-
Notifications
You must be signed in to change notification settings - Fork 14.7k
/
Copy pathtest_manager.py
326 lines (289 loc) · 11.6 KB
/
test_manager.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations
import tempfile
from typing import TYPE_CHECKING, Any
from unittest.mock import MagicMock
import pytest
from openlineage.client.event_v2 import Dataset as OpenLineageDataset
from openlineage.client.facet_v2 import (
documentation_dataset,
ownership_dataset,
schema_dataset,
)
from airflow.io.path import ObjectStoragePath
from airflow.lineage.entities import Column, File, Table, User
from airflow.models.baseoperator import BaseOperator
from airflow.models.taskinstance import TaskInstance
from airflow.providers.openlineage.extractors import OperatorLineage
from airflow.providers.openlineage.extractors.manager import ExtractorManager
from airflow.providers.openlineage.utils.utils import Asset
from airflow.utils.state import State
from tests_common.test_utils.compat import PythonOperator
from tests_common.test_utils.version_compat import AIRFLOW_V_2_10_PLUS
if TYPE_CHECKING:
from airflow.utils.context import Context
if AIRFLOW_V_2_10_PLUS:
@pytest.fixture
def hook_lineage_collector():
from airflow.lineage import hook
from airflow.providers.common.compat.lineage.hook import (
get_hook_lineage_collector,
)
hook._hook_lineage_collector = None
hook._hook_lineage_collector = hook.HookLineageCollector()
yield get_hook_lineage_collector()
hook._hook_lineage_collector = None
@pytest.mark.parametrize(
("uri", "dataset"),
(
(
"s3://bucket1/dir1/file1",
OpenLineageDataset(namespace="s3://bucket1", name="dir1/file1"),
),
(
"gs://bucket2/dir2/file2",
OpenLineageDataset(namespace="gs://bucket2", name="dir2/file2"),
),
(
"gcs://bucket3/dir3/file3",
OpenLineageDataset(namespace="gs://bucket3", name="dir3/file3"),
),
(
"hdfs://namenodehost:8020/file1",
OpenLineageDataset(namespace="hdfs://namenodehost:8020", name="file1"),
),
(
"hdfs://namenodehost/file2",
OpenLineageDataset(namespace="hdfs://namenodehost", name="file2"),
),
(
"file://localhost/etc/fstab",
OpenLineageDataset(namespace="file://localhost", name="etc/fstab"),
),
(
"file:///etc/fstab",
OpenLineageDataset(namespace="file://", name="etc/fstab"),
),
("https://test.com", OpenLineageDataset(namespace="https", name="test.com")),
(
"https://test.com?param1=test1¶m2=test2",
OpenLineageDataset(namespace="https", name="test.com"),
),
("file:test.csv", None),
("not_an_url", None),
),
)
def test_convert_to_ol_dataset_from_object_storage_uri(uri, dataset):
result = ExtractorManager.convert_to_ol_dataset_from_object_storage_uri(uri)
assert result == dataset
@pytest.mark.parametrize(
("obj", "dataset"),
(
(
OpenLineageDataset(namespace="n1", name="f1"),
OpenLineageDataset(namespace="n1", name="f1"),
),
(
File(url="s3://bucket1/dir1/file1"),
OpenLineageDataset(namespace="s3://bucket1", name="dir1/file1"),
),
(
File(url="gs://bucket2/dir2/file2"),
OpenLineageDataset(namespace="gs://bucket2", name="dir2/file2"),
),
(
File(url="gcs://bucket3/dir3/file3"),
OpenLineageDataset(namespace="gs://bucket3", name="dir3/file3"),
),
(
File(url="hdfs://namenodehost:8020/file1"),
OpenLineageDataset(namespace="hdfs://namenodehost:8020", name="file1"),
),
(
File(url="hdfs://namenodehost/file2"),
OpenLineageDataset(namespace="hdfs://namenodehost", name="file2"),
),
(
File(url="file://localhost/etc/fstab"),
OpenLineageDataset(namespace="file://localhost", name="etc/fstab"),
),
(
File(url="file:///etc/fstab"),
OpenLineageDataset(namespace="file://", name="etc/fstab"),
),
(
File(url="https://test.com"),
OpenLineageDataset(namespace="https", name="test.com"),
),
(
Table(cluster="c1", database="d1", name="t1"),
OpenLineageDataset(namespace="c1", name="d1.t1"),
),
("gs://bucket2/dir2/file2", None),
("not_an_url", None),
),
)
def test_convert_to_ol_dataset(obj, dataset):
result = ExtractorManager.convert_to_ol_dataset(obj)
assert result == dataset
def test_convert_to_ol_dataset_from_table_with_columns_and_owners():
table = Table(
cluster="c1",
database="d1",
name="t1",
columns=[
Column(name="col1", description="desc1", data_type="type1"),
Column(name="col2", description="desc2", data_type="type2"),
],
owners=[
User(email="[email protected]", first_name="Mike", last_name="Smith"),
User(email="[email protected]", first_name="Theo"),
User(email="[email protected]", last_name="Smith"),
User(email="[email protected]"),
],
description="test description",
)
expected_facets = {
"schema": schema_dataset.SchemaDatasetFacet(
fields=[
schema_dataset.SchemaDatasetFacetFields(
name="col1",
type="type1",
description="desc1",
),
schema_dataset.SchemaDatasetFacetFields(
name="col2",
type="type2",
description="desc2",
),
]
),
"ownership": ownership_dataset.OwnershipDatasetFacet(
owners=[
ownership_dataset.Owner(name="user:Mike Smith <[email protected]>", type=""),
ownership_dataset.Owner(name="user:Theo <[email protected]>", type=""),
ownership_dataset.Owner(name="user:Smith <[email protected]>", type=""),
ownership_dataset.Owner(name="user:<[email protected]>", type=""),
]
),
"documentation": documentation_dataset.DocumentationDatasetFacet(description="test description"),
}
result = ExtractorManager.convert_to_ol_dataset_from_table(table)
assert result.namespace == "c1"
assert result.name == "d1.t1"
assert result.facets == expected_facets
def test_convert_to_ol_dataset_table():
table = Table(
cluster="c1",
database="d1",
name="t1",
columns=[
Column(name="col1", description="desc1", data_type="type1"),
Column(name="col2", description="desc2", data_type="type2"),
],
owners=[
User(email="[email protected]", first_name="Mike", last_name="Smith"),
User(email="[email protected]", first_name="Theo"),
User(email="[email protected]", last_name="Smith"),
User(email="[email protected]"),
],
)
expected_facets = {
"schema": schema_dataset.SchemaDatasetFacet(
fields=[
schema_dataset.SchemaDatasetFacetFields(
name="col1",
type="type1",
description="desc1",
),
schema_dataset.SchemaDatasetFacetFields(
name="col2",
type="type2",
description="desc2",
),
]
),
"ownership": ownership_dataset.OwnershipDatasetFacet(
owners=[
ownership_dataset.Owner(name="user:Mike Smith <[email protected]>", type=""),
ownership_dataset.Owner(name="user:Theo <[email protected]>", type=""),
ownership_dataset.Owner(name="user:Smith <[email protected]>", type=""),
ownership_dataset.Owner(name="user:<[email protected]>", type=""),
]
),
}
result = ExtractorManager.convert_to_ol_dataset(table)
assert result.namespace == "c1"
assert result.name == "d1.t1"
assert result.facets == expected_facets
@pytest.mark.skipif(not AIRFLOW_V_2_10_PLUS, reason="Hook lineage works in Airflow >= 2.10.0")
def test_extractor_manager_uses_hook_level_lineage(hook_lineage_collector):
dagrun = MagicMock()
task = MagicMock()
del task.get_openlineage_facets_on_start
del task.get_openlineage_facets_on_complete
ti = MagicMock()
hook_lineage_collector.add_input_asset(None, uri="s3://bucket/input_key")
hook_lineage_collector.add_output_asset(None, uri="s3://bucket/output_key")
extractor_manager = ExtractorManager()
metadata = extractor_manager.extract_metadata(dagrun=dagrun, task=task, complete=True, task_instance=ti)
assert metadata.inputs == [OpenLineageDataset(namespace="s3://bucket", name="input_key")]
assert metadata.outputs == [OpenLineageDataset(namespace="s3://bucket", name="output_key")]
@pytest.mark.skipif(not AIRFLOW_V_2_10_PLUS, reason="Hook lineage works in Airflow >= 2.10.0")
def test_extractor_manager_does_not_use_hook_level_lineage_when_operator(
hook_lineage_collector,
):
class FakeSupportedOperator(BaseOperator):
def execute(self, context: Context) -> Any:
pass
def get_openlineage_facets_on_start(self):
return OperatorLineage(
inputs=[OpenLineageDataset(namespace="s3://bucket", name="proper_input_key")]
)
dagrun = MagicMock()
task = FakeSupportedOperator(task_id="test_task_extractor")
ti = MagicMock()
hook_lineage_collector.add_input_asset(None, uri="s3://bucket/input_key")
extractor_manager = ExtractorManager()
metadata = extractor_manager.extract_metadata(dagrun=dagrun, task=task, complete=True, task_instance=ti)
# s3://bucket/input_key not here - use data from operator
assert metadata.inputs == [OpenLineageDataset(namespace="s3://bucket", name="proper_input_key")]
assert metadata.outputs == []
@pytest.mark.db_test
@pytest.mark.skipif(not AIRFLOW_V_2_10_PLUS, reason="Hook lineage works in Airflow >= 2.10.0")
def test_extractor_manager_gets_data_from_pythonoperator(session, dag_maker, hook_lineage_collector):
path = None
with tempfile.NamedTemporaryFile() as f:
path = f.name
with dag_maker():
def use_read():
storage_path = ObjectStoragePath(path)
with storage_path.open("w") as out:
out.write("test")
task = PythonOperator(task_id="test_task_extractor_pythonoperator", python_callable=use_read)
dr = dag_maker.create_dagrun()
ti = TaskInstance(task=task, run_id=dr.run_id)
ti.refresh_from_db()
ti.state = State.QUEUED
session.merge(ti)
session.commit()
ti.run()
datasets = hook_lineage_collector.collected_assets
assert len(datasets.outputs) == 1
assert datasets.outputs[0].asset == Asset(uri=path)