Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[core] Resolve the race condition between reference counting GC and actor creation #49480

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

kevin85421
Copy link
Member

@kevin85421 kevin85421 commented Dec 29, 2024

Why are these changes needed?

Problem statement

import ray

@ray.remote
class Actor:
    pass

ray.init()

for i in range(0, 10000):
    print(i)
    Actor.options(
        name="actor",
        get_if_exists=True,
        max_restarts=-1,
    ).remote()

In the above script, Actor.remote() with get_if_exists=True is called multiple times, and the actor handle goes out of scope immediately. When get_if_exists is set to True, the process can be briefly divided into three steps. See this function for more details:

  • Step 1: Ray calls get_actor to retrieve the actor if it exists.
  • Step 2: If the actor does not exist, Ray attempts to call _remote to create a new actor.
  • Step 3: If step 2 fails to create a new actor, Ray calls get_actor again.

Next, we need to understand how get_actor and actor creation work.

  • get_actor: The most important function is ActorManager::GetNamedActorHandle. First, the actor manager checks whether the named actor's information exists in the local cache. If not, it calls the SyncGetByName function to query the GCS for information about the named actor. If the named actor is not found in either the actor manager's cache or the GCS, an error status is returned, and an exception is thrown in Python.

  • actor creation: The most important function is GcsActorManager::RegisterActor.

The bug happens in the following case:

  • (Step 1-1) The core worker cannot find the named actor in the cached_actor_name_to_ids_ cache because when the actor handle goes out of scope and the reference count of the actor reaches 0, the named actor is removed from cached_actor_name_to_ids_ by a callback function.
  • (Step 1-2) Because the core worker can't find the named actor in the cached_actor_name_to_ids_, it will call SyncGetByName to ask GCS.
  • (Step 1-3) GCS uses GcsActorManager::HandleGetNamedActorInfo to handle SyncGetByName. GCS returns Status::NotFound to the core worker under the following conditions:
    • The named actor exists in registered_actors_ and named_actors, but its ActorState is rpc::ActorTableData::DEAD.
  • (Step 1-4) The core worker receives Status::NotFound from the GCS and returns Status::NotFound, causing an exception to be raised in Python.
  • (Step 2-1) actor.py catches the exception, and then calls self._remote to create an actor.
  • (Step 2-2) GcsActorManager::RegisterActor: The request attempts to create a new actor with the same name and namespace, but a different actor ID is generated. As a result, the old named actor's actor ID remains in registered_actors_, and this request does not enter the if statement.
  • (Step 2-3) GcsActorManager::RegisterActor: Next, name and namespace are used to look up named_actors, and an error is returned because the old / new named actors share the same name and namespace. Hence, the actor creation failed.
  • (Step 3) Call get_actor again, but it will fail.
  • The GCS finally receives the notification about the actor going out of scope and then calls DestroyActor to remove the old named actor from registered_actors_ and named_actors.

To summarize, the bug occurs because the GCS has a different understanding of whether an actor exists during "get actor" and create actor operations.

  • get_actor: The GCS considers an actor nonexistent if it is not in registered_actors_, not in named_actors, or if the actor's state is DEAD. See this line for more details.
  • actor creation: The GCS considers an actor to exist if it is present in named_actors.

The differing definitions of "exist" cause the bug when the GCS fails to detect an actor going out of scope in time and calls DestroyActor to update registered_actors_ and named_actors.

Solution

The goal of the solution is to ensure the GCS has a consistent understanding of whether an actor "exists," regardless of when users attempt to "get" or "create" an actor.

The solution is that the GCS must ensure registered_actors_ and named_actors are updated if it plans to return an error status to the core worker in (Step 1-3). In this PR, we call DestroyActor to update registered_actors_ and named_actors before returning the status to the core worker.

Related issue number

Closes #48856

Checks

pytest -vs ray/tests/test_get_or_create_actor.py::test_get_or_create_named_actor
  • Without this PR, it failed all 10 times out of 10 runs.
  • With this PR, it passed all 10 times out of 10 runs.
  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • 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 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 :(

Signed-off-by: kaihsun <[email protected]>
@dentiny
Copy link
Contributor

dentiny commented Dec 29, 2024

Curious how can I reproduce the TSAN failure?

Signed-off-by: kaihsun <[email protected]>
@kevin85421 kevin85421 force-pushed the 20241226-devbox1-tmux-14-ray1 branch from d764f4e to 157f57a Compare December 29, 2024 05:04
@kevin85421 kevin85421 marked this pull request as ready for review December 29, 2024 06:14
@kevin85421 kevin85421 requested a review from a team as a code owner December 29, 2024 06:14
@kevin85421
Copy link
Member Author

@dentiny I wrote a detailed PR description. We can chat in-person if you have further questions.

@kevin85421 kevin85421 changed the title [WIP][core] Resolve the race condition between reference counting GC and actor creation [core] Resolve the race condition between reference counting GC and actor creation Dec 29, 2024
@rynewang
Copy link
Contributor

I think this relates to a similar actor refcnt issue, that I found 2023, that actually dates back to a very old unresolved issue at ~2019... but I can't find that issue. @jjyao do you have memories on actor refcount issues?

@kevin85421 kevin85421 assigned kevin85421 and unassigned dentiny and rynewang Dec 30, 2024
@jjyao jjyao added the go add ONLY when ready to merge, run all tests label Dec 30, 2024
@jjyao jjyao assigned rynewang and unassigned kevin85421 Dec 30, 2024
// conditions. Without this, GCS might tell the core worker that the actor is not
// found when the core worker tries to `get_actor`, but still think the actor exists
// when the core worker tries to create a new actor with the same name.
DestroyActor(actor_id, GenActorRefDeletedCause(GetActor(actor_id)));
Copy link
Collaborator

Choose a reason for hiding this comment

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

After #49480 an actor has three states:

  1. ALIVE
  2. DEAD and non-restartable: not in registered_actors_ and named_actors_
  3. DEAD and restartable: in registered_actors_ and named_actors_

The semantic is clear for get_if_exist if the actor is in state 1 or 2. What we need to figure out is the behavior when the actor is in state 3.

Copy link
Collaborator

Choose a reason for hiding this comment

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

One option is to bring it back to state 1.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Core] Repeated calls to create an Actor with get_or_create=True and max_restarts != 0 can fail
4 participants