Skip to content
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

Add GroupBy.shuffle_to_chunks() #9320

Merged
merged 61 commits into from
Nov 21, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
61 commits
Select commit Hold shift + click to select a range
3bc51bd
Add GroupBy.shuffle()
dcherian Aug 7, 2024
60d7619
Cleanup
dcherian Aug 7, 2024
d1429cd
Cleanup
dcherian Aug 7, 2024
31fc00e
fix
dcherian Aug 7, 2024
4583853
return groupby instance from shuffle
dcherian Aug 13, 2024
abd9dd2
Fix nD by
dcherian Aug 13, 2024
6b820aa
Merge branch 'main' into groupby-shuffle
dcherian Aug 14, 2024
0d70656
Skip if no dask
dcherian Aug 14, 2024
fafb937
fix tests
dcherian Aug 14, 2024
939db9a
Merge branch 'main' into groupby-shuffle
dcherian Aug 14, 2024
a08450e
Add `chunks` to signature
dcherian Aug 14, 2024
d0cd218
FIx self
dcherian Aug 14, 2024
4edc976
Another Self fix
dcherian Aug 14, 2024
0b42be4
Forward chunks too
dcherian Aug 14, 2024
c52734d
[revert]
dcherian Aug 14, 2024
8180625
undo flox limit
dcherian Aug 14, 2024
7897c91
[revert]
dcherian Aug 14, 2024
7773548
fix types
dcherian Aug 14, 2024
51a7723
Add DataArray.shuffle_by, Dataset.shuffle_by
dcherian Aug 15, 2024
cc95513
Add doctest
dcherian Aug 15, 2024
18f4a40
Refactor
dcherian Aug 15, 2024
f489bcf
tweak docstrings
dcherian Aug 15, 2024
ead1bb4
fix typing
dcherian Aug 15, 2024
75115d0
Fix
dcherian Aug 15, 2024
390863a
fix docstring
dcherian Aug 15, 2024
a408cb0
bump min version to dask>=2024.08.1
dcherian Aug 17, 2024
7038f37
Merge branch 'main' into groupby-shuffle
dcherian Aug 17, 2024
05a0fb4
Fix typing
dcherian Aug 17, 2024
b8e7f62
Fix types
dcherian Aug 17, 2024
6d9ed1c
Merge branch 'main' into groupby-shuffle
dcherian Aug 22, 2024
20a8cd9
Merge branch 'main' into groupby-shuffle
dcherian Aug 30, 2024
7a99c8f
remove shuffle_by for now.
dcherian Aug 30, 2024
5e2fdfb
Add tests
dcherian Aug 30, 2024
a22c7ed
Support shuffling with multiple groupers
dcherian Aug 30, 2024
2d48690
Revert "remove shuffle_by for now."
dcherian Sep 11, 2024
0679d2b
Merge branch 'main' into groupby-shuffle
dcherian Sep 12, 2024
63b3e77
Merge branch 'main' into groupby-shuffle
dcherian Sep 17, 2024
7dc5dd1
bad merge
dcherian Sep 17, 2024
bad0744
Merge branch 'main' into groupby-shuffle
dcherian Sep 18, 2024
91e4bd8
Add a test
dcherian Sep 18, 2024
0542944
Merge branch 'main' into groupby-shuffle
dcherian Nov 1, 2024
1e4f805
Add docs
dcherian Nov 1, 2024
ad502aa
bugfix
dcherian Nov 1, 2024
4b0c143
Refactor out Dataset._shuffle
dcherian Nov 2, 2024
2b2c4ab
Merge branch 'main' into groupby-shuffle
dcherian Nov 3, 2024
f624c8f
fix types
dcherian Nov 3, 2024
888e780
fix tests
dcherian Nov 3, 2024
47e5c17
Merge branch 'main' into groupby-shuffle
dcherian Nov 4, 2024
b100fb1
Handle by is chunked
dcherian Nov 4, 2024
978fad9
Merge branch 'main' into groupby-shuffle
dcherian Nov 7, 2024
d1a3fc1
Some refactoring
dcherian Nov 7, 2024
23b0cac
Merge branch 'main' into groupby-shuffle
dcherian Nov 12, 2024
d533638
Merge branch 'main' into groupby-shuffle
dcherian Nov 19, 2024
d467bc6
Remove shuffle_by
dcherian Nov 19, 2024
231533c
shuffle -> distributed_shuffle
dcherian Nov 19, 2024
c77d7c5
return xarray object from distributed_shuffle
dcherian Nov 19, 2024
bccacfe
fix
dcherian Nov 19, 2024
2d4392a
fix doctest
dcherian Nov 19, 2024
003e9f2
fix api
dcherian Nov 19, 2024
0f80c81
Rename to `shuffle_to_chunks`
dcherian Nov 20, 2024
88bef5d
update docs
dcherian Nov 21, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
44 changes: 44 additions & 0 deletions xarray/core/groupby.py
Original file line number Diff line number Diff line change
Expand Up @@ -517,6 +517,50 @@ def sizes(self) -> Mapping[Hashable, int]:
self._sizes = self._obj.isel({self._group_dim: index}).sizes
return self._sizes

def shuffle(self) -> None:
dcherian marked this conversation as resolved.
Show resolved Hide resolved
"""
Shuffle the underlying object so that all members in a group occur sequentially.

The order of appearance is not guaranteed. This method modifies the underlying Xarray
object in place.

Use this method first if you need to map a function that requires all members of a group
be in a single chunk.
"""
from xarray.core.dataarray import DataArray
from xarray.core.dataset import Dataset

(grouper,) = self.groupers
dim = self._group_dim
was_array = isinstance(self._obj, DataArray)
as_dataset = self._obj._to_temp_dataset() if was_array else self._obj

shuffled = Dataset()
for name, var in as_dataset._variables.items():
if dim not in var.dims:
shuffled[name] = var
continue
shuffled[name] = var._shuffle(indices=list(self._group_indices), dim=dim)

# Replace self._group_indices with slices
slices = []
start = 0
for idxr in self._group_indices:
if TYPE_CHECKING:
assert not isinstance(idxr, slice)
slices.append(slice(start, start + len(idxr)))
start += len(idxr)

# TODO: we have now broken the invariant
# self._group_indices ≠ self.groupers[0].group_indices
self._group_indices = tuple(slices)
if was_array:
if TYPE_CHECKING:
assert isinstance(self._obj, DataArray)
self._obj = self._obj._from_temp_dataset(shuffled)
else:
self._obj = shuffled

def map(
self,
func: Callable,
Expand Down
2 changes: 1 addition & 1 deletion xarray/core/types.py
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ def copy(
ZarrWriteModes = Literal["w", "w-", "a", "a-", "r+", "r"]

GroupKey = Any
GroupIndex = Union[int, slice, list[int]]
GroupIndex = Union[slice, list[int]]
GroupIndices = tuple[GroupIndex, ...]
Bins = Union[
int, Sequence[int], Sequence[float], Sequence[pd.Timestamp], np.ndarray, pd.Index
Expand Down
23 changes: 22 additions & 1 deletion xarray/core/variable.py
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,13 @@
maybe_coerce_to_str,
)
from xarray.namedarray.core import NamedArray, _raise_if_any_duplicate_dimensions
from xarray.namedarray.pycompat import integer_types, is_0d_dask_array, to_duck_array
from xarray.namedarray.parallelcompat import get_chunked_array_type
from xarray.namedarray.pycompat import (
integer_types,
is_0d_dask_array,
is_chunked_array,
to_duck_array,
)
from xarray.util.deprecation_helpers import deprecate_dims

NON_NUMPY_SUPPORTED_ARRAY_TYPES = (
Expand Down Expand Up @@ -998,6 +1004,21 @@ def compute(self, **kwargs):
new = self.copy(deep=False)
return new.load(**kwargs)

def _shuffle(self, indices: list[slice | list[int]], dim: Hashable) -> Self:
size = self.sizes[dim]
indices: list[list[int]] = [
list(range(*idx.indices(size))) if isinstance(idx, slice) else idx
for idx in indices
]
array = self._data
if is_chunked_array(array):
chunkmanager = get_chunked_array_type(array)
return chunkmanager.shuffle(
array, indexer=indices, axis=self.get_axis_num(dim)
)
else:
return self.isel({dim: np.concatenate(indices)})

def isel(
self,
indexers: Mapping[Any, Any] | None = None,
Expand Down
9 changes: 9 additions & 0 deletions xarray/namedarray/daskmanager.py
Original file line number Diff line number Diff line change
Expand Up @@ -251,3 +251,12 @@ def store(
targets=targets,
**kwargs,
)

def shuffle(self, x: DaskArray, indexer: list[list[int]], axis: int) -> DaskArray:
import dask.array

if not module_available("dask", minversion="2024.08.0"):
raise ValueError(
"This method is very inefficient on dask<2024.08.0. Please upgrade."
)
return dask.array.shuffle(x, indexer, axis)
dcherian marked this conversation as resolved.
Show resolved Hide resolved
5 changes: 5 additions & 0 deletions xarray/namedarray/parallelcompat.py
Original file line number Diff line number Diff line change
Expand Up @@ -364,6 +364,11 @@ def compute(
"""
raise NotImplementedError()

def shuffle(
self, x: T_ChunkedArray, indexer: list[list[int]], axis: int
) -> T_ChunkedArray:
raise NotImplementedError()

@property
def array_api(self) -> Any:
"""
Expand Down
1 change: 1 addition & 0 deletions xarray/tests/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ def _importorskip(
has_h5netcdf, requires_h5netcdf = _importorskip("h5netcdf")
has_cftime, requires_cftime = _importorskip("cftime")
has_dask, requires_dask = _importorskip("dask")
has_dask_ge_2024_08_0, _ = _importorskip("dask", minversion="2024.08.0")
with warnings.catch_warnings():
warnings.filterwarnings(
"ignore",
Expand Down
18 changes: 17 additions & 1 deletion xarray/tests/test_groupby.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
assert_identical,
create_test_data,
has_cftime,
has_dask_ge_2024_08_0,
has_flox,
requires_cftime,
requires_dask,
Expand Down Expand Up @@ -1293,11 +1294,26 @@ def test_groupby_sum(self) -> None:
assert_allclose(expected_sum_axis1, grouped.reduce(np.sum, "y"))
assert_allclose(expected_sum_axis1, grouped.sum("y"))

@pytest.mark.parametrize(
"shuffle",
[
pytest.param(
True,
marks=pytest.mark.skipif(
not has_dask_ge_2024_08_0, reason="dask too old"
),
),
False,
],
)
@pytest.mark.parametrize("method", ["sum", "mean", "median"])
def test_groupby_reductions(self, method) -> None:
def test_groupby_reductions(self, method: str, shuffle: bool) -> None:
array = self.da
grouped = array.groupby("abc")

if shuffle:
grouped.shuffle()

reduction = getattr(np, method)
expected = Dataset(
{
Expand Down
Loading