From 4da00eab26b10cd9445d7cb69373608f5685bb01 Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Mon, 20 May 2024 18:59:35 -0500 Subject: [PATCH] Raise error when sorting by categorical column in dask-cudf (#15788) 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: https://github.com/rapidsai/cudf/pull/15788 --- .../dask_cudf/dask_cudf/expr/_collection.py | 19 +++++++++++++++++++ python/dask_cudf/dask_cudf/tests/test_sort.py | 18 ++++++++++++++++-- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index d50dfb24256..926b7cfaf0e 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -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 @@ -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, diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 400600a1598..9d9fe297248 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -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]) @@ -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"], @@ -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):