Skip to content

Conversation

@codope
Copy link
Contributor

@codope codope commented Nov 13, 2025

The tests that exercised actor failures when they go out of scope, such as test_actor_ray_shutdown_called_on_del and test_actor_ray_shutdown_called_on_scope_exit were flaky. This PR fixes the flakiness by ensuring actors use graceful shutdown when GCS polling detects actor refs are deleted.

Problem
When actors go out of scope, GCS uses two mechanisms to detect reference deletion:

  1. Push model (GcsActorManager::HandleReportActorOutOfScope) - already fixed in [core] Use graceful shutdown path when actor OUT_OF_SCOPE (del actor) #57090
  2. Pull model (GcsActorManager::PollOwnerForActorRefDeleted) - was still using force kill

The pull model was calling DestroyActor(..., force_kill=true), which skips __ray_shutdown__ and immediately terminates the actor. This created a race condition: whichever mechanism completed first determined whether cleanup callbacks ran, causing test flakiness.

To fix the issue, changed PollOwnerForActorRefDeleted to use graceful shutdown with timeout (same as HandleReportActorOutOfScope). I ran all the actor failure tests that exercise this shutdown path 20 times locally, and where they failed 3/20 previously, they succeeded everytime after the fix.

@codope codope requested a review from a team as a code owner November 13, 2025 22:10
@codope codope requested a review from dayshah November 13, 2025 22:11
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 addresses flakiness in actor failure tests by ensuring graceful shutdown for actors when GCS polling detects their references are deleted. The change in GcsActorManager::PollOwnerForActorRefDeleted correctly modifies the call to DestroyActor, setting force_kill to false and providing a graceful shutdown timeout. This aligns the behavior of the pull-based reference deletion mechanism with the push-based one, fixing the described race condition. The change is clear, well-targeted, and looks correct.

@codope codope added the go add ONLY when ready to merge, run all tests label Nov 13, 2025
@codope
Copy link
Contributor Author

codope commented Nov 13, 2025

@dayshah @edoakes I have a more basic question: why do we need two different mechanisms for the same thing? It feels like the polling mechanism is only there as a backup (say when owner crashes before sending the notification).

@dayshah
Copy link
Contributor

dayshah commented Nov 13, 2025

@dayshah @edoakes I have a more basic question: why do we need two different mechanisms for the same thing? It feels like the polling mechanism is only there as a backup (say when owner crashes before sending the notification).

ohh ok I remember this now
So ref deleted is for when it's totally deleted from the ref counter, e.g. no lineage ref count too the actor is not restartable, etc. so the actor is dead dead

The out of scope is for when the ref is out of scope (people are done using it but it can still be lineage reconstructed). This means the actor can be destroyed right now but could be restarted later.

It doesn't make that much send for the two directions, we should just always do worker -> GCS and not have the long polling rpc the whole time, the GCS also knows when workers die and when the owner dies, it'll also destroy the actor. There's also some core worker ref counting pubsub for this stuff for object deletion that's a little repetitive with this. Also we don't need to send the outofscope if we send the delete one so if it's always from worker we can deduplicate that and 1 less rpc per non-restartable actor

Maybe @Sparks0219 can clean this up with the actor restart project 😃

DestroyActor(actor_id,
GenActorRefDeletedCause(GetActor(actor_id)),
/*force_kill=*/true);
/*force_kill=*/false,
Copy link
Contributor

Choose a reason for hiding this comment

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

it looks like we never set force kill to true with the new timeout changes, can we just kill the parameter completely

Copy link
Contributor Author

Choose a reason for hiding this comment

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

makes sense; can i do that in a separate PR?

@ray-gardener ray-gardener bot added the core Issues that should be addressed in Ray Core label Nov 14, 2025
@edoakes edoakes merged commit e2dce38 into ray-project:master Nov 14, 2025
6 checks passed
ArturNiederfahrenhorst pushed a commit to ArturNiederfahrenhorst/ray that referenced this pull request Nov 16, 2025
… deleted (ray-project#58605)

The tests that exercised actor failures when they go out of scope, such
as `test_actor_ray_shutdown_called_on_del` and
`test_actor_ray_shutdown_called_on_scope_exit` [were
flaky](https://buildkite.com/ray-project/postmerge/builds/14336#019a7abe-73d3-46e0-8dc2-13351e12b7c3/613-1919).
This PR fixes the flakiness by ensuring actors use graceful shutdown
when GCS polling detects actor refs are deleted.

**Problem**
When actors go out of scope, GCS uses two mechanisms to detect reference
deletion:
1. Push model (`GcsActorManager::HandleReportActorOutOfScope`) - already
fixed in ray-project#57090
2. Pull model (`GcsActorManager::PollOwnerForActorRefDeleted`) - was
still using force kill

The pull model was calling DestroyActor(..., force_kill=true), which
skips `__ray_shutdown__` and immediately terminates the actor. This
created a race condition: whichever mechanism completed first determined
whether cleanup callbacks ran, causing test flakiness.

To fix the issue, changed `PollOwnerForActorRefDeleted` to use graceful
shutdown with timeout (same as `HandleReportActorOutOfScope`). I ran all
the actor failure tests that exercise this shutdown path 20 times
locally, and where they failed 3/20 previously, they succeeded everytime
after the fix.

Signed-off-by: Sagar Sumit <[email protected]>
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
… deleted (ray-project#58605)

The tests that exercised actor failures when they go out of scope, such
as `test_actor_ray_shutdown_called_on_del` and
`test_actor_ray_shutdown_called_on_scope_exit` [were
flaky](https://buildkite.com/ray-project/postmerge/builds/14336#019a7abe-73d3-46e0-8dc2-13351e12b7c3/613-1919).
This PR fixes the flakiness by ensuring actors use graceful shutdown
when GCS polling detects actor refs are deleted.

**Problem**
When actors go out of scope, GCS uses two mechanisms to detect reference
deletion:
1. Push model (`GcsActorManager::HandleReportActorOutOfScope`) - already
fixed in ray-project#57090
2. Pull model (`GcsActorManager::PollOwnerForActorRefDeleted`) - was
still using force kill

The pull model was calling DestroyActor(..., force_kill=true), which
skips `__ray_shutdown__` and immediately terminates the actor. This
created a race condition: whichever mechanism completed first determined
whether cleanup callbacks ran, causing test flakiness.

To fix the issue, changed `PollOwnerForActorRefDeleted` to use graceful
shutdown with timeout (same as `HandleReportActorOutOfScope`). I ran all
the actor failure tests that exercise this shutdown path 20 times
locally, and where they failed 3/20 previously, they succeeded everytime
after the fix.

Signed-off-by: Sagar Sumit <[email protected]>
Signed-off-by: Aydin Abiar <[email protected]>
ykdojo pushed a commit to ykdojo/ray that referenced this pull request Nov 27, 2025
… deleted (ray-project#58605)

The tests that exercised actor failures when they go out of scope, such
as `test_actor_ray_shutdown_called_on_del` and
`test_actor_ray_shutdown_called_on_scope_exit` [were
flaky](https://buildkite.com/ray-project/postmerge/builds/14336#019a7abe-73d3-46e0-8dc2-13351e12b7c3/613-1919).
This PR fixes the flakiness by ensuring actors use graceful shutdown
when GCS polling detects actor refs are deleted.

**Problem**
When actors go out of scope, GCS uses two mechanisms to detect reference
deletion:
1. Push model (`GcsActorManager::HandleReportActorOutOfScope`) - already
fixed in ray-project#57090
2. Pull model (`GcsActorManager::PollOwnerForActorRefDeleted`) - was
still using force kill

The pull model was calling DestroyActor(..., force_kill=true), which
skips `__ray_shutdown__` and immediately terminates the actor. This
created a race condition: whichever mechanism completed first determined
whether cleanup callbacks ran, causing test flakiness.

To fix the issue, changed `PollOwnerForActorRefDeleted` to use graceful
shutdown with timeout (same as `HandleReportActorOutOfScope`). I ran all
the actor failure tests that exercise this shutdown path 20 times
locally, and where they failed 3/20 previously, they succeeded everytime
after the fix.

Signed-off-by: Sagar Sumit <[email protected]>
Signed-off-by: YK <[email protected]>
SheldonTsen pushed a commit to SheldonTsen/ray that referenced this pull request Dec 1, 2025
… deleted (ray-project#58605)

The tests that exercised actor failures when they go out of scope, such
as `test_actor_ray_shutdown_called_on_del` and
`test_actor_ray_shutdown_called_on_scope_exit` [were
flaky](https://buildkite.com/ray-project/postmerge/builds/14336#019a7abe-73d3-46e0-8dc2-13351e12b7c3/613-1919).
This PR fixes the flakiness by ensuring actors use graceful shutdown
when GCS polling detects actor refs are deleted.

**Problem**
When actors go out of scope, GCS uses two mechanisms to detect reference
deletion:
1. Push model (`GcsActorManager::HandleReportActorOutOfScope`) - already
fixed in ray-project#57090
2. Pull model (`GcsActorManager::PollOwnerForActorRefDeleted`) - was
still using force kill

The pull model was calling DestroyActor(..., force_kill=true), which
skips `__ray_shutdown__` and immediately terminates the actor. This
created a race condition: whichever mechanism completed first determined
whether cleanup callbacks ran, causing test flakiness.

To fix the issue, changed `PollOwnerForActorRefDeleted` to use graceful
shutdown with timeout (same as `HandleReportActorOutOfScope`). I ran all
the actor failure tests that exercise this shutdown path 20 times
locally, and where they failed 3/20 previously, they succeeded everytime
after the fix.

Signed-off-by: Sagar Sumit <[email protected]>
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.

4 participants