mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Update named actor API (#8559)
This commit is contained in:
parent
92c2e41dfd
commit
860eb6f13a
10 changed files with 109 additions and 140 deletions
|
@ -467,12 +467,10 @@ For even finer-grained control over training, you can use RLlib's lower-level `b
|
||||||
|
|
||||||
Global Coordination
|
Global Coordination
|
||||||
~~~~~~~~~~~~~~~~~~~
|
~~~~~~~~~~~~~~~~~~~
|
||||||
Sometimes, it is necessary to coordinate between pieces of code that live in different processes managed by RLlib. For example, it can be useful to maintain a global average of a certain variable, or centrally control a hyperparameter used by policies. Ray provides a general way to achieve this through *named actors* (learn more about Ray actors `here <actors.html>`__). As an example, consider maintaining a shared global counter that is incremented by environments and read periodically from your driver program:
|
Sometimes, it is necessary to coordinate between pieces of code that live in different processes managed by RLlib. For example, it can be useful to maintain a global average of a certain variable, or centrally control a hyperparameter used by policies. Ray provides a general way to achieve this through *detached actors* (learn more about Ray actors `here <actors.html>`__). These actors are assigned a global name and handles to them can be retrieved using these names. As an example, consider maintaining a shared global counter that is incremented by environments and read periodically from your driver program:
|
||||||
|
|
||||||
.. code-block:: python
|
.. code-block:: python
|
||||||
|
|
||||||
from ray.util import named_actors
|
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
class Counter:
|
class Counter:
|
||||||
def __init__(self):
|
def __init__(self):
|
||||||
|
@ -483,12 +481,11 @@ Sometimes, it is necessary to coordinate between pieces of code that live in dif
|
||||||
return self.count
|
return self.count
|
||||||
|
|
||||||
# on the driver
|
# on the driver
|
||||||
counter = Counter.remote()
|
counter = Counter.options(name="global_counter").remote()
|
||||||
named_actors.register_actor("global_counter", counter)
|
|
||||||
print(ray.get(counter.get.remote())) # get the latest count
|
print(ray.get(counter.get.remote())) # get the latest count
|
||||||
|
|
||||||
# in your envs
|
# in your envs
|
||||||
counter = named_actors.get_actor("global_counter")
|
counter = ray.get_actor("global_counter")
|
||||||
counter.inc.remote(1) # async call to increment the global count
|
counter.inc.remote(1) # async call to increment the global count
|
||||||
|
|
||||||
Ray actors provide high levels of performance, so in more complex cases they can be used implement communication patterns such as parameter servers and allreduce.
|
Ray actors provide high levels of performance, so in more complex cases they can be used implement communication patterns such as parameter servers and allreduce.
|
||||||
|
|
|
@ -78,6 +78,7 @@ from ray.worker import (
|
||||||
connect,
|
connect,
|
||||||
disconnect,
|
disconnect,
|
||||||
get,
|
get,
|
||||||
|
get_actor,
|
||||||
get_gpu_ids,
|
get_gpu_ids,
|
||||||
get_resource_ids,
|
get_resource_ids,
|
||||||
get_webui_url,
|
get_webui_url,
|
||||||
|
@ -126,6 +127,7 @@ __all__ = [
|
||||||
"connect",
|
"connect",
|
||||||
"disconnect",
|
"disconnect",
|
||||||
"get",
|
"get",
|
||||||
|
"get_actor",
|
||||||
"get_gpu_ids",
|
"get_gpu_ids",
|
||||||
"get_resource_ids",
|
"get_resource_ids",
|
||||||
"get_webui_url",
|
"get_webui_url",
|
||||||
|
|
|
@ -435,8 +435,7 @@ class ActorClass:
|
||||||
asyncio execution. Note that the execution order is not
|
asyncio execution. Note that the execution order is not
|
||||||
guaranteed when max_concurrency > 1.
|
guaranteed when max_concurrency > 1.
|
||||||
name: The globally unique name for the actor.
|
name: The globally unique name for the actor.
|
||||||
detached: Whether the actor should be kept alive after driver
|
detached: DEPRECATED.
|
||||||
exits.
|
|
||||||
|
|
||||||
Returns:
|
Returns:
|
||||||
A handle to the newly created actor.
|
A handle to the newly created actor.
|
||||||
|
@ -469,18 +468,16 @@ class ActorClass:
|
||||||
raise RuntimeError("Actors cannot be created before ray.init() "
|
raise RuntimeError("Actors cannot be created before ray.init() "
|
||||||
"has been called.")
|
"has been called.")
|
||||||
|
|
||||||
if detached and name is None:
|
if detached:
|
||||||
raise ValueError("Detached actors must be named. "
|
logger.warning("The detached flag is deprecated. To create a "
|
||||||
"Please use Actor._remote(name='some_name') "
|
"detached actor, use the name parameter.")
|
||||||
"to associate the name.")
|
|
||||||
|
|
||||||
if name and not detached:
|
if name is not None:
|
||||||
raise ValueError("Only detached actors can be named. "
|
if not isinstance(name, str):
|
||||||
"Please use Actor._remote(detached=True, "
|
raise TypeError("name must be None or a string, "
|
||||||
"name='some_name').")
|
"got: '{}'.".format(type(name)))
|
||||||
|
if name == "":
|
||||||
if name == "":
|
raise ValueError("Actor name cannot be an empty string.")
|
||||||
raise ValueError("Actor name cannot be an empty string.")
|
|
||||||
|
|
||||||
# Check whether the name is already taken.
|
# Check whether the name is already taken.
|
||||||
# TODO(edoakes): this check has a race condition because two drivers
|
# TODO(edoakes): this check has a race condition because two drivers
|
||||||
|
@ -489,14 +486,17 @@ class ActorClass:
|
||||||
# async call.
|
# async call.
|
||||||
if name is not None:
|
if name is not None:
|
||||||
try:
|
try:
|
||||||
ray.util.get_actor(name)
|
ray.get_actor(name)
|
||||||
except ValueError: # Name is not taken.
|
except ValueError: # Name is not taken.
|
||||||
pass
|
pass
|
||||||
else:
|
else:
|
||||||
raise ValueError(
|
raise ValueError(
|
||||||
"The name {name} is already taken. Please use "
|
"The name {name} is already taken. Please use "
|
||||||
"a different name or get existing actor using "
|
"a different name or get the existing actor using "
|
||||||
"ray.util.get_actor('{name}')".format(name=name))
|
"ray.get_actor('{name}')".format(name=name))
|
||||||
|
detached = True
|
||||||
|
else:
|
||||||
|
detached = False
|
||||||
|
|
||||||
# Set the actor's default resources if not already set. First three
|
# Set the actor's default resources if not already set. First three
|
||||||
# conditions are to check that no resources were specified in the
|
# conditions are to check that no resources were specified in the
|
||||||
|
@ -583,7 +583,7 @@ class ActorClass:
|
||||||
original_handle=True)
|
original_handle=True)
|
||||||
|
|
||||||
if name is not None and not gcs_actor_service_enabled():
|
if name is not None and not gcs_actor_service_enabled():
|
||||||
ray.util.register_actor(name, actor_handle)
|
ray.util.named_actors._register_actor(name, actor_handle)
|
||||||
|
|
||||||
return actor_handle
|
return actor_handle
|
||||||
|
|
||||||
|
@ -762,12 +762,6 @@ class ActorHandle:
|
||||||
self._ray_actor_creation_function_descriptor.class_name,
|
self._ray_actor_creation_function_descriptor.class_name,
|
||||||
self._actor_id.hex())
|
self._actor_id.hex())
|
||||||
|
|
||||||
def __ray_kill__(self):
|
|
||||||
"""Deprecated - use ray.kill() instead."""
|
|
||||||
logger.warning("actor.__ray_kill__() is deprecated and will be removed"
|
|
||||||
" in the near future. Use ray.kill(actor) instead.")
|
|
||||||
ray.kill(self)
|
|
||||||
|
|
||||||
@property
|
@property
|
||||||
def _actor_id(self):
|
def _actor_id(self):
|
||||||
return self._ray_actor_id
|
return self._ray_actor_id
|
||||||
|
|
|
@ -107,7 +107,7 @@ def init(cluster_name=None,
|
||||||
global master_actor
|
global master_actor
|
||||||
master_actor_name = format_actor_name(SERVE_MASTER_NAME, cluster_name)
|
master_actor_name = format_actor_name(SERVE_MASTER_NAME, cluster_name)
|
||||||
try:
|
try:
|
||||||
master_actor = ray.util.get_actor(master_actor_name)
|
master_actor = ray.get_actor(master_actor_name)
|
||||||
return
|
return
|
||||||
except ValueError:
|
except ValueError:
|
||||||
pass
|
pass
|
||||||
|
@ -124,7 +124,6 @@ def init(cluster_name=None,
|
||||||
# in the future.
|
# in the future.
|
||||||
http_node_id = ray.state.current_node_id()
|
http_node_id = ray.state.current_node_id()
|
||||||
master_actor = ServeMaster.options(
|
master_actor = ServeMaster.options(
|
||||||
detached=True,
|
|
||||||
name=master_actor_name,
|
name=master_actor_name,
|
||||||
max_restarts=-1,
|
max_restarts=-1,
|
||||||
).remote(cluster_name, start_server, http_node_id, http_host, http_port,
|
).remote(cluster_name, start_server, http_node_id, http_host, http_port,
|
||||||
|
|
|
@ -127,11 +127,10 @@ class ServeMaster:
|
||||||
"""
|
"""
|
||||||
router_name = format_actor_name(SERVE_ROUTER_NAME, self.cluster_name)
|
router_name = format_actor_name(SERVE_ROUTER_NAME, self.cluster_name)
|
||||||
try:
|
try:
|
||||||
self.router = ray.util.get_actor(router_name)
|
self.router = ray.get_actor(router_name)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
logger.info("Starting router with name '{}'".format(router_name))
|
logger.info("Starting router with name '{}'".format(router_name))
|
||||||
self.router = async_retryable(ray.remote(Router)).options(
|
self.router = async_retryable(ray.remote(Router)).options(
|
||||||
detached=True,
|
|
||||||
name=router_name,
|
name=router_name,
|
||||||
max_concurrency=ASYNC_CONCURRENCY,
|
max_concurrency=ASYNC_CONCURRENCY,
|
||||||
max_restarts=-1,
|
max_restarts=-1,
|
||||||
|
@ -148,13 +147,12 @@ class ServeMaster:
|
||||||
"""
|
"""
|
||||||
proxy_name = format_actor_name(SERVE_PROXY_NAME, self.cluster_name)
|
proxy_name = format_actor_name(SERVE_PROXY_NAME, self.cluster_name)
|
||||||
try:
|
try:
|
||||||
self.http_proxy = ray.util.get_actor(proxy_name)
|
self.http_proxy = ray.get_actor(proxy_name)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
logger.info(
|
logger.info(
|
||||||
"Starting HTTP proxy with name '{}' on node '{}'".format(
|
"Starting HTTP proxy with name '{}' on node '{}'".format(
|
||||||
proxy_name, node_id))
|
proxy_name, node_id))
|
||||||
self.http_proxy = async_retryable(HTTPProxyActor).options(
|
self.http_proxy = async_retryable(HTTPProxyActor).options(
|
||||||
detached=True,
|
|
||||||
name=proxy_name,
|
name=proxy_name,
|
||||||
max_concurrency=ASYNC_CONCURRENCY,
|
max_concurrency=ASYNC_CONCURRENCY,
|
||||||
max_restarts=-1,
|
max_restarts=-1,
|
||||||
|
@ -180,12 +178,11 @@ class ServeMaster:
|
||||||
metric_sink_name = format_actor_name(SERVE_METRIC_SINK_NAME,
|
metric_sink_name = format_actor_name(SERVE_METRIC_SINK_NAME,
|
||||||
self.cluster_name)
|
self.cluster_name)
|
||||||
try:
|
try:
|
||||||
self.metric_exporter = ray.util.get_actor(metric_sink_name)
|
self.metric_exporter = ray.get_actor(metric_sink_name)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
logger.info("Starting metric exporter with name '{}'".format(
|
logger.info("Starting metric exporter with name '{}'".format(
|
||||||
metric_sink_name))
|
metric_sink_name))
|
||||||
self.metric_exporter = MetricExporterActor.options(
|
self.metric_exporter = MetricExporterActor.options(
|
||||||
detached=True,
|
|
||||||
name=metric_sink_name).remote(metric_exporter_class)
|
name=metric_sink_name).remote(metric_exporter_class)
|
||||||
|
|
||||||
def get_metric_exporter(self):
|
def get_metric_exporter(self):
|
||||||
|
@ -246,7 +243,7 @@ class ServeMaster:
|
||||||
for replica_tag in replica_tags:
|
for replica_tag in replica_tags:
|
||||||
replica_name = format_actor_name(replica_tag,
|
replica_name = format_actor_name(replica_tag,
|
||||||
self.cluster_name)
|
self.cluster_name)
|
||||||
self.workers[backend_tag][replica_tag] = ray.util.get_actor(
|
self.workers[backend_tag][replica_tag] = ray.get_actor(
|
||||||
replica_name)
|
replica_name)
|
||||||
|
|
||||||
# Push configuration state to the router.
|
# Push configuration state to the router.
|
||||||
|
@ -311,7 +308,6 @@ class ServeMaster:
|
||||||
|
|
||||||
replica_name = format_actor_name(replica_tag, self.cluster_name)
|
replica_name = format_actor_name(replica_tag, self.cluster_name)
|
||||||
worker_handle = async_retryable(ray.remote(backend_worker)).options(
|
worker_handle = async_retryable(ray.remote(backend_worker)).options(
|
||||||
detached=True,
|
|
||||||
name=replica_name,
|
name=replica_name,
|
||||||
max_restarts=-1,
|
max_restarts=-1,
|
||||||
**replica_config.ray_actor_options).remote(
|
**replica_config.ray_actor_options).remote(
|
||||||
|
@ -328,7 +324,7 @@ class ServeMaster:
|
||||||
# failed after creating them but before writing a
|
# failed after creating them but before writing a
|
||||||
# checkpoint.
|
# checkpoint.
|
||||||
try:
|
try:
|
||||||
worker_handle = ray.util.get_actor(replica_tag)
|
worker_handle = ray.get_actor(replica_tag)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
worker_handle = await self._start_backend_worker(
|
worker_handle = await self._start_backend_worker(
|
||||||
backend_tag, replica_tag)
|
backend_tag, replica_tag)
|
||||||
|
@ -371,7 +367,7 @@ class ServeMaster:
|
||||||
# NOTE(edoakes): the replicas may already be stopped if we
|
# NOTE(edoakes): the replicas may already be stopped if we
|
||||||
# failed after stopping them but before writing a checkpoint.
|
# failed after stopping them but before writing a checkpoint.
|
||||||
try:
|
try:
|
||||||
replica = ray.util.get_actor(replica_tag)
|
replica = ray.get_actor(replica_tag)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
continue
|
continue
|
||||||
|
|
||||||
|
@ -384,9 +380,7 @@ class ServeMaster:
|
||||||
# use replica.__ray_terminate__, we may send it while the
|
# use replica.__ray_terminate__, we may send it while the
|
||||||
# replica is being restarted and there's no way to tell if it
|
# replica is being restarted and there's no way to tell if it
|
||||||
# successfully killed the worker or not.
|
# successfully killed the worker or not.
|
||||||
worker = ray.worker.global_worker
|
ray.kill(replica, no_restart=True)
|
||||||
# Kill the actor with no_restart=True.
|
|
||||||
worker.core_worker.kill_actor(replica._ray_actor_id, True)
|
|
||||||
|
|
||||||
self.replicas_to_stop.clear()
|
self.replicas_to_stop.clear()
|
||||||
|
|
||||||
|
|
|
@ -35,7 +35,7 @@ def test_master_failure(serve_instance):
|
||||||
response = request_with_retries("/master_failure", timeout=30)
|
response = request_with_retries("/master_failure", timeout=30)
|
||||||
assert response.text == "hello1"
|
assert response.text == "hello1"
|
||||||
|
|
||||||
ray.kill(serve.api._get_master_actor())
|
ray.kill(serve.api._get_master_actor(), no_restart=False)
|
||||||
|
|
||||||
for _ in range(10):
|
for _ in range(10):
|
||||||
response = request_with_retries("/master_failure", timeout=30)
|
response = request_with_retries("/master_failure", timeout=30)
|
||||||
|
@ -44,7 +44,7 @@ def test_master_failure(serve_instance):
|
||||||
def function():
|
def function():
|
||||||
return "hello2"
|
return "hello2"
|
||||||
|
|
||||||
ray.kill(serve.api._get_master_actor())
|
ray.kill(serve.api._get_master_actor(), no_restart=False)
|
||||||
|
|
||||||
serve.create_backend("master_failure:v2", function)
|
serve.create_backend("master_failure:v2", function)
|
||||||
serve.set_traffic("master_failure", {"master_failure:v2": 1.0})
|
serve.set_traffic("master_failure", {"master_failure:v2": 1.0})
|
||||||
|
@ -56,11 +56,11 @@ def test_master_failure(serve_instance):
|
||||||
def function():
|
def function():
|
||||||
return "hello3"
|
return "hello3"
|
||||||
|
|
||||||
ray.kill(serve.api._get_master_actor())
|
ray.kill(serve.api._get_master_actor(), no_restart=False)
|
||||||
serve.create_endpoint("master_failure_2", "/master_failure_2")
|
serve.create_endpoint("master_failure_2", "/master_failure_2")
|
||||||
ray.kill(serve.api._get_master_actor())
|
ray.kill(serve.api._get_master_actor(), no_restart=False)
|
||||||
serve.create_backend("master_failure_2", function)
|
serve.create_backend("master_failure_2", function)
|
||||||
ray.kill(serve.api._get_master_actor())
|
ray.kill(serve.api._get_master_actor(), no_restart=False)
|
||||||
serve.set_traffic("master_failure_2", {"master_failure_2": 1.0})
|
serve.set_traffic("master_failure_2", {"master_failure_2": 1.0})
|
||||||
|
|
||||||
for _ in range(10):
|
for _ in range(10):
|
||||||
|
@ -73,7 +73,7 @@ def test_master_failure(serve_instance):
|
||||||
def _kill_http_proxy():
|
def _kill_http_proxy():
|
||||||
[http_proxy] = ray.get(
|
[http_proxy] = ray.get(
|
||||||
serve.api._get_master_actor().get_http_proxy.remote())
|
serve.api._get_master_actor().get_http_proxy.remote())
|
||||||
ray.kill(http_proxy)
|
ray.kill(http_proxy, no_restart=False)
|
||||||
|
|
||||||
|
|
||||||
def test_http_proxy_failure(serve_instance):
|
def test_http_proxy_failure(serve_instance):
|
||||||
|
@ -107,7 +107,7 @@ def test_http_proxy_failure(serve_instance):
|
||||||
|
|
||||||
def _kill_router():
|
def _kill_router():
|
||||||
[router] = ray.get(serve.api._get_master_actor().get_router.remote())
|
[router] = ray.get(serve.api._get_master_actor().get_router.remote())
|
||||||
ray.kill(router)
|
ray.kill(router, no_restart=False)
|
||||||
|
|
||||||
|
|
||||||
def test_router_failure(serve_instance):
|
def test_router_failure(serve_instance):
|
||||||
|
@ -169,7 +169,7 @@ def test_worker_restart(serve_instance):
|
||||||
# Kill the worker.
|
# Kill the worker.
|
||||||
handles = _get_worker_handles("worker_failure:v1")
|
handles = _get_worker_handles("worker_failure:v1")
|
||||||
assert len(handles) == 1
|
assert len(handles) == 1
|
||||||
ray.kill(handles[0])
|
ray.kill(handles[0], no_restart=False)
|
||||||
|
|
||||||
# Wait until the worker is killed and a one is started.
|
# Wait until the worker is killed and a one is started.
|
||||||
start = time.time()
|
start = time.time()
|
||||||
|
@ -227,7 +227,7 @@ def test_worker_replica_failure(serve_instance):
|
||||||
# Kill one of the replicas.
|
# Kill one of the replicas.
|
||||||
handles = _get_worker_handles("replica_failure")
|
handles = _get_worker_handles("replica_failure")
|
||||||
assert len(handles) == 2
|
assert len(handles) == 2
|
||||||
ray.kill(handles[0])
|
ray.kill(handles[0], no_restart=False)
|
||||||
|
|
||||||
# Check that the other replica still serves requests.
|
# Check that the other replica still serves requests.
|
||||||
for _ in range(10):
|
for _ in range(10):
|
||||||
|
|
|
@ -655,63 +655,22 @@ def test_pickled_actor_handle_call_in_method_twice(ray_start_regular):
|
||||||
ray.get(b.step.remote())
|
ray.get(b.step.remote())
|
||||||
|
|
||||||
|
|
||||||
def test_register_and_get_named_actors(ray_start_regular):
|
|
||||||
# TODO(heyucongtom): We should test this from another driver.
|
|
||||||
|
|
||||||
@ray.remote
|
|
||||||
class Foo:
|
|
||||||
def __init__(self):
|
|
||||||
self.x = 0
|
|
||||||
|
|
||||||
def method(self):
|
|
||||||
self.x += 1
|
|
||||||
return self.x
|
|
||||||
|
|
||||||
f1 = Foo.remote()
|
|
||||||
# Test saving f.
|
|
||||||
ray.util.register_actor("f1", f1)
|
|
||||||
# Test getting f.
|
|
||||||
f2 = ray.util.get_actor("f1")
|
|
||||||
assert f1._actor_id == f2._actor_id
|
|
||||||
|
|
||||||
# Test same name register shall raise error.
|
|
||||||
with pytest.raises(ValueError):
|
|
||||||
ray.util.register_actor("f1", f2)
|
|
||||||
|
|
||||||
# Test register with wrong object type.
|
|
||||||
with pytest.raises(TypeError):
|
|
||||||
ray.util.register_actor("f3", 1)
|
|
||||||
|
|
||||||
# Test getting a nonexistent actor.
|
|
||||||
with pytest.raises(ValueError):
|
|
||||||
ray.util.get_actor("nonexistent")
|
|
||||||
|
|
||||||
# Test method
|
|
||||||
assert ray.get(f1.method.remote()) == 1
|
|
||||||
assert ray.get(f2.method.remote()) == 2
|
|
||||||
assert ray.get(f1.method.remote()) == 3
|
|
||||||
assert ray.get(f2.method.remote()) == 4
|
|
||||||
|
|
||||||
|
|
||||||
def test_detached_actor(ray_start_regular):
|
def test_detached_actor(ray_start_regular):
|
||||||
@ray.remote
|
@ray.remote
|
||||||
class DetachedActor:
|
class DetachedActor:
|
||||||
def ping(self):
|
def ping(self):
|
||||||
return "pong"
|
return "pong"
|
||||||
|
|
||||||
|
with pytest.raises(TypeError):
|
||||||
|
DetachedActor._remote(name=1)
|
||||||
|
|
||||||
with pytest.raises(
|
with pytest.raises(
|
||||||
ValueError, match="Actor name cannot be an empty string"):
|
ValueError, match="Actor name cannot be an empty string"):
|
||||||
DetachedActor._remote(detached=True, name="")
|
DetachedActor._remote(name="")
|
||||||
|
|
||||||
with pytest.raises(ValueError, match="Detached actors must be named"):
|
DetachedActor._remote(name="d_actor")
|
||||||
DetachedActor._remote(detached=True)
|
|
||||||
|
|
||||||
with pytest.raises(ValueError, match="Only detached actors can be named"):
|
|
||||||
DetachedActor._remote(name="d_actor")
|
|
||||||
|
|
||||||
DetachedActor._remote(detached=True, name="d_actor")
|
|
||||||
with pytest.raises(ValueError, match="Please use a different name"):
|
with pytest.raises(ValueError, match="Please use a different name"):
|
||||||
DetachedActor._remote(detached=True, name="d_actor")
|
DetachedActor._remote(name="d_actor")
|
||||||
|
|
||||||
redis_address = ray_start_regular["redis_address"]
|
redis_address = ray_start_regular["redis_address"]
|
||||||
|
|
||||||
|
@ -721,7 +680,7 @@ def test_detached_actor(ray_start_regular):
|
||||||
import ray
|
import ray
|
||||||
ray.init(address="{}")
|
ray.init(address="{}")
|
||||||
|
|
||||||
existing_actor = ray.util.get_actor("{}")
|
existing_actor = ray.get_actor("{}")
|
||||||
assert ray.get(existing_actor.ping.remote()) == "pong"
|
assert ray.get(existing_actor.ping.remote()) == "pong"
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
|
@ -729,17 +688,16 @@ class DetachedActor:
|
||||||
def ping(self):
|
def ping(self):
|
||||||
return "pong"
|
return "pong"
|
||||||
|
|
||||||
actor = DetachedActor._remote(name="{}", detached=True)
|
actor = DetachedActor._remote(name="{}")
|
||||||
ray.get(actor.ping.remote())
|
ray.get(actor.ping.remote())
|
||||||
""".format(redis_address, get_actor_name, create_actor_name)
|
""".format(redis_address, get_actor_name, create_actor_name)
|
||||||
|
|
||||||
run_string_as_driver(driver_script)
|
run_string_as_driver(driver_script)
|
||||||
detached_actor = ray.util.get_actor(create_actor_name)
|
detached_actor = ray.get_actor(create_actor_name)
|
||||||
assert ray.get(detached_actor.ping.remote()) == "pong"
|
assert ray.get(detached_actor.ping.remote()) == "pong"
|
||||||
|
|
||||||
|
|
||||||
@pytest.mark.parametrize("deprecated_codepath", [False, True])
|
def test_kill(ray_start_regular):
|
||||||
def test_kill(ray_start_regular, deprecated_codepath):
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
class Actor:
|
class Actor:
|
||||||
def hang(self):
|
def hang(self):
|
||||||
|
@ -750,17 +708,13 @@ def test_kill(ray_start_regular, deprecated_codepath):
|
||||||
result = actor.hang.remote()
|
result = actor.hang.remote()
|
||||||
ready, _ = ray.wait([result], timeout=0.5)
|
ready, _ = ray.wait([result], timeout=0.5)
|
||||||
assert len(ready) == 0
|
assert len(ready) == 0
|
||||||
if deprecated_codepath:
|
ray.kill(actor, no_restart=False)
|
||||||
actor.__ray_kill__()
|
|
||||||
else:
|
|
||||||
ray.kill(actor)
|
|
||||||
|
|
||||||
with pytest.raises(ray.exceptions.RayActorError):
|
with pytest.raises(ray.exceptions.RayActorError):
|
||||||
ray.get(result)
|
ray.get(result)
|
||||||
|
|
||||||
if not deprecated_codepath:
|
with pytest.raises(ValueError):
|
||||||
with pytest.raises(ValueError):
|
ray.kill("not_an_actor_handle")
|
||||||
ray.kill("not_an_actor_handle")
|
|
||||||
|
|
||||||
|
|
||||||
# This test verifies actor creation task failure will not
|
# This test verifies actor creation task failure will not
|
||||||
|
|
|
@ -219,8 +219,7 @@ class Counter:
|
||||||
def increment(self):
|
def increment(self):
|
||||||
self.count += 1
|
self.count += 1
|
||||||
return self.count
|
return self.count
|
||||||
counter = Counter.remote()
|
counter = Counter.options(name="Counter").remote()
|
||||||
ray.util.register_actor("Counter", counter)
|
|
||||||
time.sleep(100)
|
time.sleep(100)
|
||||||
""".format(address)
|
""".format(address)
|
||||||
|
|
||||||
|
@ -231,7 +230,7 @@ import time
|
||||||
ray.init(address="{}")
|
ray.init(address="{}")
|
||||||
while True:
|
while True:
|
||||||
try:
|
try:
|
||||||
counter = ray.util.get_actor("Counter")
|
counter = ray.get_actor("Counter")
|
||||||
break
|
break
|
||||||
except ValueError:
|
except ValueError:
|
||||||
time.sleep(1)
|
time.sleep(1)
|
||||||
|
|
|
@ -1,7 +1,11 @@
|
||||||
|
import logging
|
||||||
|
|
||||||
import ray
|
import ray
|
||||||
import ray.cloudpickle as pickle
|
import ray.cloudpickle as pickle
|
||||||
from ray.experimental.internal_kv import _internal_kv_get, _internal_kv_put
|
from ray.experimental.internal_kv import _internal_kv_get, _internal_kv_put
|
||||||
|
|
||||||
|
logger = logging.getLogger(__name__)
|
||||||
|
|
||||||
|
|
||||||
def _calculate_key(name):
|
def _calculate_key(name):
|
||||||
"""Generate a Redis key with the given name.
|
"""Generate a Redis key with the given name.
|
||||||
|
@ -15,17 +19,7 @@ def _calculate_key(name):
|
||||||
return b"Actor:" + name.encode("ascii")
|
return b"Actor:" + name.encode("ascii")
|
||||||
|
|
||||||
|
|
||||||
def get_actor(name):
|
def _get_actor(name):
|
||||||
"""Get a named actor which was previously created.
|
|
||||||
|
|
||||||
If the actor doesn't exist, an exception will be raised.
|
|
||||||
|
|
||||||
Args:
|
|
||||||
name: The name of the named actor.
|
|
||||||
|
|
||||||
Returns:
|
|
||||||
The ActorHandle object corresponding to the name.
|
|
||||||
"""
|
|
||||||
if ray._raylet.gcs_actor_service_enabled():
|
if ray._raylet.gcs_actor_service_enabled():
|
||||||
worker = ray.worker.global_worker
|
worker = ray.worker.global_worker
|
||||||
handle = worker.core_worker.get_named_actor_handle(name)
|
handle = worker.core_worker.get_named_actor_handle(name)
|
||||||
|
@ -40,13 +34,23 @@ def get_actor(name):
|
||||||
return handle
|
return handle
|
||||||
|
|
||||||
|
|
||||||
def register_actor(name, actor_handle):
|
def get_actor(name):
|
||||||
"""Register a named actor under a string key.
|
"""Get a named actor which was previously created.
|
||||||
|
|
||||||
Args:
|
If the actor doesn't exist, an exception will be raised.
|
||||||
name: The name of the named actor.
|
|
||||||
actor_handle: The actor object to be associated with this name
|
Args:
|
||||||
"""
|
name: The name of the named actor.
|
||||||
|
|
||||||
|
Returns:
|
||||||
|
The ActorHandle object corresponding to the name.
|
||||||
|
"""
|
||||||
|
logger.warning("ray.util.get_actor has been moved to ray.get_actor and "
|
||||||
|
"will be removed in the future.")
|
||||||
|
return _get_actor(name)
|
||||||
|
|
||||||
|
|
||||||
|
def _register_actor(name, actor_handle):
|
||||||
if not isinstance(name, str):
|
if not isinstance(name, str):
|
||||||
raise TypeError("The name argument must be a string.")
|
raise TypeError("The name argument must be a string.")
|
||||||
if not isinstance(actor_handle, ray.actor.ActorHandle):
|
if not isinstance(actor_handle, ray.actor.ActorHandle):
|
||||||
|
@ -56,7 +60,7 @@ def register_actor(name, actor_handle):
|
||||||
|
|
||||||
# First check if the actor already exists.
|
# First check if the actor already exists.
|
||||||
try:
|
try:
|
||||||
get_actor(name)
|
_get_actor(name)
|
||||||
exists = True
|
exists = True
|
||||||
except ValueError:
|
except ValueError:
|
||||||
exists = False
|
exists = False
|
||||||
|
@ -66,3 +70,15 @@ def register_actor(name, actor_handle):
|
||||||
|
|
||||||
# Add the actor to Redis if it does not already exist.
|
# Add the actor to Redis if it does not already exist.
|
||||||
_internal_kv_put(actor_name, pickle.dumps(actor_handle))
|
_internal_kv_put(actor_name, pickle.dumps(actor_handle))
|
||||||
|
|
||||||
|
|
||||||
|
def register_actor(name, actor_handle):
|
||||||
|
"""Register a named actor under a string key.
|
||||||
|
|
||||||
|
Args:
|
||||||
|
name: The name of the named actor.
|
||||||
|
actor_handle: The actor object to be associated with this name
|
||||||
|
"""
|
||||||
|
logger.warning("ray.util.register_actor is deprecated. To create a "
|
||||||
|
"named, detached actor, use Actor.options(name=\"name\").")
|
||||||
|
return _register_actor(name, actor_handle)
|
||||||
|
|
|
@ -1657,7 +1657,22 @@ def wait(object_ids, num_returns=1, timeout=None):
|
||||||
return ready_ids, remaining_ids
|
return ready_ids, remaining_ids
|
||||||
|
|
||||||
|
|
||||||
def kill(actor):
|
def get_actor(name):
|
||||||
|
"""Get a handle to a detached actor.
|
||||||
|
|
||||||
|
Gets a handle to a detached actor with the given name. The actor must
|
||||||
|
have been created with Actor.options(name="name").remote().
|
||||||
|
|
||||||
|
Returns:
|
||||||
|
ActorHandle to the actor.
|
||||||
|
|
||||||
|
Raises:
|
||||||
|
ValueError if the named actor does not exist.
|
||||||
|
"""
|
||||||
|
return ray.util.named_actors._get_actor(name)
|
||||||
|
|
||||||
|
|
||||||
|
def kill(actor, no_restart=True):
|
||||||
"""Kill an actor forcefully.
|
"""Kill an actor forcefully.
|
||||||
|
|
||||||
This will interrupt any running tasks on the actor, causing them to fail
|
This will interrupt any running tasks on the actor, causing them to fail
|
||||||
|
@ -1667,21 +1682,20 @@ def kill(actor):
|
||||||
you can call ``actor.__ray_terminate__.remote()`` instead to queue a
|
you can call ``actor.__ray_terminate__.remote()`` instead to queue a
|
||||||
termination task.
|
termination task.
|
||||||
|
|
||||||
In both cases, the worker is actually killed, but it will be restarted by
|
If the actor is a detached actor, subsequent calls to get its handle via
|
||||||
Ray.
|
ray.get_actor will fail.
|
||||||
|
|
||||||
If this actor is reconstructable, an attempt will be made to reconstruct
|
|
||||||
it.
|
|
||||||
|
|
||||||
Args:
|
Args:
|
||||||
actor (ActorHandle): Handle to the actor to kill.
|
actor (ActorHandle): Handle to the actor to kill.
|
||||||
|
no_restart (bool): Whether or not this actor should be restarted if
|
||||||
|
it's a restartable actor.
|
||||||
"""
|
"""
|
||||||
if not isinstance(actor, ray.actor.ActorHandle):
|
if not isinstance(actor, ray.actor.ActorHandle):
|
||||||
raise ValueError("ray.kill() only supported for actors. "
|
raise ValueError("ray.kill() only supported for actors. "
|
||||||
"Got: {}.".format(type(actor)))
|
"Got: {}.".format(type(actor)))
|
||||||
worker = ray.worker.global_worker
|
worker = ray.worker.global_worker
|
||||||
worker.check_connected()
|
worker.check_connected()
|
||||||
worker.core_worker.kill_actor(actor._ray_actor_id, False)
|
worker.core_worker.kill_actor(actor._ray_actor_id, no_restart)
|
||||||
|
|
||||||
|
|
||||||
def cancel(object_id, force=False):
|
def cancel(object_id, force=False):
|
||||||
|
|
Loading…
Add table
Reference in a new issue