-
Notifications
You must be signed in to change notification settings - Fork 903
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
Refactor Dask cuDF legacy code #17205
Merged
rapids-bot
merged 22 commits into
rapidsai:branch-24.12
from
rjzamora:refactor-dask-cudf
Nov 4, 2024
Merged
Changes from 10 commits
Commits
Show all changes
22 commits
Select commit
Hold shift + click to select a range
da9b6fb
initial refactor
rjzamora 196c0c0
rename _collection.py to core.py
rjzamora 5be712b
keep _collection name
rjzamora 957299b
move legacy io
rjzamora 86b3fa6
split out _expr.py logic
rjzamora 60853f6
fix patching
rjzamora 56644e8
Merge branch 'branch-24.12' into refactor-dask-cudf
rjzamora db4556f
tweak _patch_dask_expr
rjzamora 8dec940
Merge remote-tracking branch 'upstream/branch-24.12' into refactor-da…
rjzamora e54807f
Merge branch 'refactor-dask-cudf' of github.com:rjzamora/cudf into re…
rjzamora ec139d2
copy upstream changes
rjzamora b5f92c9
Merge remote-tracking branch 'upstream/branch-24.12' into refactor-da…
rjzamora 8fb4657
use Mads' suggestion
rjzamora 1d7c84e
remove core - not worth the tech debt
rjzamora ab56534
Merge remote-tracking branch 'upstream/branch-24.12' into refactor-da…
rjzamora 788cb24
add back core.py
rjzamora 1f54219
add deprecation warnings for various IO methods
rjzamora 95a6697
Merge remote-tracking branch 'upstream/branch-24.12' into refactor-da…
rjzamora 1b36183
add test coverage for _deprecated_api usage
rjzamora bc9897b
fix parquet test mistake
rjzamora a84128a
Merge remote-tracking branch 'upstream/branch-24.12' into refactor-da…
rjzamora ba3032a
Merge branch 'branch-24.12' into refactor-dask-cudf
rjzamora File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
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(): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Most of this PR is just moving around existing code. This change is actually new - We used to implicitly "patch" dask-expr when the |
||
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.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No longer relevant.