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] Fix the bug of eagerly clearing up input blocks #31459

Merged
merged 1 commit into from
Jan 5, 2023

Conversation

c21
Copy link
Contributor

@c21 c21 commented Jan 5, 2023

Co-authored-by: Clark Zinzow [email protected]
Signed-off-by: Cheng Su [email protected]

Why are these changes needed?

This PR is fixing the issue found in #31286. Previously we always eagerly clears up non-lazy input blocks (plan._in_blocks) when executing the plan. This is not safe as the input blocks might be used by downstream operations later.

An example of exception:

import ray
import pandas as pd

input_ds = ray.data.from_pandas(pd.DataFrame({"col1": [1, 2], "col2": [4, 5]}))
input_ds = input_ds.map_batches(lambda x:x)
input_ds = input_ds.lazy()

ds1 = input_ds.map_batches(lambda x:x)
ds1.fully_executed()
ds2 = input_ds.map_batches(lambda x:x)
ds2.fully_executed()
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/chengsu/ray/python/ray/data/dataset.py", line 3834, in fully_executed
    self._plan.execute(force_read=True)
  File "/Users/chengsu/ray/python/ray/data/_internal/plan.py", line 321, in execute
    blocks, stage_info = stage(
  File "/Users/chengsu/ray/python/ray/data/_internal/plan.py", line 683, in __call__
    if blocks._owned_by_consumer:
AttributeError: 'NoneType' object has no attribute '_owned_by_consumer'

Related issue number

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

Co-authored-by: Clark Zinzow <[email protected]>
Signed-off-by: Cheng Su <[email protected]>
Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

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

LGTM!

# Test that first map is executed twice, because ds1.fully_executed()
# clears up the previous snapshot blocks, and ds2.fully_executed()
# has to re-execute ds.map(inc) again.
assert ray.get(map_counter.get.remote()) == 2 * 10 + 10 + 10
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice test!

# If not a lazy datasource, unlink the input blocks from the plan so we
# can eagerly reclaim the input block memory after the first stage is
# done executing.
self._in_blocks = None
Copy link
Contributor

Choose a reason for hiding this comment

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

I think that removing this is safe, but when adding the block clearing logic for the new execution plan/model, we should be sure to have comprehensive memory releasing tests that covers all corners cases of our block clearing semantics.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@clarkzinzow - agreed.

@clarkzinzow clarkzinzow merged commit ce0f2a0 into ray-project:master Jan 5, 2023
@c21 c21 deleted the fix-in-block branch January 6, 2023 06:51
ericl pushed a commit that referenced this pull request Jan 6, 2023
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes:
* Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed.
* `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block.
* Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing.

TODO:
- [x] Fix all unit tests
- [x] #31459
- [x] #31460 
- [ ] Remove the behavior to eagerly compute first block for read
- [ ] #31417
- [ ] Update documentation
AmeerHajAli pushed a commit that referenced this pull request Jan 12, 2023
This PR is fixing the issue found in #31286. Previously we always eagerly clears up non-lazy input blocks (plan._in_blocks) when executing the plan. This is not safe as the input blocks might be used by downstream operations later.

Signed-off-by: Cheng Su <[email protected]>
Co-authored-by: Clark Zinzow <[email protected]>
AmeerHajAli pushed a commit that referenced this pull request Jan 12, 2023
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes:
* Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed.
* `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block.
* Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing.

TODO:
- [x] Fix all unit tests
- [x] #31459
- [x] #31460 
- [ ] Remove the behavior to eagerly compute first block for read
- [ ] #31417
- [ ] Update documentation
tamohannes pushed a commit to ju2ez/ray that referenced this pull request Jan 25, 2023
This PR is fixing the issue found in ray-project#31286. Previously we always eagerly clears up non-lazy input blocks (plan._in_blocks) when executing the plan. This is not safe as the input blocks might be used by downstream operations later.

Signed-off-by: Cheng Su <[email protected]>
Co-authored-by: Clark Zinzow <[email protected]>
Signed-off-by: tmynn <[email protected]>
tamohannes pushed a commit to ju2ez/ray that referenced this pull request Jan 25, 2023
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes:
* Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed.
* `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block.
* Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing.

TODO:
- [x] Fix all unit tests
- [x] ray-project#31459
- [x] ray-project#31460 
- [ ] Remove the behavior to eagerly compute first block for read
- [ ] ray-project#31417
- [ ] Update documentation

Signed-off-by: tmynn <[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.

3 participants