-
Notifications
You must be signed in to change notification settings - Fork 908
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Refactor Dask cuDF legacy code (#17205)
The "legacy" DataFrame API is now deprecated (dask/dask#11437). The main purpose of this PR is to start isolating legacy code in Dask cuDF. **Old layout**: ``` dask_cudf/ ├── expr/ │ ├── _collection.py │ ├── _expr.py │ ├── _groupby.py ├── io/ │ ├── tests/ │ ├── ... │ ├── parquet.py │ ├── ... ├── tests/ ├── accessors.py ├── backends.py ├── core.py ├── groupby.py ├── sorting.py ``` **New layout**: ``` dask_cudf/ ├── _expr/ │ ├── accessors.py │ ├── collection.py │ ├── expr.py │ ├── groupby.py ├── _legacy/ │ ├── io/ │ ├── core.py │ ├── groupby.py │ ├── sorting.py ├── io/ │ ├── tests/ │ ├── ... │ ├── parquet.py │ ├── ... ├── tests/ ├── backends.py ├── core.py ``` **Notes** - This PR adds some backward compatibility to the expr-based API that was previously missing: The user can now import collection classes from `dask_cudf.core` (previously led to a "silent" bug when query-planning was enabled). - The user can also import various IO functions from `dask_cudf.io` (and sub-modules like `dask_cudf.io.parquet`), but they will typically get a deprecation warning. - This PR is still technically "breaking" in the sense that the user can no longer import *some* functions/classes from `dask_cudf.io.*`. Also, the `groupby`, `sorting`, and `accessors` modules have simply moved. It *should* be uncommon for down-stream code to import from these modules. It's also worth noting that query-planning was already causing problems for these users if they *were* doing this. Authors: - Richard (Rick) Zamora (https://github.com/rjzamora) Approvers: - Mads R. B. Kristensen (https://github.com/madsbk) URL: #17205
- Loading branch information
Showing
35 changed files
with
2,795 additions
and
2,674 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
# Copyright (c) 2024, NVIDIA CORPORATION. |
File renamed without changes.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,210 @@ | ||
# Copyright (c) 2024, NVIDIA CORPORATION. | ||
import functools | ||
|
||
import dask_expr._shuffle as _shuffle_module | ||
from dask_expr import new_collection | ||
from dask_expr._cumulative import CumulativeBlockwise | ||
from dask_expr._expr import Elemwise, Expr, RenameAxis, VarColumns | ||
from dask_expr._reductions import Reduction, Var | ||
|
||
from dask.dataframe.core import ( | ||
is_dataframe_like, | ||
make_meta, | ||
meta_nonempty, | ||
) | ||
from dask.dataframe.dispatch import is_categorical_dtype | ||
from dask.typing import no_default | ||
|
||
import cudf | ||
|
||
## | ||
## Custom expressions | ||
## | ||
|
||
|
||
class RenameAxisCudf(RenameAxis): | ||
# TODO: Remove this after rename_axis is supported in cudf | ||
# (See: https://github.com/rapidsai/cudf/issues/16895) | ||
@staticmethod | ||
def operation(df, index=no_default, **kwargs): | ||
if index != no_default: | ||
df.index.name = index | ||
return df | ||
raise NotImplementedError( | ||
"Only `index` is supported for the cudf backend" | ||
) | ||
|
||
|
||
class ToCudfBackend(Elemwise): | ||
# TODO: Inherit from ToBackend when rapids-dask-dependency | ||
# is pinned to dask>=2024.8.1 | ||
_parameters = ["frame", "options"] | ||
_projection_passthrough = True | ||
_filter_passthrough = True | ||
_preserves_partitioning_information = True | ||
|
||
@staticmethod | ||
def operation(df, options): | ||
from dask_cudf.backends import to_cudf_dispatch | ||
|
||
return to_cudf_dispatch(df, **options) | ||
|
||
def _simplify_down(self): | ||
if isinstance( | ||
self.frame._meta, (cudf.DataFrame, cudf.Series, cudf.Index) | ||
): | ||
# We already have cudf data | ||
return self.frame | ||
|
||
|
||
## | ||
## Custom expression patching | ||
## | ||
|
||
|
||
# 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: | ||
return self.operands[:1] | ||
|
||
@property | ||
def _kwargs(self) -> dict: | ||
# Must pass axis and skipna as kwargs in cudf | ||
return {"axis": self.axis, "skipna": self.skipna} | ||
|
||
|
||
# The upstream Var code uses `Series.values`, and relies on numpy | ||
# for most of the logic. Unfortunately, cudf -> cupy conversion | ||
# is not supported for data containing null values. Therefore, | ||
# we must implement our own version of Var for now. This logic | ||
# is mostly copied from dask-cudf. | ||
|
||
|
||
class VarCudf(Reduction): | ||
# Uses the parallel version of Welford's online algorithm (Chan '79) | ||
# (http://i.stanford.edu/pub/cstr/reports/cs/tr/79/773/CS-TR-79-773.pdf) | ||
_parameters = [ | ||
"frame", | ||
"skipna", | ||
"ddof", | ||
"numeric_only", | ||
"split_every", | ||
] | ||
_defaults = { | ||
"skipna": True, | ||
"ddof": 1, | ||
"numeric_only": False, | ||
"split_every": False, | ||
} | ||
|
||
@functools.cached_property | ||
def _meta(self): | ||
return make_meta( | ||
meta_nonempty(self.frame._meta).var( | ||
skipna=self.skipna, numeric_only=self.numeric_only | ||
) | ||
) | ||
|
||
@property | ||
def chunk_kwargs(self): | ||
return dict(skipna=self.skipna, numeric_only=self.numeric_only) | ||
|
||
@property | ||
def combine_kwargs(self): | ||
return {} | ||
|
||
@property | ||
def aggregate_kwargs(self): | ||
return dict(ddof=self.ddof) | ||
|
||
@classmethod | ||
def reduction_chunk(cls, x, skipna=True, numeric_only=False): | ||
kwargs = {"numeric_only": numeric_only} if is_dataframe_like(x) else {} | ||
if skipna or numeric_only: | ||
n = x.count(**kwargs) | ||
kwargs["skipna"] = skipna | ||
avg = x.mean(**kwargs) | ||
else: | ||
# Not skipping nulls, so might as well | ||
# avoid the full `count` operation | ||
n = len(x) | ||
kwargs["skipna"] = skipna | ||
avg = x.sum(**kwargs) / n | ||
if numeric_only: | ||
# Workaround for cudf bug | ||
# (see: https://github.com/rapidsai/cudf/issues/13731) | ||
x = x[n.index] | ||
m2 = ((x - avg) ** 2).sum(**kwargs) | ||
return n, avg, m2 | ||
|
||
@classmethod | ||
def reduction_combine(cls, parts): | ||
n, avg, m2 = parts[0] | ||
for i in range(1, len(parts)): | ||
n_a, avg_a, m2_a = n, avg, m2 | ||
n_b, avg_b, m2_b = parts[i] | ||
n = n_a + n_b | ||
avg = (n_a * avg_a + n_b * avg_b) / n | ||
delta = avg_b - avg_a | ||
m2 = m2_a + m2_b + delta**2 * n_a * n_b / n | ||
return n, avg, m2 | ||
|
||
@classmethod | ||
def reduction_aggregate(cls, vals, ddof=1): | ||
vals = cls.reduction_combine(vals) | ||
n, _, m2 = vals | ||
return m2 / (n - ddof) | ||
|
||
|
||
def _patched_var( | ||
self, | ||
axis=0, | ||
skipna=True, | ||
ddof=1, | ||
numeric_only=False, | ||
split_every=False, | ||
): | ||
if axis == 0: | ||
if hasattr(self._meta, "to_pandas"): | ||
return VarCudf(self, skipna, ddof, numeric_only, split_every) | ||
else: | ||
return Var(self, skipna, ddof, numeric_only, split_every) | ||
elif axis == 1: | ||
return VarColumns(self, skipna, ddof, numeric_only) | ||
else: | ||
raise ValueError(f"axis={axis} not supported. Please specify 0 or 1") | ||
|
||
|
||
# Temporary work-around for missing cudf + categorical support | ||
# See: https://github.com/rapidsai/cudf/issues/11795 | ||
# TODO: Fix RepartitionQuantiles and remove this in cudf>24.06 | ||
|
||
_original_get_divisions = _shuffle_module._get_divisions | ||
|
||
|
||
def _patched_get_divisions(frame, other, *args, **kwargs): | ||
# NOTE: The following two lines contains the "patch" | ||
# (we simply convert the partitioning column to pandas) | ||
if is_categorical_dtype(other._meta.dtype) and hasattr( | ||
other.frame._meta, "to_pandas" | ||
): | ||
other = new_collection(other).to_backend("pandas")._expr | ||
|
||
# Call "original" function | ||
return _original_get_divisions(frame, other, *args, **kwargs) | ||
|
||
|
||
_PATCHED = False | ||
|
||
|
||
def _patch_dask_expr(): | ||
global _PATCHED | ||
|
||
if not _PATCHED: | ||
CumulativeBlockwise._args = PatchCumulativeBlockwise._args | ||
CumulativeBlockwise._kwargs = PatchCumulativeBlockwise._kwargs | ||
Expr.var = _patched_var | ||
_shuffle_module._get_divisions = _patched_get_divisions | ||
_PATCHED = True |
Oops, something went wrong.