def _load_actor_class_from_gcs(self, job_id, actor_creation_function_descriptor): """Load actor class from GCS.""" key = (b"ActorClass:" + job_id.binary() + b":" + actor_creation_function_descriptor.function_id.binary()) # Wait for the actor class key to have been imported by the # import thread. TODO(rkn): It shouldn't be possible to end # up in an infinite loop here, but we should push an error to # the driver if too much time is spent here. while key not in self.imported_actor_classes: time.sleep(0.001) # Fetch raw data from GCS. (job_id_str, class_name, module, pickled_class, actor_method_names) = self._worker.redis_client.hmget( key, ["job_id", "class_name", "module", "class", "actor_method_names"]) class_name = ensure_str(class_name) module_name = ensure_str(module) job_id = ray.JobID(job_id_str) actor_method_names = json.loads(ensure_str(actor_method_names)) actor_class = None try: with self.lock: actor_class = pickle.loads(pickled_class) except Exception: logger.exception("Failed to load actor class %s.", class_name) # The actor class failed to be unpickled, create a fake actor # class instead (just to produce error messages and to prevent # the driver from hanging). actor_class = self._create_fake_actor_class( class_name, actor_method_names) # If an exception was thrown when the actor was imported, we record # the traceback and notify the scheduler of the failure. traceback_str = ray._private.utils.format_error_message( traceback.format_exc()) # Log the error message. push_error_to_driver( self._worker, ray_constants.REGISTER_ACTOR_PUSH_ERROR, f"Failed to unpickle actor class '{class_name}' " f"for actor ID {self._worker.actor_id.hex()}. " f"Traceback:\n{traceback_str}", job_id=job_id) # TODO(rkn): In the future, it might make sense to have the worker # exit here. However, currently that would lead to hanging if # someone calls ray.get on a method invoked on the actor. # The below line is necessary. Because in the driver process, # if the function is defined in the file where the python script # was started from, its module is `__main__`. # However in the worker process, the `__main__` module is a # different module, which is `default_worker.py` actor_class.__module__ = module_name return actor_class
def fetch_and_register_remote_function(self, key): """Import a remote function.""" (job_id_str, function_id_str, function_name, serialized_function, module, max_calls) = self._worker.redis_client.hmget( key, [ "job_id", "function_id", "function_name", "function", "module", "max_calls" ]) function_id = ray.FunctionID(function_id_str) job_id = ray.JobID(job_id_str) function_name = decode(function_name) max_calls = int(max_calls) module = decode(module) # This function is called by ImportThread. This operation needs to be # atomic. Otherwise, there is race condition. Another thread may use # the temporary function above before the real function is ready. with self.lock: self._num_task_executions[job_id][function_id] = 0 try: function = pickle.loads(serialized_function) except Exception: def f(*args, **kwargs): raise RuntimeError( "This function was not imported properly.") # Use a placeholder method when function pickled failed self._function_execution_info[job_id][function_id] = ( FunctionExecutionInfo( function=f, function_name=function_name, max_calls=max_calls)) # If an exception was thrown when the remote function was # imported, we record the traceback and notify the scheduler # of the failure. traceback_str = format_error_message(traceback.format_exc()) # Log the error message. push_error_to_driver( self._worker, ray_constants.REGISTER_REMOTE_FUNCTION_PUSH_ERROR, "Failed to unpickle the remote function " f"'{function_name}' with " f"function ID {function_id.hex()}. " f"Traceback:\n{traceback_str}", job_id=job_id) else: # The below line is necessary. Because in the driver process, # if the function is defined in the file where the python # script was started from, its module is `__main__`. # However in the worker process, the `__main__` module is a # different module, which is `default_worker.py` function.__module__ = module self._function_execution_info[job_id][function_id] = ( FunctionExecutionInfo( function=function, function_name=function_name, max_calls=max_calls)) # Add the function to the function table. self._worker.redis_client.rpush( b"FunctionTable:" + function_id.binary(), self._worker.worker_id)