Skip to content

[core] Increment ref count when creating an ObjectRef to prevent object from going out of scope#21719

Merged
stephanie-wang merged 14 commits intoray-project:masterfrom
stephanie-wang:pin-object-refs
Feb 4, 2022
Merged

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

Conversation

@stephanie-wang
Copy link
Copy Markdown
Contributor

@stephanie-wang stephanie-wang commented Jan 20, 2022

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.

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

@stephanie-wang stephanie-wang added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jan 20, 2022
@stephanie-wang stephanie-wang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jan 24, 2022
@stephanie-wang
Copy link
Copy Markdown
Contributor Author

I still need to fix up the ref counting in Java and C++ language frontends, but the Python version is ready for review.

@stephanie-wang
Copy link
Copy Markdown
Contributor Author

Oh turns out the C++ frontend doesn't actually do ref counting.

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.

Looks good, but I don't have the state to give a really detailed review. Maybe trigger nightly tests to flush out any edge cases?

prepare_args_internal(core_worker, language, args, args_vector,
function_descriptor, put_arg_ids)
except Exception as e:
# An error occurred during arg serialization. We must remove the
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Hmm, is it possible the refs haven't been added yet?

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.

We don't add to this list unless we've already incremented the local ref. Updated the variable name to clarify.

@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 Jan 25, 2022
@stephanie-wang stephanie-wang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jan 27, 2022
@stephanie-wang
Copy link
Copy Markdown
Contributor Author

Ran nightly, I think all of these test failures are from master:

https://buildkite.com/ray-project/periodic-ci/builds/2519

@ericl
Copy link
Copy Markdown
Contributor

ericl commented Jan 27, 2022

LINT errors, etc. Can you rebase?

@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 Jan 27, 2022
@bveeramani
Copy link
Copy Markdown
Member

‼️ ACTION REQUIRED ‼️

We've switched our code formatter from YAPF to Black (see #21311).

To prevent issues with merging your code, here's what you'll need to do:

  1. Install Black
pip install -I black==21.12b0
  1. Format changed files with Black
curl -o format-changed.sh https://gist.githubusercontent.com/bveeramani/42ef0e9e387b755a8a735b084af976f2/raw/7631276790765d555c423b8db2b679fd957b984a/format-changed.sh
chmod +x ./format-changed.sh
./format-changed.sh
rm format-changed.sh
  1. Commit your changes.
git add --all
git commit -m "Format Python code with Black"
  1. Merge master into your branch.
git pull upstream master
  1. Resolve merge conflicts (if necessary).

After running these steps, you'll have the updated format.sh.

@stephanie-wang
Copy link
Copy Markdown
Contributor Author

Test failures look unrelated (on the last commit it passed all but Java, and Java's now passing).

@stephanie-wang stephanie-wang merged commit e3af828 into ray-project:master Feb 4, 2022
@stephanie-wang stephanie-wang deleted the pin-object-refs branch February 4, 2022 01:31
rkooo567 added a commit to rkooo567/ray that referenced this pull request Feb 4, 2022
…ent object from going out of scope (ray-project#21719)"

This reverts commit e3af828.
stephanie-wang added a commit that referenced this pull request Feb 8, 2022
…ct from going out of scope (#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 #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.
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.

3 participants