Skip to content

Conversation

@yancanmao
Copy link
Contributor

Description

Summary

When a borrower process releases an object reference (ref count reaches 0) and then dies immediately, the owner may crash with a RAY_CHECK failure inside ReferenceCounter::CleanupBorrowersOnRefRemoved() .

This is caused by a race in the WORKER_REF_REMOVED_CHANNEL pub/sub path: both message_published_callback (published ref-removed reply) and publisher_failed_callback (long-polling connection failure) can be posted to the subscriber IO service, and both may execute, leading to duplicate cleanup.

Error logs:

[2026-01-26 17:37:24,322 C 2247115 2247193] reference_counter.cc:1224:  An unexpected system state has occurred. You have likely discovered a bug in Ray. Please report this issue at https://github.com/ray-project/ray/issues and we'll work with you to fix it. Check failed: it->second.mutable_borrow()->borrowers.erase(borrower_addr) 
*** StackTrace Information ***
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x16fdb9a) [0x7f1c26698b9a] ray::operator<<()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(_ZN3ray6RayLogD1Ev+0x444) [0x7f1c2669b344] ray::RayLog::~RayLog()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(_ZN3ray4core16ReferenceCounter28CleanupBorrowersOnRefRemovedERKN4absl12lts_2023080213flat_hash_mapINS_8ObjectIDENS1_9ReferenceENS3_13hash_internal4HashIS5_EESt8equal_toIS5_ESaISt4pairIKS5_S6_EEEERSD_RKNS_3rpc7AddressE+0x2bc) [0x7f1c25ae971c] ray::core::ReferenceCounter::CleanupBorrowersOnRefRemoved()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0xb4e887) [0x7f1c25ae9887] ray::core::ReferenceCounter::WaitForRefRemoved()::{lambda()#1}::operator()()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x1032b8c) [0x7f1c25fcdb8c] EventTracker::RecordExecution()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x1027c1a) [0x7f1c25fc2c1a] std::_Function_handler<>::_M_invoke()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x102c4e2) [0x7f1c25fc74e2] boost::asio::detail::wait_handler<>::do_complete()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x15a40bf) [0x7f1c2653f0bf] boost::asio::detail::scheduler::do_run_one()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x15a59d1) [0x7f1c265409d1] boost::asio::detail::scheduler::run()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0x15a6091) [0x7f1c26541091] boost::asio::io_context::run()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0xa7183a) [0x7f1c25a0c83a] ray::core::CoreWorkerProcessImpl::CreateCoreWorker()::{lambda()#1}::operator()()
/data00/home/yancan.mao/workspace/ray-myc/python/ray/_raylet.so(+0xba6ce9) [0x7f1c25b41ce9] thread_proxy
/lib/x86_64-linux-gnu/libc.so.6(+0x891f5) [0x7f1c2730c1f5]
/lib/x86_64-linux-gnu/libc.so.6(+0x10989c) [0x7f1c2738c89c]

Analysis

The owner subscribes in ReferenceCounter::WaitForRefRemoved() and provides two callbacks:

  • message_published_callback : runs when the borrower publishes a ref-removed reply.
  • publisher_failed_callback : runs when the long-polling connection fails (borrower died / connection broken).

If the borrower publishes the ref-removed message but dies before the subscriber IO service executes message_published_callback , the subscriber can also enqueue publisher_failed_callback . Both callbacks may run, causing the owner to attempt borrower cleanup twice:

  • second borrowers.erase(borrower_addr) fails the RAY_CHECK
  • and/or second Unsubscribe(...) returns false and fails the RAY_CHECK

Solution

Make the cleanup path idempotent / deduplicated:
In ReferenceCounter::CleanupBorrowersOnRefRemoved :

  • Replace the RAY_CHECK on borrowers.erase(borrower_addr) with a conditional:
    • if the borrower has already been removed, treat it as a no-op and skip the rest of cleanup.

Related issues

Fixes #60494

Additional information

Optional: Add implementation details, API changes, usage examples, screenshots, etc.

@yancanmao yancanmao requested a review from a team as a code owner January 26, 2026 10:29
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request effectively addresses a critical race condition in ReferenceCounter::CleanupBorrowersOnRefRemoved by making the cleanup process idempotent. The introduction of early exit conditions based on whether the object or borrower has already been cleaned up prevents RAY_CHECK failures and improves the robustness of the reference counting mechanism. This is a solid fix for a potential crash scenario.

Comment on lines 1245 to 1246
// Erase the previous borrower.
auto it = object_id_refs_.find(object_id);
RAY_CHECK(it != object_id_refs_.end()) << object_id;
RAY_CHECK(it->second.mutable_borrow()->borrowers.erase(borrower_addr));
DeleteReferenceInternal(it, nullptr);
Copy link
Contributor

Choose a reason for hiding this comment

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

medium

The comment "Erase the previous borrower." on line 1245 is now slightly misleading. The actual erasure of the borrower from the borrowers set happens earlier on line 1235. Line 1246 DeleteReferenceInternal(it, nullptr); is responsible for potentially deleting the entire Reference object if its overall reference count reaches zero, not just erasing the borrower. Consider updating the comment for better clarity.

Suggested change
// Erase the previous borrower.
auto it = object_id_refs_.find(object_id);
RAY_CHECK(it != object_id_refs_.end()) << object_id;
RAY_CHECK(it->second.mutable_borrow()->borrowers.erase(borrower_addr));
DeleteReferenceInternal(it, nullptr);
// Delete the reference if its overall count reaches zero.
DeleteReferenceInternal(it, nullptr);

@ray-gardener ray-gardener bot added core Issues that should be addressed in Ray Core community-contribution Contributed by the community labels Jan 26, 2026
@edoakes
Copy link
Collaborator

edoakes commented Jan 27, 2026

@dayshah @ZacAttack PTAL

Copy link

@cursor cursor bot left a comment

Choose a reason for hiding this comment

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

Cursor Bugbot has reviewed your changes and found 1 potential issue.

Signed-off-by: Mao Yancan <[email protected]>
@dayshah dayshah self-assigned this Jan 30, 2026
Copy link
Contributor

@dayshah dayshah left a comment

Choose a reason for hiding this comment

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

Thanks for the fix! Seems like this was pretty broken. The idea to make it idempotent sounds right to me but there's some possible side-effects that i'm concerned about.

Can you also write a unit test for the core worker or reference counter that tests this scenario

return;
}

MergeRemoteBorrowers(object_id, borrower_addr, new_borrower_refs);
Copy link
Contributor

Choose a reason for hiding this comment

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

There's a possible behavior change here

Before we would MergeRemoteBorrowers even if the ref had been deleted from object_id_refs_ as long as it's in borrowed_refs_. And then MergeRemoteBorrowers would add it into object_id_refs_ - now that doesn't happen.

I think if the borrower death notification comes first, followed by the actual message from the borrower (pretty unlikely but still possible) we could end up in an unexpected state.

it = object_id_refs_.find(object_id);
if (it == object_id_refs_.end()) {
return;
}
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 it's only possible for MergeRemoteBorrowers to add to object_id_refs_ but it can't ever remove the obj id from it.

Signed-off-by: Mao Yancan <[email protected]>
@yancanmao
Copy link
Contributor Author

Thanks @dayshah for the review! I agree with your points and have moved the object_id_refs_ check after MergeRemoteBorrowers. To avoid dangling objects, I also removed the early return after erasing the borrower (it->second.mutable_borrow()->borrowers.erase(borrower_addr)). A unit test has been added to cover this race condition.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-contribution Contributed by the community core Issues that should be addressed in Ray Core

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Core] Crash in ReferenceCounter::CleanupBorrowersOnRefRemoved due to check failed

3 participants