-
Notifications
You must be signed in to change notification settings - Fork 5.9k
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
base: master
Are you sure you want to change the base?
[core] Resolve the race condition between reference counting GC and actor creation #49480
Conversation
Signed-off-by: kaihsun <[email protected]>
Curious how can I reproduce the TSAN failure? |
Signed-off-by: kaihsun <[email protected]>
d764f4e
to
157f57a
Compare
@dentiny I wrote a detailed PR description. We can chat in-person if you have further questions. |
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? |
// 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))); |
There was a problem hiding this comment.
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:
- ALIVE
- DEAD and non-restartable: not in
registered_actors_
andnamed_actors_
- 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.
There was a problem hiding this comment.
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.
Why are these changes needed?
Problem statement
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:
get_actor
to retrieve the actor if it exists._remote
to create a new actor.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:
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 fromcached_actor_name_to_ids_
by a callback function.cached_actor_name_to_ids_
, it will call SyncGetByName to ask GCS.SyncGetByName
. GCS returns Status::NotFound to the core worker under the following conditions:registered_actors_
andnamed_actors
, but itsActorState
isrpc::ActorTableData::DEAD
.Status::NotFound
from the GCS and returnsStatus::NotFound
, causing an exception to be raised in Python.actor.py
catches the exception, and then calls self._remote to create an actor.name
andnamespace
, but a different actor ID is generated. As a result, the old named actor's actor ID remains inregistered_actors_
, and this request does not enter the if statement.name
andnamespace
are used to look upnamed_actors
, and an error is returned because the old / new named actors share the samename
andnamespace
. Hence, the actor creation failed.get_actor
again, but it will fail.DestroyActor
to remove the old named actor fromregistered_actors_
andnamed_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 inregistered_actors_
, not innamed_actors
, or if the actor's state is DEAD. See this line for more details.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 updateregistered_actors_
andnamed_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_
andnamed_actors
are updated if it plans to return an error status to the core worker in (Step 1-3). In this PR, we callDestroyActor
to updateregistered_actors_
andnamed_actors
before returning the status to the core worker.Related issue number
Closes #48856
Checks
git commit -s
) in this PR.scripts/format.sh
to lint the changes in this PR.method in Tune, I've added it in
doc/source/tune/api/
under thecorresponding
.rst
file.