forked from twitter-forks/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathtest_dag_serialization.py
713 lines (617 loc) · 29.5 KB
/
test_dag_serialization.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
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
#
# 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.
"""Unit tests for stringified DAGs."""
import multiprocessing
import os
import unittest
from datetime import datetime, timedelta
from glob import glob
from unittest import mock
from dateutil.relativedelta import FR, relativedelta
from parameterized import parameterized
from airflow.hooks.base_hook import BaseHook
from airflow.models import DAG, Connection, DagBag, TaskInstance
from airflow.models.baseoperator import BaseOperator
from airflow.operators.bash import BashOperator
from airflow.operators.subdag_operator import SubDagOperator
from airflow.serialization.json_schema import load_dag_schema_dict
from airflow.serialization.serialized_objects import SerializedBaseOperator, SerializedDAG
from tests.test_utils.mock_operators import CustomOperator, CustomOpLink, GoogleLink
serialized_simple_dag_ground_truth = {
"__version": 1,
"dag": {
"default_args": {
"__type": "dict",
"__var": {
"depends_on_past": False,
"retries": 1,
"retry_delay": {
"__type": "timedelta",
"__var": 300.0
}
}
},
"start_date": 1564617600.0,
"is_paused_upon_creation": False,
"_dag_id": "simple_dag",
"fileloc": None,
"tasks": [
{
"task_id": "bash_task",
"owner": "airflow",
"retries": 1,
"retry_delay": 300.0,
"_downstream_task_ids": [],
"_inlets": [],
"_outlets": [],
"ui_color": "#f0ede4",
"ui_fgcolor": "#000",
"template_fields": ['bash_command', 'env'],
"bash_command": "echo {{ task.task_id }}",
"_task_type": "BashOperator",
"_task_module": "airflow.operators.bash",
"pool": "default_pool",
},
{
"task_id": "custom_task",
"retries": 1,
"retry_delay": 300.0,
"_downstream_task_ids": [],
"_inlets": [],
"_outlets": [],
"_operator_extra_links": [{"tests.test_utils.mock_operators.CustomOpLink": {}}],
"ui_color": "#fff",
"ui_fgcolor": "#000",
"template_fields": ['bash_command'],
"_task_type": "CustomOperator",
"_task_module": "tests.test_utils.mock_operators",
"pool": "default_pool",
},
],
"timezone": "UTC",
"_access_control": {
"__type": "dict",
"__var": {
"test_role": {
"__type": "set",
"__var": [
"can_dag_read",
"can_dag_edit"
]
}
}
}
},
}
ROOT_FOLDER = os.path.realpath(
os.path.join(os.path.dirname(os.path.realpath(__file__)), os.pardir, os.pardir)
)
def make_example_dags(module_path):
"""Loads DAGs from a module for test."""
dagbag = DagBag(module_path)
return dagbag.dags
def make_simple_dag():
"""Make very simple DAG to verify serialization result."""
with DAG(
dag_id='simple_dag',
default_args={
"retries": 1,
"retry_delay": timedelta(minutes=5),
"depends_on_past": False,
},
start_date=datetime(2019, 8, 1),
is_paused_upon_creation=False,
access_control={
"test_role": {"can_dag_read", "can_dag_edit"}
}
) as dag:
CustomOperator(task_id='custom_task')
BashOperator(task_id='bash_task', bash_command='echo {{ task.task_id }}', owner='airflow')
return {'simple_dag': dag}
def make_user_defined_macro_filter_dag():
""" Make DAGs with user defined macros and filters using locally defined methods.
For Webserver, we do not include ``user_defined_macros`` & ``user_defined_filters``.
The examples here test:
(1) functions can be successfully displayed on UI;
(2) templates with function macros have been rendered before serialization.
"""
def compute_next_execution_date(dag, execution_date):
return dag.following_schedule(execution_date)
default_args = {
'start_date': datetime(2019, 7, 10)
}
dag = DAG(
'user_defined_macro_filter_dag',
default_args=default_args,
user_defined_macros={
'next_execution_date': compute_next_execution_date,
},
user_defined_filters={
'hello': lambda name: 'Hello %s' % name
},
catchup=False
)
BashOperator(
task_id='echo',
bash_command='echo "{{ next_execution_date(dag, execution_date) }}"',
dag=dag,
)
return {dag.dag_id: dag}
def collect_dags():
"""Collects DAGs to test."""
dags = {}
dags.update(make_simple_dag())
dags.update(make_user_defined_macro_filter_dag())
patterns = [
"airflow/example_dags",
"airflow/providers/*/example_dags",
"airflow/providers/*/*/example_dags",
]
for pattern in patterns:
for directory in glob(f"{ROOT_FOLDER}/{pattern}"):
dags.update(make_example_dags(directory))
# Filter subdags as they are stored in same row in Serialized Dag table
dags = {dag_id: dag for dag_id, dag in dags.items() if not dag.is_subdag}
return dags
def serialize_subprocess(queue):
"""Validate pickle in a subprocess."""
dags = collect_dags()
for dag in dags.values():
queue.put(SerializedDAG.to_json(dag))
queue.put(None)
class TestStringifiedDAGs(unittest.TestCase):
"""Unit tests for stringified DAGs."""
def setUp(self):
super().setUp()
BaseHook.get_connection = mock.Mock(
return_value=Connection(
extra=('{'
'"project_id": "mock", '
'"location": "mock", '
'"instance": "mock", '
'"database_type": "postgres", '
'"use_proxy": "False", '
'"use_ssl": "False"'
'}')))
self.maxDiff = None # pylint: disable=invalid-name
def test_serialization(self):
"""Serialization and deserialization should work for every DAG and Operator."""
dags = collect_dags()
serialized_dags = {}
for _, v in dags.items():
dag = SerializedDAG.to_dict(v)
SerializedDAG.validate_schema(dag)
serialized_dags[v.dag_id] = dag
# Compares with the ground truth of JSON string.
self.validate_serialized_dag(
serialized_dags['simple_dag'],
serialized_simple_dag_ground_truth)
def validate_serialized_dag(self, json_dag, ground_truth_dag):
"""Verify serialized DAGs match the ground truth."""
self.assertTrue(
json_dag['dag']['fileloc'].split('/')[-1] == 'test_dag_serialization.py')
json_dag['dag']['fileloc'] = None
def sorted_serialized_dag(dag_dict: dict):
"""
Sorts the "tasks" list and "access_control" permissions in the
serialised dag python dictionary. This is needed as the order of
items should not matter but assertEqual would fail if the order of
items changes in the dag dictionary
"""
dag_dict["dag"]["tasks"] = sorted(dag_dict["dag"]["tasks"],
key=lambda x: sorted(x.keys()))
dag_dict["dag"]["_access_control"]["__var"]["test_role"]["__var"] = sorted(
dag_dict["dag"]["_access_control"]["__var"]["test_role"]["__var"]
)
return dag_dict
self.assertEqual(sorted_serialized_dag(ground_truth_dag),
sorted_serialized_dag(json_dag))
def test_deserialization(self):
"""A serialized DAG can be deserialized in another process."""
queue = multiprocessing.Queue()
proc = multiprocessing.Process(
target=serialize_subprocess, args=(queue,))
proc.daemon = True
proc.start()
stringified_dags = {}
while True:
v = queue.get()
if v is None:
break
dag = SerializedDAG.from_json(v)
self.assertTrue(isinstance(dag, DAG))
stringified_dags[dag.dag_id] = dag
dags = collect_dags()
self.assertTrue(set(stringified_dags.keys()) == set(dags.keys()))
# Verify deserialized DAGs.
for dag_id in stringified_dags:
self.validate_deserialized_dag(stringified_dags[dag_id], dags[dag_id])
example_skip_dag = stringified_dags['example_skip_dag']
skip_operator_1_task = example_skip_dag.task_dict['skip_operator_1']
self.validate_deserialized_task(
skip_operator_1_task, 'DummySkipOperator', '#e8b7e4', '#000')
# Verify that the DAG object has 'full_filepath' attribute
# and is equal to fileloc
self.assertTrue(hasattr(example_skip_dag, 'full_filepath'))
self.assertEqual(example_skip_dag.full_filepath, example_skip_dag.fileloc)
example_subdag_operator = stringified_dags['example_subdag_operator']
section_1_task = example_subdag_operator.task_dict['section-1']
self.validate_deserialized_task(
section_1_task,
SubDagOperator.__name__,
SubDagOperator.ui_color,
SubDagOperator.ui_fgcolor
)
def validate_deserialized_dag(self, serialized_dag, dag):
"""
Verify that all example DAGs work with DAG Serialization by
checking fields between Serialized Dags & non-Serialized Dags
"""
fields_to_check = [
"task_ids", "params", "fileloc", "max_active_runs", "concurrency",
"is_paused_upon_creation", "doc_md", "safe_dag_id", "is_subdag",
"catchup", "description", "start_date", "end_date", "parent_dag",
"template_searchpath", "_access_control", "dagrun_timeout"
]
# fields_to_check = dag.get_serialized_fields()
for field in fields_to_check:
self.assertEqual(getattr(serialized_dag, field), getattr(dag, field))
def validate_deserialized_task(self, task, task_type, ui_color, ui_fgcolor):
"""Verify non-airflow operators are casted to BaseOperator."""
self.assertTrue(isinstance(task, SerializedBaseOperator))
# Verify the original operator class is recorded for UI.
self.assertTrue(task.task_type == task_type)
self.assertTrue(task.ui_color == ui_color)
self.assertTrue(task.ui_fgcolor == ui_fgcolor)
# Check that for Deserialised task, task.subdag is None for all other Operators
# except for the SubDagOperator where task.subdag is an instance of DAG object
if task.task_type == "SubDagOperator":
self.assertIsNotNone(task.subdag)
self.assertTrue(isinstance(task.subdag, DAG))
else:
self.assertIsNone(task.subdag)
self.assertEqual({}, task.params)
self.assertEqual({}, task.executor_config)
@parameterized.expand([
(datetime(2019, 8, 1), None, datetime(2019, 8, 1)),
(datetime(2019, 8, 1), datetime(2019, 8, 2), datetime(2019, 8, 2)),
(datetime(2019, 8, 1), datetime(2019, 7, 30), datetime(2019, 8, 1)),
])
def test_deserialization_start_date(self,
dag_start_date,
task_start_date,
expected_task_start_date):
dag = DAG(dag_id='simple_dag', start_date=dag_start_date)
BaseOperator(task_id='simple_task', dag=dag, start_date=task_start_date)
serialized_dag = SerializedDAG.to_dict(dag)
if not task_start_date or dag_start_date >= task_start_date:
# If dag.start_date > task.start_date -> task.start_date=dag.start_date
# because of the logic in dag.add_task()
self.assertNotIn("start_date", serialized_dag["dag"]["tasks"][0])
else:
self.assertIn("start_date", serialized_dag["dag"]["tasks"][0])
dag = SerializedDAG.from_dict(serialized_dag)
simple_task = dag.task_dict["simple_task"]
self.assertEqual(simple_task.start_date, expected_task_start_date)
@parameterized.expand([
(datetime(2019, 8, 1), None, datetime(2019, 8, 1)),
(datetime(2019, 8, 1), datetime(2019, 8, 2), datetime(2019, 8, 1)),
(datetime(2019, 8, 1), datetime(2019, 7, 30), datetime(2019, 7, 30)),
])
def test_deserialization_end_date(self,
dag_end_date,
task_end_date,
expected_task_end_date):
dag = DAG(dag_id='simple_dag', start_date=datetime(2019, 8, 1),
end_date=dag_end_date)
BaseOperator(task_id='simple_task', dag=dag, end_date=task_end_date)
serialized_dag = SerializedDAG.to_dict(dag)
if not task_end_date or dag_end_date <= task_end_date:
# If dag.end_date < task.end_date -> task.end_date=dag.end_date
# because of the logic in dag.add_task()
self.assertNotIn("end_date", serialized_dag["dag"]["tasks"][0])
else:
self.assertIn("end_date", serialized_dag["dag"]["tasks"][0])
dag = SerializedDAG.from_dict(serialized_dag)
simple_task = dag.task_dict["simple_task"]
self.assertEqual(simple_task.end_date, expected_task_end_date)
@parameterized.expand([
(None, None, None),
("@weekly", "@weekly", "0 0 * * 0"),
("@once", "@once", None),
({"__type": "timedelta", "__var": 86400.0}, timedelta(days=1), timedelta(days=1)),
])
def test_deserialization_schedule_interval(
self, serialized_schedule_interval, expected_schedule_interval, expected_n_schedule_interval
):
serialized = {
"__version": 1,
"dag": {
"default_args": {"__type": "dict", "__var": {}},
"_dag_id": "simple_dag",
"fileloc": __file__,
"tasks": [],
"timezone": "UTC",
"schedule_interval": serialized_schedule_interval,
},
}
SerializedDAG.validate_schema(serialized)
dag = SerializedDAG.from_dict(serialized)
self.assertEqual(dag.schedule_interval, expected_schedule_interval)
self.assertEqual(dag.normalized_schedule_interval, expected_n_schedule_interval)
@parameterized.expand([
(relativedelta(days=-1), {"__type": "relativedelta", "__var": {"days": -1}}),
(relativedelta(month=1, days=-1), {"__type": "relativedelta", "__var": {"month": 1, "days": -1}}),
# Every friday
(relativedelta(weekday=FR), {"__type": "relativedelta", "__var": {"weekday": [4]}}),
# Every second friday
(relativedelta(weekday=FR(2)), {"__type": "relativedelta", "__var": {"weekday": [4, 2]}})
])
def test_roundtrip_relativedelta(self, val, expected):
serialized = SerializedDAG._serialize(val)
self.assertDictEqual(serialized, expected)
round_tripped = SerializedDAG._deserialize(serialized)
self.assertEqual(val, round_tripped)
@parameterized.expand([
(None, {}),
({"param_1": "value_1"}, {"param_1": "value_1"}),
])
def test_dag_params_roundtrip(self, val, expected_val):
"""
Test that params work both on Serialized DAGs & Tasks
"""
dag = DAG(dag_id='simple_dag', params=val)
BaseOperator(task_id='simple_task', dag=dag, start_date=datetime(2019, 8, 1))
serialized_dag = SerializedDAG.to_dict(dag)
if val:
self.assertIn("params", serialized_dag["dag"])
else:
self.assertNotIn("params", serialized_dag["dag"])
deserialized_dag = SerializedDAG.from_dict(serialized_dag)
deserialized_simple_task = deserialized_dag.task_dict["simple_task"]
self.assertEqual(expected_val, deserialized_dag.params)
self.assertEqual(expected_val, deserialized_simple_task.params)
@parameterized.expand([
(None, {}),
({"param_1": "value_1"}, {"param_1": "value_1"}),
])
def test_task_params_roundtrip(self, val, expected_val):
"""
Test that params work both on Serialized DAGs & Tasks
"""
dag = DAG(dag_id='simple_dag')
BaseOperator(task_id='simple_task', dag=dag, params=val,
start_date=datetime(2019, 8, 1))
serialized_dag = SerializedDAG.to_dict(dag)
if val:
self.assertIn("params", serialized_dag["dag"]["tasks"][0])
else:
self.assertNotIn("params", serialized_dag["dag"]["tasks"][0])
deserialized_dag = SerializedDAG.from_dict(serialized_dag)
deserialized_simple_task = deserialized_dag.task_dict["simple_task"]
self.assertEqual(expected_val, deserialized_simple_task.params)
def test_extra_serialized_field_and_operator_links(self):
"""
Assert extra field exists & OperatorLinks defined in Plugins and inbuilt Operator Links.
This tests also depends on GoogleLink() registered as a plugin
in tests/plugins/test_plugin.py
The function tests that if extra operator links are registered in plugin
in ``operator_extra_links`` and the same is also defined in
the Operator in ``BaseOperator.operator_extra_links``, it has the correct
extra link.
"""
test_date = datetime(2019, 8, 1)
dag = DAG(dag_id='simple_dag', start_date=test_date)
CustomOperator(task_id='simple_task', dag=dag, bash_command="true")
serialized_dag = SerializedDAG.to_dict(dag)
self.assertIn("bash_command", serialized_dag["dag"]["tasks"][0])
dag = SerializedDAG.from_dict(serialized_dag)
simple_task = dag.task_dict["simple_task"]
self.assertEqual(getattr(simple_task, "bash_command"), "true")
#########################################################
# Verify Operator Links work with Serialized Operator
#########################################################
# Check Serialized version of operator link only contains the inbuilt Op Link
self.assertEqual(
serialized_dag["dag"]["tasks"][0]["_operator_extra_links"],
[{'tests.test_utils.mock_operators.CustomOpLink': {}}]
)
# Test all the extra_links are set
self.assertCountEqual(simple_task.extra_links, ['Google Custom', 'airflow', 'github', 'google'])
ti = TaskInstance(task=simple_task, execution_date=test_date)
ti.xcom_push('search_query', "dummy_value_1")
# Test Deserialized inbuilt link
custom_inbuilt_link = simple_task.get_extra_links(test_date, CustomOpLink.name)
self.assertEqual('http://google.com/custom_base_link?search=dummy_value_1', custom_inbuilt_link)
# Test Deserialized link registered via Airflow Plugin
google_link_from_plugin = simple_task.get_extra_links(test_date, GoogleLink.name)
self.assertEqual("https://www.google.com", google_link_from_plugin)
def test_extra_serialized_field_and_multiple_operator_links(self):
"""
Assert extra field exists & OperatorLinks defined in Plugins and inbuilt Operator Links.
This tests also depends on GoogleLink() registered as a plugin
in tests/plugins/test_plugin.py
The function tests that if extra operator links are registered in plugin
in ``operator_extra_links`` and the same is also defined in
the Operator in ``BaseOperator.operator_extra_links``, it has the correct
extra link.
"""
test_date = datetime(2019, 8, 1)
dag = DAG(dag_id='simple_dag', start_date=test_date)
CustomOperator(task_id='simple_task', dag=dag, bash_command=["echo", "true"])
serialized_dag = SerializedDAG.to_dict(dag)
self.assertIn("bash_command", serialized_dag["dag"]["tasks"][0])
dag = SerializedDAG.from_dict(serialized_dag)
simple_task = dag.task_dict["simple_task"]
self.assertEqual(getattr(simple_task, "bash_command"), ["echo", "true"])
#########################################################
# Verify Operator Links work with Serialized Operator
#########################################################
# Check Serialized version of operator link only contains the inbuilt Op Link
self.assertEqual(
serialized_dag["dag"]["tasks"][0]["_operator_extra_links"],
[
{'tests.test_utils.mock_operators.CustomBaseIndexOpLink': {'index': 0}},
{'tests.test_utils.mock_operators.CustomBaseIndexOpLink': {'index': 1}},
]
)
# Test all the extra_links are set
self.assertCountEqual(simple_task.extra_links, [
'BigQuery Console #1', 'BigQuery Console #2', 'airflow', 'github', 'google'])
ti = TaskInstance(task=simple_task, execution_date=test_date)
ti.xcom_push('search_query', ["dummy_value_1", "dummy_value_2"])
# Test Deserialized inbuilt link #1
custom_inbuilt_link = simple_task.get_extra_links(test_date, "BigQuery Console #1")
self.assertEqual('https://console.cloud.google.com/bigquery?j=dummy_value_1', custom_inbuilt_link)
# Test Deserialized inbuilt link #2
custom_inbuilt_link = simple_task.get_extra_links(test_date, "BigQuery Console #2")
self.assertEqual('https://console.cloud.google.com/bigquery?j=dummy_value_2', custom_inbuilt_link)
# Test Deserialized link registered via Airflow Plugin
google_link_from_plugin = simple_task.get_extra_links(test_date, GoogleLink.name)
self.assertEqual("https://www.google.com", google_link_from_plugin)
class ClassWithCustomAttributes:
"""
Class for testing purpose: allows to create objects with custom attributes in one single statement.
"""
def __init__(self, **kwargs):
for key, value in kwargs.items():
setattr(self, key, value)
def __str__(self):
return "{}({})".format(self.__class__.__name__, str(self.__dict__))
def __repr__(self):
return self.__str__()
def __eq__(self, other):
return self.__dict__ == other.__dict__
def __ne__(self, other):
return not self.__eq__(other)
@parameterized.expand([
(None, None),
([], []),
({}, {}),
("{{ task.task_id }}", "{{ task.task_id }}"),
(["{{ task.task_id }}", "{{ task.task_id }}"]),
({"foo": "{{ task.task_id }}"}, {"foo": "{{ task.task_id }}"}),
({"foo": {"bar": "{{ task.task_id }}"}}, {"foo": {"bar": "{{ task.task_id }}"}}),
(
[{"foo1": {"bar": "{{ task.task_id }}"}}, {"foo2": {"bar": "{{ task.task_id }}"}}],
[{"foo1": {"bar": "{{ task.task_id }}"}}, {"foo2": {"bar": "{{ task.task_id }}"}}],
),
(
{"foo": {"bar": {"{{ task.task_id }}": ["sar"]}}},
{"foo": {"bar": {"{{ task.task_id }}": ["sar"]}}}),
(
ClassWithCustomAttributes(
att1="{{ task.task_id }}", att2="{{ task.task_id }}", template_fields=["att1"]),
"ClassWithCustomAttributes("
"{'att1': '{{ task.task_id }}', 'att2': '{{ task.task_id }}', 'template_fields': ['att1']})",
),
(
ClassWithCustomAttributes(nested1=ClassWithCustomAttributes(att1="{{ task.task_id }}",
att2="{{ task.task_id }}",
template_fields=["att1"]),
nested2=ClassWithCustomAttributes(att3="{{ task.task_id }}",
att4="{{ task.task_id }}",
template_fields=["att3"]),
template_fields=["nested1"]),
"ClassWithCustomAttributes("
"{'nested1': ClassWithCustomAttributes({'att1': '{{ task.task_id }}', "
"'att2': '{{ task.task_id }}', 'template_fields': ['att1']}), "
"'nested2': ClassWithCustomAttributes({'att3': '{{ task.task_id }}', "
"'att4': '{{ task.task_id }}', 'template_fields': ['att3']}), 'template_fields': ['nested1']})",
),
])
def test_templated_fields_exist_in_serialized_dag(self, templated_field, expected_field):
"""
Test that templated_fields exists for all Operators in Serialized DAG
Since we don't want to inflate arbitrary python objects (it poses a RCE/security risk etc.)
we want check that non-"basic" objects are turned in to strings after deserializing.
"""
dag = DAG("test_serialized_template_fields", start_date=datetime(2019, 8, 1))
with dag:
BashOperator(task_id="test", bash_command=templated_field)
serialized_dag = SerializedDAG.to_dict(dag)
deserialized_dag = SerializedDAG.from_dict(serialized_dag)
deserialized_test_task = deserialized_dag.task_dict["test"]
self.assertEqual(expected_field, getattr(deserialized_test_task, "bash_command"))
def test_dag_serialized_fields_with_schema(self):
"""
Additional Properties are disabled on DAGs. This test verifies that all the
keys in DAG.get_serialized_fields are listed in Schema definition.
"""
dag_schema: dict = load_dag_schema_dict()["definitions"]["dag"]["properties"]
# The parameters we add manually in Serialization needs to be ignored
ignored_keys: set = {"is_subdag", "tasks"}
dag_params: set = set(dag_schema.keys()) - ignored_keys
self.assertEqual(set(DAG.get_serialized_fields()), dag_params)
def test_no_new_fields_added_to_base_operator(self):
"""
This test verifies that there are no new fields added to BaseOperator. And reminds that
tests should be added for it.
"""
base_operator = BaseOperator(task_id="10")
fields = base_operator.__dict__
self.assertEqual({'_dag': None,
'_downstream_task_ids': set(),
'_inlets': [],
'_log': base_operator.log,
'_outlets': [],
'_upstream_task_ids': set(),
'depends_on_past': False,
'do_xcom_push': True,
'email': None,
'email_on_failure': True,
'email_on_retry': True,
'end_date': None,
'execution_timeout': None,
'executor_config': {},
'inlets': [],
'max_retry_delay': None,
'on_execute_callback': None,
'on_failure_callback': None,
'on_retry_callback': None,
'on_success_callback': None,
'outlets': [],
'owner': 'airflow',
'params': {},
'pool': 'default_pool',
'pool_slots': 1,
'priority_weight': 1,
'queue': 'default',
'resources': None,
'retries': 0,
'retry_delay': timedelta(0, 300),
'retry_exponential_backoff': False,
'run_as_user': None,
'sla': None,
'start_date': None,
'subdag': None,
'task_concurrency': None,
'task_id': '10',
'trigger_rule': 'all_success',
'wait_for_downstream': False,
'weight_rule': 'downstream'}, fields,
"""
!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
ACTION NEEDED! PLEASE READ THIS CAREFULLY AND CORRECT TESTS CAREFULLY
Some fields were added to the BaseOperator! Please add them to the list above and make sure that
you add support for DAG serialization - you should add the field to
`airflow/serialization/schema.json` - they should have correct type defined there.
Note that we do not support versioning yet so you should only add optional fields to BaseOperator.
!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
"""
)