mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Try fixing actor handle destruction on py2 (#6076)
This commit is contained in:
parent
1c5446851a
commit
fbad6f543b
2 changed files with 16 additions and 4 deletions
|
@ -95,6 +95,8 @@ class ActorMethod(object):
|
|||
# actor method handles to remote functions.
|
||||
if hardref:
|
||||
self._actor_hard_ref = actor
|
||||
else:
|
||||
self._actor_hard_ref = None
|
||||
|
||||
def __call__(self, *args, **kwargs):
|
||||
raise Exception("Actor methods cannot be called directly. Instead "
|
||||
|
@ -110,7 +112,7 @@ class ActorMethod(object):
|
|||
num_return_vals = self._num_return_vals
|
||||
|
||||
def invocation(args, kwargs):
|
||||
actor = self._actor_ref()
|
||||
actor = self._actor_hard_ref or self._actor_ref()
|
||||
if actor is None:
|
||||
raise RuntimeError("Lost reference to actor")
|
||||
return actor._actor_method_call(
|
||||
|
@ -615,9 +617,13 @@ class ActorHandle(object):
|
|||
self._ray_class_name)
|
||||
return
|
||||
if worker.connected and self._ray_original_handle:
|
||||
# TODO(rkn): Should we be passing in the actor cursor as a
|
||||
# dependency here?
|
||||
self.__ray_terminate__.remote()
|
||||
# Note: in py2 the weakref is destroyed prior to calling __del__
|
||||
# so we need to set the hardref here briefly
|
||||
try:
|
||||
self.__ray_terminate__._actor_hard_ref = self
|
||||
self.__ray_terminate__.remote()
|
||||
finally:
|
||||
self.__ray_terminate__._actor_hard_ref = None
|
||||
|
||||
@property
|
||||
def _actor_id(self):
|
||||
|
|
|
@ -805,6 +805,8 @@ def test_keyword_args(ray_start_regular):
|
|||
assert ray.get(f3.remote(4)) == 4
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
sys.version_info < (3, 0), reason="This test requires Python 3.")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_regular", [{
|
||||
"local_mode": True
|
||||
|
@ -840,6 +842,8 @@ def test_args_starkwargs(ray_start_regular):
|
|||
ray.get(remote_test_function.remote(local_method, actor_method))
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
sys.version_info < (3, 0), reason="This test requires Python 3.")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_regular", [{
|
||||
"local_mode": True
|
||||
|
@ -881,6 +885,8 @@ def test_args_named_and_star(ray_start_regular):
|
|||
ray.get(remote_test_function.remote(local_method, actor_method))
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
sys.version_info < (3, 0), reason="This test requires Python 3.")
|
||||
@pytest.mark.parametrize(
|
||||
"ray_start_regular", [{
|
||||
"local_mode": True
|
||||
|
|
Loading…
Add table
Reference in a new issue