Update named actor API (#8559)

This commit is contained in:
Edward Oakes 2020-05-24 20:08:03 -05:00 committed by GitHub
parent 92c2e41dfd
commit 860eb6f13a
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
10 changed files with 109 additions and 140 deletions

View file

@ -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.

View file

@ -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",

View file

@ -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

View file

@ -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,

View file

@ -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()

View file

@ -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):

View file

@ -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

View file

@ -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)

View file

@ -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)

View file

@ -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):