From f39f44c30bcb33923af5b58974adbb9367c63a2a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 24 Apr 2024 08:06:24 -0700 Subject: [PATCH 1/5] deprecate to/from_dask_dataframe --- docs/dask_cudf/source/api.rst | 3 +- python/dask_cudf/dask_cudf/core.py | 30 ++++++++++++++++--- .../dask_cudf/io/tests/test_parquet.py | 2 +- python/dask_cudf/dask_cudf/tests/test_core.py | 26 ++++++++++++++++ .../dask_cudf/dask_cudf/tests/test_groupby.py | 4 +-- 5 files changed, 56 insertions(+), 9 deletions(-) diff --git a/docs/dask_cudf/source/api.rst b/docs/dask_cudf/source/api.rst index db32f4bbcb3..ab10f4af4fa 100644 --- a/docs/dask_cudf/source/api.rst +++ b/docs/dask_cudf/source/api.rst @@ -13,12 +13,11 @@ Creating and storing DataFrames of DataFrames from a variety of storage formats. For on-disk data that are not supported directly in Dask-cuDF, we recommend using Dask's data reading facilities, followed by calling -:func:`.from_dask_dataframe` to obtain a Dask-cuDF object. +:meth:`*.to_backend("cudf")` to obtain a Dask-cuDF object. .. automodule:: dask_cudf :members: from_cudf, - from_dask_dataframe, from_delayed, read_csv, read_json, diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index bfe58531a73..190652d58c9 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -55,9 +55,20 @@ def __repr__(self): @_dask_cudf_nvtx_annotate def to_dask_dataframe(self, **kwargs): - """Create a dask.dataframe object from a dask_cudf object""" - nullable = kwargs.get("nullable", False) - return self.map_partitions(M.to_pandas, nullable=nullable) + """Create a dask.dataframe object from a dask_cudf object + + WARNING: This API is deprecated, and may not work properly + when query-planning is active. Please use `*.to_backend("pandas")` + to convert the underlying data to pandas. + """ + + warnings.warn( + "The `to_dask_dataframe` API is now deprecated. " + "Please use `*.to_backend('pandas')` instead.", + FutureWarning, + ) + + return self.to_backend("pandas", **kwargs) concat = dd.concat @@ -733,6 +744,10 @@ def from_dask_dataframe(df): Convert a Dask :class:`dask.dataframe.DataFrame` to a Dask-cuDF one. + WARNING: This API is deprecated, and may not work properly + when query-planning is active. Please use `*.to_backend("cudf")` + to convert the underlying data to cudf. + Parameters ---------- df : dask.dataframe.DataFrame @@ -742,7 +757,14 @@ def from_dask_dataframe(df): ------- dask_cudf.DataFrame : A new Dask collection backed by cuDF objects """ - return df.map_partitions(cudf.from_pandas) + + warnings.warn( + "The `to_dask_dataframe` API is now deprecated. " + "Please use `*.to_backend('pandas')` instead.", + FutureWarning, + ) + + return df.to_backend("cudf") for name in ( 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 68460653119..8ca27df8fec 100644 --- a/python/dask_cudf/dask_cudf/io/tests/test_parquet.py +++ b/python/dask_cudf/dask_cudf/io/tests/test_parquet.py @@ -113,7 +113,7 @@ def test_roundtrip_from_dask_none_index_false(tmpdir): @pytest.mark.parametrize("write_meta", [True, False]) def test_roundtrip_from_dask_cudf(tmpdir, write_meta): tmpdir = str(tmpdir) - gddf = dask_cudf.from_dask_dataframe(ddf) + gddf = ddf.to_backend("cudf") gddf.to_parquet(tmpdir, write_metadata_file=write_meta) gddf2 = dask_cudf.read_parquet(tmpdir, calculate_divisions=True) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index c6918c94559..af59df04053 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -32,6 +32,32 @@ def test_from_dict_backend_dispatch(): dd.assert_eq(expect, ddf) +@xfail_dask_expr("API collision with dask-expr") +def test_to_dask_dataframe_deprecated(): + gdf = cudf.DataFrame({"a": range(100)}) + ddf = dd.from_pandas(gdf, npartitions=2) + assert isinstance(ddf._meta, cudf.DataFrame) + + with pytest.warns(FutureWarning, match="API is now deprecated"): + assert isinstance( + ddf.to_dask_dataframe()._meta, + pd.DataFrame, + ) + + +@xfail_dask_expr("API collision with dask-expr") +def test_from_dask_dataframe_deprecated(): + gdf = pd.DataFrame({"a": range(100)}) + ddf = dd.from_pandas(gdf, npartitions=2) + assert isinstance(ddf._meta, pd.DataFrame) + + with pytest.warns(FutureWarning, match="API is now deprecated"): + assert isinstance( + dask_cudf.from_dask_dataframe(ddf)._meta, + cudf.DataFrame, + ) + + def test_to_backend(): np.random.seed(0) data = { diff --git a/python/dask_cudf/dask_cudf/tests/test_groupby.py b/python/dask_cudf/dask_cudf/tests/test_groupby.py index 3bb3e3b0bb8..1e22dd95475 100644 --- a/python/dask_cudf/dask_cudf/tests/test_groupby.py +++ b/python/dask_cudf/dask_cudf/tests/test_groupby.py @@ -562,9 +562,9 @@ def test_groupby_reset_index_string_name(): def test_groupby_categorical_key(): # See https://github.com/rapidsai/cudf/issues/4608 df = dask.datasets.timeseries() - gddf = dask_cudf.from_dask_dataframe(df) + gddf = df.to_backend("cudf") gddf["name"] = gddf["name"].astype("category") - ddf = gddf.to_dask_dataframe() + ddf = gddf.to_backend("pandas") got = gddf.groupby("name", sort=True).agg( {"x": ["mean", "max"], "y": ["mean", "count"]} From f484049e7dda6501e8037d03a906f5873580f3a6 Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Mon, 29 Apr 2024 12:50:03 -0500 Subject: [PATCH 2/5] Apply suggestions from code review Co-authored-by: Charles Blackmon-Luca <20627856+charlesbluca@users.noreply.github.com> --- python/dask_cudf/dask_cudf/core.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 190652d58c9..3f0cfeb6d2c 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -759,8 +759,8 @@ def from_dask_dataframe(df): """ warnings.warn( - "The `to_dask_dataframe` API is now deprecated. " - "Please use `*.to_backend('pandas')` instead.", + "The `from_dask_dataframe` API is now deprecated. " + "Please use `*.to_backend('cudf')` instead.", FutureWarning, ) From 820927bc279a13de6113712fd9015f9750890257 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 29 Apr 2024 11:39:34 -0700 Subject: [PATCH 3/5] override upstream API --- .../dask_cudf/dask_cudf/expr/_collection.py | 25 ++++++++++++++++--- python/dask_cudf/dask_cudf/tests/test_core.py | 2 -- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 516e35a4335..ffac5b776e8 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -1,5 +1,6 @@ # Copyright (c) 2024, NVIDIA CORPORATION. +import warnings from functools import cached_property from dask_expr import ( @@ -49,7 +50,25 @@ def var( ) -class DataFrame(VarMixin, DXDataFrame): +class CommonMixin: + def to_dask_dataframe(self, **kwargs): + """Create a dask.dataframe object from a dask_cudf object + + WARNING: This API is deprecated, and may not work properly. + Please use `*.to_backend("pandas")` to convert the + underlying data to pandas. + """ + + warnings.warn( + "The `to_dask_dataframe` API is now deprecated. " + "Please use `*.to_backend('pandas')` instead.", + FutureWarning, + ) + + return self.to_backend("pandas", **kwargs) + + +class DataFrame(VarMixin, CommonMixin, DXDataFrame): @classmethod def from_dict(cls, *args, **kwargs): with config.set({"dataframe.backend": "cudf"}): @@ -94,7 +113,7 @@ def read_text(*args, **kwargs): return from_legacy_dataframe(ddf) -class Series(VarMixin, DXSeries): +class Series(VarMixin, CommonMixin, DXSeries): def groupby(self, by, **kwargs): from dask_cudf.expr._groupby import SeriesGroupBy @@ -113,7 +132,7 @@ def struct(self): return StructMethods(self) -class Index(DXIndex): +class Index(CommonMixin, DXIndex): pass # Same as pandas (for now) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index af59df04053..4878d44d636 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -32,7 +32,6 @@ def test_from_dict_backend_dispatch(): dd.assert_eq(expect, ddf) -@xfail_dask_expr("API collision with dask-expr") def test_to_dask_dataframe_deprecated(): gdf = cudf.DataFrame({"a": range(100)}) ddf = dd.from_pandas(gdf, npartitions=2) @@ -45,7 +44,6 @@ def test_to_dask_dataframe_deprecated(): ) -@xfail_dask_expr("API collision with dask-expr") def test_from_dask_dataframe_deprecated(): gdf = pd.DataFrame({"a": range(100)}) ddf = dd.from_pandas(gdf, npartitions=2) From d0c2b1243a907bbd0b45b05f0349e56782cf5851 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 30 Apr 2024 07:43:25 -0700 Subject: [PATCH 4/5] Use CudfFrameBase --- .../dask_cudf/dask_cudf/expr/_collection.py | 46 +++++++++---------- 1 file changed, 22 insertions(+), 24 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index ffac5b776e8..7196c53deab 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -23,9 +23,25 @@ ## -# VarMixin can be removed if cudf#15179 is addressed. -# See: https://github.com/rapidsai/cudf/issues/15179 -class VarMixin: +class CudfFrameBase(FrameBase): + def to_dask_dataframe(self, **kwargs): + """Create a dask.dataframe object from a dask_cudf object + + WARNING: This API is deprecated, and may not work properly. + Please use `*.to_backend("pandas")` to convert the + underlying data to pandas. + """ + + warnings.warn( + "The `to_dask_dataframe` API is now deprecated. " + "Please use `*.to_backend('pandas')` instead.", + FutureWarning, + ) + + return self.to_backend("pandas", **kwargs) + + # var can be removed if cudf#15179 is addressed. + # See: https://github.com/rapidsai/cudf/issues/15179 def var( self, axis=0, @@ -50,25 +66,7 @@ def var( ) -class CommonMixin: - def to_dask_dataframe(self, **kwargs): - """Create a dask.dataframe object from a dask_cudf object - - WARNING: This API is deprecated, and may not work properly. - Please use `*.to_backend("pandas")` to convert the - underlying data to pandas. - """ - - warnings.warn( - "The `to_dask_dataframe` API is now deprecated. " - "Please use `*.to_backend('pandas')` instead.", - FutureWarning, - ) - - return self.to_backend("pandas", **kwargs) - - -class DataFrame(VarMixin, CommonMixin, DXDataFrame): +class DataFrame(CudfFrameBase, DXDataFrame): @classmethod def from_dict(cls, *args, **kwargs): with config.set({"dataframe.backend": "cudf"}): @@ -113,7 +111,7 @@ def read_text(*args, **kwargs): return from_legacy_dataframe(ddf) -class Series(VarMixin, CommonMixin, DXSeries): +class Series(CudfFrameBase, DXSeries): def groupby(self, by, **kwargs): from dask_cudf.expr._groupby import SeriesGroupBy @@ -132,7 +130,7 @@ def struct(self): return StructMethods(self) -class Index(CommonMixin, DXIndex): +class Index(CudfFrameBase, DXIndex): pass # Same as pandas (for now) From 0627e66bdf76ae9d2c5a9541ac3e7a10b140d187 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 30 Apr 2024 08:07:33 -0700 Subject: [PATCH 5/5] modify inheritance order --- python/dask_cudf/dask_cudf/expr/_collection.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/expr/_collection.py b/python/dask_cudf/dask_cudf/expr/_collection.py index 7196c53deab..605a81f0fcd 100644 --- a/python/dask_cudf/dask_cudf/expr/_collection.py +++ b/python/dask_cudf/dask_cudf/expr/_collection.py @@ -66,7 +66,7 @@ def var( ) -class DataFrame(CudfFrameBase, DXDataFrame): +class DataFrame(DXDataFrame, CudfFrameBase): @classmethod def from_dict(cls, *args, **kwargs): with config.set({"dataframe.backend": "cudf"}): @@ -111,7 +111,7 @@ def read_text(*args, **kwargs): return from_legacy_dataframe(ddf) -class Series(CudfFrameBase, DXSeries): +class Series(DXSeries, CudfFrameBase): def groupby(self, by, **kwargs): from dask_cudf.expr._groupby import SeriesGroupBy @@ -130,7 +130,7 @@ def struct(self): return StructMethods(self) -class Index(CudfFrameBase, DXIndex): +class Index(DXIndex, CudfFrameBase): pass # Same as pandas (for now)