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

Fix categorical-accessor support and testing in dask-cudf #15591

Merged
merged 18 commits into from
May 1, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
18 commits
Select commit Hold shift + click to select a range
cb046bb
fix categorical support for dask-expr - needs upstream fix to clear_k…
rjzamora Apr 19, 2024
7a45bde
Merge remote-tracking branch 'upstream/branch-24.06' into fix-categor…
rjzamora Apr 22, 2024
057fdcb
Merge remote-tracking branch 'upstream/branch-24.06' into fix-categor…
rjzamora Apr 24, 2024
fde1651
adjust tests
rjzamora Apr 24, 2024
c0d972f
Merge remote-tracking branch 'upstream/branch-24.06' into fix-categor…
rjzamora Apr 25, 2024
c2bc812
roll back Series.__dask_tokenize__ change in favor of simpler fix
rjzamora Apr 25, 2024
0f01712
normalize categories just in case the list is too long for repr
rjzamora Apr 25, 2024
b4e7c66
Update python/cudf/cudf/core/frame.py
rjzamora Apr 25, 2024
d901e20
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora Apr 26, 2024
f38d62e
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora Apr 26, 2024
e229267
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora Apr 29, 2024
ea0616b
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora Apr 29, 2024
6f0ee4c
Merge remote-tracking branch 'upstream/branch-24.06' into fix-categor…
rjzamora Apr 30, 2024
5432d20
use dask version instead of dask-expr version for lt_version
rjzamora Apr 30, 2024
164cc2d
update test
rjzamora Apr 30, 2024
5913c8d
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora Apr 30, 2024
e41ad69
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora May 1, 2024
88e8383
Merge branch 'branch-24.06' into fix-categorical-support
rjzamora May 1, 2024
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
12 changes: 12 additions & 0 deletions python/cudf/cudf/core/series.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@
to_cudf_compatible_scalar,
)
from cudf.utils.nvtx_annotation import _cudf_nvtx_annotate
from cudf.utils.utils import _warn_no_dask_cudf


def _format_percentile_names(percentiles):
Expand Down Expand Up @@ -3658,6 +3659,17 @@ def where(self, cond, other=None, inplace=False):
inplace=inplace,
)

@_cudf_nvtx_annotate
@_warn_no_dask_cudf
def __dask_tokenize__(self):
from dask.base import normalize_token

return [
type(self),
str(self.dtype),
normalize_token(self.to_pandas()),
]
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm honestly not sure why the Frame.__dask_tokenize__ definition (which looks very similar) isn't being used?

Whatever the reason may be, test_categorical_compare_ordered fails without this fix, because different Series objects end up being tokenized to the same value, and the corresponding expressions are cached between tests. (general message: Unique/proper tokenization is very important when query-planning is active)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like before this was using IndexedFrame.__dask_tokenize__ where the data was being hashed as self.hash_values().values_host as opposed to self.to_pandas() (might be difference here for categorical)

Might be a better fix if IndexedFrame.__dask_tokenize__ uses to_pandas() instead of self.hash_values? Additionally if we want to use this fix as-is I think we would need to also incorporate the self.index?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like before this was using IndexedFrame.dask_tokenize where the data was being hashed as self.hash_values().values_host as opposed to self.to_pandas() (might be difference here for categorical)

Aha! Thanks for pointing that out @mroeschke !

This is actually a problem we ran into before and fixed in Frame.__dask_tokenize__. It turns out that normalize_token(self._dtypes) doesn't work very well. The more reliable thing to do is actually use str(self._dtypes). With that said, dtypes with many categories may not be completely/well represented by str(self._dtypes). Therefore, I just added an extra line to explicitly normalize the actual categories for each categorical dtype.

Might be a better fix if IndexedFrame.dask_tokenize uses to_pandas() instead of self.hash_values?

I think you are right that this is probably the safest and most robust thing to do. However, I am still hesitant to remove the hash_values code path. Right now, we avoid moving more than two columns (the hashed values, and the index) to host memory when a cudf object is tokenized. The overhead difference may not be dramatic, but it would be nice to avoid moving the whole thing to pandas.



def make_binop_func(op):
# This function is used to wrap binary operations in Frame with an
Expand Down
19 changes: 15 additions & 4 deletions python/dask_cudf/dask_cudf/tests/test_accessor.py
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,8 @@ def test_categorical_accessor_initialization2(data):
dsr.cat


@xfail_dask_expr("TODO: Unexplained dask-expr failure")
# TODO: Remove this once we are pinned to dask>=2024.5.0
@xfail_dask_expr("Requires: https://github.com/dask/dask/pull/11059")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wonder if the lt_version param of this marker should also account for the dask-core version, since the dask-expr doesn't have a super established release cycle yet?

That way, in addition to leaving this TODO we could also do something like lt_version=2024.5.0 to make sure that things fail loudly here once that dask-core version becomes available.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I was thinking the same thing. I was actually going to submit a dedicated PR to revise the xfail_dask_expr utility, but might as well do it here :)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay - Thanks again for the suggestion. The xfail_dask_expr/skip_dask_expr utilities have been updated.

@pytest.mark.parametrize("data", [data_cat_1()])
def test_categorical_basic(data):
cat = data.copy()
Expand Down Expand Up @@ -203,7 +204,6 @@ 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].copy()
Expand Down Expand Up @@ -274,7 +274,6 @@ 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")
Expand All @@ -283,7 +282,19 @@ def test_categorical_as_known():
pdf = dd.from_pandas(pd.DataFrame({"col_1": [0, 1, 2, 3]}), npartitions=2)
pdf["col_1"] = pdf["col_1"].astype("category")
expected = pdf["col_1"].cat.as_known()
dd.assert_eq(expected, actual)

# Note: Categories may be ordered differently in
# cudf and pandas. Therefore, we need to compare
# the global set of categories (before and after
# calling `compute`), then we need to check that
# the initial order of rows was preserved.
assert set(expected.cat.categories) == set(
actual.cat.categories.values_host
)
assert set(expected.compute().cat.categories) == set(
actual.compute().cat.categories.values_host
)
dd.assert_eq(expected, actual.astype(expected.dtype))


def test_str_slice():
Expand Down
Loading