def resume_all(include_failed: bool = False) -> Dict[str, ray.ObjectRef]: """Resume all resumable workflow jobs. This can be used after cluster restart to resume all tasks. Args: with_failed: Whether to resume FAILED workflows. Examples: >>> workflow_step = failed_job.step() >>> output = workflow_step.run_async(workflow_id="failed_job") >>> try: >>> ray.get(output) >>> except Exception: >>> print("JobFailed") >>> jobs = workflow.list_all() >>> assert jobs == [("failed_job", workflow.FAILED)] >>> assert workflow.resume_all( >>> include_failed=True).get("failed_job") is not None Returns: A list of (workflow_id, returned_obj_ref) resumed. """ ensure_ray_initialized() return execution.resume_all(include_failed)
def get_actor(actor_id: str) -> "VirtualActor": """Get an virtual actor. Args: actor_id: The ID of the actor. Returns: A virtual actor. """ ensure_ray_initialized() return virtual_actor_class.get_actor(actor_id, storage_base.get_global_storage())
def get_status(workflow_id: str) -> WorkflowStatus: """Get the status for a given workflow. Args: workflow_id: The workflow to query. Examples: >>> workflow_step = trip.step() >>> output = workflow_step.run(workflow_id="trip") >>> assert workflow.SUCCESSFUL == workflow.get_status("trip") Returns: The status of that workflow """ ensure_ray_initialized() if not isinstance(workflow_id, str): raise TypeError("workflow_id has to be a string type.") return execution.get_status(workflow_id)
def list_all( status_filter: Optional[Union[Union[WorkflowStatus, str], Set[Union[WorkflowStatus, str]]]] = None ) -> List[Tuple[str, WorkflowStatus]]: """List all workflows matching a given status filter. Args: status: If given, only returns workflow with that status. This can be a single status or set of statuses. The string form of the status is also acceptable, i.e., "RUNNING"/"FAILED"/"SUCCESSFUL"/"CANCELED"/"RESUMABLE". Examples: >>> workflow_step = long_running_job.step() >>> wf = workflow_step.run_async(workflow_id="long_running_job") >>> jobs = workflow.list_all() >>> assert jobs == [ ("long_running_job", workflow.RUNNING) ] >>> ray.get(wf) >>> jobs = workflow.list_all({workflow.RUNNING}) >>> assert jobs == [] >>> jobs = workflow.list_all(workflow.SUCCESSFUL) >>> assert jobs == [ ("long_running_job", workflow.SUCCESSFUL) ] Returns: A list of tuple with workflow id and workflow status """ ensure_ray_initialized() if isinstance(status_filter, str): status_filter = set({WorkflowStatus(status_filter)}) elif isinstance(status_filter, WorkflowStatus): status_filter = set({status_filter}) elif isinstance(status_filter, set): if all([isinstance(s, str) for s in status_filter]): status_filter = {WorkflowStatus(s) for s in status_filter} elif not all([isinstance(s, WorkflowStatus) for s in status_filter]): raise TypeError("status_filter contains element which is not" " a type of `WorkflowStatus or str`." f" {status_filter}") elif status_filter is None: status_filter = set(WorkflowStatus.__members__.keys()) else: raise TypeError( "status_filter must be WorkflowStatus or a set of WorkflowStatus.") return execution.list_all(status_filter)
def cancel(workflow_id: str) -> None: """Cancel a workflow. Args: workflow_id: The workflow to cancel. Examples: >>> workflow_step = some_job.step() >>> output = workflow_step.run_async(workflow_id="some_job") >>> workflow.cancel(workflow_id="some_job") >>> assert [("some_job", workflow.CANCELED)] == workflow.list_all() Returns: None """ ensure_ray_initialized() if not isinstance(workflow_id, str): raise TypeError("workflow_id has to be a string type.") return execution.cancel(workflow_id)
def resume(workflow_id: str) -> ray.ObjectRef: """Resume a workflow. Resume a workflow and retrieve its output. If the workflow was incomplete, it will be re-executed from its checkpointed outputs. If the workflow was complete, returns the result immediately. Examples: >>> trip = start_trip.step() >>> res1 = trip.run_async(workflow_id="trip1") >>> res2 = workflow.resume("trip1") >>> assert ray.get(res1) == ray.get(res2) Args: workflow_id: The id of the workflow to resume. Returns: An object reference that can be used to retrieve the workflow result. """ ensure_ray_initialized() return execution.resume(workflow_id)
def get_output(workflow_id: str, *, name: Optional[str] = None) -> ray.ObjectRef: """Get the output of a running workflow. Args: workflow_id: The workflow to get the output of. name: If set, fetch the specific step instead of the output of the workflow. Examples: >>> trip = start_trip.options(name="trip").step() >>> res1 = trip.run_async(workflow_id="trip1") >>> # you could "get_output()" in another machine >>> res2 = workflow.get_output("trip1") >>> assert ray.get(res1) == ray.get(res2) >>> step_output = workflow.get_output("trip1", "trip") >>> assert ray.get(step_output) == ray.get(res1) Returns: An object reference that can be used to retrieve the workflow result. """ ensure_ray_initialized() return execution.get_output(workflow_id, name)
def prepare_inputs(): ensure_ray_initialized() return serialization_context.make_workflow_inputs( flattened_args)