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]: Enable get_actor_name for actor runtime context #39347

Merged
merged 10 commits into from
Sep 11, 2023
6 changes: 6 additions & 0 deletions python/ray/_private/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -495,6 +495,12 @@ def actor_id(self):
return self.core_worker.get_actor_id()
return ActorID.nil()

@property
def actor_name(self):
if hasattr(self, "core_worker"):
return self.core_worker.get_actor_name().decode("utf-8")
return None

@property
def current_task_id(self):
return self.core_worker.get_current_task_id()
Expand Down
3 changes: 3 additions & 0 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -3073,6 +3073,9 @@ cdef class CoreWorker:
return ActorID(
CCoreWorkerProcess.GetCoreWorker().GetActorId().Binary())

def get_actor_name(self):
return CCoreWorkerProcess.GetCoreWorker().GetActorName()

def get_placement_group_id(self):
return PlacementGroupID(
CCoreWorkerProcess.GetCoreWorker()
Expand Down
1 change: 1 addition & 0 deletions python/ray/includes/libcoreworker.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,7 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
CWorkerID GetWorkerID()
c_bool ShouldCaptureChildTasksInPlacementGroup()
const CActorID &GetActorId()
const c_string GetActorName()
void SetActorTitle(const c_string &title)
void SetActorReprName(const c_string &repr_name)
void SetWebuiDisplay(const c_string &key, const c_string &message)
Expand Down
23 changes: 22 additions & 1 deletion python/ray/runtime_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -254,12 +254,33 @@ def get_actor_id(self) -> Optional[str]:
if self.worker.mode != ray._private.worker.WORKER_MODE:
logger.warning(
"This method is only available when the process is a "
"worker. Current mode: {self.worker.mode}"
f"worker. Current mode: {self.worker.mode}"
)
return None
actor_id = self.worker.actor_id
return actor_id.hex() if not actor_id.is_nil() else None

def get_actor_name(self) -> Optional[str]:
"""Get the current actor name of this worker.

This shouldn't be used in a driver process.
The name is in string format.

Returns:
The current actor name of this worker.
If a current worker is an actor, and
if actor name doesn't exist, it returns an empty string.
If a current worker is not an actor, it returns None.
"""
# only worker mode has actor_id
if self.worker.mode != ray._private.worker.WORKER_MODE:
logger.warning(
"This method is only available when the process is a "
f"worker. Current mode: {self.worker.mode}"
)
actor_id = self.worker.actor_id
return self.worker.actor_name if not actor_id.is_nil() else None

@property
def namespace(self):
"""Get the current namespace of this worker.
Expand Down
24 changes: 24 additions & 0 deletions python/ray/tests/test_runtime_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -345,6 +345,30 @@ def foo(self):
actor = FooActor.remote()
ray.get(actor.foo.remote())

# actor name
@ray.remote
class NamedActor:
def name(self):
return ray.get_runtime_context().get_actor_name()

ACTOR_NAME = "actor_name"
named_actor = NamedActor.options(name=ACTOR_NAME).remote()
assert ray.get(named_actor.name.remote()) == ACTOR_NAME

jonathan-anyscale marked this conversation as resolved.
Show resolved Hide resolved
# unnamed actor name
unnamed_actor = NamedActor.options().remote()
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you add tests to verify all the specs from the API?

Make sure if it is a task, it returns "None".
Make sure if it is a driver, it returns "None".
Make sure if it has no actor name, it returns "".
Make sure if it is an actor and has a name, it returns the name

assert ray.get(unnamed_actor.name.remote()) == ""

# task actor name
@ray.remote
def task_actor_name():
ray.get_runtime_context().get_actor_name()

assert ray.get(task_actor_name.remote()) is None

# driver actor name
assert rtc.get_actor_name() is None


def test_auto_init(shutdown_only):
assert not ray.is_initialized()
Expand Down
5 changes: 5 additions & 0 deletions src/ray/core_worker/core_worker.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2486,6 +2486,11 @@ CoreWorker::ListNamedActorsLocalMode() {
return std::make_pair(actors, Status::OK());
}

const std::string CoreWorker::GetActorName() const {
absl::MutexLock lock(&mutex_);
return actor_manager_->GetActorHandle(actor_id_)->GetName();
}

const ResourceMappingType CoreWorker::GetResourceIDs() const {
absl::MutexLock lock(&mutex_);
return *resource_ids_;
Expand Down
2 changes: 2 additions & 0 deletions src/ray/core_worker/core_worker.h
Original file line number Diff line number Diff line change
Expand Up @@ -973,6 +973,8 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler {

const ActorID &GetActorId() const { return actor_id_; }

const std::string GetActorName() const;

// Get the resource IDs available to this worker (as assigned by the raylet).
const ResourceMappingType GetResourceIDs() const;

Expand Down
Loading