mirror of
https://github.com/vale981/ray
synced 2025-03-05 18:11:42 -05:00
Change actor.__ray_kill__() to ray.kill(actor) (#7360)
This commit is contained in:
parent
3fc162f93c
commit
93fe4b0b58
6 changed files with 44 additions and 16 deletions
|
@ -147,7 +147,7 @@ collected. The ``ObjectID`` resulting from the task can be waited on to wait
|
|||
for the actor to exit (calling ``ray.get()`` on it will raise a ``RayActorError``).
|
||||
Note that this method of termination will wait until any previously submitted
|
||||
tasks finish executing. If you want to terminate an actor immediately, you can
|
||||
call ``actor_handle.__ray_kill__()``. This will cause the actor to exit immediately
|
||||
call ``ray.kill(actor_handle)``. This will cause the actor to exit immediately
|
||||
and any pending tasks to fail. Any exit handlers installed in the actor using
|
||||
``atexit`` will be called.
|
||||
|
||||
|
|
|
@ -13,6 +13,8 @@ Ray Package Reference
|
|||
|
||||
.. autofunction:: ray.put
|
||||
|
||||
.. autofunction:: ray.kill
|
||||
|
||||
.. autofunction:: ray.get_gpu_ids
|
||||
|
||||
.. autofunction:: ray.get_resource_ids
|
||||
|
|
|
@ -83,6 +83,7 @@ from ray.worker import (
|
|||
init,
|
||||
is_initialized,
|
||||
put,
|
||||
kill,
|
||||
register_custom_serializer,
|
||||
remote,
|
||||
shutdown,
|
||||
|
@ -135,6 +136,7 @@ __all__ = [
|
|||
"profile",
|
||||
"projects",
|
||||
"put",
|
||||
"kill",
|
||||
"register_custom_serializer",
|
||||
"remote",
|
||||
"shutdown",
|
||||
|
|
|
@ -783,18 +783,10 @@ class ActorHandle:
|
|||
self.__ray_terminate__._actor_hard_ref = None
|
||||
|
||||
def __ray_kill__(self):
|
||||
"""Kill the actor that this actor handle refers to immediately.
|
||||
|
||||
This will cause any outstanding tasks submitted to the actor to fail
|
||||
and the actor to exit in the same way as if it crashed. In general,
|
||||
you should prefer to just delete the actor handle and let it clean up
|
||||
gracefull.
|
||||
|
||||
Returns:
|
||||
None.
|
||||
"""
|
||||
worker = ray.worker.get_global_worker()
|
||||
worker.core_worker.kill_actor(self._ray_actor_id)
|
||||
"""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
|
||||
def _actor_id(self):
|
||||
|
|
|
@ -686,7 +686,8 @@ ray.get(actor.ping.remote())
|
|||
assert ray.get(detached_actor.ping.remote()) == "pong"
|
||||
|
||||
|
||||
def test_kill(ray_start_regular):
|
||||
@pytest.mark.parametrize("deprecated_codepath", [False, True])
|
||||
def test_kill(ray_start_regular, deprecated_codepath):
|
||||
@ray.remote
|
||||
class Actor:
|
||||
def hang(self):
|
||||
|
@ -697,10 +698,18 @@ def test_kill(ray_start_regular):
|
|||
result = actor.hang.remote()
|
||||
ready, _ = ray.wait([result], timeout=0.5)
|
||||
assert len(ready) == 0
|
||||
actor.__ray_kill__()
|
||||
if deprecated_codepath:
|
||||
actor.__ray_kill__()
|
||||
else:
|
||||
ray.kill(actor)
|
||||
|
||||
with pytest.raises(ray.exceptions.RayActorError):
|
||||
ray.get(result)
|
||||
|
||||
if not deprecated_codepath:
|
||||
with pytest.raises(ValueError):
|
||||
ray.kill("not_an_actor_handle")
|
||||
|
||||
|
||||
# This test verifies actor creation task failure will not
|
||||
# hang the caller.
|
||||
|
|
|
@ -1641,6 +1641,29 @@ def wait(object_ids, num_returns=1, timeout=None):
|
|||
return ready_ids, remaining_ids
|
||||
|
||||
|
||||
def kill(actor):
|
||||
"""Kill an actor forcefully.
|
||||
|
||||
This will interrupt any running tasks on the actor, causing them to fail
|
||||
immediately. Any atexit handlers installed in the actor will still be run.
|
||||
|
||||
If you want to kill the actor but let pending tasks finish,
|
||||
you can call ``actor.__ray_terminate__.remote()`` instead to queue a
|
||||
termination task.
|
||||
|
||||
If this actor is reconstructable, it will be attempted to be reconstructed.
|
||||
|
||||
Args:
|
||||
actor (ActorHandle): Handle to the actor to kill.
|
||||
"""
|
||||
if not isinstance(actor, ray.actor.ActorHandle):
|
||||
raise ValueError("ray.kill() only supported for actors. "
|
||||
"Got: {}.".format(type(actor)))
|
||||
|
||||
worker = ray.worker.get_global_worker()
|
||||
worker.core_worker.kill_actor(actor._ray_actor_id)
|
||||
|
||||
|
||||
def _mode(worker=global_worker):
|
||||
"""This is a wrapper around worker.mode.
|
||||
|
||||
|
@ -1775,7 +1798,7 @@ def remote(*args, **kwargs):
|
|||
Running remote actors will be terminated when the actor handle to them
|
||||
in Python is deleted, which will cause them to complete any outstanding
|
||||
work and then shut down. If you want to kill them immediately, you can
|
||||
also call ``actor_handle.__ray_kill__()``.
|
||||
also call ``ray.kill(actor)``.
|
||||
"""
|
||||
worker = get_global_worker()
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue