mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Make actor handles work in Python mode. (#2283)
* Make actor handles work in local mode. * Add test for actor handles in local mode.
This commit is contained in:
parent
ff2217251f
commit
800f7cc77d
2 changed files with 18 additions and 4 deletions
|
@ -884,13 +884,15 @@ class ActorHandle(object):
|
|||
"actor_id": self._ray_actor_id.id(),
|
||||
"class_name": self._ray_class_name,
|
||||
"actor_forks": self._ray_actor_forks,
|
||||
"actor_cursor": self._ray_actor_cursor.id(),
|
||||
"actor_cursor": self._ray_actor_cursor.id()
|
||||
if self._ray_actor_cursor is not None else None,
|
||||
"actor_counter": 0, # Reset the actor counter.
|
||||
"actor_method_names": self._ray_actor_method_names,
|
||||
"method_signatures": self._ray_method_signatures,
|
||||
"method_num_return_vals": self._ray_method_num_return_vals,
|
||||
"actor_creation_dummy_object_id": self.
|
||||
_ray_actor_creation_dummy_object_id.id(),
|
||||
_ray_actor_creation_dummy_object_id.id()
|
||||
if self._ray_actor_creation_dummy_object_id is not None else None,
|
||||
"actor_method_cpus": self._ray_actor_method_cpus,
|
||||
"actor_driver_id": self._ray_actor_driver_id.id(),
|
||||
"previous_actor_handle_id": self._ray_actor_handle_id.id()
|
||||
|
@ -929,12 +931,14 @@ class ActorHandle(object):
|
|||
self.__init__(
|
||||
ray.ObjectID(state["actor_id"]),
|
||||
state["class_name"],
|
||||
ray.ObjectID(state["actor_cursor"]),
|
||||
ray.ObjectID(state["actor_cursor"])
|
||||
if state["actor_cursor"] is not None else None,
|
||||
state["actor_counter"],
|
||||
state["actor_method_names"],
|
||||
state["method_signatures"],
|
||||
state["method_num_return_vals"],
|
||||
ray.ObjectID(state["actor_creation_dummy_object_id"]),
|
||||
ray.ObjectID(state["actor_creation_dummy_object_id"])
|
||||
if state["actor_creation_dummy_object_id"] is not None else None,
|
||||
state["actor_method_cpus"],
|
||||
actor_driver_id,
|
||||
actor_handle_id=actor_handle_id,
|
||||
|
|
|
@ -1189,6 +1189,16 @@ class PythonModeTest(unittest.TestCase):
|
|||
test_array[0] = -1
|
||||
assert_equal(test_array, test_actor.get_array.remote())
|
||||
|
||||
# Check that actor handles work in Python mode.
|
||||
|
||||
@ray.remote
|
||||
def use_actor_handle(handle):
|
||||
array = np.ones(10)
|
||||
handle.set_array.remote(array)
|
||||
assert np.alltrue(array == ray.get(handle.get_array.remote()))
|
||||
|
||||
ray.get(use_actor_handle.remote(test_actor))
|
||||
|
||||
|
||||
class ResourcesTest(unittest.TestCase):
|
||||
def tearDown(self):
|
||||
|
|
Loading…
Add table
Reference in a new issue