def cancel(workflow_id: str) -> None: try: workflow_manager = get_management_actor() ray.get(workflow_manager.cancel_workflow.remote(workflow_id)) except ValueError: wf_store = workflow_storage.get_workflow_storage(workflow_id) wf_store.save_workflow_meta(WorkflowMetaData(WorkflowStatus.CANCELED))
def list_all( status_filter: Set[WorkflowStatus] ) -> List[Tuple[str, WorkflowStatus]]: try: workflow_manager = get_management_actor() except ValueError: workflow_manager = None if workflow_manager is None: runnings = [] else: runnings = ray.get(workflow_manager.list_running_workflow.remote()) if WorkflowStatus.RUNNING in status_filter and len(status_filter) == 1: return [(r, WorkflowStatus.RUNNING) for r in runnings] runnings = set(runnings) # Here we don't have workflow id, so use empty one instead store = workflow_storage.get_workflow_storage("") ret = [] for (k, s) in store.list_workflow(): if s == WorkflowStatus.RUNNING and k not in runnings: s = WorkflowStatus.RESUMABLE if s in status_filter: ret.append((k, s)) return ret
def run(entry_workflow: Workflow, workflow_id: Optional[str] = None, overwrite: bool = True) -> ray.ObjectRef: """Run a workflow asynchronously. # TODO(suquark): The current "run" always overwrite existing workflow. # We need to fix this later. """ store = get_global_storage() assert ray.is_initialized() if workflow_id is None: # Workflow ID format: {Entry workflow UUID}.{Unix time to nanoseconds} workflow_id = f"{entry_workflow.id}.{time.time():.9f}" logger.info(f"Workflow job created. [id=\"{workflow_id}\", storage_url=" f"\"{store.storage_url}\"].") # checkpoint the workflow ws = workflow_storage.get_workflow_storage(workflow_id) commit_step(ws, "", entry_workflow) workflow_manager = get_or_create_management_actor() ignore_existing = (entry_workflow.data.step_type != StepType.FUNCTION) # NOTE: It is important to 'ray.get' the returned output. This # ensures caller of 'run()' holds the reference to the workflow # result. Otherwise if the actor removes the reference of the # workflow output, the caller may fail to resolve the result. result: "WorkflowExecutionResult" = ray.get( workflow_manager.run_or_resume.remote(workflow_id, ignore_existing)) if entry_workflow.data.step_type == StepType.FUNCTION: return flatten_workflow_output(workflow_id, result.persisted_output) else: return flatten_workflow_output(workflow_id, result.volatile_output)
def _recover_workflow_step(input_object_refs: List[str], input_workflows: List[Any], instant_workflow_inputs: Dict[int, StepID]): """A workflow step that recovers the output of an unfinished step. Args: input_object_refs: The object refs in the argument of the (original) step. input_workflows: The workflows in the argument of the (original) step. They are resolved into physical objects (i.e. the output of the workflows) here. They come from other recover workflows we construct recursively. instant_workflow_inputs: Same as 'input_workflows', but they come point to workflow steps that have output checkpoints. They override corresponding workflows in 'input_workflows'. Returns: The output of the recovered step. """ reader = workflow_storage.get_workflow_storage() for index, _step_id in instant_workflow_inputs.items(): # override input workflows with instant workflows input_workflows[index] = reader.load_step_output(_step_id) input_object_refs = [reader.load_object_ref(r) for r in input_object_refs] step_id = workflow_context.get_current_step_id() func: Callable = reader.load_step_func_body(step_id) args, kwargs = reader.load_step_args(step_id, input_workflows, input_object_refs) return func(*args, **kwargs)
def test_shortcut(workflow_start_regular): assert recursive_chain.step(0).run(workflow_id="shortcut") == 100 # the shortcut points to the step with output checkpoint store = workflow_storage.get_workflow_storage("shortcut") step_id = store.get_entrypoint_step_id() output_step_id = store.inspect_step(step_id).output_step_id assert store.inspect_step(output_step_id).output_object_valid
def test_embedded_objectrefs(workflow_start_regular): workflow_id = test_workflow_storage.__name__ class ObjectRefsWrapper: def __init__(self, refs): self.refs = refs wf_storage = workflow_storage.WorkflowStorage(workflow_id, storage.get_global_storage()) url = storage.get_global_storage().storage_url wrapped = ObjectRefsWrapper([ray.put(1), ray.put(2)]) asyncio_run(wf_storage._put(["key"], wrapped)) # Be extremely explicit about shutting down. We want to make sure the # `_get` call deserializes the full object and puts it in the object store. # Shutting down the cluster should guarantee we don't accidently get the # old object and pass the test. ray.shutdown() subprocess.check_output("ray stop --force", shell=True) workflow.init(url) storage2 = get_workflow_storage(workflow_id) result = asyncio_run(storage2._get(["key"])) assert ray.get(result.refs) == [1, 2]
def _workflow_step_executor(func: Callable, context: workflow_context.WorkflowStepContext, step_id: "StepID", step_inputs: "StepInputTupleToResolve", outer_most_step_id: "StepID", catch_exceptions: bool, step_max_retries: int) -> Any: """Executor function for workflow step. Args: func: The workflow step function. context: Workflow step context. Used to access correct storage etc. step_id: The ID of the step. step_inputs: The inputs tuple of the step. outer_most_step_id: See "step_executor.execute_workflow" for explanation. catch_exceptions: If set to be true, return (Optional[Result], Optional[Error]) instead of Result. step_max_retries: Max number of retries encounter of a failure. Returns: Workflow step output. """ ret = None err = None # step_max_retries are for application level failure. # For ray failure, we should use max_retries. from ray.experimental.workflow.common import WorkflowStatus from ray.experimental.workflow.common import Workflow for _ in range(step_max_retries): try: workflow_context.update_workflow_step_context(context, step_id) args, kwargs = _resolve_step_inputs(step_inputs) # Running the actual step function ret = func(*args, **kwargs) # Save workflow output store = workflow_storage.get_workflow_storage() commit_step(store, step_id, ret, outer_most_step_id) if isinstance(ret, Workflow): # execute sub-workflow ret = execute_workflow(ret, outer_most_step_id) err = None break except BaseException as e: err = e if catch_exceptions: _record_step_status(step_id, WorkflowStatus.FINISHED) return (ret, err) else: if err is not None: _record_step_status(step_id, WorkflowStatus.RESUMABLE) raise err _record_step_status(step_id, WorkflowStatus.FINISHED) return ret
def _workflow_step_executor(step_type: StepType, func: Callable, context: workflow_context.WorkflowStepContext, step_id: "StepID", step_inputs: "StepInputTupleToResolve", outer_most_step_id: "StepID", catch_exceptions: bool, max_retries: int, last_step_of_workflow: bool) -> Any: """Executor function for workflow step. Args: step_type: The type of workflow step. func: The workflow step function. context: Workflow step context. Used to access correct storage etc. step_id: The ID of the step. step_inputs: The inputs tuple of the step. outer_most_step_id: See "step_executor.execute_workflow" for explanation. catch_exceptions: If set to be true, return (Optional[Result], Optional[Error]) instead of Result. max_retries: Max number of retries encounter of a failure. last_step_of_workflow: The step that generates the output of the workflow (including nested steps). Returns: Workflow step output. """ workflow_context.update_workflow_step_context(context, step_id) args, kwargs = _resolve_step_inputs(step_inputs) state, output = _wrap_run(func, step_type, step_id, catch_exceptions, max_retries, *args, **kwargs) if step_type != StepType.READONLY_ACTOR_METHOD: store = workflow_storage.get_workflow_storage() # Save workflow output commit_step(store, step_id, state, outer_most_step_id) # We MUST execute the workflow after saving the output. if isinstance(state, Workflow): if step_type == StepType.FUNCTION: # execute sub-workflow state = execute_workflow(state, outer_most_step_id, last_step_of_workflow) else: # TODO(suquark): Support returning a workflow inside # a virtual actor. raise TypeError("Only a workflow step function " "can return a workflow.") elif last_step_of_workflow: # advance the progress of the workflow store.advance_progress(step_id) _record_step_status(step_id, WorkflowStatus.SUCCESSFUL) logger.info(get_step_status_info(WorkflowStatus.SUCCESSFUL)) return state, output
def get_status(workflow_id: str) -> Optional[WorkflowStatus]: try: workflow_manager = get_management_actor() running = ray.get( workflow_manager.is_workflow_running.remote(workflow_id)) except Exception: running = False if running: return WorkflowStatus.RUNNING store = workflow_storage.get_workflow_storage(workflow_id) meta = store.load_workflow_meta() if meta is None: raise ValueError(f"No such workflow_id {workflow_id}") return meta.status
def run(entry_workflow: Workflow, workflow_id: Optional[str] = None) -> ray.ObjectRef: """Run a workflow asynchronously. """ store = get_global_storage() assert ray.is_initialized() if workflow_id is None: # Workflow ID format: {Entry workflow UUID}.{Unix time to nanoseconds} workflow_id = f"{str(uuid.uuid4())}.{time.time():.9f}" logger.info(f"Workflow job created. [id=\"{workflow_id}\", storage_url=" f"\"{store.storage_url}\"].") with workflow_context.workflow_step_context(workflow_id, store.storage_url): # checkpoint the workflow ws = workflow_storage.get_workflow_storage(workflow_id) wf_exists = True try: ws.get_entrypoint_step_id() except Exception: wf_exists = False # We only commit for # - virtual actor tasks: it's dynamic tasks, so we always add # - it's a new workflow # TODO (yic): follow up with force rerun if entry_workflow.data.step_type != StepType.FUNCTION or not wf_exists: commit_step(ws, "", entry_workflow, None) workflow_manager = get_or_create_management_actor() ignore_existing = (entry_workflow.data.step_type != StepType.FUNCTION) # NOTE: It is important to 'ray.get' the returned output. This # ensures caller of 'run()' holds the reference to the workflow # result. Otherwise if the actor removes the reference of the # workflow output, the caller may fail to resolve the result. result: "WorkflowExecutionResult" = ray.get( workflow_manager.run_or_resume.remote(workflow_id, ignore_existing)) if entry_workflow.data.step_type == StepType.FUNCTION: return flatten_workflow_output(workflow_id, result.persisted_output) else: return flatten_workflow_output(workflow_id, result.volatile_output)
def _resolve_dynamic_workflow_refs(workflow_refs: "List[WorkflowRef]"): """Get the output of a workflow step with the step ID at runtime. We lookup the output by the following order: 1. Query cached step output in the workflow manager. Fetch the physical output object. 2. If failed to fetch the physical output object, look into the storage to see whether the output is checkpointed. Load the checkpoint. 3. If failed to load the checkpoint, resume the step and get the output. """ workflow_manager = get_or_create_management_actor() context = workflow_context.get_workflow_step_context() workflow_id = context.workflow_id storage_url = context.storage_url workflow_ref_mapping = [] for workflow_ref in workflow_refs: step_ref = ray.get( workflow_manager.get_cached_step_output.remote( workflow_id, workflow_ref.step_id)) get_cached_step = False if step_ref is not None: try: output, _ = _resolve_object_ref(step_ref) get_cached_step = True except Exception: get_cached_step = False if not get_cached_step: wf_store = workflow_storage.get_workflow_storage() try: output = wf_store.load_step_output(workflow_ref.step_id) except DataLoadError: current_step_id = workflow_context.get_current_step_id() logger.warning("Failed to get the output of step " f"{workflow_ref.step_id}. Trying to resume it. " f"Current step: '{current_step_id}'") step_ref = recovery.resume_workflow_step( workflow_id, workflow_ref.step_id, storage_url).persisted_output output, _ = _resolve_object_ref(step_ref) workflow_ref_mapping.append(output) return workflow_ref_mapping
def run(entry_workflow: Workflow, workflow_id: Optional[str] = None) -> ray.ObjectRef: """Run a workflow asynchronously. See "api.run()" for details.""" store = get_global_storage() assert ray.is_initialized() if workflow_id is None: # Workflow ID format: {Entry workflow UUID}.{Unix time to nanoseconds} workflow_id = f"{entry_workflow.id}.{time.time():.9f}" logger.info(f"Workflow job created. [id=\"{workflow_id}\", storage_url=" f"\"{store.storage_url}\"].") # checkpoint the workflow ws = workflow_storage.get_workflow_storage(workflow_id) commit_step(ws, "", entry_workflow) workflow_manager = get_or_create_management_actor() # NOTE: It is important to 'ray.get' the returned output. This # ensures caller of 'run()' holds the reference to the workflow # result. Otherwise if the actor removes the reference of the # workflow output, the caller may fail to resolve the result. output = ray.get(workflow_manager.run_or_resume.remote(workflow_id)) return flatten_workflow_output(workflow_id, output)
def _workflow_step_executor(step_type: StepType, func: Callable, context: workflow_context.WorkflowStepContext, step_id: "StepID", baked_inputs: "_BakedWorkflowInputs", outer_most_step_id: "StepID", catch_exceptions: bool, max_retries: int, last_step_of_workflow: bool) -> Any: """Executor function for workflow step. Args: step_type: The type of workflow step. func: The workflow step function. context: Workflow step context. Used to access correct storage etc. step_id: The ID of the step. baked_inputs: The processed inputs for the step. outer_most_step_id: See "step_executor.execute_workflow" for explanation. catch_exceptions: If set to be true, return (Optional[Result], Optional[Error]) instead of Result. max_retries: Max number of retries encounter of a failure. last_step_of_workflow: The step that generates the output of the workflow (including nested steps). Returns: Workflow step output. """ workflow_context.update_workflow_step_context(context, step_id) args, kwargs = _resolve_step_inputs(baked_inputs) persisted_output, volatile_output = _wrap_run(func, step_type, step_id, catch_exceptions, max_retries, *args, **kwargs) if step_type != StepType.READONLY_ACTOR_METHOD: store = workflow_storage.get_workflow_storage() # Save workflow output commit_step(store, step_id, persisted_output, outer_most_step_id) # We MUST execute the workflow after saving the output. if isinstance(persisted_output, Workflow): if step_type == StepType.FUNCTION: # Passing down outer most step so inner nested steps would # access the same outer most step. if not outer_most_step_id: # The current workflow step returns a nested workflow, and # there is no outer step for the current step. So the # current step is the outer most step for the inner nested # workflow steps. outer_most_step_id = workflow_context.get_current_step_id() # execute sub-workflow persisted_output = execute_workflow( persisted_output, outer_most_step_id, last_step_of_workflow).persisted_output else: # TODO(suquark): Support returning a workflow inside # a virtual actor. raise TypeError("Only a workflow step function " "can return a workflow.") elif last_step_of_workflow: # advance the progress of the workflow store.advance_progress(step_id) _record_step_status(step_id, WorkflowStatus.SUCCESSFUL) logger.info(get_step_status_info(WorkflowStatus.SUCCESSFUL)) return persisted_output, volatile_output
def _workflow_step_executor( step_type: StepType, func: Callable, context: workflow_context.WorkflowStepContext, step_id: "StepID", baked_inputs: "_BakedWorkflowInputs", outer_most_step_id: "StepID", catch_exceptions: bool, max_retries: int, last_step_of_workflow: bool) -> Any: """Executor function for workflow step. Args: step_type: The type of workflow step. func: The workflow step function. context: Workflow step context. Used to access correct storage etc. step_id: The ID of the step. baked_inputs: The processed inputs for the step. outer_most_step_id: See "step_executor.execute_workflow" for explanation. catch_exceptions: If set to be true, return (Optional[Result], Optional[Error]) instead of Result. max_retries: Max number of retries encounter of a failure. last_step_of_workflow: The step that generates the output of the workflow (including nested steps). Returns: Workflow step output. """ workflow_context.update_workflow_step_context(context, step_id) args, kwargs = _resolve_step_inputs(baked_inputs) store = workflow_storage.get_workflow_storage() try: persisted_output, volatile_output = _wrap_run( func, step_type, step_id, catch_exceptions, max_retries, *args, **kwargs) except Exception as e: commit_step(store, step_id, None, e, outer_most_step_id) raise e if step_type == StepType.READONLY_ACTOR_METHOD: if isinstance(volatile_output, Workflow): raise TypeError( "Returning a Workflow from a readonly virtual actor " "is not allowed.") assert not isinstance(persisted_output, Workflow) else: store = workflow_storage.get_workflow_storage() commit_step(store, step_id, persisted_output, None, outer_most_step_id) if isinstance(persisted_output, Workflow): if step_type == StepType.FUNCTION: # Passing down outer most step so inner nested steps would # access the same outer most step. if not outer_most_step_id: # The current workflow step returns a nested workflow, and # there is no outer step for the current step. So the # current step is the outer most step for the inner nested # workflow steps. outer_most_step_id = workflow_context.get_current_step_id() assert volatile_output is None # execute sub-workflow result = execute_workflow(persisted_output, outer_most_step_id, last_step_of_workflow) # When virtual actor returns a workflow in the method, # the volatile_output and persisted_output will be put together persisted_output = result.persisted_output volatile_output = result.volatile_output elif last_step_of_workflow: # advance the progress of the workflow store.advance_progress(step_id) _record_step_status(step_id, WorkflowStatus.SUCCESSFUL) logger.info(get_step_status_info(WorkflowStatus.SUCCESSFUL)) if isinstance(volatile_output, Workflow): # This is the case where a step method is called in the virtual actor. # We need to run the method to get the final result. assert step_type == StepType.ACTOR_METHOD volatile_output = volatile_output.run_async( workflow_context.get_current_workflow_id()) return persisted_output, volatile_output