[core] Increment ref count when creating an ObjectRef to prevent object from going out of scope#22120
Merged
stephanie-wang merged 16 commits intoray-project:masterfrom Feb 8, 2022
Merged
Conversation
stephanie-wang
commented
Feb 4, 2022
| NodeID::FromBinary(rpc_address_.raylet_id())); | ||
| auto status = Put(object, contained_object_ids, *object_id, /*pin_object=*/true); | ||
| if (!status.ok()) { | ||
| reference_counter_->RemoveOwnedObject(*object_id); |
Contributor
Author
There was a problem hiding this comment.
This is the fix compared to the previous PR (we no longer need the RemoveOwnedObject call because the ref will get deleted when we remove the local ref).
simonsays1980
pushed a commit
to simonsays1980/ray
that referenced
this pull request
Feb 27, 2022
…ct from going out of scope (ray-project#22120) When a Ray program first creates an ObjectRef (via ray.put or task call), we add it with a ref count of 0 in the C++ backend because the language frontend will increment the initial local ref once we return the allocated ObjectID, then delete the local ref once the ObjectRef goes out of scope. Thus, there is a brief window where the object ref will appear to be out of scope. This can cause problems with async protocols that check whether the object is in scope or not, such as the previous bug fixed in ray-project#19910. Now that we plan to enable lineage reconstruction to automatically recover lost objects, this race condition can also be problematic because we use the ref count to decide whether an object needs to be recovered or not. This PR avoids these race conditions by incrementing the local ref count in the C++ backend when executing ray.put() and task calls. The frontend is then responsible for skipping the initial local ref increment when creating the ObjectRef. This is the same fix used in ray-project#19910, but generalized to all initial ObjectRefs. This is a re-merge for ray-project#21719 with a fix for removing the owned object ref if creation fails.
6 tasks
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Why are these changes needed?
When a Ray program first creates an ObjectRef (via ray.put or task call), we add it with a ref count of 0 in the C++ backend because the language frontend will increment the initial local ref once we return the allocated ObjectID, then delete the local ref once the ObjectRef goes out of scope. Thus, there is a brief window where the object ref will appear to be out of scope.
This can cause problems with async protocols that check whether the object is in scope or not, such as the previous bug fixed in #19910. Now that we plan to enable lineage reconstruction to automatically recover lost objects, this race condition can also be problematic because we use the ref count to decide whether an object needs to be recovered or not.
This PR avoids these race conditions by incrementing the local ref count in the C++ backend when executing ray.put() and task calls. The frontend is then responsible for skipping the initial local ref increment when creating the ObjectRef. This is the same fix used in #19910, but generalized to all initial ObjectRefs.
This is a re-merge for #21719 with a fix for removing the owned object ref if creation fails.