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

Conversation

rjzamora
Copy link
Member

Description

Related to #15027

Adds a minor tokenization fix, and adjusts testing for categorical-accessor support.

Checklist

  • I am familiar with the Contributing Guidelines.
  • New or existing tests cover these changes.
  • The documentation is up to date with these changes.

@rjzamora rjzamora added bug Something isn't working 2 - In Progress Currently a work in progress dask Dask issue non-breaking Non-breaking change labels Apr 24, 2024
@rjzamora rjzamora self-assigned this Apr 24, 2024
@github-actions github-actions bot added the Python Affects Python cuDF API. label Apr 24, 2024
Comment on lines 3662 to 3671
@_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.

@rjzamora rjzamora marked this pull request as ready for review April 24, 2024 15:18
@rjzamora rjzamora requested review from a team as code owners April 24, 2024 15:18
@rjzamora rjzamora added 3 - Ready for Review Ready for review by team and removed 2 - In Progress Currently a work in progress labels Apr 24, 2024
@@ -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.

@rjzamora rjzamora added 5 - Ready to Merge Testing and reviews complete, ready to merge and removed 3 - Ready for Review Ready for review by team labels May 1, 2024
@rjzamora
Copy link
Member Author

rjzamora commented May 1, 2024

/merge

@rapids-bot rapids-bot bot merged commit 67d427d into rapidsai:branch-24.06 May 1, 2024
69 checks passed
@rjzamora rjzamora deleted the fix-categorical-support branch May 1, 2024 21:09
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
5 - Ready to Merge Testing and reviews complete, ready to merge bug Something isn't working dask Dask issue non-breaking Non-breaking change Python Affects Python cuDF API.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants