Skip to content

Conversation

@JackGammack
Copy link
Contributor

@JackGammack JackGammack commented Aug 26, 2025

Why are these changes needed?

Using FileBasedDatasource or ParquetDatasource with a very large number of files causes OOM when creating read tasks. The full list of file paths is stored in self, causing it to persist to every read task, leading to this warning:

The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.

When using a small number of blocks, OOM does not occur because the large file list is not repeated so many times. But when setting high parallelism with override_num_blocks, OOM occurs.

This is because the full list of paths is added to self._unresolved_paths. This attribute isn't currently used anywhere in ray. This PR removes self._unresolved_paths to alleviate unexpected high memory usage with very large numbers of files.

Related issue number

Similar to this issue with Iceberg: #49054

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

…source to reduce memory usage

Signed-off-by: Jack Gammack <jgammack@etsy.com>
Signed-off-by: Jack Gammack <jgammack@etsy.com>
@JackGammack JackGammack changed the title [data] Fix performance degradation with FileBasedDatasource & ParquetDatasource causing high memory usage [data] Fix high memory usage with FileBasedDatasource & ParquetDatasource with large numbers of files Aug 26, 2025
@JackGammack JackGammack changed the title [data] Fix high memory usage with FileBasedDatasource & ParquetDatasource with large numbers of files [data] Fix high memory usage with FileBasedDatasource & ParquetDatasource when using large numbers of files Aug 26, 2025
@JackGammack JackGammack changed the title [data] Fix high memory usage with FileBasedDatasource & ParquetDatasource when using large numbers of files [data] Fix high memory usage with FileBasedDatasource & ParquetDatasource when using a large number of files Aug 27, 2025
@JackGammack JackGammack marked this pull request as ready for review August 27, 2025 01:22
@JackGammack JackGammack requested a review from a team as a code owner August 27, 2025 01:22
@ray-gardener ray-gardener bot added docs An issue or change related to documentation data Ray Data-related issues community-contribution Contributed by the community labels Aug 27, 2025
@gvspraveen gvspraveen added the go add ONLY when ready to merge, run all tests label Sep 1, 2025
@gvspraveen
Copy link
Contributor

Thank you for the contribution. And good catch on high mem for large number of blocks.

On closer look, _unresolved_paths appears to be not used anywhere. @bveeramani @omatthew98 Do we still need it or should we delete it?
cc @JDarDagran who may have more thoughts on this (this was introduced in #50332).

@omatthew98
Copy link
Contributor

Do we still need it or should we delete it?

Yeah it is not used anywhere and I don't see any reason to keep it. The only argument for keeping it would be to ensure that we don't break any assumptions for downstream consumers (e.g. users who extend FileBasedDatasource or ParquetDatasource), but I doubt there are any users doing that and if they are they aren't following a public API to do so.

@gvspraveen

@JackGammack
Copy link
Contributor Author

Yes, I'm happy to update this to just remove it. The unresolved paths aren't used anywhere, and aren't accessible by users using the read_parquet etc API since the datasource is created internally and not passed outside of that function. The PRs that added the unresolved paths were relatively recent, so I thought they might want to be kept.

Signed-off-by: Jack Gammack <jgammack@etsy.com>
Signed-off-by: Jack Gammack <jgammack@etsy.com>
@JDarDagran
Copy link
Contributor

@gvspraveen thanks for pinging me. I'm using this for lineage tracking purpose which was not yet open-sourced. The relation is not that clear - I'm looking at the object through ray.data._internal.execution.execution_callback.ExecutionCallback.

Maybe those unresolved_paths could be put into GCS, similarly to resolved paths?

        # Read tasks serialize `FileBasedDatasource` instances, and the list of paths
        # can be large. To avoid slow serialization speeds, we store a reference to
        # the paths rather than the paths themselves.
        self._paths_ref = ray.put(paths)

@JackGammack
Copy link
Contributor Author

I think keeping it in the object store is okay, the size of the many file paths isn't so bad if it's just stored once. I only run into issues when I increase the number of blocks over 2,000 or so

@JDarDagran
Copy link
Contributor

I think keeping it in the object store is okay, the size of the many file paths isn't so bad if it's just stored once. I only run into issues when I increase the number of blocks over 2,000 or so

Thanks! I'd really love to avoid losing that metadata.

Signed-off-by: Jack Gammack <jgammack@etsy.com>
Signed-off-by: Jack Gammack <jgammack@etsy.com>
Copy link
Contributor

@omatthew98 omatthew98 left a comment

Choose a reason for hiding this comment

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

Thanks @JDarDagran for the input, thanks @JackGammack for the change and iteration. LGTM.

@JackGammack
Copy link
Contributor Author

A couple things to note after updating:

  1. I didn't add the helper to ray.get the actual unresolved paths as is done here, but could if you wanted. The test case shows getting the unresolved paths from the object store
  2. I noticed that the ParquetDatasource doesn't store the resolved paths in the object store the way FileBasedDatasource does, but instead in self._pq_paths. So it could also have memory issues for large numbers of files and blocks. But the ParquetDatasourcealready has issues with large numbers of files because of the metadata collection

Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

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

Thank you for your contribution @JackGammack!

ray.get_runtime_context().get_node_id(), soft=False
)

self._unresolved_paths = paths
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 actually just delete this field

Copy link
Contributor Author

@JackGammack JackGammack Sep 2, 2025

Choose a reason for hiding this comment

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

Are you saying to remove self._unresolved_paths_ref for ParquetDatasource, but keep it for FileBasedDatasource? Or is this an older comment

Copy link
Contributor

Choose a reason for hiding this comment

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

I see no good reason for us not to nuke it in both places

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I do agree if it's not used anywhere currently especially with the big performance issue, but I'm not aware of the other ongoing work. It should be very easy to add back.

I have updated to just remove self._unresolved_paths now

JackGammack and others added 3 commits September 4, 2025 08:53
Signed-off-by: Jack Gammack <jgammack@etsy.com>
Signed-off-by: Jack Gammack <jgammack@etsy.com>
@alexeykudinkin alexeykudinkin merged commit 83fb562 into ray-project:master Sep 4, 2025
5 checks passed
sampan-s-nayak pushed a commit to sampan-s-nayak/ray that referenced this pull request Sep 8, 2025
…urce when using a large number of files (ray-project#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: ray-project#49054

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
Signed-off-by: sampan <sampan@anyscale.com>
jugalshah291 pushed a commit to jugalshah291/ray_fork that referenced this pull request Sep 11, 2025
…urce when using a large number of files (ray-project#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: ray-project#49054

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
Signed-off-by: jugalshah291 <shah.jugal291@gmail.com>
wyhong3103 pushed a commit to wyhong3103/ray that referenced this pull request Sep 12, 2025
…urce when using a large number of files (ray-project#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: ray-project#49054

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
Signed-off-by: yenhong.wong <yenhong.wong@grabtaxi.com>
alexwang177 pushed a commit to pinterest/ray that referenced this pull request Sep 17, 2025
…urce when using a large number of files (ray-project#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: ray-project#49054 

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
dstrodtman pushed a commit that referenced this pull request Oct 6, 2025
…urce when using a large number of files (#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: #49054

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
Signed-off-by: Douglas Strodtman <douglas@anyscale.com>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
…urce when using a large number of files (ray-project#55978)

## Why are these changes needed?

Using `FileBasedDatasource` or `ParquetDatasource` with a very large
number of files causes OOM when creating read tasks. The full list of
file paths is stored in `self`, causing it to persist to every read
task, leading to this warning:
```
The serialized size of your read function named 'read_task_fn' is 49.8MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

When using a small number of blocks, OOM does not occur because the
large file list is not repeated so many times. But when setting high
parallelism with `override_num_blocks`, OOM occurs.

This is because the full list of paths is added to
`self._unresolved_paths`. This attribute isn't currently used anywhere
in ray. This PR removes `self._unresolved_paths` to alleviate unexpected
high memory usage with very large numbers of files.

## Related issue number

Similar to this issue with Iceberg: ray-project#49054 

## 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.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] 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: Jack Gammack <jgammack@etsy.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-contribution Contributed by the community data Ray Data-related issues docs An issue or change related to documentation go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants