Skip to content

[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
stephanie-wang:pin-object-refs
Feb 8, 2022
Merged

[core] Increment ref count when creating an ObjectRef to prevent object from going out of scope#22120
stephanie-wang merged 16 commits intoray-project:masterfrom
stephanie-wang:pin-object-refs

Conversation

@stephanie-wang
Copy link
Copy Markdown
Contributor

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.

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);
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

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).

Copy link
Copy Markdown
Contributor

@ericl ericl left a comment

Choose a reason for hiding this comment

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

LGTM if all tests pass

@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 Feb 5, 2022
@stephanie-wang stephanie-wang merged commit dcd96ca into ray-project:master Feb 8, 2022
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.
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.

2 participants