-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
[DNM] Scatter shuffle proof-of-concept #5473
Closed
Closed
Conversation
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Fixes dask#4959 `get_client` was calling the private `Worker._get_client` method when it ran within a task. `_get_client` should really have been called `_make_client`, since it created a new client every time. The simplest correct thing to do instead would have been to use the `Worker.client` property, which caches this instance. In order to pass the `timeout` parameter through though, I changed `Worker.get_client` to actually match its docstring and always return the same instance.
If you accidentally pass Futures created by a different Client into `Client.gather`, you'll get a `CancelledError`. This is confusing and misleading. An explicit check for this would have made discovering dask#5466 much easier. And since there are probably plenty of other race conditions regarding default clients in multiple threads, hopefully a better error message will save someone else time in the future too.
This is probably a good idea in general (xref dask#4959), but it particularly helps with performance deserializing Futures, which have a fastpath through `Client.current` that bypasses a number of unnecessarily slow things that `get_client` does before it checks `Client.current`.
When a key is scattered from a task and written directly to worker storage, the Client immediately sets the Future's state to `"finished"`. There's no need for the scheduler to also tell the client that that key is finished; it already knows. This saves a bit of scheduler time and a comms roundtrip.
Helps with setting the current client in worker while deserializing. Implementation referenced from python/cpython#9688
This was really slow and probably doesn't matter when the future is coming from a worker. But probably not safe to remove in general?
We don't need to report back to the client that its key was cancelled. But this shouldn't be exposed and may be wrong.
Just want to see how it affects performance
Hoping this speeds up the transfer of Futures; makes no sense in general though.
fjetter
reviewed
Oct 28, 2021
"op": "update-graph", | ||
"tasks": {}, | ||
"keys": [stringify(self.key)], | ||
"client": c.id, |
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.
I think this is about updating the who_wants
on scheduler side. however, I don't know for sure
2 tasks
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
This is a semi-working proof-of-concept of a different approach to scalable(ish) shuffling. The idea is basically this: tracking millions of tasks is too much overhead on the scheduler, but what about millions of keys? Keys (static pieces of data) should be a bit easier to deal with: they don't have dependencies, they don't need to be scheduled, etc. Perhaps we can get the scheduler to track tens of millions of keys with relatively little effort?
The idea is the same as what I proposed in dask/dask#7613 (comment): tasks split up input data and
scatter
it, returning Futures. Other tasks take the Futures andgather
them, fetching the data.In detail
So within an input task, we:
scatter
all of those shards (pieces), getting back Futures referencing themscatter
within a task: just it takes the data, stores it in the worker's memory, and informs the scheduler that the data is there. No communication with other workers; basically lets you hand off multiple pieces of data to the system to be tracked without running a separate task to produce each piece.Then each output task depends on every one of these input tasks:
gather
those Futures, transferring the data from the source worker to the current workerWith this approach, we keep the graph size
O(N)
in the number of partitions. The number of keys is stillO(N^2)
, but we're hoping we can handle keys much more efficiently (initial results indicate this is maybe true with some more careful thought).There are a few reasons I find this approach appealing compared to #5435:
[^1] though because disk IO blocks the event loop (#4424), performance is awful with this PR unless you disable spilling. When the loop is blocked, data can't transfer, which stops up the whole operation—the spice must flow!
However, there are also some big downsides:
O(N^2)
component. A 100,000-partition DataFrame would require 10 billion keys for all the sub-partitions. I'm not sure we'll ever be able to handle that scale on the scheduler. In practice, I think this approach would have a lower upper bound in scalability than [DNM] Peer-to-peer shuffle design #5435.So how does this perform? Well, the first try was abysmal. But there was a lot of very low-hanging fruit I've either fixed or hacked past in this PR.
Currently, it's slower than a task-based shuffle for small datasets locally on my machine:
dask.datasets.timeseries(start="2000-01-01", end="2001-01-01")
: 44secshuffle="scatter
, 22secshuffle="tasks"
But it can handle bigger datasets that tasks cannot:
dask.datasets.timeseries(start="2000-01-01", end="2005-01-01")
: ~5minshuffle="scatter"
,shuffle="tasks"
failedYet weirdly, on a cluster, it performs much worse. I'm not sure why yet; it has something to do with when all the futures get replicated to every worker. Generally what's hurting performance is:
O(N^2 * W)
! But is it even necessary?)The bottom line is that this doesn't even come close to working as well as dask/dask#8223. And it's almost certainly not the most performant or scalable design for shuffling we could make.
But what I find appealing about this is that, because it's not circumventing dask, the improvements needed to make this work tend to be things we should do anyway, and that benefit the whole system, even for users who aren't shuffling. There's nothing new to add; it's just making current features as robust, performant, and correct as they should be.
Note that the point of this PR is just to record for posterity that somebody has tried this once. I do not plan to continue working on this PR or this approach currently.