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

Fix bug in which actor classes are not exported multiple times. #4838

Merged
merged 1 commit into from
May 23, 2019
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
15 changes: 10 additions & 5 deletions python/ray/actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -186,8 +186,9 @@ class ActorClass(object):
task.
_resources: The default resources required by the actor creation task.
_actor_method_cpus: The number of CPUs required by actor method tasks.
_exported: True if the actor class has been exported and false
otherwise.
_last_export_session: The index of the last session in which the remote
function was exported. This is used to determine if we need to
export the remote function again.
_actor_methods: The actor methods.
_method_decorators: Optional decorators that should be applied to the
method invocation function before invoking the actor methods. These
Expand All @@ -208,7 +209,7 @@ def __init__(self, modified_class, class_id, max_reconstructions, num_cpus,
self._num_cpus = num_cpus
self._num_gpus = num_gpus
self._resources = resources
self._exported = False
self._last_export_session = None
Copy link
Collaborator Author

Choose a reason for hiding this comment

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

We initialize this to None instead of to global_worker._session_index because we don't actually export the actor class here.


self._actor_methods = inspect.getmembers(
self._modified_class, ray.utils.is_function_or_method)
Expand Down Expand Up @@ -341,10 +342,14 @@ def _remote(self,
*copy.deepcopy(args), **copy.deepcopy(kwargs))
else:
# Export the actor.
if not self._exported:
if (self._last_export_session is None
or self._last_export_session < worker._session_index):
# If this actor class was exported in a previous session, we
# need to export this function again, because current GCS
# doesn't have it.
self._last_export_session = worker._session_index
Copy link
Collaborator Author

Choose a reason for hiding this comment

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

This more or less mimics

if self._last_export_session < worker._session_index:
# If this function was exported in a previous session, we need to
# export this function again, because current GCS doesn't have it.
self._last_export_session = worker._session_index
worker.function_actor_manager.export(self)

worker.function_actor_manager.export_actor_class(
self._modified_class, self._actor_method_names)
self._exported = True

resources = ray.utils.resources_from_resource_arguments(
cpus_to_use, self._num_gpus, self._resources, num_cpus,
Expand Down
4 changes: 3 additions & 1 deletion python/ray/remote_function.py
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,9 @@ class RemoteFunction(object):
return the resulting ObjectIDs. For an example, see
"test_decorated_function" in "python/ray/tests/test_basic.py".
_function_signature: The function signature.
_last_export_session: The index of the last session in which the remote
function was exported. This is used to determine if we need to
export the remote function again.
"""

def __init__(self, function, num_cpus, num_gpus, resources,
Expand All @@ -68,7 +71,6 @@ def __init__(self, function, num_cpus, num_gpus, resources,

# Export the function.
worker = ray.worker.get_global_worker()
# In which session this function was exported last time.
self._last_export_session = worker._session_index
worker.function_actor_manager.export(self)

Expand Down
26 changes: 26 additions & 0 deletions python/ray/tests/test_basic.py
Original file line number Diff line number Diff line change
Expand Up @@ -2942,3 +2942,29 @@ def get_postprocessor(object_ids, values):

assert ray.get(
[ray.put(i) for i in [0, 1, 3, 5, -1, -3, 4]]) == [1, 3, 5, 4]


def test_export_after_shutdown(ray_start_regular):
# This test checks that we can use actor and remote function definitions
# across multiple Ray sessions.

@ray.remote
def f():
pass

@ray.remote
class Actor(object):
def method(self):
pass

ray.get(f.remote())
a = Actor.remote()
ray.get(a.method.remote())

ray.shutdown()

# Start Ray and use the remote function and actor again.
ray.init(num_cpus=1)
ray.get(f.remote())
a = Actor.remote()
ray.get(a.method.remote())