Skip to content

Commit 9948e8c

Browse files
guoyuhongrobertnishihara
authored andcommitted
Move function/actor exporting & loading code to function_manager.py (ray-project#3003)
Move function/actor exporting & loading code to function_manager.py to prepare the code change for function descriptor for python.
1 parent d73ee36 commit 9948e8c

File tree

6 files changed

+559
-502
lines changed

6 files changed

+559
-502
lines changed

python/ray/actor.py

Lines changed: 24 additions & 275 deletions
Original file line numberDiff line numberDiff line change
@@ -5,31 +5,19 @@
55
import copy
66
import hashlib
77
import inspect
8-
import json
98
import traceback
109

1110
import ray.cloudpickle as pickle
11+
from ray.function_manager import FunctionActorManager
1212
import ray.local_scheduler
1313
import ray.ray_constants as ray_constants
1414
import ray.signature as signature
1515
import ray.worker
16-
from ray.utils import (
17-
decode,
18-
_random_string,
19-
check_oversized_pickle,
20-
is_cython,
21-
push_error_to_driver,
22-
)
16+
from ray.utils import _random_string
2317

2418
DEFAULT_ACTOR_METHOD_NUM_RETURN_VALS = 1
2519

2620

27-
def is_classmethod(f):
28-
"""Returns whether the given method is a classmethod."""
29-
30-
return hasattr(f, "__self__") and f.__self__ is not None
31-
32-
3321
def compute_actor_handle_id(actor_handle_id, num_forks):
3422
"""Deterministically compute an actor handle ID.
3523
@@ -96,24 +84,6 @@ def compute_actor_creation_function_id(class_id):
9684
return ray.ObjectID(class_id)
9785

9886

99-
def compute_actor_method_function_id(class_name, attr):
100-
"""Get the function ID corresponding to an actor method.
101-
102-
Args:
103-
class_name (str): The class name of the actor.
104-
attr (str): The attribute name of the method.
105-
106-
Returns:
107-
Function ID corresponding to the method.
108-
"""
109-
function_id_hash = hashlib.sha1()
110-
function_id_hash.update(class_name.encode("ascii"))
111-
function_id_hash.update(attr.encode("ascii"))
112-
function_id = function_id_hash.digest()
113-
assert len(function_id) == ray_constants.ID_SIZE
114-
return ray.ObjectID(function_id)
115-
116-
11787
def set_actor_checkpoint(worker, actor_id, checkpoint_index, checkpoint,
11888
frontier):
11989
"""Set the most recent checkpoint associated with a given actor ID.
@@ -134,28 +104,6 @@ def set_actor_checkpoint(worker, actor_id, checkpoint_index, checkpoint,
134104
})
135105

136106

137-
def get_actor_checkpoint(worker, actor_id):
138-
"""Get the most recent checkpoint associated with a given actor ID.
139-
140-
Args:
141-
worker: The worker to use to get the checkpoint.
142-
actor_id: The actor ID of the actor to get the checkpoint for.
143-
144-
Returns:
145-
If a checkpoint exists, this returns a tuple of the number of tasks
146-
included in the checkpoint, the saved checkpoint state, and the
147-
task frontier at the time of the checkpoint. If no checkpoint
148-
exists, all objects are set to None. The checkpoint index is the .
149-
executed on the actor before the checkpoint was made.
150-
"""
151-
actor_key = b"Actor:" + actor_id
152-
checkpoint_index, checkpoint, frontier = worker.redis_client.hmget(
153-
actor_key, ["checkpoint_index", "checkpoint", "frontier"])
154-
if checkpoint_index is not None:
155-
checkpoint_index = int(checkpoint_index)
156-
return checkpoint_index, checkpoint, frontier
157-
158-
159107
def save_and_log_checkpoint(worker, actor):
160108
"""Save a checkpoint on the actor and log any errors.
161109
@@ -205,219 +153,26 @@ def restore_and_log_checkpoint(worker, actor):
205153
return checkpoint_resumed
206154

207155

208-
def make_actor_method_executor(worker, method_name, method, actor_imported):
209-
"""Make an executor that wraps a user-defined actor method.
210-
211-
The wrapped method updates the worker's internal state and performs any
212-
necessary checkpointing operations.
156+
def get_actor_checkpoint(worker, actor_id):
157+
"""Get the most recent checkpoint associated with a given actor ID.
213158
214159
Args:
215-
worker (Worker): The worker that is executing the actor.
216-
method_name (str): The name of the actor method.
217-
method (instancemethod): The actor method to wrap. This should be a
218-
method defined on the actor class and should therefore take an
219-
instance of the actor as the first argument.
220-
actor_imported (bool): Whether the actor has been imported.
221-
Checkpointing operations will not be run if this is set to False.
160+
worker: The worker to use to get the checkpoint.
161+
actor_id: The actor ID of the actor to get the checkpoint for.
222162
223163
Returns:
224-
A function that executes the given actor method on the worker's stored
225-
instance of the actor. The function also updates the worker's
226-
internal state to record the executed method.
227-
"""
228-
229-
def actor_method_executor(dummy_return_id, actor, *args):
230-
# Update the actor's task counter to reflect the task we're about to
231-
# execute.
232-
worker.actor_task_counter += 1
233-
234-
# If this is the first task to execute on the actor, try to resume from
235-
# a checkpoint.
236-
if actor_imported and worker.actor_task_counter == 1:
237-
checkpoint_resumed = restore_and_log_checkpoint(worker, actor)
238-
if checkpoint_resumed:
239-
# NOTE(swang): Since we did not actually execute the __init__
240-
# method, this will put None as the return value. If the
241-
# __init__ method is supposed to return multiple values, an
242-
# exception will be logged.
243-
return
244-
245-
# Determine whether we should checkpoint the actor.
246-
checkpointing_on = (actor_imported
247-
and worker.actor_checkpoint_interval > 0)
248-
# We should checkpoint the actor if user checkpointing is on, we've
249-
# executed checkpoint_interval tasks since the last checkpoint, and the
250-
# method we're about to execute is not a checkpoint.
251-
save_checkpoint = (
252-
checkpointing_on and
253-
(worker.actor_task_counter % worker.actor_checkpoint_interval == 0
254-
and method_name != "__ray_checkpoint__"))
255-
256-
# Execute the assigned method and save a checkpoint if necessary.
257-
try:
258-
if is_classmethod(method):
259-
method_returns = method(*args)
260-
else:
261-
method_returns = method(actor, *args)
262-
except Exception:
263-
# Save the checkpoint before allowing the method exception to be
264-
# thrown.
265-
if save_checkpoint:
266-
save_and_log_checkpoint(worker, actor)
267-
raise
268-
else:
269-
# Save the checkpoint before returning the method's return values.
270-
if save_checkpoint:
271-
save_and_log_checkpoint(worker, actor)
272-
return method_returns
273-
274-
return actor_method_executor
275-
276-
277-
def fetch_and_register_actor(actor_class_key, worker):
278-
"""Import an actor.
279-
280-
This will be called by the worker's import thread when the worker receives
281-
the actor_class export, assuming that the worker is an actor for that
282-
class.
283-
284-
Args:
285-
actor_class_key: The key in Redis to use to fetch the actor.
286-
worker: The worker to use.
287-
"""
288-
actor_id_str = worker.actor_id
289-
(driver_id, class_id, class_name, module, pickled_class,
290-
checkpoint_interval, actor_method_names) = worker.redis_client.hmget(
291-
actor_class_key, [
292-
"driver_id", "class_id", "class_name", "module", "class",
293-
"checkpoint_interval", "actor_method_names"
294-
])
295-
296-
class_name = decode(class_name)
297-
module = decode(module)
298-
checkpoint_interval = int(checkpoint_interval)
299-
actor_method_names = json.loads(decode(actor_method_names))
300-
301-
# Create a temporary actor with some temporary methods so that if the actor
302-
# fails to be unpickled, the temporary actor can be used (just to produce
303-
# error messages and to prevent the driver from hanging).
304-
class TemporaryActor(object):
305-
pass
306-
307-
worker.actors[actor_id_str] = TemporaryActor()
308-
worker.actor_checkpoint_interval = checkpoint_interval
309-
310-
def temporary_actor_method(*xs):
311-
raise Exception("The actor with name {} failed to be imported, and so "
312-
"cannot execute this method".format(class_name))
313-
314-
# Register the actor method executors.
315-
for actor_method_name in actor_method_names:
316-
function_id = compute_actor_method_function_id(class_name,
317-
actor_method_name).id()
318-
temporary_executor = make_actor_method_executor(
319-
worker,
320-
actor_method_name,
321-
temporary_actor_method,
322-
actor_imported=False)
323-
worker.function_execution_info[driver_id][function_id] = (
324-
ray.worker.FunctionExecutionInfo(
325-
function=temporary_executor,
326-
function_name=actor_method_name,
327-
max_calls=0))
328-
worker.num_task_executions[driver_id][function_id] = 0
329-
330-
try:
331-
unpickled_class = pickle.loads(pickled_class)
332-
worker.actor_class = unpickled_class
333-
except Exception:
334-
# If an exception was thrown when the actor was imported, we record the
335-
# traceback and notify the scheduler of the failure.
336-
traceback_str = ray.utils.format_error_message(traceback.format_exc())
337-
# Log the error message.
338-
push_error_to_driver(
339-
worker,
340-
ray_constants.REGISTER_ACTOR_PUSH_ERROR,
341-
traceback_str,
342-
driver_id,
343-
data={"actor_id": actor_id_str})
344-
# TODO(rkn): In the future, it might make sense to have the worker exit
345-
# here. However, currently that would lead to hanging if someone calls
346-
# ray.get on a method invoked on the actor.
347-
else:
348-
# TODO(pcm): Why is the below line necessary?
349-
unpickled_class.__module__ = module
350-
worker.actors[actor_id_str] = unpickled_class.__new__(unpickled_class)
351-
352-
def pred(x):
353-
return (inspect.isfunction(x) or inspect.ismethod(x)
354-
or is_cython(x))
355-
356-
actor_methods = inspect.getmembers(unpickled_class, predicate=pred)
357-
for actor_method_name, actor_method in actor_methods:
358-
function_id = compute_actor_method_function_id(
359-
class_name, actor_method_name).id()
360-
executor = make_actor_method_executor(
361-
worker, actor_method_name, actor_method, actor_imported=True)
362-
worker.function_execution_info[driver_id][function_id] = (
363-
ray.worker.FunctionExecutionInfo(
364-
function=executor,
365-
function_name=actor_method_name,
366-
max_calls=0))
367-
# We do not set worker.function_properties[driver_id][function_id]
368-
# because we currently do need the actor worker to submit new tasks
369-
# for the actor.
370-
371-
372-
def publish_actor_class_to_key(key, actor_class_info, worker):
373-
"""Push an actor class definition to Redis.
374-
375-
The is factored out as a separate function because it is also called
376-
on cached actor class definitions when a worker connects for the first
377-
time.
378-
379-
Args:
380-
key: The key to store the actor class info at.
381-
actor_class_info: Information about the actor class.
382-
worker: The worker to use to connect to Redis.
164+
If a checkpoint exists, this returns a tuple of the number of tasks
165+
included in the checkpoint, the saved checkpoint state, and the
166+
task frontier at the time of the checkpoint. If no checkpoint
167+
exists, all objects are set to None. The checkpoint index is the .
168+
executed on the actor before the checkpoint was made.
383169
"""
384-
# We set the driver ID here because it may not have been available when the
385-
# actor class was defined.
386-
actor_class_info["driver_id"] = worker.task_driver_id.id()
387-
worker.redis_client.hmset(key, actor_class_info)
388-
worker.redis_client.rpush("Exports", key)
389-
390-
391-
def export_actor_class(class_id, Class, actor_method_names,
392-
checkpoint_interval, worker):
393-
key = b"ActorClass:" + class_id
394-
actor_class_info = {
395-
"class_name": Class.__name__,
396-
"module": Class.__module__,
397-
"class": pickle.dumps(Class),
398-
"checkpoint_interval": checkpoint_interval,
399-
"actor_method_names": json.dumps(list(actor_method_names))
400-
}
401-
402-
check_oversized_pickle(actor_class_info["class"],
403-
actor_class_info["class_name"], "actor", worker)
404-
405-
if worker.mode is None:
406-
# This means that 'ray.init()' has not been called yet and so we must
407-
# cache the actor class definition and export it when 'ray.init()' is
408-
# called.
409-
assert worker.cached_remote_functions_and_actors is not None
410-
worker.cached_remote_functions_and_actors.append(
411-
("actor", (key, actor_class_info)))
412-
# This caching code path is currently not used because we only export
413-
# actor class definitions lazily when we instantiate the actor for the
414-
# first time.
415-
assert False, "This should be unreachable."
416-
else:
417-
publish_actor_class_to_key(key, actor_class_info, worker)
418-
# TODO(rkn): Currently we allow actor classes to be defined within tasks.
419-
# I tried to disable this, but it may be necessary because of
420-
# https://github.com/ray-project/ray/issues/1146.
170+
actor_key = b"Actor:" + actor_id
171+
checkpoint_index, checkpoint, frontier = worker.redis_client.hmget(
172+
actor_key, ["checkpoint_index", "checkpoint", "frontier"])
173+
if checkpoint_index is not None:
174+
checkpoint_index = int(checkpoint_index)
175+
return checkpoint_index, checkpoint, frontier
421176

422177

423178
def method(*args, **kwargs):
@@ -518,13 +273,8 @@ def __init__(self, modified_class, class_id, checkpoint_interval, num_cpus,
518273
self._actor_method_cpus = actor_method_cpus
519274
self._exported = False
520275

521-
# Get the actor methods of the given class.
522-
def pred(x):
523-
return (inspect.isfunction(x) or inspect.ismethod(x)
524-
or is_cython(x))
525-
526276
self._actor_methods = inspect.getmembers(
527-
self._modified_class, predicate=pred)
277+
self._modified_class, ray.utils.is_function_or_method)
528278
# Extract the signatures of each of the methods. This will be used
529279
# to catch some errors if the methods are called with inappropriate
530280
# arguments.
@@ -537,7 +287,7 @@ def pred(x):
537287
# don't support, there may not be much the user can do about it.
538288
signature.check_signature_supported(method, warn=True)
539289
self._method_signatures[method_name] = signature.extract_signature(
540-
method, ignore_first=not is_classmethod(method))
290+
method, ignore_first=not ray.utils.is_class_method(method))
541291

542292
# Set the default number of return values for this method.
543293
if hasattr(method, "__ray_num_return_vals__"):
@@ -614,9 +364,9 @@ def _submit(self,
614364
else:
615365
# Export the actor.
616366
if not self._exported:
617-
export_actor_class(self._class_id, self._modified_class,
618-
self._actor_method_names,
619-
self._checkpoint_interval, worker)
367+
worker.function_actor_manager.export_actor_class(
368+
self._class_id, self._modified_class,
369+
self._actor_method_names, self._checkpoint_interval)
620370
self._exported = True
621371

622372
resources = ray.utils.resources_from_resource_arguments(
@@ -801,8 +551,8 @@ def _actor_method_call(self,
801551
else:
802552
actor_handle_id = self._ray_actor_handle_id
803553

804-
function_id = compute_actor_method_function_id(self._ray_class_name,
805-
method_name)
554+
function_id = FunctionActorManager.compute_actor_method_function_id(
555+
self._ray_class_name, method_name)
806556
object_ids = worker.submit_task(
807557
function_id,
808558
args,
@@ -1068,5 +818,4 @@ def __ray_checkpoint_restore__(self):
1068818
resources, actor_method_cpus)
1069819

1070820

1071-
ray.worker.global_worker.fetch_and_register_actor = fetch_and_register_actor
1072821
ray.worker.global_worker.make_actor = make_actor

0 commit comments

Comments
 (0)