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

[REVIEW] Fetch correct grouping keys agg of dask groupby #9022

Merged
merged 3 commits into from
Aug 13, 2021
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
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
17 changes: 15 additions & 2 deletions python/dask_cudf/dask_cudf/groupby.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
from dask.dataframe.groupby import DataFrameGroupBy, SeriesGroupBy
from dask.highlevelgraph import HighLevelGraph

import cudf


class CudfDataFrameGroupBy(DataFrameGroupBy):
def __init__(self, *args, **kwargs):
Expand Down Expand Up @@ -76,12 +78,23 @@ def aggregate(self, arg, split_every=None, split_out=1):
}
if (
isinstance(self.obj, DaskDataFrame)
and isinstance(self.index, (str, list))
and (
isinstance(self.index, str)
or (
isinstance(self.index, list)
and all(isinstance(x, str) for x in self.index)
)
)
and _is_supported(arg, _supported)
):
if isinstance(self._meta.grouping.keys, cudf.MultiIndex):
keys = self._meta.grouping.keys.names
else:
keys = self._meta.grouping.keys.name

return groupby_agg(
self.obj,
self.index,
keys,
arg,
split_every=split_every,
split_out=split_out,
Expand Down
14 changes: 14 additions & 0 deletions python/dask_cudf/dask_cudf/tests/test_groupby.py
Original file line number Diff line number Diff line change
Expand Up @@ -631,3 +631,17 @@ def test_groupby_first_last(data, agg):
getattr(gdf.groupby("a"), agg)(),
getattr(gddf.groupby("a"), agg)().compute(),
)


def test_groupby_with_list_of_series():
df = cudf.DataFrame({"a": [1, 2, 3, 4, 5]})
gdf = dask_cudf.from_cudf(df, npartitions=2)
gs = cudf.Series([1, 1, 1, 2, 2], name="id")
ggs = dask_cudf.from_cudf(gs, npartitions=2)

ddf = dd.from_pandas(df.to_pandas(), npartitions=2)
pgs = dd.from_pandas(gs.to_pandas(), npartitions=2)

dd.assert_eq(
gdf.groupby([ggs]).agg(["sum"]), ddf.groupby([pgs]).agg(["sum"])
)