-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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] Weakref semantics for actor handles retrieved from self or by name #45699
[core] Weakref semantics for actor handles retrieved from self or by name #45699
Conversation
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.
Try to understand the issue:
- (1) If the actor handle is obtained from "out-of-band" (either
ray.get_runtime_context().current_actor
orray.get_actor
), the reference count will not increment. - (2) However, when the "out-of-band" actor handles go out of the scope. The reference count will decrement.
- (3) When the "in-band" actor task finishes, we can't find the actor handle in the reference count table.
Is my understanding correct?
This PR loosens the check in step (4) to only throw an assertion failure if the ObjectID is a real object, i.e. not an actor handle.
I don't have enough context about reference counting, but this solution seems a bit strange to me. Why wouldn't the reference count be incremented when there is an out-of-band actor handle? Hence, (3) will not fail.
This PR appears to not correct the incorrect value in the reference count table. Instead, it looses the assertion. What happens if the reference count value is wrong? Will it affect the correctness of Ray applications?
/*deduct_local_ref=*/true, | ||
&borrowed_refs); | ||
if (!ref_found) { | ||
RAY_CHECK(ObjectID::IsActorID(borrowed_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.
Should we make those "out-of-band" cases in-band instead so that reference counting is correct. With this PR, it's still possible that reference count table is 0 while the application still has a reference to the actor handle.
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
# Update ref counting for the actor handle | ||
self.add_contained_object_ref(actor_handle_id) | ||
return _actor_handle_deserializer, (serialized,) | ||
if not weak_ref: |
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.
Is weak_ref
equivalent to out-of-band? If they are equivalent, maybe we can update add_contained_object_ref
instead, which already has some logic to determine whether the serialization is in-band or out-of-band.
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.
It is not quite equivalent. add_contained_ref
is used to track which ObjectRefs are passed inside other objects. But we don't want to count weakref ActorHandles stored inside other objects towards the ref count. Will add a comment.
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.
Thank you for the explanation!
return self.make_actor_handle( | ||
CCoreWorkerProcess.GetCoreWorker().GetActorHandle(c_actor_id)) | ||
CCoreWorkerProcess.GetCoreWorker().GetActorHandle(c_actor_id), | ||
weak_ref) | ||
|
||
def get_named_actor_handle(self, const c_string &name, |
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.
Is this to handle ray.get_actor
?
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.
It's used in a few places, I think for both get_actor and getting the self actor.
@@ -4429,13 +4436,15 @@ cdef class CoreWorker: | |||
name, ray_namespace)) | |||
check_status(named_actor_handle_pair.second) | |||
|
|||
return self.make_actor_handle(named_actor_handle_pair.first) | |||
return self.make_actor_handle(named_actor_handle_pair.first, | |||
weak_ref=True) | |||
|
|||
def get_actor_handle(self, ActorID 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.
Is this to handle runtime_context().current_actor
?
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.
This is for get_actor
by name.
reference_counter_->AddOwnedObject(actor_creation_return_id, | ||
/*inner_ids=*/{}, | ||
caller_address, | ||
call_site, | ||
/*object_size*/ -1, | ||
/*is_reconstructable=*/true, | ||
/*add_local_ref=*/false); |
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.
Is /*add_local_ref=*/false
a bug?
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.
It used to be right because we would add the local ref in the call to AddActorHandle, but now we might as well do it when we add the owned object.
src/ray/core_worker/actor_manager.cc
Outdated
AddNewActorHandle(std::move(actor_handle), | ||
call_site, | ||
caller_address, | ||
/*is_detached*/ true); |
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.
Must a named actor be a detached actor? I am not sure why is_detached = True
here.
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.
Hmm I just copied this from before, but good catch, I don't think it needs to be. I'll try updating to is_detached = false
and see if it breaks anything.
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.
Oh actually maybe the arg is just named poorly. I think it is more about whether we own the actor or not, will update.
A lot of CI tests fail. Maybe we need to rebase with the master branch. |
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.
The change looks good to me. The CI has a lot of failures. Maybe we need to rebase with the master branch?
@@ -389,7 +389,9 @@ std::string AbstractRayRuntime::DeserializeAndRegisterActorHandle( | |||
const std::string &serialized_actor_handle) { |
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.
So we fix the reference cycle of a <-> a but what about a <-> b?
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.
I don't think we've ever supported GC for cycles with strong refs. But if they both got the actors by name, then it should work. Will add a test.
This reverts commit 87cdf57.
…f or by name (ray-project#45699)" This reverts commit d729815.
…from self or by name (ray-project#45699)"" This reverts commit aaa6364.
Why are these changes needed?
The distributed ref counting protocol assumes that ObjectRefs and ActorHandles are only ever passed in-band, through task arguments and return values. However, it is possible to get actor handles "out-of-band" through two APIs:
ray.get_runtime_context().current_actor
returns a handle to theself
actorray.get_actor
can get an actor handle by nameWe used to assert that when a task finished, its arguments were still registered in the ref count table. That way, we could return the remaining ref count back to the task's caller. This assertion is too strong and causes crash in the following case:
ray.get_runtime_context().current_actor
This PR updates ref counting for actors to distinguish between strong refs and weak refs. Strong refs are handles passed via task args and return values from the original handle. These are tracked by the ref counting protocol and will keep the actor alive. Weak refs are handles created from the above cases and are not tracked by the ref counting protocol (doing so would require additional protocols to notify the owner of the new handle).
This PR also fixes the related #45704 by not incrementing the self ref count on the actor itself when the actor is first created.
Related issue number
Closes #45704.
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.