@@ -267,7 +267,7 @@ def set_mode(self, mode):
267
267
print any information about errors because some of the tests
268
268
intentionally fail.
269
269
270
- args :
270
+ Args :
271
271
mode: One of SCRIPT_MODE, WORKER_MODE, PYTHON_MODE, and
272
272
SILENT_MODE.
273
273
"""
@@ -363,11 +363,6 @@ def put_object(self, object_id, value):
363
363
"do this, you can wrap the ObjectID in a list and "
364
364
"call 'put' on it (or return it)." )
365
365
366
- if isinstance (value , ray .actor .ActorHandleParent ):
367
- raise Exception ("Calling 'put' on an actor handle is currently "
368
- "not allowed (similarly, returning an actor "
369
- "handle from a remote function is not allowed)." )
370
-
371
366
# Serialize and put the object in the object store.
372
367
try :
373
368
self .store_and_register (object_id , value )
@@ -525,7 +520,8 @@ def submit_task(self,
525
520
num_return_vals = None ,
526
521
num_cpus = None ,
527
522
num_gpus = None ,
528
- resources = None ):
523
+ resources = None ,
524
+ driver_id = None ):
529
525
"""Submit a remote task to the scheduler.
530
526
531
527
Tell the scheduler to schedule the execution of the function with ID
@@ -552,6 +548,11 @@ def submit_task(self,
552
548
num_cpus: The number of CPUs required by this task.
553
549
num_gpus: The number of GPUs required by this task.
554
550
resources: The resource requirements for this task.
551
+ driver_id: The ID of the relevant driver. This is almost always the
552
+ driver ID of the driver that is currently running. However, in
553
+ the exceptional case that an actor task is being dispatched to
554
+ an actor created by a different driver, this should be the
555
+ driver ID of the driver that created the actor.
555
556
556
557
Returns:
557
558
The return object IDs for this task.
@@ -579,9 +580,6 @@ def submit_task(self,
579
580
for arg in args :
580
581
if isinstance (arg , ray .local_scheduler .ObjectID ):
581
582
args_for_local_scheduler .append (arg )
582
- elif isinstance (arg , ray .actor .ActorHandleParent ):
583
- args_for_local_scheduler .append (
584
- put (ray .actor .wrap_actor_handle (arg )))
585
583
elif ray .local_scheduler .check_simple_value (arg ):
586
584
args_for_local_scheduler .append (arg )
587
585
else :
@@ -591,9 +589,12 @@ def submit_task(self,
591
589
if execution_dependencies is None :
592
590
execution_dependencies = []
593
591
592
+ if driver_id is None :
593
+ driver_id = self .task_driver_id
594
+
594
595
# Look up the various function properties.
595
- function_properties = self .function_properties [
596
- self . task_driver_id . id ()][ function_id .id ()]
596
+ function_properties = self .function_properties [driver_id . id ()][
597
+ function_id .id ()]
597
598
598
599
if num_return_vals is None :
599
600
num_return_vals = function_properties .num_return_vals
@@ -610,8 +611,7 @@ def submit_task(self,
610
611
611
612
# Submit the task to local scheduler.
612
613
task = ray .local_scheduler .Task (
613
- self .task_driver_id ,
614
- ray .local_scheduler .ObjectID (
614
+ driver_id , ray .local_scheduler .ObjectID (
615
615
function_id .id ()), args_for_local_scheduler ,
616
616
num_return_vals , self .current_task_id , self .task_index ,
617
617
actor_creation_id , actor_creation_dummy_object_id , actor_id ,
@@ -749,8 +749,6 @@ def _get_arguments_for_execution(self, function_name, serialized_args):
749
749
# created this object failed, and we should propagate the
750
750
# error message here.
751
751
raise RayGetArgumentError (function_name , i , arg , argument )
752
- elif isinstance (argument , ray .actor .ActorHandleWrapper ):
753
- argument = ray .actor .unwrap_actor_handle (self , argument )
754
752
else :
755
753
# pass the argument by value
756
754
argument = arg
@@ -779,6 +777,10 @@ def _store_outputs_in_objstore(self, object_ids, outputs):
779
777
passed into this function.
780
778
"""
781
779
for i in range (len (object_ids )):
780
+ if isinstance (outputs [i ], ray .actor .ActorHandle ):
781
+ raise Exception ("Returning an actor handle from a remote "
782
+ "function is not allowed)." )
783
+
782
784
self .put_object (object_ids [i ], outputs [i ])
783
785
784
786
def _process_task (self , task ):
@@ -1137,18 +1139,39 @@ def _initialize_serialization(worker=global_worker):
1137
1139
pyarrow .register_torch_serialization_handlers (worker .serialization_context )
1138
1140
1139
1141
# Define a custom serializer and deserializer for handling Object IDs.
1140
- def objectid_custom_serializer (obj ):
1142
+ def object_id_custom_serializer (obj ):
1141
1143
return obj .id ()
1142
1144
1143
- def objectid_custom_deserializer (serialized_obj ):
1145
+ def object_id_custom_deserializer (serialized_obj ):
1144
1146
return ray .local_scheduler .ObjectID (serialized_obj )
1145
1147
1148
+ # We register this serializer on each worker instead of calling
1149
+ # register_custom_serializer from the driver so that isinstance still
1150
+ # works.
1146
1151
worker .serialization_context .register_type (
1147
1152
ray .local_scheduler .ObjectID ,
1148
1153
"ray.ObjectID" ,
1149
1154
pickle = False ,
1150
- custom_serializer = objectid_custom_serializer ,
1151
- custom_deserializer = objectid_custom_deserializer )
1155
+ custom_serializer = object_id_custom_serializer ,
1156
+ custom_deserializer = object_id_custom_deserializer )
1157
+
1158
+ def actor_handle_serializer (obj ):
1159
+ return obj ._serialization_helper (True )
1160
+
1161
+ def actor_handle_deserializer (serialized_obj ):
1162
+ new_handle = ray .actor .ActorHandle .__new__ (ray .actor .ActorHandle )
1163
+ new_handle ._deserialization_helper (serialized_obj , True )
1164
+ return new_handle
1165
+
1166
+ # We register this serializer on each worker instead of calling
1167
+ # register_custom_serializer from the driver so that isinstance still
1168
+ # works.
1169
+ worker .serialization_context .register_type (
1170
+ ray .actor .ActorHandle ,
1171
+ "ray.ActorHandle" ,
1172
+ pickle = False ,
1173
+ custom_serializer = actor_handle_serializer ,
1174
+ custom_deserializer = actor_handle_deserializer )
1152
1175
1153
1176
if worker .mode in [SCRIPT_MODE , SILENT_MODE ]:
1154
1177
# These should only be called on the driver because
@@ -1161,8 +1184,6 @@ def objectid_custom_deserializer(serialized_obj):
1161
1184
register_custom_serializer (type (lambda : 0 ), use_pickle = True )
1162
1185
# Tell Ray to serialize types with pickle.
1163
1186
register_custom_serializer (type (int ), use_pickle = True )
1164
- # Ray can serialize actor handles that have been wrapped.
1165
- register_custom_serializer (ray .actor .ActorHandleWrapper , use_dict = True )
1166
1187
# Tell Ray to serialize FunctionSignatures as dictionaries. This is
1167
1188
# used when passing around actor handles.
1168
1189
register_custom_serializer (
0 commit comments