1
+ import asyncio
1
2
from abc import ABC , abstractmethod
2
3
from collections .abc import Iterator
3
4
from itertools import chain
4
5
from typing import Optional
5
6
6
7
from dagster import AssetMaterialization
8
+ from dagster ._core .definitions .metadata .metadata_value import MetadataValue
7
9
from dagster ._core .events import (
8
10
AssetMaterializationPlannedData ,
9
11
DagsterEvent ,
21
23
from dagster_airlift .core .airflow_defs_data import AirflowDefinitionsData
22
24
from dagster_airlift .core .airflow_instance import AirflowInstance
23
25
from dagster_airlift .core .monitoring_job .utils import (
26
+ extract_metadata_from_logs ,
24
27
get_dagster_run_for_airflow_repr ,
25
28
structured_log ,
26
29
)
@@ -122,6 +125,7 @@ def persist_state(
122
125
@record
123
126
class TaskInstanceCompleted (AirflowEvent ):
124
127
task_instance : TaskInstance
128
+ metadata : dict [str , MetadataValue ]
125
129
126
130
@property
127
131
def timestamp (self ) -> float :
@@ -137,7 +141,9 @@ def persist_state(
137
141
for asset in airflow_data .mapped_asset_keys_by_task_handle [self .task_instance .task_handle ]:
138
142
# IMPROVEME: Add metadata to the materialization event.
139
143
_report_materialization (
140
- context , corresponding_run , AssetMaterialization (asset_key = asset )
144
+ context ,
145
+ corresponding_run ,
146
+ AssetMaterialization (asset_key = asset , metadata = self .metadata ),
141
147
)
142
148
143
149
@@ -254,6 +260,37 @@ def _process_completed_runs(
254
260
break
255
261
256
262
263
+ async def _retrieve_logs_for_task_instance (
264
+ context : OpExecutionContext ,
265
+ airflow_instance : AirflowInstance ,
266
+ task_instance : TaskInstance ,
267
+ ) -> TaskInstanceCompleted :
268
+ logs = airflow_instance .get_task_instance_logs (
269
+ task_instance .dag_id ,
270
+ task_instance .task_id ,
271
+ task_instance .run_id ,
272
+ task_instance .try_number ,
273
+ )
274
+ metadata = extract_metadata_from_logs (context , logs )
275
+
276
+ return TaskInstanceCompleted (task_instance = task_instance , metadata = metadata )
277
+
278
+
279
+ async def _async_process_task_instances (
280
+ context : OpExecutionContext ,
281
+ airflow_instance : AirflowInstance ,
282
+ task_instances : list [TaskInstance ],
283
+ ) -> Iterator [TaskInstanceCompleted ]:
284
+ results = await asyncio .gather (
285
+ * (
286
+ _retrieve_logs_for_task_instance (context , airflow_instance , task_instance )
287
+ for task_instance in task_instances
288
+ )
289
+ )
290
+
291
+ return results
292
+
293
+
257
294
def _process_task_instances (
258
295
context : OpExecutionContext ,
259
296
airflow_data : AirflowDefinitionsData ,
@@ -275,9 +312,7 @@ def _process_task_instances(
275
312
context ,
276
313
f"Found { len (task_instances )} completed task instances in the time range { datetime_from_timestamp (range_start )} to { datetime_from_timestamp (range_end )} " ,
277
314
)
278
- yield from (
279
- TaskInstanceCompleted (task_instance = task_instance ) for task_instance in task_instances
280
- )
315
+ yield from asyncio .run (_async_process_task_instances (context , airflow_instance , task_instances ))
281
316
282
317
283
318
def persist_events (
0 commit comments