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

[Data] support batch_format for Sort and Aggregate #48287

Merged
merged 8 commits into from
Nov 13, 2024

Conversation

xingyu-long
Copy link
Contributor

@xingyu-long xingyu-long commented Oct 27, 2024

Why are these changes needed?

While we calling xxx.map_groups(..., batch_format="..."), we may invoke sort function and creating empty blocks which still uses pyarrow by default. And, when we invoke another sort call on top of it, we will hit AttributeError: 'DataFrame' object has no attribute 'num_rows' since we uses first block type. (However, we may have different blocks). See more details in #46748

Related issue number

Close #46748

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@xingyu-long
Copy link
Contributor Author

Hi @scottjlee, could you take a look when you have time? Thanks a lot!

Copy link
Contributor

@scottjlee scottjlee left a comment

Choose a reason for hiding this comment

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

Great start, have a suggestion on how we can set the batch_format without having it explicitly set at Dataset level.

@xingyu-long xingyu-long requested a review from scottjlee October 29, 2024 05:04
@xingyu-long xingyu-long requested a review from scottjlee November 3, 2024 18:39
@xingyu-long xingyu-long changed the title [Data] support batch_format for Sort [Data] support batch_format for Sort and Aggregate Nov 3, 2024
Copy link
Contributor

@scottjlee scottjlee 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 your contribution @xingyu-long !

@scottjlee scottjlee self-assigned this Nov 8, 2024
@scottjlee scottjlee added the go add ONLY when ready to merge, run all tests label Nov 8, 2024
Comment on lines +22 to +25
for node in op.post_order_iter():
nodes.appendleft(node)

while len(nodes) > 0:
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's combine these 2 loops

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Looking through the codebase, I was trying to use same coding style for this matter. it seems not worth it to combine? Thanks!

Copy link
Contributor

Choose a reason for hiding this comment

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

We can't just blindly follow the patterns, these need to make sense, right?

What's the motivation for first collecting the nodes into a queue, instead of traversing the iterator directly?

Signed-off-by: Xingyu Long <[email protected]>
@scottjlee scottjlee enabled auto-merge (squash) November 13, 2024 18:23
@scottjlee scottjlee merged commit 3f195b4 into ray-project:master Nov 13, 2024
6 checks passed
JP-sDEV pushed a commit to JP-sDEV/ray that referenced this pull request Nov 14, 2024
## Why are these changes needed?
While we calling `xxx.map_groups(..., batch_format="...")`, we may
invoke sort function and creating empty blocks which still uses pyarrow
by default. And, when we invoke another sort call on top of it, we will
hit `AttributeError: 'DataFrame' object has no attribute 'num_rows'`
since we uses first block type. (However, we may have different blocks).
See more details in ray-project#46748

## Related issue number

Close ray-project#46748

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [x] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Xingyu Long <[email protected]>
Co-authored-by: Scott Lee <[email protected]>
mohitjain2504 pushed a commit to mohitjain2504/ray that referenced this pull request Nov 15, 2024
## Why are these changes needed?
While we calling `xxx.map_groups(..., batch_format="...")`, we may
invoke sort function and creating empty blocks which still uses pyarrow
by default. And, when we invoke another sort call on top of it, we will
hit `AttributeError: 'DataFrame' object has no attribute 'num_rows'`
since we uses first block type. (However, we may have different blocks).
See more details in ray-project#46748

## Related issue number

Close ray-project#46748

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [x] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Xingyu Long <[email protected]>
Co-authored-by: Scott Lee <[email protected]>
Signed-off-by: mohitjain2504 <[email protected]>
dentiny pushed a commit to dentiny/ray that referenced this pull request Dec 7, 2024
## Why are these changes needed?
While we calling `xxx.map_groups(..., batch_format="...")`, we may
invoke sort function and creating empty blocks which still uses pyarrow
by default. And, when we invoke another sort call on top of it, we will
hit `AttributeError: 'DataFrame' object has no attribute 'num_rows'`
since we uses first block type. (However, we may have different blocks).
See more details in ray-project#46748

## Related issue number

Close ray-project#46748

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [x] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Xingyu Long <[email protected]>
Co-authored-by: Scott Lee <[email protected]>
Signed-off-by: hjiang <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Data] Sorting on grouped data not working with pandas format
3 participants