-
Notifications
You must be signed in to change notification settings - Fork 920
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
Make a plan for sort_values/set_index #2272
Comments
NVIDIA folks have asked if there is some way to integrate an MPI or NCCL enabled multi-gpu sort into Dask for improved efficiency. My initial reaction to this is that it's likely to be difficult to integrate smoothly in a way that respects other Dask management like resilience, spilling to disk, load balancing, and so on. Lets expand on this. First, if we have a multi-node sorting algorithm, we can always treat it how we treat XGBoost. Dask gives up control to some other higher performance system, it does its thing, we claim control back. If anything fails during this stage then we just retry the whole thing. We give up on any kind of memory management or load balancing during this process and just hope that the external system can handle things well without blowing up. Second question is if we just have a single-node multi-GPU system, maybe we can use that? This is also a bit tricky currently, but we might be able to make structural changes to Dask to make it less tricky. The cost-benefit analysis of those changes might make this undesirable though. Currently the approach most people seem to be using with Dask and GPUs is to have one Dask worker per GPU. Currently Dask workers don't have any knowledge of other Dask workers on the same node, so there isn't anything built up to handle local collective action. We would be doing something similar to what is done above where we would more or less stop Dask from doing its normal task-scheduling thing, hand-write a bunch of control flow, hope nothing breaks, run custom code, and then have Dask take back control when we're done. Both are totally doable, but would require us to build something like dask-xgboost, and raise general concerns around memory management, resilience, diagnostics, load balancing, spilling to disk, and so forth. We lose a lot of Dask's management when we switch into this mode. So maybe Dask should start thinking more about collective actions. This is coming up often enough that it probably deserves more attention. That's a large conversation though and probably requires dedicated time from someone somewhat deeply familiar with Dask scheduler internals. I think that, short term, we should continue with the current approach of using efficient single-core/gpu sorts and shuffling techniques currently done in dask.dataframe. We should tune these to the extent that we can both by making the single-gpu sort algorithms faster, the book keeping operations faster, and the communication faster. If this isn't enough then we should investigate collective actions as discussed above, but that should be part of a larger effort than just sorting. |
I agree with this approach. Thrust has a good single GPU sorting, and I think UCX should help tremendously. Any thoughts on book keeping, or was that more of a placeholder for if we need it? |
There are two forms of book keeping that are relevant here:
|
@sklam can you expand on the motivation behind your use of batcher sort net? Why was this decided on rather than the approach take in the mainline dask dataframe codebase? |
@mrocklin, I was trying to avoid the following of the mainline sort/shuffle:
I chose the sorting network because:
|
OK, so these concerns seem similar to the concerns we had while designing the various shuffle algorithms for dask dataframe/bag. I'm not seeing anything here that is highly specific to GPU computation. (please correct me if I'm wrong). My inclination then is to try to unify both CPU and GPU around a single implemenatation, probably starting with the one in dask.dataframe, but then maybe we would consider batcher-sorting-networks as an alternative for both, rather than just for dask-cudf, after doing some more extensive benchmarking. Thanks for the information @sklam , I really appreciate the continuity here. |
I recently took a look at Dask dataframe's task-based shuffle and improved docstrings here in order to help others dive in: https://github.com/dask/dask/pull/4674/files I think that we need the following:
|
Also, just to direct people, I think that the core function that we'll have to make work is |
@mrocklin @datametrician I have a vested interest in seeing this work succeed, and will begin by implementing the solution path you've outlined above. If we need to get into the specifics of MPI/NCCL, we can cross that bridge later. I think a general solution which puts Dask first is going to help the ecosystem the most. |
A minimal test would look something like the following: import cudf, dask.dataframe as dd
# Make a dataframe, we'd like to divide the data by y
df = cudf.DataFrame({'x': [1, 2, 3, 4, 5, 6], 'y': [0, 4, 0, 4, 0, 4]})
# Split it up into a few partitions with Dask
ddf = dd.from_pandas(df, npartitions=3)
# Try to create a new dataframe with two partitions sorted on the y column, split by y 0->2 and 2->4
out = dd.shuffle.rearrange_by_divisions(ddf, column='y', divisions=[0, 2, 4], shuffle='tasks')
# compute in a single thread so it's easy to use %pdb and %debug
out.compute(scheduler='single-threaded') |
I just tried this and ran into an minor problem of cudf.DataFrame.drop not supporting the axis= keyword. As a suggestion, these errors are easier to identify if you remove some of Dask's error reporting with the following diff diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py
index 6a08af9..894fba6 100644
--- a/dask/dataframe/core.py
+++ b/dask/dataframe/core.py
@@ -3736,8 +3736,8 @@ def _emulate(func, *args, **kwargs):
Apply a function using args / kwargs. If arguments contain dd.DataFrame /
dd.Series, using internal cache (``_meta``) for calculation
"""
- with raise_on_meta_error(funcname(func), udf=kwargs.pop('udf', False)):
- return func(*_extract_meta(args, True), **_extract_meta(kwargs, True))
+ kwargs.pop('udf')
+ return func(*_extract_meta(args, True), **_extract_meta(kwargs, True)) |
I imagine that, like with the groupby aggregations work, this will end up triggering many small PRs in cudf. |
Potential fix here: #1396 |
Next thing I run into, searchsorted In [1]: import cudf, dask.dataframe as dd
...:
...: # Make a dataframe, we'd like to divide the data by y
...: df = cudf.DataFrame({'x': [1, 2, 3, 4, 5, 6], 'y': [0, 4, 0, 4, 0, 4]})
...:
...: # Split it up into a few partitions with Dask
...: ddf = dd.from_pandas(df, npartitions=3)
...:
...: # Try to create a new dataframe with two partitions sorted on the y column, split by y 0->2 and 2->4
...: dd.shuffle.rearrange_by_divisions(ddf, column='y', divisions=[0, 2, 4], shuffle='tasks')
Out[1]: <dask_cudf.DataFrame | 40 tasks | 2 npartitions>
In [2]: _.compute(scheduler='single-threaded')
---------------------------------------------------------------------------
TypeError Traceback (most recent call last)
... <removed for clarity>
TypeError: can't compute boolean for <class 'cudf.dataframe.series.Series'>
In [3]: debug
> /home/nfs/mrocklin/cudf/python/cudf/dataframe/series.py(325)__bool__()
323 into a boolean.
324 """
--> 325 raise TypeError("can't compute boolean for {!r}".format(type(self)))
326
327 def values_to_string(self, nrows=None):
ipdb> up
> /home/nfs/mrocklin/miniconda/envs/cudf/lib/python3.7/site-packages/pandas/core/series.py(2337)searchsorted()
2335 sorter = ensure_platform_int(sorter)
2336 result = self._values.searchsorted(Series(value)._values,
-> 2337 side=side, sorter=sorter)
2338
2339 return result[0] if is_scalar(value) else result
ipdb>
> /home/nfs/mrocklin/dask/dask/dataframe/shuffle.py(434)set_partitions_pre()
432
433 def set_partitions_pre(s, divisions):
--> 434 partitions = pd.Series(divisions).searchsorted(s, side='right') - 1
435 partitions[(s >= divisions[-1]).values] = len(divisions) - 2
436 return partitions
ipdb> |
Searchsorted work is happening here: #2156 |
Just a minor update here - After #2156 goes through, the next issue we run into is the very next line of
Since |
Adding a simple @property
def values(self):
return self._values
@property
def _values(self):
return self.to_pandas().values Test: In [1]: import cudf, dask.dataframe as dd
...:
...: # Make a dataframe, we'd like to divide the data by y
...: df = cudf.DataFrame({'x': [1, 2, 3, 4, 5, 6], 'y': [0, 4, 0, 4, 0, 4]})
...:
...: # Split it up into a few partitions with Dask
...: ddf = dd.from_pandas(df, npartitions=3)
...:
...: # Try to create a new dataframe with two partitions sorted on the y column, split by y 0->2 and 2->4
...: out = dd.shuffle.rearrange_by_divisions(ddf, column='y', divisions=[0, 2, 4], shuffle='tasks')
...:
...: # compute in a single thread so it's easy to use %pdb and %debug
...: print(out.compute(scheduler='single-threaded'))
x y
0 1 0
2 3 0
4 5 0
1 2 4
3 4 4
5 6 4 @kkraus14 - Is there a reason we might want to avoid adding a |
So copying an entire column from device to host? That would be very expensive. |
One issue I see above is a hard dependency on pandas.
If partitions were to be a cudf series instead of a pandas series , |
@jrhemstad Sorry - That was a silly question! @ayushdg - Right. I guess I am just unsure of the best way (if possible) to handle both pandas and cudf using the same logic in Dask, but I haven't exactly tried much yet :) |
Some options to explore:
I agree that it would be good not to call See also some conversation about |
Also see: #2373 which attempts to add |
@mrocklin I explored some options to avoid copying the entire column from device to host. I made this simple gist with some experiments. For a dataframe with 1e8 rows, the dask+pandas version takes about 13.5s, while the dask+cudf version takes about 7.6s. If I use a Note that the gist also shows that the cudf version of |
7.6s sounds very high, what's taking the time in this situation? Could you share a profile? |
@kkraus14 agreed - I'll take a closer look and collect a profile |
We're probably ready to remove some of the older dask-cudf implementations of set_index/merge/join. I've raised this as a separate issue here: #2598 |
I thought I'd poke this. @harrism @jrhemstad do either of you have thoughts on what a timeline for the partition_map splitting operation could look like? We're quickly reaching a state where this will be our only remaining blocker. |
@mrocklin yes, @jrhemstad and I have been a plan. A "fastest time to solution, not necessarily fastest solution" plan is described in #2677. Please review. |
Now that all of the issues mentioned above have closed, and cupy packaging has been resolved, are we able to continue deprecating the custom dask_cudf sort code? My hope is doing so will yield performance and scalability improvements. Several workflows have recently had problems with the dask_cudf implementation of set_index and sort_values. If it's useful, I'm happy to open new issues with example unexpected behavior, but a quick read of the above makes me feel like you all have a good handle on the problem. |
@randerzander Can you clarify the pieces that are still custom to dask_cudf? dask_cudf should now be using the main-line dask implementations of If your comment is more about performance improvements/optimizations, then I can say we are certainly in the process of exploring ways to (1) Improve merge/join performance to better match the expected/available performance, and (2) improve the stability of operations like Note that I am currently working on a simple (but experimental) modification to the upstream-dask |
To what extent is this issue satisfied by the current work on providing multi-column distributed sorting primitives (table quantiles, k-way merge) in libcudf? |
100%, this is the primary target for that work. |
@kkraus14 both of those new primitives were added. Does this need to stay open? |
I defer to @rjzamora if he'd like to keep this open for future discussion or if this can now be closed. |
We should probably close this once #4308 is merged. That should provide dask_cudf with the general sorting/set_index functionality targeted by this discussion. After that, the remaining work will be to push as mach dask_cudf code as possible into main-line dask (which can probably be tracked separately). |
Now that #4308 is merged, it probably makes sense to close this. We can open new issues in the future to discuss any performance/functionality challenges that come up. Also, the task of moving the new |
@randerzander - Feel free to reopen this if the current |
It would be nice to be able to use the
set_index
method to sort the dataframe by a particular column.There are currently two implementations for this, one in dask.dataframe and one in dask-cudf which uses a batcher sorting net. While most dask-cudf code has been removed in favor of the dask.dataframe implementations this sorting code has remained, mostly because I don't understand it fully, and don't know if there was a reason for this particular implementation.
Why was this implementation chosen? Was this discussed somewhere? Alternatively @sklam, do you have any information here?
cc @kkraus14 @randerzander
The text was updated successfully, but these errors were encountered: