Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Prototype named actors. #2129

Merged
merged 6 commits into from
May 24, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -869,7 +869,8 @@ def _serialization_helper(self, ray_forking):
_ray_actor_creation_dummy_object_id.id(),
"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(),
"previous_actor_handle_id": self._ray_actor_handle_id.id()
if self._ray_actor_handle_id else None,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this change needed?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm.. ok I see that the test failures without it.

"ray_forking": ray_forking
}

Expand Down
3 changes: 2 additions & 1 deletion python/ray/experimental/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,10 +7,11 @@
flush_redis_unsafe, flush_task_and_object_metadata_unsafe,
flush_finished_tasks_unsafe, flush_evicted_objects_unsafe,
_flush_finished_tasks_unsafe_shard, _flush_evicted_objects_unsafe_shard)
from .named_actors import get_actor, register_actor

__all__ = [
"TensorFlowVariables", "flush_redis_unsafe",
"flush_task_and_object_metadata_unsafe", "flush_finished_tasks_unsafe",
"flush_evicted_objects_unsafe", "_flush_finished_tasks_unsafe_shard",
"_flush_evicted_objects_unsafe_shard"
"_flush_evicted_objects_unsafe_shard", "get_actor", "register_actor"
]
61 changes: 61 additions & 0 deletions python/ray/experimental/named_actors.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
from __future__ import absolute_import
from __future__ import division
from __future__ import print_function

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's reserve assertions for checking things that should never actually happen. The errors (in both register_actor and get_actor) can happen if the user passes in the wrong values, so we should raise some sort of exception

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Handled.

import ray
import ray.cloudpickle as pickle


def _calculate_key(name):
"""Generate a Redis key with the given name.

Args:
name: The name of the named actor.

Returns:
The key to use for storing a named actor in Redis.
"""
return b"Actor:" + name.encode("ascii")


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.
"""
worker = ray.worker.get_global_worker()
actor_hash = _calculate_key(name)
pickled_state = worker.redis_client.hget(actor_hash, name)
if pickled_state is None:
raise ValueError("The actor with name={} doesn't exist".format(name))
handle = pickle.loads(pickled_state)
return 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
"""
worker = ray.worker.get_global_worker()
if not isinstance(name, str):
raise TypeError("The name argument must be a string.")
if not isinstance(actor_handle, ray.actor.ActorHandle):
raise TypeError("The actor_handle argument must be an ActorHandle "
"object.")
actor_hash = _calculate_key(name)
pickled_state = pickle.dumps(actor_handle)

# Add the actor to Redis if it does not already exist.
updated = worker.redis_client.hsetnx(actor_hash, name, pickled_state)
if updated == 0:
raise ValueError(
"Error: the actor with name={} already exists".format(name))
38 changes: 38 additions & 0 deletions test/actor_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -1906,6 +1906,44 @@ def method(self):
# we should also test this from a different driver.
ray.get(new_f.method.remote())

def testRegisterAndGetNamedActors(self):
# TODO(heyucongtom): We should test this from another driver.
ray.worker.init(num_workers=1)

@ray.remote
class Foo(object):
def __init__(self):
self.x = 0

def method(self):
self.x += 1
return self.x

f1 = Foo.remote()
# Test saving f.
ray.experimental.register_actor("f1", f1)
# Test getting f.
f2 = ray.experimental.get_actor("f1")
self.assertEqual(f1._actor_id, f2._actor_id)

# Test same name register shall raise error.
with self.assertRaises(ValueError):
ray.experimental.register_actor("f1", f2)

# Test register with wrong object type.
with self.assertRaises(TypeError):
ray.experimental.register_actor("f3", 1)

# Test getting a nonexistent actor.
with self.assertRaises(ValueError):
ray.experimental.get_actor("nonexistent")

# Test method
self.assertEqual(ray.get(f1.method.remote()), 1)
self.assertEqual(ray.get(f2.method.remote()), 2)
self.assertEqual(ray.get(f1.method.remote()), 3)
self.assertEqual(ray.get(f2.method.remote()), 4)


class ActorPlacementAndResources(unittest.TestCase):
def tearDown(self):
Expand Down