Skip to content

Commit

Permalink
Simplify imports and exports and provide driver isolation for remote …
Browse files Browse the repository at this point in the history
…functions. (ray-project#288)

* Remove import counter and export counter.

* Provide isolation between drivers for remote functions.

* Add test for driver function isolation.

* Hash source code into function ID to reduce likelihood of collisions.

* Fix failure test example.

* Replace assertTrue with assertIn to improve failure messages in tests.

* Fix failure test.
  • Loading branch information
robertnishihara authored and pcmoritz committed Feb 16, 2017
1 parent 883f945 commit 88a5b4e
Show file tree
Hide file tree
Showing 6 changed files with 250 additions and 179 deletions.
44 changes: 21 additions & 23 deletions python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,18 +27,19 @@ def get_actor_method_function_id(attr):
Returns:
Function ID corresponding to the method.
"""
function_id = hashlib.sha1()
function_id.update(attr.encode("ascii"))
return photon.ObjectID(function_id.digest())
function_id_hash = hashlib.sha1()
function_id_hash.update(attr.encode("ascii"))
function_id = function_id_hash.digest()
assert len(function_id) == 20
return photon.ObjectID(function_id)

def fetch_and_register_actor(key, worker):
"""Import an actor."""
driver_id, actor_id_str, actor_name, module, pickled_class, class_export_counter = \
worker.redis_client.hmget(key, ["driver_id", "actor_id", "name", "module", "class", "class_export_counter"])
driver_id, actor_id_str, actor_name, module, pickled_class = \
worker.redis_client.hmget(key, ["driver_id", "actor_id", "name", "module", "class"])
actor_id = photon.ObjectID(actor_id_str)
actor_name = actor_name.decode("ascii")
module = module.decode("ascii")
class_export_counter = int(class_export_counter)
try:
unpickled_class = pickling.loads(pickled_class)
except:
Expand All @@ -49,45 +50,43 @@ def fetch_and_register_actor(key, worker):
worker.actors[actor_id_str] = unpickled_class.__new__(unpickled_class)
for (k, v) in inspect.getmembers(unpickled_class, predicate=(lambda x: inspect.isfunction(x) or inspect.ismethod(x))):
function_id = get_actor_method_function_id(k).id()
worker.function_names[function_id] = k
worker.functions[function_id] = v
worker.functions[driver_id][function_id] = (k, v)
# We do not set worker.function_properties[driver_id][function_id] because
# we currently do need the actor worker to submit new tasks for the actor.

def export_actor(actor_id, Class, worker):
def export_actor(actor_id, Class, actor_method_names, worker):
"""Export an actor to redis.
Args:
actor_id: The ID of the actor.
Class: Name of the class to be exported as an actor.
worker: The worker class
actor_method_names (list): A list of the names of this actor's methods.
"""
ray.worker.check_main_thread()
if worker.mode is None:
raise NotImplemented("TODO(pcm): Cache actors")
key = "Actor:{}".format(actor_id.id())
pickled_class = pickling.dumps(Class)

# For now, all actor methods have 1 return value and require 0 CPUs and GPUs.
driver_id = worker.task_driver_id.id()
for actor_method_name in actor_method_names:
function_id = get_actor_method_function_id(actor_method_name).id()
worker.function_properties[driver_id][function_id] = (1, 0, 0)

# Select a local scheduler for the actor.
local_schedulers = state.get_local_schedulers()
local_scheduler_id = random.choice(local_schedulers)

worker.redis_client.publish("actor_notifications", actor_id.id() + local_scheduler_id)

# The export counter is computed differently depending on whether we are
# currently in a driver or a worker.
if worker.mode in [ray.SCRIPT_MODE, ray.SILENT_MODE]:
export_counter = worker.driver_export_counter
elif worker.mode == ray.WORKER_MODE:
# We don't actually need export counters for actors.
export_counter = 0
d = {"driver_id": worker.task_driver_id.id(),
d = {"driver_id": driver_id,
"actor_id": actor_id.id(),
"name": Class.__name__,
"module": Class.__module__,
"class": pickled_class,
"class_export_counter": export_counter}
"class": pickled_class}
worker.redis_client.hmset(key, d)
worker.redis_client.rpush("Exports", key)
worker.driver_export_counter += 1

def actor(Class):
# The function actor_method_call gets called if somebody tries to call a
Expand All @@ -105,7 +104,6 @@ def actor_method_call(actor_id, attr, *args, **kwargs):
num_cpus = 0
num_gpus = 0
object_ids = ray.worker.global_worker.submit_task(function_id, "", args,
num_cpus, num_gpus,
actor_id=actor_id)
if len(object_ids) == 1:
return object_ids[0]
Expand All @@ -116,7 +114,7 @@ class NewClass(object):
def __init__(self, *args, **kwargs):
self._ray_actor_id = random_actor_id()
self._ray_actor_methods = {k: v for (k, v) in inspect.getmembers(Class, predicate=(lambda x: inspect.isfunction(x) or inspect.ismethod(x)))}
export_actor(self._ray_actor_id, Class, ray.worker.global_worker)
export_actor(self._ray_actor_id, Class, self._ray_actor_methods, ray.worker.global_worker)
# Call __init__ as a remote function.
if "__init__" in self._ray_actor_methods.keys():
actor_method_call(self._ray_actor_id, "__init__", *args, **kwargs)
Expand Down
Loading

0 comments on commit 88a5b4e

Please sign in to comment.