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

[Datasets] Avoid unnecessary reads when truncating a dataset with ds.limit() #27343

Conversation

clarkzinzow
Copy link
Contributor

@clarkzinzow clarkzinzow commented Aug 1, 2022

Datasets currently eagerly kicks off all read tasks when truncating a dataset immediately after a read via ray.data.read_*().limit(); this results in a lot of wasted computation and unnecessary object store bloat, especially when trying to poke at a very small subset of the data.

This PR avoids these unnecessary reads by truncating the blocklist to the minimum number of blocks needed to meet the row limit before doing the actual block splitting, thereby avoiding materialization of unnecessary read tasks in the common splitting path.

Related issue number

Closes #27340

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

@jianoaix
Copy link
Contributor

jianoaix commented Aug 2, 2022

Nice, we should make all transform APIs real and not run as consumption APIs.

python/ray/data/dataset.py Show resolved Hide resolved
@@ -112,6 +112,30 @@ def split_by_bytes(self, bytes_per_split: int) -> List["BlockList"]:
)
return output

def truncate_by_rows(self, limit: int) -> "BlockList":
Copy link
Contributor

Choose a reason for hiding this comment

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

It looks truncate_by_blocks, since this is working at block level with constraint to cover desired num of rows.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We're truncating to the number of rows given, similar to above split_by_bytes where we're splitting by the number of bytes given.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ok. I think one thing important is that we don't split block in order to make up the desired bytes or rows. I don't have a good naming suggestion though (maybe just compensate it with comments).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's already indicated in the docstring so I think it should be fine.

python/ray/data/datasource/datasource.py Outdated Show resolved Hide resolved
python/ray/data/datasource/datasource.py Outdated Show resolved Hide resolved
@clarkzinzow clarkzinzow force-pushed the datasets/fix/limit-no-redundant-reads branch from d3847a0 to 3dc9eee Compare August 4, 2022 23:08
@@ -112,6 +112,30 @@ def split_by_bytes(self, bytes_per_split: int) -> List["BlockList"]:
)
return output

def truncate_by_rows(self, limit: int) -> "BlockList":
Copy link
Contributor

Choose a reason for hiding this comment

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

Ok. I think one thing important is that we don't split block in order to make up the desired bytes or rows. I don't have a good naming suggestion though (maybe just compensate it with comments).

@clarkzinzow clarkzinzow merged commit 313d553 into ray-project:master Aug 5, 2022
clarkzinzow added a commit to clarkzinzow/ray that referenced this pull request Aug 5, 2022
….limit()` (ray-project#27343)

Datasets currently eagerly kicks off all read tasks when truncating a dataset immediately after a read via ray.data.read_*().limit(); this results in a lot of wasted computation and unnecessary object store bloat, especially when trying to poke at a very small subset of the data.

This PR avoids these unnecessary reads by truncating the blocklist to the minimum number of blocks needed to meet the row limit before doing the actual block splitting, thereby avoiding materialization of unnecessary read tasks in the common splitting path.
scv119 pushed a commit that referenced this pull request Aug 6, 2022
…datasets with `.limit()` (#27585)

* [Datasets] Avoid unnecessary reads when truncating a dataset with `ds.limit()` (#27343)

Datasets currently eagerly kicks off all read tasks when truncating a dataset immediately after a read via ray.data.read_*().limit(); this results in a lot of wasted computation and unnecessary object store bloat, especially when trying to poke at a very small subset of the data.

This PR avoids these unnecessary reads by truncating the blocklist to the minimum number of blocks needed to meet the row limit before doing the actual block splitting, thereby avoiding materialization of unnecessary read tasks in the common splitting path.

* [Datasets] [Docs] Improve `.limit()` and `.take()` docstrings (#27367)

Improve docstrings for .limit() and .take(), making the distinction more clear.

Signed-off-by: Clark Zinzow <[email protected]>
Stefan-1313 pushed a commit to Stefan-1313/ray_mod that referenced this pull request Aug 18, 2022
….limit()` (ray-project#27343)

Datasets currently eagerly kicks off all read tasks when truncating a dataset immediately after a read via ray.data.read_*().limit(); this results in a lot of wasted computation and unnecessary object store bloat, especially when trying to poke at a very small subset of the data.

This PR avoids these unnecessary reads by truncating the blocklist to the minimum number of blocks needed to meet the row limit before doing the actual block splitting, thereby avoiding materialization of unnecessary read tasks in the common splitting path.

Signed-off-by: Stefan van der Kleij <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Datasets] Avoid unnecessary execution/read to improve limit() performance
5 participants