kacpermuda commented on code in PR #39520:
URL: https://github.com/apache/airflow/pull/39520#discussion_r1624774970
##########
airflow/providers/openlineage/utils/utils.py:
##########
@@ -262,20 +271,151 @@ def get_airflow_run_facet(
task_instance: TaskInstance,
task: BaseOperator,
task_uuid: str,
-):
+) -> dict[str, BaseFacet]:
return {
- "airflow": attrs.asdict(
- AirflowRunFacet(
- dag=DagInfo(dag),
- dagRun=DagRunInfo(dag_run),
- taskInstance=TaskInstanceInfo(task_instance),
- task=TaskInfo(task),
- taskUuid=task_uuid,
- )
+ "airflow": AirflowRunFacet(
+ dag=DagInfo(dag),
+ dagRun=DagRunInfo(dag_run),
+ taskInstance=TaskInstanceInfo(task_instance),
+ task=TaskInfo(task),
+ taskUuid=task_uuid,
+ )
+ }
+
+
+def get_airflow_job_facet(dag_run: DagRun) -> dict[str, BaseFacet]:
+ if not dag_run.dag:
+ return {}
+ return {
+ "airflowDag": AirflowJobFacet(
+ taskTree=_get_dag_tree(dag_run.dag),
+ taskGroups=_get_task_groups_details(dag_run.dag),
+ tasks=_get_tasks_details(dag_run.dag),
+ )
+ }
+
+
+def get_airflow_state_run_facet(dag_run: DagRun) -> dict[str, BaseFacet]:
+ return {
+ "airflowState": AirflowStateRunFacet(
+ dagRunState=dag_run.get_state(),
+ tasksState={
+ ti.task_id: ti.state
+ for ti in dag_run.fetch_task_instances(dag_id=dag_run.dag_id,
run_id=dag_run.run_id)
Review Comment:
There was a particular concern about this line as it calls Airflow internal
API and this method is called within DAG level listener methods (we are on a
scheduler). I've tested it in breeze, local env and astro cloud, using
different executors (Celery, Local, K8s) and under some pressure (100-2000
concurrent dag runs). **I've encountered NO issues**, but please let me know if
there is a better, more reliable way of testing it. AFAIK, this feature is
production ready.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]