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] Ray Data heap OOMs when using fused pandas #46785

Closed
bveeramani opened this issue Jul 25, 2024 · 0 comments · Fixed by #46939
Closed

[Data] Ray Data heap OOMs when using fused pandas #46785

bveeramani opened this issue Jul 25, 2024 · 0 comments · Fixed by #46939
Assignees
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P2 Important issue, but not time-critical

Comments

@bveeramani
Copy link
Member

bveeramani commented Jul 25, 2024

What happened + What you expected to happen

Ray Data map tasks buffer outputs until the buffer contains at least 128 MiB of data. However, Ray Data doesn't correctly count the size of some pandas DataFrames, so if you use the pandas batch format, Ray Data might buffer too much data and your task will heap OOM.

See the related #44577 for more information.

Versions / Dependencies

2.32

Reproduction script

This doesn't work.

import numpy as np

import ray

ray.init(num_cpus=1)


def yield_blocks(batch):
    while True:
        yield {"data": np.zeros((128, 1024, 1024), dtype=np.uint8)}


def convert_to_dataframe(batch):
    return batch


ds = (
    ray.data.range(1, override_num_blocks=1)
    .map_batches(yield_blocks, batch_size=None)
    .map_batches(convert_to_dataframe, batch_size=None, batch_format="pandas")
)
for _ in ds.iter_batches(batch_size=None):  # This should eventually heap OOM
    pass

This does work. For some reason, pandas' memory estimate is correct in this case but not the previous.

import numpy as np
import pandas as pd

import ray

ray.init(num_cpus=1)


def yield_blocks(block):
    while True:
        yield pd.DataFrame({"data": 128 * [np.zeros((1024, 1024), dtype=np.uint8)]})


ds = ray.data.range(1, override_num_blocks=1).map_batches(yield_blocks, batch_size=None)
for _ in ds.iter_batches(batch_size=None):
    pass

Issue Severity

Medium: It is a significant difficulty but I can work around it.

@bveeramani bveeramani added bug Something that is supposed to be working; but isn't P0 Issues that should be fixed in short order data Ray Data-related issues labels Jul 25, 2024
@c21 c21 assigned bveeramani and unassigned c21 Aug 15, 2024
@bveeramani bveeramani added P2 Important issue, but not time-critical and removed P0 Issues that should be fixed in short order labels Aug 26, 2024
MortalHappiness pushed a commit to MortalHappiness/ray that referenced this issue Nov 22, 2024
## Why are these changes needed?

close ray-project#46785
Current the memory usage for pandas is not accurate when it's object, so
we just implement to calculated it in recursion in case of nested.
## Related issue number

closes ray-project#46785, closes
ray-project#48506

## 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 :(

---------

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: Richard Liaw <rliaw@berkeley.edu>
Co-authored-by: Richard Liaw <rliaw@berkeley.edu>
jecsand838 pushed a commit to jecsand838/ray that referenced this issue Dec 4, 2024
## Why are these changes needed?

close ray-project#46785
Current the memory usage for pandas is not accurate when it's object, so
we just implement to calculated it in recursion in case of nested.
## Related issue number

closes ray-project#46785, closes
ray-project#48506

## 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 :(

---------

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: Richard Liaw <rliaw@berkeley.edu>
Co-authored-by: Richard Liaw <rliaw@berkeley.edu>
Signed-off-by: Connor Sanders <connor@elastiflow.com>
dentiny pushed a commit to dentiny/ray that referenced this issue Dec 7, 2024
## Why are these changes needed?

close ray-project#46785
Current the memory usage for pandas is not accurate when it's object, so
we just implement to calculated it in recursion in case of nested.
## Related issue number

closes ray-project#46785, closes
ray-project#48506

## 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 :(

---------

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: Richard Liaw <rliaw@berkeley.edu>
Co-authored-by: Richard Liaw <rliaw@berkeley.edu>
Signed-off-by: hjiang <dentinyhao@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P2 Important issue, but not time-critical
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants