From 0da06d0137e57e98ed52fb10373546f0023c4466 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 9 Jan 2024 10:07:41 -0800 Subject: [PATCH 01/74] fix groupby get-group --- python/cudf/cudf/core/groupby/groupby.py | 2 +- python/cudf/cudf/tests/groupby/test_indexing.py | 13 ++++++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 73e6774f5ce..7418dbb202f 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -377,7 +377,7 @@ def get_group(self, name, obj=None): if obj is None: obj = self.obj - return obj.loc[self.groups[name]] + return obj.loc[self.groups[name].drop_duplicates()] @_cudf_nvtx_annotate def size(self): diff --git a/python/cudf/cudf/tests/groupby/test_indexing.py b/python/cudf/cudf/tests/groupby/test_indexing.py index 06777c8e6af..57e8bc1c2d8 100644 --- a/python/cudf/cudf/tests/groupby/test_indexing.py +++ b/python/cudf/cudf/tests/groupby/test_indexing.py @@ -1 +1,12 @@ -# Copyright (c) 2023, NVIDIA CORPORATION. +# Copyright (c) 2023-2024, NVIDIA CORPORATION. +import cudf +from cudf.testing._utils import assert_eq + + +def test_rank_return_type_compatible_mode(): + # in compatible mode, rank() always returns floats + df = cudf.DataFrame({"a": range(10), "b": [0] * 10}, index=[0] * 10) + pdf = df.to_pandas() + expect = pdf.groupby("b").get_group(0) + result = df.groupby("b").get_group(0) + assert_eq(expect, result) From 3a4a5a0e91581754e4a6007429a1e2aff6b90fea Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 9 Jan 2024 10:11:47 -0800 Subject: [PATCH 02/74] update copyright --- python/cudf/cudf/core/groupby/groupby.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 7418dbb202f..3ee14835572 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. import copy import itertools From 675964c117d862a9087bd32f61fae7d618c544ed Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 16 Jan 2024 08:25:00 -0800 Subject: [PATCH 03/74] add new backend dispatching --- python/dask_cudf/dask_cudf/backends.py | 69 ++++++++++++++++++++++++++ python/dask_cudf/pyproject.toml | 2 + 2 files changed, 71 insertions(+) diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 387643587d1..b86c22c348f 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -638,3 +638,72 @@ def read_hdf(*args, **kwargs): return from_dask_dataframe( _default_backend(dd.read_hdf, *args, **kwargs) ) + + +# Define "cudf" backend engine to be registered with Dask-Expressions +class CudfDXBackendEntrypoint(DataFrameBackendEntrypoint): + """Backend-entrypoint class for Dask-Expressions + + This class is registered under the name "cudf" for the + ``dask-expr.dataframe.backends`` entrypoint in ``setup.cfg``. + Dask-DataFrame will use the methods defined in this class + in place of ``dask_expr.`` when the + "dataframe.backend" configuration is set to "cudf": + + Examples + -------- + >>> import dask + >>> import dask_expr + >>> with dask.config.set({"dataframe.backend": "cudf"}): + ... ddf = dx.from_dict({"a": range(10)}) + >>> type(ddf._meta) + + """ + + @classmethod + def to_backend_dispatch(cls): + return CudfBackendEntrypoint.to_backend_dispatch() + + @classmethod + def to_backend(cls, *args, **kwargs): + return CudfBackendEntrypoint.to_backend(*args, **kwargs) + + @staticmethod + def from_dict( + data, + npartitions, + orient="columns", + dtype=None, + columns=None, + constructor=cudf.DataFrame, + ): + import dask_expr as dx + + return _default_backend( + dx.from_dict, + data, + npartitions=npartitions, + orient=orient, + dtype=dtype, + columns=columns, + constructor=constructor, + ) + + @staticmethod + def read_parquet(*args, **kwargs): + import dask_expr as dx + + # Dask-Expressions can handle this (for now) + return _default_backend( + dx.read_parquet, + *args, + **kwargs, + ) + + @staticmethod + def read_csv(*args, **kwargs): + import dask_expr as dx + + from dask_cudf.io import read_csv + + return dx.from_dask_dataframe(read_csv(*args, **kwargs)) diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 33065da6e8d..5675b1f7bd2 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -38,6 +38,8 @@ classifiers = [ [project.entry-points."dask.dataframe.backends"] cudf = "dask_cudf.backends:CudfBackendEntrypoint" +[project.entry-points."dask_expr.dataframe.backends"] +cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ From b26c2709fca794ea31de82614fa2b9b01cbd412a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 17 Jan 2024 10:31:23 -0800 Subject: [PATCH 04/74] add meta-based dispatching patch for demonstration --- python/dask_cudf/dask_cudf/backends.py | 104 +++++++++++++++++++++---- 1 file changed, 90 insertions(+), 14 deletions(-) diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index b86c22c348f..1798b891609 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -1,5 +1,7 @@ # Copyright (c) 2020-2023, NVIDIA CORPORATION. +from __future__ import annotations +import functools import warnings from collections.abc import Iterator @@ -689,21 +691,95 @@ def from_dict( constructor=constructor, ) - @staticmethod - def read_parquet(*args, **kwargs): - import dask_expr as dx - # Dask-Expressions can handle this (for now) - return _default_backend( - dx.read_parquet, - *args, - **kwargs, - ) +try: + # Import and monkey-patch dask-expr for cudf support + from dask_expr._collection import ( + DataFrame as DXDataFrame, + Index as DXIndex, + Series as DXSeries, + ) + from dask_expr._cumulative import CumulativeBlockwise, TakeLast + from dask_expr._dispatch import get_collection_type + from dask_expr._expr import Expr - @staticmethod - def read_csv(*args, **kwargs): - import dask_expr as dx + __ext_dispatch_classes = {} # Track registered "external" dispatch classes - from dask_cudf.io import read_csv + def register_dispatch(cls, meta_type, ext_cls=None): + """Register an external/custom expression dispatch""" + + def wrapper(ext_cls): + if cls not in __ext_dispatch_classes: + __ext_dispatch_classes[cls] = Dispatch( + f"{cls.__qualname__}_dispatch" + ) + if isinstance(meta_type, tuple): + for t in meta_type: + __ext_dispatch_classes[cls].register(t, ext_cls) + else: + __ext_dispatch_classes[cls].register(meta_type, ext_cls) + return ext_cls + + return wrapper(ext_cls) if ext_cls is not None else wrapper + + def _override_new_expr(cls, *args, **kwargs): + """Override the __new__ method of an Expr class""" + if args and isinstance(args[0], Expr): + meta = args[0]._meta + try: + use_cls = __ext_dispatch_classes[cls].dispatch(type(meta)) + except (KeyError, TypeError): + use_cls = None # Default case + if use_cls: + return use_cls(*args, **kwargs) + return object.__new__(cls) + + # Monkey-patch `Expr` with meta-based dispatching + Expr.register_dispatch = classmethod(register_dispatch) + Expr.__new__ = _override_new_expr + + ## + ## Custom expression classes + ## + + @CumulativeBlockwise.register_dispatch((cudf.DataFrame, cudf.Series)) + class CumulativeBlockwiseCudf(CumulativeBlockwise): + @functools.cached_property + def _args(self) -> list: + return self.operands[:1] + + @functools.cached_property + def _kwargs(self) -> dict: + # Must pass axis and skipna as kwargs in cudf + return {"axis": self.axis, "skipna": self.skipna} + + @TakeLast.register_dispatch((cudf.DataFrame, cudf.Series)) + class TakeLastCudf(TakeLast): + @staticmethod + def operation(a, skipna=True): + if not len(a): + return a + if skipna: + a = a.bfill() + # Cannot use `squeeze` with cudf + return a.tail(n=1).iloc[0] + + ## + ## Custom collection classes + ## + + @get_collection_type.register(cudf.DataFrame) + class DataFrameCudf(DXDataFrame): + pass # Same as pandas (for now) + + @get_collection_type.register(cudf.Series) + class SeriesCudf(DXSeries): + pass # Same as pandas (for now) + + @get_collection_type.register(cudf.BaseIndex) + class IndexCudf(DXIndex): + pass # Same as pandas (for now) - return dx.from_dask_dataframe(read_csv(*args, **kwargs)) +except ImportError: + # Compatible dask_expr version not installed + pass From b6a4cac6cb6003cf61fd3768ee06953981f7b789 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 17 Jan 2024 10:41:37 -0800 Subject: [PATCH 05/74] fix get_collection_type registration --- python/dask_cudf/dask_cudf/backends.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 1798b891609..1b26b86fbb8 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -768,18 +768,19 @@ def operation(a, skipna=True): ## Custom collection classes ## - @get_collection_type.register(cudf.DataFrame) class DataFrameCudf(DXDataFrame): pass # Same as pandas (for now) - @get_collection_type.register(cudf.Series) class SeriesCudf(DXSeries): pass # Same as pandas (for now) - @get_collection_type.register(cudf.BaseIndex) class IndexCudf(DXIndex): pass # Same as pandas (for now) + get_collection_type.register(cudf.DataFrame, lambda _: DataFrameCudf) + get_collection_type.register(cudf.Series, lambda _: SeriesCudf) + get_collection_type.register(cudf.BaseIndex, lambda _: IndexCudf) + except ImportError: # Compatible dask_expr version not installed pass From a6ee37ad881c98cdeccf2f8272aa8add7b6aea79 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 18 Jan 2024 12:02:07 -0800 Subject: [PATCH 06/74] re-org --- python/dask_cudf/dask_cudf/backends.py | 93 +------------------ .../dask_cudf/expr_backend/__init__.py | 10 ++ .../dask_cudf/expr_backend/_collection.py | 53 +++++++++++ .../dask_cudf/dask_cudf/expr_backend/_expr.py | 35 +++++++ .../dask_cudf/expr_backend/_groupby.py | 15 +++ .../dask_cudf/expr_backend/_utils.py | 44 +++++++++ 6 files changed, 159 insertions(+), 91 deletions(-) create mode 100644 python/dask_cudf/dask_cudf/expr_backend/__init__.py create mode 100644 python/dask_cudf/dask_cudf/expr_backend/_collection.py create mode 100644 python/dask_cudf/dask_cudf/expr_backend/_expr.py create mode 100644 python/dask_cudf/dask_cudf/expr_backend/_groupby.py create mode 100644 python/dask_cudf/dask_cudf/expr_backend/_utils.py diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 1b26b86fbb8..2f0ae91dfad 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -1,7 +1,5 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. -from __future__ import annotations +# Copyright (c) 2020-2024, NVIDIA CORPORATION. -import functools import warnings from collections.abc import Iterator @@ -693,94 +691,7 @@ def from_dict( try: - # Import and monkey-patch dask-expr for cudf support - from dask_expr._collection import ( - DataFrame as DXDataFrame, - Index as DXIndex, - Series as DXSeries, - ) - from dask_expr._cumulative import CumulativeBlockwise, TakeLast - from dask_expr._dispatch import get_collection_type - from dask_expr._expr import Expr - - __ext_dispatch_classes = {} # Track registered "external" dispatch classes - - def register_dispatch(cls, meta_type, ext_cls=None): - """Register an external/custom expression dispatch""" - - def wrapper(ext_cls): - if cls not in __ext_dispatch_classes: - __ext_dispatch_classes[cls] = Dispatch( - f"{cls.__qualname__}_dispatch" - ) - if isinstance(meta_type, tuple): - for t in meta_type: - __ext_dispatch_classes[cls].register(t, ext_cls) - else: - __ext_dispatch_classes[cls].register(meta_type, ext_cls) - return ext_cls - - return wrapper(ext_cls) if ext_cls is not None else wrapper - - def _override_new_expr(cls, *args, **kwargs): - """Override the __new__ method of an Expr class""" - if args and isinstance(args[0], Expr): - meta = args[0]._meta - try: - use_cls = __ext_dispatch_classes[cls].dispatch(type(meta)) - except (KeyError, TypeError): - use_cls = None # Default case - if use_cls: - return use_cls(*args, **kwargs) - return object.__new__(cls) - - # Monkey-patch `Expr` with meta-based dispatching - Expr.register_dispatch = classmethod(register_dispatch) - Expr.__new__ = _override_new_expr - - ## - ## Custom expression classes - ## - - @CumulativeBlockwise.register_dispatch((cudf.DataFrame, cudf.Series)) - class CumulativeBlockwiseCudf(CumulativeBlockwise): - @functools.cached_property - def _args(self) -> list: - return self.operands[:1] - - @functools.cached_property - def _kwargs(self) -> dict: - # Must pass axis and skipna as kwargs in cudf - return {"axis": self.axis, "skipna": self.skipna} - - @TakeLast.register_dispatch((cudf.DataFrame, cudf.Series)) - class TakeLastCudf(TakeLast): - @staticmethod - def operation(a, skipna=True): - if not len(a): - return a - if skipna: - a = a.bfill() - # Cannot use `squeeze` with cudf - return a.tail(n=1).iloc[0] - - ## - ## Custom collection classes - ## - - class DataFrameCudf(DXDataFrame): - pass # Same as pandas (for now) - - class SeriesCudf(DXSeries): - pass # Same as pandas (for now) - - class IndexCudf(DXIndex): - pass # Same as pandas (for now) - - get_collection_type.register(cudf.DataFrame, lambda _: DataFrameCudf) - get_collection_type.register(cudf.Series, lambda _: SeriesCudf) - get_collection_type.register(cudf.BaseIndex, lambda _: IndexCudf) - + import dask_cudf.expr_backend # noqa: F401 except ImportError: # Compatible dask_expr version not installed pass diff --git a/python/dask_cudf/dask_cudf/expr_backend/__init__.py b/python/dask_cudf/dask_cudf/expr_backend/__init__.py new file mode 100644 index 00000000000..492d74e0185 --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr_backend/__init__.py @@ -0,0 +1,10 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +# Monkey-patch `Expr` with meta-based dispatching +from dask_cudf.expr_backend._dispatch_utils import patch_dask_expr + +patch_dask_expr() + +# Import the "cudf" backend +import dask_cudf.expr_backend._collection # noqa: F401 +import dask_cudf.expr_backend._expr # noqa: F401 diff --git a/python/dask_cudf/dask_cudf/expr_backend/_collection.py b/python/dask_cudf/dask_cudf/expr_backend/_collection.py new file mode 100644 index 00000000000..fbd875a218d --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr_backend/_collection.py @@ -0,0 +1,53 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +from dask_expr import DataFrame, FrameBase, Index, Series, get_collection_type + +import cudf + +## +## Custom collection classes +## + + +class DataFrameCudf(DataFrame): + def groupby( + self, + by, + group_keys=True, + sort=None, + observed=None, + dropna=None, + **kwargs, + ): + from dask_cudf.expr_backend._groupby import GroupByCudf + + if isinstance(by, FrameBase) and not isinstance(by, Series): + raise ValueError( + f"`by` must be a column name or list of columns, got {by}." + ) + + return GroupByCudf( + self, + by, + group_keys=group_keys, + sort=sort, + observed=observed, + dropna=dropna, + **kwargs, + ) + + +class SeriesCudf(Series): + def groupby(self, by, **kwargs): + from dask_cudf.expr_backend._groupby import SeriesGroupByCudf + + return SeriesGroupByCudf(self, by, **kwargs) + + +class IndexCudf(Index): + pass # Same as pandas (for now) + + +get_collection_type.register(cudf.DataFrame, lambda _: DataFrameCudf) +get_collection_type.register(cudf.Series, lambda _: SeriesCudf) +get_collection_type.register(cudf.BaseIndex, lambda _: IndexCudf) diff --git a/python/dask_cudf/dask_cudf/expr_backend/_expr.py b/python/dask_cudf/dask_cudf/expr_backend/_expr.py new file mode 100644 index 00000000000..65f38f0f310 --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr_backend/_expr.py @@ -0,0 +1,35 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +import functools + +from dask_expr._cumulative import CumulativeBlockwise, TakeLast + +import cudf + +## +## Custom expression classes +## + + +@CumulativeBlockwise.register_dispatch((cudf.DataFrame, cudf.Series)) +class CumulativeBlockwiseCudf(CumulativeBlockwise): + @functools.cached_property + def _args(self) -> list: + return self.operands[:1] + + @functools.cached_property + def _kwargs(self) -> dict: + # Must pass axis and skipna as kwargs in cudf + return {"axis": self.axis, "skipna": self.skipna} + + +@TakeLast.register_dispatch((cudf.DataFrame, cudf.Series)) +class TakeLastCudf(TakeLast): + @staticmethod + def operation(a, skipna=True): + if not len(a): + return a + if skipna: + a = a.bfill() + # Cannot use `squeeze` with cudf + return a.tail(n=1).iloc[0] diff --git a/python/dask_cudf/dask_cudf/expr_backend/_groupby.py b/python/dask_cudf/dask_cudf/expr_backend/_groupby.py new file mode 100644 index 00000000000..884bba41716 --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr_backend/_groupby.py @@ -0,0 +1,15 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +from dask_expr._groupby import GroupBy, SeriesGroupBy + + +class GroupByCudf(GroupBy): + def __init__(self, *args, observed=None, **kwargs): + observed = observed if observed is not None else True + super().__init__(*args, observed=observed, **kwargs) + + +class SeriesGroupByCudf(SeriesGroupBy): + def __init__(self, *args, observed=None, **kwargs): + observed = observed if observed is not None else True + super().__init__(*args, observed=observed, **kwargs) diff --git a/python/dask_cudf/dask_cudf/expr_backend/_utils.py b/python/dask_cudf/dask_cudf/expr_backend/_utils.py new file mode 100644 index 00000000000..3ed8fb20219 --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr_backend/_utils.py @@ -0,0 +1,44 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +from dask_expr._expr import Expr + +from dask.utils import Dispatch + +__ext_dispatch_classes = {} # Track registered "external" dispatch classes + + +def register_dispatch(cls, meta_type, ext_cls=None): + """Register an external/custom expression dispatch""" + + def wrapper(ext_cls): + if cls not in __ext_dispatch_classes: + __ext_dispatch_classes[cls] = Dispatch( + f"{cls.__qualname__}_dispatch" + ) + if isinstance(meta_type, tuple): + for t in meta_type: + __ext_dispatch_classes[cls].register(t, ext_cls) + else: + __ext_dispatch_classes[cls].register(meta_type, ext_cls) + return ext_cls + + return wrapper(ext_cls) if ext_cls is not None else wrapper + + +def _override_new_expr(cls, *args, **kwargs): + """Override the __new__ method of an Expr class""" + if args and isinstance(args[0], Expr): + meta = args[0]._meta + try: + use_cls = __ext_dispatch_classes[cls].dispatch(type(meta)) + except (KeyError, TypeError): + use_cls = None # Default case + if use_cls: + return use_cls(*args, **kwargs) + return object.__new__(cls) + + +def patch_dask_expr(): + """Monkey-patch `Expr` with meta-based dispatching""" + Expr.register_dispatch = classmethod(register_dispatch) + Expr.__new__ = _override_new_expr From 40687bd635da567183c7dbebf8df1097d9f35856 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 19 Jan 2024 10:52:17 -0800 Subject: [PATCH 07/74] rename --- python/dask_cudf/dask_cudf/backends.py | 2 +- .../_utils.py => expr/__init__.py} | 10 +++-- .../{expr_backend => expr}/_collection.py | 30 ++++++++----- .../dask_cudf/{expr_backend => expr}/_expr.py | 0 python/dask_cudf/dask_cudf/expr/_groupby.py | 44 +++++++++++++++++++ .../dask_cudf/expr_backend/__init__.py | 10 ----- .../dask_cudf/expr_backend/_groupby.py | 15 ------- 7 files changed, 69 insertions(+), 42 deletions(-) rename python/dask_cudf/dask_cudf/{expr_backend/_utils.py => expr/__init__.py} (85%) rename python/dask_cudf/dask_cudf/{expr_backend => expr}/_collection.py (54%) rename python/dask_cudf/dask_cudf/{expr_backend => expr}/_expr.py (100%) create mode 100644 python/dask_cudf/dask_cudf/expr/_groupby.py delete mode 100644 python/dask_cudf/dask_cudf/expr_backend/__init__.py delete mode 100644 python/dask_cudf/dask_cudf/expr_backend/_groupby.py diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 2f0ae91dfad..8884e917e0a 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -691,7 +691,7 @@ def from_dict( try: - import dask_cudf.expr_backend # noqa: F401 + import dask_cudf.expr # noqa: F401 except ImportError: # Compatible dask_expr version not installed pass diff --git a/python/dask_cudf/dask_cudf/expr_backend/_utils.py b/python/dask_cudf/dask_cudf/expr/__init__.py similarity index 85% rename from python/dask_cudf/dask_cudf/expr_backend/_utils.py rename to python/dask_cudf/dask_cudf/expr/__init__.py index 3ed8fb20219..6056b57cc19 100644 --- a/python/dask_cudf/dask_cudf/expr_backend/_utils.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -38,7 +38,9 @@ def _override_new_expr(cls, *args, **kwargs): return object.__new__(cls) -def patch_dask_expr(): - """Monkey-patch `Expr` with meta-based dispatching""" - Expr.register_dispatch = classmethod(register_dispatch) - Expr.__new__ = _override_new_expr +Expr.register_dispatch = classmethod(register_dispatch) +Expr.__new__ = _override_new_expr + +# Make sure custom expressions and collections are defined +import dask_cudf.expr._collection +import dask_cudf.expr._expr diff --git a/python/dask_cudf/dask_cudf/expr_backend/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py similarity index 54% rename from python/dask_cudf/dask_cudf/expr_backend/_collection.py rename to python/dask_cudf/dask_cudf/expr/_collection.py index fbd875a218d..6fff3260beb 100644 --- a/python/dask_cudf/dask_cudf/expr_backend/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -1,6 +1,12 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -from dask_expr import DataFrame, FrameBase, Index, Series, get_collection_type +from dask_expr import ( + DataFrame as DXDataFrame, + FrameBase, + Index as DXIndex, + Series as DXSeries, + get_collection_type, +) import cudf @@ -9,7 +15,7 @@ ## -class DataFrameCudf(DataFrame): +class DataFrame(DXDataFrame): def groupby( self, by, @@ -19,14 +25,14 @@ def groupby( dropna=None, **kwargs, ): - from dask_cudf.expr_backend._groupby import GroupByCudf + from dask_cudf.expr._groupby import GroupBy - if isinstance(by, FrameBase) and not isinstance(by, Series): + if isinstance(by, FrameBase) and not isinstance(by, DXSeries): raise ValueError( f"`by` must be a column name or list of columns, got {by}." ) - return GroupByCudf( + return GroupBy( self, by, group_keys=group_keys, @@ -37,17 +43,17 @@ def groupby( ) -class SeriesCudf(Series): +class Series(DXSeries): def groupby(self, by, **kwargs): - from dask_cudf.expr_backend._groupby import SeriesGroupByCudf + from dask_cudf.expr._groupby import SeriesGroupBy - return SeriesGroupByCudf(self, by, **kwargs) + return SeriesGroupBy(self, by, **kwargs) -class IndexCudf(Index): +class Index(DXIndex): pass # Same as pandas (for now) -get_collection_type.register(cudf.DataFrame, lambda _: DataFrameCudf) -get_collection_type.register(cudf.Series, lambda _: SeriesCudf) -get_collection_type.register(cudf.BaseIndex, lambda _: IndexCudf) +get_collection_type.register(cudf.DataFrame, lambda _: DataFrame) +get_collection_type.register(cudf.Series, lambda _: Series) +get_collection_type.register(cudf.BaseIndex, lambda _: Index) diff --git a/python/dask_cudf/dask_cudf/expr_backend/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py similarity index 100% rename from python/dask_cudf/dask_cudf/expr_backend/_expr.py rename to python/dask_cudf/dask_cudf/expr/_expr.py diff --git a/python/dask_cudf/dask_cudf/expr/_groupby.py b/python/dask_cudf/dask_cudf/expr/_groupby.py new file mode 100644 index 00000000000..8e67932fd3c --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr/_groupby.py @@ -0,0 +1,44 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +from dask_expr._groupby import ( + GroupBy as DXGroupBy, + SeriesGroupBy as DXSeriesGroupBy, +) +from dask_expr._util import is_scalar + +## +## Custom groupby classes +## + + +class GroupBy(DXGroupBy): + def __init__(self, *args, observed=None, **kwargs): + observed = observed if observed is not None else True + super().__init__(*args, observed=observed, **kwargs) + + def __getitem__(self, key): + if is_scalar(key): + return SeriesGroupBy( + self.obj, + by=self.by, + slice=key, + sort=self.sort, + dropna=self.dropna, + observed=self.observed, + ) + g = GroupBy( + self.obj, + by=self.by, + slice=key, + sort=self.sort, + dropna=self.dropna, + observed=self.observed, + group_keys=self.group_keys, + ) + return g + + +class SeriesGroupBy(DXSeriesGroupBy): + def __init__(self, *args, observed=None, **kwargs): + observed = observed if observed is not None else True + super().__init__(*args, observed=observed, **kwargs) diff --git a/python/dask_cudf/dask_cudf/expr_backend/__init__.py b/python/dask_cudf/dask_cudf/expr_backend/__init__.py deleted file mode 100644 index 492d74e0185..00000000000 --- a/python/dask_cudf/dask_cudf/expr_backend/__init__.py +++ /dev/null @@ -1,10 +0,0 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. - -# Monkey-patch `Expr` with meta-based dispatching -from dask_cudf.expr_backend._dispatch_utils import patch_dask_expr - -patch_dask_expr() - -# Import the "cudf" backend -import dask_cudf.expr_backend._collection # noqa: F401 -import dask_cudf.expr_backend._expr # noqa: F401 diff --git a/python/dask_cudf/dask_cudf/expr_backend/_groupby.py b/python/dask_cudf/dask_cudf/expr_backend/_groupby.py deleted file mode 100644 index 884bba41716..00000000000 --- a/python/dask_cudf/dask_cudf/expr_backend/_groupby.py +++ /dev/null @@ -1,15 +0,0 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. - -from dask_expr._groupby import GroupBy, SeriesGroupBy - - -class GroupByCudf(GroupBy): - def __init__(self, *args, observed=None, **kwargs): - observed = observed if observed is not None else True - super().__init__(*args, observed=observed, **kwargs) - - -class SeriesGroupByCudf(SeriesGroupBy): - def __init__(self, *args, observed=None, **kwargs): - observed = observed if observed is not None else True - super().__init__(*args, observed=observed, **kwargs) From 04a1b24552255a3fe7249de9a9c617abc9c54243 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 19 Jan 2024 10:56:31 -0800 Subject: [PATCH 08/74] comment --- python/dask_cudf/dask_cudf/backends.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 8884e917e0a..21d42926ffc 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -640,7 +640,7 @@ def read_hdf(*args, **kwargs): ) -# Define "cudf" backend engine to be registered with Dask-Expressions +# Define "cudf" backend entrypoint for dask-expr class CudfDXBackendEntrypoint(DataFrameBackendEntrypoint): """Backend-entrypoint class for Dask-Expressions @@ -690,6 +690,7 @@ def from_dict( ) +# Import/register cudf-specific classes for dask-expr try: import dask_cudf.expr # noqa: F401 except ImportError: From eb9fc88835cde8b210b983c913c991dd6099f148 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 22 Jan 2024 11:55:57 -0800 Subject: [PATCH 09/74] update __dask_tokenize__ --- python/cudf/cudf/core/frame.py | 6 ++++-- python/cudf/cudf/core/indexed_frame.py | 8 +++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/core/frame.py b/python/cudf/cudf/core/frame.py index fc313a62fd0..2680dccfd34 100644 --- a/python/cudf/cudf/core/frame.py +++ b/python/cudf/cudf/core/frame.py @@ -2853,10 +2853,12 @@ def _repeat( @_cudf_nvtx_annotate @_warn_no_dask_cudf def __dask_tokenize__(self): + from dask.base import normalize_token + return [ type(self), - self._dtypes, - self.to_pandas(), + normalize_token(self._dtypes), + normalize_token(self.to_pandas()), ] diff --git a/python/cudf/cudf/core/indexed_frame.py b/python/cudf/cudf/core/indexed_frame.py index 3e564919090..6320d1fb5fb 100644 --- a/python/cudf/cudf/core/indexed_frame.py +++ b/python/cudf/cudf/core/indexed_frame.py @@ -5198,11 +5198,13 @@ def convert_dtypes( @_warn_no_dask_cudf def __dask_tokenize__(self): + from dask.base import normalize_token + return [ type(self), - self._dtypes, - self.index, - self.hash_values().values_host, + normalize_token(self._dtypes), + normalize_token(self.index), + normalize_token(self.hash_values().values_host), ] From 61fae84659eef9e6ef6f5197da1acfae77f7333e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 22 Jan 2024 11:57:58 -0800 Subject: [PATCH 10/74] update __dask_tokenize__ --- python/cudf/cudf/core/frame.py | 6 ++++-- python/cudf/cudf/core/indexed_frame.py | 8 +++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/python/cudf/cudf/core/frame.py b/python/cudf/cudf/core/frame.py index fc313a62fd0..2680dccfd34 100644 --- a/python/cudf/cudf/core/frame.py +++ b/python/cudf/cudf/core/frame.py @@ -2853,10 +2853,12 @@ def _repeat( @_cudf_nvtx_annotate @_warn_no_dask_cudf def __dask_tokenize__(self): + from dask.base import normalize_token + return [ type(self), - self._dtypes, - self.to_pandas(), + normalize_token(self._dtypes), + normalize_token(self.to_pandas()), ] diff --git a/python/cudf/cudf/core/indexed_frame.py b/python/cudf/cudf/core/indexed_frame.py index 3e564919090..6320d1fb5fb 100644 --- a/python/cudf/cudf/core/indexed_frame.py +++ b/python/cudf/cudf/core/indexed_frame.py @@ -5198,11 +5198,13 @@ def convert_dtypes( @_warn_no_dask_cudf def __dask_tokenize__(self): + from dask.base import normalize_token + return [ type(self), - self._dtypes, - self.index, - self.hash_values().values_host, + normalize_token(self._dtypes), + normalize_token(self.index), + normalize_token(self.hash_values().values_host), ] From 92a36d5858b3527e79904fc978883a669f10dbcc Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 22 Jan 2024 12:03:01 -0800 Subject: [PATCH 11/74] add test coverage --- python/dask_cudf/dask_cudf/tests/test_dispatch.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index c64e25fd437..f37e82d4c4e 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -1,5 +1,7 @@ # Copyright (c) 2021-2023, NVIDIA CORPORATION. +from datetime import datetime + import numpy as np import pandas as pd import pytest @@ -82,6 +84,16 @@ def test_deterministic_tokenize(index): assert tokenize(df2) == tokenize(df2) +def test_deterministic_tokenize_multiindex(): + dt = datetime.strptime("1995-03-15", "%Y-%m-%d") + index = cudf.MultiIndex( + levels=[[1, 2], [dt]], + codes=[[0, 1], [0, 0]], + ) + df = cudf.DataFrame(index=index) + assert tokenize(df) == tokenize(df) + + @pytest.mark.parametrize("preserve_index", [True, False]) def test_pyarrow_schema_dispatch(preserve_index): from dask.dataframe.dispatch import ( From c3d69f96b55c4cae6e1d4e329a81c50d0375d74b Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 23 Jan 2024 05:17:25 -0800 Subject: [PATCH 12/74] fix date --- python/dask_cudf/dask_cudf/tests/test_dispatch.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index f37e82d4c4e..76703206726 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2023, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. from datetime import datetime From ed11879b2fb85dcd405ad320e4d8b90d0c9b0eef Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 23 Jan 2024 12:28:53 -0800 Subject: [PATCH 13/74] remove __new__ patchhing - we want to avoid checking _meta upon creation --- python/dask_cudf/dask_cudf/expr/__init__.py | 41 --------------------- python/dask_cudf/dask_cudf/expr/_expr.py | 11 ++++-- 2 files changed, 7 insertions(+), 45 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index 6056b57cc19..c18bd1b47ea 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -1,46 +1,5 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -from dask_expr._expr import Expr - -from dask.utils import Dispatch - -__ext_dispatch_classes = {} # Track registered "external" dispatch classes - - -def register_dispatch(cls, meta_type, ext_cls=None): - """Register an external/custom expression dispatch""" - - def wrapper(ext_cls): - if cls not in __ext_dispatch_classes: - __ext_dispatch_classes[cls] = Dispatch( - f"{cls.__qualname__}_dispatch" - ) - if isinstance(meta_type, tuple): - for t in meta_type: - __ext_dispatch_classes[cls].register(t, ext_cls) - else: - __ext_dispatch_classes[cls].register(meta_type, ext_cls) - return ext_cls - - return wrapper(ext_cls) if ext_cls is not None else wrapper - - -def _override_new_expr(cls, *args, **kwargs): - """Override the __new__ method of an Expr class""" - if args and isinstance(args[0], Expr): - meta = args[0]._meta - try: - use_cls = __ext_dispatch_classes[cls].dispatch(type(meta)) - except (KeyError, TypeError): - use_cls = None # Default case - if use_cls: - return use_cls(*args, **kwargs) - return object.__new__(cls) - - -Expr.register_dispatch = classmethod(register_dispatch) -Expr.__new__ = _override_new_expr - # Make sure custom expressions and collections are defined import dask_cudf.expr._collection import dask_cudf.expr._expr diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 65f38f0f310..6e5e0739c80 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -4,14 +4,11 @@ from dask_expr._cumulative import CumulativeBlockwise, TakeLast -import cudf - ## ## Custom expression classes ## -@CumulativeBlockwise.register_dispatch((cudf.DataFrame, cudf.Series)) class CumulativeBlockwiseCudf(CumulativeBlockwise): @functools.cached_property def _args(self) -> list: @@ -23,7 +20,10 @@ def _kwargs(self) -> dict: return {"axis": self.axis, "skipna": self.skipna} -@TakeLast.register_dispatch((cudf.DataFrame, cudf.Series)) +CumulativeBlockwise._args = CumulativeBlockwiseCudf._args +CumulativeBlockwise._kwargs = CumulativeBlockwiseCudf._kwargs + + class TakeLastCudf(TakeLast): @staticmethod def operation(a, skipna=True): @@ -33,3 +33,6 @@ def operation(a, skipna=True): a = a.bfill() # Cannot use `squeeze` with cudf return a.tail(n=1).iloc[0] + + +TakeLast.operation = staticmethod(TakeLastCudf.operation) From 7b984b22774f18c01f628aef525f0eb37477f63f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 23 Jan 2024 12:56:25 -0800 Subject: [PATCH 14/74] Avoid unnecessary caching --- python/dask_cudf/dask_cudf/expr/_expr.py | 32 +++++++++++------------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 6e5e0739c80..0dd61045231 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -1,38 +1,34 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -import functools - from dask_expr._cumulative import CumulativeBlockwise, TakeLast ## -## Custom expression classes +## Custom expression patching ## -class CumulativeBlockwiseCudf(CumulativeBlockwise): - @functools.cached_property +class PatchCumulativeBlockwise(CumulativeBlockwise): + @property def _args(self) -> list: return self.operands[:1] - @functools.cached_property + @property def _kwargs(self) -> dict: # Must pass axis and skipna as kwargs in cudf return {"axis": self.axis, "skipna": self.skipna} -CumulativeBlockwise._args = CumulativeBlockwiseCudf._args -CumulativeBlockwise._kwargs = CumulativeBlockwiseCudf._kwargs +CumulativeBlockwise._args = PatchCumulativeBlockwise._args +CumulativeBlockwise._kwargs = PatchCumulativeBlockwise._kwargs -class TakeLastCudf(TakeLast): - @staticmethod - def operation(a, skipna=True): - if not len(a): - return a - if skipna: - a = a.bfill() - # Cannot use `squeeze` with cudf - return a.tail(n=1).iloc[0] +def _takelast(a, skipna=True): + if not len(a): + return a + if skipna: + a = a.bfill() + # Cannot use `squeeze` with cudf + return a.tail(n=1).iloc[0] -TakeLast.operation = staticmethod(TakeLastCudf.operation) +TakeLast.operation = staticmethod(_takelast) From c01002be6b66ddae923290e715e0387c34fc790a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 30 Jan 2024 14:43:45 -0800 Subject: [PATCH 15/74] move dask tests that depend on cudf code --- python/cudf/cudf/tests/test_dask.py | 70 +++++++++++++++++-- .../dask_cudf/tests/test_dispatch.py | 53 -------------- 2 files changed, 63 insertions(+), 60 deletions(-) diff --git a/python/cudf/cudf/tests/test_dask.py b/python/cudf/cudf/tests/test_dask.py index 3af21b4a7ff..e7eccb5cd36 100644 --- a/python/cudf/cudf/tests/test_dask.py +++ b/python/cudf/cudf/tests/test_dask.py @@ -1,17 +1,19 @@ -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2019-2024, NVIDIA CORPORATION. + +from datetime import datetime import pytest import cudf -is_dataframe_like = pytest.importorskip( - "dask.dataframe.utils" -).is_dataframe_like -is_index_like = pytest.importorskip("dask.dataframe.utils").is_index_like -is_series_like = pytest.importorskip("dask.dataframe.utils").is_series_like - def test_is_dataframe_like(): + is_dataframe_like = pytest.importorskip( + "dask.dataframe.utils" + ).is_dataframe_like + is_index_like = pytest.importorskip("dask.dataframe.utils").is_index_like + is_series_like = pytest.importorskip("dask.dataframe.utils").is_series_like + df = cudf.DataFrame({"x": [1, 2, 3]}) assert is_dataframe_like(df) assert is_series_like(df.x) @@ -19,3 +21,57 @@ def test_is_dataframe_like(): assert not is_dataframe_like(df.x) assert not is_series_like(df) assert not is_index_like(df) + + +@pytest.mark.parametrize("index", [None, [1, 2] * 5]) +def test_deterministic_tokenize(index): + tokenize = pytest.importorskip("dask.base").tokenize + + # Checks that `dask.base.normalize_token` correctly + # dispatches to the logic defined in `backends.py` + # (making `tokenize()` deterministic). + df = cudf.DataFrame( + {"A": range(10), "B": ["dog", "cat"] * 5, "C": range(10, 0, -1)}, + index=index, + ) + + # Matching data should produce the same token + assert tokenize(df) == tokenize(df) + assert tokenize(df.A) == tokenize(df.A) + assert tokenize(df.index) == tokenize(df.index) + assert tokenize(df) == tokenize(df.copy(deep=True)) + assert tokenize(df.A) == tokenize(df.A.copy(deep=True)) + assert tokenize(df.index) == tokenize(df.index.copy(deep=True)) + + # Modifying a column element should change the token + original_token = tokenize(df) + original_token_a = tokenize(df.A) + df.A.iloc[2] = 10 + assert original_token != tokenize(df) + assert original_token_a != tokenize(df.A) + + # Modifying an index element should change the token + original_token = tokenize(df) + original_token_index = tokenize(df.index) + new_index = df.index.values + new_index[2] = 10 + df.index = new_index + assert original_token != tokenize(df) + assert original_token_index != tokenize(df.index) + + # Check MultiIndex case + df2 = df.set_index(["B", "C"], drop=False) + assert tokenize(df) != tokenize(df2) + assert tokenize(df2) == tokenize(df2) + + +def test_deterministic_tokenize_multiindex(): + tokenize = pytest.importorskip("dask.base").tokenize + + dt = datetime.strptime("1995-03-15", "%Y-%m-%d") + index = cudf.MultiIndex( + levels=[[1, 2], [dt]], + codes=[[0, 1], [0, 0]], + ) + df = cudf.DataFrame(index=index) + assert tokenize(df) == tokenize(df) diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index 76703206726..34d7e0f1b3c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -1,12 +1,9 @@ # Copyright (c) 2021-2024, NVIDIA CORPORATION. -from datetime import datetime - import numpy as np import pandas as pd import pytest -from dask.base import tokenize from dask.dataframe import assert_eq from dask.dataframe.methods import is_categorical_dtype @@ -44,56 +41,6 @@ def test_pyarrow_conversion_dispatch(preserve_index): assert not isinstance(df2.index, cudf.RangeIndex) -@pytest.mark.parametrize("index", [None, [1, 2] * 5]) -def test_deterministic_tokenize(index): - # Checks that `dask.base.normalize_token` correctly - # dispatches to the logic defined in `backends.py` - # (making `tokenize()` deterministic). - df = cudf.DataFrame( - {"A": range(10), "B": ["dog", "cat"] * 5, "C": range(10, 0, -1)}, - index=index, - ) - - # Matching data should produce the same token - assert tokenize(df) == tokenize(df) - assert tokenize(df.A) == tokenize(df.A) - assert tokenize(df.index) == tokenize(df.index) - assert tokenize(df) == tokenize(df.copy(deep=True)) - assert tokenize(df.A) == tokenize(df.A.copy(deep=True)) - assert tokenize(df.index) == tokenize(df.index.copy(deep=True)) - - # Modifying a column element should change the token - original_token = tokenize(df) - original_token_a = tokenize(df.A) - df.A.iloc[2] = 10 - assert original_token != tokenize(df) - assert original_token_a != tokenize(df.A) - - # Modifying an index element should change the token - original_token = tokenize(df) - original_token_index = tokenize(df.index) - new_index = df.index.values - new_index[2] = 10 - df.index = new_index - assert original_token != tokenize(df) - assert original_token_index != tokenize(df.index) - - # Check MultiIndex case - df2 = df.set_index(["B", "C"], drop=False) - assert tokenize(df) != tokenize(df2) - assert tokenize(df2) == tokenize(df2) - - -def test_deterministic_tokenize_multiindex(): - dt = datetime.strptime("1995-03-15", "%Y-%m-%d") - index = cudf.MultiIndex( - levels=[[1, 2], [dt]], - codes=[[0, 1], [0, 0]], - ) - df = cudf.DataFrame(index=index) - assert tokenize(df) == tokenize(df) - - @pytest.mark.parametrize("preserve_index", [True, False]) def test_pyarrow_schema_dispatch(preserve_index): from dask.dataframe.dispatch import ( From 90defebd6ec21320a3176f17e04f54159ed05d5a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 30 Jan 2024 18:28:56 -0800 Subject: [PATCH 16/74] revert change --- python/cudf/cudf/tests/test_dask.py | 70 ++----------------- .../dask_cudf/tests/test_dispatch.py | 53 ++++++++++++++ 2 files changed, 60 insertions(+), 63 deletions(-) diff --git a/python/cudf/cudf/tests/test_dask.py b/python/cudf/cudf/tests/test_dask.py index e7eccb5cd36..3af21b4a7ff 100644 --- a/python/cudf/cudf/tests/test_dask.py +++ b/python/cudf/cudf/tests/test_dask.py @@ -1,19 +1,17 @@ -# Copyright (c) 2019-2024, NVIDIA CORPORATION. - -from datetime import datetime +# Copyright (c) 2019, NVIDIA CORPORATION. import pytest import cudf +is_dataframe_like = pytest.importorskip( + "dask.dataframe.utils" +).is_dataframe_like +is_index_like = pytest.importorskip("dask.dataframe.utils").is_index_like +is_series_like = pytest.importorskip("dask.dataframe.utils").is_series_like -def test_is_dataframe_like(): - is_dataframe_like = pytest.importorskip( - "dask.dataframe.utils" - ).is_dataframe_like - is_index_like = pytest.importorskip("dask.dataframe.utils").is_index_like - is_series_like = pytest.importorskip("dask.dataframe.utils").is_series_like +def test_is_dataframe_like(): df = cudf.DataFrame({"x": [1, 2, 3]}) assert is_dataframe_like(df) assert is_series_like(df.x) @@ -21,57 +19,3 @@ def test_is_dataframe_like(): assert not is_dataframe_like(df.x) assert not is_series_like(df) assert not is_index_like(df) - - -@pytest.mark.parametrize("index", [None, [1, 2] * 5]) -def test_deterministic_tokenize(index): - tokenize = pytest.importorskip("dask.base").tokenize - - # Checks that `dask.base.normalize_token` correctly - # dispatches to the logic defined in `backends.py` - # (making `tokenize()` deterministic). - df = cudf.DataFrame( - {"A": range(10), "B": ["dog", "cat"] * 5, "C": range(10, 0, -1)}, - index=index, - ) - - # Matching data should produce the same token - assert tokenize(df) == tokenize(df) - assert tokenize(df.A) == tokenize(df.A) - assert tokenize(df.index) == tokenize(df.index) - assert tokenize(df) == tokenize(df.copy(deep=True)) - assert tokenize(df.A) == tokenize(df.A.copy(deep=True)) - assert tokenize(df.index) == tokenize(df.index.copy(deep=True)) - - # Modifying a column element should change the token - original_token = tokenize(df) - original_token_a = tokenize(df.A) - df.A.iloc[2] = 10 - assert original_token != tokenize(df) - assert original_token_a != tokenize(df.A) - - # Modifying an index element should change the token - original_token = tokenize(df) - original_token_index = tokenize(df.index) - new_index = df.index.values - new_index[2] = 10 - df.index = new_index - assert original_token != tokenize(df) - assert original_token_index != tokenize(df.index) - - # Check MultiIndex case - df2 = df.set_index(["B", "C"], drop=False) - assert tokenize(df) != tokenize(df2) - assert tokenize(df2) == tokenize(df2) - - -def test_deterministic_tokenize_multiindex(): - tokenize = pytest.importorskip("dask.base").tokenize - - dt = datetime.strptime("1995-03-15", "%Y-%m-%d") - index = cudf.MultiIndex( - levels=[[1, 2], [dt]], - codes=[[0, 1], [0, 0]], - ) - df = cudf.DataFrame(index=index) - assert tokenize(df) == tokenize(df) diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index 34d7e0f1b3c..76703206726 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -1,9 +1,12 @@ # Copyright (c) 2021-2024, NVIDIA CORPORATION. +from datetime import datetime + import numpy as np import pandas as pd import pytest +from dask.base import tokenize from dask.dataframe import assert_eq from dask.dataframe.methods import is_categorical_dtype @@ -41,6 +44,56 @@ def test_pyarrow_conversion_dispatch(preserve_index): assert not isinstance(df2.index, cudf.RangeIndex) +@pytest.mark.parametrize("index", [None, [1, 2] * 5]) +def test_deterministic_tokenize(index): + # Checks that `dask.base.normalize_token` correctly + # dispatches to the logic defined in `backends.py` + # (making `tokenize()` deterministic). + df = cudf.DataFrame( + {"A": range(10), "B": ["dog", "cat"] * 5, "C": range(10, 0, -1)}, + index=index, + ) + + # Matching data should produce the same token + assert tokenize(df) == tokenize(df) + assert tokenize(df.A) == tokenize(df.A) + assert tokenize(df.index) == tokenize(df.index) + assert tokenize(df) == tokenize(df.copy(deep=True)) + assert tokenize(df.A) == tokenize(df.A.copy(deep=True)) + assert tokenize(df.index) == tokenize(df.index.copy(deep=True)) + + # Modifying a column element should change the token + original_token = tokenize(df) + original_token_a = tokenize(df.A) + df.A.iloc[2] = 10 + assert original_token != tokenize(df) + assert original_token_a != tokenize(df.A) + + # Modifying an index element should change the token + original_token = tokenize(df) + original_token_index = tokenize(df.index) + new_index = df.index.values + new_index[2] = 10 + df.index = new_index + assert original_token != tokenize(df) + assert original_token_index != tokenize(df.index) + + # Check MultiIndex case + df2 = df.set_index(["B", "C"], drop=False) + assert tokenize(df) != tokenize(df2) + assert tokenize(df2) == tokenize(df2) + + +def test_deterministic_tokenize_multiindex(): + dt = datetime.strptime("1995-03-15", "%Y-%m-%d") + index = cudf.MultiIndex( + levels=[[1, 2], [dt]], + codes=[[0, 1], [0, 0]], + ) + df = cudf.DataFrame(index=index) + assert tokenize(df) == tokenize(df) + + @pytest.mark.parametrize("preserve_index", [True, False]) def test_pyarrow_schema_dispatch(preserve_index): from dask.dataframe.dispatch import ( From 4df89a081c9fe4964271e0d993a21f056f0884cf Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 08:43:05 -0800 Subject: [PATCH 17/74] add initial testing --- ci/test_python_other.sh | 11 + .../all_cuda-118_arch-x86_64.yaml | 2 + python/dask_cudf/dask_cudf/core.py | 4 +- python/dask_cudf/dask_cudf/expr/__init__.py | 10 +- .../dask_cudf/expr/tests/__init__.py | 10 + .../dask_cudf/expr/tests/test_core.py | 846 ++++++++++++++++++ 6 files changed, 880 insertions(+), 3 deletions(-) create mode 100644 python/dask_cudf/dask_cudf/expr/tests/__init__.py create mode 100644 python/dask_cudf/dask_cudf/expr/tests/test_core.py diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 25c1d681029..b5c46deaab5 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,6 +30,17 @@ pytest \ . popd +# Dask-expr tests should be skipped if dask_expr is not installed +rapids-logger "pytest dask_cudf + dask-expr" +pushd python/dask_cudf/dask_cudf/expr +DASK_DATAFRAME__QUERY_PLANNING=True pytest \ + --cache-clear \ + --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ + --numprocesses=8 \ + --dist=loadscope \ + . +popd + rapids-logger "pytest custreamz" pushd python/custreamz/custreamz pytest \ diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index fa4ef8ddf68..4c4084b9648 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -103,4 +103,6 @@ dependencies: - zlib>=1.2.13 - pip: - git+https://github.com/python-streamz/streamz.git@master + # Dask expressions should eventually be added to rapids-dask-dependency + - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 name: all_cuda-118_arch-x86_64 diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index b051b21790e..05f2f9ebdcb 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -711,7 +711,9 @@ def from_cudf(data, npartitions=None, chunksize=None, sort=True, name=None): rather than pandas objects.\n """ ) - + textwrap.dedent(dd.from_pandas.__doc__) + # TODO: `dd.from_pandas.__doc__` is empty when + # `DASK_DATAFRAME__QUERY_PLANNING=True` + + textwrap.dedent(dd.from_pandas.__doc__ or "") ) diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index c18bd1b47ea..37d2822efaa 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -1,5 +1,11 @@ # Copyright (c) 2024, NVIDIA CORPORATION. # Make sure custom expressions and collections are defined -import dask_cudf.expr._collection -import dask_cudf.expr._expr +try: + import dask_cudf.expr._collection + import dask_cudf.expr._expr + + _expr_support = True +except ImportError: + # Dask Expressions not installed + _expr_support = False diff --git a/python/dask_cudf/dask_cudf/expr/tests/__init__.py b/python/dask_cudf/dask_cudf/expr/tests/__init__.py new file mode 100644 index 00000000000..8b1377ff5ee --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr/tests/__init__.py @@ -0,0 +1,10 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +import pytest + +from dask import config + +from dask_cudf.expr import _expr_support + +if not _expr_support or not config.get("dataframe.query-planning", False): + pytest.skip(allow_module_level=True) diff --git a/python/dask_cudf/dask_cudf/expr/tests/test_core.py b/python/dask_cudf/dask_cudf/expr/tests/test_core.py new file mode 100644 index 00000000000..70a1108564a --- /dev/null +++ b/python/dask_cudf/dask_cudf/expr/tests/test_core.py @@ -0,0 +1,846 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. + +import random + +import cupy as cp +import numpy as np +import pandas as pd +import pytest +from packaging import version + +import dask +from dask import dataframe as dd +from dask.dataframe.core import make_meta as dask_make_meta, meta_nonempty +from dask.utils import M + +import cudf + +from dask_cudf.backends import hash_object_cudf + + +def test_from_dict_backend_dispatch(): + # Test ddf.from_dict cudf-backend dispatch + np.random.seed(0) + data = { + "x": np.random.randint(0, 5, size=10000), + "y": np.random.normal(size=10000), + } + expect = cudf.DataFrame(data) + with dask.config.set({"dataframe.backend": "cudf"}): + ddf = dd.from_dict(data, npartitions=2) + assert isinstance(ddf._meta, cudf.DataFrame) + dd.assert_eq(expect, ddf) + + +def test_to_backend(): + np.random.seed(0) + data = { + "x": np.random.randint(0, 5, size=10000), + "y": np.random.normal(size=10000), + } + with dask.config.set({"dataframe.backend": "pandas"}): + ddf = dd.from_dict(data, npartitions=2) + assert isinstance(ddf._meta, pd.DataFrame) + + gdf = ddf.to_backend("cudf") + assert isinstance(gdf._meta, cudf.DataFrame) + dd.assert_eq(cudf.DataFrame(data), ddf) + + assert isinstance(gdf.to_backend()._meta, pd.DataFrame) + + +def test_to_backend_kwargs(): + data = {"x": [0, 2, np.nan, 3, 4, 5]} + with dask.config.set({"dataframe.backend": "pandas"}): + dser = dd.from_dict(data, npartitions=2)["x"] + assert isinstance(dser._meta, pd.Series) + + # Using `nan_as_null=False` will result in a cudf-backed + # Series with a NaN element (ranther than ) + gser_nan = dser.to_backend("cudf", nan_as_null=False) + assert isinstance(gser_nan._meta, cudf.Series) + assert np.isnan(gser_nan.compute()).sum() == 1 + + # Using `nan_as_null=True` will result in a cudf-backed + # Series with a element (ranther than NaN) + gser_null = dser.to_backend("cudf", nan_as_null=True) + assert isinstance(gser_null._meta, cudf.Series) + assert np.isnan(gser_null.compute()).sum() == 0 + + # Check `nullable` argument for `cudf.Series.to_pandas` + dser_null = gser_null.to_backend("pandas", nullable=False) + assert dser_null.compute().dtype == "float" + dser_null = gser_null.to_backend("pandas", nullable=True) + assert isinstance(dser_null.compute().dtype, pd.Float64Dtype) + + # Check unsupported arguments + with pytest.raises(ValueError, match="pandas-to-cudf"): + dser.to_backend("cudf", bad_arg=True) + + with pytest.raises(ValueError, match="cudf-to-cudf"): + gser_null.to_backend("cudf", bad_arg=True) + + with pytest.raises(ValueError, match="cudf-to-pandas"): + gser_null.to_backend("pandas", bad_arg=True) + + +def test_from_pandas(): + np.random.seed(0) + + df = pd.DataFrame( + { + "x": np.random.randint(0, 5, size=10000), + "y": np.random.normal(size=10000), + } + ) + + gdf = cudf.DataFrame.from_pandas(df) + + # Test simple around to/from cudf + ingested = dd.from_pandas(gdf, npartitions=2) + dd.assert_eq(ingested, df) + + # Test conversion back to pandas + ddf = ingested.to_backend("pandas") + dd.assert_eq(ddf, df) + + +def test_from_pandas_multiindex_raises(): + df = cudf.DataFrame({"x": list("abc"), "y": [1, 2, 3], "z": [1, 2, 3]}) + + with pytest.raises(NotImplementedError): + # Dask does not support MultiIndex yet + dd.from_pandas(df.set_index(["x", "y"]), npartitions=1) + + +def test_from_pandas_with_generic_idx(): + cdf = cudf.DataFrame( + { + "a": list(range(20)), + "b": list(reversed(range(20))), + "c": list(range(20)), + } + ) + + ddf = dd.from_pandas(cdf, npartitions=2) + + assert isinstance(ddf.index.compute(), cudf.RangeIndex) + dd.assert_eq(ddf.loc[1:2, ["a"]], cdf.loc[1:2, ["a"]]) + + +def _fragmented_gdf(df, nsplit): + n = len(df) + + # Split dataframe in *nsplit* + subdivsize = n // nsplit + starts = [i * subdivsize for i in range(nsplit)] + ends = starts[1:] + [None] + frags = [df[s:e] for s, e in zip(starts, ends)] + return frags + + +def test_query(): + np.random.seed(0) + + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=10), "y": np.random.normal(size=10)} + ) + gdf = cudf.DataFrame.from_pandas(df) + expr = "x > 2" + + dd.assert_eq(gdf.query(expr), df.query(expr)) + + queried = dd.from_pandas(gdf, npartitions=2).query(expr) + + got = queried + expect = gdf.query(expr) + + dd.assert_eq(got, expect) + + +def test_query_local_dict(): + np.random.seed(0) + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=10), "y": np.random.normal(size=10)} + ) + gdf = cudf.DataFrame.from_pandas(df) + ddf = dd.from_pandas(gdf, npartitions=2) + + val = 2 + + gdf_queried = gdf.query("x > @val") + ddf_queried = ddf.query("x > @val", local_dict={"val": val}) + + dd.assert_eq(gdf_queried, ddf_queried) + + +def test_head(): + np.random.seed(0) + df = pd.DataFrame( + { + "x": np.random.randint(0, 5, size=100), + "y": np.random.normal(size=100), + } + ) + gdf = cudf.DataFrame.from_pandas(df) + dgf = dd.from_pandas(gdf, npartitions=2) + + dd.assert_eq(dgf.head(), df.head()) + + +@pytest.mark.parametrize("nelem", [10, 200, 1333]) +def test_set_index(nelem): + with dask.config.set(scheduler="single-threaded"): + np.random.seed(0) + # Use unique index range as the sort may not be stable-ordering + x = np.arange(nelem) + np.random.shuffle(x) + df = pd.DataFrame( + {"x": x, "y": np.random.randint(0, nelem, size=nelem)} + ) + ddf = dd.from_pandas(df, npartitions=2) + ddf2 = ddf.to_backend("cudf") + + expect = ddf.set_index("x") + got = ddf2.set_index("x") + + dd.assert_eq(expect, got, check_index=False, check_divisions=False) + + +def assert_frame_equal_by_index_group(expect, got): + assert sorted(expect.columns) == sorted(got.columns) + assert sorted(set(got.index)) == sorted(set(expect.index)) + # Note the set_index sort is not stable, + unique_values = sorted(set(got.index)) + for iv in unique_values: + sr_expect = expect.loc[[iv]] + sr_got = got.loc[[iv]] + + for k in expect.columns: + # Sort each column before we compare them + sorted_expect = sr_expect.sort_values(k)[k] + sorted_got = sr_got.sort_values(k)[k] + np.testing.assert_array_equal(sorted_expect, sorted_got) + + +@pytest.mark.parametrize("nelem", [10, 200, 1333]) +def test_set_index_2(nelem): + with dask.config.set(scheduler="single-threaded"): + np.random.seed(0) + df = pd.DataFrame( + { + "x": 100 + np.random.randint(0, nelem // 2, size=nelem), + "y": np.random.normal(size=nelem), + } + ) + expect = df.set_index("x").sort_index() + + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) + res = dgf.set_index("x") # sort by default + got = res.compute().to_pandas() + + assert_frame_equal_by_index_group(expect, got) + + +def test_set_index_w_series(): + with dask.config.set(scheduler="single-threaded"): + nelem = 20 + np.random.seed(0) + df = pd.DataFrame( + { + "x": 100 + np.random.randint(0, nelem // 2, size=nelem), + "y": np.random.normal(size=nelem), + } + ) + expect = df.set_index(df.x).sort_index() + + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) + res = dgf.set_index(dgf.x) # sort by default + got = res.compute().to_pandas() + + dd.assert_eq(expect, got) + + +def test_set_index_sorted(): + with dask.config.set(scheduler="single-threaded"): + df1 = pd.DataFrame({"val": [4, 3, 2, 1, 0], "id": [0, 1, 3, 5, 7]}) + ddf1 = dd.from_pandas(df1, npartitions=2) + + gdf1 = cudf.from_pandas(df1) + gddf1 = dd.from_pandas(gdf1, npartitions=2) + + expect = ddf1.set_index("id", sorted=True) + got = gddf1.set_index("id", sorted=True) + + dd.assert_eq(expect, got) + + with pytest.raises(ValueError): + # Cannot set `sorted=True` for non-sorted column + gddf1.set_index("val", sorted=True) + + +@pytest.mark.parametrize("nelem", [10, 200, 1333]) +@pytest.mark.parametrize("index", [None, "myindex"]) +def test_rearrange_by_divisions(nelem, index): + with dask.config.set(scheduler="single-threaded"): + np.random.seed(0) + df = pd.DataFrame( + { + "x": np.random.randint(0, 20, size=nelem), + "y": np.random.normal(size=nelem), + "z": np.random.choice(["dog", "cat", "bird"], nelem), + } + ) + df["z"] = df["z"].astype("category") + + ddf1 = dd.from_pandas(df, npartitions=4) + gdf1 = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) + ddf1.index.name = index + gdf1.index.name = index + divisions = (0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20) + + expect = dd.shuffle.rearrange_by_divisions( + ddf1, "x", divisions=divisions, shuffle_method="tasks" + ) + result = dd.shuffle.rearrange_by_divisions( + gdf1, "x", divisions=divisions, shuffle_method="tasks" + ) + dd.assert_eq(expect, result) + + +def test_assign(): + np.random.seed(0) + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} + ) + + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) + pdcol = pd.Series(np.arange(20) + 1000) + newcol = dd.from_pandas(cudf.Series(pdcol), npartitions=dgf.npartitions) + got = dgf.assign(z=newcol) + + dd.assert_eq(got.loc[:, ["x", "y"]], df) + np.testing.assert_array_equal(got["z"].compute().values_host, pdcol) + + +@pytest.mark.parametrize("data_type", ["int8", "int16", "int32", "int64"]) +def test_setitem_scalar_integer(data_type): + np.random.seed(0) + scalar = np.random.randint(0, 100, dtype=data_type) + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} + ) + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) + + df["z"] = scalar + dgf["z"] = scalar + + got = dgf.compute().to_pandas() + np.testing.assert_array_equal(got["z"], df["z"]) + + +@pytest.mark.parametrize("data_type", ["float32", "float64"]) +def test_setitem_scalar_float(data_type): + np.random.seed(0) + scalar = np.random.randn(1).astype(data_type)[0] + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} + ) + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) + + df["z"] = scalar + dgf["z"] = scalar + + got = dgf.compute().to_pandas() + np.testing.assert_array_equal(got["z"], df["z"]) + + +def test_setitem_scalar_datetime(): + np.random.seed(0) + scalar = np.int64(np.random.randint(0, 100)).astype("datetime64[ms]") + df = pd.DataFrame( + {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} + ) + dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) + + df["z"] = scalar + dgf["z"] = scalar + + got = dgf.compute().to_pandas() + np.testing.assert_array_equal(got["z"], df["z"]) + + +@pytest.mark.parametrize("start", [1, 2, 5]) +@pytest.mark.parametrize("stop", [1, 3, 7]) +def test_repartition_simple_divisions(start, stop): + pdf = pd.DataFrame({"x": range(100)}) + + pdf = dd.from_pandas(pdf, npartitions=start) + gdf = pdf.map_partitions(cudf.DataFrame.from_pandas) + + a = pdf.repartition(npartitions=stop) + b = gdf.repartition(npartitions=stop) + assert a.divisions == b.divisions + + dd.assert_eq(a, b) + + +@pytest.mark.parametrize("npartitions", [2, 17, 20]) +def test_repartition_hash_staged(npartitions): + by = ["b"] + datarange = 35 + size = 100 + gdf = cudf.DataFrame( + { + "a": np.arange(size, dtype="int64"), + "b": np.random.randint(datarange, size=size), + } + ) + # WARNING: Specific npartitions-max_branch combination + # was specifically chosen to cover changes in #4676 + npartitions_initial = 17 + ddf = dd.from_pandas(gdf, npartitions=npartitions_initial) + ddf_new = ddf.shuffle( + on=by, ignore_index=True, npartitions=npartitions, max_branch=4 + ) + + # Make sure we are getting a dask_cudf dataframe + assert type(ddf_new) == type(ddf) + + # Check that the length was preserved + assert len(ddf_new) == len(ddf) + + # Check that the partitions have unique keys, + # and that the key values are preserved + expect_unique = gdf[by].drop_duplicates().sort_values(by) + got_unique = cudf.concat( + [ + part[by].compute().drop_duplicates() + for part in ddf_new[by].partitions + ], + ignore_index=True, + ).sort_values(by) + dd.assert_eq(got_unique, expect_unique, check_index=False) + + +@pytest.mark.parametrize("by", [["b"], ["c"], ["d"], ["b", "c"]]) +@pytest.mark.parametrize("npartitions", [3, 4, 5]) +@pytest.mark.parametrize("max_branch", [3, 32]) +def test_repartition_hash(by, npartitions, max_branch): + npartitions_i = 4 + datarange = 26 + size = 100 + gdf = cudf.DataFrame( + { + "a": np.arange(0, stop=size, dtype="int64"), + "b": np.random.randint(datarange, size=size), + "c": np.random.choice(list("abcdefgh"), size=size), + "d": np.random.choice(np.arange(26), size=size), + } + ) + gdf.d = gdf.d.astype("datetime64[ms]") + ddf = dd.from_pandas(gdf, npartitions=npartitions_i) + ddf_new = ddf.shuffle( + on=by, + ignore_index=True, + npartitions=npartitions, + max_branch=max_branch, + ) + + # Check that the length was preserved + assert len(ddf_new) == len(ddf) + + # Check that the partitions have unique keys, + # and that the key values are preserved + expect_unique = gdf[by].drop_duplicates().sort_values(by) + got_unique = cudf.concat( + [ + part[by].compute().drop_duplicates() + for part in ddf_new[by].partitions + ], + ignore_index=True, + ).sort_values(by) + dd.assert_eq(got_unique, expect_unique, check_index=False) + + +def test_repartition_no_extra_row(): + # see https://github.com/rapidsai/cudf/issues/11930 + gdf = cudf.DataFrame({"a": [10, 20, 30], "b": [1, 2, 3]}).set_index("a") + ddf = dd.from_pandas(gdf, npartitions=1) + ddf_new = ddf.repartition([0, 5, 10, 30], force=True) + dd.assert_eq(ddf, ddf_new) + dd.assert_eq(gdf, ddf_new) + + +@pytest.fixture +def pdf(): + return pd.DataFrame( + {"x": [1, 2, 3, 4, 5, 6], "y": [11.0, 12.0, 13.0, 14.0, 15.0, 16.0]} + ) + + +@pytest.fixture +def gdf(pdf): + return cudf.from_pandas(pdf) + + +@pytest.fixture +def ddf(pdf): + return dd.from_pandas(pdf, npartitions=3) + + +@pytest.fixture +def gddf(gdf): + return dd.from_pandas(gdf, npartitions=3) + + +@pytest.mark.parametrize( + "func", + [ + lambda df: df + 1, + lambda df: df.index, + lambda df: df.x.sum(), + lambda df: df.x.astype(float), + lambda df: df.assign(z=df.x.astype("int")), + ], +) +def test_unary_ops(func, gdf, gddf): + p = func(gdf) + g = func(gddf) + + # Fixed in https://github.com/dask/dask/pull/4657 + if isinstance(p, cudf.Index): + if version.parse(dask.__version__) < version.parse("1.1.6"): + pytest.skip( + "dask.dataframe assert_eq index check hardcoded to " + "pandas prior to 1.1.6 release" + ) + + dd.assert_eq(p, g, check_names=False) + + +@pytest.mark.parametrize("series", [True, False]) +def test_concat(gdf, gddf, series): + if series: + gdf = gdf.x + gddf = gddf.x + a = ( + cudf.concat([gdf, gdf + 1, gdf + 2]) + .sort_values() + .reset_index(drop=True) + ) + b = ( + dd.concat([gddf, gddf + 1, gddf + 2], interleave_partitions=True) + .compute() + .sort_values() + .reset_index(drop=True) + ) + else: + a = ( + cudf.concat([gdf, gdf + 1, gdf + 2]) + .sort_values("x") + .reset_index(drop=True) + ) + b = ( + dd.concat([gddf, gddf + 1, gddf + 2], interleave_partitions=True) + .compute() + .sort_values("x") + .reset_index(drop=True) + ) + dd.assert_eq(a, b) + + +def test_boolean_index(gdf, gddf): + gdf2 = gdf[gdf.x > 2] + gddf2 = gddf[gddf.x > 2] + + dd.assert_eq(gdf2, gddf2) + + +def test_drop(gdf, gddf): + gdf2 = gdf.drop(columns="x") + gddf2 = gddf.drop(columns="x").compute() + + dd.assert_eq(gdf2, gddf2) + + +@pytest.mark.parametrize("deep", [True, False]) +@pytest.mark.parametrize("index", [True, False]) +def test_memory_usage(gdf, gddf, index, deep): + dd.assert_eq( + gdf.memory_usage(deep=deep, index=index), + gddf.memory_usage(deep=deep, index=index), + ) + + +@pytest.mark.parametrize("index", [True, False]) +def test_hash_object_dispatch(index): + obj = cudf.DataFrame( + {"x": ["a", "b", "c"], "y": [1, 2, 3], "z": [1, 1, 0]}, index=[2, 4, 6] + ) + + # DataFrame + result = dd.core.hash_object_dispatch(obj, index=index) + expected = hash_object_cudf(obj, index=index) + assert isinstance(result, cudf.Series) + dd.assert_eq(result, expected) + + # Series + result = dd.core.hash_object_dispatch(obj["x"], index=index) + expected = hash_object_cudf(obj["x"], index=index) + assert isinstance(result, cudf.Series) + dd.assert_eq(result, expected) + + # DataFrame with MultiIndex + obj_multi = obj.set_index(["x", "z"], drop=True) + result = dd.core.hash_object_dispatch(obj_multi, index=index) + expected = hash_object_cudf(obj_multi, index=index) + assert isinstance(result, cudf.Series) + dd.assert_eq(result, expected) + + +@pytest.mark.parametrize( + "index", + [ + "int8", + "int32", + "int64", + "float64", + "strings", + "cats", + "time_s", + "time_ms", + "time_ns", + ["int32", "int64"], + ["int8", "float64", "strings"], + ["cats", "int8", "float64"], + ["time_ms", "cats"], + ], +) +def test_make_meta_backends(index): + dtypes = ["int8", "int32", "int64", "float64"] + df = cudf.DataFrame( + {dt: np.arange(start=0, stop=3, dtype=dt) for dt in dtypes} + ) + df["strings"] = ["cat", "dog", "fish"] + df["cats"] = df["strings"].astype("category") + df["time_s"] = np.array( + ["2018-10-07", "2018-10-08", "2018-10-09"], dtype="datetime64[s]" + ) + df["time_ms"] = df["time_s"].astype("datetime64[ms]") + df["time_ns"] = df["time_s"].astype("datetime64[ns]") + df = df.set_index(index) + + # Check "empty" metadata types + chk_meta = dask_make_meta(df) + dd.assert_eq(chk_meta.dtypes, df.dtypes) + + # Check "non-empty" metadata types + chk_meta_nonempty = meta_nonempty(df) + dd.assert_eq(chk_meta.dtypes, chk_meta_nonempty.dtypes) + + # Check dask code path if not MultiIndex + if not isinstance(df.index, cudf.MultiIndex): + ddf = dd.from_pandas(df, npartitions=1) + + # Check "empty" metadata types + dd.assert_eq(ddf._meta.dtypes, df.dtypes) + + # Check "non-empty" metadata types + dd.assert_eq(ddf._meta.dtypes, ddf._meta_nonempty.dtypes) + + +@pytest.mark.parametrize( + "data", + [ + pd.Series([], dtype="float64"), + pd.DataFrame({"abc": [], "xyz": []}), + pd.Series([1, 2, 10, 11]), + pd.DataFrame({"abc": [1, 2, 10, 11], "xyz": [100, 12, 120, 1]}), + ], +) +def test_dataframe_series_replace(data): + pdf = data.copy() + gdf = cudf.from_pandas(pdf) + + ddf = dd.from_pandas(gdf, npartitions=5) + + dd.assert_eq(ddf.replace(1, 2), pdf.replace(1, 2)) + + +def test_dataframe_assign_col(): + df = cudf.DataFrame(list(range(100))) + pdf = pd.DataFrame(list(range(100))) + + ddf = dd.from_pandas(df, npartitions=4) + ddf["fold"] = 0 + ddf["fold"] = ddf["fold"].map_partitions( + lambda cudf_df: cudf.Series(cp.random.randint(0, 4, len(cudf_df))) + ) + + pddf = dd.from_pandas(pdf, npartitions=4) + pddf["fold"] = 0 + pddf["fold"] = pddf["fold"].map_partitions( + lambda p_df: pd.Series(np.random.randint(0, 4, len(p_df))) + ) + + dd.assert_eq(ddf[0], pddf[0]) + dd.assert_eq(len(ddf["fold"]), len(pddf["fold"])) + + +def test_dataframe_set_index(): + random.seed(0) + df = cudf.datasets.randomdata(26, dtypes={"a": float, "b": int}) + df["str"] = list("abcdefghijklmnopqrstuvwxyz") + pdf = df.to_pandas() + + with dask.config.set({"dataframe.convert-string": False}): + ddf = dd.from_pandas(df, npartitions=4) + ddf = ddf.set_index("str") + + pddf = dd.from_pandas(pdf, npartitions=4) + pddf = pddf.set_index("str") + + from cudf.testing._utils import assert_eq + + assert_eq(ddf.compute(), pddf.compute()) + + +@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") +def test_series_describe(): + random.seed(0) + sr = cudf.datasets.randomdata(20)["x"] + psr = sr.to_pandas() + + dsr = dd.from_pandas(sr, npartitions=4) + pdsr = dd.from_pandas(psr, npartitions=4) + + dd.assert_eq( + dsr.describe(), + pdsr.describe(), + rtol=1e-3, + ) + + +@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") +def test_dataframe_describe(): + random.seed(0) + df = cudf.datasets.randomdata(20) + pdf = df.to_pandas() + + ddf = dd.from_pandas(df, npartitions=4) + pddf = dd.from_pandas(pdf, npartitions=4) + + dd.assert_eq( + ddf.describe(), pddf.describe(), check_exact=False, atol=0.0001 + ) + + +@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") +def test_zero_std_describe(): + num = 84886781 + df = cudf.DataFrame( + { + "x": np.full((20,), num, dtype=np.float64), + "y": np.full((20,), num, dtype=np.float64), + } + ) + pdf = df.to_pandas() + ddf = dd.from_pandas(df, npartitions=4) + pddf = dd.from_pandas(pdf, npartitions=4) + + dd.assert_eq(ddf.describe(), pddf.describe(), rtol=1e-3) + + +def test_large_numbers_var(): + num = 8488678001 + df = cudf.DataFrame( + { + "x": np.arange(num, num + 1000, dtype=np.float64), + "y": np.arange(num, num + 1000, dtype=np.float64), + } + ) + pdf = df.to_pandas() + ddf = dd.from_pandas(df, npartitions=4) + pddf = dd.from_pandas(pdf, npartitions=4) + + dd.assert_eq(ddf.var(), pddf.var(), rtol=1e-3) + + +def test_index_map_partitions(): + # https://github.com/rapidsai/cudf/issues/6738 + + ddf = dd.from_pandas(pd.DataFrame({"a": range(10)}), npartitions=2) + mins_pd = ddf.index.map_partitions(M.min, meta=ddf.index).compute() + + gddf = dd.from_pandas(cudf.DataFrame({"a": range(10)}), npartitions=2) + mins_gd = gddf.index.map_partitions(M.min, meta=gddf.index).compute() + + dd.assert_eq(mins_pd, mins_gd) + + +def test_merging_categorical_columns(): + df_1 = cudf.DataFrame( + {"id_1": [0, 1, 2, 3], "cat_col": ["a", "b", "f", "f"]} + ) + ddf_1 = dd.from_pandas(df_1, npartitions=2) + ddf_1 = dd.categorical.categorize(ddf_1, columns=["cat_col"]) + + df_2 = cudf.DataFrame( + {"id_2": [111, 112, 113], "cat_col": ["g", "h", "f"]} + ) + ddf_2 = dd.from_pandas(df_2, npartitions=2) + ddf_2 = dd.categorical.categorize(ddf_2, columns=["cat_col"]) + + expected = cudf.DataFrame( + { + "id_1": [2, 3], + "cat_col": cudf.Series( + ["f", "f"], + dtype=cudf.CategoricalDtype( + categories=["a", "b", "f", "g", "h"], ordered=False + ), + ), + "id_2": [113, 113], + } + ) + with pytest.warns(UserWarning, match="mismatch"): + dd.assert_eq(ddf_1.merge(ddf_2), expected) + + +def test_correct_meta(): + try: + from dask.dataframe.dispatch import make_meta_obj # noqa: F401 + except ImportError: + pytest.skip("need make_meta_obj to be preset") + + # Need these local imports in this specific order. + # For context: https://github.com/rapidsai/cudf/issues/7946 + import pandas as pd + + from dask import dataframe as dd + + import dask_cudf # noqa: F401 + + df = pd.DataFrame({"a": [3, 4], "b": [1, 2]}) + ddf = dd.from_pandas(df, npartitions=1) + emb = ddf["a"].apply(pd.Series, meta={"c0": "int64", "c1": "int64"}) + + assert isinstance(emb, dd.DataFrame) + assert isinstance(emb._meta, pd.DataFrame) + + +def test_categorical_dtype_round_trip(): + s = cudf.Series(4 * ["foo"], dtype="category") + assert s.dtype.ordered is False + + ds = dd.from_pandas(s, npartitions=2) + pds = dd.from_pandas(s.to_pandas(), npartitions=2) + dd.assert_eq(ds, pds) + assert ds.dtype.ordered is False + + # Below validations are required, see: + # https://github.com/rapidsai/cudf/issues/11487#issuecomment-1208912383 + actual = ds.compute() + expected = pds.compute() + assert actual.dtype.ordered == expected.dtype.ordered From 08e57611b3116335d9a15ecb53cf6cb3a2a7ae0d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 08:47:38 -0800 Subject: [PATCH 18/74] remove comment --- conda/environments/all_cuda-118_arch-x86_64.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 4c4084b9648..7dfb8668632 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -103,6 +103,5 @@ dependencies: - zlib>=1.2.13 - pip: - git+https://github.com/python-streamz/streamz.git@master - # Dask expressions should eventually be added to rapids-dask-dependency - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 name: all_cuda-118_arch-x86_64 From c3f61e8a339067bf11bfc6d7d8e96c1d2eb5ce15 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 09:42:53 -0800 Subject: [PATCH 19/74] try to fix pre-commit failures --- conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +- conda/environments/all_cuda-120_arch-x86_64.yaml | 1 + dependencies.yaml | 4 ++++ python/dask_cudf/pyproject.toml | 2 ++ 4 files changed, 8 insertions(+), 1 deletion(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 7dfb8668632..5d7c54b38bd 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -102,6 +102,6 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: - - git+https://github.com/python-streamz/streamz.git@master - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-118_arch-x86_64 diff --git a/conda/environments/all_cuda-120_arch-x86_64.yaml b/conda/environments/all_cuda-120_arch-x86_64.yaml index a8be9d65c43..74d0516ba8d 100644 --- a/conda/environments/all_cuda-120_arch-x86_64.yaml +++ b/conda/environments/all_cuda-120_arch-x86_64.yaml @@ -100,5 +100,6 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: + - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-120_arch-x86_64 diff --git a/dependencies.yaml b/dependencies.yaml index 90b0527479a..658074ebe0e 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -661,6 +661,10 @@ dependencies: packages: - dask-cuda==24.4.* - *numba + - pip + - pip: + # This should eventually move to rapids-dask-dependency + - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index bc481f1bc6d..7f3cee177af 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -45,9 +45,11 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" test = [ "dask-cuda==24.4.*", "numba>=0.57", + "pip", "pytest-cov", "pytest-xdist", "pytest<8", + {pip = ["git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20"]}, ] # This list was generated by `rapids-dependency-file-generator`. To make changes, edit ../../dependencies.yaml and run `rapids-dependency-file-generator`. [project.urls] From 58a3350eeac4ddb88e6afbc0b1e14a44ea6e0d6e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 09:47:29 -0800 Subject: [PATCH 20/74] try to fix pre-commit failures again --- ci/test_python_other.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index b5c46deaab5..57b22f0e369 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -1,5 +1,5 @@ #!/bin/bash -# Copyright (c) 2022-2023, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. # Common setup steps shared by Python test jobs source "$(dirname "$0")/test_python_common.sh" From b385950f1eca8f84e328483bda2ca8418a4e5557 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 10:53:33 -0800 Subject: [PATCH 21/74] Use advice from Charles --- dependencies.yaml | 7 ++++++- python/dask_cudf/pyproject.toml | 3 +-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/dependencies.yaml b/dependencies.yaml index 658074ebe0e..b1aebec6956 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -661,10 +661,15 @@ dependencies: packages: - dask-cuda==24.4.* - *numba + - output_types: conda + packages: - pip - pip: # This should eventually move to rapids-dask-dependency - - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + - output_types: [requirements, pyproject] + packages: + - *dask_expr_tip depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 7f3cee177af..f4ef876f076 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,12 +44,11 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", + "git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20", "numba>=0.57", - "pip", "pytest-cov", "pytest-xdist", "pytest<8", - {pip = ["git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20"]}, ] # This list was generated by `rapids-dependency-file-generator`. To make changes, edit ../../dependencies.yaml and run `rapids-dependency-file-generator`. [project.urls] From 880e9991124c99f90477da1a0fee40df10201533 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 12:03:38 -0800 Subject: [PATCH 22/74] move tests and hold off on env changes for now --- ci/test_python_other.sh | 20 +- .../all_cuda-118_arch-x86_64.yaml | 1 - .../all_cuda-120_arch-x86_64.yaml | 1 - dependencies.yaml | 18 +- python/dask_cudf/dask_cudf/backends.py | 1 - .../dask_cudf/expr/tests/__init__.py | 10 - .../dask_cudf/expr/tests/test_core.py | 846 ------------------ .../dask_cudf/tests/test_accessor.py | 5 + .../dask_cudf/tests/test_applymap.py | 5 +- .../dask_cudf/dask_cudf/tests/test_binops.py | 5 +- python/dask_cudf/dask_cudf/tests/test_core.py | 139 ++- .../dask_cudf/tests/test_delayed_io.py | 4 + .../dask_cudf/tests/test_dispatch.py | 5 + .../dask_cudf/tests/test_distributed.py | 5 + .../dask_cudf/dask_cudf/tests/test_groupby.py | 4 + python/dask_cudf/dask_cudf/tests/test_join.py | 5 + .../dask_cudf/dask_cudf/tests/test_onehot.py | 4 + .../dask_cudf/tests/test_reductions.py | 4 + python/dask_cudf/dask_cudf/tests/test_sort.py | 4 + .../dask_cudf/dask_cudf/tests/test_struct.py | 4 + python/dask_cudf/dask_cudf/tests/utils.py | 28 + python/dask_cudf/pyproject.toml | 1 - 22 files changed, 163 insertions(+), 956 deletions(-) delete mode 100644 python/dask_cudf/dask_cudf/expr/tests/__init__.py delete mode 100644 python/dask_cudf/dask_cudf/expr/tests/test_core.py diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 57b22f0e369..d926fc95a52 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,16 +30,16 @@ pytest \ . popd -# Dask-expr tests should be skipped if dask_expr is not installed -rapids-logger "pytest dask_cudf + dask-expr" -pushd python/dask_cudf/dask_cudf/expr -DASK_DATAFRAME__QUERY_PLANNING=True pytest \ - --cache-clear \ - --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ - --numprocesses=8 \ - --dist=loadscope \ - . -popd +# # Dask-expr tests should be skipped if dask_expr is not installed +# rapids-logger "pytest dask_cudf + dask-expr" +# pushd python/dask_cudf/dask_cudf/expr +# DASK_DATAFRAME__QUERY_PLANNING=True pytest \ +# --cache-clear \ +# --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ +# --numprocesses=8 \ +# --dist=loadscope \ +# . +# popd rapids-logger "pytest custreamz" pushd python/custreamz/custreamz diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 5d7c54b38bd..fa4ef8ddf68 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -102,6 +102,5 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: - - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-118_arch-x86_64 diff --git a/conda/environments/all_cuda-120_arch-x86_64.yaml b/conda/environments/all_cuda-120_arch-x86_64.yaml index 74d0516ba8d..a8be9d65c43 100644 --- a/conda/environments/all_cuda-120_arch-x86_64.yaml +++ b/conda/environments/all_cuda-120_arch-x86_64.yaml @@ -100,6 +100,5 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: - - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-120_arch-x86_64 diff --git a/dependencies.yaml b/dependencies.yaml index b1aebec6956..07b08fae9cf 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -661,15 +661,15 @@ dependencies: packages: - dask-cuda==24.4.* - *numba - - output_types: conda - packages: - - pip - - pip: - # This should eventually move to rapids-dask-dependency - - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - - output_types: [requirements, pyproject] - packages: - - *dask_expr_tip + # - output_types: conda + # packages: + # - pip + # - pip: + # # This should eventually move to rapids-dask-dependency + # - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + # - output_types: [requirements, pyproject] + # packages: + # - *dask_expr_tip depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 0fa68b0bd0e..36df95be4b6 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -691,5 +691,4 @@ def from_dict( try: import dask_cudf.expr # noqa: F401 except ImportError: - # Compatible dask_expr version not installed pass diff --git a/python/dask_cudf/dask_cudf/expr/tests/__init__.py b/python/dask_cudf/dask_cudf/expr/tests/__init__.py deleted file mode 100644 index 8b1377ff5ee..00000000000 --- a/python/dask_cudf/dask_cudf/expr/tests/__init__.py +++ /dev/null @@ -1,10 +0,0 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. - -import pytest - -from dask import config - -from dask_cudf.expr import _expr_support - -if not _expr_support or not config.get("dataframe.query-planning", False): - pytest.skip(allow_module_level=True) diff --git a/python/dask_cudf/dask_cudf/expr/tests/test_core.py b/python/dask_cudf/dask_cudf/expr/tests/test_core.py deleted file mode 100644 index 70a1108564a..00000000000 --- a/python/dask_cudf/dask_cudf/expr/tests/test_core.py +++ /dev/null @@ -1,846 +0,0 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. - -import random - -import cupy as cp -import numpy as np -import pandas as pd -import pytest -from packaging import version - -import dask -from dask import dataframe as dd -from dask.dataframe.core import make_meta as dask_make_meta, meta_nonempty -from dask.utils import M - -import cudf - -from dask_cudf.backends import hash_object_cudf - - -def test_from_dict_backend_dispatch(): - # Test ddf.from_dict cudf-backend dispatch - np.random.seed(0) - data = { - "x": np.random.randint(0, 5, size=10000), - "y": np.random.normal(size=10000), - } - expect = cudf.DataFrame(data) - with dask.config.set({"dataframe.backend": "cudf"}): - ddf = dd.from_dict(data, npartitions=2) - assert isinstance(ddf._meta, cudf.DataFrame) - dd.assert_eq(expect, ddf) - - -def test_to_backend(): - np.random.seed(0) - data = { - "x": np.random.randint(0, 5, size=10000), - "y": np.random.normal(size=10000), - } - with dask.config.set({"dataframe.backend": "pandas"}): - ddf = dd.from_dict(data, npartitions=2) - assert isinstance(ddf._meta, pd.DataFrame) - - gdf = ddf.to_backend("cudf") - assert isinstance(gdf._meta, cudf.DataFrame) - dd.assert_eq(cudf.DataFrame(data), ddf) - - assert isinstance(gdf.to_backend()._meta, pd.DataFrame) - - -def test_to_backend_kwargs(): - data = {"x": [0, 2, np.nan, 3, 4, 5]} - with dask.config.set({"dataframe.backend": "pandas"}): - dser = dd.from_dict(data, npartitions=2)["x"] - assert isinstance(dser._meta, pd.Series) - - # Using `nan_as_null=False` will result in a cudf-backed - # Series with a NaN element (ranther than ) - gser_nan = dser.to_backend("cudf", nan_as_null=False) - assert isinstance(gser_nan._meta, cudf.Series) - assert np.isnan(gser_nan.compute()).sum() == 1 - - # Using `nan_as_null=True` will result in a cudf-backed - # Series with a element (ranther than NaN) - gser_null = dser.to_backend("cudf", nan_as_null=True) - assert isinstance(gser_null._meta, cudf.Series) - assert np.isnan(gser_null.compute()).sum() == 0 - - # Check `nullable` argument for `cudf.Series.to_pandas` - dser_null = gser_null.to_backend("pandas", nullable=False) - assert dser_null.compute().dtype == "float" - dser_null = gser_null.to_backend("pandas", nullable=True) - assert isinstance(dser_null.compute().dtype, pd.Float64Dtype) - - # Check unsupported arguments - with pytest.raises(ValueError, match="pandas-to-cudf"): - dser.to_backend("cudf", bad_arg=True) - - with pytest.raises(ValueError, match="cudf-to-cudf"): - gser_null.to_backend("cudf", bad_arg=True) - - with pytest.raises(ValueError, match="cudf-to-pandas"): - gser_null.to_backend("pandas", bad_arg=True) - - -def test_from_pandas(): - np.random.seed(0) - - df = pd.DataFrame( - { - "x": np.random.randint(0, 5, size=10000), - "y": np.random.normal(size=10000), - } - ) - - gdf = cudf.DataFrame.from_pandas(df) - - # Test simple around to/from cudf - ingested = dd.from_pandas(gdf, npartitions=2) - dd.assert_eq(ingested, df) - - # Test conversion back to pandas - ddf = ingested.to_backend("pandas") - dd.assert_eq(ddf, df) - - -def test_from_pandas_multiindex_raises(): - df = cudf.DataFrame({"x": list("abc"), "y": [1, 2, 3], "z": [1, 2, 3]}) - - with pytest.raises(NotImplementedError): - # Dask does not support MultiIndex yet - dd.from_pandas(df.set_index(["x", "y"]), npartitions=1) - - -def test_from_pandas_with_generic_idx(): - cdf = cudf.DataFrame( - { - "a": list(range(20)), - "b": list(reversed(range(20))), - "c": list(range(20)), - } - ) - - ddf = dd.from_pandas(cdf, npartitions=2) - - assert isinstance(ddf.index.compute(), cudf.RangeIndex) - dd.assert_eq(ddf.loc[1:2, ["a"]], cdf.loc[1:2, ["a"]]) - - -def _fragmented_gdf(df, nsplit): - n = len(df) - - # Split dataframe in *nsplit* - subdivsize = n // nsplit - starts = [i * subdivsize for i in range(nsplit)] - ends = starts[1:] + [None] - frags = [df[s:e] for s, e in zip(starts, ends)] - return frags - - -def test_query(): - np.random.seed(0) - - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=10), "y": np.random.normal(size=10)} - ) - gdf = cudf.DataFrame.from_pandas(df) - expr = "x > 2" - - dd.assert_eq(gdf.query(expr), df.query(expr)) - - queried = dd.from_pandas(gdf, npartitions=2).query(expr) - - got = queried - expect = gdf.query(expr) - - dd.assert_eq(got, expect) - - -def test_query_local_dict(): - np.random.seed(0) - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=10), "y": np.random.normal(size=10)} - ) - gdf = cudf.DataFrame.from_pandas(df) - ddf = dd.from_pandas(gdf, npartitions=2) - - val = 2 - - gdf_queried = gdf.query("x > @val") - ddf_queried = ddf.query("x > @val", local_dict={"val": val}) - - dd.assert_eq(gdf_queried, ddf_queried) - - -def test_head(): - np.random.seed(0) - df = pd.DataFrame( - { - "x": np.random.randint(0, 5, size=100), - "y": np.random.normal(size=100), - } - ) - gdf = cudf.DataFrame.from_pandas(df) - dgf = dd.from_pandas(gdf, npartitions=2) - - dd.assert_eq(dgf.head(), df.head()) - - -@pytest.mark.parametrize("nelem", [10, 200, 1333]) -def test_set_index(nelem): - with dask.config.set(scheduler="single-threaded"): - np.random.seed(0) - # Use unique index range as the sort may not be stable-ordering - x = np.arange(nelem) - np.random.shuffle(x) - df = pd.DataFrame( - {"x": x, "y": np.random.randint(0, nelem, size=nelem)} - ) - ddf = dd.from_pandas(df, npartitions=2) - ddf2 = ddf.to_backend("cudf") - - expect = ddf.set_index("x") - got = ddf2.set_index("x") - - dd.assert_eq(expect, got, check_index=False, check_divisions=False) - - -def assert_frame_equal_by_index_group(expect, got): - assert sorted(expect.columns) == sorted(got.columns) - assert sorted(set(got.index)) == sorted(set(expect.index)) - # Note the set_index sort is not stable, - unique_values = sorted(set(got.index)) - for iv in unique_values: - sr_expect = expect.loc[[iv]] - sr_got = got.loc[[iv]] - - for k in expect.columns: - # Sort each column before we compare them - sorted_expect = sr_expect.sort_values(k)[k] - sorted_got = sr_got.sort_values(k)[k] - np.testing.assert_array_equal(sorted_expect, sorted_got) - - -@pytest.mark.parametrize("nelem", [10, 200, 1333]) -def test_set_index_2(nelem): - with dask.config.set(scheduler="single-threaded"): - np.random.seed(0) - df = pd.DataFrame( - { - "x": 100 + np.random.randint(0, nelem // 2, size=nelem), - "y": np.random.normal(size=nelem), - } - ) - expect = df.set_index("x").sort_index() - - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) - res = dgf.set_index("x") # sort by default - got = res.compute().to_pandas() - - assert_frame_equal_by_index_group(expect, got) - - -def test_set_index_w_series(): - with dask.config.set(scheduler="single-threaded"): - nelem = 20 - np.random.seed(0) - df = pd.DataFrame( - { - "x": 100 + np.random.randint(0, nelem // 2, size=nelem), - "y": np.random.normal(size=nelem), - } - ) - expect = df.set_index(df.x).sort_index() - - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) - res = dgf.set_index(dgf.x) # sort by default - got = res.compute().to_pandas() - - dd.assert_eq(expect, got) - - -def test_set_index_sorted(): - with dask.config.set(scheduler="single-threaded"): - df1 = pd.DataFrame({"val": [4, 3, 2, 1, 0], "id": [0, 1, 3, 5, 7]}) - ddf1 = dd.from_pandas(df1, npartitions=2) - - gdf1 = cudf.from_pandas(df1) - gddf1 = dd.from_pandas(gdf1, npartitions=2) - - expect = ddf1.set_index("id", sorted=True) - got = gddf1.set_index("id", sorted=True) - - dd.assert_eq(expect, got) - - with pytest.raises(ValueError): - # Cannot set `sorted=True` for non-sorted column - gddf1.set_index("val", sorted=True) - - -@pytest.mark.parametrize("nelem", [10, 200, 1333]) -@pytest.mark.parametrize("index", [None, "myindex"]) -def test_rearrange_by_divisions(nelem, index): - with dask.config.set(scheduler="single-threaded"): - np.random.seed(0) - df = pd.DataFrame( - { - "x": np.random.randint(0, 20, size=nelem), - "y": np.random.normal(size=nelem), - "z": np.random.choice(["dog", "cat", "bird"], nelem), - } - ) - df["z"] = df["z"].astype("category") - - ddf1 = dd.from_pandas(df, npartitions=4) - gdf1 = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) - ddf1.index.name = index - gdf1.index.name = index - divisions = (0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20) - - expect = dd.shuffle.rearrange_by_divisions( - ddf1, "x", divisions=divisions, shuffle_method="tasks" - ) - result = dd.shuffle.rearrange_by_divisions( - gdf1, "x", divisions=divisions, shuffle_method="tasks" - ) - dd.assert_eq(expect, result) - - -def test_assign(): - np.random.seed(0) - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} - ) - - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) - pdcol = pd.Series(np.arange(20) + 1000) - newcol = dd.from_pandas(cudf.Series(pdcol), npartitions=dgf.npartitions) - got = dgf.assign(z=newcol) - - dd.assert_eq(got.loc[:, ["x", "y"]], df) - np.testing.assert_array_equal(got["z"].compute().values_host, pdcol) - - -@pytest.mark.parametrize("data_type", ["int8", "int16", "int32", "int64"]) -def test_setitem_scalar_integer(data_type): - np.random.seed(0) - scalar = np.random.randint(0, 100, dtype=data_type) - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} - ) - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) - - df["z"] = scalar - dgf["z"] = scalar - - got = dgf.compute().to_pandas() - np.testing.assert_array_equal(got["z"], df["z"]) - - -@pytest.mark.parametrize("data_type", ["float32", "float64"]) -def test_setitem_scalar_float(data_type): - np.random.seed(0) - scalar = np.random.randn(1).astype(data_type)[0] - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} - ) - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) - - df["z"] = scalar - dgf["z"] = scalar - - got = dgf.compute().to_pandas() - np.testing.assert_array_equal(got["z"], df["z"]) - - -def test_setitem_scalar_datetime(): - np.random.seed(0) - scalar = np.int64(np.random.randint(0, 100)).astype("datetime64[ms]") - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} - ) - dgf = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=2) - - df["z"] = scalar - dgf["z"] = scalar - - got = dgf.compute().to_pandas() - np.testing.assert_array_equal(got["z"], df["z"]) - - -@pytest.mark.parametrize("start", [1, 2, 5]) -@pytest.mark.parametrize("stop", [1, 3, 7]) -def test_repartition_simple_divisions(start, stop): - pdf = pd.DataFrame({"x": range(100)}) - - pdf = dd.from_pandas(pdf, npartitions=start) - gdf = pdf.map_partitions(cudf.DataFrame.from_pandas) - - a = pdf.repartition(npartitions=stop) - b = gdf.repartition(npartitions=stop) - assert a.divisions == b.divisions - - dd.assert_eq(a, b) - - -@pytest.mark.parametrize("npartitions", [2, 17, 20]) -def test_repartition_hash_staged(npartitions): - by = ["b"] - datarange = 35 - size = 100 - gdf = cudf.DataFrame( - { - "a": np.arange(size, dtype="int64"), - "b": np.random.randint(datarange, size=size), - } - ) - # WARNING: Specific npartitions-max_branch combination - # was specifically chosen to cover changes in #4676 - npartitions_initial = 17 - ddf = dd.from_pandas(gdf, npartitions=npartitions_initial) - ddf_new = ddf.shuffle( - on=by, ignore_index=True, npartitions=npartitions, max_branch=4 - ) - - # Make sure we are getting a dask_cudf dataframe - assert type(ddf_new) == type(ddf) - - # Check that the length was preserved - assert len(ddf_new) == len(ddf) - - # Check that the partitions have unique keys, - # and that the key values are preserved - expect_unique = gdf[by].drop_duplicates().sort_values(by) - got_unique = cudf.concat( - [ - part[by].compute().drop_duplicates() - for part in ddf_new[by].partitions - ], - ignore_index=True, - ).sort_values(by) - dd.assert_eq(got_unique, expect_unique, check_index=False) - - -@pytest.mark.parametrize("by", [["b"], ["c"], ["d"], ["b", "c"]]) -@pytest.mark.parametrize("npartitions", [3, 4, 5]) -@pytest.mark.parametrize("max_branch", [3, 32]) -def test_repartition_hash(by, npartitions, max_branch): - npartitions_i = 4 - datarange = 26 - size = 100 - gdf = cudf.DataFrame( - { - "a": np.arange(0, stop=size, dtype="int64"), - "b": np.random.randint(datarange, size=size), - "c": np.random.choice(list("abcdefgh"), size=size), - "d": np.random.choice(np.arange(26), size=size), - } - ) - gdf.d = gdf.d.astype("datetime64[ms]") - ddf = dd.from_pandas(gdf, npartitions=npartitions_i) - ddf_new = ddf.shuffle( - on=by, - ignore_index=True, - npartitions=npartitions, - max_branch=max_branch, - ) - - # Check that the length was preserved - assert len(ddf_new) == len(ddf) - - # Check that the partitions have unique keys, - # and that the key values are preserved - expect_unique = gdf[by].drop_duplicates().sort_values(by) - got_unique = cudf.concat( - [ - part[by].compute().drop_duplicates() - for part in ddf_new[by].partitions - ], - ignore_index=True, - ).sort_values(by) - dd.assert_eq(got_unique, expect_unique, check_index=False) - - -def test_repartition_no_extra_row(): - # see https://github.com/rapidsai/cudf/issues/11930 - gdf = cudf.DataFrame({"a": [10, 20, 30], "b": [1, 2, 3]}).set_index("a") - ddf = dd.from_pandas(gdf, npartitions=1) - ddf_new = ddf.repartition([0, 5, 10, 30], force=True) - dd.assert_eq(ddf, ddf_new) - dd.assert_eq(gdf, ddf_new) - - -@pytest.fixture -def pdf(): - return pd.DataFrame( - {"x": [1, 2, 3, 4, 5, 6], "y": [11.0, 12.0, 13.0, 14.0, 15.0, 16.0]} - ) - - -@pytest.fixture -def gdf(pdf): - return cudf.from_pandas(pdf) - - -@pytest.fixture -def ddf(pdf): - return dd.from_pandas(pdf, npartitions=3) - - -@pytest.fixture -def gddf(gdf): - return dd.from_pandas(gdf, npartitions=3) - - -@pytest.mark.parametrize( - "func", - [ - lambda df: df + 1, - lambda df: df.index, - lambda df: df.x.sum(), - lambda df: df.x.astype(float), - lambda df: df.assign(z=df.x.astype("int")), - ], -) -def test_unary_ops(func, gdf, gddf): - p = func(gdf) - g = func(gddf) - - # Fixed in https://github.com/dask/dask/pull/4657 - if isinstance(p, cudf.Index): - if version.parse(dask.__version__) < version.parse("1.1.6"): - pytest.skip( - "dask.dataframe assert_eq index check hardcoded to " - "pandas prior to 1.1.6 release" - ) - - dd.assert_eq(p, g, check_names=False) - - -@pytest.mark.parametrize("series", [True, False]) -def test_concat(gdf, gddf, series): - if series: - gdf = gdf.x - gddf = gddf.x - a = ( - cudf.concat([gdf, gdf + 1, gdf + 2]) - .sort_values() - .reset_index(drop=True) - ) - b = ( - dd.concat([gddf, gddf + 1, gddf + 2], interleave_partitions=True) - .compute() - .sort_values() - .reset_index(drop=True) - ) - else: - a = ( - cudf.concat([gdf, gdf + 1, gdf + 2]) - .sort_values("x") - .reset_index(drop=True) - ) - b = ( - dd.concat([gddf, gddf + 1, gddf + 2], interleave_partitions=True) - .compute() - .sort_values("x") - .reset_index(drop=True) - ) - dd.assert_eq(a, b) - - -def test_boolean_index(gdf, gddf): - gdf2 = gdf[gdf.x > 2] - gddf2 = gddf[gddf.x > 2] - - dd.assert_eq(gdf2, gddf2) - - -def test_drop(gdf, gddf): - gdf2 = gdf.drop(columns="x") - gddf2 = gddf.drop(columns="x").compute() - - dd.assert_eq(gdf2, gddf2) - - -@pytest.mark.parametrize("deep", [True, False]) -@pytest.mark.parametrize("index", [True, False]) -def test_memory_usage(gdf, gddf, index, deep): - dd.assert_eq( - gdf.memory_usage(deep=deep, index=index), - gddf.memory_usage(deep=deep, index=index), - ) - - -@pytest.mark.parametrize("index", [True, False]) -def test_hash_object_dispatch(index): - obj = cudf.DataFrame( - {"x": ["a", "b", "c"], "y": [1, 2, 3], "z": [1, 1, 0]}, index=[2, 4, 6] - ) - - # DataFrame - result = dd.core.hash_object_dispatch(obj, index=index) - expected = hash_object_cudf(obj, index=index) - assert isinstance(result, cudf.Series) - dd.assert_eq(result, expected) - - # Series - result = dd.core.hash_object_dispatch(obj["x"], index=index) - expected = hash_object_cudf(obj["x"], index=index) - assert isinstance(result, cudf.Series) - dd.assert_eq(result, expected) - - # DataFrame with MultiIndex - obj_multi = obj.set_index(["x", "z"], drop=True) - result = dd.core.hash_object_dispatch(obj_multi, index=index) - expected = hash_object_cudf(obj_multi, index=index) - assert isinstance(result, cudf.Series) - dd.assert_eq(result, expected) - - -@pytest.mark.parametrize( - "index", - [ - "int8", - "int32", - "int64", - "float64", - "strings", - "cats", - "time_s", - "time_ms", - "time_ns", - ["int32", "int64"], - ["int8", "float64", "strings"], - ["cats", "int8", "float64"], - ["time_ms", "cats"], - ], -) -def test_make_meta_backends(index): - dtypes = ["int8", "int32", "int64", "float64"] - df = cudf.DataFrame( - {dt: np.arange(start=0, stop=3, dtype=dt) for dt in dtypes} - ) - df["strings"] = ["cat", "dog", "fish"] - df["cats"] = df["strings"].astype("category") - df["time_s"] = np.array( - ["2018-10-07", "2018-10-08", "2018-10-09"], dtype="datetime64[s]" - ) - df["time_ms"] = df["time_s"].astype("datetime64[ms]") - df["time_ns"] = df["time_s"].astype("datetime64[ns]") - df = df.set_index(index) - - # Check "empty" metadata types - chk_meta = dask_make_meta(df) - dd.assert_eq(chk_meta.dtypes, df.dtypes) - - # Check "non-empty" metadata types - chk_meta_nonempty = meta_nonempty(df) - dd.assert_eq(chk_meta.dtypes, chk_meta_nonempty.dtypes) - - # Check dask code path if not MultiIndex - if not isinstance(df.index, cudf.MultiIndex): - ddf = dd.from_pandas(df, npartitions=1) - - # Check "empty" metadata types - dd.assert_eq(ddf._meta.dtypes, df.dtypes) - - # Check "non-empty" metadata types - dd.assert_eq(ddf._meta.dtypes, ddf._meta_nonempty.dtypes) - - -@pytest.mark.parametrize( - "data", - [ - pd.Series([], dtype="float64"), - pd.DataFrame({"abc": [], "xyz": []}), - pd.Series([1, 2, 10, 11]), - pd.DataFrame({"abc": [1, 2, 10, 11], "xyz": [100, 12, 120, 1]}), - ], -) -def test_dataframe_series_replace(data): - pdf = data.copy() - gdf = cudf.from_pandas(pdf) - - ddf = dd.from_pandas(gdf, npartitions=5) - - dd.assert_eq(ddf.replace(1, 2), pdf.replace(1, 2)) - - -def test_dataframe_assign_col(): - df = cudf.DataFrame(list(range(100))) - pdf = pd.DataFrame(list(range(100))) - - ddf = dd.from_pandas(df, npartitions=4) - ddf["fold"] = 0 - ddf["fold"] = ddf["fold"].map_partitions( - lambda cudf_df: cudf.Series(cp.random.randint(0, 4, len(cudf_df))) - ) - - pddf = dd.from_pandas(pdf, npartitions=4) - pddf["fold"] = 0 - pddf["fold"] = pddf["fold"].map_partitions( - lambda p_df: pd.Series(np.random.randint(0, 4, len(p_df))) - ) - - dd.assert_eq(ddf[0], pddf[0]) - dd.assert_eq(len(ddf["fold"]), len(pddf["fold"])) - - -def test_dataframe_set_index(): - random.seed(0) - df = cudf.datasets.randomdata(26, dtypes={"a": float, "b": int}) - df["str"] = list("abcdefghijklmnopqrstuvwxyz") - pdf = df.to_pandas() - - with dask.config.set({"dataframe.convert-string": False}): - ddf = dd.from_pandas(df, npartitions=4) - ddf = ddf.set_index("str") - - pddf = dd.from_pandas(pdf, npartitions=4) - pddf = pddf.set_index("str") - - from cudf.testing._utils import assert_eq - - assert_eq(ddf.compute(), pddf.compute()) - - -@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") -def test_series_describe(): - random.seed(0) - sr = cudf.datasets.randomdata(20)["x"] - psr = sr.to_pandas() - - dsr = dd.from_pandas(sr, npartitions=4) - pdsr = dd.from_pandas(psr, npartitions=4) - - dd.assert_eq( - dsr.describe(), - pdsr.describe(), - rtol=1e-3, - ) - - -@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") -def test_dataframe_describe(): - random.seed(0) - df = cudf.datasets.randomdata(20) - pdf = df.to_pandas() - - ddf = dd.from_pandas(df, npartitions=4) - pddf = dd.from_pandas(pdf, npartitions=4) - - dd.assert_eq( - ddf.describe(), pddf.describe(), check_exact=False, atol=0.0001 - ) - - -@pytest.mark.xfail(reason="describe is not cudf compatible in dask-expr") -def test_zero_std_describe(): - num = 84886781 - df = cudf.DataFrame( - { - "x": np.full((20,), num, dtype=np.float64), - "y": np.full((20,), num, dtype=np.float64), - } - ) - pdf = df.to_pandas() - ddf = dd.from_pandas(df, npartitions=4) - pddf = dd.from_pandas(pdf, npartitions=4) - - dd.assert_eq(ddf.describe(), pddf.describe(), rtol=1e-3) - - -def test_large_numbers_var(): - num = 8488678001 - df = cudf.DataFrame( - { - "x": np.arange(num, num + 1000, dtype=np.float64), - "y": np.arange(num, num + 1000, dtype=np.float64), - } - ) - pdf = df.to_pandas() - ddf = dd.from_pandas(df, npartitions=4) - pddf = dd.from_pandas(pdf, npartitions=4) - - dd.assert_eq(ddf.var(), pddf.var(), rtol=1e-3) - - -def test_index_map_partitions(): - # https://github.com/rapidsai/cudf/issues/6738 - - ddf = dd.from_pandas(pd.DataFrame({"a": range(10)}), npartitions=2) - mins_pd = ddf.index.map_partitions(M.min, meta=ddf.index).compute() - - gddf = dd.from_pandas(cudf.DataFrame({"a": range(10)}), npartitions=2) - mins_gd = gddf.index.map_partitions(M.min, meta=gddf.index).compute() - - dd.assert_eq(mins_pd, mins_gd) - - -def test_merging_categorical_columns(): - df_1 = cudf.DataFrame( - {"id_1": [0, 1, 2, 3], "cat_col": ["a", "b", "f", "f"]} - ) - ddf_1 = dd.from_pandas(df_1, npartitions=2) - ddf_1 = dd.categorical.categorize(ddf_1, columns=["cat_col"]) - - df_2 = cudf.DataFrame( - {"id_2": [111, 112, 113], "cat_col": ["g", "h", "f"]} - ) - ddf_2 = dd.from_pandas(df_2, npartitions=2) - ddf_2 = dd.categorical.categorize(ddf_2, columns=["cat_col"]) - - expected = cudf.DataFrame( - { - "id_1": [2, 3], - "cat_col": cudf.Series( - ["f", "f"], - dtype=cudf.CategoricalDtype( - categories=["a", "b", "f", "g", "h"], ordered=False - ), - ), - "id_2": [113, 113], - } - ) - with pytest.warns(UserWarning, match="mismatch"): - dd.assert_eq(ddf_1.merge(ddf_2), expected) - - -def test_correct_meta(): - try: - from dask.dataframe.dispatch import make_meta_obj # noqa: F401 - except ImportError: - pytest.skip("need make_meta_obj to be preset") - - # Need these local imports in this specific order. - # For context: https://github.com/rapidsai/cudf/issues/7946 - import pandas as pd - - from dask import dataframe as dd - - import dask_cudf # noqa: F401 - - df = pd.DataFrame({"a": [3, 4], "b": [1, 2]}) - ddf = dd.from_pandas(df, npartitions=1) - emb = ddf["a"].apply(pd.Series, meta={"c0": "int64", "c1": "int64"}) - - assert isinstance(emb, dd.DataFrame) - assert isinstance(emb._meta, pd.DataFrame) - - -def test_categorical_dtype_round_trip(): - s = cudf.Series(4 * ["foo"], dtype="category") - assert s.dtype.ordered is False - - ds = dd.from_pandas(s, npartitions=2) - pds = dd.from_pandas(s.to_pandas(), npartitions=2) - dd.assert_eq(ds, pds) - assert ds.dtype.ordered is False - - # Below validations are required, see: - # https://github.com/rapidsai/cudf/issues/11487#issuecomment-1208912383 - actual = ds.compute() - expected = pds.compute() - assert actual.dtype.ordered == expected.dtype.ordered diff --git a/python/dask_cudf/dask_cudf/tests/test_accessor.py b/python/dask_cudf/dask_cudf/tests/test_accessor.py index a6a457d98a4..24003499e5c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_accessor.py +++ b/python/dask_cudf/dask_cudf/tests/test_accessor.py @@ -12,6 +12,11 @@ from cudf.testing._utils import assert_eq, does_not_raise import dask_cudf as dgd +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() + ############################################################################# # Datetime Accessor # diff --git a/python/dask_cudf/dask_cudf/tests/test_applymap.py b/python/dask_cudf/dask_cudf/tests/test_applymap.py index 929f00ec296..90b121e9d57 100644 --- a/python/dask_cudf/dask_cudf/tests/test_applymap.py +++ b/python/dask_cudf/dask_cudf/tests/test_applymap.py @@ -5,7 +5,10 @@ from dask import dataframe as dd -from dask_cudf.tests.utils import _make_random_frame +from dask_cudf.tests.utils import _make_random_frame, skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() @pytest.mark.parametrize( diff --git a/python/dask_cudf/dask_cudf/tests/test_binops.py b/python/dask_cudf/dask_cudf/tests/test_binops.py index 87bd401accd..b222718a5f6 100644 --- a/python/dask_cudf/dask_cudf/tests/test_binops.py +++ b/python/dask_cudf/dask_cudf/tests/test_binops.py @@ -10,7 +10,10 @@ import cudf -from dask_cudf.tests.utils import _make_random_frame +from dask_cudf.tests.utils import _make_random_frame, skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() def _make_empty_frame(npartitions=2): diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index afe2a050695..af6b31255f6 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2024, NVIDIA CORPORATION. +# Copyright (c) 2024, NVIDIA CORPORATION. import random @@ -15,7 +15,13 @@ import cudf -import dask_cudf as dgd +import dask_cudf +from dask_cudf.backends import hash_object_cudf +from dask_cudf.tests.utils import ( + DASK_EXPR_ENABLED, + skip_dask_expr, + xfail_dask_expr, +) def test_from_dict_backend_dispatch(): @@ -28,7 +34,9 @@ def test_from_dict_backend_dispatch(): expect = cudf.DataFrame(data) with dask.config.set({"dataframe.backend": "cudf"}): ddf = dd.from_dict(data, npartitions=2) - assert isinstance(ddf, dgd.DataFrame) + if not DASK_EXPR_ENABLED: + assert isinstance(ddf, dask_cudf.DataFrame) + assert isinstance(ddf._meta, cudf.DataFrame) dd.assert_eq(expect, ddf) @@ -43,7 +51,9 @@ def test_to_backend(): assert isinstance(ddf._meta, pd.DataFrame) gdf = ddf.to_backend("cudf") - assert isinstance(gdf, dgd.DataFrame) + if not DASK_EXPR_ENABLED: + assert isinstance(gdf, dask_cudf.DataFrame) + assert isinstance(gdf._meta, cudf.DataFrame) dd.assert_eq(cudf.DataFrame(data), ddf) assert isinstance(gdf.to_backend()._meta, pd.DataFrame) @@ -58,13 +68,17 @@ def test_to_backend_kwargs(): # Using `nan_as_null=False` will result in a cudf-backed # Series with a NaN element (ranther than ) gser_nan = dser.to_backend("cudf", nan_as_null=False) - assert isinstance(gser_nan, dgd.Series) + if not DASK_EXPR_ENABLED: + assert isinstance(gser_nan, dask_cudf.Series) + assert isinstance(gser_nan._meta, cudf.Series) assert np.isnan(gser_nan.compute()).sum() == 1 # Using `nan_as_null=True` will result in a cudf-backed # Series with a element (ranther than NaN) gser_null = dser.to_backend("cudf", nan_as_null=True) - assert isinstance(gser_null, dgd.Series) + if not DASK_EXPR_ENABLED: + assert isinstance(gser_null, dask_cudf.Series) + assert isinstance(gser_null._meta, cudf.Series) assert np.isnan(gser_null.compute()).sum() == 0 # Check `nullable` argument for `cudf.Series.to_pandas` @@ -84,7 +98,7 @@ def test_to_backend_kwargs(): gser_null.to_backend("pandas", bad_arg=True) -def test_from_cudf(): +def test_from_pandas(): np.random.seed(0) df = pd.DataFrame( @@ -96,24 +110,24 @@ def test_from_cudf(): gdf = cudf.DataFrame.from_pandas(df) - # Test simple around to/from dask + # Test simple around to/from cudf ingested = dd.from_pandas(gdf, npartitions=2) dd.assert_eq(ingested, df) - # Test conversion to dask.dataframe - ddf = ingested.to_dask_dataframe() + # Test conversion back to pandas + ddf = ingested.to_backend("pandas") dd.assert_eq(ddf, df) -def test_from_cudf_multiindex_raises(): +def test_from_pandas_multiindex_raises(): df = cudf.DataFrame({"x": list("abc"), "y": [1, 2, 3], "z": [1, 2, 3]}) with pytest.raises(NotImplementedError): - # dask_cudf does not support MultiIndex yet - dgd.from_cudf(df.set_index(["x", "y"])) + # Dask does not support MultiIndex yet + dd.from_pandas(df.set_index(["x", "y"]), npartitions=1) -def test_from_cudf_with_generic_idx(): +def test_from_pandas_with_generic_idx(): cdf = cudf.DataFrame( { "a": list(range(20)), @@ -122,7 +136,7 @@ def test_from_cudf_with_generic_idx(): } ) - ddf = dgd.from_cudf(cdf, npartitions=2) + ddf = dd.from_pandas(cdf, npartitions=2) assert isinstance(ddf.index.compute(), cudf.RangeIndex) dd.assert_eq(ddf.loc[1:2, ["a"]], cdf.loc[1:2, ["a"]]) @@ -164,7 +178,7 @@ def test_query_local_dict(): {"x": np.random.randint(0, 5, size=10), "y": np.random.normal(size=10)} ) gdf = cudf.DataFrame.from_pandas(df) - ddf = dgd.from_cudf(gdf, npartitions=2) + ddf = dd.from_pandas(gdf, npartitions=2) val = 2 @@ -188,22 +202,8 @@ def test_head(): dd.assert_eq(dgf.head(), df.head()) -def test_from_dask_dataframe(): - np.random.seed(0) - df = pd.DataFrame( - {"x": np.random.randint(0, 5, size=20), "y": np.random.normal(size=20)} - ) - ddf = dd.from_pandas(df, npartitions=2) - dgdf = ddf.map_partitions(cudf.from_pandas) - got = dgdf.compute().to_pandas() - expect = df - - dd.assert_eq(got, expect) - - @pytest.mark.parametrize("nelem", [10, 200, 1333]) -@pytest.mark.parametrize("divisions", [None, "quantile"]) -def test_set_index(nelem, divisions): +def test_set_index(nelem): with dask.config.set(scheduler="single-threaded"): np.random.seed(0) # Use unique index range as the sort may not be stable-ordering @@ -213,14 +213,15 @@ def test_set_index(nelem, divisions): {"x": x, "y": np.random.randint(0, nelem, size=nelem)} ) ddf = dd.from_pandas(df, npartitions=2) - dgdf = ddf.map_partitions(cudf.from_pandas) + ddf2 = ddf.to_backend("cudf") expect = ddf.set_index("x") - got = dgdf.set_index("x", divisions=divisions) + got = ddf2.set_index("x") dd.assert_eq(expect, got, check_index=False, check_divisions=False) +@skip_dask_expr() @pytest.mark.parametrize("by", ["a", "b"]) @pytest.mark.parametrize("nelem", [10, 500]) @pytest.mark.parametrize("nparts", [1, 10]) @@ -270,7 +271,6 @@ def test_set_index_2(nelem): assert_frame_equal_by_index_group(expect, got) -@pytest.mark.xfail(reason="dask's index name '__dask_cudf.index' is correct") def test_set_index_w_series(): with dask.config.set(scheduler="single-threaded"): nelem = 20 @@ -296,7 +296,7 @@ def test_set_index_sorted(): ddf1 = dd.from_pandas(df1, npartitions=2) gdf1 = cudf.from_pandas(df1) - gddf1 = dgd.from_cudf(gdf1, npartitions=2) + gddf1 = dd.from_pandas(gdf1, npartitions=2) expect = ddf1.set_index("id", sorted=True) got = gddf1.set_index("id", sorted=True) @@ -323,7 +323,7 @@ def test_rearrange_by_divisions(nelem, index): df["z"] = df["z"].astype("category") ddf1 = dd.from_pandas(df, npartitions=4) - gdf1 = dgd.from_cudf(cudf.DataFrame.from_pandas(df), npartitions=4) + gdf1 = dd.from_pandas(cudf.DataFrame.from_pandas(df), npartitions=4) ddf1.index.name = index gdf1.index.name = index divisions = (0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20) @@ -399,6 +399,7 @@ def test_setitem_scalar_datetime(): np.testing.assert_array_equal(got["z"], df["z"]) +@skip_dask_expr() @pytest.mark.parametrize( "func", [ @@ -436,6 +437,7 @@ def test_repr(func): assert gddf._repr_html_() +@skip_dask_expr() @pytest.mark.skip(reason="datetime indexes not fully supported in cudf") @pytest.mark.parametrize("start", ["1d", "5d", "1w", "12h"]) @pytest.mark.parametrize("stop", ["1d", "3d", "8h"]) @@ -487,7 +489,7 @@ def test_repartition_hash_staged(npartitions): # WARNING: Specific npartitions-max_branch combination # was specifically chosen to cover changes in #4676 npartitions_initial = 17 - ddf = dgd.from_cudf(gdf, npartitions=npartitions_initial) + ddf = dd.from_pandas(gdf, npartitions=npartitions_initial) ddf_new = ddf.shuffle( on=by, ignore_index=True, npartitions=npartitions, max_branch=4 ) @@ -527,7 +529,7 @@ def test_repartition_hash(by, npartitions, max_branch): } ) gdf.d = gdf.d.astype("datetime64[ms]") - ddf = dgd.from_cudf(gdf, npartitions=npartitions_i) + ddf = dd.from_pandas(gdf, npartitions=npartitions_i) ddf_new = ddf.shuffle( on=by, ignore_index=True, @@ -554,7 +556,7 @@ def test_repartition_hash(by, npartitions, max_branch): def test_repartition_no_extra_row(): # see https://github.com/rapidsai/cudf/issues/11930 gdf = cudf.DataFrame({"a": [10, 20, 30], "b": [1, 2, 3]}).set_index("a") - ddf = dgd.from_cudf(gdf, npartitions=1) + ddf = dd.from_pandas(gdf, npartitions=1) ddf_new = ddf.repartition([0, 5, 10, 30], force=True) dd.assert_eq(ddf, ddf_new) dd.assert_eq(gdf, ddf_new) @@ -669,20 +671,20 @@ def test_hash_object_dispatch(index): # DataFrame result = dd.core.hash_object_dispatch(obj, index=index) - expected = dgd.backends.hash_object_cudf(obj, index=index) + expected = hash_object_cudf(obj, index=index) assert isinstance(result, cudf.Series) dd.assert_eq(result, expected) # Series result = dd.core.hash_object_dispatch(obj["x"], index=index) - expected = dgd.backends.hash_object_cudf(obj["x"], index=index) + expected = hash_object_cudf(obj["x"], index=index) assert isinstance(result, cudf.Series) dd.assert_eq(result, expected) # DataFrame with MultiIndex obj_multi = obj.set_index(["x", "z"], drop=True) result = dd.core.hash_object_dispatch(obj_multi, index=index) - expected = dgd.backends.hash_object_cudf(obj_multi, index=index) + expected = hash_object_cudf(obj_multi, index=index) assert isinstance(result, cudf.Series) dd.assert_eq(result, expected) @@ -729,7 +731,7 @@ def test_make_meta_backends(index): # Check dask code path if not MultiIndex if not isinstance(df.index, cudf.MultiIndex): - ddf = dgd.from_cudf(df, npartitions=1) + ddf = dd.from_pandas(df, npartitions=1) # Check "empty" metadata types dd.assert_eq(ddf._meta.dtypes, df.dtypes) @@ -751,7 +753,7 @@ def test_dataframe_series_replace(data): pdf = data.copy() gdf = cudf.from_pandas(pdf) - ddf = dgd.from_cudf(gdf, npartitions=5) + ddf = dd.from_pandas(gdf, npartitions=5) dd.assert_eq(ddf.replace(1, 2), pdf.replace(1, 2)) @@ -760,16 +762,16 @@ def test_dataframe_assign_col(): df = cudf.DataFrame(list(range(100))) pdf = pd.DataFrame(list(range(100))) - ddf = dgd.from_cudf(df, npartitions=4) + ddf = dd.from_pandas(df, npartitions=4) ddf["fold"] = 0 ddf["fold"] = ddf["fold"].map_partitions( - lambda cudf_df: cp.random.randint(0, 4, len(cudf_df)) + lambda cudf_df: cudf.Series(cp.random.randint(0, 4, len(cudf_df))) ) pddf = dd.from_pandas(pdf, npartitions=4) pddf["fold"] = 0 pddf["fold"] = pddf["fold"].map_partitions( - lambda p_df: np.random.randint(0, 4, len(p_df)) + lambda p_df: pd.Series(np.random.randint(0, 4, len(p_df))) ) dd.assert_eq(ddf[0], pddf[0]) @@ -783,7 +785,7 @@ def test_dataframe_set_index(): pdf = df.to_pandas() with dask.config.set({"dataframe.convert-string": False}): - ddf = dgd.from_cudf(df, npartitions=4) + ddf = dd.from_pandas(df, npartitions=4) ddf = ddf.set_index("str") pddf = dd.from_pandas(pdf, npartitions=4) @@ -794,12 +796,13 @@ def test_dataframe_set_index(): assert_eq(ddf.compute(), pddf.compute()) +@xfail_dask_expr("Insufficient describe support in dask-expr") def test_series_describe(): random.seed(0) sr = cudf.datasets.randomdata(20)["x"] psr = sr.to_pandas() - dsr = dgd.from_cudf(sr, npartitions=4) + dsr = dd.from_pandas(sr, npartitions=4) pdsr = dd.from_pandas(psr, npartitions=4) dd.assert_eq( @@ -809,12 +812,13 @@ def test_series_describe(): ) +@xfail_dask_expr("Insufficient describe support in dask-expr") def test_dataframe_describe(): random.seed(0) df = cudf.datasets.randomdata(20) pdf = df.to_pandas() - ddf = dgd.from_cudf(df, npartitions=4) + ddf = dd.from_pandas(df, npartitions=4) pddf = dd.from_pandas(pdf, npartitions=4) dd.assert_eq( @@ -822,6 +826,7 @@ def test_dataframe_describe(): ) +@xfail_dask_expr("Insufficient describe support in dask-expr") def test_zero_std_describe(): num = 84886781 df = cudf.DataFrame( @@ -831,7 +836,7 @@ def test_zero_std_describe(): } ) pdf = df.to_pandas() - ddf = dgd.from_cudf(df, npartitions=4) + ddf = dd.from_pandas(df, npartitions=4) pddf = dd.from_pandas(pdf, npartitions=4) dd.assert_eq(ddf.describe(), pddf.describe(), rtol=1e-3) @@ -846,7 +851,7 @@ def test_large_numbers_var(): } ) pdf = df.to_pandas() - ddf = dgd.from_cudf(df, npartitions=4) + ddf = dd.from_pandas(df, npartitions=4) pddf = dd.from_pandas(pdf, npartitions=4) dd.assert_eq(ddf.var(), pddf.var(), rtol=1e-3) @@ -858,37 +863,25 @@ def test_index_map_partitions(): ddf = dd.from_pandas(pd.DataFrame({"a": range(10)}), npartitions=2) mins_pd = ddf.index.map_partitions(M.min, meta=ddf.index).compute() - gddf = dgd.from_cudf(cudf.DataFrame({"a": range(10)}), npartitions=2) + gddf = dd.from_pandas(cudf.DataFrame({"a": range(10)}), npartitions=2) mins_gd = gddf.index.map_partitions(M.min, meta=gddf.index).compute() dd.assert_eq(mins_pd, mins_gd) def test_merging_categorical_columns(): - try: - from dask.dataframe.dispatch import ( # noqa: F401 - union_categoricals_dispatch, - ) - except ImportError: - pytest.skip( - "need a version of dask that has union_categoricals_dispatch" - ) - df_1 = cudf.DataFrame( {"id_1": [0, 1, 2, 3], "cat_col": ["a", "b", "f", "f"]} ) - - ddf_1 = dgd.from_cudf(df_1, npartitions=2) - + ddf_1 = dd.from_pandas(df_1, npartitions=2) ddf_1 = dd.categorical.categorize(ddf_1, columns=["cat_col"]) df_2 = cudf.DataFrame( {"id_2": [111, 112, 113], "cat_col": ["g", "h", "f"]} ) - - ddf_2 = dgd.from_cudf(df_2, npartitions=2) - + ddf_2 = dd.from_pandas(df_2, npartitions=2) ddf_2 = dd.categorical.categorize(ddf_2, columns=["cat_col"]) + expected = cudf.DataFrame( { "id_1": [2, 3], @@ -901,15 +894,11 @@ def test_merging_categorical_columns(): "id_2": [113, 113], } ) - dd.assert_eq(ddf_1.merge(ddf_2), expected) + with pytest.warns(UserWarning, match="mismatch"): + dd.assert_eq(ddf_1.merge(ddf_2), expected) def test_correct_meta(): - try: - from dask.dataframe.dispatch import make_meta_obj # noqa: F401 - except ImportError: - pytest.skip("need make_meta_obj to be preset") - # Need these local imports in this specific order. # For context: https://github.com/rapidsai/cudf/issues/7946 import pandas as pd @@ -930,7 +919,7 @@ def test_categorical_dtype_round_trip(): s = cudf.Series(4 * ["foo"], dtype="category") assert s.dtype.ordered is False - ds = dgd.from_cudf(s, npartitions=2) + ds = dd.from_pandas(s, npartitions=2) pds = dd.from_pandas(s.to_pandas(), npartitions=2) dd.assert_eq(ds, pds) assert ds.dtype.ordered is False diff --git a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py index 6c68d92a8df..b5ffb357ca2 100644 --- a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py +++ b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py @@ -13,6 +13,10 @@ import cudf as gd import dask_cudf as dgd +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() @delayed diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index 76703206726..3016aa0ae01 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -12,6 +12,11 @@ import cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() + def test_is_categorical_dispatch(): assert is_categorical_dtype(pd.CategoricalDtype([1, 2, 3])) diff --git a/python/dask_cudf/dask_cudf/tests/test_distributed.py b/python/dask_cudf/dask_cudf/tests/test_distributed.py index 39eadb45c91..5e15f33b18d 100644 --- a/python/dask_cudf/dask_cudf/tests/test_distributed.py +++ b/python/dask_cudf/dask_cudf/tests/test_distributed.py @@ -12,6 +12,11 @@ from cudf.testing._utils import assert_eq import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() + dask_cuda = pytest.importorskip("dask_cuda") diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index c8cc6e65fa5..ab1782fd79a 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -12,6 +12,10 @@ import dask_cudf from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() def assert_cudf_groupby_layers(ddf): diff --git a/python/dask_cudf/dask_cudf/tests/test_join.py b/python/dask_cudf/dask_cudf/tests/test_join.py index 73fd37df6fa..beaba8e5bfd 100644 --- a/python/dask_cudf/dask_cudf/tests/test_join.py +++ b/python/dask_cudf/dask_cudf/tests/test_join.py @@ -11,6 +11,11 @@ import cudf import dask_cudf as dgd +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() + param_nrows = [5, 10, 50, 100] diff --git a/python/dask_cudf/dask_cudf/tests/test_onehot.py b/python/dask_cudf/dask_cudf/tests/test_onehot.py index 6453d843467..506e2475898 100644 --- a/python/dask_cudf/dask_cudf/tests/test_onehot.py +++ b/python/dask_cudf/dask_cudf/tests/test_onehot.py @@ -8,6 +8,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() def test_get_dummies_cat(): diff --git a/python/dask_cudf/dask_cudf/tests/test_reductions.py b/python/dask_cudf/dask_cudf/tests/test_reductions.py index e347e8be9e4..85916f4b74c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_reductions.py +++ b/python/dask_cudf/dask_cudf/tests/test_reductions.py @@ -10,6 +10,10 @@ import cudf import dask_cudf as dgd +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() def _make_random_frame(nelem, npartitions=2): diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 8cf621da1bf..97bd594456d 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -10,6 +10,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() @pytest.mark.parametrize("ascending", [True, False]) diff --git a/python/dask_cudf/dask_cudf/tests/test_struct.py b/python/dask_cudf/dask_cudf/tests/test_struct.py index 6abac4cf53b..75a445ebd4e 100644 --- a/python/dask_cudf/dask_cudf/tests/test_struct.py +++ b/python/dask_cudf/dask_cudf/tests/test_struct.py @@ -5,6 +5,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() @pytest.mark.parametrize( diff --git a/python/dask_cudf/dask_cudf/tests/utils.py b/python/dask_cudf/dask_cudf/tests/utils.py index 88a2116fb0a..7bb996f9d3f 100644 --- a/python/dask_cudf/dask_cudf/tests/utils.py +++ b/python/dask_cudf/dask_cudf/tests/utils.py @@ -2,8 +2,10 @@ import numpy as np import pandas as pd +import pytest import dask.dataframe as dd +from dask import config import cudf @@ -19,3 +21,29 @@ def _make_random_frame(nelem, npartitions=2, include_na=False): gdf = cudf.DataFrame.from_pandas(df) dgf = dd.from_pandas(gdf, npartitions=npartitions) return df, dgf + + +def _dask_expr_enabled(): + from dask_cudf.expr import _expr_support + + return _expr_support and config.get("dataframe.query-planning", False) + + +DASK_EXPR_ENABLED = _dask_expr_enabled() +_default_reason = "Not compatible with dask-expr" + + +def skip_module_dask_expr(reason=_default_reason): + if DASK_EXPR_ENABLED: + pytest.skip( + allow_module_level=True, + reason=reason, + ) + + +def skip_dask_expr(reason=_default_reason): + return pytest.mark.skipif(DASK_EXPR_ENABLED, reason=reason) + + +def xfail_dask_expr(reason=_default_reason): + return pytest.mark.xfail(DASK_EXPR_ENABLED, reason=reason) diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index f4ef876f076..bc481f1bc6d 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,7 +44,6 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", - "git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20", "numba>=0.57", "pytest-cov", "pytest-xdist", From fdc59a04ed2ee5e50f8ca981a25ba0f44fa42035 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 12:12:39 -0800 Subject: [PATCH 23/74] add back env changes with charles' suggestion --- ci/test_python_other.sh | 20 +++++++++--------- .../all_cuda-118_arch-x86_64.yaml | 1 + .../all_cuda-120_arch-x86_64.yaml | 1 + dependencies.yaml | 21 +++++++++++-------- .../dask_cudf/tests/test_applymap.py | 2 +- .../dask_cudf/dask_cudf/tests/test_binops.py | 2 +- python/dask_cudf/dask_cudf/tests/test_core.py | 2 +- .../dask_cudf/tests/test_delayed_io.py | 2 +- python/dask_cudf/dask_cudf/tests/test_join.py | 2 +- .../dask_cudf/dask_cudf/tests/test_onehot.py | 2 +- .../dask_cudf/dask_cudf/tests/test_struct.py | 2 +- python/dask_cudf/pyproject.toml | 1 + 12 files changed, 32 insertions(+), 26 deletions(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index d926fc95a52..9e25805351c 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,16 +30,16 @@ pytest \ . popd -# # Dask-expr tests should be skipped if dask_expr is not installed -# rapids-logger "pytest dask_cudf + dask-expr" -# pushd python/dask_cudf/dask_cudf/expr -# DASK_DATAFRAME__QUERY_PLANNING=True pytest \ -# --cache-clear \ -# --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ -# --numprocesses=8 \ -# --dist=loadscope \ -# . -# popd +# Dask-expr tests should be skipped if dask_expr is not installed +rapids-logger "pytest dask_cudf + dask-expr" +pushd python/dask_cudf/dask_cudf +DASK_DATAFRAME__QUERY_PLANNING=True pytest \ + --cache-clear \ + --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ + --numprocesses=8 \ + --dist=loadscope \ + . +popd rapids-logger "pytest custreamz" pushd python/custreamz/custreamz diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index fa4ef8ddf68..5d7c54b38bd 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -102,5 +102,6 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: + - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-118_arch-x86_64 diff --git a/conda/environments/all_cuda-120_arch-x86_64.yaml b/conda/environments/all_cuda-120_arch-x86_64.yaml index a8be9d65c43..74d0516ba8d 100644 --- a/conda/environments/all_cuda-120_arch-x86_64.yaml +++ b/conda/environments/all_cuda-120_arch-x86_64.yaml @@ -100,5 +100,6 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: + - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-120_arch-x86_64 diff --git a/dependencies.yaml b/dependencies.yaml index 07b08fae9cf..9d4574801de 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -661,15 +661,18 @@ dependencies: packages: - dask-cuda==24.4.* - *numba - # - output_types: conda - # packages: - # - pip - # - pip: - # # This should eventually move to rapids-dask-dependency - # - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - # - output_types: [requirements, pyproject] - # packages: - # - *dask_expr_tip + - output_types: conda + packages: + - pip + - pip: + # This should eventually move to rapids-dask-dependency + - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + - output_types: requirements + packages: + - *dask_expr_tip + - output_types: pyproject + packages: + - dask-expr@git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/dask_cudf/tests/test_applymap.py b/python/dask_cudf/dask_cudf/tests/test_applymap.py index 90b121e9d57..32125d31702 100644 --- a/python/dask_cudf/dask_cudf/tests/test_applymap.py +++ b/python/dask_cudf/dask_cudf/tests/test_applymap.py @@ -1,4 +1,4 @@ -# Copyright (c) 2022, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. import pytest from pandas import NA diff --git a/python/dask_cudf/dask_cudf/tests/test_binops.py b/python/dask_cudf/dask_cudf/tests/test_binops.py index b222718a5f6..7bf585e6076 100644 --- a/python/dask_cudf/dask_cudf/tests/test_binops.py +++ b/python/dask_cudf/dask_cudf/tests/test_binops.py @@ -1,4 +1,4 @@ -# Copyright (c) 2022, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. import operator diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index af6b31255f6..abf84e80eee 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -1,4 +1,4 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. import random diff --git a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py index b5ffb357ca2..eec075756f0 100644 --- a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py +++ b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# Copyright (c) 2019-2024, NVIDIA CORPORATION. """ Test IO with dask.delayed API diff --git a/python/dask_cudf/dask_cudf/tests/test_join.py b/python/dask_cudf/dask_cudf/tests/test_join.py index beaba8e5bfd..5affb65f7a2 100644 --- a/python/dask_cudf/dask_cudf/tests/test_join.py +++ b/python/dask_cudf/dask_cudf/tests/test_join.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019-2023, NVIDIA CORPORATION. +# Copyright (c) 2019-2024, NVIDIA CORPORATION. from functools import partial diff --git a/python/dask_cudf/dask_cudf/tests/test_onehot.py b/python/dask_cudf/dask_cudf/tests/test_onehot.py index 506e2475898..3575aca6968 100644 --- a/python/dask_cudf/dask_cudf/tests/test_onehot.py +++ b/python/dask_cudf/dask_cudf/tests/test_onehot.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# Copyright (c) 2019-2024, NVIDIA CORPORATION. import pandas as pd import pytest diff --git a/python/dask_cudf/dask_cudf/tests/test_struct.py b/python/dask_cudf/dask_cudf/tests/test_struct.py index 75a445ebd4e..1c143496504 100644 --- a/python/dask_cudf/dask_cudf/tests/test_struct.py +++ b/python/dask_cudf/dask_cudf/tests/test_struct.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2022, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. import pytest diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index bc481f1bc6d..b17253f2258 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,6 +44,7 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", + "dask-expr@git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20", "numba>=0.57", "pytest-cov", "pytest-xdist", From 793b56e616b41e969ce4b84e4d6ad0337fd881d1 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 12:16:48 -0800 Subject: [PATCH 24/74] cw fix --- python/dask_cudf/dask_cudf/tests/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/tests/utils.py b/python/dask_cudf/dask_cudf/tests/utils.py index 7bb996f9d3f..f1b27ce59cb 100644 --- a/python/dask_cudf/dask_cudf/tests/utils.py +++ b/python/dask_cudf/dask_cudf/tests/utils.py @@ -1,4 +1,4 @@ -# Copyright (c) 2022, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. import numpy as np import pandas as pd From 838a897ba8a477641b4908cc66d93f942a6e0cfa Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:09:31 -0800 Subject: [PATCH 25/74] skip IO tests for now --- ci/test_wheel_dask_cudf.sh | 3 +++ python/dask_cudf/dask_cudf/io/tests/test_csv.py | 4 ++++ python/dask_cudf/dask_cudf/io/tests/test_json.py | 4 ++++ python/dask_cudf/dask_cudf/io/tests/test_orc.py | 4 ++++ python/dask_cudf/dask_cudf/io/tests/test_parquet.py | 4 ++++ python/dask_cudf/dask_cudf/io/tests/test_s3.py | 4 ++++ python/dask_cudf/dask_cudf/io/tests/test_text.py | 4 ++++ 7 files changed, 27 insertions(+) diff --git a/ci/test_wheel_dask_cudf.sh b/ci/test_wheel_dask_cudf.sh index e9162b816aa..c9c3ab14c52 100755 --- a/ci/test_wheel_dask_cudf.sh +++ b/ci/test_wheel_dask_cudf.sh @@ -28,3 +28,6 @@ python -m pip install $(echo ./dist/dask_cudf*.whl)[test] # Run tests in dask_cudf/tests and dask_cudf/io/tests python -m pytest -n 8 ./python/dask_cudf/dask_cudf/ + +# Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr +DASK_DATAFRAME__QUERY_PLANNING=True python -m pytest -n 8 ./python/dask_cudf/dask_cudf/ diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index a35a9f1be48..a9694efd70a 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -14,6 +14,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() @pytest.fixture diff --git a/python/dask_cudf/dask_cudf/io/tests/test_json.py b/python/dask_cudf/dask_cudf/io/tests/test_json.py index 5e06832ed94..68e0101a85c 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_json.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_json.py @@ -10,6 +10,10 @@ from dask.utils import tmpfile import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() def test_read_json_backend_dispatch(tmp_path): diff --git a/python/dask_cudf/dask_cudf/io/tests/test_orc.py b/python/dask_cudf/dask_cudf/io/tests/test_orc.py index c2be75e8ddd..347ce036f04 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_orc.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_orc.py @@ -12,6 +12,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() cur_dir = os.path.dirname(__file__) sample_orc = os.path.join(cur_dir, "data/orc/sample.orc") diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index 583d4b07f6f..af3368043bf 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -16,6 +16,10 @@ from cudf.core._compat import PANDAS_GE_200 import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() # Check if create_metadata_file is supported by # the current dask.dataframe version diff --git a/python/dask_cudf/dask_cudf/io/tests/test_s3.py b/python/dask_cudf/dask_cudf/io/tests/test_s3.py index 7614ea38d6a..27e64f397f4 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_s3.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_s3.py @@ -10,6 +10,10 @@ import pytest import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() moto = pytest.importorskip("moto", minversion="3.1.6") boto3 = pytest.importorskip("boto3") diff --git a/python/dask_cudf/dask_cudf/io/tests/test_text.py b/python/dask_cudf/dask_cudf/io/tests/test_text.py index a14eec1fea9..7caafae2e42 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_text.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_text.py @@ -9,6 +9,10 @@ import cudf import dask_cudf +from dask_cudf.tests.utils import skip_module_dask_expr + +# No dask-expr support +skip_module_dask_expr() cur_dir = os.path.dirname(__file__) text_file = os.path.join(cur_dir, "data/text/sample.pgn") From 59e9b8b6736f70b093bc0802e4e65cb82a624aaf Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:11:11 -0800 Subject: [PATCH 26/74] remove dask-expr tests from conda-python-other-tests for now --- ci/test_python_other.sh | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 9e25805351c..cda07f2d199 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,17 +30,6 @@ pytest \ . popd -# Dask-expr tests should be skipped if dask_expr is not installed -rapids-logger "pytest dask_cudf + dask-expr" -pushd python/dask_cudf/dask_cudf -DASK_DATAFRAME__QUERY_PLANNING=True pytest \ - --cache-clear \ - --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ - --numprocesses=8 \ - --dist=loadscope \ - . -popd - rapids-logger "pytest custreamz" pushd python/custreamz/custreamz pytest \ From 916a0e478cfd32798f91453242d0c0136e8ac9ef Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:13:34 -0800 Subject: [PATCH 27/74] remove extra cw change --- ci/test_python_other.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index cda07f2d199..25c1d681029 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -1,5 +1,5 @@ #!/bin/bash -# Copyright (c) 2022-2024, NVIDIA CORPORATION. +# Copyright (c) 2022-2023, NVIDIA CORPORATION. # Common setup steps shared by Python test jobs source "$(dirname "$0")/test_python_common.sh" From 6a4bbae4b7f10a98ef8f74ead562c5249528a5a9 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:18:28 -0800 Subject: [PATCH 28/74] fix cw dates --- python/dask_cudf/dask_cudf/io/tests/test_s3.py | 2 +- python/dask_cudf/dask_cudf/io/tests/test_text.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/io/tests/test_s3.py b/python/dask_cudf/dask_cudf/io/tests/test_s3.py index 27e64f397f4..299a1655248 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_s3.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_s3.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. import os import socket diff --git a/python/dask_cudf/dask_cudf/io/tests/test_text.py b/python/dask_cudf/dask_cudf/io/tests/test_text.py index 7caafae2e42..c05916c7977 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_text.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_text.py @@ -1,4 +1,4 @@ -# Copyright (c) 2022, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. import os From 9e296ffc463602c937541759691fcd633a00b781 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:23:05 -0800 Subject: [PATCH 29/74] fix cw dates --- ci/test_wheel_dask_cudf.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/test_wheel_dask_cudf.sh b/ci/test_wheel_dask_cudf.sh index c9c3ab14c52..aa2283aa4fa 100755 --- a/ci/test_wheel_dask_cudf.sh +++ b/ci/test_wheel_dask_cudf.sh @@ -1,5 +1,5 @@ #!/bin/bash -# Copyright (c) 2023, NVIDIA CORPORATION. +# Copyright (c) 2023-2024, NVIDIA CORPORATION. set -eou pipefail From 3a2beb460886bccb8eca93843e4ee9b807d3c403 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 13:55:07 -0800 Subject: [PATCH 30/74] add csv test coverage --- python/dask_cudf/dask_cudf/core.py | 13 ++++- .../dask_cudf/dask_cudf/io/tests/test_csv.py | 56 ++++++++++--------- 2 files changed, 39 insertions(+), 30 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 05f2f9ebdcb..a63dde544d3 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -8,7 +8,7 @@ import pandas as pd from tlz import partition_all -from dask import dataframe as dd +from dask import config, dataframe as dd from dask.base import normalize_token, tokenize from dask.dataframe.core import ( Scalar, @@ -690,13 +690,20 @@ def from_cudf(data, npartitions=None, chunksize=None, sort=True, name=None): "dask_cudf does not support MultiIndex Dataframes." ) - name = name or ("from_cudf-" + tokenize(data, npartitions or chunksize)) + # Dask-expr doesn't support the `name` argument + name = {} + if not config.get("dataframe.query-planning", False): + name = { + "name": name + or ("from_cudf-" + tokenize(data, npartitions or chunksize)) + } + return dd.from_pandas( data, npartitions=npartitions, chunksize=chunksize, sort=sort, - name=name, + **name, ) diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index a9694efd70a..7e8a1dd8f82 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -14,10 +14,12 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import DASK_EXPR_ENABLED, skip_dask_expr -# No dask-expr support -skip_module_dask_expr() + +def _read_csv(*args, **kwargs): + with dask.config.set({"dataframe.backend": "cudf"}): + return dd.read_csv(*args, **kwargs) @pytest.fixture @@ -62,12 +64,14 @@ def test_csv_roundtrip_backend_dispatch(tmp_path): ddf = dask_cudf.from_cudf(df, npartitions=2) csv_path = str(tmp_path / "data-*.csv") ddf.to_csv(csv_path, index=False) - with dask.config.set({"dataframe.backend": "cudf"}): - ddf2 = dd.read_csv(csv_path) - assert isinstance(ddf2, dask_cudf.DataFrame) + ddf2 = _read_csv(csv_path) + if not DASK_EXPR_ENABLED: + assert isinstance(ddf2, dask_cudf.DataFrame) + assert isinstance(ddf2._meta, cudf.DataFrame) dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) +@skip_dask_expr() def test_csv_roundtrip(tmp_path): df = cudf.DataFrame({"x": [1, 2, 3, 4], "id": ["a", "b", "c", "d"]}) ddf = dask_cudf.from_cudf(df, npartitions=2) @@ -85,7 +89,7 @@ def test_csv_roundtrip_filepath(tmp_path): ddf.to_csv(f"file://{stmp_path}", index=False) - ddf2 = dask_cudf.read_csv(f"file://{stmp_path}") + ddf2 = _read_csv(f"file://{stmp_path}") dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) @@ -97,25 +101,25 @@ def test_read_csv(tmp_path): csv_path = str(tmp_path / "data-*.csv") df.to_csv(csv_path, index=False) - df2 = dask_cudf.read_csv(csv_path) + df2 = _read_csv(csv_path) dd.assert_eq(df, df2) # file path test stmp_path = str(csv_path) - df3 = dask_cudf.read_csv(f"file://{stmp_path}") + df3 = _read_csv(f"file://{stmp_path}") dd.assert_eq(df2, df3) # file list test list_paths = [ os.path.join(tmp_path, fname) for fname in sorted(os.listdir(tmp_path)) ] - df4 = dask_cudf.read_csv(list_paths) + df4 = _read_csv(list_paths) dd.assert_eq(df, df4) def test_raises_FileNotFoundError(): with pytest.raises(FileNotFoundError): - dask_cudf.read_csv("foo.csv") + _read_csv("foo.csv") def test_read_csv_w_bytes(tmp_path): @@ -125,7 +129,7 @@ def test_read_csv_w_bytes(tmp_path): df = pd.DataFrame(dict(x=np.arange(20), y=np.arange(20))) df.to_csv(tmp_path / "data-*.csv", index=False) - df2 = dask_cudf.read_csv(tmp_path / "*.csv", blocksize="50 B") + df2 = _read_csv(tmp_path / "*.csv", blocksize="50 B") assert df2.npartitions == 3 dd.assert_eq(df2, df, check_index=False) @@ -135,7 +139,7 @@ def test_read_csv_compression(tmp_path): df.to_csv(tmp_path / "data.csv.gz", index=False) with pytest.warns(UserWarning) as w: - df2 = dask_cudf.read_csv(tmp_path / "*.csv.gz", blocksize="50 B") + df2 = _read_csv(tmp_path / "*.csv.gz", blocksize="50 B") assert len(w) == 1 msg = str(w[0].message) @@ -145,7 +149,7 @@ def test_read_csv_compression(tmp_path): dd.assert_eq(df2, df, check_index=False) with warnings.catch_warnings(record=True) as record: - df2 = dask_cudf.read_csv(tmp_path / "*.csv.gz", blocksize=None) + df2 = _read_csv(tmp_path / "*.csv.gz", blocksize=None) assert not record @@ -163,7 +167,7 @@ def test_read_csv_compression_file_list(tmp_path): fp.write(lines.encode("utf-8")) ddf_cpu = dd.read_csv(files, compression="gzip").compute() - ddf_gpu = dask_cudf.read_csv(files, compression="gzip").compute() + ddf_gpu = _read_csv(files, compression="gzip").compute() dd.assert_eq(ddf_cpu, ddf_gpu) @@ -186,12 +190,12 @@ def test_read_csv_blocksize_none(tmp_path, compression, size): typ = None df.to_csv(path, index=False, compression=compression) - df2 = dask_cudf.read_csv(path, blocksize=None, dtype=typ) + df2 = _read_csv(path, blocksize=None, dtype=typ) dd.assert_eq(df, df2) # Test chunksize deprecation with pytest.warns(FutureWarning, match="deprecated"): - df3 = dask_cudf.read_csv(path, chunksize=None, dtype=typ) + df3 = _read_csv(path, chunksize=None, dtype=typ) dd.assert_eq(df, df3) @@ -207,7 +211,7 @@ def test_csv_reader_usecols(tmp_path, dtype): csv_path = str(tmp_path / "usecols_data.csv") df.to_csv(csv_path, index=False) ddf = dask_cudf.from_cudf(df[["b", "c"]], npartitions=5) - ddf2 = dask_cudf.read_csv(csv_path, usecols=["b", "c"], dtype=dtype) + ddf2 = _read_csv(csv_path, usecols=["b", "c"], dtype=dtype) dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) @@ -215,7 +219,7 @@ def test_csv_reader_usecols(tmp_path, dtype): def test_read_csv_skiprows(csv_begin_bad_lines): # Repro from Issue#13552 ddf_cpu = dd.read_csv(csv_begin_bad_lines, skiprows=3).compute() - ddf_gpu = dask_cudf.read_csv(csv_begin_bad_lines, skiprows=3).compute() + ddf_gpu = _read_csv(csv_begin_bad_lines, skiprows=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu) @@ -223,7 +227,7 @@ def test_read_csv_skiprows(csv_begin_bad_lines): def test_read_csv_skiprows_error(csv_begin_bad_lines): # Repro from Issue#13552 with pytest.raises(ValueError): - dask_cudf.read_csv( + _read_csv( csv_begin_bad_lines, skiprows=3, blocksize="100 MiB" ).compute() @@ -232,7 +236,7 @@ def test_read_csv_skipfooter(csv_end_bad_lines): # Repro from Issue#13552 with dask.config.set({"dataframe.convert-string": False}): ddf_cpu = dd.read_csv(csv_end_bad_lines, skipfooter=3).compute() - ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, skipfooter=3).compute() + ddf_gpu = _read_csv(csv_end_bad_lines, skipfooter=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu, check_dtype=False) @@ -244,27 +248,25 @@ def test_read_csv_skipfooter_arrow_string_fail(request, csv_end_bad_lines): ) ) ddf_cpu = dd.read_csv(csv_end_bad_lines, skipfooter=3).compute() - ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, skipfooter=3).compute() + ddf_gpu = _read_csv(csv_end_bad_lines, skipfooter=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu, check_dtype=False) def test_read_csv_skipfooter_error(csv_end_bad_lines): with pytest.raises(ValueError): - dask_cudf.read_csv( + _read_csv( csv_end_bad_lines, skipfooter=3, blocksize="100 MiB" ).compute() def test_read_csv_nrows(csv_end_bad_lines): ddf_cpu = pd.read_csv(csv_end_bad_lines, nrows=2) - ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, nrows=2).compute() + ddf_gpu = _read_csv(csv_end_bad_lines, nrows=2).compute() dd.assert_eq(ddf_cpu, ddf_gpu) def test_read_csv_nrows_error(csv_end_bad_lines): with pytest.raises(ValueError): - dask_cudf.read_csv( - csv_end_bad_lines, nrows=2, blocksize="100 MiB" - ).compute() + _read_csv(csv_end_bad_lines, nrows=2, blocksize="100 MiB").compute() From b28b816056cdbaf0e600fe7ffd9499c8e743a8e7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 31 Jan 2024 17:10:13 -0800 Subject: [PATCH 31/74] add parquet test coverage --- ci/test_wheel_dask_cudf.sh | 1 + .../dask_cudf/dask_cudf/io/tests/test_csv.py | 7 +- .../dask_cudf/io/tests/test_parquet.py | 138 +++++++++--------- 3 files changed, 77 insertions(+), 69 deletions(-) diff --git a/ci/test_wheel_dask_cudf.sh b/ci/test_wheel_dask_cudf.sh index aa2283aa4fa..10458ef3475 100755 --- a/ci/test_wheel_dask_cudf.sh +++ b/ci/test_wheel_dask_cudf.sh @@ -30,4 +30,5 @@ python -m pip install $(echo ./dist/dask_cudf*.whl)[test] python -m pytest -n 8 ./python/dask_cudf/dask_cudf/ # Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr +echo "Running dask-cudf tests with dask-expr enabled." DASK_DATAFRAME__QUERY_PLANNING=True python -m pytest -n 8 ./python/dask_cudf/dask_cudf/ diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index 7e8a1dd8f82..21e376f2dbe 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -18,8 +18,11 @@ def _read_csv(*args, **kwargs): - with dask.config.set({"dataframe.backend": "cudf"}): - return dd.read_csv(*args, **kwargs) + if DASK_EXPR_ENABLED: + with dask.config.set({"dataframe.backend": "cudf"}): + return dd.read_csv(*args, **kwargs) + else: + return dask_cudf.read_csv(*args, **kwargs) @pytest.fixture diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index af3368043bf..a864db01aa4 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -16,10 +16,20 @@ from cudf.core._compat import PANDAS_GE_200 import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import ( + DASK_EXPR_ENABLED, + skip_dask_expr, + xfail_dask_expr, +) + + +def _read_parquet(*args, **kwargs): + if DASK_EXPR_ENABLED: + with dask.config.set({"dataframe.backend": "cudf"}): + return dd.read_parquet(*args, **kwargs) + else: + return dask_cudf.read_parquet(*args, **kwargs) -# No dask-expr support -skip_module_dask_expr() # Check if create_metadata_file is supported by # the current dask.dataframe version @@ -46,7 +56,9 @@ def test_roundtrip_backend_dispatch(tmpdir): ddf.to_parquet(tmpdir, engine="pyarrow") with dask.config.set({"dataframe.backend": "cudf"}): ddf2 = dd.read_parquet(tmpdir, index=False) - assert isinstance(ddf2, dask_cudf.DataFrame) + if not DASK_EXPR_ENABLED: + assert isinstance(ddf2, dask_cudf.DataFrame) + assert isinstance(ddf2._meta, cudf.DataFrame) dd.assert_eq(ddf.reset_index(drop=False), ddf2) @@ -63,43 +75,39 @@ def test_roundtrip_from_dask(tmpdir, divisions, write_metadata_file): ) # Read list of parquet files - ddf2 = dask_cudf.read_parquet(files, calculate_divisions=divisions) + ddf2 = _read_parquet(files, calculate_divisions=divisions) dd.assert_eq(ddf, ddf2, check_divisions=divisions) # Specify columns=['x'] - ddf2 = dask_cudf.read_parquet( - files, columns=["x"], calculate_divisions=divisions - ) + ddf2 = _read_parquet(files, columns=["x"], calculate_divisions=divisions) dd.assert_eq(ddf[["x"]], ddf2, check_divisions=divisions) - # Specify columns='y' - ddf2 = dask_cudf.read_parquet( - files, columns="y", calculate_divisions=divisions - ) - dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) + if not DASK_EXPR_ENABLED: # Dask-expr returns Series + # Specify columns='y' + ddf2 = _read_parquet(files, columns="y", calculate_divisions=divisions) + dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) # Now include metadata - ddf2 = dask_cudf.read_parquet(tmpdir, calculate_divisions=divisions) + ddf2 = _read_parquet(tmpdir, calculate_divisions=divisions) dd.assert_eq(ddf, ddf2, check_divisions=divisions) # Specify columns=['x'] (with metadata) - ddf2 = dask_cudf.read_parquet( - tmpdir, columns=["x"], calculate_divisions=divisions - ) + ddf2 = _read_parquet(tmpdir, columns=["x"], calculate_divisions=divisions) dd.assert_eq(ddf[["x"]], ddf2, check_divisions=divisions) - # Specify columns='y' (with metadata) - ddf2 = dask_cudf.read_parquet( - tmpdir, columns="y", calculate_divisions=divisions - ) - dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) + if not DASK_EXPR_ENABLED: # Dask-expr returns Series + # Specify columns='y' (with metadata) + ddf2 = _read_parquet( + tmpdir, columns="y", calculate_divisions=divisions + ) + dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) def test_roundtrip_from_dask_index_false(tmpdir): tmpdir = str(tmpdir) ddf.to_parquet(tmpdir, engine="pyarrow") - ddf2 = dask_cudf.read_parquet(tmpdir, index=False) + ddf2 = _read_parquet(tmpdir, index=False) dd.assert_eq(ddf.reset_index(drop=False), ddf2) @@ -110,7 +118,7 @@ def test_roundtrip_from_dask_none_index_false(tmpdir): df2 = ddf.reset_index(drop=True).compute() df2.to_parquet(path, engine="pyarrow") - ddf3 = dask_cudf.read_parquet(path, index=False) + ddf3 = _read_parquet(path, index=False) dd.assert_eq(df2, ddf3) @@ -120,7 +128,7 @@ def test_roundtrip_from_dask_cudf(tmpdir, write_meta): gddf = dask_cudf.from_dask_dataframe(ddf) gddf.to_parquet(tmpdir, write_metadata_file=write_meta) - gddf2 = dask_cudf.read_parquet(tmpdir, calculate_divisions=True) + gddf2 = _read_parquet(tmpdir, calculate_divisions=True) dd.assert_eq(gddf, gddf2) @@ -131,7 +139,7 @@ def test_roundtrip_none_rangeindex(tmpdir): index=pd.RangeIndex(start=5, stop=9), ) dask_cudf.from_cudf(gdf, npartitions=2).to_parquet(fn) - ddf2 = dask_cudf.read_parquet(fn) + ddf2 = _read_parquet(fn) dd.assert_eq(gdf, ddf2, check_index=True) @@ -142,13 +150,13 @@ def test_roundtrip_from_pandas(tmpdir): dfp = df.copy() dfp.to_parquet(fn, engine="pyarrow", index=False) dfp = dfp.reset_index(drop=True) - ddf2 = dask_cudf.read_parquet(fn) + ddf2 = _read_parquet(fn) dd.assert_eq(dfp, ddf2, check_index=True) # Now, specifying an index dfp = df.copy() dfp.to_parquet(fn, engine="pyarrow", index=True) - ddf2 = dask_cudf.read_parquet(fn, index=["index"]) + ddf2 = _read_parquet(fn, index=["index"]) dd.assert_eq(dfp, ddf2, check_index=True) @@ -160,7 +168,7 @@ def test_strings(tmpdir): dfp.set_index("a", inplace=True, drop=True) ddf2 = dd.from_pandas(dfp, npartitions=2) ddf2.to_parquet(fn, engine="pyarrow") - read_df = dask_cudf.read_parquet(fn, index=["a"]) + read_df = _read_parquet(fn, index=["a"]) dd.assert_eq(ddf2, read_df.compute().to_pandas()) @@ -169,7 +177,7 @@ def test_dask_timeseries_from_pandas(tmpdir): ddf2 = dask.datasets.timeseries(freq="D") pdf = ddf2.compute() pdf.to_parquet(fn, engine="pyarrow") - read_df = dask_cudf.read_parquet(fn) + read_df = _read_parquet(fn) # Workaround until following issue is fixed: # https://github.com/apache/arrow/issues/33321 dd.assert_eq(ddf2, read_df.compute(), check_index_type=not PANDAS_GE_200) @@ -181,14 +189,13 @@ def test_dask_timeseries_from_dask(tmpdir, index, divisions): fn = str(tmpdir) ddf2 = dask.datasets.timeseries(freq="D") ddf2.to_parquet(fn, engine="pyarrow", write_index=index) - read_df = dask_cudf.read_parquet( - fn, index=index, calculate_divisions=divisions - ) + read_df = _read_parquet(fn, index=index, calculate_divisions=divisions) dd.assert_eq( ddf2, read_df, check_divisions=(divisions and index), check_index=index ) +@xfail_dask_expr("Categorical column support") @pytest.mark.parametrize("index", [False, None]) @pytest.mark.parametrize("divisions", [False, True]) def test_dask_timeseries_from_daskcudf(tmpdir, index, divisions): @@ -198,9 +205,7 @@ def test_dask_timeseries_from_daskcudf(tmpdir, index, divisions): ) ddf2.name = ddf2.name.astype("object") ddf2.to_parquet(fn, write_index=index) - read_df = dask_cudf.read_parquet( - fn, index=index, calculate_divisions=divisions - ) + read_df = _read_parquet(fn, index=index, calculate_divisions=divisions) dd.assert_eq( ddf2, read_df, check_divisions=(divisions and index), check_index=index ) @@ -215,7 +220,7 @@ def test_empty(tmpdir, index): ddf2 = dd.from_pandas(dfp, npartitions=2) ddf2.to_parquet(fn, write_index=index, engine="pyarrow") - read_df = dask_cudf.read_parquet(fn) + read_df = _read_parquet(fn) dd.assert_eq(ddf2, read_df.compute()) @@ -227,20 +232,18 @@ def test_filters(tmpdir): ddf.to_parquet(tmp_path, engine="pyarrow") - a = dask_cudf.read_parquet( - tmp_path, filters=[("x", ">", 4)], split_row_groups=True - ) + a = _read_parquet(tmp_path, filters=[("x", ">", 4)], split_row_groups=True) assert a.npartitions == 3 assert (a.x > 3).all().compute() - b = dask_cudf.read_parquet( + b = _read_parquet( tmp_path, filters=[("y", "==", "c")], split_row_groups=True ) assert b.npartitions == 1 b = b.compute().to_pandas() assert (b.y == "c").all() - c = dask_cudf.read_parquet( + c = _read_parquet( tmp_path, filters=[("y", "==", "c"), ("x", ">", 6)], split_row_groups=True, @@ -268,17 +271,13 @@ def test_isna_filters(tmpdir, null, numeric): # Test "is" col = "i" if numeric else "j" filters = [(col, "is", null)] - out = dask_cudf.read_parquet( - tmp_path, filters=filters, split_row_groups=True - ) + out = _read_parquet(tmp_path, filters=filters, split_row_groups=True) assert len(out) == 2 assert list(out.x.compute().values) == [4, 5] # Test "is not" filters = [(col, "is not", null)] - out = dask_cudf.read_parquet( - tmp_path, filters=filters, split_row_groups=True - ) + out = _read_parquet(tmp_path, filters=filters, split_row_groups=True) assert len(out) == 8 assert list(out.x.compute().values) == [0, 1, 2, 3, 6, 7, 8, 9] @@ -291,15 +290,18 @@ def test_filters_at_row_group_level(tmpdir): ddf.to_parquet(tmp_path, engine="pyarrow", row_group_size=10 / 5) - a = dask_cudf.read_parquet( + a = _read_parquet( tmp_path, filters=[("x", "==", 1)], split_row_groups=True ) assert a.npartitions == 1 assert (a.shape[0] == 1).compute() - ddf.to_parquet(tmp_path, engine="pyarrow", row_group_size=1) + # Note: Use overwrite=True to ignore cache (for now) + ddf.to_parquet( + tmp_path, engine="pyarrow", row_group_size=1, overwrite=True + ) - b = dask_cudf.read_parquet( + b = _read_parquet( tmp_path, filters=[("x", "==", 1)], split_row_groups=True ) assert b.npartitions == 1 @@ -334,7 +336,7 @@ def test_roundtrip_from_dask_partitioned(tmpdir, parts, daskcudf, metadata): partition_on=parts, ) df_read = dd.read_parquet(tmpdir, engine="pyarrow") - gdf_read = dask_cudf.read_parquet(tmpdir) + gdf_read = _read_parquet(tmpdir) # TODO: Avoid column selection after `CudfEngine` # can be aligned with dask/dask#6534 @@ -357,7 +359,7 @@ def test_roundtrip_from_dask_partitioned(tmpdir, parts, daskcudf, metadata): df_read = dd.read_parquet( tmpdir, engine="pyarrow", aggregate_files="year", split_row_groups=2 ) - gdf_read = dask_cudf.read_parquet( + gdf_read = _read_parquet( tmpdir, aggregate_files="year", split_row_groups=2 ) dd.assert_eq(df_read, gdf_read) @@ -391,7 +393,7 @@ def test_split_row_groups(tmpdir, row_groups, index): write_metadata_file=True, ) - ddf2 = dask_cudf.read_parquet( + ddf2 = _read_parquet( str(tmpdir), split_row_groups=row_groups, ) @@ -428,7 +430,7 @@ def test_create_metadata_file(tmpdir, partition_on): # Check that we can now read the ddf # with the _metadata file present - ddf2 = dask_cudf.read_parquet( + ddf2 = _read_parquet( tmpdir, split_row_groups=False, index="myindex", @@ -441,6 +443,7 @@ def test_create_metadata_file(tmpdir, partition_on): dd.assert_eq(ddf1, ddf2) +@xfail_dask_expr("dtypes are inconsistent") @need_create_meta def test_create_metadata_file_inconsistent_schema(tmpdir): # NOTE: This test demonstrates that the CudfEngine @@ -462,7 +465,7 @@ def test_create_metadata_file_inconsistent_schema(tmpdir): # New pyarrow-dataset base can handle an inconsistent # schema (even without a _metadata file), but computing # and dtype validation may fail - ddf1 = dask_cudf.read_parquet(str(tmpdir), calculate_divisions=True) + ddf1 = _read_parquet(str(tmpdir), calculate_divisions=True) # Add global metadata file. # Dask-CuDF can do this without requiring schema @@ -471,7 +474,7 @@ def test_create_metadata_file_inconsistent_schema(tmpdir): # Check that we can still read the ddf # with the _metadata file present - ddf2 = dask_cudf.read_parquet(str(tmpdir), calculate_divisions=True) + ddf2 = _read_parquet(str(tmpdir), calculate_divisions=True) # Check that the result is the same with and # without the _metadata file. Note that we must @@ -499,7 +502,7 @@ def test_cudf_dtypes_from_pandas(tmpdir, data): dfp.to_parquet(fn, engine="pyarrow", index=True) # Use `split_row_groups=True` to avoid "fast path" where # schema is not is passed through in older Dask versions - ddf2 = dask_cudf.read_parquet(fn, split_row_groups=True) + ddf2 = _read_parquet(fn, split_row_groups=True) dd.assert_eq(cudf.from_pandas(dfp), ddf2) @@ -517,19 +520,21 @@ def test_cudf_list_struct_write(tmpdir): temp_file = str(tmpdir.join("list_struct.parquet")) ddf.to_parquet(temp_file) - new_ddf = dask_cudf.read_parquet(temp_file) + new_ddf = _read_parquet(temp_file) dd.assert_eq(df, new_ddf) +@skip_dask_expr("Not necessary in dask-expr") def test_check_file_size(tmpdir): # Test simple file-size check to help warn users # of upstream change to `split_row_groups` default fn = str(tmpdir.join("test.parquet")) cudf.DataFrame({"a": np.arange(1000)}).to_parquet(fn) with pytest.warns(match="large parquet file"): - dask_cudf.read_parquet(fn, check_file_size=1).compute() + _read_parquet(fn, check_file_size=1).compute() +@xfail_dask_expr("HivePartitioning cannot be hashed") def test_null_partition(tmpdir): import pyarrow as pa from pyarrow.dataset import HivePartitioning @@ -542,7 +547,7 @@ def test_null_partition(tmpdir): assert len(fns) == 3 partitioning = HivePartitioning(pa.schema([("id", pa.int64())])) - ddf_read = dask_cudf.read_parquet( + ddf_read = _read_parquet( str(tmpdir), dataset={"partitioning": partitioning}, ) @@ -559,11 +564,10 @@ def test_nullable_schema_mismatch(tmpdir): path1 = str(tmpdir.join("test.1.parquet")) cudf.DataFrame.from_dict({"a": [1, 2, 3]}).to_parquet(path0) cudf.DataFrame.from_dict({"a": [4, 5, None]}).to_parquet(path1) - with dask.config.set({"dataframe.backend": "cudf"}): - ddf = dd.read_parquet( - [path0, path1], split_row_groups=2, aggregate_files=True - ) - expect = pd.read_parquet([path0, path1]) + ddf = _read_parquet( + [path0, path1], split_row_groups=2, aggregate_files=True + ) + expect = pd.read_parquet([path0, path1]) dd.assert_eq(ddf, expect, check_index=False) @@ -587,7 +591,7 @@ def test_parquet_read_filter_and_project(tmpdir): # Read back with filter and projection columns = ["b"] filters = [[("a", "==", 5), ("c", ">", 20)]] - got = dask_cudf.read_parquet(path, columns=columns, filters=filters) + got = _read_parquet(path, columns=columns, filters=filters) # Check result expected = df[(df.a == 5) & (df.c > 20)][columns].reset_index(drop=True) From 68be23c7ad4ca2e4dd322cff4589b30ea858d0eb Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 1 Feb 2024 07:52:57 -0800 Subject: [PATCH 32/74] adjust top-level dask_cudf API --- python/dask_cudf/dask_cudf/__init__.py | 65 +++++++--- python/dask_cudf/dask_cudf/backends.py | 6 +- python/dask_cudf/dask_cudf/expr/__init__.py | 22 ++-- .../dask_cudf/dask_cudf/io/tests/test_csv.py | 50 ++++---- .../dask_cudf/io/tests/test_parquet.py | 113 ++++++++++-------- python/dask_cudf/dask_cudf/tests/utils.py | 10 +- 6 files changed, 147 insertions(+), 119 deletions(-) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index c152a9e6a81..ebce33e19eb 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -1,28 +1,57 @@ -# Copyright (c) 2018-2023, NVIDIA CORPORATION. +# Copyright (c) 2018-2024, NVIDIA CORPORATION. +import dask.dataframe as dd +from dask import config from dask.dataframe import from_delayed import cudf from . import backends from ._version import __git_commit__, __version__ -from .core import DataFrame, Series, concat, from_cudf, from_dask_dataframe -from .groupby import groupby_agg -from .io import read_csv, read_json, read_orc, read_text, to_orc - -try: - from .io import read_parquet -except ImportError: - pass - -__all__ = [ - "DataFrame", - "Series", - "from_cudf", - "from_dask_dataframe", - "concat", - "from_delayed", -] +from .core import concat, from_cudf, from_dask_dataframe +from .expr import DASK_EXPR_ENABLED + + +def read_csv(*args, **kwargs): + with config.set({"dataframe.backend": "cudf"}): + return dd.read_csv(*args, **kwargs) + + +def read_json(*args, **kwargs): + with config.set({"dataframe.backend": "cudf"}): + return dd.read_json(*args, **kwargs) + + +def read_orc(*args, **kwargs): + with config.set({"dataframe.backend": "cudf"}): + return dd.read_orc(*args, **kwargs) + + +def read_parquet(*args, **kwargs): + with config.set({"dataframe.backend": "cudf"}): + return dd.read_parquet(*args, **kwargs) + + +if DASK_EXPR_ENABLED: + __all__ = [ + "from_cudf", + "from_dask_dataframe", + "concat", + "from_delayed", + ] +else: + from .core import DataFrame, Series + from .groupby import groupby_agg + from .io import read_text, to_orc + + __all__ = [ + "DataFrame", + "Series", + "from_cudf", + "from_dask_dataframe", + "concat", + "from_delayed", + ] if not hasattr(cudf.DataFrame, "mean"): cudf.DataFrame.mean = None diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index c6b0b9a64a1..3401bc494d6 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -625,15 +625,13 @@ def read_csv(*args, **kwargs): @staticmethod def read_hdf(*args, **kwargs): - from dask_cudf import from_dask_dataframe - # HDF5 reader not yet implemented in cudf warnings.warn( "read_hdf is not yet implemented in cudf/dask_cudf. " "Moving to cudf from pandas. Expect poor performance!" ) - return from_dask_dataframe( - _default_backend(dd.read_hdf, *args, **kwargs) + return _default_backend(dd.read_hdf, *args, **kwargs).to_backend( + "cudf" ) diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index 37d2822efaa..93652861018 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -1,11 +1,17 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -# Make sure custom expressions and collections are defined -try: - import dask_cudf.expr._collection - import dask_cudf.expr._expr +from dask import config - _expr_support = True -except ImportError: - # Dask Expressions not installed - _expr_support = False +DASK_EXPR_ENABLED = False +if config.get("dataframe.query-planning", False): + # Make sure custom expressions and collections are defined + try: + import dask_cudf.expr._collection + import dask_cudf.expr._expr + + DASK_EXPR_ENABLED = True + except ImportError: + # Dask Expressions not installed. + # Dask DataFrame should have already thrown an error + # before we got here. + pass diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index 21e376f2dbe..31334e10fb0 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -17,14 +17,6 @@ from dask_cudf.tests.utils import DASK_EXPR_ENABLED, skip_dask_expr -def _read_csv(*args, **kwargs): - if DASK_EXPR_ENABLED: - with dask.config.set({"dataframe.backend": "cudf"}): - return dd.read_csv(*args, **kwargs) - else: - return dask_cudf.read_csv(*args, **kwargs) - - @pytest.fixture def csv_begin_bad_lines(tmp_path): lines = """x @@ -67,7 +59,7 @@ def test_csv_roundtrip_backend_dispatch(tmp_path): ddf = dask_cudf.from_cudf(df, npartitions=2) csv_path = str(tmp_path / "data-*.csv") ddf.to_csv(csv_path, index=False) - ddf2 = _read_csv(csv_path) + ddf2 = dask_cudf.read_csv(csv_path) if not DASK_EXPR_ENABLED: assert isinstance(ddf2, dask_cudf.DataFrame) assert isinstance(ddf2._meta, cudf.DataFrame) @@ -92,7 +84,7 @@ def test_csv_roundtrip_filepath(tmp_path): ddf.to_csv(f"file://{stmp_path}", index=False) - ddf2 = _read_csv(f"file://{stmp_path}") + ddf2 = dask_cudf.read_csv(f"file://{stmp_path}") dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) @@ -104,25 +96,25 @@ def test_read_csv(tmp_path): csv_path = str(tmp_path / "data-*.csv") df.to_csv(csv_path, index=False) - df2 = _read_csv(csv_path) + df2 = dask_cudf.read_csv(csv_path) dd.assert_eq(df, df2) # file path test stmp_path = str(csv_path) - df3 = _read_csv(f"file://{stmp_path}") + df3 = dask_cudf.read_csv(f"file://{stmp_path}") dd.assert_eq(df2, df3) # file list test list_paths = [ os.path.join(tmp_path, fname) for fname in sorted(os.listdir(tmp_path)) ] - df4 = _read_csv(list_paths) + df4 = dask_cudf.read_csv(list_paths) dd.assert_eq(df, df4) def test_raises_FileNotFoundError(): with pytest.raises(FileNotFoundError): - _read_csv("foo.csv") + dask_cudf.read_csv("foo.csv") def test_read_csv_w_bytes(tmp_path): @@ -132,7 +124,7 @@ def test_read_csv_w_bytes(tmp_path): df = pd.DataFrame(dict(x=np.arange(20), y=np.arange(20))) df.to_csv(tmp_path / "data-*.csv", index=False) - df2 = _read_csv(tmp_path / "*.csv", blocksize="50 B") + df2 = dask_cudf.read_csv(tmp_path / "*.csv", blocksize="50 B") assert df2.npartitions == 3 dd.assert_eq(df2, df, check_index=False) @@ -142,7 +134,7 @@ def test_read_csv_compression(tmp_path): df.to_csv(tmp_path / "data.csv.gz", index=False) with pytest.warns(UserWarning) as w: - df2 = _read_csv(tmp_path / "*.csv.gz", blocksize="50 B") + df2 = dask_cudf.read_csv(tmp_path / "*.csv.gz", blocksize="50 B") assert len(w) == 1 msg = str(w[0].message) @@ -152,7 +144,7 @@ def test_read_csv_compression(tmp_path): dd.assert_eq(df2, df, check_index=False) with warnings.catch_warnings(record=True) as record: - df2 = _read_csv(tmp_path / "*.csv.gz", blocksize=None) + df2 = dask_cudf.read_csv(tmp_path / "*.csv.gz", blocksize=None) assert not record @@ -170,7 +162,7 @@ def test_read_csv_compression_file_list(tmp_path): fp.write(lines.encode("utf-8")) ddf_cpu = dd.read_csv(files, compression="gzip").compute() - ddf_gpu = _read_csv(files, compression="gzip").compute() + ddf_gpu = dask_cudf.read_csv(files, compression="gzip").compute() dd.assert_eq(ddf_cpu, ddf_gpu) @@ -193,12 +185,12 @@ def test_read_csv_blocksize_none(tmp_path, compression, size): typ = None df.to_csv(path, index=False, compression=compression) - df2 = _read_csv(path, blocksize=None, dtype=typ) + df2 = dask_cudf.read_csv(path, blocksize=None, dtype=typ) dd.assert_eq(df, df2) # Test chunksize deprecation with pytest.warns(FutureWarning, match="deprecated"): - df3 = _read_csv(path, chunksize=None, dtype=typ) + df3 = dask_cudf.read_csv(path, chunksize=None, dtype=typ) dd.assert_eq(df, df3) @@ -214,7 +206,7 @@ def test_csv_reader_usecols(tmp_path, dtype): csv_path = str(tmp_path / "usecols_data.csv") df.to_csv(csv_path, index=False) ddf = dask_cudf.from_cudf(df[["b", "c"]], npartitions=5) - ddf2 = _read_csv(csv_path, usecols=["b", "c"], dtype=dtype) + ddf2 = dask_cudf.read_csv(csv_path, usecols=["b", "c"], dtype=dtype) dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) @@ -222,7 +214,7 @@ def test_csv_reader_usecols(tmp_path, dtype): def test_read_csv_skiprows(csv_begin_bad_lines): # Repro from Issue#13552 ddf_cpu = dd.read_csv(csv_begin_bad_lines, skiprows=3).compute() - ddf_gpu = _read_csv(csv_begin_bad_lines, skiprows=3).compute() + ddf_gpu = dask_cudf.read_csv(csv_begin_bad_lines, skiprows=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu) @@ -230,7 +222,7 @@ def test_read_csv_skiprows(csv_begin_bad_lines): def test_read_csv_skiprows_error(csv_begin_bad_lines): # Repro from Issue#13552 with pytest.raises(ValueError): - _read_csv( + dask_cudf.read_csv( csv_begin_bad_lines, skiprows=3, blocksize="100 MiB" ).compute() @@ -239,7 +231,7 @@ def test_read_csv_skipfooter(csv_end_bad_lines): # Repro from Issue#13552 with dask.config.set({"dataframe.convert-string": False}): ddf_cpu = dd.read_csv(csv_end_bad_lines, skipfooter=3).compute() - ddf_gpu = _read_csv(csv_end_bad_lines, skipfooter=3).compute() + ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, skipfooter=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu, check_dtype=False) @@ -251,25 +243,27 @@ def test_read_csv_skipfooter_arrow_string_fail(request, csv_end_bad_lines): ) ) ddf_cpu = dd.read_csv(csv_end_bad_lines, skipfooter=3).compute() - ddf_gpu = _read_csv(csv_end_bad_lines, skipfooter=3).compute() + ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, skipfooter=3).compute() dd.assert_eq(ddf_cpu, ddf_gpu, check_dtype=False) def test_read_csv_skipfooter_error(csv_end_bad_lines): with pytest.raises(ValueError): - _read_csv( + dask_cudf.read_csv( csv_end_bad_lines, skipfooter=3, blocksize="100 MiB" ).compute() def test_read_csv_nrows(csv_end_bad_lines): ddf_cpu = pd.read_csv(csv_end_bad_lines, nrows=2) - ddf_gpu = _read_csv(csv_end_bad_lines, nrows=2).compute() + ddf_gpu = dask_cudf.read_csv(csv_end_bad_lines, nrows=2).compute() dd.assert_eq(ddf_cpu, ddf_gpu) def test_read_csv_nrows_error(csv_end_bad_lines): with pytest.raises(ValueError): - _read_csv(csv_end_bad_lines, nrows=2, blocksize="100 MiB").compute() + dask_cudf.read_csv( + csv_end_bad_lines, nrows=2, blocksize="100 MiB" + ).compute() diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index a864db01aa4..da3477a3e31 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -22,15 +22,6 @@ xfail_dask_expr, ) - -def _read_parquet(*args, **kwargs): - if DASK_EXPR_ENABLED: - with dask.config.set({"dataframe.backend": "cudf"}): - return dd.read_parquet(*args, **kwargs) - else: - return dask_cudf.read_parquet(*args, **kwargs) - - # Check if create_metadata_file is supported by # the current dask.dataframe version need_create_meta = pytest.mark.skipif( @@ -75,39 +66,43 @@ def test_roundtrip_from_dask(tmpdir, divisions, write_metadata_file): ) # Read list of parquet files - ddf2 = _read_parquet(files, calculate_divisions=divisions) + ddf2 = dask_cudf.read_parquet(files, calculate_divisions=divisions) dd.assert_eq(ddf, ddf2, check_divisions=divisions) # Specify columns=['x'] - ddf2 = _read_parquet(files, columns=["x"], calculate_divisions=divisions) + ddf2 = dask_cudf.read_parquet( + files, columns=["x"], calculate_divisions=divisions + ) dd.assert_eq(ddf[["x"]], ddf2, check_divisions=divisions) - if not DASK_EXPR_ENABLED: # Dask-expr returns Series - # Specify columns='y' - ddf2 = _read_parquet(files, columns="y", calculate_divisions=divisions) - dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) + # Specify columns='y' + ddf2 = dask_cudf.read_parquet( + files, columns="y", calculate_divisions=divisions + ) + dd.assert_eq(ddf["y"], ddf2, check_divisions=divisions) # Now include metadata - ddf2 = _read_parquet(tmpdir, calculate_divisions=divisions) + ddf2 = dask_cudf.read_parquet(tmpdir, calculate_divisions=divisions) dd.assert_eq(ddf, ddf2, check_divisions=divisions) # Specify columns=['x'] (with metadata) - ddf2 = _read_parquet(tmpdir, columns=["x"], calculate_divisions=divisions) + ddf2 = dask_cudf.read_parquet( + tmpdir, columns=["x"], calculate_divisions=divisions + ) dd.assert_eq(ddf[["x"]], ddf2, check_divisions=divisions) - if not DASK_EXPR_ENABLED: # Dask-expr returns Series - # Specify columns='y' (with metadata) - ddf2 = _read_parquet( - tmpdir, columns="y", calculate_divisions=divisions - ) - dd.assert_eq(ddf[["y"]], ddf2, check_divisions=divisions) + # Specify columns='y' (with metadata) + ddf2 = dask_cudf.read_parquet( + tmpdir, columns="y", calculate_divisions=divisions + ) + dd.assert_eq(ddf["y"], ddf2, check_divisions=divisions) def test_roundtrip_from_dask_index_false(tmpdir): tmpdir = str(tmpdir) ddf.to_parquet(tmpdir, engine="pyarrow") - ddf2 = _read_parquet(tmpdir, index=False) + ddf2 = dask_cudf.read_parquet(tmpdir, index=False) dd.assert_eq(ddf.reset_index(drop=False), ddf2) @@ -118,7 +113,7 @@ def test_roundtrip_from_dask_none_index_false(tmpdir): df2 = ddf.reset_index(drop=True).compute() df2.to_parquet(path, engine="pyarrow") - ddf3 = _read_parquet(path, index=False) + ddf3 = dask_cudf.read_parquet(path, index=False) dd.assert_eq(df2, ddf3) @@ -128,7 +123,7 @@ def test_roundtrip_from_dask_cudf(tmpdir, write_meta): gddf = dask_cudf.from_dask_dataframe(ddf) gddf.to_parquet(tmpdir, write_metadata_file=write_meta) - gddf2 = _read_parquet(tmpdir, calculate_divisions=True) + gddf2 = dask_cudf.read_parquet(tmpdir, calculate_divisions=True) dd.assert_eq(gddf, gddf2) @@ -139,7 +134,7 @@ def test_roundtrip_none_rangeindex(tmpdir): index=pd.RangeIndex(start=5, stop=9), ) dask_cudf.from_cudf(gdf, npartitions=2).to_parquet(fn) - ddf2 = _read_parquet(fn) + ddf2 = dask_cudf.read_parquet(fn) dd.assert_eq(gdf, ddf2, check_index=True) @@ -150,13 +145,13 @@ def test_roundtrip_from_pandas(tmpdir): dfp = df.copy() dfp.to_parquet(fn, engine="pyarrow", index=False) dfp = dfp.reset_index(drop=True) - ddf2 = _read_parquet(fn) + ddf2 = dask_cudf.read_parquet(fn) dd.assert_eq(dfp, ddf2, check_index=True) # Now, specifying an index dfp = df.copy() dfp.to_parquet(fn, engine="pyarrow", index=True) - ddf2 = _read_parquet(fn, index=["index"]) + ddf2 = dask_cudf.read_parquet(fn, index=["index"]) dd.assert_eq(dfp, ddf2, check_index=True) @@ -168,7 +163,7 @@ def test_strings(tmpdir): dfp.set_index("a", inplace=True, drop=True) ddf2 = dd.from_pandas(dfp, npartitions=2) ddf2.to_parquet(fn, engine="pyarrow") - read_df = _read_parquet(fn, index=["a"]) + read_df = dask_cudf.read_parquet(fn, index=["a"]) dd.assert_eq(ddf2, read_df.compute().to_pandas()) @@ -177,7 +172,7 @@ def test_dask_timeseries_from_pandas(tmpdir): ddf2 = dask.datasets.timeseries(freq="D") pdf = ddf2.compute() pdf.to_parquet(fn, engine="pyarrow") - read_df = _read_parquet(fn) + read_df = dask_cudf.read_parquet(fn) # Workaround until following issue is fixed: # https://github.com/apache/arrow/issues/33321 dd.assert_eq(ddf2, read_df.compute(), check_index_type=not PANDAS_GE_200) @@ -189,7 +184,9 @@ def test_dask_timeseries_from_dask(tmpdir, index, divisions): fn = str(tmpdir) ddf2 = dask.datasets.timeseries(freq="D") ddf2.to_parquet(fn, engine="pyarrow", write_index=index) - read_df = _read_parquet(fn, index=index, calculate_divisions=divisions) + read_df = dask_cudf.read_parquet( + fn, index=index, calculate_divisions=divisions + ) dd.assert_eq( ddf2, read_df, check_divisions=(divisions and index), check_index=index ) @@ -205,7 +202,9 @@ def test_dask_timeseries_from_daskcudf(tmpdir, index, divisions): ) ddf2.name = ddf2.name.astype("object") ddf2.to_parquet(fn, write_index=index) - read_df = _read_parquet(fn, index=index, calculate_divisions=divisions) + read_df = dask_cudf.read_parquet( + fn, index=index, calculate_divisions=divisions + ) dd.assert_eq( ddf2, read_df, check_divisions=(divisions and index), check_index=index ) @@ -220,7 +219,7 @@ def test_empty(tmpdir, index): ddf2 = dd.from_pandas(dfp, npartitions=2) ddf2.to_parquet(fn, write_index=index, engine="pyarrow") - read_df = _read_parquet(fn) + read_df = dask_cudf.read_parquet(fn) dd.assert_eq(ddf2, read_df.compute()) @@ -232,18 +231,20 @@ def test_filters(tmpdir): ddf.to_parquet(tmp_path, engine="pyarrow") - a = _read_parquet(tmp_path, filters=[("x", ">", 4)], split_row_groups=True) + a = dask_cudf.read_parquet( + tmp_path, filters=[("x", ">", 4)], split_row_groups=True + ) assert a.npartitions == 3 assert (a.x > 3).all().compute() - b = _read_parquet( + b = dask_cudf.read_parquet( tmp_path, filters=[("y", "==", "c")], split_row_groups=True ) assert b.npartitions == 1 b = b.compute().to_pandas() assert (b.y == "c").all() - c = _read_parquet( + c = dask_cudf.read_parquet( tmp_path, filters=[("y", "==", "c"), ("x", ">", 6)], split_row_groups=True, @@ -271,13 +272,17 @@ def test_isna_filters(tmpdir, null, numeric): # Test "is" col = "i" if numeric else "j" filters = [(col, "is", null)] - out = _read_parquet(tmp_path, filters=filters, split_row_groups=True) + out = dask_cudf.read_parquet( + tmp_path, filters=filters, split_row_groups=True + ) assert len(out) == 2 assert list(out.x.compute().values) == [4, 5] # Test "is not" filters = [(col, "is not", null)] - out = _read_parquet(tmp_path, filters=filters, split_row_groups=True) + out = dask_cudf.read_parquet( + tmp_path, filters=filters, split_row_groups=True + ) assert len(out) == 8 assert list(out.x.compute().values) == [0, 1, 2, 3, 6, 7, 8, 9] @@ -290,7 +295,7 @@ def test_filters_at_row_group_level(tmpdir): ddf.to_parquet(tmp_path, engine="pyarrow", row_group_size=10 / 5) - a = _read_parquet( + a = dask_cudf.read_parquet( tmp_path, filters=[("x", "==", 1)], split_row_groups=True ) assert a.npartitions == 1 @@ -301,7 +306,7 @@ def test_filters_at_row_group_level(tmpdir): tmp_path, engine="pyarrow", row_group_size=1, overwrite=True ) - b = _read_parquet( + b = dask_cudf.read_parquet( tmp_path, filters=[("x", "==", 1)], split_row_groups=True ) assert b.npartitions == 1 @@ -336,7 +341,7 @@ def test_roundtrip_from_dask_partitioned(tmpdir, parts, daskcudf, metadata): partition_on=parts, ) df_read = dd.read_parquet(tmpdir, engine="pyarrow") - gdf_read = _read_parquet(tmpdir) + gdf_read = dask_cudf.read_parquet(tmpdir) # TODO: Avoid column selection after `CudfEngine` # can be aligned with dask/dask#6534 @@ -359,7 +364,7 @@ def test_roundtrip_from_dask_partitioned(tmpdir, parts, daskcudf, metadata): df_read = dd.read_parquet( tmpdir, engine="pyarrow", aggregate_files="year", split_row_groups=2 ) - gdf_read = _read_parquet( + gdf_read = dask_cudf.read_parquet( tmpdir, aggregate_files="year", split_row_groups=2 ) dd.assert_eq(df_read, gdf_read) @@ -393,7 +398,7 @@ def test_split_row_groups(tmpdir, row_groups, index): write_metadata_file=True, ) - ddf2 = _read_parquet( + ddf2 = dask_cudf.read_parquet( str(tmpdir), split_row_groups=row_groups, ) @@ -430,7 +435,7 @@ def test_create_metadata_file(tmpdir, partition_on): # Check that we can now read the ddf # with the _metadata file present - ddf2 = _read_parquet( + ddf2 = dask_cudf.read_parquet( tmpdir, split_row_groups=False, index="myindex", @@ -465,7 +470,7 @@ def test_create_metadata_file_inconsistent_schema(tmpdir): # New pyarrow-dataset base can handle an inconsistent # schema (even without a _metadata file), but computing # and dtype validation may fail - ddf1 = _read_parquet(str(tmpdir), calculate_divisions=True) + ddf1 = dask_cudf.read_parquet(str(tmpdir), calculate_divisions=True) # Add global metadata file. # Dask-CuDF can do this without requiring schema @@ -474,7 +479,7 @@ def test_create_metadata_file_inconsistent_schema(tmpdir): # Check that we can still read the ddf # with the _metadata file present - ddf2 = _read_parquet(str(tmpdir), calculate_divisions=True) + ddf2 = dask_cudf.read_parquet(str(tmpdir), calculate_divisions=True) # Check that the result is the same with and # without the _metadata file. Note that we must @@ -502,7 +507,7 @@ def test_cudf_dtypes_from_pandas(tmpdir, data): dfp.to_parquet(fn, engine="pyarrow", index=True) # Use `split_row_groups=True` to avoid "fast path" where # schema is not is passed through in older Dask versions - ddf2 = _read_parquet(fn, split_row_groups=True) + ddf2 = dask_cudf.read_parquet(fn, split_row_groups=True) dd.assert_eq(cudf.from_pandas(dfp), ddf2) @@ -520,7 +525,7 @@ def test_cudf_list_struct_write(tmpdir): temp_file = str(tmpdir.join("list_struct.parquet")) ddf.to_parquet(temp_file) - new_ddf = _read_parquet(temp_file) + new_ddf = dask_cudf.read_parquet(temp_file) dd.assert_eq(df, new_ddf) @@ -531,7 +536,9 @@ def test_check_file_size(tmpdir): fn = str(tmpdir.join("test.parquet")) cudf.DataFrame({"a": np.arange(1000)}).to_parquet(fn) with pytest.warns(match="large parquet file"): - _read_parquet(fn, check_file_size=1).compute() + # Need to use `dask_cudf.io` path + # TODO: Remove outdated `check_file_size` functionality + dask_cudf.io.read_parquet(fn, check_file_size=1).compute() @xfail_dask_expr("HivePartitioning cannot be hashed") @@ -547,7 +554,7 @@ def test_null_partition(tmpdir): assert len(fns) == 3 partitioning = HivePartitioning(pa.schema([("id", pa.int64())])) - ddf_read = _read_parquet( + ddf_read = dask_cudf.read_parquet( str(tmpdir), dataset={"partitioning": partitioning}, ) @@ -564,7 +571,7 @@ def test_nullable_schema_mismatch(tmpdir): path1 = str(tmpdir.join("test.1.parquet")) cudf.DataFrame.from_dict({"a": [1, 2, 3]}).to_parquet(path0) cudf.DataFrame.from_dict({"a": [4, 5, None]}).to_parquet(path1) - ddf = _read_parquet( + ddf = dask_cudf.read_parquet( [path0, path1], split_row_groups=2, aggregate_files=True ) expect = pd.read_parquet([path0, path1]) @@ -591,7 +598,7 @@ def test_parquet_read_filter_and_project(tmpdir): # Read back with filter and projection columns = ["b"] filters = [[("a", "==", 5), ("c", ">", 20)]] - got = _read_parquet(path, columns=columns, filters=filters) + got = dask_cudf.read_parquet(path, columns=columns, filters=filters) # Check result expected = df[(df.a == 5) & (df.c > 20)][columns].reset_index(drop=True) diff --git a/python/dask_cudf/dask_cudf/tests/utils.py b/python/dask_cudf/dask_cudf/tests/utils.py index f1b27ce59cb..18ad10bfe3a 100644 --- a/python/dask_cudf/dask_cudf/tests/utils.py +++ b/python/dask_cudf/dask_cudf/tests/utils.py @@ -5,10 +5,11 @@ import pytest import dask.dataframe as dd -from dask import config import cudf +from dask_cudf.expr import DASK_EXPR_ENABLED + def _make_random_frame(nelem, npartitions=2, include_na=False): df = pd.DataFrame( @@ -23,13 +24,6 @@ def _make_random_frame(nelem, npartitions=2, include_na=False): return df, dgf -def _dask_expr_enabled(): - from dask_cudf.expr import _expr_support - - return _expr_support and config.get("dataframe.query-planning", False) - - -DASK_EXPR_ENABLED = _dask_expr_enabled() _default_reason = "Not compatible with dask-expr" From 542f696ab0d4a00268584121e204e25f042bd253 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 1 Feb 2024 08:52:43 -0800 Subject: [PATCH 33/74] improve/simplify UX --- python/dask_cudf/dask_cudf/__init__.py | 41 ++++++++++++------- .../dask_cudf/dask_cudf/io/tests/test_csv.py | 6 +-- .../dask_cudf/io/tests/test_parquet.py | 10 +---- python/dask_cudf/dask_cudf/tests/test_core.py | 22 +++------- 4 files changed, 35 insertions(+), 44 deletions(-) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index ebce33e19eb..0b23dcb4a1d 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -32,26 +32,37 @@ def read_parquet(*args, **kwargs): return dd.read_parquet(*args, **kwargs) +def raise_not_implemented_error(attr_name): + def inner_func(*args, **kwargs): + raise NotImplementedError( + f"Top-level {attr_name} API is not available for dask-expr." + ) + + return inner_func + + if DASK_EXPR_ENABLED: - __all__ = [ - "from_cudf", - "from_dask_dataframe", - "concat", - "from_delayed", - ] + from .expr._collection import DataFrame, Index, Series + + groupby_agg = raise_not_implemented_error("groupby_agg") + read_text = raise_not_implemented_error("read_text") + to_orc = raise_not_implemented_error("to_orc") else: - from .core import DataFrame, Series + from .core import DataFrame, Index, Series from .groupby import groupby_agg from .io import read_text, to_orc - __all__ = [ - "DataFrame", - "Series", - "from_cudf", - "from_dask_dataframe", - "concat", - "from_delayed", - ] + +__all__ = [ + "DataFrame", + "Series", + "Index", + "from_cudf", + "from_dask_dataframe", + "concat", + "from_delayed", +] + if not hasattr(cudf.DataFrame, "mean"): cudf.DataFrame.mean = None diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index 31334e10fb0..226818fd141 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -14,7 +14,7 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import DASK_EXPR_ENABLED, skip_dask_expr +from dask_cudf.tests.utils import skip_dask_expr @pytest.fixture @@ -60,9 +60,7 @@ def test_csv_roundtrip_backend_dispatch(tmp_path): csv_path = str(tmp_path / "data-*.csv") ddf.to_csv(csv_path, index=False) ddf2 = dask_cudf.read_csv(csv_path) - if not DASK_EXPR_ENABLED: - assert isinstance(ddf2, dask_cudf.DataFrame) - assert isinstance(ddf2._meta, cudf.DataFrame) + assert isinstance(ddf2, dask_cudf.DataFrame) dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index da3477a3e31..10a382f0317 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -16,11 +16,7 @@ from cudf.core._compat import PANDAS_GE_200 import dask_cudf -from dask_cudf.tests.utils import ( - DASK_EXPR_ENABLED, - skip_dask_expr, - xfail_dask_expr, -) +from dask_cudf.tests.utils import skip_dask_expr, xfail_dask_expr # Check if create_metadata_file is supported by # the current dask.dataframe version @@ -47,9 +43,7 @@ def test_roundtrip_backend_dispatch(tmpdir): ddf.to_parquet(tmpdir, engine="pyarrow") with dask.config.set({"dataframe.backend": "cudf"}): ddf2 = dd.read_parquet(tmpdir, index=False) - if not DASK_EXPR_ENABLED: - assert isinstance(ddf2, dask_cudf.DataFrame) - assert isinstance(ddf2._meta, cudf.DataFrame) + assert isinstance(ddf2, dask_cudf.DataFrame) dd.assert_eq(ddf.reset_index(drop=False), ddf2) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index abf84e80eee..15a57ca7c3d 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -17,11 +17,7 @@ import dask_cudf from dask_cudf.backends import hash_object_cudf -from dask_cudf.tests.utils import ( - DASK_EXPR_ENABLED, - skip_dask_expr, - xfail_dask_expr, -) +from dask_cudf.tests.utils import skip_dask_expr, xfail_dask_expr def test_from_dict_backend_dispatch(): @@ -34,9 +30,7 @@ def test_from_dict_backend_dispatch(): expect = cudf.DataFrame(data) with dask.config.set({"dataframe.backend": "cudf"}): ddf = dd.from_dict(data, npartitions=2) - if not DASK_EXPR_ENABLED: - assert isinstance(ddf, dask_cudf.DataFrame) - assert isinstance(ddf._meta, cudf.DataFrame) + assert isinstance(ddf, dask_cudf.DataFrame) dd.assert_eq(expect, ddf) @@ -51,9 +45,7 @@ def test_to_backend(): assert isinstance(ddf._meta, pd.DataFrame) gdf = ddf.to_backend("cudf") - if not DASK_EXPR_ENABLED: - assert isinstance(gdf, dask_cudf.DataFrame) - assert isinstance(gdf._meta, cudf.DataFrame) + assert isinstance(gdf, dask_cudf.DataFrame) dd.assert_eq(cudf.DataFrame(data), ddf) assert isinstance(gdf.to_backend()._meta, pd.DataFrame) @@ -68,17 +60,13 @@ def test_to_backend_kwargs(): # Using `nan_as_null=False` will result in a cudf-backed # Series with a NaN element (ranther than ) gser_nan = dser.to_backend("cudf", nan_as_null=False) - if not DASK_EXPR_ENABLED: - assert isinstance(gser_nan, dask_cudf.Series) - assert isinstance(gser_nan._meta, cudf.Series) + assert isinstance(gser_nan, dask_cudf.Series) assert np.isnan(gser_nan.compute()).sum() == 1 # Using `nan_as_null=True` will result in a cudf-backed # Series with a element (ranther than NaN) gser_null = dser.to_backend("cudf", nan_as_null=True) - if not DASK_EXPR_ENABLED: - assert isinstance(gser_null, dask_cudf.Series) - assert isinstance(gser_null._meta, cudf.Series) + assert isinstance(gser_null, dask_cudf.Series) assert np.isnan(gser_null.compute()).sum() == 0 # Check `nullable` argument for `cudf.Series.to_pandas` From 20d6f313d4762edf27e0f5bcec79b17ddeafdd82 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 1 Feb 2024 09:52:59 -0800 Subject: [PATCH 34/74] fix from_dict --- python/dask_cudf/dask_cudf/expr/_collection.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 6fff3260beb..33585824e11 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -8,6 +8,8 @@ get_collection_type, ) +from dask import config + import cudf ## @@ -16,6 +18,11 @@ class DataFrame(DXDataFrame): + @classmethod + def from_dict(cls, *args, **kwargs): + with config.set({"dataframe.backend": "cudf"}): + return DXDataFrame.from_dict(*args, **kwargs) + def groupby( self, by, From e922d46463ce6788c1a6aa6b2707aeec48769c8e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 1 Feb 2024 11:13:26 -0800 Subject: [PATCH 35/74] remove extra utility --- python/dask_cudf/dask_cudf/io/tests/test_json.py | 4 ++-- python/dask_cudf/dask_cudf/io/tests/test_orc.py | 4 ++-- python/dask_cudf/dask_cudf/io/tests/test_s3.py | 4 ++-- python/dask_cudf/dask_cudf/io/tests/test_text.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_accessor.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_applymap.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_binops.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_delayed_io.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_dispatch.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_distributed.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_groupby.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_join.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_onehot.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_reductions.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_sort.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_struct.py | 4 ++-- python/dask_cudf/dask_cudf/tests/utils.py | 8 -------- 17 files changed, 32 insertions(+), 40 deletions(-) diff --git a/python/dask_cudf/dask_cudf/io/tests/test_json.py b/python/dask_cudf/dask_cudf/io/tests/test_json.py index 68e0101a85c..a2b1d7fc114 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_json.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_json.py @@ -10,10 +10,10 @@ from dask.utils import tmpfile import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def test_read_json_backend_dispatch(tmp_path): diff --git a/python/dask_cudf/dask_cudf/io/tests/test_orc.py b/python/dask_cudf/dask_cudf/io/tests/test_orc.py index 347ce036f04..8ccb7a7bfe7 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_orc.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_orc.py @@ -12,10 +12,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() cur_dir = os.path.dirname(__file__) sample_orc = os.path.join(cur_dir, "data/orc/sample.orc") diff --git a/python/dask_cudf/dask_cudf/io/tests/test_s3.py b/python/dask_cudf/dask_cudf/io/tests/test_s3.py index 299a1655248..f4a6fabdb60 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_s3.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_s3.py @@ -10,10 +10,10 @@ import pytest import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() moto = pytest.importorskip("moto", minversion="3.1.6") boto3 = pytest.importorskip("boto3") diff --git a/python/dask_cudf/dask_cudf/io/tests/test_text.py b/python/dask_cudf/dask_cudf/io/tests/test_text.py index c05916c7977..d3dcd386d0d 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_text.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_text.py @@ -9,10 +9,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() cur_dir = os.path.dirname(__file__) text_file = os.path.join(cur_dir, "data/text/sample.pgn") diff --git a/python/dask_cudf/dask_cudf/tests/test_accessor.py b/python/dask_cudf/dask_cudf/tests/test_accessor.py index e2ca4fc5285..24139a60b50 100644 --- a/python/dask_cudf/dask_cudf/tests/test_accessor.py +++ b/python/dask_cudf/dask_cudf/tests/test_accessor.py @@ -12,10 +12,10 @@ from cudf.testing._utils import assert_eq, does_not_raise import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() ############################################################################# diff --git a/python/dask_cudf/dask_cudf/tests/test_applymap.py b/python/dask_cudf/dask_cudf/tests/test_applymap.py index 32125d31702..521f273c812 100644 --- a/python/dask_cudf/dask_cudf/tests/test_applymap.py +++ b/python/dask_cudf/dask_cudf/tests/test_applymap.py @@ -5,10 +5,10 @@ from dask import dataframe as dd -from dask_cudf.tests.utils import _make_random_frame, skip_module_dask_expr +from dask_cudf.tests.utils import _make_random_frame, skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() @pytest.mark.parametrize( diff --git a/python/dask_cudf/dask_cudf/tests/test_binops.py b/python/dask_cudf/dask_cudf/tests/test_binops.py index 7bf585e6076..276fb1e6203 100644 --- a/python/dask_cudf/dask_cudf/tests/test_binops.py +++ b/python/dask_cudf/dask_cudf/tests/test_binops.py @@ -10,10 +10,10 @@ import cudf -from dask_cudf.tests.utils import _make_random_frame, skip_module_dask_expr +from dask_cudf.tests.utils import _make_random_frame, skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def _make_empty_frame(npartitions=2): diff --git a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py index 41259ea1fdb..174b285724c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py +++ b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py @@ -13,10 +13,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() @delayed diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index 3016aa0ae01..8decf20d595 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -12,10 +12,10 @@ import cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def test_is_categorical_dispatch(): diff --git a/python/dask_cudf/dask_cudf/tests/test_distributed.py b/python/dask_cudf/dask_cudf/tests/test_distributed.py index 5e15f33b18d..8413abb103f 100644 --- a/python/dask_cudf/dask_cudf/tests/test_distributed.py +++ b/python/dask_cudf/dask_cudf/tests/test_distributed.py @@ -12,10 +12,10 @@ from cudf.testing._utils import assert_eq import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() dask_cuda = pytest.importorskip("dask_cuda") diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index ab1782fd79a..10536c7c7ad 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -12,10 +12,10 @@ import dask_cudf from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def assert_cudf_groupby_layers(ddf): diff --git a/python/dask_cudf/dask_cudf/tests/test_join.py b/python/dask_cudf/dask_cudf/tests/test_join.py index df9580c9378..cec5e55fe19 100644 --- a/python/dask_cudf/dask_cudf/tests/test_join.py +++ b/python/dask_cudf/dask_cudf/tests/test_join.py @@ -11,10 +11,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() param_nrows = [5, 10, 50, 100] diff --git a/python/dask_cudf/dask_cudf/tests/test_onehot.py b/python/dask_cudf/dask_cudf/tests/test_onehot.py index 3575aca6968..3d00e56d07e 100644 --- a/python/dask_cudf/dask_cudf/tests/test_onehot.py +++ b/python/dask_cudf/dask_cudf/tests/test_onehot.py @@ -8,10 +8,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def test_get_dummies_cat(): diff --git a/python/dask_cudf/dask_cudf/tests/test_reductions.py b/python/dask_cudf/dask_cudf/tests/test_reductions.py index 01ac8dca1ff..81ca32eb30c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_reductions.py +++ b/python/dask_cudf/dask_cudf/tests/test_reductions.py @@ -10,10 +10,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() def _make_random_frame(nelem, npartitions=2): diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 97bd594456d..030d122594f 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -10,10 +10,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() @pytest.mark.parametrize("ascending", [True, False]) diff --git a/python/dask_cudf/dask_cudf/tests/test_struct.py b/python/dask_cudf/dask_cudf/tests/test_struct.py index 1c143496504..c0d1fe04072 100644 --- a/python/dask_cudf/dask_cudf/tests/test_struct.py +++ b/python/dask_cudf/dask_cudf/tests/test_struct.py @@ -5,10 +5,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_module_dask_expr +from dask_cudf.tests.utils import skip_dask_expr # No dask-expr support -skip_module_dask_expr() +pytestmark = skip_dask_expr() @pytest.mark.parametrize( diff --git a/python/dask_cudf/dask_cudf/tests/utils.py b/python/dask_cudf/dask_cudf/tests/utils.py index 18ad10bfe3a..ce0e046e26f 100644 --- a/python/dask_cudf/dask_cudf/tests/utils.py +++ b/python/dask_cudf/dask_cudf/tests/utils.py @@ -27,14 +27,6 @@ def _make_random_frame(nelem, npartitions=2, include_na=False): _default_reason = "Not compatible with dask-expr" -def skip_module_dask_expr(reason=_default_reason): - if DASK_EXPR_ENABLED: - pytest.skip( - allow_module_level=True, - reason=reason, - ) - - def skip_dask_expr(reason=_default_reason): return pytest.mark.skipif(DASK_EXPR_ENABLED, reason=reason) From 2634419d8250c17097f13aebb5a5ef091d2bcb2e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 2 Feb 2024 07:30:39 -0800 Subject: [PATCH 36/74] fix direct support with dask-expr --- python/dask_cudf/dask_cudf/__init__.py | 4 +-- python/dask_cudf/dask_cudf/backends.py | 10 +++++-- python/dask_cudf/dask_cudf/expr/__init__.py | 30 ++++++++++++--------- python/dask_cudf/dask_cudf/tests/utils.py | 6 ++--- 4 files changed, 31 insertions(+), 19 deletions(-) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index 0b23dcb4a1d..95c44421dfb 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -9,7 +9,7 @@ from . import backends from ._version import __git_commit__, __version__ from .core import concat, from_cudf, from_dask_dataframe -from .expr import DASK_EXPR_ENABLED +from .expr import QUERY_PLANNING_ON def read_csv(*args, **kwargs): @@ -41,7 +41,7 @@ def inner_func(*args, **kwargs): return inner_func -if DASK_EXPR_ENABLED: +if QUERY_PLANNING_ON: from .expr._collection import DataFrame, Index, Series groupby_agg = raise_not_implemented_error("groupby_agg") diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 3401bc494d6..561ad9524f3 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -492,8 +492,14 @@ def join(dfs): else: return cudf.concat(dfs) - dumps = partial(pickle.dumps, protocol=pickle.HIGHEST_PROTOCOL) - return partial(partd.Encode, dumps, pickle.loads, join) + def _dumps(*args, **kwargs): + dumps = partial(pickle.dumps, protocol=pickle.HIGHEST_PROTOCOL) + return dumps(*args, **kwargs) + + def _loads(*args, **kwargs): + return pickle.loads(*args, **kwargs) + + return partial(partd.Encode, _dumps, _loads, join) except ImportError: pass diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index 93652861018..8210b4bd0b5 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -2,16 +2,22 @@ from dask import config -DASK_EXPR_ENABLED = False -if config.get("dataframe.query-planning", False): - # Make sure custom expressions and collections are defined - try: - import dask_cudf.expr._collection - import dask_cudf.expr._expr +# Check if dask-dataframe is using dask-expr +QUERY_PLANNING_ON = config.get("dataframe.query-planning", False) - DASK_EXPR_ENABLED = True - except ImportError: - # Dask Expressions not installed. - # Dask DataFrame should have already thrown an error - # before we got here. - pass +# Register custom expressions and collections +try: + import dask_cudf.expr._collection + import dask_cudf.expr._expr + + DASK_EXPR_ENABLED = True # Dask-expr is installed +except ImportError as err: + DASK_EXPR_ENABLED = False # Dask-expr is not installed + if QUERY_PLANNING_ON: + # Dask *should* raise an error before this. + # However, we can still raise here to be certain. + raise RuntimeError( + "Failed to register the 'cudf' backend for dask-expr." + " Please make sure you have dask-expr installed.\n" + f"Error Message: {err}" + ) diff --git a/python/dask_cudf/dask_cudf/tests/utils.py b/python/dask_cudf/dask_cudf/tests/utils.py index ce0e046e26f..e838b8d63bc 100644 --- a/python/dask_cudf/dask_cudf/tests/utils.py +++ b/python/dask_cudf/dask_cudf/tests/utils.py @@ -8,7 +8,7 @@ import cudf -from dask_cudf.expr import DASK_EXPR_ENABLED +from dask_cudf.expr import QUERY_PLANNING_ON def _make_random_frame(nelem, npartitions=2, include_na=False): @@ -28,8 +28,8 @@ def _make_random_frame(nelem, npartitions=2, include_na=False): def skip_dask_expr(reason=_default_reason): - return pytest.mark.skipif(DASK_EXPR_ENABLED, reason=reason) + return pytest.mark.skipif(QUERY_PLANNING_ON, reason=reason) def xfail_dask_expr(reason=_default_reason): - return pytest.mark.xfail(DASK_EXPR_ENABLED, reason=reason) + return pytest.mark.xfail(QUERY_PLANNING_ON, reason=reason) From c75e3b69172f8ecb2a93fc5c6c6d9208c26980b3 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 5 Feb 2024 13:35:55 -0800 Subject: [PATCH 37/74] debugging groupby failures --- python/dask_cudf/dask_cudf/expr/_expr.py | 16 +++++ .../dask_cudf/dask_cudf/tests/test_groupby.py | 65 +++++++++---------- 2 files changed, 48 insertions(+), 33 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 0dd61045231..404d977bb2b 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -1,6 +1,7 @@ # Copyright (c) 2024, NVIDIA CORPORATION. from dask_expr._cumulative import CumulativeBlockwise, TakeLast +from dask_expr._shuffle import DiskShuffle ## ## Custom expression patching @@ -32,3 +33,18 @@ def _takelast(a, skipna=True): TakeLast.operation = staticmethod(_takelast) + + +def _shuffle_group(df, col, _filter, p): + from dask.dataframe.shuffle import ensure_cleanup_on_exception + + with ensure_cleanup_on_exception(p): + # import pdb; pdb.set_trace() + # _, part_offsets, part_keys, grouped_df = df.groupby(col)._grouped() + + g = df.groupby(col) + d = {i: g.get_group(i) for i in g.groups if i in _filter} + p.append(d, fsync=True) + + +DiskShuffle._shuffle_group = staticmethod(_shuffle_group) diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 10536c7c7ad..cba80fcbe03 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -12,10 +12,7 @@ import dask_cudf from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() +from dask_cudf.tests.utils import QUERY_PLANNING_ON, xfail_dask_expr def assert_cudf_groupby_layers(ddf): @@ -53,45 +50,42 @@ def pdf(request): @pytest.mark.parametrize("aggregation", OPTIMIZED_AGGS) @pytest.mark.parametrize("series", [False, True]) def test_groupby_basic(series, aggregation, pdf): + if QUERY_PLANNING_ON and aggregation == "collect": + pytest.skip("Dask-expr does not support 'collect' yet.") + gdf = cudf.DataFrame.from_pandas(pdf) - gdf_grouped = gdf.groupby("xx") - ddf_grouped = dask_cudf.from_cudf(gdf, npartitions=5).groupby("xx") + gdf_grouped = gdf.groupby("xx", dropna=True) + ddf_grouped = dask_cudf.from_cudf(gdf, npartitions=5).groupby( + "xx", dropna=True + ) if series: - gdf_grouped = gdf_grouped.xx - ddf_grouped = ddf_grouped.xx + gdf_grouped = gdf_grouped.x + ddf_grouped = ddf_grouped.x check_dtype = aggregation != "count" expect = getattr(gdf_grouped, aggregation)() actual = getattr(ddf_grouped, aggregation)() - assert_cudf_groupby_layers(actual) + if not QUERY_PLANNING_ON: + assert_cudf_groupby_layers(actual) dd.assert_eq(expect, actual, check_dtype=check_dtype) - expect = gdf_grouped.agg({"xx": aggregation}) - actual = ddf_grouped.agg({"xx": aggregation}) + if not series: + expect = gdf_grouped.agg({"x": aggregation}) + actual = ddf_grouped.agg({"x": aggregation}) - assert_cudf_groupby_layers(actual) + if not QUERY_PLANNING_ON: + assert_cudf_groupby_layers(actual) - dd.assert_eq(expect, actual, check_dtype=check_dtype) + dd.assert_eq(expect, actual, check_dtype=check_dtype) # TODO: explore adding support with `.agg()` @pytest.mark.parametrize("series", [True, False]) -@pytest.mark.parametrize( - "aggregation", - [ - "cumsum", - pytest.param( - "cumcount", - marks=pytest.mark.xfail( - reason="https://github.com/rapidsai/cudf/issues/13390" - ), - ), - ], -) +@pytest.mark.parametrize("aggregation", ["cumsum", "cumcount"]) def test_groupby_cumulative(aggregation, pdf, series): gdf = cudf.DataFrame.from_pandas(pdf) ddf = dask_cudf.from_cudf(gdf, npartitions=5) @@ -113,17 +107,20 @@ def test_groupby_cumulative(aggregation, pdf, series): @pytest.mark.parametrize( "func", [ + # See: https://github.com/rapidsai/cudf/issues/14957 lambda df, agg: df.groupby("xx").agg({"y": agg}), - lambda df, agg: df.groupby("xx").y.agg({"y": agg}), + # lambda df, agg: df.groupby("xx").y.agg({"y": agg}), lambda df, agg: df.groupby("xx").agg([agg]), - lambda df, agg: df.groupby("xx").y.agg([agg]), + # lambda df, agg: df.groupby("xx").y.agg([agg]), lambda df, agg: df.groupby("xx").agg(agg), - lambda df, agg: df.groupby("xx").y.agg(agg), + # lambda df, agg: df.groupby("xx").y.agg(agg), ], ) def test_groupby_agg(func, aggregation, pdf): - gdf = cudf.DataFrame.from_pandas(pdf) + if QUERY_PLANNING_ON and aggregation == "collect": + pytest.skip("Dask-expr does not support 'collect' yet.") + gdf = cudf.DataFrame.from_pandas(pdf) ddf = dask_cudf.from_cudf(gdf, npartitions=5) actual = func(ddf, aggregation) @@ -131,11 +128,12 @@ def test_groupby_agg(func, aggregation, pdf): check_dtype = aggregation != "count" - assert_cudf_groupby_layers(actual) + if not QUERY_PLANNING_ON: + assert_cudf_groupby_layers(actual) - # groupby.agg should add an explicit getitem layer - # to improve/enable column projection - assert hlg_layer(actual.dask, "getitem") + # groupby.agg should add an explicit getitem layer + # to improve/enable column projection + assert hlg_layer(actual.dask, "getitem") dd.assert_eq(expect, actual, check_names=False, check_dtype=check_dtype) @@ -797,6 +795,7 @@ def test_groupby_nested_dict(func): dd.assert_eq(a, b) +@xfail_dask_expr("https://github.com/rapidsai/cudf/issues/14957") @pytest.mark.parametrize( "func", [ From f417619149bccdff2f194131edd1d3782f68fcfc Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 5 Feb 2024 13:42:28 -0800 Subject: [PATCH 38/74] make sure series aggregate used the proper logic --- python/cudf/cudf/core/groupby/groupby.py | 2 ++ python/cudf/cudf/tests/groupby/test_agg.py | 12 +++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 1f08abdc7fc..78593f20421 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -2640,6 +2640,8 @@ def agg(self, func): return result + aggregate = agg + def apply(self, func, *args): result = super().apply(func, *args) diff --git a/python/cudf/cudf/tests/groupby/test_agg.py b/python/cudf/cudf/tests/groupby/test_agg.py index 7919ee4a9f1..d63da855011 100644 --- a/python/cudf/cudf/tests/groupby/test_agg.py +++ b/python/cudf/cudf/tests/groupby/test_agg.py @@ -1,4 +1,4 @@ -# Copyright (c) 2023, NVIDIA CORPORATION. +# Copyright (c) 2023-2024, NVIDIA CORPORATION. import numpy as np import pytest @@ -16,3 +16,13 @@ def test_agg_count_dtype(empty): df = df.iloc[:0] result = df.groupby("a").agg({"c": "count"}) assert result["c"].dtype == np.dtype("int64") + + +@pytest.mark.parametrize("attr", ["agg", "aggregate"]) +def test_series_agg(attr): + df = cudf.DataFrame({"a": [1, 2] * 2, "b": [0] * 4}) + pdf = df.to_pandas() + agg = getattr(df.groupby("a")["a"], attr)("count") + pd_agg = getattr(pdf.groupby(["a"])["a"], attr)("count") + + assert agg.ndim == pd_agg.ndim From 348523728398bfcfb22d547079e27e81b451c10c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 6 Feb 2024 08:13:07 -0800 Subject: [PATCH 39/74] further align groupby API and tests --- python/dask_cudf/dask_cudf/backends.py | 10 ++---- python/dask_cudf/dask_cudf/expr/_expr.py | 15 +++++--- .../dask_cudf/dask_cudf/tests/test_groupby.py | 34 ++++++++++++------- 3 files changed, 34 insertions(+), 25 deletions(-) diff --git a/python/dask_cudf/dask_cudf/backends.py b/python/dask_cudf/dask_cudf/backends.py index 561ad9524f3..3401bc494d6 100644 --- a/python/dask_cudf/dask_cudf/backends.py +++ b/python/dask_cudf/dask_cudf/backends.py @@ -492,14 +492,8 @@ def join(dfs): else: return cudf.concat(dfs) - def _dumps(*args, **kwargs): - dumps = partial(pickle.dumps, protocol=pickle.HIGHEST_PROTOCOL) - return dumps(*args, **kwargs) - - def _loads(*args, **kwargs): - return pickle.loads(*args, **kwargs) - - return partial(partd.Encode, _dumps, _loads, join) + dumps = partial(pickle.dumps, protocol=pickle.HIGHEST_PROTOCOL) + return partial(partd.Encode, dumps, pickle.loads, join) except ImportError: pass diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 404d977bb2b..4f0ba2a91fa 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -39,11 +39,18 @@ def _shuffle_group(df, col, _filter, p): from dask.dataframe.shuffle import ensure_cleanup_on_exception with ensure_cleanup_on_exception(p): - # import pdb; pdb.set_trace() - # _, part_offsets, part_keys, grouped_df = df.groupby(col)._grouped() - g = df.groupby(col) - d = {i: g.get_group(i) for i in g.groups if i in _filter} + if hasattr(g, "_grouped"): + # Avoid `get_group` for cudf data. + # See: https://github.com/rapidsai/cudf/issues/14955 + keys, part_offsets, _, grouped_df = df.groupby(col)._grouped() + d = { + k: grouped_df.iloc[part_offsets[i] : part_offsets[i + 1]] + for i, k in enumerate(keys.to_pandas()) + if k in _filter + } + else: + d = {i: g.get_group(i) for i in g.groups if i in _filter} p.append(d, fsync=True) diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index cba80fcbe03..19f3311787f 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -12,7 +12,11 @@ import dask_cudf from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized -from dask_cudf.tests.utils import QUERY_PLANNING_ON, xfail_dask_expr +from dask_cudf.tests.utils import ( + QUERY_PLANNING_ON, + skip_dask_expr, + xfail_dask_expr, +) def assert_cudf_groupby_layers(ddf): @@ -576,6 +580,7 @@ def test_groupby_categorical_key(): dd.assert_eq(expect, got) +@xfail_dask_expr("as_index not supported in dask-expr") @pytest.mark.parametrize("as_index", [True, False]) @pytest.mark.parametrize("split_out", ["use_dask_default", 1, 2]) @pytest.mark.parametrize("split_every", [False, 4]) @@ -664,6 +669,7 @@ def test_groupby_agg_params(npartitions, split_every, split_out, as_index): dd.assert_eq(gf, pf) +@skip_dask_expr("Callable arguments not supported by dask-expr") @pytest.mark.parametrize( "aggregations", [(sum, "sum"), (max, "max"), (min, "min")] ) @@ -752,6 +758,7 @@ def test_groupby_first_last(data, agg): ) +@xfail_dask_expr("Co-alignment check fails in dask-expr") def test_groupby_with_list_of_series(): df = cudf.DataFrame({"a": [1, 2, 3, 4, 5]}) gdf = dask_cudf.from_cudf(df, npartitions=2) @@ -766,6 +773,7 @@ def test_groupby_with_list_of_series(): ) +@xfail_dask_expr("Nested renamer not supported in dask-expr") @pytest.mark.parametrize( "func", [ @@ -795,7 +803,6 @@ def test_groupby_nested_dict(func): dd.assert_eq(a, b) -@xfail_dask_expr("https://github.com/rapidsai/cudf/issues/14957") @pytest.mark.parametrize( "func", [ @@ -819,12 +826,12 @@ def test_groupby_all_columns(func): ) ddf = dd.from_pandas(pdf, npartitions=5) - gddf = ddf.map_partitions(cudf.from_pandas) + gddf = ddf.to_backend("cudf") expect = func(ddf) actual = func(gddf) - dd.assert_eq(expect, actual) + dd.assert_eq(expect, actual, check_names=not QUERY_PLANNING_ON) def test_groupby_shuffle(): @@ -862,13 +869,14 @@ def test_groupby_shuffle(): got = gddf.groupby("a", sort=False).agg(spec, split_out=2) dd.assert_eq(expect, got.compute().sort_index()) - # Sorted aggregation fails with split_out>1 when shuffle is False - # (sort=True, split_out=2, shuffle_method=False) - with pytest.raises(ValueError): - gddf.groupby("a", sort=True).agg( - spec, shuffle_method=False, split_out=2 - ) + if not QUERY_PLANNING_ON: + # Sorted aggregation fails with split_out>1 when shuffle is False + # (sort=True, split_out=2, shuffle_method=False) + with pytest.raises(ValueError): + gddf.groupby("a", sort=True).agg( + spec, shuffle_method=False, split_out=2 + ) - # Check shuffle kwarg deprecation - with pytest.warns(match="'shuffle' keyword is deprecated"): - gddf.groupby("a", sort=True).agg(spec, shuffle=False) + # Check shuffle kwarg deprecation + with pytest.warns(match="'shuffle' keyword is deprecated"): + gddf.groupby("a", sort=True).agg(spec, shuffle=False) From 2046aa81701954d89adcfae8baccb2982f13c4c2 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 7 Feb 2024 17:21:12 -0800 Subject: [PATCH 40/74] opt groupby --- python/dask_cudf/dask_cudf/expr/_expr.py | 149 ++++++++++++++++++ python/dask_cudf/dask_cudf/expr/_groupby.py | 49 ++++++ python/dask_cudf/dask_cudf/groupby.py | 11 +- .../dask_cudf/dask_cudf/tests/test_groupby.py | 10 +- 4 files changed, 209 insertions(+), 10 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 4f0ba2a91fa..cfcf8f027d0 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -1,8 +1,22 @@ # Copyright (c) 2024, NVIDIA CORPORATION. +import functools + from dask_expr._cumulative import CumulativeBlockwise, TakeLast +from dask_expr._groupby import ( + DecomposableGroupbyAggregation, + GroupbyAggregation, +) from dask_expr._shuffle import DiskShuffle +from dask.dataframe.core import _concat + +from dask_cudf.groupby import ( + _finalize_gb_agg, + _groupby_partition_agg, + _tree_node_agg, +) + ## ## Custom expression patching ## @@ -55,3 +69,138 @@ def _shuffle_group(df, col, _filter, p): DiskShuffle._shuffle_group = staticmethod(_shuffle_group) + + +class CudfGroupbyAggregation(GroupbyAggregation): + @functools.cached_property + def _meta(self): + from dask.dataframe.dispatch import make_meta, meta_nonempty + + meta = meta_nonempty(self.frame._meta) + meta = meta.groupby( + self._by_meta, + # **_as_dict("observed", self.observed), + # **_as_dict("dropna", self.dropna), + ) + if self._slice is not None: + meta = meta[self._slice] + meta = meta.aggregate(self.arg) + # import pdb; pdb.set_trace() + return make_meta(meta) + + def _lower(self): + return DecomposableCudfGroupbyAggregation( + self.frame, + self.arg, + self.observed, + self.dropna, + self.split_every, + self.split_out, + self.sort, + self.shuffle_method, + self._slice, + *self.by, + ) + + +class DecomposableCudfGroupbyAggregation(DecomposableGroupbyAggregation): + sep = "___" + + @property + def shuffle_by_index(self): + return False # We always group by column(s) + + @functools.cached_property + def spec_info(self): + if isinstance(self.arg, (dict, list)): + aggs = self.arg.copy() + else: + aggs = self.arg + + if self._slice and not isinstance(aggs, dict): + aggs = {self._slice: aggs} + + gb_cols = self._by_columns + if isinstance(gb_cols, str): + gb_cols = [gb_cols] + columns = [c for c in self.frame.columns if c not in gb_cols] + if not isinstance(aggs, dict): + aggs = {col: aggs for col in columns} + + # Assert if our output will have a MultiIndex; this will be the case if + # any value in the `aggs` dict is not a string (i.e. multiple/named + # aggregations per column) + str_cols_out = True + aggs_renames = {} + for col in aggs: + if isinstance(aggs[col], str) or callable(aggs[col]): + aggs[col] = [aggs[col]] + elif isinstance(aggs[col], dict): + str_cols_out = False + col_aggs = [] + for k, v in aggs[col].items(): + aggs_renames[col, v] = k + col_aggs.append(v) + aggs[col] = col_aggs + else: + str_cols_out = False + if col in gb_cols: + columns.append(col) + + return { + "aggs": aggs, + "columns": columns, + "str_cols_out": str_cols_out, + "aggs_renames": aggs_renames, + } + + @classmethod + def chunk(cls, df, *by, **kwargs): + return _groupby_partition_agg(df, **kwargs) + + @classmethod + def combine(cls, inputs, **kwargs): + return _tree_node_agg(_concat(inputs), **kwargs) + + @classmethod + def aggregate(cls, inputs, **kwargs): + return _finalize_gb_agg(_concat(inputs), **kwargs) + + @property + def chunk_kwargs(self) -> dict: + dropna = True if self.dropna is None else self.dropna + return { + "gb_cols": self._by_columns, + "aggs": self.spec_info["aggs"], + "columns": self.spec_info["columns"], + "dropna": dropna, + "sort": self.sort, + "sep": self.sep, + } + + @property + def combine_kwargs(self) -> dict: + dropna = True if self.dropna is None else self.dropna + return { + "gb_cols": self._by_columns, + "dropna": dropna, + "sort": self.sort, + "sep": self.sep, + } + + @property + def aggregate_kwargs(self) -> dict: + dropna = True if self.dropna is None else self.dropna + final_columns = self._slice or self._meta.columns + return { + "gb_cols": self._by_columns, + "aggs": self.spec_info["aggs"], + "columns": self.spec_info["columns"], + "final_columns": final_columns, + "as_index": True, + "dropna": dropna, + "sort": self.sort, + "sep": self.sep, + "str_cols_out": self.spec_info["str_cols_out"], + "aggs_renames": self.spec_info["aggs_renames"], + } diff --git a/python/dask_cudf/dask_cudf/expr/_groupby.py b/python/dask_cudf/dask_cudf/expr/_groupby.py index 8e67932fd3c..3908195489f 100644 --- a/python/dask_cudf/dask_cudf/expr/_groupby.py +++ b/python/dask_cudf/dask_cudf/expr/_groupby.py @@ -1,11 +1,15 @@ # Copyright (c) 2024, NVIDIA CORPORATION. +from dask_expr._collection import new_collection from dask_expr._groupby import ( GroupBy as DXGroupBy, + GroupbyAggregation, SeriesGroupBy as DXSeriesGroupBy, ) from dask_expr._util import is_scalar +from dask_cudf.expr._expr import CudfGroupbyAggregation + ## ## Custom groupby classes ## @@ -37,8 +41,53 @@ def __getitem__(self, key): ) return g + def aggregate(self, *args, **kwargs): + return _aggregation(self, *args, **kwargs) + class SeriesGroupBy(DXSeriesGroupBy): def __init__(self, *args, observed=None, **kwargs): observed = observed if observed is not None else True super().__init__(*args, observed=observed, **kwargs) + + def aggregate(self, *args, **kwargs): + return _aggregation(self, *args, **kwargs) + + +def _aggregation( + gb, arg=None, split_every=8, split_out=1, shuffle_method=None, **kwargs +): + from dask_cudf.groupby import ( + OPTIMIZED_AGGS, + _aggs_optimized, + _redirect_aggs, + ) + + if arg is None: + raise NotImplementedError("arg=None not supported") + + if arg == "size": + return gb.size() + + arg = _redirect_aggs(arg) + if _aggs_optimized(arg, OPTIMIZED_AGGS) and hasattr( + gb.obj._meta, "to_pandas" + ): + cls = CudfGroupbyAggregation + else: + cls = GroupbyAggregation + + return new_collection( + cls( + gb.obj.expr, + arg, + gb.observed, + gb.dropna, + split_every, + split_out, + gb.sort, + shuffle_method, + gb._slice, + *gb.by, + ) + ) diff --git a/python/dask_cudf/dask_cudf/groupby.py b/python/dask_cudf/dask_cudf/groupby.py index 43ad4f0fee3..04c680eede3 100644 --- a/python/dask_cudf/dask_cudf/groupby.py +++ b/python/dask_cudf/dask_cudf/groupby.py @@ -777,6 +777,7 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): """ agg_dict = {} + flatten = [] for col in df.columns: if col in gb_cols: continue @@ -785,6 +786,8 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): agg_dict[col] = ["sum"] elif agg in OPTIMIZED_AGGS: agg_dict[col] = [agg] + if agg == "collect": + flatten.append(col) else: raise ValueError(f"Unexpected aggregation: {agg}") @@ -798,6 +801,11 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): for name in gb.columns ] gb.columns = output_columns + + # Avoid nested lists + for col in flatten: + gb[col] = gb[col].list.concat() + # Return with deterministic column ordering return gb[sorted(output_columns)] @@ -873,9 +881,6 @@ def _finalize_gb_agg( gb.drop(columns=[sum_name], inplace=True) if "count" not in agg_list: gb.drop(columns=[count_name], inplace=True) - if "collect" in agg_list: - collect_name = _make_name((col, "collect"), sep=sep) - gb[collect_name] = gb[collect_name].list.concat() # Ensure sorted keys if `sort=True` if sort: diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 19f3311787f..4e18c3fc3c6 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -111,19 +111,15 @@ def test_groupby_cumulative(aggregation, pdf, series): @pytest.mark.parametrize( "func", [ - # See: https://github.com/rapidsai/cudf/issues/14957 lambda df, agg: df.groupby("xx").agg({"y": agg}), - # lambda df, agg: df.groupby("xx").y.agg({"y": agg}), + lambda df, agg: df.groupby("xx").y.agg({"y": agg}), lambda df, agg: df.groupby("xx").agg([agg]), - # lambda df, agg: df.groupby("xx").y.agg([agg]), + lambda df, agg: df.groupby("xx").y.agg([agg]), lambda df, agg: df.groupby("xx").agg(agg), - # lambda df, agg: df.groupby("xx").y.agg(agg), + lambda df, agg: df.groupby("xx").y.agg(agg), ], ) def test_groupby_agg(func, aggregation, pdf): - if QUERY_PLANNING_ON and aggregation == "collect": - pytest.skip("Dask-expr does not support 'collect' yet.") - gdf = cudf.DataFrame.from_pandas(pdf) ddf = dask_cudf.from_cudf(gdf, npartitions=5) From dd9faf7c7a67cbe023e0b3298d562c0a87704d3d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 8 Feb 2024 07:33:25 -0800 Subject: [PATCH 41/74] cleanup --- python/dask_cudf/dask_cudf/expr/_expr.py | 21 +++---------------- python/dask_cudf/dask_cudf/expr/_groupby.py | 13 +++++++++--- .../dask_cudf/dask_cudf/tests/test_groupby.py | 7 +------ 3 files changed, 14 insertions(+), 27 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index cfcf8f027d0..db931889bc9 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -72,22 +72,6 @@ def _shuffle_group(df, col, _filter, p): class CudfGroupbyAggregation(GroupbyAggregation): - @functools.cached_property - def _meta(self): - from dask.dataframe.dispatch import make_meta, meta_nonempty - - meta = meta_nonempty(self.frame._meta) - meta = meta.groupby( - self._by_meta, - # **_as_dict("observed", self.observed), - # **_as_dict("dropna", self.dropna), - ) - if self._slice is not None: - meta = meta[self._slice] - meta = meta.aggregate(self.arg) - # import pdb; pdb.set_trace() - return make_meta(meta) - def _lower(self): return DecomposableCudfGroupbyAggregation( self.frame, @@ -108,7 +92,7 @@ class DecomposableCudfGroupbyAggregation(DecomposableGroupbyAggregation): @property def shuffle_by_index(self): - return False # We always group by column(s) + return False # We always group by column(s) in dask-cudf @functools.cached_property def spec_info(self): @@ -156,6 +140,7 @@ def spec_info(self): @classmethod def chunk(cls, df, *by, **kwargs): + # `by` columns are already specified in kwargs return _groupby_partition_agg(df, **kwargs) @classmethod @@ -197,7 +182,7 @@ def aggregate_kwargs(self) -> dict: "aggs": self.spec_info["aggs"], "columns": self.spec_info["columns"], "final_columns": final_columns, - "as_index": True, + "as_index": True, # False not supported in dask-expr "dropna": dropna, "sort": self.sort, "sep": self.sep, diff --git a/python/dask_cudf/dask_cudf/expr/_groupby.py b/python/dask_cudf/dask_cudf/expr/_groupby.py index 3908195489f..384498b0e33 100644 --- a/python/dask_cudf/dask_cudf/expr/_groupby.py +++ b/python/dask_cudf/dask_cudf/expr/_groupby.py @@ -55,7 +55,12 @@ def aggregate(self, *args, **kwargs): def _aggregation( - gb, arg=None, split_every=8, split_out=1, shuffle_method=None, **kwargs + gb, + arg=None, + split_every=8, + split_out=1, + shuffle_method=None, + _use_optimized=True, ): from dask_cudf.groupby import ( OPTIMIZED_AGGS, @@ -70,8 +75,10 @@ def _aggregation( return gb.size() arg = _redirect_aggs(arg) - if _aggs_optimized(arg, OPTIMIZED_AGGS) and hasattr( - gb.obj._meta, "to_pandas" + if ( + _use_optimized + and _aggs_optimized(arg, OPTIMIZED_AGGS) + and hasattr(gb.obj._meta, "to_pandas") ): cls = CudfGroupbyAggregation else: diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 4e18c3fc3c6..35c247bc870 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -12,11 +12,7 @@ import dask_cudf from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized -from dask_cudf.tests.utils import ( - QUERY_PLANNING_ON, - skip_dask_expr, - xfail_dask_expr, -) +from dask_cudf.tests.utils import QUERY_PLANNING_ON, xfail_dask_expr def assert_cudf_groupby_layers(ddf): @@ -665,7 +661,6 @@ def test_groupby_agg_params(npartitions, split_every, split_out, as_index): dd.assert_eq(gf, pf) -@skip_dask_expr("Callable arguments not supported by dask-expr") @pytest.mark.parametrize( "aggregations", [(sum, "sum"), (max, "max"), (min, "min")] ) From 146fd7d1c64c696f3305a19ab4ed467fa4123059 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 8 Feb 2024 17:11:52 -0800 Subject: [PATCH 42/74] only use optimized single-agg code path for collect --- python/dask_cudf/dask_cudf/expr/_groupby.py | 42 +++++++++++++++---- .../dask_cudf/dask_cudf/tests/test_groupby.py | 3 -- 2 files changed, 33 insertions(+), 12 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_groupby.py b/python/dask_cudf/dask_cudf/expr/_groupby.py index 384498b0e33..0e9bb6cecfa 100644 --- a/python/dask_cudf/dask_cudf/expr/_groupby.py +++ b/python/dask_cudf/dask_cudf/expr/_groupby.py @@ -1,5 +1,7 @@ # Copyright (c) 2024, NVIDIA CORPORATION. +from functools import partial + from dask_expr._collection import new_collection from dask_expr._groupby import ( GroupBy as DXGroupBy, @@ -9,6 +11,7 @@ from dask_expr._util import is_scalar from dask_cudf.expr._expr import CudfGroupbyAggregation +from dask_cudf.groupby import OPTIMIZED_AGGS ## ## Custom groupby classes @@ -19,6 +22,11 @@ class GroupBy(DXGroupBy): def __init__(self, *args, observed=None, **kwargs): observed = observed if observed is not None else True super().__init__(*args, observed=observed, **kwargs) + # Add optimized aggregation code paths + for agg in OPTIMIZED_AGGS: + setattr(self, agg, partial(single_agg, self, agg)) + setattr(self, "agg", partial(groupby_agg, self)) + setattr(self, "aggregate", partial(groupby_agg, self)) def __getitem__(self, key): if is_scalar(key): @@ -41,26 +49,42 @@ def __getitem__(self, key): ) return g - def aggregate(self, *args, **kwargs): - return _aggregation(self, *args, **kwargs) - class SeriesGroupBy(DXSeriesGroupBy): def __init__(self, *args, observed=None, **kwargs): observed = observed if observed is not None else True super().__init__(*args, observed=observed, **kwargs) - - def aggregate(self, *args, **kwargs): - return _aggregation(self, *args, **kwargs) + # Add optimized aggregation code paths + for agg in OPTIMIZED_AGGS: + setattr(self, agg, partial(single_agg, self, agg)) + setattr(self, "agg", partial(groupby_agg, self)) + setattr(self, "aggregate", partial(groupby_agg, self)) + + +def single_agg(gb, agg_name, **kwargs): + _optimized = kwargs.pop("_optimized", agg_name == "collect") + if _optimized and hasattr(gb.obj._meta, "to_pandas"): + if gb._slice is None: + if isinstance(gb.by, list): + agg = {c: agg_name for c in gb.obj.columns if c not in gb.by} + else: + agg = {c: agg_name for c in gb.obj.columns if c != gb.by} + else: + agg = {gb._slice: agg_name} + + return groupby_agg(gb, agg, **kwargs) + else: + # Fall back to dask-expr code path + return getattr(super(type(gb), gb), agg_name)(**kwargs) -def _aggregation( +def groupby_agg( gb, arg=None, split_every=8, split_out=1, shuffle_method=None, - _use_optimized=True, + _optimized=True, ): from dask_cudf.groupby import ( OPTIMIZED_AGGS, @@ -76,7 +100,7 @@ def _aggregation( arg = _redirect_aggs(arg) if ( - _use_optimized + _optimized and _aggs_optimized(arg, OPTIMIZED_AGGS) and hasattr(gb.obj._meta, "to_pandas") ): diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 35c247bc870..aa9cca8d737 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -50,9 +50,6 @@ def pdf(request): @pytest.mark.parametrize("aggregation", OPTIMIZED_AGGS) @pytest.mark.parametrize("series", [False, True]) def test_groupby_basic(series, aggregation, pdf): - if QUERY_PLANNING_ON and aggregation == "collect": - pytest.skip("Dask-expr does not support 'collect' yet.") - gdf = cudf.DataFrame.from_pandas(pdf) gdf_grouped = gdf.groupby("xx", dropna=True) ddf_grouped = dask_cudf.from_cudf(gdf, npartitions=5).groupby( From 6de8946ad3a9f767aece3a6ecd93a315e01afdcf Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 9 Feb 2024 07:35:17 -0800 Subject: [PATCH 43/74] expand test coverage --- .../dask_cudf/dask_cudf/expr/_collection.py | 32 +++++++++++++++++-- python/dask_cudf/dask_cudf/expr/_expr.py | 14 ++++++++ .../dask_cudf/tests/test_delayed_io.py | 4 --- .../dask_cudf/tests/test_dispatch.py | 5 --- .../dask_cudf/tests/test_distributed.py | 5 --- python/dask_cudf/dask_cudf/tests/test_join.py | 9 ++---- .../dask_cudf/tests/test_reductions.py | 6 +--- 7 files changed, 47 insertions(+), 28 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 33585824e11..9e842819a0b 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -7,8 +7,11 @@ Series as DXSeries, get_collection_type, ) +from dask_expr._collection import new_collection +from dask_expr._util import _raise_if_object_series from dask import config +from dask.dataframe.core import is_dataframe_like import cudf @@ -17,7 +20,32 @@ ## -class DataFrame(DXDataFrame): +class CudfMixin: + def var( + self, + axis=0, + skipna=True, + ddof=1, + numeric_only=False, + split_every=False, + **kwargs, + ): + _raise_if_object_series(self, "var") + axis = self._validate_axis(axis) + self._meta.var(axis=axis, skipna=skipna, numeric_only=numeric_only) + frame = self + if is_dataframe_like(self._meta) and numeric_only: + # Convert to pandas - cudf does something weird here + index = self._meta.to_pandas().var(numeric_only=True).index + frame = frame[list(index)] + return new_collection( + frame.expr.var( + axis, skipna, ddof, numeric_only, split_every=split_every + ) + ) + + +class DataFrame(CudfMixin, DXDataFrame): @classmethod def from_dict(cls, *args, **kwargs): with config.set({"dataframe.backend": "cudf"}): @@ -50,7 +78,7 @@ def groupby( ) -class Series(DXSeries): +class Series(CudfMixin, DXSeries): def groupby(self, by, **kwargs): from dask_cudf.expr._groupby import SeriesGroupBy diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index db931889bc9..7da7419c3ba 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -7,6 +7,7 @@ DecomposableGroupbyAggregation, GroupbyAggregation, ) +from dask_expr._reductions import Var from dask_expr._shuffle import DiskShuffle from dask.dataframe.core import _concat @@ -49,6 +50,19 @@ def _takelast(a, skipna=True): TakeLast.operation = staticmethod(_takelast) +_dx_reduction_aggregate = Var.reduction_aggregate + + +def _reduction_aggregate(*args, **kwargs): + result = _dx_reduction_aggregate(*args, **kwargs) + if result.ndim == 0: + return result.tolist() + return result + + +Var.reduction_aggregate = staticmethod(_reduction_aggregate) + + def _shuffle_group(df, col, _filter, p): from dask.dataframe.shuffle import ensure_cleanup_on_exception diff --git a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py index 174b285724c..e6fb58ad6df 100644 --- a/python/dask_cudf/dask_cudf/tests/test_delayed_io.py +++ b/python/dask_cudf/dask_cudf/tests/test_delayed_io.py @@ -13,10 +13,6 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() @delayed diff --git a/python/dask_cudf/dask_cudf/tests/test_dispatch.py b/python/dask_cudf/dask_cudf/tests/test_dispatch.py index 8decf20d595..76703206726 100644 --- a/python/dask_cudf/dask_cudf/tests/test_dispatch.py +++ b/python/dask_cudf/dask_cudf/tests/test_dispatch.py @@ -12,11 +12,6 @@ import cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() - def test_is_categorical_dispatch(): assert is_categorical_dtype(pd.CategoricalDtype([1, 2, 3])) diff --git a/python/dask_cudf/dask_cudf/tests/test_distributed.py b/python/dask_cudf/dask_cudf/tests/test_distributed.py index 8413abb103f..39eadb45c91 100644 --- a/python/dask_cudf/dask_cudf/tests/test_distributed.py +++ b/python/dask_cudf/dask_cudf/tests/test_distributed.py @@ -12,11 +12,6 @@ from cudf.testing._utils import assert_eq import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() - dask_cuda = pytest.importorskip("dask_cuda") diff --git a/python/dask_cudf/dask_cudf/tests/test_join.py b/python/dask_cudf/dask_cudf/tests/test_join.py index cec5e55fe19..42ecc130298 100644 --- a/python/dask_cudf/dask_cudf/tests/test_join.py +++ b/python/dask_cudf/dask_cudf/tests/test_join.py @@ -11,11 +11,6 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() - param_nrows = [5, 10, 50, 100] @@ -168,7 +163,7 @@ def test_merge_left( } ) - expect = left.merge(right, on=("x", "y"), how=how) + expect = left.merge(right, on=["x", "y"], how=how) def normalize(df): return ( @@ -181,7 +176,7 @@ def normalize(df): left = dask_cudf.from_cudf(left, chunksize=chunksize) right = dask_cudf.from_cudf(right, chunksize=chunksize) - result = left.merge(right, on=("x", "y"), how=how).compute( + result = left.merge(right, on=["x", "y"], how=how).compute( scheduler="single-threaded" ) diff --git a/python/dask_cudf/dask_cudf/tests/test_reductions.py b/python/dask_cudf/dask_cudf/tests/test_reductions.py index 81ca32eb30c..c3056f2607c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_reductions.py +++ b/python/dask_cudf/dask_cudf/tests/test_reductions.py @@ -10,10 +10,6 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() def _make_random_frame(nelem, npartitions=2): @@ -72,7 +68,7 @@ def test_series_reduce(reducer): ) def test_rowwise_reductions(data, op): gddf = dask_cudf.from_cudf(data, npartitions=10) - pddf = gddf.to_dask_dataframe() + pddf = gddf.to_backend("pandas") with dask.config.set({"dataframe.convert-string": False}): if op in ("var", "std"): From 65dd6adaa72e97ad9be2c67efda7342f2dcf69df Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 9 Feb 2024 07:50:40 -0800 Subject: [PATCH 44/74] add sort testing (limited support for now) --- python/dask_cudf/dask_cudf/tests/test_sort.py | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 030d122594f..ab75ad39817 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -10,14 +10,26 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() +from dask_cudf.tests.utils import xfail_dask_expr @pytest.mark.parametrize("ascending", [True, False]) -@pytest.mark.parametrize("by", ["a", "b", "c", "d", ["a", "b"], ["c", "d"]]) +@pytest.mark.parametrize( + "by", + [ + "a", + "b", + "c", + pytest.param( + "d", + marks=xfail_dask_expr( + "Dask-expr fails to sort by categorical column." + ), + ), + ["a", "b"], + ["c", "d"], + ], +) @pytest.mark.parametrize("nelem", [10, 500]) @pytest.mark.parametrize("nparts", [1, 10]) def test_sort_values(nelem, nparts, by, ascending): @@ -60,6 +72,7 @@ def test_sort_repartition(): dd.assert_eq(len(new_ddf), len(ddf)) +@xfail_dask_expr("dask-expr code path fails with nulls") @pytest.mark.parametrize("na_position", ["first", "last"]) @pytest.mark.parametrize("ascending", [True, False]) @pytest.mark.parametrize("by", ["a", "b", ["a", "b"]]) From 1a8eabe2918bbe618f6345a8b6e47f60485b46d9 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 9 Feb 2024 11:01:38 -0800 Subject: [PATCH 45/74] further expand test coverage --- python/dask_cudf/dask_cudf/tests/test_applymap.py | 9 +++------ python/dask_cudf/dask_cudf/tests/test_binops.py | 5 +---- python/dask_cudf/dask_cudf/tests/test_onehot.py | 4 ++-- python/dask_cudf/dask_cudf/tests/test_sort.py | 4 ---- 4 files changed, 6 insertions(+), 16 deletions(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_applymap.py b/python/dask_cudf/dask_cudf/tests/test_applymap.py index 521f273c812..d84235481c3 100644 --- a/python/dask_cudf/dask_cudf/tests/test_applymap.py +++ b/python/dask_cudf/dask_cudf/tests/test_applymap.py @@ -5,10 +5,7 @@ from dask import dataframe as dd -from dask_cudf.tests.utils import _make_random_frame, skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() +from dask_cudf.tests.utils import _make_random_frame @pytest.mark.parametrize( @@ -27,6 +24,6 @@ def test_applymap_basic(func, has_na): dpdf = dd.from_pandas(pdf, npartitions=dgdf.npartitions) - expect = dpdf.applymap(func) - got = dgdf.applymap(func) + expect = dpdf.map(func) + got = dgdf.map(func) dd.assert_eq(expect, got, check_dtype=False) diff --git a/python/dask_cudf/dask_cudf/tests/test_binops.py b/python/dask_cudf/dask_cudf/tests/test_binops.py index 276fb1e6203..770f7368d56 100644 --- a/python/dask_cudf/dask_cudf/tests/test_binops.py +++ b/python/dask_cudf/dask_cudf/tests/test_binops.py @@ -10,10 +10,7 @@ import cudf -from dask_cudf.tests.utils import _make_random_frame, skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() +from dask_cudf.tests.utils import _make_random_frame def _make_empty_frame(npartitions=2): diff --git a/python/dask_cudf/dask_cudf/tests/test_onehot.py b/python/dask_cudf/dask_cudf/tests/test_onehot.py index 3d00e56d07e..96646f85f74 100644 --- a/python/dask_cudf/dask_cudf/tests/test_onehot.py +++ b/python/dask_cudf/dask_cudf/tests/test_onehot.py @@ -8,10 +8,10 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr +from dask_cudf.tests.utils import xfail_dask_expr # No dask-expr support -pytestmark = skip_dask_expr() +pytestmark = xfail_dask_expr("limited get_dummy support in dask-expr + cudf") def test_get_dummies_cat(): diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index ab75ad39817..9184ad996ad 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -134,10 +134,6 @@ def test_sort_values_empty_string(by): def test_disk_shuffle(): - try: - from dask.dataframe.dispatch import partd_encode_dispatch # noqa: F401 - except ImportError: - pytest.skip("need a version of dask that has partd_encode_dispatch") df = cudf.DataFrame({"a": [1, 2, 3] * 20, "b": [4, 5, 6, 7] * 15}) ddf = dd.from_pandas(df, npartitions=4) got = dd.DataFrame.shuffle(ddf, "a", shuffle_method="disk") From e68d1070acfe556d8ee9f675236c107d53588a78 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 12 Feb 2024 07:43:05 -0800 Subject: [PATCH 46/74] fix normalization --- python/cudf/cudf/core/frame.py | 4 ++-- python/cudf/cudf/core/index.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/cudf/cudf/core/frame.py b/python/cudf/cudf/core/frame.py index 79005193b4e..32a39765208 100644 --- a/python/cudf/cudf/core/frame.py +++ b/python/cudf/cudf/core/frame.py @@ -1957,8 +1957,8 @@ def __dask_tokenize__(self): from dask.base import normalize_token return [ - type(self), - normalize_token(self._dtypes), + str(type(self)), + str(self._dtypes), normalize_token(self.to_pandas()), ] diff --git a/python/cudf/cudf/core/index.py b/python/cudf/cudf/core/index.py index ea8ba154922..af00a5d524e 100644 --- a/python/cudf/cudf/core/index.py +++ b/python/cudf/cudf/core/index.py @@ -960,7 +960,7 @@ def __abs__(self): @_warn_no_dask_cudf def __dask_tokenize__(self): - return (type(self), self.start, self.stop, self.step) + return (str(type(self)), self.start, self.stop, self.step) class Index(SingleColumnFrame, BaseIndex, metaclass=IndexMeta): From faa54fc256a15d95fb53cb41c80153a288d72c2d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 12 Feb 2024 08:37:23 -0800 Subject: [PATCH 47/74] add test coverage for accessorts, but some tun-explained test failures remain --- python/dask_cudf/dask_cudf/expr/_collection.py | 14 ++++++++++++++ python/dask_cudf/dask_cudf/tests/test_accessor.py | 13 ++++++------- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 9e842819a0b..4f7938fc161 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -1,5 +1,7 @@ # Copyright (c) 2024, NVIDIA CORPORATION. +from functools import cached_property + from dask_expr import ( DataFrame as DXDataFrame, FrameBase, @@ -84,6 +86,18 @@ def groupby(self, by, **kwargs): return SeriesGroupBy(self, by, **kwargs) + @cached_property + def list(self): + from dask_cudf.accessors import ListMethods + + return ListMethods(self) + + @cached_property + def struct(self): + from dask_cudf.accessors import StructMethods + + return StructMethods(self) + class Index(DXIndex): pass # Same as pandas (for now) diff --git a/python/dask_cudf/dask_cudf/tests/test_accessor.py b/python/dask_cudf/dask_cudf/tests/test_accessor.py index 24139a60b50..ebb8e4be187 100644 --- a/python/dask_cudf/dask_cudf/tests/test_accessor.py +++ b/python/dask_cudf/dask_cudf/tests/test_accessor.py @@ -12,11 +12,7 @@ from cudf.testing._utils import assert_eq, does_not_raise import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() - +from dask_cudf.tests.utils import xfail_dask_expr ############################################################################# # Datetime Accessor # @@ -115,6 +111,7 @@ def test_categorical_accessor_initialization2(data): dsr.cat +@xfail_dask_expr("TODO: Unexplained dask-expr failure") @pytest.mark.parametrize("data", [data_cat_1()]) def test_categorical_basic(data): cat = data.copy() @@ -206,10 +203,11 @@ def test_categorical_compare_unordered(data): dsr < dsr +@xfail_dask_expr("TODO: Unexplained dask-expr failure") @pytest.mark.parametrize("data", [data_cat_3()]) def test_categorical_compare_ordered(data): - cat1 = data[0] - cat2 = data[1] + cat1 = data[0].copy() + cat2 = data[1].copy() pdsr1 = pd.Series(cat1) pdsr2 = pd.Series(cat2) sr1 = Series(cat1) @@ -276,6 +274,7 @@ def test_categorical_categories(): ) +@xfail_dask_expr("TODO: Unexplained dask-expr failure") def test_categorical_as_known(): df = dask_cudf.from_cudf(DataFrame({"col_1": [0, 1, 2, 3]}), npartitions=2) df["col_1"] = df["col_1"].astype("category") From 569d01fea42f4f92d97ae7a610de8349990803dd Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 12 Feb 2024 09:11:30 -0800 Subject: [PATCH 48/74] allow test_struct.py tests --- python/dask_cudf/dask_cudf/tests/test_struct.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_struct.py b/python/dask_cudf/dask_cudf/tests/test_struct.py index c0d1fe04072..ad861193674 100644 --- a/python/dask_cudf/dask_cudf/tests/test_struct.py +++ b/python/dask_cudf/dask_cudf/tests/test_struct.py @@ -5,10 +5,6 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr - -# No dask-expr support -pytestmark = skip_dask_expr() @pytest.mark.parametrize( From f7cd39390845329a5c101682a240474dd7035df2 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 14 Feb 2024 07:30:20 -0800 Subject: [PATCH 49/74] xfail test that needs newer version of dask-expr --- python/dask_cudf/dask_cudf/tests/test_groupby.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index aa9cca8d737..05537435afb 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -696,6 +696,7 @@ def test_is_supported(arg, supported): assert _aggs_optimized(arg, OPTIMIZED_AGGS) is supported +@xfail_dask_expr("Fails on older versions of dask-expr") def test_groupby_unique_lists(): df = pd.DataFrame({"a": [0, 0, 0, 1, 1, 1], "b": [10, 10, 10, 7, 8, 9]}) ddf = dd.from_pandas(df, 2) From 6844647a0cefe37a810ef7a6825c5d143a17b91c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 22 Feb 2024 11:35:34 -0800 Subject: [PATCH 50/74] simnplify dask-expr dependency --- conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +- conda/environments/all_cuda-122_arch-x86_64.yaml | 2 +- dependencies.yaml | 13 +------------ python/dask_cudf/pyproject.toml | 2 +- 4 files changed, 4 insertions(+), 15 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index 37a2757c27e..2d662818421 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -29,6 +29,7 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* +- dask-expr==0.4.0 - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 @@ -102,6 +103,5 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: - - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-118_arch-x86_64 diff --git a/conda/environments/all_cuda-122_arch-x86_64.yaml b/conda/environments/all_cuda-122_arch-x86_64.yaml index 546e2485cef..89e1a948224 100644 --- a/conda/environments/all_cuda-122_arch-x86_64.yaml +++ b/conda/environments/all_cuda-122_arch-x86_64.yaml @@ -30,6 +30,7 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* +- dask-expr==0.4.0 - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 @@ -100,6 +101,5 @@ dependencies: - typing_extensions>=4.0.0 - zlib>=1.2.13 - pip: - - git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - git+https://github.com/python-streamz/streamz.git@master name: all_cuda-122_arch-x86_64 diff --git a/dependencies.yaml b/dependencies.yaml index 9e0edeca558..fefaf01be53 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -649,18 +649,7 @@ dependencies: packages: - dask-cuda==24.4.* - *numba - - output_types: conda - packages: - - pip - - pip: - # This should eventually move to rapids-dask-dependency - - &dask_expr_tip git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 - - output_types: requirements - packages: - - *dask_expr_tip - - output_types: pyproject - packages: - - dask-expr@git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20 + - dask-expr==0.4.0 depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 24954bc7cf6..976e8b36d64 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,7 +44,7 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", - "dask-expr@git+https://github.com/dask-contrib/dask-expr.git@b588a9e15e90e0567061664ffc01374786686e20", + "dask-expr==0.4.0", "numba>=0.57", "pytest-cov", "pytest-xdist", From 18c6dbfc3eaaebd12c04b606b0fa1afb00c05e43 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 22 Feb 2024 13:37:00 -0800 Subject: [PATCH 51/74] remove extra variable --- python/dask_cudf/dask_cudf/expr/__init__.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index 8210b4bd0b5..867ea1be84e 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -10,9 +10,7 @@ import dask_cudf.expr._collection import dask_cudf.expr._expr - DASK_EXPR_ENABLED = True # Dask-expr is installed except ImportError as err: - DASK_EXPR_ENABLED = False # Dask-expr is not installed if QUERY_PLANNING_ON: # Dask *should* raise an error before this. # However, we can still raise here to be certain. From 8c601543ce9a7ad982ebc880c30b2ce45579b987 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 22 Feb 2024 16:07:11 -0800 Subject: [PATCH 52/74] handle newer versions of dask-expr --- python/dask_cudf/dask_cudf/__init__.py | 8 +++++++- python/dask_cudf/dask_cudf/tests/test_core.py | 3 ++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index 95c44421dfb..c66e85ed2af 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -1,7 +1,13 @@ # Copyright (c) 2018-2024, NVIDIA CORPORATION. -import dask.dataframe as dd from dask import config + +# For dask>2024.2.0, we can silence the loud deprecation +# warning before importing `dask.dataframe` (this won't +# do anything for dask==2024.2.0) +config.set({"dataframe.query-planning-warning": False}) + +import dask.dataframe as dd from dask.dataframe import from_delayed import cudf diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 76733828ea9..bbfb7dfe2ed 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -336,7 +336,8 @@ def test_assign(): newcol = dd.from_pandas(cudf.Series(pdcol), npartitions=dgf.npartitions) got = dgf.assign(z=newcol) - dd.assert_eq(got.loc[:, ["x", "y"]], df) + # TODO: Using `loc[:, ["x", "y"]]` may be broken with dask-expr API + dd.assert_eq(got[["x", "y"]], df) np.testing.assert_array_equal(got["z"].compute().values_host, pdcol) From 5a6b1c29bed9d33473efe1fe6cf476d97e28f3d2 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 26 Feb 2024 06:47:44 -0800 Subject: [PATCH 53/74] add conda testing for dask-expr --- ci/test_python_other.sh | 11 +++++++++++ ci/test_wheel_dask_cudf.sh | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 25c1d681029..f84cd50e6a5 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,6 +30,17 @@ pytest \ . popd +# Dask-expr tests should be skipped if dask_expr is not installed +rapids-logger "pytest dask_cudf + dask_expr" +pushd python/dask_cudf/dask_cudf/expr +DASK_DATAFRAME__QUERY_PLANNING=True pytest \ + --cache-clear \ + --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ + --numprocesses=8 \ + --dist=loadscope \ + . +popd + rapids-logger "pytest custreamz" pushd python/custreamz/custreamz pytest \ diff --git a/ci/test_wheel_dask_cudf.sh b/ci/test_wheel_dask_cudf.sh index dad6823b598..802cbb9bd94 100755 --- a/ci/test_wheel_dask_cudf.sh +++ b/ci/test_wheel_dask_cudf.sh @@ -40,7 +40,7 @@ python -m pytest \ popd # Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr -rapids-logger "pytest dask_cudf + dask-expr" +rapids-logger "pytest dask_cudf + dask_expr" pushd python/dask_cudf/dask_cudf DASK_DATAFRAME__QUERY_PLANNING=True python -m pytest \ --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ From ed9d2aaee0ea3783a03d26874e5cb187cabe9914 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 26 Feb 2024 06:49:44 -0800 Subject: [PATCH 54/74] change conda test --- ci/test_python_other.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index f84cd50e6a5..1f12bfece7d 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -30,7 +30,7 @@ pytest \ . popd -# Dask-expr tests should be skipped if dask_expr is not installed +# Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr rapids-logger "pytest dask_cudf + dask_expr" pushd python/dask_cudf/dask_cudf/expr DASK_DATAFRAME__QUERY_PLANNING=True pytest \ From bdf6fdc55b311d047bc513a3e0b7d36c756210f7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 26 Feb 2024 06:53:44 -0800 Subject: [PATCH 55/74] fix cw date --- ci/test_python_other.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 1f12bfece7d..28eec705cbd 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -1,5 +1,5 @@ #!/bin/bash -# Copyright (c) 2022-2023, NVIDIA CORPORATION. +# Copyright (c) 2022-2024, NVIDIA CORPORATION. # Common setup steps shared by Python test jobs source "$(dirname "$0")/test_python_common.sh" @@ -32,7 +32,7 @@ popd # Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr rapids-logger "pytest dask_cudf + dask_expr" -pushd python/dask_cudf/dask_cudf/expr +pushd python/dask_cudf/dask_cudf DASK_DATAFRAME__QUERY_PLANNING=True pytest \ --cache-clear \ --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ From 4f2fa9fd86dd6a649c40a3768665f4f5c9578d48 Mon Sep 17 00:00:00 2001 From: Lawrence Mitchell Date: Mon, 26 Feb 2024 18:54:04 +0000 Subject: [PATCH 56/74] Fix GroupBy.get_group and GroupBy.indices These are supposed to index based on row indices, not row labels. - Closes #14955 --- python/cudf/cudf/core/groupby/groupby.py | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 9612349a607..0073ef3ad08 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -363,13 +363,22 @@ def indices(self): >>> df.groupby(by=["a"]).indices {10: array([0, 1]), 40: array([2])} """ - group_names, offsets, _, grouped_values = self._grouped() + offsets, group_keys, (indices,) = self._groupby.groups( + [ + cudf.core.column.as_column( + range(len(self.obj)), dtype=size_type_dtype + ) + ] + ) + group_keys = libcudf.stream_compaction.drop_duplicates(group_keys) + if len(group_keys) > 1: + index = cudf.MultiIndex.from_arrays(group_keys) + else: + (group_keys,) = group_keys + index = cudf.Index(group_keys) return dict( - zip( - group_names.to_pandas(), - np.split(grouped_values.index.values, offsets[1:-1]), - ) + zip(index.to_pandas(), cp.split(indices.values, offsets[1:-1])) ) @_cudf_nvtx_annotate @@ -414,8 +423,7 @@ def get_group(self, name, obj=None): "instead of ``gb.get_group(name, obj=df)``.", FutureWarning, ) - - return obj.loc[self.groups[name].drop_duplicates()] + return obj.iloc[self.indices[name]] @_cudf_nvtx_annotate def size(self): From d9d077fc63e63729b315e0e489d15fa549daaf85 Mon Sep 17 00:00:00 2001 From: Lawrence Mitchell Date: Mon, 26 Feb 2024 19:02:01 +0000 Subject: [PATCH 57/74] Add test --- .../cudf/cudf/tests/groupby/test_groupby_obj.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 python/cudf/cudf/tests/groupby/test_groupby_obj.py diff --git a/python/cudf/cudf/tests/groupby/test_groupby_obj.py b/python/cudf/cudf/tests/groupby/test_groupby_obj.py new file mode 100644 index 00000000000..04b483e08dc --- /dev/null +++ b/python/cudf/cudf/tests/groupby/test_groupby_obj.py @@ -0,0 +1,15 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +from numpy.testing import assert_array_equal + +import cudf +from cudf.testing._utils import assert_eq + + +def test_groupby_14955(): + # https://github.com/rapidsai/cudf/issues/14955 + df = cudf.DataFrame({"a": [1, 2] * 2}, index=[0] * 4) + agg = df.groupby("a") + pagg = df.to_pandas().groupby("a") + for key in agg.groups: + assert_array_equal(pagg.indices[key], agg.indices[key].get()) + assert_eq(pagg.get_group(key), agg.get_group(key)) From 15c6b233b341a952bbd33effc355a79709f0ff4b Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 27 Feb 2024 06:54:48 -0800 Subject: [PATCH 58/74] small code-review changes --- python/dask_cudf/dask_cudf/expr/_collection.py | 6 +++--- python/dask_cudf/dask_cudf/io/tests/test_parquet.py | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 4f7938fc161..0c3725d0536 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -22,7 +22,7 @@ ## -class CudfMixin: +class VarMixin: def var( self, axis=0, @@ -47,7 +47,7 @@ def var( ) -class DataFrame(CudfMixin, DXDataFrame): +class DataFrame(VarMixin, DXDataFrame): @classmethod def from_dict(cls, *args, **kwargs): with config.set({"dataframe.backend": "cudf"}): @@ -80,7 +80,7 @@ def groupby( ) -class Series(CudfMixin, DXSeries): +class Series(VarMixin, DXSeries): def groupby(self, by, **kwargs): from dask_cudf.expr._groupby import SeriesGroupBy diff --git a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py index 06102dccbc4..de2a735b2ce 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -294,7 +294,8 @@ def test_filters_at_row_group_level(tmpdir): assert a.npartitions == 1 assert (a.shape[0] == 1).compute() - # Note: Use overwrite=True to ignore cache (for now) + # Overwrite=True can be removed for dask-expr>=0.4.1 + # See: https://github.com/dask-contrib/dask-expr/issues/800 ddf.to_parquet( tmp_path, engine="pyarrow", row_group_size=1, overwrite=True ) From f53683736b96962c89de5384bb49a9c5ec0995da Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 09:40:03 -0800 Subject: [PATCH 59/74] remove optimized groupby code path since it is a lot of fragile code with little benefit --- python/dask_cudf/dask_cudf/expr/_expr.py | 134 ------------------ python/dask_cudf/dask_cudf/expr/_groupby.py | 84 +---------- .../dask_cudf/dask_cudf/tests/test_groupby.py | 14 +- 3 files changed, 16 insertions(+), 216 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 7da7419c3ba..38b38e18ca1 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -1,23 +1,9 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -import functools - from dask_expr._cumulative import CumulativeBlockwise, TakeLast -from dask_expr._groupby import ( - DecomposableGroupbyAggregation, - GroupbyAggregation, -) from dask_expr._reductions import Var from dask_expr._shuffle import DiskShuffle -from dask.dataframe.core import _concat - -from dask_cudf.groupby import ( - _finalize_gb_agg, - _groupby_partition_agg, - _tree_node_agg, -) - ## ## Custom expression patching ## @@ -83,123 +69,3 @@ def _shuffle_group(df, col, _filter, p): DiskShuffle._shuffle_group = staticmethod(_shuffle_group) - - -class CudfGroupbyAggregation(GroupbyAggregation): - def _lower(self): - return DecomposableCudfGroupbyAggregation( - self.frame, - self.arg, - self.observed, - self.dropna, - self.split_every, - self.split_out, - self.sort, - self.shuffle_method, - self._slice, - *self.by, - ) - - -class DecomposableCudfGroupbyAggregation(DecomposableGroupbyAggregation): - sep = "___" - - @property - def shuffle_by_index(self): - return False # We always group by column(s) in dask-cudf - - @functools.cached_property - def spec_info(self): - if isinstance(self.arg, (dict, list)): - aggs = self.arg.copy() - else: - aggs = self.arg - - if self._slice and not isinstance(aggs, dict): - aggs = {self._slice: aggs} - - gb_cols = self._by_columns - if isinstance(gb_cols, str): - gb_cols = [gb_cols] - columns = [c for c in self.frame.columns if c not in gb_cols] - if not isinstance(aggs, dict): - aggs = {col: aggs for col in columns} - - # Assert if our output will have a MultiIndex; this will be the case if - # any value in the `aggs` dict is not a string (i.e. multiple/named - # aggregations per column) - str_cols_out = True - aggs_renames = {} - for col in aggs: - if isinstance(aggs[col], str) or callable(aggs[col]): - aggs[col] = [aggs[col]] - elif isinstance(aggs[col], dict): - str_cols_out = False - col_aggs = [] - for k, v in aggs[col].items(): - aggs_renames[col, v] = k - col_aggs.append(v) - aggs[col] = col_aggs - else: - str_cols_out = False - if col in gb_cols: - columns.append(col) - - return { - "aggs": aggs, - "columns": columns, - "str_cols_out": str_cols_out, - "aggs_renames": aggs_renames, - } - - @classmethod - def chunk(cls, df, *by, **kwargs): - # `by` columns are already specified in kwargs - return _groupby_partition_agg(df, **kwargs) - - @classmethod - def combine(cls, inputs, **kwargs): - return _tree_node_agg(_concat(inputs), **kwargs) - - @classmethod - def aggregate(cls, inputs, **kwargs): - return _finalize_gb_agg(_concat(inputs), **kwargs) - - @property - def chunk_kwargs(self) -> dict: - dropna = True if self.dropna is None else self.dropna - return { - "gb_cols": self._by_columns, - "aggs": self.spec_info["aggs"], - "columns": self.spec_info["columns"], - "dropna": dropna, - "sort": self.sort, - "sep": self.sep, - } - - @property - def combine_kwargs(self) -> dict: - dropna = True if self.dropna is None else self.dropna - return { - "gb_cols": self._by_columns, - "dropna": dropna, - "sort": self.sort, - "sep": self.sep, - } - - @property - def aggregate_kwargs(self) -> dict: - dropna = True if self.dropna is None else self.dropna - final_columns = self._slice or self._meta.columns - return { - "gb_cols": self._by_columns, - "aggs": self.spec_info["aggs"], - "columns": self.spec_info["columns"], - "final_columns": final_columns, - "as_index": True, # False not supported in dask-expr - "dropna": dropna, - "sort": self.sort, - "sep": self.sep, - "str_cols_out": self.spec_info["str_cols_out"], - "aggs_renames": self.spec_info["aggs_renames"], - } diff --git a/python/dask_cudf/dask_cudf/expr/_groupby.py b/python/dask_cudf/dask_cudf/expr/_groupby.py index 0e9bb6cecfa..7f275151f75 100644 --- a/python/dask_cudf/dask_cudf/expr/_groupby.py +++ b/python/dask_cudf/dask_cudf/expr/_groupby.py @@ -1,32 +1,24 @@ # Copyright (c) 2024, NVIDIA CORPORATION. -from functools import partial - -from dask_expr._collection import new_collection from dask_expr._groupby import ( GroupBy as DXGroupBy, - GroupbyAggregation, SeriesGroupBy as DXSeriesGroupBy, ) from dask_expr._util import is_scalar -from dask_cudf.expr._expr import CudfGroupbyAggregation -from dask_cudf.groupby import OPTIMIZED_AGGS - ## ## Custom groupby classes ## +# TODO: These classes are mostly a work-around for missing +# `observed=False` support. +# See: https://github.com/rapidsai/cudf/issues/15173 + class GroupBy(DXGroupBy): def __init__(self, *args, observed=None, **kwargs): observed = observed if observed is not None else True super().__init__(*args, observed=observed, **kwargs) - # Add optimized aggregation code paths - for agg in OPTIMIZED_AGGS: - setattr(self, agg, partial(single_agg, self, agg)) - setattr(self, "agg", partial(groupby_agg, self)) - setattr(self, "aggregate", partial(groupby_agg, self)) def __getitem__(self, key): if is_scalar(key): @@ -54,71 +46,3 @@ class SeriesGroupBy(DXSeriesGroupBy): def __init__(self, *args, observed=None, **kwargs): observed = observed if observed is not None else True super().__init__(*args, observed=observed, **kwargs) - # Add optimized aggregation code paths - for agg in OPTIMIZED_AGGS: - setattr(self, agg, partial(single_agg, self, agg)) - setattr(self, "agg", partial(groupby_agg, self)) - setattr(self, "aggregate", partial(groupby_agg, self)) - - -def single_agg(gb, agg_name, **kwargs): - _optimized = kwargs.pop("_optimized", agg_name == "collect") - if _optimized and hasattr(gb.obj._meta, "to_pandas"): - if gb._slice is None: - if isinstance(gb.by, list): - agg = {c: agg_name for c in gb.obj.columns if c not in gb.by} - else: - agg = {c: agg_name for c in gb.obj.columns if c != gb.by} - else: - agg = {gb._slice: agg_name} - - return groupby_agg(gb, agg, **kwargs) - else: - # Fall back to dask-expr code path - return getattr(super(type(gb), gb), agg_name)(**kwargs) - - -def groupby_agg( - gb, - arg=None, - split_every=8, - split_out=1, - shuffle_method=None, - _optimized=True, -): - from dask_cudf.groupby import ( - OPTIMIZED_AGGS, - _aggs_optimized, - _redirect_aggs, - ) - - if arg is None: - raise NotImplementedError("arg=None not supported") - - if arg == "size": - return gb.size() - - arg = _redirect_aggs(arg) - if ( - _optimized - and _aggs_optimized(arg, OPTIMIZED_AGGS) - and hasattr(gb.obj._meta, "to_pandas") - ): - cls = CudfGroupbyAggregation - else: - cls = GroupbyAggregation - - return new_collection( - cls( - gb.obj.expr, - arg, - gb.observed, - gb.dropna, - split_every, - split_out, - gb.sort, - shuffle_method, - gb._slice, - *gb.by, - ) - ) diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 05537435afb..51b149ddc90 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -14,6 +14,16 @@ from dask_cudf.groupby import OPTIMIZED_AGGS, _aggs_optimized from dask_cudf.tests.utils import QUERY_PLANNING_ON, xfail_dask_expr +# XFAIL "collect" tests for now +agg_params = [agg for agg in OPTIMIZED_AGGS if agg != "collect"] +if QUERY_PLANNING_ON: + agg_params.append( + # TODO: "collect" not supported with dask-expr yet + pytest.param("collect", marks=pytest.mark.xfail) + ) +else: + agg_params.append("collect") + def assert_cudf_groupby_layers(ddf): for prefix in ("cudf-aggregate-chunk", "cudf-aggregate-agg"): @@ -47,7 +57,7 @@ def pdf(request): return pdf -@pytest.mark.parametrize("aggregation", OPTIMIZED_AGGS) +@pytest.mark.parametrize("aggregation", agg_params) @pytest.mark.parametrize("series", [False, True]) def test_groupby_basic(series, aggregation, pdf): gdf = cudf.DataFrame.from_pandas(pdf) @@ -100,7 +110,7 @@ def test_groupby_cumulative(aggregation, pdf, series): dd.assert_eq(a, b) -@pytest.mark.parametrize("aggregation", OPTIMIZED_AGGS) +@pytest.mark.parametrize("aggregation", agg_params) @pytest.mark.parametrize( "func", [ From 761492629a1644bed01f86c8f93043cbf8d37ab3 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 10:20:44 -0800 Subject: [PATCH 60/74] simplify tokenization fix --- python/cudf/cudf/core/frame.py | 4 ++-- python/cudf/cudf/core/index.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/cudf/cudf/core/frame.py b/python/cudf/cudf/core/frame.py index 32a39765208..3d6b83f328c 100644 --- a/python/cudf/cudf/core/frame.py +++ b/python/cudf/cudf/core/frame.py @@ -1957,8 +1957,8 @@ def __dask_tokenize__(self): from dask.base import normalize_token return [ - str(type(self)), - str(self._dtypes), + type(self), + self._dtypes, normalize_token(self.to_pandas()), ] diff --git a/python/cudf/cudf/core/index.py b/python/cudf/cudf/core/index.py index 4b259153328..1b9893d1256 100644 --- a/python/cudf/cudf/core/index.py +++ b/python/cudf/cudf/core/index.py @@ -960,7 +960,7 @@ def __abs__(self): @_warn_no_dask_cudf def __dask_tokenize__(self): - return (str(type(self)), self.start, self.stop, self.step) + return (type(self), self.start, self.stop, self.step) class Index(SingleColumnFrame, BaseIndex, metaclass=IndexMeta): From d3a97c7f2bbc1dfdb17d37537b83b907216d61ae Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 11:33:34 -0800 Subject: [PATCH 61/74] add comments to expression patchhes --- python/dask_cudf/dask_cudf/expr/_expr.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index 38b38e18ca1..fc50f63c1c0 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -9,6 +9,8 @@ ## +# This can be removed after cudf#15176 is addressed. +# See: https://github.com/rapidsai/cudf/issues/15176 class PatchCumulativeBlockwise(CumulativeBlockwise): @property def _args(self) -> list: @@ -24,6 +26,9 @@ def _kwargs(self) -> dict: CumulativeBlockwise._kwargs = PatchCumulativeBlockwise._kwargs +# This can be removed if squeeze support is added to cudf, +# or if squeeze is removed from the dask-expr logic. +# See: https://github.com/rapidsai/cudf/issues/15177 def _takelast(a, skipna=True): if not len(a): return a @@ -36,19 +41,26 @@ def _takelast(a, skipna=True): TakeLast.operation = staticmethod(_takelast) +# This patch accounts for differences between +# numpy and cupy behavior. It may make sense +# to move this logic upstream. _dx_reduction_aggregate = Var.reduction_aggregate def _reduction_aggregate(*args, **kwargs): result = _dx_reduction_aggregate(*args, **kwargs) if result.ndim == 0: - return result.tolist() + # cupy will sometimes produce a 0d array, and + # we need to convert it to a scalar. + return result.item() return result Var.reduction_aggregate = staticmethod(_reduction_aggregate) +# This patch should be removed afer cudf#15143 is merged. +# See: https://github.com/rapidsai/cudf/pull/15143 def _shuffle_group(df, col, _filter, p): from dask.dataframe.shuffle import ensure_cleanup_on_exception From 02547e9e066a6197317b15eef990aa2314874bd7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 12:05:42 -0800 Subject: [PATCH 62/74] roll back unused groupby changes --- python/dask_cudf/dask_cudf/groupby.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/python/dask_cudf/dask_cudf/groupby.py b/python/dask_cudf/dask_cudf/groupby.py index 04c680eede3..9afb4796891 100644 --- a/python/dask_cudf/dask_cudf/groupby.py +++ b/python/dask_cudf/dask_cudf/groupby.py @@ -777,7 +777,6 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): """ agg_dict = {} - flatten = [] for col in df.columns: if col in gb_cols: continue @@ -786,8 +785,6 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): agg_dict[col] = ["sum"] elif agg in OPTIMIZED_AGGS: agg_dict[col] = [agg] - if agg == "collect": - flatten.append(col) else: raise ValueError(f"Unexpected aggregation: {agg}") @@ -802,10 +799,6 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): ] gb.columns = output_columns - # Avoid nested lists - for col in flatten: - gb[col] = gb[col].list.concat() - # Return with deterministic column ordering return gb[sorted(output_columns)] @@ -881,6 +874,9 @@ def _finalize_gb_agg( gb.drop(columns=[sum_name], inplace=True) if "count" not in agg_list: gb.drop(columns=[count_name], inplace=True) + if "collect" in agg_list: + collect_name = _make_name((col, "collect"), sep=sep) + gb[collect_name] = gb[collect_name].list.concat() # Ensure sorted keys if `sort=True` if sort: From e2ed6ffc18cd007ffa38f6e1cbe642dfae01fa72 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 12:06:43 -0800 Subject: [PATCH 63/74] remove space --- python/dask_cudf/dask_cudf/groupby.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/groupby.py b/python/dask_cudf/dask_cudf/groupby.py index 9afb4796891..43ad4f0fee3 100644 --- a/python/dask_cudf/dask_cudf/groupby.py +++ b/python/dask_cudf/dask_cudf/groupby.py @@ -798,7 +798,6 @@ def _tree_node_agg(df, gb_cols, dropna, sort, sep): for name in gb.columns ] gb.columns = output_columns - # Return with deterministic column ordering return gb[sorted(output_columns)] From ef2fefc5d99232e890e795136cd1e20fdfdc6452 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 12:29:16 -0800 Subject: [PATCH 64/74] update dependencies --- conda/environments/all_cuda-118_arch-x86_64.yaml | 2 +- conda/environments/all_cuda-122_arch-x86_64.yaml | 2 +- dependencies.yaml | 2 +- python/dask_cudf/dask_cudf/expr/_collection.py | 2 ++ python/dask_cudf/pyproject.toml | 2 +- 5 files changed, 6 insertions(+), 4 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index c682bc6d259..a540cb2b432 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -29,7 +29,7 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* -- dask-expr==0.4.0 +- dask-expr - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 diff --git a/conda/environments/all_cuda-122_arch-x86_64.yaml b/conda/environments/all_cuda-122_arch-x86_64.yaml index 69ba4e4bf7c..56aaba45179 100644 --- a/conda/environments/all_cuda-122_arch-x86_64.yaml +++ b/conda/environments/all_cuda-122_arch-x86_64.yaml @@ -30,7 +30,7 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* -- dask-expr==0.4.0 +- dask-expr - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 diff --git a/dependencies.yaml b/dependencies.yaml index b3b5d714888..6695cc31298 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -648,7 +648,7 @@ dependencies: packages: - dask-cuda==24.4.* - *numba - - dask-expr==0.4.0 + - dask-expr depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 0c3725d0536..b2f92aeddda 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -22,6 +22,8 @@ ## +# VarMixin can be removed if cudf#15179 is addressed. +# See: https://github.com/rapidsai/cudf/issues/15179 class VarMixin: def var( self, diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index b5dda5b8618..39dc25cf2e9 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,7 +44,7 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", - "dask-expr==0.4.0", + "dask-expr", "numba>=0.57", "pytest-cov", "pytest-xdist", From 47392c3df86fcf9454f9e15f575affca6cd4fcc9 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 17:16:58 -0800 Subject: [PATCH 65/74] reduce diff --- python/dask_cudf/dask_cudf/tests/test_binops.py | 2 +- python/dask_cudf/dask_cudf/tests/test_core.py | 2 +- python/dask_cudf/dask_cudf/tests/test_struct.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_binops.py b/python/dask_cudf/dask_cudf/tests/test_binops.py index 770f7368d56..87bd401accd 100644 --- a/python/dask_cudf/dask_cudf/tests/test_binops.py +++ b/python/dask_cudf/dask_cudf/tests/test_binops.py @@ -1,4 +1,4 @@ -# Copyright (c) 2022-2024, NVIDIA CORPORATION. +# Copyright (c) 2022, NVIDIA CORPORATION. import operator diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index bbfb7dfe2ed..68b578caac6 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -336,7 +336,7 @@ def test_assign(): newcol = dd.from_pandas(cudf.Series(pdcol), npartitions=dgf.npartitions) got = dgf.assign(z=newcol) - # TODO: Using `loc[:, ["x", "y"]]` may be broken with dask-expr API + # Using `loc[:, ["x", "y"]]` was broken for dask-expr 0.4.0 dd.assert_eq(got[["x", "y"]], df) np.testing.assert_array_equal(got["z"].compute().values_host, pdcol) diff --git a/python/dask_cudf/dask_cudf/tests/test_struct.py b/python/dask_cudf/dask_cudf/tests/test_struct.py index ad861193674..6abac4cf53b 100644 --- a/python/dask_cudf/dask_cudf/tests/test_struct.py +++ b/python/dask_cudf/dask_cudf/tests/test_struct.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2024, NVIDIA CORPORATION. +# Copyright (c) 2021-2022, NVIDIA CORPORATION. import pytest From 481c655bca4c8bd8eed2edab9ed362742292c8f8 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 28 Feb 2024 18:57:35 -0800 Subject: [PATCH 66/74] adjust tokenization change --- ci/test_python_other.sh | 5 +---- python/cudf/cudf/core/frame.py | 2 +- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/ci/test_python_other.sh b/ci/test_python_other.sh index 091fa9414e6..b5239f17863 100755 --- a/ci/test_python_other.sh +++ b/ci/test_python_other.sh @@ -31,14 +31,11 @@ rapids-logger "pytest dask_cudf" # Run tests in dask_cudf/tests and dask_cudf/io/tests with dask-expr rapids-logger "pytest dask_cudf + dask_expr" -pushd python/dask_cudf/dask_cudf -DASK_DATAFRAME__QUERY_PLANNING=True pytest \ - --cache-clear \ +DASK_DATAFRAME__QUERY_PLANNING=True ./ci/run_dask_cudf_pytests.sh \ --junitxml="${RAPIDS_TESTS_DIR}/junit-dask-cudf-expr.xml" \ --numprocesses=8 \ --dist=loadscope \ . -popd rapids-logger "pytest custreamz" ./ci/run_custreamz_pytests.sh \ diff --git a/python/cudf/cudf/core/frame.py b/python/cudf/cudf/core/frame.py index 3d6b83f328c..809bdb4e6d1 100644 --- a/python/cudf/cudf/core/frame.py +++ b/python/cudf/cudf/core/frame.py @@ -1958,7 +1958,7 @@ def __dask_tokenize__(self): return [ type(self), - self._dtypes, + str(self._dtypes), normalize_token(self.to_pandas()), ] From 026b98b5409275020510d91bc1f370b2262aa2a5 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 29 Feb 2024 07:42:43 -0800 Subject: [PATCH 67/74] xfail groupby test that fails for older dask-expr version --- dependencies.yaml | 1 + python/dask_cudf/dask_cudf/tests/test_groupby.py | 1 + 2 files changed, 2 insertions(+) diff --git a/dependencies.yaml b/dependencies.yaml index 6695cc31298..3a575e9cb5f 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -648,6 +648,7 @@ dependencies: packages: - dask-cuda==24.4.* - *numba + # TODO: Remove once dask-expr is in rapids-dask-dependency - dask-expr depends_on_cudf: common: diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 51b149ddc90..bd10fd57f8e 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -668,6 +668,7 @@ def test_groupby_agg_params(npartitions, split_every, split_out, as_index): dd.assert_eq(gf, pf) +@xfail_dask_expr("Newer dask-expr version needed") @pytest.mark.parametrize( "aggregations", [(sum, "sum"), (max, "max"), (min, "min")] ) From 67f8be9ea80c25554896c29c79d35d6931eeb641 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 29 Feb 2024 09:03:19 -0800 Subject: [PATCH 68/74] remove explicit dask-expr dependency as it should now be covered by rapids-dask-dependency --- conda/environments/all_cuda-118_arch-x86_64.yaml | 1 - conda/environments/all_cuda-122_arch-x86_64.yaml | 1 - dependencies.yaml | 2 -- python/dask_cudf/pyproject.toml | 1 - 4 files changed, 5 deletions(-) diff --git a/conda/environments/all_cuda-118_arch-x86_64.yaml b/conda/environments/all_cuda-118_arch-x86_64.yaml index a540cb2b432..79b786fe012 100644 --- a/conda/environments/all_cuda-118_arch-x86_64.yaml +++ b/conda/environments/all_cuda-118_arch-x86_64.yaml @@ -29,7 +29,6 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* -- dask-expr - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 diff --git a/conda/environments/all_cuda-122_arch-x86_64.yaml b/conda/environments/all_cuda-122_arch-x86_64.yaml index 56aaba45179..66a4ee57238 100644 --- a/conda/environments/all_cuda-122_arch-x86_64.yaml +++ b/conda/environments/all_cuda-122_arch-x86_64.yaml @@ -30,7 +30,6 @@ dependencies: - cxx-compiler - cython>=3.0.3 - dask-cuda==24.4.* -- dask-expr - dlpack>=0.5,<0.6.0a0 - doxygen=1.9.1 - fastavro>=0.22.9 diff --git a/dependencies.yaml b/dependencies.yaml index 3a575e9cb5f..4281e907862 100644 --- a/dependencies.yaml +++ b/dependencies.yaml @@ -648,8 +648,6 @@ dependencies: packages: - dask-cuda==24.4.* - *numba - # TODO: Remove once dask-expr is in rapids-dask-dependency - - dask-expr depends_on_cudf: common: - output_types: conda diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 39dc25cf2e9..1efa10133b3 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -44,7 +44,6 @@ cudf = "dask_cudf.backends:CudfDXBackendEntrypoint" [project.optional-dependencies] test = [ "dask-cuda==24.4.*", - "dask-expr", "numba>=0.57", "pytest-cov", "pytest-xdist", From be88773dd14bed4d6b3c5ba1200d0c5ea64ce40c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 29 Feb 2024 10:02:23 -0800 Subject: [PATCH 69/74] remove more code - sits on top of #15143 --- python/dask_cudf/dask_cudf/expr/_expr.py | 25 ------------------- .../dask_cudf/dask_cudf/io/tests/test_csv.py | 5 ++-- python/dask_cudf/dask_cudf/tests/test_core.py | 5 ++-- 3 files changed, 4 insertions(+), 31 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index fc50f63c1c0..cbe7a71cb73 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -2,7 +2,6 @@ from dask_expr._cumulative import CumulativeBlockwise, TakeLast from dask_expr._reductions import Var -from dask_expr._shuffle import DiskShuffle ## ## Custom expression patching @@ -57,27 +56,3 @@ def _reduction_aggregate(*args, **kwargs): Var.reduction_aggregate = staticmethod(_reduction_aggregate) - - -# This patch should be removed afer cudf#15143 is merged. -# See: https://github.com/rapidsai/cudf/pull/15143 -def _shuffle_group(df, col, _filter, p): - from dask.dataframe.shuffle import ensure_cleanup_on_exception - - with ensure_cleanup_on_exception(p): - g = df.groupby(col) - if hasattr(g, "_grouped"): - # Avoid `get_group` for cudf data. - # See: https://github.com/rapidsai/cudf/issues/14955 - keys, part_offsets, _, grouped_df = df.groupby(col)._grouped() - d = { - k: grouped_df.iloc[part_offsets[i] : part_offsets[i + 1]] - for i, k in enumerate(keys.to_pandas()) - if k in _filter - } - else: - d = {i: g.get_group(i) for i in g.groups if i in _filter} - p.append(d, fsync=True) - - -DiskShuffle._shuffle_group = staticmethod(_shuffle_group) diff --git a/python/dask_cudf/dask_cudf/io/tests/test_csv.py b/python/dask_cudf/dask_cudf/io/tests/test_csv.py index 226818fd141..a35a9f1be48 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_csv.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_csv.py @@ -14,7 +14,6 @@ import cudf import dask_cudf -from dask_cudf.tests.utils import skip_dask_expr @pytest.fixture @@ -59,12 +58,12 @@ def test_csv_roundtrip_backend_dispatch(tmp_path): ddf = dask_cudf.from_cudf(df, npartitions=2) csv_path = str(tmp_path / "data-*.csv") ddf.to_csv(csv_path, index=False) - ddf2 = dask_cudf.read_csv(csv_path) + with dask.config.set({"dataframe.backend": "cudf"}): + ddf2 = dd.read_csv(csv_path) assert isinstance(ddf2, dask_cudf.DataFrame) dd.assert_eq(ddf, ddf2, check_divisions=False, check_index=False) -@skip_dask_expr() def test_csv_roundtrip(tmp_path): df = cudf.DataFrame({"x": [1, 2, 3, 4], "id": ["a", "b", "c", "d"]}) ddf = dask_cudf.from_cudf(df, npartitions=2) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 68b578caac6..8a2f3414fd1 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -207,7 +207,7 @@ def test_set_index(nelem): dd.assert_eq(expect, got, check_index=False, check_divisions=False) -@skip_dask_expr() +@xfail_dask_expr("missing support for divisions='quantile'") @pytest.mark.parametrize("by", ["a", "b"]) @pytest.mark.parametrize("nelem", [10, 500]) @pytest.mark.parametrize("nparts", [1, 10]) @@ -388,7 +388,7 @@ def test_setitem_scalar_datetime(): np.testing.assert_array_equal(got["z"], df["z"]) -@skip_dask_expr() +@skip_dask_expr("Not relevant for dask-expr") @pytest.mark.parametrize( "func", [ @@ -426,7 +426,6 @@ def test_repr(func): assert gddf._repr_html_() -@skip_dask_expr() @pytest.mark.skip(reason="datetime indexes not fully supported in cudf") @pytest.mark.parametrize("start", ["1d", "5d", "1w", "12h"]) @pytest.mark.parametrize("stop", ["1d", "3d", "8h"]) From 32bfff3aa822532e5703183cf980a6c024df1562 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 1 Mar 2024 11:06:06 -0800 Subject: [PATCH 70/74] roll back shuffle-group fix for now --- python/cudf/cudf/core/groupby/groupby.py | 50 ++++++++----------- .../cudf/tests/groupby/test_groupby_obj.py | 15 ------ python/dask_cudf/dask_cudf/expr/_expr.py | 25 ++++++++++ 3 files changed, 46 insertions(+), 44 deletions(-) delete mode 100644 python/cudf/cudf/tests/groupby/test_groupby_obj.py diff --git a/python/cudf/cudf/core/groupby/groupby.py b/python/cudf/cudf/core/groupby/groupby.py index 0073ef3ad08..e4370be304a 100644 --- a/python/cudf/cudf/core/groupby/groupby.py +++ b/python/cudf/cudf/core/groupby/groupby.py @@ -109,11 +109,11 @@ def _is_row_of(chunk, obj): Parrot 30.0 Parrot 20.0 Name: Max Speed, dtype: float64 ->>> ser.groupby(level=0).mean() +>>> ser.groupby(level=0, sort=True).mean() Falcon 370.0 Parrot 25.0 Name: Max Speed, dtype: float64 ->>> ser.groupby(ser > 100).mean() +>>> ser.groupby(ser > 100, sort=True).mean() Max Speed False 25.0 True 370.0 @@ -133,7 +133,7 @@ def _is_row_of(chunk, obj): 1 Falcon 370.0 2 Parrot 24.0 3 Parrot 26.0 ->>> df.groupby(['Animal']).mean() +>>> df.groupby(['Animal'], sort=True).mean() Max Speed Animal Falcon 375.0 @@ -151,22 +151,22 @@ def _is_row_of(chunk, obj): Wild 350.0 Parrot Captive 30.0 Wild 20.0 ->>> df.groupby(level=0).mean() +>>> df.groupby(level=0, sort=True).mean() Max Speed Animal Falcon 370.0 Parrot 25.0 ->>> df.groupby(level="Type").mean() +>>> df.groupby(level="Type", sort=True).mean() Max Speed Type -Wild 185.0 Captive 210.0 +Wild 185.0 >>> df = cudf.DataFrame({{'A': 'a a b'.split(), ... 'B': [1,2,3], ... 'C': [4,6,5]}}) ->>> g1 = df.groupby('A', group_keys=False) ->>> g2 = df.groupby('A', group_keys=True) +>>> g1 = df.groupby('A', group_keys=False, sort=True) +>>> g2 = df.groupby('A', group_keys=True, sort=True) Notice that ``g1`` have ``g2`` have two groups, ``a`` and ``b``, and only differ in their ``group_keys`` argument. Calling `apply` in various ways, @@ -363,22 +363,13 @@ def indices(self): >>> df.groupby(by=["a"]).indices {10: array([0, 1]), 40: array([2])} """ - offsets, group_keys, (indices,) = self._groupby.groups( - [ - cudf.core.column.as_column( - range(len(self.obj)), dtype=size_type_dtype - ) - ] - ) + group_names, offsets, _, grouped_values = self._grouped() - group_keys = libcudf.stream_compaction.drop_duplicates(group_keys) - if len(group_keys) > 1: - index = cudf.MultiIndex.from_arrays(group_keys) - else: - (group_keys,) = group_keys - index = cudf.Index(group_keys) return dict( - zip(index.to_pandas(), cp.split(indices.values, offsets[1:-1])) + zip( + group_names.to_pandas(), + np.split(grouped_values.index.values, offsets[1:-1]), + ) ) @_cudf_nvtx_annotate @@ -423,7 +414,8 @@ def get_group(self, name, obj=None): "instead of ``gb.get_group(name, obj=df)``.", FutureWarning, ) - return obj.iloc[self.indices[name]] + + return obj.loc[self.groups[name].drop_duplicates()] @_cudf_nvtx_annotate def size(self): @@ -547,11 +539,11 @@ def agg(self, func): ... 'b': [1, 2, 3], ... 'c': [2, 2, 1] ... }) - >>> a.groupby('a').agg('sum') + >>> a.groupby('a', sort=True).agg('sum') b c a - 2 3 1 1 3 4 + 2 3 1 Specifying a list of aggregations to perform on each column. @@ -561,12 +553,12 @@ def agg(self, func): ... 'b': [1, 2, 3], ... 'c': [2, 2, 1] ... }) - >>> a.groupby('a').agg(['sum', 'min']) + >>> a.groupby('a', sort=True).agg(['sum', 'min']) b c sum min sum min a - 2 3 3 1 1 1 3 1 4 2 + 2 3 3 1 1 Using a dict to specify aggregations to perform per column. @@ -576,12 +568,12 @@ def agg(self, func): ... 'b': [1, 2, 3], ... 'c': [2, 2, 1] ... }) - >>> a.groupby('a').agg({'a': 'max', 'b': ['min', 'mean']}) + >>> a.groupby('a', sort=True).agg({'a': 'max', 'b': ['min', 'mean']}) a b max min mean a - 2 2 3 3.0 1 1 1 1.5 + 2 2 3 3.0 Using lambdas/callables to specify aggregations taking parameters. diff --git a/python/cudf/cudf/tests/groupby/test_groupby_obj.py b/python/cudf/cudf/tests/groupby/test_groupby_obj.py deleted file mode 100644 index 04b483e08dc..00000000000 --- a/python/cudf/cudf/tests/groupby/test_groupby_obj.py +++ /dev/null @@ -1,15 +0,0 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. -from numpy.testing import assert_array_equal - -import cudf -from cudf.testing._utils import assert_eq - - -def test_groupby_14955(): - # https://github.com/rapidsai/cudf/issues/14955 - df = cudf.DataFrame({"a": [1, 2] * 2}, index=[0] * 4) - agg = df.groupby("a") - pagg = df.to_pandas().groupby("a") - for key in agg.groups: - assert_array_equal(pagg.indices[key], agg.indices[key].get()) - assert_eq(pagg.get_group(key), agg.get_group(key)) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index cbe7a71cb73..fc50f63c1c0 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -2,6 +2,7 @@ from dask_expr._cumulative import CumulativeBlockwise, TakeLast from dask_expr._reductions import Var +from dask_expr._shuffle import DiskShuffle ## ## Custom expression patching @@ -56,3 +57,27 @@ def _reduction_aggregate(*args, **kwargs): Var.reduction_aggregate = staticmethod(_reduction_aggregate) + + +# This patch should be removed afer cudf#15143 is merged. +# See: https://github.com/rapidsai/cudf/pull/15143 +def _shuffle_group(df, col, _filter, p): + from dask.dataframe.shuffle import ensure_cleanup_on_exception + + with ensure_cleanup_on_exception(p): + g = df.groupby(col) + if hasattr(g, "_grouped"): + # Avoid `get_group` for cudf data. + # See: https://github.com/rapidsai/cudf/issues/14955 + keys, part_offsets, _, grouped_df = df.groupby(col)._grouped() + d = { + k: grouped_df.iloc[part_offsets[i] : part_offsets[i + 1]] + for i, k in enumerate(keys.to_pandas()) + if k in _filter + } + else: + d = {i: g.get_group(i) for i in g.groups if i in _filter} + p.append(d, fsync=True) + + +DiskShuffle._shuffle_group = staticmethod(_shuffle_group) From ff4e4381c3e385a565ca34939907f22fba8dd934 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 5 Mar 2024 06:53:35 -0800 Subject: [PATCH 71/74] update cw date --- python/dask_cudf/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/dask_cudf/pyproject.toml b/python/dask_cudf/pyproject.toml index 7f91f4098ae..21aaa17a6c7 100644 --- a/python/dask_cudf/pyproject.toml +++ b/python/dask_cudf/pyproject.toml @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2023, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. [build-system] build-backend = "setuptools.build_meta" From 363d203a4075744d808e7855db18c7552d699b6f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 5 Mar 2024 09:05:17 -0800 Subject: [PATCH 72/74] improve check --- python/dask_cudf/dask_cudf/core.py | 6 ++++-- python/dask_cudf/dask_cudf/expr/__init__.py | 5 +++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index a63dde544d3..e6d9417fe7e 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -8,7 +8,7 @@ import pandas as pd from tlz import partition_all -from dask import config, dataframe as dd +from dask import dataframe as dd from dask.base import normalize_token, tokenize from dask.dataframe.core import ( Scalar, @@ -685,6 +685,8 @@ def reduction( @_dask_cudf_nvtx_annotate def from_cudf(data, npartitions=None, chunksize=None, sort=True, name=None): + from dask_cudf import QUERY_PLANNING_ON + if isinstance(getattr(data, "index", None), cudf.MultiIndex): raise NotImplementedError( "dask_cudf does not support MultiIndex Dataframes." @@ -692,7 +694,7 @@ def from_cudf(data, npartitions=None, chunksize=None, sort=True, name=None): # Dask-expr doesn't support the `name` argument name = {} - if not config.get("dataframe.query-planning", False): + if not QUERY_PLANNING_ON: name = { "name": name or ("from_cudf-" + tokenize(data, npartitions or chunksize)) diff --git a/python/dask_cudf/dask_cudf/expr/__init__.py b/python/dask_cudf/dask_cudf/expr/__init__.py index 867ea1be84e..c36dd0abcb9 100644 --- a/python/dask_cudf/dask_cudf/expr/__init__.py +++ b/python/dask_cudf/dask_cudf/expr/__init__.py @@ -2,8 +2,9 @@ from dask import config -# Check if dask-dataframe is using dask-expr -QUERY_PLANNING_ON = config.get("dataframe.query-planning", False) +# Check if dask-dataframe is using dask-expr. +# For dask>=2024.3.0, a null value will default to True +QUERY_PLANNING_ON = config.get("dataframe.query-planning", None) is not False # Register custom expressions and collections try: From a330d06f2563e0a294867178d4ae611ef8ef1d08 Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Tue, 5 Mar 2024 12:39:47 -0600 Subject: [PATCH 73/74] Apply suggestions from code review Co-authored-by: Lawrence Mitchell --- python/dask_cudf/dask_cudf/core.py | 1 + python/dask_cudf/dask_cudf/expr/_expr.py | 22 ---------------------- 2 files changed, 1 insertion(+), 22 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index e6d9417fe7e..bfe58531a73 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -722,6 +722,7 @@ def from_cudf(data, npartitions=None, chunksize=None, sort=True, name=None): ) # TODO: `dd.from_pandas.__doc__` is empty when # `DASK_DATAFRAME__QUERY_PLANNING=True` + # since dask-expr does not provide a docstring for from_pandas. + textwrap.dedent(dd.from_pandas.__doc__ or "") ) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index fc50f63c1c0..ea980ffc5c8 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -59,25 +59,3 @@ def _reduction_aggregate(*args, **kwargs): Var.reduction_aggregate = staticmethod(_reduction_aggregate) -# This patch should be removed afer cudf#15143 is merged. -# See: https://github.com/rapidsai/cudf/pull/15143 -def _shuffle_group(df, col, _filter, p): - from dask.dataframe.shuffle import ensure_cleanup_on_exception - - with ensure_cleanup_on_exception(p): - g = df.groupby(col) - if hasattr(g, "_grouped"): - # Avoid `get_group` for cudf data. - # See: https://github.com/rapidsai/cudf/issues/14955 - keys, part_offsets, _, grouped_df = df.groupby(col)._grouped() - d = { - k: grouped_df.iloc[part_offsets[i] : part_offsets[i + 1]] - for i, k in enumerate(keys.to_pandas()) - if k in _filter - } - else: - d = {i: g.get_group(i) for i in g.groups if i in _filter} - p.append(d, fsync=True) - - -DiskShuffle._shuffle_group = staticmethod(_shuffle_group) From 6d8e7acefd8ee19fa328ef23fc6b6aa762595876 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 5 Mar 2024 10:45:47 -0800 Subject: [PATCH 74/74] make ci happy --- python/dask_cudf/dask_cudf/expr/_expr.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_expr.py b/python/dask_cudf/dask_cudf/expr/_expr.py index ea980ffc5c8..cbe7a71cb73 100644 --- a/python/dask_cudf/dask_cudf/expr/_expr.py +++ b/python/dask_cudf/dask_cudf/expr/_expr.py @@ -2,7 +2,6 @@ from dask_expr._cumulative import CumulativeBlockwise, TakeLast from dask_expr._reductions import Var -from dask_expr._shuffle import DiskShuffle ## ## Custom expression patching @@ -57,5 +56,3 @@ def _reduction_aggregate(*args, **kwargs): Var.reduction_aggregate = staticmethod(_reduction_aggregate) - -