-
-
Notifications
You must be signed in to change notification settings - Fork 1.1k
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
Groupby-map is slow with out of order indices #9220
Comments
For Dask Array people, here is a tiny test: def test_out_of_order_slicing():
x = da.random.random((10, 20), chunks=(10, 10))
idx = np.random.randint(0, x.shape[1], x.shape[1])
y = x[:, idx]
assert y.npartitions < x.npartitions * 2 My sense is that we want to embrace an |
Reducing the number of tasks is easy if we just merged adjacent slices. However, in that scenario neither task number not network is the limiting factor but memory. If we truly care/need this kind of random access take, the best move would probably be to have a p2p version of this as well. |
Yes. groupby-map is quite popular (sadly).
Yes. This is a high-impact problem to solve.
This assumes that you can shove a whole group into one block, which is not a good assumption. I think the core problem is that we need Xarray to give dask/cubed/whatever all the information (indices that sort by group) in one call rather than a for-loop over groups. Which suggests that the missing primitive is a That way a user is free to rechunk to a single block if it makes sense and dask/whatever is free to optimize the indexing in one go. This For an example of the indices see arr.groupby("time.dayofyear")._group_indices
It feels like you've already solved a much harder problem at some level... it "just" needs to be hooked up to slicing.
This is fine. The iteration is hidden behind an abstraction already.
Yes I can help, I've been refactoring all this code recently. PS: Arkouda does something like this for groupby problems --- IIRC they always sort by key. |
alternatively we need |
Once array libraries define such a primitive, we now have a natural place for xarray to inteface with it - the |
Here's an array-only implementation from xarray.core.groupby import _inverse_permutation_indices
def center(x: np.ndarray):
return x - x.mean(axis=-1, keepdims=True)
def identity(x):
return x
func = identity
dask_array = arr["arr"].data
numpy_array = dask_array.compute()
indices = arr.groupby("time.dayofyear")._group_indices
group_sizes = [len(i) for i in indices]
# We could have Xarray do this shuffling when setting up
# the groupby problem only for chunked array types
shuffled = np.take(
dask_array,
indices=np.concatenate(indices),
axis=-1,
# TODO: this is the new argument
# chunks_hint = group_sizes,
)
# reset _group_indices for the shuffled array
summed = np.cumsum(group_sizes).tolist()
slices = [slice(None, summed[0])] + [
slice(summed[i], summed[i + 1]) for i in range(len(summed) - 1)
]
# This is the apply-combine step
applied = np.concatenate(
[func(shuffled[..., slicer]) for slicer in slices],
axis=-1,
)
# Now shuffle back, in case this was a groupby-transform, not a groupby-reduce
reordering = _inverse_permutation_indices(indices, N=dask_array.shape[-1])
result = np.take(
applied,
indices=reordering,
axis=-1,
# TODO: this is the new argument
# chunks_hint=dask_array.chunks,
)
np.testing.assert_equal(numpy_array, result) The What's not clear to me is whether just optimizing the |
I think what you would want Is basically the shuffle (i.e. take) and then call map_blocks with the transformer / reducer. This is what DataFrames in Dask do as well. I am a bit hesitant about
I'd rather have control about this in dask by default, creating a block per group might blow up the graph if we are unlucky with a high cardinality grouper. You can split the groups in map_blocks again if needed, hiding the cardinality of the grouper from the system |
OK now I am finally understanding what you mean here. The UDF is expecting to receive only one group, we can still satisfy that interface by inserting a for-loop-over-groups in the map-blocks call. 👍
For one, without an extra arg, dask doesn't know where the group boundaries are. Another option is to pass:
A low cardinality grouper is also important to consider, particularly with small chunk sizes. For example. arr = xr.DataArray(
# time chunks of size-1 are common!
da.random.random((100, 100, len(time)), chunks=(100, 100, 1)),
dims=["lat", "lon", "time"],
coords={"lat": lat, "lon": lon, "time": time},
name="arr",
).to_dataset()
with xr.set_options(use_flox=False):
arr.groupby("time.year").mean().compute() ^ Today, this is a tree-reduction reduce because we dispatch to |
I definitely agree that the shuffle map_blocks solution shouldn't be the default for reducers. A tree reduction is totally fine if you have a low cardinality grouper and will most likely outperform every shuffle implementation we can come up with. We have to come up with a threshold starting from where the shuffle map_blocks approach makes sense for reducers.
Yeah we need some marker where the group boundaries are, I totally agree with you here. What I would want to avoid is that the group boundaries necessarily define the chunks and then you would do:
For context: in DataFrame land this works as follow:
We push the actual groupby operation into the map_partitions call. For arrays you would "re-do" the groupby in the map_blocks call to ensure that the UDF gets one group at a time, but we don't have to create a task per group with this approach, rather one per chunk which should keep the graph size smaller and still parallelises very well. |
After thinking over it over the weekend, I think it'd be nice to have at least two APIs.
cc @phofl |
That sounds good to me. So the current shuffle implementation doesn't do any rechunking, assuming you have a group that is very large you would expect us to change the length of the chunks across other dimensions correct? re take: I have a pr dask/dask#11267 that hooks the shuffle approach into take, this means that we preserve the chunk size along the axis that we are applying the take indexer but it's a lot more efficient than an out of order take indexer (this is what I benchmarked here). This is something that would be helpful if we expose this toplevel I assume? |
Yes you'd absolutely have to.
Isn't it already exposed as
How does this do with repeated indices? #2745 |
This isn't yet implemented, I'll add this as a follow up.
good point, this should end up in the same place (I'll double check that to be sure though)
It places them in different chunks if you have too many of them. It looks at the average input chunk size along the dimension your are indexing and then uses this as the target chunk size for the output. It would preserve the 100, 100 chunks in the example in the other issue. |
Perfect! We'll need this for groupby-binary-ops ( Lines 655 to 668 in 1ac19c4
|
Some updates after a chat with @phofl First off, amazing progress on reducing memory usage in dask 2024.08.0! TODO:
|
What is your issue?
I think that this is a longstanding problem. Sorry if I missed an existing github issue.
I was looking at an Dask-array-backed Xarray workload with @phofl and we were both concerned about some performance we were seeing with groupby-aggregations called with out-of-order indices. Here is a minimal example:
Performance here is bad in a few ways:
We think that what is happening here looks like this:
For steps (1) and (3) above performance is bad in a way that we can reduce to a dask array performance issue. Here is a small reproducer for that:
We think that we can make this better on our end, and can take that away as homework.
However for step (2) we think that this probably has to be a change in xarray. Ideally xarray would call something like
map_blocks
, rather than iterate through each group. This would be a special-case for dask-array. Is this ok?Also, we think that this has a lot of impact throughout xarray, but are not sure. Is this also the code path taken in sum/max/etc..? (assuming that
flox
is not around). Mostly we're curious how much we all should prioritize this.Asks
Some questions:
The text was updated successfully, but these errors were encountered: