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

Object GC for block splitting inside the dataset splitting #26196

Merged
merged 4 commits into from
Jul 12, 2022

Conversation

jianoaix
Copy link
Contributor

@jianoaix jianoaix commented Jun 29, 2022

Why are these changes needed?

The pipeline will spill objects when splitting the dataset into multiple equal parts.

Related issue number

#25249

Checks

  • 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 jianoaix changed the title Object GC for block splitting insdie the dataset splitting Object GC for block splitting inside the dataset splitting Jun 29, 2022
@@ -3400,6 +3400,11 @@ def _split(
left_metadata.append(ray.get(m0))
right_blocks.append(b1)
right_metadata.append(ray.get(m1))
# If return_right_half is requested, the input block b will be copied
# into b0 and b1. In such case, we can safely clear b if this is in
# lazy mode.
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm isn't it copied in both cases?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

At the level of this method, the right half may not copied and conditioned on this boolean: https://sourcegraph.com/github.com/ray-project/ray@master/-/blob/python/ray/data/dataset.py?L3689

Copy link
Contributor

Choose a reason for hiding this comment

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

But the left half is always copied. And the right half is either copied or None. It seems good to clear in either case.

# into b0 and b1. In such case, we can safely clear b if this is in
# lazy mode.
if return_right_half and self._lazy:
ray._private.internal_api.free(b, local_only=False)
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we more generally auto-free on any transform in lazy mode? It feels a little ad-hoc right now, like we should improve our block handling abstractions. Though we can defer this to future work.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, we should do it more cleanly. We may need to have an overhaul of execution semantics, e.g. 1) we execute the plan when it's split(), regardless of whether it's lazy or not; 2) we need to have the clear semantics of eager v.s. cached/pinned dataset (via running ds.fully_executed()), the later will make the dataset like an eager one, but running fully_executed() isn't going to make a lazy dataset eager.

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.

Since this is force-clearing on the base blocks of the dataset, we should be careful that this won't break certain fan-out patterns. E.g.:

ds = ray.data.from_items(list(range(1000))).experimental_lazy()
ds.limit(10).show()
ds.show()

ds2 = ray.data.from_items(list(range(1000))).experimental_lazy()
dses = ds.split_at_indices([250, 750])
for ds in dses:
    ds.show()
ds2.show()

The first should work since we're only clearing if return_right_half=True (although agreed with @ericl that that shouldn't matter), but I think the second might fail.

@ericl ericl added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jun 30, 2022
@jianoaix
Copy link
Contributor Author

jianoaix commented Jul 1, 2022

I looked more into this regarding safety. It can happen actually quite easily (not even need to fanout or use from_items()). For example, this will crash:

ds1 = ray.data.range(100, parallelism=10).map_batches(lambda x: x).experimental_lazy() # map_batches() can be dropped
ds1_splits = split(3, equal=True)
for ds in ds1_splits:
    for batch in ds.iter_batches():
        pass
for batch in ds1.iter_batches():
    pass

It's generally more tricky to do eager object gc for dataset than pipeline, as the later is protected by the API semantics that it can be consumed once at most.

We might still be able to do eager gc for lazy dataset, if we move to a semantics like "each time to execute a lazy dataset, it will be executed from the beginning" ("beginning" would mean, for example, the source input blocks, or some intermediate dataset which is known to be pinned and cannot be cleared before unpinned, the output of eager dataset). This will need a revamp work (filed #26264) first (at least part of it).

One hack we may try is to flag the Dataset (with an extra parameter) whether it's used in DatasetPipeline processing, and if so we clear the input block after split.

@jianoaix
Copy link
Contributor Author

jianoaix commented Jul 7, 2022

@ericl @clarkzinzow This should be ready to review.

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!

@ericl ericl merged commit 45ba0e3 into ray-project:master Jul 12, 2022
jianoaix pushed a commit to jianoaix/ray that referenced this pull request Jul 12, 2022
clarkzinzow added a commit to clarkzinzow/ray that referenced this pull request Jul 13, 2022
clarkzinzow added a commit that referenced this pull request Jul 13, 2022
…26196)" (#26495)

This reverts commit 45ba0e3.

Failures in the Train GPU job started popping up involving lost references around when this PR was merged; there was an ongoing failure that was reverted that overlaps this PR, but this PR is the most likely culprit for this particular lost reference issue, so we should try reverting the PR.

- Flakey test tracker: https://flakey-tests.ray.io/
- Example failure: https://buildkite.com/ray-project/ray-builders-branch/builds/8585#0181f423-0fe2-42b5-9dd8-47d2c7f9efa7
jianoaix pushed a commit to jianoaix/ray that referenced this pull request Jul 14, 2022
ericl pushed a commit that referenced this pull request Jul 22, 2022
…uns the plan (#26650)

Having the indicator about who's running the stage and who created a blocklist will enable the eager memory releasing.

This is an alternative with better abstraction to #26196.

Note: this doesn't work for Dataset.split() yet, will do in a followup PR.
Rohan138 pushed a commit to Rohan138/ray that referenced this pull request Jul 28, 2022
…uns the plan (ray-project#26650)

Having the indicator about who's running the stage and who created a blocklist will enable the eager memory releasing.

This is an alternative with better abstraction to ray-project#26196.

Note: this doesn't work for Dataset.split() yet, will do in a followup PR.
Signed-off-by: Rohan138 <[email protected]>
Stefan-1313 pushed a commit to Stefan-1313/ray_mod that referenced this pull request Aug 18, 2022
…ct#26196)

The pipeline will spill objects when splitting the dataset into multiple equal parts.

Co-authored-by: Ubuntu <[email protected]>
Signed-off-by: Stefan van der Kleij <[email protected]>
Stefan-1313 pushed a commit to Stefan-1313/ray_mod that referenced this pull request Aug 18, 2022
…ay-project#26196)" (ray-project#26495)

This reverts commit 45ba0e3.

Failures in the Train GPU job started popping up involving lost references around when this PR was merged; there was an ongoing failure that was reverted that overlaps this PR, but this PR is the most likely culprit for this particular lost reference issue, so we should try reverting the PR.

- Flakey test tracker: https://flakey-tests.ray.io/
- Example failure: https://buildkite.com/ray-project/ray-builders-branch/builds/8585#0181f423-0fe2-42b5-9dd8-47d2c7f9efa7

Signed-off-by: Stefan van der Kleij <[email protected]>
Stefan-1313 pushed a commit to Stefan-1313/ray_mod that referenced this pull request Aug 18, 2022
…uns the plan (ray-project#26650)

Having the indicator about who's running the stage and who created a blocklist will enable the eager memory releasing.

This is an alternative with better abstraction to ray-project#26196.

Note: this doesn't work for Dataset.split() yet, will do in a followup PR.
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
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants