mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Fix actor handle id creation when actor handle was pickled (#3074)
This commit is contained in:
parent
3c0803e7e9
commit
2c52d9dfa0
2 changed files with 35 additions and 0 deletions
|
@ -501,6 +501,7 @@ class ActorHandle(object):
|
|||
self._ray_actor_method_cpus = actor_method_cpus
|
||||
self._ray_actor_driver_id = actor_driver_id
|
||||
self._ray_previous_actor_handle_id = previous_actor_handle_id
|
||||
self._ray_previously_generated_actor_handle_id = None
|
||||
|
||||
def _actor_method_call(self,
|
||||
method_name,
|
||||
|
@ -554,10 +555,22 @@ class ActorHandle(object):
|
|||
|
||||
is_actor_checkpoint_method = (method_name == "__ray_checkpoint__")
|
||||
|
||||
# Right now, if the actor handle has been pickled, we create a
|
||||
# temporary actor handle id for invocations.
|
||||
# TODO(pcm): This still leads to a lot of actor handles being
|
||||
# created, there should be a better way to handle pickled
|
||||
# actor handles.
|
||||
if self._ray_actor_handle_id is None:
|
||||
actor_handle_id = compute_actor_handle_id_non_forked(
|
||||
self._ray_actor_id, self._ray_previous_actor_handle_id,
|
||||
worker.current_task_id)
|
||||
# Each new task creates a new actor handle id, so we need to
|
||||
# reset the actor counter to 0
|
||||
if (actor_handle_id !=
|
||||
self._ray_previously_generated_actor_handle_id):
|
||||
self._ray_actor_counter = 0
|
||||
self._ray_previously_generated_actor_handle_id = (
|
||||
actor_handle_id)
|
||||
else:
|
||||
actor_handle_id = self._ray_actor_handle_id
|
||||
|
||||
|
|
|
@ -1968,6 +1968,28 @@ def test_pickling_actor_handle(ray_start_regular):
|
|||
ray.get(new_f.method.remote())
|
||||
|
||||
|
||||
def test_pickled_actor_handle_call_in_method_twice(ray_start_regular):
|
||||
@ray.remote
|
||||
class Actor1(object):
|
||||
def f(self):
|
||||
return 1
|
||||
|
||||
@ray.remote
|
||||
class Actor2(object):
|
||||
def __init__(self, constructor):
|
||||
self.actor = constructor()
|
||||
|
||||
def step(self):
|
||||
ray.get(self.actor.f.remote())
|
||||
|
||||
a = Actor1.remote()
|
||||
|
||||
b = Actor2.remote(lambda: a)
|
||||
|
||||
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.
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue