Skip to content

Commit

Permalink
[core] Add generic __ray_ready__ method to Actor classes (ray-proje…
Browse files Browse the repository at this point in the history
…ct#31997)

We currently have no canonical way to await actors. Users can define their own _is-ready_ methods, schedule a future, and await these, but this has to be done for every actor class separately.

This does not match other patterns - e.g. we have `actor.__ray_terminate__.remote()` for actor termination and `placement_group.ready()` for placement group ready futures.

This PR adds a new `__ray_ready__` magic actor method that just returns `True`. It can be used to await actors becoming ready (newly scheduled actors), and actors having processed all of their other enqueued tasks.

Signed-off-by: Kai Fricke <kai@anyscale.com>
  • Loading branch information
krfricke authored and clarence-wu committed Jan 31, 2023
1 parent 1824e75 commit 5c18973
Show file tree
Hide file tree
Showing 2 changed files with 25 additions and 3 deletions.
7 changes: 5 additions & 2 deletions python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -1316,11 +1316,14 @@ def _modify_class(cls):
"'class ClassName(object):' instead of 'class ClassName:'."
)

# Modify the class to have an additional method that will be used for
# terminating the worker.
# Modify the class to have additional methods
# for checking actor alive status and to terminate the worker.
class Class(cls):
__ray_actor_class__ = cls # The original actor class

def __ray_ready__(self):
return True

def __ray_terminate__(self):
worker = ray._private.worker.global_worker
if worker.mode != ray.LOCAL_MODE:
Expand Down
21 changes: 20 additions & 1 deletion python/ray/tests/test_actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -1108,6 +1108,20 @@ def check_file_written():
os.unlink(tmpfile.name)


def test_actor_ready(ray_start_regular_shared):
@ray.remote
class Actor:
pass

actor = Actor.remote()

with pytest.raises(TypeError):
# Method can't be called directly
actor.__ray_ready__()

assert ray.get(actor.__ray_ready__.remote())


def test_return_actor_handle_from_actor(ray_start_regular_shared):
@ray.remote
class Inner:
Expand Down Expand Up @@ -1148,7 +1162,12 @@ def method_one(self) -> None:
assert methods == ["method_one"]

all_methods = set(dir(f))
assert all_methods == {"__init__", "method_one", "__ray_terminate__"}
assert all_methods == {
"__init__",
"method_one",
"__ray_ready__",
"__ray_terminate__",
}

method_options = [fn for fn in dir(f.method_one) if not fn.startswith("_")]

Expand Down

0 comments on commit 5c18973

Please sign in to comment.