New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Change to dask.order
: be more eager at times
#7929
Conversation
`dask.order` will now more eagerly compute dependents if the parent is expected to be released soon. The simplified explanation is that if running a single dependent will allow the parent to be released from memory, then we should do it. Although this is a wash--because we computed one thing and released another, so net zero--this can have big implications. Computing more eagerly earlier can give more opportunities for releasing memory (such as computing one thing and releasing two). The more complicated (and correct) explanation is that we compute a dependent if doing so will let the parent be released *soon*. Computing the depenend *may* allow the parent to be released immediately, but not necessarily so. But, there is the expectation that the parent will be able to be released *soon*, because all of its other dependents are already in one of the inner stacks. Hence, this can introduce local sub-optimal behavior (more work and more memory) with the goal of being more globally optimal. I think this is pretty cool!
Give up trying to add this same trick to the "slow path". Doing so is tricky.
dask.order
. Be more eager at times.dask.order
: be more eager at times
I think this is ready for review. CC @mrocklin, because I know you love looking at I gave up trying to get 65 in the second plot above to run sooner. Doing so is tricky, because of course anything to do with I know changing |
dask/order.py
Outdated
if num_needed[item] == 0: | ||
deps = dependents[item] | ||
key = partition_keys[item] | ||
if key < item_key: | ||
# We have three reasonable choices for item_key: first, last, min. | ||
# I don't have a principled reason for choosing any of the options. | ||
# So, let's choose min. It may be the most conservative by avoiding | ||
# accidentally going down an expensive-to-compute path. | ||
item_key = key | ||
continue |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
FYI, if we remove these lines, then the code logic is exactly equivalent to what it was before.
Specifically, the excess memory usage in the first graph above is caused by holding onto 1, 5, 10, 14, 20, 24, 29, 33, 40, and 44. That's 10 total. The second graph holds onto 1, 6, 18, 44, and 58. That's 5 total excess. Better. Had we been able to run 65 earlier and keep everything else the same, then we would have no excess memory usage, and we'd have ordered this graph more-or-less perfectly. Do we expect the distributed scheduler to run 65 earlier so it can free 1? If so, would it then soon discover "aha, I can run 66 so I can free 6 and 65!"? Or, would the distributed scheduler get enticed by the abundant parallelism available? |
For reference, this is what I consider to be perfect ordering for this graph: Unlike the current PR, the "eager" nodes when ordered perfectly don't "skip the line". For example, in this graph, 2 and 3 come before 4, and 5-9 come before 10. This is preferable, because it avoids being locally sub-optimal, which poses a risk if the inner stacks grow very large before being able to go back and clean up the sub-optimal behavior (I consider this risk on the current PR to be small, but the risk is there). I think I can probably update I think the likely rewards of the current PR outweigh the risks, but, with ordering, there's always a chance I'm wrong. Moreover, if I manage to update |
…available. This is much like the previous version, but better (and more complicated). Unlike before, this is now not over-eager. It will only eagerly compute a node when it is known that doing so will free a node.
Can one of the admins verify this patch? |
Updated. It now orders the example graph perfectly as shown above. The other tests and my battery of examples still run fine. I still need to run benchmarks and add comments to explain what's going on (so if anybody wants to review what I'm doing, maybe wait until I add comments). This added a bit more complexity, but I think it's worth it as long as it's not significantly slower. |
assert total <= 32 # ideally, this should be 2 * 16 = 32 | ||
assert total <= 110 # ideally, this should be 2 * 16 = 32 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#6668 might be another interesting case to look at. |
This adds several hundred lines to the implementation. Should we be concerned about performance at all here? Should we be concerned about code complexity (although maybe the ship has already sailed there). |
I will check. Should be fine 🤞 , but I expect it may be slightly slower.
Heh, yeah, that ship has probably already sailed, and the purpose is for performance. I could tidy up the code with functions, but adding a few more function calls and a few more conditionals in the primary loop will come at a performance cost. Consider
Thanks for the suggestion @shoyer. This is indeed part of my battery of examples, yet is one I have difficulty grokking. There is no difference in the memory usage profile when running the code from the first post between main branch and this PR. There are slight differences in the ordering, and I haven't convinced myself which is better. Let me show you: n = 125 * 2
dask.visualize(evaluate(x1[:n], y1[:n], x2[:n], y2[:n]), optimize_graph=True,
color="order", cmap="autumn", node_attr={"penwidth": "4"}) main branchThis PRn = 125 * 3
dask.visualize(evaluate(x1[:n], y1[:n], x2[:n], y2[:n]), optimize_graph=True,
color="order", cmap="autumn", node_attr={"penwidth": "4"}) main branchThis PRIf we go by visual aesthetics alone, this PR is more appealing to me for these examples. In the last graph, we can finally see the three distinct groups of calculations by color. In terms of broader justification for this PR, I expect a dynamic scheduler to employ similar behavior to run a task to free a dependency. A dynamic scheduler has the benefit of knowing data sizes. I think this behavior makes sense for static scheduling too. By more eagerly running a task to free a dependency, there will be more opportunities for running a task to free many dependencies. |
There are some serious performance regressions with the latest changes (and also some performance improvements). Interestingly, the original commit in this PR doesn't have any performance regressions. I should be able to shake things down enough to perform well again. |
I fixed the performance regression with no change to behavior. Here are the benchmark results. I ran each set once on main branch and the PR on Python 3.9, and tried to do so fairly: main branch· Running 12 total benchmarks (1 commits * 1 environments * 12 benchmarks)
[ 0.00%] ·· Benchmarking existing-py_Users_ewelch_miniconda3_envs_dask39_bin_python
[ 4.17%] ··· Running (order.OrderCholesky.time_order_cholesky--)....
[ 20.83%] ··· Running (order.OrderFullLayers.time_order_full_layers--)..
[ 29.17%] ··· Running (order.OrderLinearFull.time_order_linear_full--)....
[ 45.83%] ··· Running (order.OrderRechunkTranspose.time_order_rechunk_transpose--)..
[ 54.17%] ··· order.OrderCholesky.time_order_cholesky 396±2ms
[ 58.33%] ··· order.OrderCholesky.time_order_cholesky_lower 384±3ms
[ 62.50%] ··· order.OrderCholeskyMixed.time_order_cholesky_mixed 292±3ms
[ 66.67%] ··· order.OrderCholeskyMixed.time_order_cholesky_mixed_lower 256±2ms
[ 70.83%] ··· order.OrderFullLayers.time_order_full_layers ok
[ 70.83%] ··· ============ ============
param1
------------ ------------
(1, 50000) 354±2ms
(2, 10000) 975±10ms
(10, 1000) 364±4ms
(100, 20) 259±4ms
(500, 2) 358±10ms
(9999, 1) 65.9±0.8ms
(50000, 1) 261±4ms
============ ============
[ 75.00%] ··· order.OrderLinalgSolves.time_order_linalg_solve 290±3ms
[ 79.17%] ··· order.OrderLinearFull.time_order_linear_full 841±4ms
[ 83.33%] ··· order.OrderLinearWithDanglers.time_order_linear_danglers ok
[ 83.33%] ··· ============ =========
param1
------------ ---------
(2, 10000) 162±3ms
(5, 5000) 193±2ms
============ =========
[ 87.50%] ··· order.OrderManySubgraphs.time_order_many_subgraphs ok
[ 87.50%] ··· =========== =========
param1
----------- ---------
(1, 9999) 155±3ms
(3, 3333) 181±3ms
(10, 999) 192±3ms
(30, 303) 166±3ms
(100, 99) 186±2ms
(999, 10) 193±1ms
=========== =========
[ 91.67%] ··· order.OrderMapOverlap.time_order_mapoverlap ok
[ 91.67%] ··· =========================================== =========
param1
------------------------------------------- ---------
((10000.0, 10000.0), (200, 200), 1) 508±3ms
((1000, 1000, 1000), (100, 100, 100), 10) 535±4ms
=========================================== =========
[ 95.83%] ··· order.OrderRechunkTranspose.time_order_rechunk_transpose 582±6ms
[100.00%] ··· order.OrderSVD.time_order_svd 47.4±1ms This PR· Running 12 total benchmarks (1 commits * 1 environments * 12 benchmarks)
[ 0.00%] ·· Benchmarking existing-py_Users_ewelch_miniconda3_envs_dask39_bin_python
[ 4.17%] ··· Running (order.OrderCholesky.time_order_cholesky--)....
[ 20.83%] ··· Running (order.OrderFullLayers.time_order_full_layers--)..
[ 29.17%] ··· Running (order.OrderLinearFull.time_order_linear_full--)....
[ 45.83%] ··· Running (order.OrderRechunkTranspose.time_order_rechunk_transpose--)..
[ 54.17%] ··· order.OrderCholesky.time_order_cholesky 397±5ms
[ 58.33%] ··· order.OrderCholesky.time_order_cholesky_lower 390±3ms
[ 62.50%] ··· order.OrderCholeskyMixed.time_order_cholesky_mixed 297±3ms
[ 66.67%] ··· order.OrderCholeskyMixed.time_order_cholesky_mixed_lower 257±3ms
[ 70.83%] ··· order.OrderFullLayers.time_order_full_layers ok
[ 70.83%] ··· ============ ==========
param1
------------ ----------
(1, 50000) 355±2ms
(2, 10000) 926±8ms
(10, 1000) 367±2ms
(100, 20) 259±4ms
(500, 2) 356±2ms
(9999, 1) 72.2±2ms
(50000, 1) 283±2ms
============ ==========
[ 75.00%] ··· order.OrderLinalgSolves.time_order_linalg_solve 302±1ms
[ 79.17%] ··· order.OrderLinearFull.time_order_linear_full 851±1ms
[ 83.33%] ··· order.OrderLinearWithDanglers.time_order_linear_danglers ok
[ 83.33%] ··· ============ =========
param1
------------ ---------
(2, 10000) 130±3ms
(5, 5000) 177±3ms
============ =========
[ 87.50%] ··· order.OrderManySubgraphs.time_order_many_subgraphs ok
[ 87.50%] ··· =========== =========
param1
----------- ---------
(1, 9999) 161±5ms
(3, 3333) 183±4ms
(10, 999) 190±6ms
(30, 303) 174±5ms
(100, 99) 186±4ms
(999, 10) 197±2ms
=========== =========
[ 91.67%] ··· order.OrderMapOverlap.time_order_mapoverlap ok
[ 91.67%] ··· =========================================== =========
param1
------------------------------------------- ---------
((10000.0, 10000.0), (200, 200), 1) 521±2ms
((1000, 1000, 1000), (100, 100, 100), 10) 548±4ms
=========================================== =========
[ 95.83%] ··· order.OrderRechunkTranspose.time_order_rechunk_transpose 579±5ms
[100.00%] ··· order.OrderSVD.time_order_svd 47.0±1ms Most are pretty much identical. Some may be slightly slower or slightly faster, but nothing to get excited or worried about that I see. So, I'd say we're good here! I may be able to squeeze out a tiny bit more, we'll see. (btw, setting aliases for methods such as Up next: code comments, and maybe try to simplify some things. |
Okay, I think this is ready! Whew. I don't expect anybody else to go through the logic line-by-line, but maybe others can peruse the code and read the explanatory code comments, and I'll be happy to clarify anything. Also, I'm always happy to make and share more graphs. As with any change to |
for dep in root_nodes: | ||
num_needed[dep] -= 1 | ||
# Use remove here to complain loudly if our assumptions change | ||
dep2.remove(dep) # Safe to mutate |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not covered. Why not? Are there not that many examples where skip_root_node
is True, or is this somehow guaranteed not to run?
elif key2 < partition_keys[item]: | ||
next_nodes[key2].append([dep2]) | ||
else: | ||
later_nodes[key2].append([dep2]) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not covered. Interesting.
You know what I would love to have when considering changes to It's not always obvious which ordering is better (such as examples from #5859). |
I added these to help investigate dask#7929 The names could probably be better. This also needs documented and tested, but I thought I'd share, because, you know, pretty plots! I think all options here are potentially useful. Anything else we might want to show? Speaking of pretty plots, I'll share some tomorrow. Good night!
Alright, I've looked more closely at some challenging graphs from previous order issue using diagnostics from #7992 (which is very nice!), and I haven't found any issues. #7992 also lets me create a line plot of number of dependencies in memory over time. This makes me feel more comfortable with this PR. |
# A lingering question is: what should we use for `item`? `item_key` is used to | ||
# determine whether each dependent goes to `next_nodes` or `later_nodes`. Currently, | ||
# we use the last value of `item` (i.e., we don't do anything). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For posterity, this is the main question in this PR that I'm not quite satisfied with. We sometimes compare the dependent partition_keys to the partition_key of the current item, but when we process the dependents of singles
, we don't have a good item
to use.
In practice, it doesn't seem to matter what we use here. I experimented setting item to a faux min_item
that has partition_key[min_item] == -1
, which causes dependents to go to later_nodes
instead of next_nodes
. Again, I couldn't find anywhere where this actually made a difference. Another option might be to set item
to the item on the top of the inner stack.
Ok we discussed this at the maintainer meeting a few weeks ago and everyone was in agreement that if @eriknw is happy, we are happy. I just forgot to hit the merge button. |
@eriknw just wanted to check in that you were finished with this PR. We're planning to release this Friday (xref dask/community#180) and I wanted to double check that you're confident in the change here |
Hi @jrbourbeau, yes, I think it's good to go! 🚀 |
Added to help investigate #7929
dask.order
will now more eagerly compute dependents if the parent is expected to be released soon.The simplified explanation is that if running a single dependent will allow the parent to be released from memory, then we should do it. Although this is a wash--because we computed one thing and released another, so net zero--this can have big implications. Computing more eagerly earlier can give more opportunities for releasing memory (such as computing one thing and releasing two).
The more complicated (and correct) explanation is that we compute a dependent if doing so will let the parent be released soon. Computing the dependent may allow the parent to be released immediately, but not necessarily so. But, there is the expectation that the parent will be able to be released soon, because all of its other dependents are already in one of the inner stacks.
Hence, this can introduce locally sub-optimal behavior (more work and more memory) with the goal of being more globally optimal. I think this is pretty cool!
Here's an example of previous behavior:
Note the pairs (2, 51), (6, 52), (11, 55), etc. Ideally, we'd like to compute e.g. 51 much earlier so we can release 1.
Now this is how the first commit of this PR behaves:
Visually, it's clear that things look better, right? Except, of course, that we'd want to run 65 much sooner. But, look at 12. We run it before running 13 (ideally, maybe we'd like to switch the order, but that would be complicated to implement). Normally we would have just done 13 and left 12 for much later. But, because we do 12, and because we do the same trick for 17, 12 and 17 can now both be released by running 18. The block on the right, 23-48, looks amazing to me.
I still have a little more work to do. I can make 65 run sooner.
This has very little impact to existing tests and my battery of examples (shared on previous issues), so I'm pretty confident in the first commit.
Many thanks to @AGoose for providing the example that made me think of this change: https://gist.github.com/agoose77/c56c1b8b924d66648a571266794b2664
Also, thanks to @gjoseph92 for creating and suggesting the tool that AGoose used to create the example:
https://github.com/gjoseph92/dask-noop
- [ ] Closes #xxxxblack dask
/flake8 dask
/isort dask