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

Allow custom sort functions for dask-cudf sort_values #9789

Merged
merged 7 commits into from
Jan 14, 2022

Conversation

charlesbluca
Copy link
Member

Similar to dask/dask#8345, this PR allows the sorting function called on each partition in last step of dask-cudf's sort_values to be generalized, along with the kwargs that are supplied to it. This allows sort_values to be extended to support more complex ascending / null position handling.

The context for this PR is a desire to simplify the sorting algorithm used by dask-sql; since it only really differs from dask-cudf's sorting algorithm in that it uses a custom sorting function, it seems like it would be easier to allow for that extension upstream rather than duplicate code in dask-sql.

@charlesbluca charlesbluca added 2 - In Progress Currently a work in progress dask-cudf improvement Improvement / enhancement to an existing function non-breaking Non-breaking change labels Nov 29, 2021
@github-actions github-actions bot added the Python Affects Python cuDF API. label Nov 29, 2021
@charlesbluca charlesbluca added 3 - Ready for Review Ready for review by team and removed 2 - In Progress Currently a work in progress labels Nov 29, 2021
@charlesbluca charlesbluca marked this pull request as ready for review November 29, 2021 19:00
@charlesbluca charlesbluca requested a review from a team as a code owner November 29, 2021 19:00
@codecov
Copy link

codecov bot commented Nov 29, 2021

Codecov Report

Merging #9789 (bc9291c) into branch-22.02 (967a333) will decrease coverage by 0.04%.
The diff coverage is n/a.

Impacted file tree graph

@@               Coverage Diff                @@
##           branch-22.02    #9789      +/-   ##
================================================
- Coverage         10.49%   10.44%   -0.05%     
================================================
  Files               119      119              
  Lines             20305    20476     +171     
================================================
+ Hits               2130     2139       +9     
- Misses            18175    18337     +162     
Impacted Files Coverage Δ
python/custreamz/custreamz/kafka.py 29.16% <0.00%> (-0.63%) ⬇️
python/dask_cudf/dask_cudf/sorting.py 92.66% <0.00%> (-0.25%) ⬇️
python/dask_cudf/dask_cudf/core.py 70.85% <0.00%> (-0.17%) ⬇️
python/cudf/cudf/__init__.py 0.00% <0.00%> (ø)
python/cudf/cudf/core/frame.py 0.00% <0.00%> (ø)
python/cudf/cudf/core/index.py 0.00% <0.00%> (ø)
python/cudf/cudf/io/parquet.py 0.00% <0.00%> (ø)
python/cudf/cudf/core/series.py 0.00% <0.00%> (ø)
python/cudf/cudf/utils/utils.py 0.00% <0.00%> (ø)
python/cudf/cudf/utils/dtypes.py 0.00% <0.00%> (ø)
... and 20 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update 0d5ec7f...bc9291c. Read the comment docs.

@charlesbluca
Copy link
Member Author

rerun tests

@charlesbluca
Copy link
Member Author

rerun tests

@github-actions
Copy link

github-actions bot commented Jan 8, 2022

This PR has been labeled inactive-30d due to no recent activity in the past 30 days. Please close this PR if it is no longer required. Otherwise, please respond with a comment indicating any updates. This PR will be labeled inactive-90d if there is no activity in the next 60 days.

Copy link
Member

@rjzamora rjzamora left a comment

Choose a reason for hiding this comment

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

The general change here makes sense to me, thanks for working on this @charlesbluca !

My main comment/suggestion is to avoid "breaking" API changes by moving the default-handling logic into sorting.sort_values.

python/dask_cudf/dask_cudf/core.py Show resolved Hide resolved
Comment on lines 268 to 269
df4 = df3.map_partitions(
M.sort_values, by, ascending=ascending, na_position=na_position
)
df4 = df3.map_partitions(sort_function, **sort_function_kwargs)
Copy link
Member

Choose a reason for hiding this comment

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

Something feels off here. We are requiring that the user specify sort_function, but the API makes it seem optional. I worry that we are now silently ignoring acsending and na_position (and maybe even by?).

What if down-stream users are implementing code with sorting.sort_values directly? I don't think that is good/recommended practice, but the API we are changing seems "public" to me (making this a breaking change).

Perhaps a simpler (non-breaking) solution would be to remove most of the changes from DataFrame.sort_values, pass through sort_function and sort_function_kwargs into here, and implement the sort_function/sort_function_kwargs default logic here (in sorting.sort_values). Does this seem reasonable?

Copy link
Member Author

Choose a reason for hiding this comment

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

That makes sense and is a valid concern - my only comment is that we ideally still want to allow for custom sorting functions in the npartitions == 1 case that is handled directly in DataFrame.sort_values, so I think it might also make sense to move the following logic:

        if self.npartitions == 1:
            df = self.map_partitions(sort_function, **sort_kwargs)

into sorting.sort_values as well, unless there's a reason that's not immediately obvious to me why we would want to keep the single partition case separate?

Also noting that this is also a concern for the upstream implementation of this, so depending on what we decide on here I will open up a follow up PR to address this in Dask.

Copy link
Member

Choose a reason for hiding this comment

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

Also noting that this is also a concern for the upstream implementation of this, so depending on what we decide on here I will open up a follow up PR to address this in Dask.

Good point! I definitely like the simplification you made here. So it probably makes sense to do something similar upstream.

python/dask_cudf/dask_cudf/tests/test_sort.py Show resolved Hide resolved
Copy link
Member

@rjzamora rjzamora left a comment

Choose a reason for hiding this comment

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

Thanks for revising this @charlesbluca! Everything looks great to me.

@charlesbluca
Copy link
Member Author

@gpucibot merge

@rapids-bot rapids-bot bot merged commit ca77542 into rapidsai:branch-22.02 Jan 14, 2022
jsignell pushed a commit to dask/dask that referenced this pull request Feb 18, 2022
This PR moves the handling of custom sorting functions to `shuffle.sort_values`, so that usages of the internal `sort_values` function will not have to manually specify a default `sort_function` and `sort_function_kwargs`.

This originated as a concern in the downstream implementation of this in rapidsai/cudf#9789
@charlesbluca charlesbluca deleted the custom-sort-functions branch July 19, 2022 14:26
@vyasr vyasr added dask Dask issue and removed dask-cudf labels Feb 23, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
3 - Ready for Review Ready for review by team dask Dask issue improvement Improvement / enhancement to an existing function 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