Skip to content

Commit

Permalink
Raise error when sorting by categorical column in dask-cudf (#15788)
Browse files Browse the repository at this point in the history
Some dask-cudf tests are currently producing a segfault when sorting by categorical columns. These tests were already marked as "xfail". This PR goes one step further, and raises an error in the top-level `sort_values` API. This `NotImplementedError` can be removed as soon as the problem is fixed up-stream (working on this now, but probably won't be available for 24.06).

Authors:
  - Richard (Rick) Zamora (https://github.com/rjzamora)
  - GALI PREM SAGAR (https://github.com/galipremsagar)

Approvers:
  - GALI PREM SAGAR (https://github.com/galipremsagar)

URL: #15788
  • Loading branch information
rjzamora authored May 20, 2024
1 parent 58f4526 commit 4da00ea
Show file tree
Hide file tree
Showing 2 changed files with 35 additions and 2 deletions.
19 changes: 19 additions & 0 deletions python/dask_cudf/dask_cudf/expr/_collection.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

from dask import config
from dask.dataframe.core import is_dataframe_like
from dask.dataframe.dispatch import is_categorical_dtype

import cudf

Expand Down Expand Up @@ -81,6 +82,24 @@ def from_dict(cls, *args, **kwargs):
with config.set({"dataframe.backend": "cudf"}):
return DXDataFrame.from_dict(*args, **kwargs)

def sort_values(
self,
by,
**kwargs,
):
# Raise if the first column is categorical, otherwise the
# upstream divisions logic may produce errors
# (See: https://github.com/rapidsai/cudf/issues/11795)
check_by = by[0] if isinstance(by, list) else by
if is_categorical_dtype(self.dtypes.get(check_by, None)):
raise NotImplementedError(
"Dask-cudf does not support sorting on categorical "
"columns when query-planning is enabled. Please use "
"the legacy API for now."
f"\n{_LEGACY_WORKAROUND}",
)
return super().sort_values(by, **kwargs)

def groupby(
self,
by,
Expand Down
18 changes: 16 additions & 2 deletions python/dask_cudf/dask_cudf/tests/test_sort.py
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@
import cudf

import dask_cudf
from dask_cudf.tests.utils import xfail_dask_expr
from dask_cudf.tests.utils import QUERY_PLANNING_ON, xfail_dask_expr


@pytest.mark.parametrize("ascending", [True, False])
Expand All @@ -23,7 +23,7 @@
pytest.param(
"d",
marks=xfail_dask_expr(
"Dask-expr fails to sort by categorical column."
"Possible segfault when sorting by categorical column.",
),
),
["a", "b"],
Expand All @@ -47,6 +47,20 @@ def test_sort_values(nelem, nparts, by, ascending):
dd.assert_eq(got, expect, check_index=False)


@pytest.mark.parametrize("by", ["b", ["b", "a"]])
def test_sort_values_categorical_raises(by):
df = cudf.DataFrame()
df["a"] = np.ascontiguousarray(np.arange(10)[::-1])
df["b"] = df["a"].astype("category")
ddf = dd.from_pandas(df, npartitions=10)

if QUERY_PLANNING_ON:
with pytest.raises(
NotImplementedError, match="sorting on categorical"
):
ddf.sort_values(by=by)


@pytest.mark.parametrize("ascending", [True, False])
@pytest.mark.parametrize("by", ["a", "b", ["a", "b"]])
def test_sort_values_single_partition(by, ascending):
Expand Down

0 comments on commit 4da00ea

Please sign in to comment.