Skip to content

[core] Make WaitForActorRefDeleted RPC Fault Tolerant#57116

Merged
jjyao merged 23 commits intoray-project:masterfrom
Sparks0219:joshlee/make-wait-for-actor-ref-deleted-fault-tolerant
Oct 9, 2025
Merged

[core] Make WaitForActorRefDeleted RPC Fault Tolerant#57116
jjyao merged 23 commits intoray-project:masterfrom
Sparks0219:joshlee/make-wait-for-actor-ref-deleted-fault-tolerant

Conversation

@Sparks0219
Copy link
Copy Markdown
Contributor

@Sparks0219 Sparks0219 commented Oct 2, 2025

Why are these changes needed?

Making WaitForActorRefDeleted RPC Fault Tolerant. Added c++ unit tests to verify idempotency of HandleWaitForActorRefDeleted, and added a python integration test.

Related issue number

Closes #53797

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run pre-commit jobs to lint the changes in this PR. (pre-commit setup)
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • 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 :(

Note

Cursor Bugbot is generating a summary for commit e3234ca. Configure here.

Signed-off-by: joshlee <joshlee@anyscale.com>
@Sparks0219 Sparks0219 requested a review from a team as a code owner October 2, 2025 01:08
@Sparks0219 Sparks0219 added the go add ONLY when ready to merge, run all tests label Oct 2, 2025
Copy link
Copy Markdown
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 makes the WaitForActorRefDeleted RPC fault-tolerant by enabling retries. This is a good improvement for the robustness of actor lifecycle management. The changes include updating the RPC client to be retryable, modifying the WaitForActorRefDeleted method signature to use move semantics for better performance, and adding comprehensive tests. The new C++ tests for idempotency on the server side are well-structured. I've suggested a small improvement to the new Python test to cover response failures in addition to request failures, and a minor style improvement in the C++ tests to mark unused lambda parameters for better readability. Overall, this is a solid contribution to Ray's fault tolerance.

Comment on lines +39 to +52
@pytest.mark.parametrize("deterministic_failure", ["request"])
def test_wait_for_actor_ref_deleted_rpc_retry_and_idempotency(
monkeypatch, shutdown_only, deterministic_failure
):
"""Test that WaitForActorRefDeleted RPC retries work correctly.
Verify that the RPC is idempotent when network failures occur.
The GCS actor manager will trigger this RPC during actor initialization
to monitor when the actor handles have gone out of scope and the actor should be destroyed.
"""

monkeypatch.setenv(
"RAY_testing_rpc_failure",
"CoreWorkerService.grpc_client.WaitForActorRefDeleted=1:100:0",
)
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.

medium

The test test_wait_for_actor_ref_deleted_rpc_retry_and_idempotency is only parameterized to test request failures. For better test coverage, it should also be parameterized to test response failures, similar to test_get_object_status_rpc_retry_and_idempotency.

Additionally, the deterministic_failure parameter is currently unused in the test body.

Suggested change
@pytest.mark.parametrize("deterministic_failure", ["request"])
def test_wait_for_actor_ref_deleted_rpc_retry_and_idempotency(
monkeypatch, shutdown_only, deterministic_failure
):
"""Test that WaitForActorRefDeleted RPC retries work correctly.
Verify that the RPC is idempotent when network failures occur.
The GCS actor manager will trigger this RPC during actor initialization
to monitor when the actor handles have gone out of scope and the actor should be destroyed.
"""
monkeypatch.setenv(
"RAY_testing_rpc_failure",
"CoreWorkerService.grpc_client.WaitForActorRefDeleted=1:100:0",
)
@pytest.mark.parametrize("deterministic_failure", ["request", "response"])
def test_wait_for_actor_ref_deleted_rpc_retry_and_idempotency(
monkeypatch, shutdown_only, deterministic_failure
):
"""Test that WaitForActorRefDeleted RPC retries work correctly.
Verify that the RPC is idempotent when network failures occur.
The GCS actor manager will trigger this RPC during actor initialization
to monitor when the actor handles have gone out of scope and the actor should be destroyed.
"""
monkeypatch.setenv(
"RAY_testing_rpc_failure",
"CoreWorkerService.grpc_client.WaitForActorRefDeleted=1:"
+ ("100:0" if deterministic_failure == "request" else "0:100"),
)

Comment on lines +995 to +997
[&callback_count](Status s,
std::function<void()> success,
std::function<void()> failure) { callback_count++; });
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.

medium

The lambda parameters s, success, and failure are unused. It's good practice to either omit their names or mark them as unused to improve readability and signal intent.

Suggested change
[&callback_count](Status s,
std::function<void()> success,
std::function<void()> failure) { callback_count++; });
[&callback_count](Status /*s*/,
std::function<void()> /*success*/,
std::function<void()> /*failure*/) { callback_count++; });

Comment on lines +1076 to +1078
[&callback_count](Status s,
std::function<void()> success,
std::function<void()> failure) { callback_count++; });
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.

medium

The lambda parameters s, success, and failure are unused. It's good practice to either omit their names or mark them as unused to improve readability and signal intent.

Suggested change
[&callback_count](Status s,
std::function<void()> success,
std::function<void()> failure) { callback_count++; });
[&callback_count](Status /*s*/,
std::function<void()> /*success*/,
std::function<void()> /*failure*/) { callback_count++; });

@ray-gardener ray-gardener bot added the core Issues that should be addressed in Ray Core label Oct 2, 2025
rpc::WaitForActorRefDeletedReply reply2;

// For requests containing the same ActorID, the send_reply_callback is overwritten
// and only the last callback is triggered.
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.

should assert false in the first callback and assert true in the second callback

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.

Put an ASSERT_TRUE(false) in the first callback since I wanted to guarantee it wouldn't be called.

Signed-off-by: joshlee <joshlee@anyscale.com>
cursor[bot]

This comment was marked as outdated.

Signed-off-by: joshlee <joshlee@anyscale.com>
@Sparks0219 Sparks0219 requested a review from dayshah October 2, 2025 05:52
Signed-off-by: joshlee <joshlee@anyscale.com>
cursor[bot]

This comment was marked as outdated.

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
cursor[bot]

This comment was marked as outdated.

Signed-off-by: joshlee <joshlee@anyscale.com>
assert final_result == [0, 2, 4, 6, 8]


@pytest.mark.parametrize("deterministic_failure", ["request"])
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.

why not also response?

Copy link
Copy Markdown
Contributor Author

@Sparks0219 Sparks0219 Oct 7, 2025

Choose a reason for hiding this comment

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

I think it's a bit tricky to test response because there's another RPC that the owner uses to notify the GCS that the actor goes out of scope: AsyncReportActorOutOfScope so even if the response is lost the GCS will still deallocate the actor.

Request is easy because if it's lost the callback immediately calls DestroyActor so we get an "actor creation cancelled" error

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.

can you put a comment there so the person after knows


result = ray.get(actor.ping.remote())
assert result == "pong"

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.

we should do

del actor
// check to make sure actor is marked as dead with cause equals REF_DELETED

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.

AsyncReportActorOutOfScope will report this, note the prior comment

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.

I mean we should add some checks to make sure actor is marked as dead with cause equals REF_DELETED

Signed-off-by: joshlee <joshlee@anyscale.com>
cursor[bot]

This comment was marked as outdated.

Copy link
Copy Markdown
Contributor

@jjyao jjyao left a comment

Choose a reason for hiding this comment

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

LGTM

return false;
}
it->second.on_object_ref_delete = callback;
it->second.object_ref_deleted_callbacks.push_back(callback);
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.

Suggested change
it->second.object_ref_deleted_callbacks.push_back(callback);
it->second.object_ref_deleted_callbacks.push_back(std::move(callback));

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.

Makes sense, done

/// Returns true if the object was in the reference table and the callback was added
/// else false.
bool SetObjectRefDeletedCallback(const ObjectID &object_id,
bool AddObjectRefDeletedCallback(const ObjectID &object_id,
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.

Update docstring

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.

Ah shoot, thanks for catching

Signed-off-by: joshlee <joshlee@anyscale.com>
wait_request,
auto client = worker_client_pool_.GetOrConnect(it->second.address_);
client->WaitForActorRefDeleted(
std::move(wait_request),
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Bug: RPC Client Pooling Breaks Retry Mechanism

The WaitForActorRefDeleted RPC now fetches a new client from the pool for each call. This breaks the retryable RPC mechanism, which relies on the same client instance to maintain state across retries.

Fix in Cursor Fix in Web

Copy link
Copy Markdown

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

Bug: Missing `const` Qualifier in Callback Parameter

The AddObjectRefDeletedCallback override in ReferenceCounter has a callback parameter that's missing the const qualifier, which differs from the virtual method in ReferenceCounterInterface. This signature mismatch prevents proper overriding and will cause compilation errors.

src/ray/core_worker/reference_counter.h#L48-L51

bool foreign_owner_already_monitoring = false) = 0;
virtual void AddOwnedObject(
const ObjectID &object_id,
const std::vector<ObjectID> &contained_ids,

Fix in Cursor Fix in Web


wait_request,
auto client = worker_client_pool_.GetOrConnect(it->second.address_);
client->WaitForActorRefDeleted(
std::move(wait_request),
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Bug: Moved Object Access Causes Undefined Behavior

Moving wait_request invalidates the object. While the lambda doesn't use it, accessing a moved-from object can lead to undefined behavior.

Fix in Cursor Fix in Web

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.

omg cursor hates the moves

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.

yea and the client pool too, chill AI

Signed-off-by: joshlee <joshlee@anyscale.com>
wait_request,
auto client = worker_client_pool_.GetOrConnect(it->second.address_);
client->WaitForActorRefDeleted(
std::move(wait_request),
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Bug: RPC Retry Fails Due to Empty Request

The wait_request is moved into the RPC call, making it empty for any retry attempts. This, combined with fetching a new client from the pool for each call, can lead to failed or inconsistent RPC retries.

Fix in Cursor Fix in Web

Signed-off-by: joshlee <joshlee@anyscale.com>

/// The callback for each request is stored in the reference counter due to retries
/// and message reordering where the callback of the retry of the request could be
/// overwritten by the callback of the initial request.
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Bug: Double Move of respond Causes Undefined Behavior

The respond callback is moved into the AsyncWaitForActorRegisterFinish lambda capture. If actor registration succeeds, the respond object, already in a moved-from state, is moved a second time into WaitForActorRefDeleted, causing undefined behavior.

Fix in Cursor Fix in Web

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.

it got the problem wrong, the problem is actually just that the second move doesn't really move because function captures are const by default

Copy link
Copy Markdown
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.

also update the Failure line in the .proto file, not sure if we've missed that for more rpc's...

just a nit

}
if (it->second.on_object_ref_delete) {
it->second.on_object_ref_delete(it->first);
if (it->second.object_ref_deleted_callbacks.size() > 0) {
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.

why do you need this if, it's the same without it

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.

oh bleh good catch thanks

wait_request,
auto client = worker_client_pool_.GetOrConnect(it->second.address_);
client->WaitForActorRefDeleted(
std::move(wait_request),
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.

omg cursor hates the moves

@Sparks0219
Copy link
Copy Markdown
Contributor Author

also update the Failure line in the .proto file, not sure if we've missed that for more rpc's...

UHHH yea it's a little outdated lol, I went through the list on the doc and updated all the missing ones

Signed-off-by: joshlee <joshlee@anyscale.com>
@Sparks0219 Sparks0219 requested review from dayshah and jjyao October 9, 2025 18:41
@jjyao jjyao enabled auto-merge (squash) October 9, 2025 19:38
@jjyao jjyao merged commit cef177f into ray-project:master Oct 9, 2025
7 checks passed
pavitrabhalla pushed a commit to superserve-ai/ray that referenced this pull request Oct 10, 2025
)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Pavitra Bhalla <pavitra@rayai.com>
joshkodi pushed a commit to joshkodi/ray that referenced this pull request Oct 13, 2025
)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Josh Kodi <joshkodi@gmail.com>
ArturNiederfahrenhorst pushed a commit to ArturNiederfahrenhorst/ray that referenced this pull request Oct 13, 2025
harshit-anyscale pushed a commit that referenced this pull request Oct 15, 2025
Signed-off-by: joshlee <joshlee@anyscale.com>
justinyeh1995 pushed a commit to justinyeh1995/ray that referenced this pull request Oct 20, 2025
xinyuangui2 pushed a commit to xinyuangui2/ray that referenced this pull request Oct 22, 2025
)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: xgui <xgui@anyscale.com>
elliot-barn pushed a commit that referenced this pull request Oct 23, 2025
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Aydin Abiar <aydin@anyscale.com>
Future-Outlier pushed a commit to Future-Outlier/ray that referenced this pull request Dec 7, 2025
)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Future-Outlier <eric901201@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Core] Transient network failure on RPC WaitForActorRefDeleted causes actor registration fail

4 participants