2021-12-20 20:24:07 -05:00
Workflow Metadata
=================
Observability is important for workflows - sometimes we not only want
to get the output, but also want to gain insights on the internal
states (e.g., to measure the performance or find bottlenecks).
Workflow metadata provides several stats that help understand
2022-05-05 22:22:51 -07:00
the workflow, from basic running status and task options to performance
2021-12-20 20:24:07 -05:00
and user-imposed metadata.
Retrieving metadata
-------------------
Workflow metadata can be retrieved with `` workflow.get_metadata(workflow_id) `` .
For example:
.. code-block :: python
2022-05-05 22:22:51 -07:00
@ray.remote
2021-12-20 20:24:07 -05:00
def add(left: int, right: int) -> int:
return left + right
2022-05-05 22:22:51 -07:00
workflow.create(add.bind(10, 20)).run("add_example")
2021-12-20 20:24:07 -05:00
workflow_metadata = workflow.get_metadata("add_example")
assert workflow_metadata["status"] == "SUCCESSFUL"
assert "start_time" in workflow_metadata["stats"]
assert "end_time" in workflow_metadata["stats"]
2022-05-05 22:22:51 -07:00
You can also retrieve metadata for individual workflow tasks by
providing the task name:
2021-12-20 20:24:07 -05:00
.. code-block :: python
2022-05-16 15:41:14 -07:00
workflow.create(add.options(**workflow.options(name="add_task")).bind(10, 20)).run("add_example_2")
2021-12-20 20:24:07 -05:00
2022-05-05 22:22:51 -07:00
task_metadata = workflow.get_metadata("add_example_2", name="add_task")
2021-12-20 20:24:07 -05:00
2022-05-05 22:22:51 -07:00
assert "task_options" in task_metadata
2021-12-20 20:24:07 -05:00
assert "start_time" in workflow_metadata["stats"]
assert "end_time" in workflow_metadata["stats"]
User-defined metadata
---------------------
2022-05-05 22:22:51 -07:00
Custom metadata can be added to a workflow or a workflow task by the user,
2021-12-20 20:24:07 -05:00
which is useful when you want to attach some extra information to the
2022-05-05 22:22:51 -07:00
workflow or workflow task.
2021-12-20 20:24:07 -05:00
- workflow-level metadata can be added via `` .run(metadata=metadata) ``
2022-05-16 15:41:14 -07:00
- task-level metadata can be added via `` .options(**workflow.options(metadata=metadata)) `` or in the decorator `` @workflow.options(metadata=metadata) ``
2021-12-20 20:24:07 -05:00
.. code-block :: python
2022-05-16 15:41:14 -07:00
workflow.create(add.options(**workflow.options(name="add_task", metadata={"task_k": "task_v"})).bind(10, 20))\
2021-12-20 20:24:07 -05:00
.run("add_example_3", metadata={"workflow_k": "workflow_v"})
assert workflow.get_metadata("add_example_3")["user_metadata"] == {"workflow_k": "workflow_v"}
2022-05-05 22:22:51 -07:00
assert workflow.get_metadata("add_example_3", name="add_task")["user_metadata"] == {"task_k": "task_v"}
2021-12-20 20:24:07 -05:00
**Note: user-defined metadata must be a python dictionary with values that are
JSON serializable.**
Available Metrics
-----------------
**Workflow level**
- status: workflow states, can be one of RUNNING, FAILED, RESUMABLE, CANCELED, or SUCCESSFUL.
- user_metadata: a python dictionary of custom metadata by the user via `` workflow.run() `` .
- stats: workflow running stats, including workflow start time and end time.
2022-05-16 15:41:14 -07:00
**Task level**
2021-12-20 20:24:07 -05:00
2022-05-16 15:41:14 -07:00
- name: name of the task, either provided by the user via `` task.options(**workflow.options(name=xxx)) `` or generated by the system.
2022-05-05 22:22:51 -07:00
- task_options: options of the task, either provided by the user via `` task.options() `` or default by system.
- user_metadata: a python dictionary of custom metadata by the user via `` task.options() `` .
- stats: task running stats, including task start time and end time.
2021-12-20 20:24:07 -05:00
Notes
-----
1. Unlike `` get_output() `` , `` get_metadata() `` returns an immediate
result for the time it is called, this also means not all fields will
be available in the result if corresponding metadata is not available
(e.g., `` metadata["stats"]["end_time"] `` won't be available until the workflow
is completed).
.. code-block :: python
2022-05-05 22:22:51 -07:00
@ray.remote
2021-12-20 20:24:07 -05:00
def simple():
flag.touch() # touch a file here
time.sleep(1000)
return 0
2022-05-05 22:22:51 -07:00
workflow.create(simple.bind()).run_async(workflow_id)
2021-12-20 20:24:07 -05:00
2022-05-05 22:22:51 -07:00
# make sure workflow task starts running
2021-12-20 20:24:07 -05:00
while not flag.exists():
time.sleep(1)
workflow_metadata = workflow.get_metadata(workflow_id)
assert workflow_metadata["status"] == "RUNNING"
assert "start_time" in workflow_metadata["stats"]
assert "end_time" not in workflow_metadata["stats"]
workflow.cancel(workflow_id)
workflow_metadata = workflow.get_metadata(workflow_id)
assert workflow_metadata["status"] == "CANCELED"
assert "start_time" in workflow_metadata["stats"]
assert "end_time" not in workflow_metadata["stats"]
2. For resumed workflows, the current behavior is that "stats" will
be updated whenever a workflow is resumed.
.. code-block :: python
2022-05-05 22:22:51 -07:00
workflow_id = "simple"
error_flag = tmp_path / "error"
error_flag.touch()
@ray.remote
2021-12-20 20:24:07 -05:00
def simple():
if error_flag.exists():
raise ValueError()
return 0
2022-05-05 22:22:51 -07:00
with pytest.raises(ray.exceptions.RaySystemError):
workflow.create(simple.bind()).run(workflow_id)
2021-12-20 20:24:07 -05:00
workflow_metadata_failed = workflow.get_metadata(workflow_id)
assert workflow_metadata_failed["status"] == "FAILED"
2022-05-05 22:22:51 -07:00
# remove flag to make task success
2021-12-20 20:24:07 -05:00
error_flag.unlink()
ref = workflow.resume(workflow_id)
assert ray.get(ref) == 0
workflow_metadata_resumed = workflow.get_metadata(workflow_id)
assert workflow_metadata_resumed["status"] == "SUCCESSFUL"
2022-05-05 22:22:51 -07:00
# make sure resume updated running metrics
assert workflow_metadata_resumed["stats"]["start_time"] > workflow_metadata_failed["stats"]["start_time"]
2021-12-20 20:24:07 -05:00
assert workflow_metadata_resumed["stats"]["end_time"] > workflow_metadata_failed["stats"]["end_time"]