def get_proto_job_config(self): """Return the protobuf structure of JobConfig.""" # TODO(edoakes): this is really unfortunate, but JobConfig is imported # all over the place so this causes circular imports. We should remove # this dependency and pass in a validated runtime_env instead. from ray.utils import get_runtime_env_info if self._cached_pb is None: pb = gcs_utils.JobConfig() if self.ray_namespace is None: pb.ray_namespace = str(uuid.uuid4()) else: pb.ray_namespace = self.ray_namespace pb.num_java_workers_per_process = self.num_java_workers_per_process pb.jvm_options.extend(self.jvm_options) pb.code_search_path.extend(self.code_search_path) for k, v in self.metadata.items(): pb.metadata[k] = v parsed_env = self._validate_runtime_env() pb.runtime_env_info.CopyFrom( get_runtime_env_info( parsed_env, is_job_runtime_env=True, serialize=False, )) if self._default_actor_lifetime is not None: pb.default_actor_lifetime = self._default_actor_lifetime self._cached_pb = pb return self._cached_pb
def _remote(self, args=None, kwargs=None, **task_options): """Submit the remote function for execution.""" # We pop the "max_calls" coming from "@ray.remote" here. We no longer need # it in "_remote()". task_options.pop("max_calls", None) if client_mode_should_convert(auto_init=True): return client_mode_convert_function(self, args, kwargs, **task_options) worker = ray.worker.global_worker worker.check_connected() # If this function was not exported in this session and job, we need to # export this function again, because the current GCS doesn't have it. if (not self._is_cross_language and self._last_export_session_and_job != worker.current_session_and_job): self._function_descriptor = PythonFunctionDescriptor.from_function( self._function, self._uuid) # There is an interesting question here. If the remote function is # used by a subsequent driver (in the same script), should the # second driver pickle the function again? If yes, then the remote # function definition can differ in the second driver (e.g., if # variables in its closure have changed). We probably want the # behavior of the remote function in the second driver to be # independent of whether or not the function was invoked by the # first driver. This is an argument for repickling the function, # which we do here. try: self._pickled_function = pickle.dumps(self._function) except TypeError as e: msg = ( "Could not serialize the function " f"{self._function_descriptor.repr}. Check " "https://docs.ray.io/en/master/ray-core/objects/serialization.html#troubleshooting " # noqa "for more information.") raise TypeError(msg) from e self._last_export_session_and_job = worker.current_session_and_job worker.function_actor_manager.export(self) kwargs = {} if kwargs is None else kwargs args = [] if args is None else args # fill task required options for k, v in ray_option_utils.task_options.items(): task_options[k] = task_options.get(k, v.default_value) # "max_calls" already takes effects and should not apply again. # Remove the default value here. task_options.pop("max_calls", None) # TODO(suquark): cleanup these fields name = task_options["name"] runtime_env = parse_runtime_env(task_options["runtime_env"]) placement_group = task_options["placement_group"] placement_group_bundle_index = task_options[ "placement_group_bundle_index"] placement_group_capture_child_tasks = task_options[ "placement_group_capture_child_tasks"] scheduling_strategy = task_options["scheduling_strategy"] num_returns = task_options["num_returns"] max_retries = task_options["max_retries"] retry_exceptions = task_options["retry_exceptions"] resources = ray._private.utils.resources_from_ray_options(task_options) if scheduling_strategy is None or isinstance( scheduling_strategy, PlacementGroupSchedulingStrategy): if isinstance(scheduling_strategy, PlacementGroupSchedulingStrategy): placement_group = scheduling_strategy.placement_group placement_group_bundle_index = ( scheduling_strategy.placement_group_bundle_index) placement_group_capture_child_tasks = ( scheduling_strategy.placement_group_capture_child_tasks) if placement_group_capture_child_tasks is None: placement_group_capture_child_tasks = ( worker.should_capture_child_tasks_in_placement_group) placement_group = configure_placement_group_based_on_context( placement_group_capture_child_tasks, placement_group_bundle_index, resources, {}, # no placement_resources for tasks self._function_descriptor.function_name, placement_group=placement_group, ) if not placement_group.is_empty: scheduling_strategy = PlacementGroupSchedulingStrategy( placement_group, placement_group_bundle_index, placement_group_capture_child_tasks, ) else: scheduling_strategy = "DEFAULT" serialized_runtime_env_info = None if runtime_env is not None: serialized_runtime_env_info = get_runtime_env_info( runtime_env, is_job_runtime_env=False, serialize=True, ) def invocation(args, kwargs): if self._is_cross_language: list_args = cross_language.format_args(worker, args, kwargs) elif not args and not kwargs and not self._function_signature: list_args = [] else: list_args = ray._private.signature.flatten_args( self._function_signature, args, kwargs) if worker.mode == ray.worker.LOCAL_MODE: assert ( not self._is_cross_language ), "Cross language remote function cannot be executed locally." object_refs = worker.core_worker.submit_task( self._language, self._function_descriptor, list_args, name if name is not None else "", num_returns, resources, max_retries, retry_exceptions, scheduling_strategy, worker.debugger_breakpoint, serialized_runtime_env_info or "{}", ) # Reset worker's debug context from the last "remote" command # (which applies only to this .remote call). worker.debugger_breakpoint = b"" if len(object_refs) == 1: return object_refs[0] elif len(object_refs) > 1: return object_refs if self._decorator is not None: invocation = self._decorator(invocation) return invocation(args, kwargs)
def _remote( self, args=None, kwargs=None, num_returns=None, num_cpus=None, num_gpus=None, memory=None, object_store_memory=None, accelerator_type=None, resources=None, max_retries=None, retry_exceptions=None, placement_group="default", placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, runtime_env=None, name="", scheduling_strategy: SchedulingStrategyT = None, ): """Submit the remote function for execution.""" if client_mode_should_convert(auto_init=True): return client_mode_convert_function( self, args, kwargs, num_returns=num_returns, num_cpus=num_cpus, num_gpus=num_gpus, memory=memory, object_store_memory=object_store_memory, accelerator_type=accelerator_type, resources=resources, max_retries=max_retries, retry_exceptions=retry_exceptions, placement_group=placement_group, placement_group_bundle_index=placement_group_bundle_index, placement_group_capture_child_tasks=( placement_group_capture_child_tasks), runtime_env=runtime_env, name=name, scheduling_strategy=scheduling_strategy, ) worker = ray.worker.global_worker worker.check_connected() # If this function was not exported in this session and job, we need to # export this function again, because the current GCS doesn't have it. if (not self._is_cross_language and self._last_export_session_and_job != worker.current_session_and_job): self._function_descriptor = PythonFunctionDescriptor.from_function( self._function, self._uuid) # There is an interesting question here. If the remote function is # used by a subsequent driver (in the same script), should the # second driver pickle the function again? If yes, then the remote # function definition can differ in the second driver (e.g., if # variables in its closure have changed). We probably want the # behavior of the remote function in the second driver to be # independent of whether or not the function was invoked by the # first driver. This is an argument for repickling the function, # which we do here. try: self._pickled_function = pickle.dumps(self._function) except TypeError as e: msg = ( "Could not serialize the function " f"{self._function_descriptor.repr}. Check " "https://docs.ray.io/en/master/serialization.html#troubleshooting " # noqa "for more information.") raise TypeError(msg) from e self._last_export_session_and_job = worker.current_session_and_job worker.function_actor_manager.export(self) kwargs = {} if kwargs is None else kwargs args = [] if args is None else args if num_returns is None: num_returns = self._num_returns if max_retries is None: max_retries = self._max_retries if retry_exceptions is None: retry_exceptions = self._retry_exceptions if scheduling_strategy is None: scheduling_strategy = self._scheduling_strategy resources = ray._private.utils.resources_from_resource_arguments( self._num_cpus, self._num_gpus, self._memory, self._object_store_memory, self._resources, self._accelerator_type, num_cpus, num_gpus, memory, object_store_memory, resources, accelerator_type, ) if (placement_group != "default") and (scheduling_strategy is not None): raise ValueError("Placement groups should be specified via the " "scheduling_strategy option. " "The placement_group option is deprecated.") if scheduling_strategy is None or isinstance( scheduling_strategy, PlacementGroupSchedulingStrategy): if isinstance(scheduling_strategy, PlacementGroupSchedulingStrategy): placement_group = scheduling_strategy.placement_group placement_group_bundle_index = ( scheduling_strategy.placement_group_bundle_index) placement_group_capture_child_tasks = ( scheduling_strategy.placement_group_capture_child_tasks) if placement_group_capture_child_tasks is None: placement_group_capture_child_tasks = ( worker.should_capture_child_tasks_in_placement_group) if placement_group == "default": placement_group = self._placement_group placement_group = configure_placement_group_based_on_context( placement_group_capture_child_tasks, placement_group_bundle_index, resources, {}, # no placement_resources for tasks self._function_descriptor.function_name, placement_group=placement_group, ) if not placement_group.is_empty: scheduling_strategy = PlacementGroupSchedulingStrategy( placement_group, placement_group_bundle_index, placement_group_capture_child_tasks, ) else: scheduling_strategy = "DEFAULT" if not runtime_env or runtime_env == "{}": runtime_env = self._runtime_env serialized_runtime_env_info = None if runtime_env is not None: serialized_runtime_env_info = get_runtime_env_info( runtime_env, is_job_runtime_env=False, serialize=True, ) def invocation(args, kwargs): if self._is_cross_language: list_args = cross_language.format_args(worker, args, kwargs) elif not args and not kwargs and not self._function_signature: list_args = [] else: list_args = ray._private.signature.flatten_args( self._function_signature, args, kwargs) if worker.mode == ray.worker.LOCAL_MODE: assert not self._is_cross_language, ( "Cross language remote function " "cannot be executed locally.") object_refs = worker.core_worker.submit_task( self._language, self._function_descriptor, list_args, name, num_returns, resources, max_retries, retry_exceptions, scheduling_strategy, worker.debugger_breakpoint, serialized_runtime_env_info or "{}", ) # Reset worker's debug context from the last "remote" command # (which applies only to this .remote call). worker.debugger_breakpoint = b"" if len(object_refs) == 1: return object_refs[0] elif len(object_refs) > 1: return object_refs if self._decorator is not None: invocation = self._decorator(invocation) return invocation(args, kwargs)