apache-airflow-providers-openlineage 1.3.1rc1__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.

Potentially problematic release.


This version of apache-airflow-providers-openlineage might be problematic. Click here for more details.

@@ -0,0 +1,375 @@
1
+ # Licensed to the Apache Software Foundation (ASF) under one
2
+ # or more contributor license agreements. See the NOTICE file
3
+ # distributed with this work for additional information
4
+ # regarding copyright ownership. The ASF licenses this file
5
+ # to you under the Apache License, Version 2.0 (the
6
+ # "License"); you may not use this file except in compliance
7
+ # with the License. You may obtain a copy of the License at
8
+ #
9
+ # http://www.apache.org/licenses/LICENSE-2.0
10
+ #
11
+ # Unless required by applicable law or agreed to in writing,
12
+ # software distributed under the License is distributed on an
13
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14
+ # KIND, either express or implied. See the License for the
15
+ # specific language governing permissions and limitations
16
+ # under the License.
17
+ from __future__ import annotations
18
+
19
+ import os
20
+ import uuid
21
+ from typing import TYPE_CHECKING
22
+
23
+ import yaml
24
+ from openlineage.client import OpenLineageClient, set_producer
25
+ from openlineage.client.facet import (
26
+ BaseFacet,
27
+ DocumentationJobFacet,
28
+ ErrorMessageRunFacet,
29
+ NominalTimeRunFacet,
30
+ OwnershipJobFacet,
31
+ OwnershipJobFacetOwners,
32
+ ParentRunFacet,
33
+ ProcessingEngineRunFacet,
34
+ SourceCodeLocationJobFacet,
35
+ )
36
+ from openlineage.client.run import Job, Run, RunEvent, RunState
37
+
38
+ from airflow.configuration import conf
39
+ from airflow.providers.openlineage import __version__ as OPENLINEAGE_PROVIDER_VERSION
40
+ from airflow.providers.openlineage.utils.utils import OpenLineageRedactor
41
+ from airflow.stats import Stats
42
+ from airflow.utils.log.logging_mixin import LoggingMixin
43
+
44
+ if TYPE_CHECKING:
45
+ from airflow.models.dagrun import DagRun
46
+ from airflow.providers.openlineage.extractors import OperatorLineage
47
+ from airflow.utils.log.secrets_masker import SecretsMasker
48
+
49
+ _DAG_DEFAULT_NAMESPACE = "default"
50
+
51
+ _DAG_NAMESPACE = conf.get(
52
+ "openlineage", "namespace", fallback=os.getenv("OPENLINEAGE_NAMESPACE", _DAG_DEFAULT_NAMESPACE)
53
+ )
54
+
55
+ _PRODUCER = f"https://github.com/apache/airflow/tree/providers-openlineage/{OPENLINEAGE_PROVIDER_VERSION}"
56
+
57
+ set_producer(_PRODUCER)
58
+
59
+
60
+ class OpenLineageAdapter(LoggingMixin):
61
+ """Translate Airflow metadata to OpenLineage events instead of creating them from Airflow code."""
62
+
63
+ def __init__(self, client: OpenLineageClient | None = None, secrets_masker: SecretsMasker | None = None):
64
+ super().__init__()
65
+ self._client = client
66
+ if not secrets_masker:
67
+ from airflow.utils.log.secrets_masker import _secrets_masker
68
+
69
+ secrets_masker = _secrets_masker()
70
+ self._redacter = OpenLineageRedactor.from_masker(secrets_masker)
71
+
72
+ def get_or_create_openlineage_client(self) -> OpenLineageClient:
73
+ if not self._client:
74
+ config = self.get_openlineage_config()
75
+ if config:
76
+ self._client = OpenLineageClient.from_dict(config=config)
77
+ else:
78
+ self._client = OpenLineageClient.from_environment()
79
+ return self._client
80
+
81
+ def get_openlineage_config(self) -> dict | None:
82
+ # First, try to read from YAML file
83
+ openlineage_config_path = conf.get("openlineage", "config_path")
84
+ if openlineage_config_path:
85
+ config = self._read_yaml_config(openlineage_config_path)
86
+ if config:
87
+ return config.get("transport", None)
88
+ # Second, try to get transport config
89
+ transport = conf.getjson("openlineage", "transport")
90
+ if not transport:
91
+ return None
92
+ elif not isinstance(transport, dict):
93
+ raise ValueError(f"{transport} is not a dict")
94
+ return transport
95
+
96
+ def _read_yaml_config(self, path: str) -> dict | None:
97
+ with open(path) as config_file:
98
+ return yaml.safe_load(config_file)
99
+
100
+ @staticmethod
101
+ def build_dag_run_id(dag_id, dag_run_id):
102
+ return str(uuid.uuid3(uuid.NAMESPACE_URL, f"{_DAG_NAMESPACE}.{dag_id}.{dag_run_id}"))
103
+
104
+ @staticmethod
105
+ def build_task_instance_run_id(task_id, execution_date, try_number):
106
+ return str(
107
+ uuid.uuid3(
108
+ uuid.NAMESPACE_URL,
109
+ f"{_DAG_NAMESPACE}.{task_id}.{execution_date}.{try_number}",
110
+ )
111
+ )
112
+
113
+ def emit(self, event: RunEvent):
114
+ if not self._client:
115
+ self._client = self.get_or_create_openlineage_client()
116
+ redacted_event: RunEvent = self._redacter.redact(event, max_depth=20) # type: ignore[assignment]
117
+ try:
118
+ with Stats.timer("ol.emit.attempts"):
119
+ return self._client.emit(redacted_event)
120
+ except Exception as e:
121
+ Stats.incr("ol.emit.failed")
122
+ self.log.warning("Failed to emit OpenLineage event of id %s", event.run.runId)
123
+ self.log.debug("OpenLineage emission failure: %s", e)
124
+
125
+ def start_task(
126
+ self,
127
+ run_id: str,
128
+ job_name: str,
129
+ job_description: str,
130
+ event_time: str,
131
+ parent_job_name: str | None,
132
+ parent_run_id: str | None,
133
+ code_location: str | None,
134
+ nominal_start_time: str | None,
135
+ nominal_end_time: str | None,
136
+ owners: list[str],
137
+ task: OperatorLineage | None,
138
+ run_facets: dict[str, BaseFacet] | None = None, # Custom run facets
139
+ ):
140
+ """
141
+ Emits openlineage event of type START.
142
+
143
+ :param run_id: globally unique identifier of task in dag run
144
+ :param job_name: globally unique identifier of task in dag
145
+ :param job_description: user provided description of job
146
+ :param event_time:
147
+ :param parent_job_name: the name of the parent job (typically the DAG,
148
+ but possibly a task group)
149
+ :param parent_run_id: identifier of job spawning this task
150
+ :param code_location: file path or URL of DAG file
151
+ :param nominal_start_time: scheduled time of dag run
152
+ :param nominal_end_time: following schedule of dag run
153
+ :param owners: list of owners of DAG
154
+ :param task: metadata container with information extracted from operator
155
+ :param run_facets: custom run facets
156
+ """
157
+ from airflow.version import version as AIRFLOW_VERSION
158
+
159
+ processing_engine_version_facet = ProcessingEngineRunFacet(
160
+ version=AIRFLOW_VERSION,
161
+ name="Airflow",
162
+ openlineageAdapterVersion=OPENLINEAGE_PROVIDER_VERSION,
163
+ )
164
+
165
+ if not run_facets:
166
+ run_facets = {}
167
+ if task:
168
+ run_facets = {**task.run_facets, **run_facets}
169
+ run_facets["processing_engine"] = processing_engine_version_facet # type: ignore
170
+ event = RunEvent(
171
+ eventType=RunState.START,
172
+ eventTime=event_time,
173
+ run=self._build_run(
174
+ run_id=run_id,
175
+ job_name=job_name,
176
+ parent_job_name=parent_job_name,
177
+ parent_run_id=parent_run_id,
178
+ nominal_start_time=nominal_start_time,
179
+ nominal_end_time=nominal_end_time,
180
+ run_facets=run_facets,
181
+ ),
182
+ job=self._build_job(
183
+ job_name=job_name,
184
+ job_description=job_description,
185
+ code_location=code_location,
186
+ owners=owners,
187
+ job_facets=task.job_facets if task else None,
188
+ ),
189
+ inputs=task.inputs if task else [],
190
+ outputs=task.outputs if task else [],
191
+ producer=_PRODUCER,
192
+ )
193
+ self.emit(event)
194
+
195
+ def complete_task(
196
+ self,
197
+ run_id: str,
198
+ job_name: str,
199
+ parent_job_name: str | None,
200
+ parent_run_id: str | None,
201
+ end_time: str,
202
+ task: OperatorLineage,
203
+ ):
204
+ """
205
+ Emits openlineage event of type COMPLETE.
206
+
207
+ :param run_id: globally unique identifier of task in dag run
208
+ :param job_name: globally unique identifier of task between dags
209
+ :param parent_job_name: the name of the parent job (typically the DAG,
210
+ but possibly a task group)
211
+ :param parent_run_id: identifier of job spawning this task
212
+ :param end_time: time of task completion
213
+ :param task: metadata container with information extracted from operator
214
+ """
215
+ event = RunEvent(
216
+ eventType=RunState.COMPLETE,
217
+ eventTime=end_time,
218
+ run=self._build_run(
219
+ run_id=run_id,
220
+ job_name=job_name,
221
+ parent_job_name=parent_job_name,
222
+ parent_run_id=parent_run_id,
223
+ run_facets=task.run_facets,
224
+ ),
225
+ job=self._build_job(job_name, job_facets=task.job_facets),
226
+ inputs=task.inputs,
227
+ outputs=task.outputs,
228
+ producer=_PRODUCER,
229
+ )
230
+ self.emit(event)
231
+
232
+ def fail_task(
233
+ self,
234
+ run_id: str,
235
+ job_name: str,
236
+ parent_job_name: str | None,
237
+ parent_run_id: str | None,
238
+ end_time: str,
239
+ task: OperatorLineage,
240
+ ):
241
+ """
242
+ Emits openlineage event of type FAIL.
243
+
244
+ :param run_id: globally unique identifier of task in dag run
245
+ :param job_name: globally unique identifier of task between dags
246
+ :param parent_job_name: the name of the parent job (typically the DAG,
247
+ but possibly a task group)
248
+ :param parent_run_id: identifier of job spawning this task
249
+ :param end_time: time of task completion
250
+ :param task: metadata container with information extracted from operator
251
+ """
252
+ event = RunEvent(
253
+ eventType=RunState.FAIL,
254
+ eventTime=end_time,
255
+ run=self._build_run(
256
+ run_id=run_id,
257
+ job_name=job_name,
258
+ parent_job_name=parent_job_name,
259
+ parent_run_id=parent_run_id,
260
+ run_facets=task.run_facets,
261
+ ),
262
+ job=self._build_job(job_name, job_facets=task.job_facets),
263
+ inputs=task.inputs,
264
+ outputs=task.outputs,
265
+ producer=_PRODUCER,
266
+ )
267
+ self.emit(event)
268
+
269
+ def dag_started(
270
+ self,
271
+ dag_run: DagRun,
272
+ msg: str,
273
+ nominal_start_time: str,
274
+ nominal_end_time: str,
275
+ ):
276
+ event = RunEvent(
277
+ eventType=RunState.START,
278
+ eventTime=dag_run.start_date.isoformat(),
279
+ job=Job(name=dag_run.dag_id, namespace=_DAG_NAMESPACE),
280
+ run=self._build_run(
281
+ run_id=self.build_dag_run_id(dag_run.dag_id, dag_run.run_id),
282
+ job_name=dag_run.dag_id,
283
+ nominal_start_time=nominal_start_time,
284
+ nominal_end_time=nominal_end_time,
285
+ ),
286
+ inputs=[],
287
+ outputs=[],
288
+ producer=_PRODUCER,
289
+ )
290
+ self.emit(event)
291
+
292
+ def dag_success(self, dag_run: DagRun, msg: str):
293
+ event = RunEvent(
294
+ eventType=RunState.COMPLETE,
295
+ eventTime=dag_run.end_date.isoformat(),
296
+ job=Job(name=dag_run.dag_id, namespace=_DAG_NAMESPACE),
297
+ run=Run(runId=self.build_dag_run_id(dag_run.dag_id, dag_run.run_id)),
298
+ inputs=[],
299
+ outputs=[],
300
+ producer=_PRODUCER,
301
+ )
302
+ self.emit(event)
303
+
304
+ def dag_failed(self, dag_run: DagRun, msg: str):
305
+ event = RunEvent(
306
+ eventType=RunState.FAIL,
307
+ eventTime=dag_run.end_date.isoformat(),
308
+ job=Job(name=dag_run.dag_id, namespace=_DAG_NAMESPACE),
309
+ run=Run(
310
+ runId=self.build_dag_run_id(dag_run.dag_id, dag_run.run_id),
311
+ facets={"errorMessage": ErrorMessageRunFacet(message=msg, programmingLanguage="python")},
312
+ ),
313
+ inputs=[],
314
+ outputs=[],
315
+ producer=_PRODUCER,
316
+ )
317
+ self.emit(event)
318
+
319
+ @staticmethod
320
+ def _build_run(
321
+ run_id: str,
322
+ job_name: str,
323
+ parent_job_name: str | None = None,
324
+ parent_run_id: str | None = None,
325
+ nominal_start_time: str | None = None,
326
+ nominal_end_time: str | None = None,
327
+ run_facets: dict[str, BaseFacet] | None = None,
328
+ ) -> Run:
329
+ facets: dict[str, BaseFacet] = {}
330
+ if nominal_start_time:
331
+ facets.update({"nominalTime": NominalTimeRunFacet(nominal_start_time, nominal_end_time)})
332
+ if parent_run_id:
333
+ parent_run_facet = ParentRunFacet.create(
334
+ runId=parent_run_id,
335
+ namespace=_DAG_NAMESPACE,
336
+ name=parent_job_name or job_name,
337
+ )
338
+ facets.update(
339
+ {
340
+ "parent": parent_run_facet,
341
+ "parentRun": parent_run_facet, # Keep sending this for the backward compatibility
342
+ }
343
+ )
344
+
345
+ if run_facets:
346
+ facets.update(run_facets)
347
+
348
+ return Run(run_id, facets)
349
+
350
+ @staticmethod
351
+ def _build_job(
352
+ job_name: str,
353
+ job_description: str | None = None,
354
+ code_location: str | None = None,
355
+ owners: list[str] | None = None,
356
+ job_facets: dict[str, BaseFacet] | None = None,
357
+ ):
358
+ facets: dict[str, BaseFacet] = {}
359
+
360
+ if job_description:
361
+ facets.update({"documentation": DocumentationJobFacet(description=job_description)})
362
+ if code_location:
363
+ facets.update({"sourceCodeLocation": SourceCodeLocationJobFacet("", url=code_location)})
364
+ if owners:
365
+ facets.update(
366
+ {
367
+ "ownership": OwnershipJobFacet(
368
+ owners=[OwnershipJobFacetOwners(name=owner) for owner in owners]
369
+ )
370
+ }
371
+ )
372
+ if job_facets:
373
+ facets = {**facets, **job_facets}
374
+
375
+ return Job(_DAG_NAMESPACE, job_name, facets)
@@ -0,0 +1,73 @@
1
+ # Licensed to the Apache Software Foundation (ASF) under one
2
+ # or more contributor license agreements. See the NOTICE file
3
+ # distributed with this work for additional information
4
+ # regarding copyright ownership. The ASF licenses this file
5
+ # to you under the Apache License, Version 2.0 (the
6
+ # "License"); you may not use this file except in compliance
7
+ # with the License. You may obtain a copy of the License at
8
+ #
9
+ # http://www.apache.org/licenses/LICENSE-2.0
10
+ #
11
+ # Unless required by applicable law or agreed to in writing,
12
+ # software distributed under the License is distributed on an
13
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14
+ # KIND, either express or implied. See the License for the
15
+ # specific language governing permissions and limitations
16
+ # under the License.
17
+ from __future__ import annotations
18
+
19
+ from attrs import define
20
+ from openlineage.client.facet import BaseFacet
21
+ from openlineage.client.utils import RedactMixin
22
+
23
+
24
+ @define(slots=False)
25
+ class AirflowMappedTaskRunFacet(BaseFacet):
26
+ """Run facet containing information about mapped tasks."""
27
+
28
+ mapIndex: int
29
+ operatorClass: str
30
+
31
+ _additional_skip_redact = ["operatorClass"]
32
+
33
+ @classmethod
34
+ def from_task_instance(cls, task_instance):
35
+ task = task_instance.task
36
+ from airflow.providers.openlineage.utils.utils import get_operator_class
37
+
38
+ return cls(
39
+ mapIndex=task_instance.map_index,
40
+ operatorClass=f"{get_operator_class(task).__module__}.{get_operator_class(task).__name__}",
41
+ )
42
+
43
+
44
+ @define(slots=False)
45
+ class AirflowRunFacet(BaseFacet):
46
+ """Composite Airflow run facet."""
47
+
48
+ dag: dict
49
+ dagRun: dict
50
+ task: dict
51
+ taskInstance: dict
52
+ taskUuid: str
53
+
54
+
55
+ @define(slots=False)
56
+ class UnknownOperatorInstance(RedactMixin):
57
+ """Describes an unknown operator.
58
+
59
+ This specifies the (class) name of the operator and its properties.
60
+ """
61
+
62
+ name: str
63
+ properties: dict[str, object]
64
+ type: str = "operator"
65
+
66
+ _skip_redact = ["name", "type"]
67
+
68
+
69
+ @define(slots=False)
70
+ class UnknownOperatorAttributeRunFacet(BaseFacet):
71
+ """RunFacet that describes unknown operators in an Airflow DAG."""
72
+
73
+ unknownItems: list[UnknownOperatorInstance]
@@ -0,0 +1,226 @@
1
+ # Licensed to the Apache Software Foundation (ASF) under one
2
+ # or more contributor license agreements. See the NOTICE file
3
+ # distributed with this work for additional information
4
+ # regarding copyright ownership. The ASF licenses this file
5
+ # to you under the Apache License, Version 2.0 (the
6
+ # "License"); you may not use this file except in compliance
7
+ # with the License. You may obtain a copy of the License at
8
+ #
9
+ # http://www.apache.org/licenses/LICENSE-2.0
10
+ #
11
+ # Unless required by applicable law or agreed to in writing,
12
+ # software distributed under the License is distributed on an
13
+ # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14
+ # KIND, either express or implied. See the License for the
15
+ # specific language governing permissions and limitations
16
+ # under the License.
17
+ from __future__ import annotations
18
+
19
+ import logging
20
+ from concurrent.futures import ThreadPoolExecutor
21
+ from datetime import datetime
22
+ from typing import TYPE_CHECKING
23
+
24
+ from airflow.listeners import hookimpl
25
+ from airflow.providers.openlineage.extractors import ExtractorManager
26
+ from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter
27
+ from airflow.providers.openlineage.utils.utils import (
28
+ get_airflow_run_facet,
29
+ get_custom_facets,
30
+ get_job_name,
31
+ print_warning,
32
+ )
33
+ from airflow.utils.timeout import timeout
34
+
35
+ if TYPE_CHECKING:
36
+ from sqlalchemy.orm import Session
37
+
38
+ from airflow.models import DagRun, TaskInstance
39
+
40
+ _openlineage_listener: OpenLineageListener | None = None
41
+
42
+
43
+ class OpenLineageListener:
44
+ """OpenLineage listener sends events on task instance and dag run starts, completes and failures."""
45
+
46
+ def __init__(self):
47
+ self._executor = None
48
+ self.log = logging.getLogger(__name__)
49
+ self.extractor_manager = ExtractorManager()
50
+ self.adapter = OpenLineageAdapter()
51
+
52
+ @hookimpl
53
+ def on_task_instance_running(
54
+ self,
55
+ previous_state,
56
+ task_instance: TaskInstance,
57
+ session: Session, # This will always be QUEUED
58
+ ):
59
+ if not hasattr(task_instance, "task"):
60
+ self.log.warning(
61
+ f"No task set for TI object task_id: {task_instance.task_id} - "
62
+ f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}"
63
+ )
64
+ return
65
+
66
+ self.log.debug("OpenLineage listener got notification about task instance start")
67
+ dagrun = task_instance.dag_run
68
+ task = task_instance.task
69
+ dag = task.dag
70
+
71
+ @print_warning(self.log)
72
+ def on_running():
73
+ # that's a workaround to detect task running from deferred state
74
+ # we return here because Airflow 2.3 needs task from deferred state
75
+ if task_instance.next_method is not None:
76
+ return
77
+ parent_run_id = self.adapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
78
+
79
+ task_uuid = self.adapter.build_task_instance_run_id(
80
+ task.task_id, task_instance.execution_date, task_instance.try_number
81
+ )
82
+
83
+ task_metadata = self.extractor_manager.extract_metadata(dagrun, task)
84
+
85
+ start_date = task_instance.start_date if task_instance.start_date else datetime.now()
86
+ data_interval_start = (
87
+ dagrun.data_interval_start.isoformat() if dagrun.data_interval_start else None
88
+ )
89
+ data_interval_end = dagrun.data_interval_end.isoformat() if dagrun.data_interval_end else None
90
+
91
+ self.adapter.start_task(
92
+ run_id=task_uuid,
93
+ job_name=get_job_name(task),
94
+ job_description=dag.description,
95
+ event_time=start_date.isoformat(),
96
+ parent_job_name=dag.dag_id,
97
+ parent_run_id=parent_run_id,
98
+ code_location=None,
99
+ nominal_start_time=data_interval_start,
100
+ nominal_end_time=data_interval_end,
101
+ owners=dag.owner.split(", "),
102
+ task=task_metadata,
103
+ run_facets={
104
+ **get_custom_facets(task_instance),
105
+ **get_airflow_run_facet(dagrun, dag, task_instance, task, task_uuid),
106
+ },
107
+ )
108
+
109
+ on_running()
110
+
111
+ @hookimpl
112
+ def on_task_instance_success(self, previous_state, task_instance: TaskInstance, session):
113
+ self.log.debug("OpenLineage listener got notification about task instance success")
114
+
115
+ dagrun = task_instance.dag_run
116
+ task = task_instance.task
117
+ dag = task.dag
118
+
119
+ task_uuid = OpenLineageAdapter.build_task_instance_run_id(
120
+ task.task_id, task_instance.execution_date, task_instance.try_number - 1
121
+ )
122
+
123
+ @print_warning(self.log)
124
+ def on_success():
125
+ parent_run_id = OpenLineageAdapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
126
+
127
+ task_metadata = self.extractor_manager.extract_metadata(
128
+ dagrun, task, complete=True, task_instance=task_instance
129
+ )
130
+
131
+ end_date = task_instance.end_date if task_instance.end_date else datetime.now()
132
+
133
+ self.adapter.complete_task(
134
+ run_id=task_uuid,
135
+ job_name=get_job_name(task),
136
+ parent_job_name=dag.dag_id,
137
+ parent_run_id=parent_run_id,
138
+ end_time=end_date.isoformat(),
139
+ task=task_metadata,
140
+ )
141
+
142
+ on_success()
143
+
144
+ @hookimpl
145
+ def on_task_instance_failed(self, previous_state, task_instance: TaskInstance, session):
146
+ self.log.debug("OpenLineage listener got notification about task instance failure")
147
+
148
+ dagrun = task_instance.dag_run
149
+ task = task_instance.task
150
+ dag = task.dag
151
+
152
+ task_uuid = OpenLineageAdapter.build_task_instance_run_id(
153
+ task.task_id, task_instance.execution_date, task_instance.try_number
154
+ )
155
+
156
+ @print_warning(self.log)
157
+ def on_failure():
158
+ parent_run_id = OpenLineageAdapter.build_dag_run_id(dag.dag_id, dagrun.run_id)
159
+
160
+ task_metadata = self.extractor_manager.extract_metadata(
161
+ dagrun, task, complete=True, task_instance=task_instance
162
+ )
163
+
164
+ end_date = task_instance.end_date if task_instance.end_date else datetime.now()
165
+
166
+ self.adapter.fail_task(
167
+ run_id=task_uuid,
168
+ job_name=get_job_name(task),
169
+ parent_job_name=dag.dag_id,
170
+ parent_run_id=parent_run_id,
171
+ end_time=end_date.isoformat(),
172
+ task=task_metadata,
173
+ )
174
+
175
+ on_failure()
176
+
177
+ @property
178
+ def executor(self):
179
+ if not self._executor:
180
+ self._executor = ThreadPoolExecutor(max_workers=8, thread_name_prefix="openlineage_")
181
+ return self._executor
182
+
183
+ @hookimpl
184
+ def on_starting(self, component):
185
+ self.log.debug("on_starting: %s", component.__class__.__name__)
186
+
187
+ @hookimpl
188
+ def before_stopping(self, component):
189
+ self.log.debug("before_stopping: %s", component.__class__.__name__)
190
+ # TODO: configure this with Airflow config
191
+ with timeout(30):
192
+ self.executor.shutdown(wait=True)
193
+
194
+ @hookimpl
195
+ def on_dag_run_running(self, dag_run: DagRun, msg: str):
196
+ data_interval_start = dag_run.data_interval_start.isoformat() if dag_run.data_interval_start else None
197
+ data_interval_end = dag_run.data_interval_end.isoformat() if dag_run.data_interval_end else None
198
+ self.executor.submit(
199
+ self.adapter.dag_started,
200
+ dag_run=dag_run,
201
+ msg=msg,
202
+ nominal_start_time=data_interval_start,
203
+ nominal_end_time=data_interval_end,
204
+ )
205
+
206
+ @hookimpl
207
+ def on_dag_run_success(self, dag_run: DagRun, msg: str):
208
+ if not self.executor:
209
+ self.log.debug("Executor have not started before `on_dag_run_success`")
210
+ return
211
+ self.executor.submit(self.adapter.dag_success, dag_run=dag_run, msg=msg)
212
+
213
+ @hookimpl
214
+ def on_dag_run_failed(self, dag_run: DagRun, msg: str):
215
+ if not self.executor:
216
+ self.log.debug("Executor have not started before `on_dag_run_failed`")
217
+ return
218
+ self.executor.submit(self.adapter.dag_failed, dag_run=dag_run, msg=msg)
219
+
220
+
221
+ def get_openlineage_listener() -> OpenLineageListener:
222
+ """Get singleton listener manager."""
223
+ global _openlineage_listener
224
+ if not _openlineage_listener:
225
+ _openlineage_listener = OpenLineageListener()
226
+ return _openlineage_listener