Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support conversion to/from cudf in dask.dataframe.core.to_backend #12380

Merged
merged 24 commits into from
Feb 8, 2023
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
Show all changes
24 commits
Select commit Hold shift + click to select a range
fd190ab
support DataFrame.to_backend in dask_cudf
rjzamora Dec 13, 2022
29eb466
formatting
rjzamora Dec 13, 2022
5df1191
Merge branch 'branch-23.02' into to-backend
rjzamora Jan 27, 2023
b5b8337
trigger formatting
rjzamora Jan 27, 2023
ceff3e2
roll back comment chang
rjzamora Jan 27, 2023
a995644
Merge branch 'branch-23.02' into to-backend
rjzamora Jan 30, 2023
a08e7d1
update test
rjzamora Jan 30, 2023
7592fd5
remove try/except
rjzamora Jan 30, 2023
0c6e8f1
Merge remote-tracking branch 'upstream/branch-23.02' into to-backend
rjzamora Jan 30, 2023
6c72cb7
move to 23.04
rjzamora Jan 31, 2023
d685001
Merge remote-tracking branch 'upstream/branch-23.04' into to-backend
rjzamora Jan 31, 2023
7916664
improve test coverage
rjzamora Jan 31, 2023
d89825e
Merge branch 'branch-23.04' into to-backend
rjzamora Jan 31, 2023
351232b
Merge branch 'branch-23.04' into to-backend
rjzamora Jan 31, 2023
ed04e3c
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 1, 2023
b0c0193
Merge remote-tracking branch 'upstream/branch-23.04' into to-backend
rjzamora Feb 1, 2023
7624864
roll back unnecessary change
rjzamora Feb 1, 2023
0242b5e
Merge branch 'to-backend' of https://github.com/rjzamora/cudf into to…
rjzamora Feb 1, 2023
46a99cb
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 2, 2023
d56ff3f
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 3, 2023
e3239b0
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 6, 2023
1099f8b
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 6, 2023
7d10ccf
Merge branch 'branch-23.04' into to-backend
rjzamora Feb 7, 2023
8f947b4
Merge branch 'branch-23.04' into to-backend
galipremsagar Feb 8, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
33 changes: 30 additions & 3 deletions python/dask_cudf/dask_cudf/backends.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
make_meta_obj,
)
from dask.sizeof import sizeof as sizeof_dispatch
from dask.utils import is_arraylike
from dask.utils import is_arraylike, Dispatch

import cudf
from cudf.api.types import is_string_dtype
Expand Down Expand Up @@ -447,10 +447,26 @@ def _default_backend(func, *args, **kwargs):


try:
from dask.dataframe.backends import DataFrameBackendEntrypoint, PandasBackendEntrypoint
rjzamora marked this conversation as resolved.
Show resolved Hide resolved

# Define "cudf" backend engine to be registered with Dask
from dask.dataframe.backends import DataFrameBackendEntrypoint
if hasattr(PandasBackendEntrypoint, "to_backend_dispatch"):
to_pandas_dispatch = PandasBackendEntrypoint.to_backend_dispatch()

@to_pandas_dispatch.register((cudf.DataFrame, cudf.Series, cudf.Index))
def to_pandas_dispatch_from_cudf(data, **kwargs):
return data.to_pandas(**kwargs)

to_cudf_dispatch = Dispatch("to_cudf_dispatch")

@to_cudf_dispatch.register((pd.DataFrame, pd.Series, pd.Index))
def to_cudf_dispatch_from_pandas(data, **kwargs):
return cudf.from_pandas(data, **kwargs)

@to_cudf_dispatch.register((cudf.DataFrame, cudf.Series, cudf.Index))
def to_cudf_dispatch_from_cudf(data, **kwargs):
return data

# Define "cudf" backend engine to be registered with Dask
class CudfBackendEntrypoint(DataFrameBackendEntrypoint):
"""Backend-entrypoint class for Dask-DataFrame

Expand All @@ -470,6 +486,17 @@ class CudfBackendEntrypoint(DataFrameBackendEntrypoint):
<class 'dask_cudf.core.DataFrame'>
"""

@classmethod
def to_backend_dispatch(cls):
return to_cudf_dispatch

@classmethod
def to_backend(cls, data: dd.core._Frame, **kwargs):
if isinstance(data._meta, (cudf.DataFrame, cudf.Series, cudf.Index)):
# Already a cudf-backed collection
return data
return data.map_partitions(cls.to_backend_dispatch(), **kwargs)

@staticmethod
def from_dict(
data,
Expand Down
25 changes: 23 additions & 2 deletions python/dask_cudf/dask_cudf/tests/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
import numpy as np
import pandas as pd
import pytest
from packaging import version

import dask
from dask import dataframe as dd
Expand All @@ -31,6 +32,28 @@ def test_from_dict_backend_dispatch():
dd.assert_eq(expect, ddf)


def test_to_backend():
# Test DataFrame.to_backend
# Requires Dask>=2023.1.1
if version.parse(dask.__version__) < version.parse("2023.1.1"):
pytest.skip("to_backend not supported in this version of Dask.")
rjzamora marked this conversation as resolved.
Show resolved Hide resolved

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, dgd.DataFrame)
dd.assert_eq(cudf.DataFrame(data), ddf)

assert isinstance(gdf.to_backend()._meta, pd.DataFrame)


def test_from_cudf():
np.random.seed(0)

Expand Down Expand Up @@ -547,8 +570,6 @@ def test_unary_ops(func, gdf, gddf):

# Fixed in https://github.com/dask/dask/pull/4657
if isinstance(p, cudf.Index):
from packaging import version

if version.parse(dask.__version__) < version.parse("1.1.6"):
pytest.skip(
"dask.dataframe assert_eq index check hardcoded to "
Expand Down