forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
models.py
1470 lines (1230 loc) · 52.2 KB
/
models.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
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# -*- coding: utf-8 -*-
#
# Licensed 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 absolute_import
from __future__ import division
from __future__ import print_function
from __future__ import unicode_literals
import datetime
import logging
import os
import unittest
import time
from airflow import models, settings, AirflowException
from airflow.exceptions import AirflowSkipException
from airflow.jobs import BackfillJob
from airflow.models import DAG, TaskInstance as TI
from airflow.models import State as ST
from airflow.models import DagModel, DagStat
from airflow.models import clear_task_instances
from airflow.models import XCom
from airflow.operators.dummy_operator import DummyOperator
from airflow.operators.bash_operator import BashOperator
from airflow.operators.python_operator import PythonOperator
from airflow.operators.python_operator import ShortCircuitOperator
from airflow.ti_deps.deps.trigger_rule_dep import TriggerRuleDep
from airflow.utils.state import State
from airflow.utils.trigger_rule import TriggerRule
from mock import patch
from parameterized import parameterized
DEFAULT_DATE = datetime.datetime(2016, 1, 1)
TEST_DAGS_FOLDER = os.path.join(
os.path.dirname(os.path.realpath(__file__)), 'dags')
class DagTest(unittest.TestCase):
def test_parms_not_passed_is_empty_dict(self):
"""
Test that when 'params' is _not_ passed to a new Dag, that the params
attribute is set to an empty dictionary.
"""
dag = models.DAG('test-dag')
self.assertEqual(dict, type(dag.params))
self.assertEqual(0, len(dag.params))
def test_params_passed_and_params_in_default_args_no_override(self):
"""
Test that when 'params' exists as a key passed to the default_args dict
in addition to params being passed explicitly as an argument to the
dag, that the 'params' key of the default_args dict is merged with the
dict of the params argument.
"""
params1 = {'parameter1': 1}
params2 = {'parameter2': 2}
dag = models.DAG('test-dag',
default_args={'params': params1},
params=params2)
params_combined = params1.copy()
params_combined.update(params2)
self.assertEqual(params_combined, dag.params)
def test_dag_as_context_manager(self):
"""
Test DAG as a context manager.
When used as a context manager, Operators are automatically added to
the DAG (unless they specifiy a different DAG)
"""
dag = DAG(
'dag',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
dag2 = DAG(
'dag2',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner2'})
with dag:
op1 = DummyOperator(task_id='op1')
op2 = DummyOperator(task_id='op2', dag=dag2)
self.assertIs(op1.dag, dag)
self.assertEqual(op1.owner, 'owner1')
self.assertIs(op2.dag, dag2)
self.assertEqual(op2.owner, 'owner2')
with dag2:
op3 = DummyOperator(task_id='op3')
self.assertIs(op3.dag, dag2)
self.assertEqual(op3.owner, 'owner2')
with dag:
with dag2:
op4 = DummyOperator(task_id='op4')
op5 = DummyOperator(task_id='op5')
self.assertIs(op4.dag, dag2)
self.assertIs(op5.dag, dag)
self.assertEqual(op4.owner, 'owner2')
self.assertEqual(op5.owner, 'owner1')
with DAG('creating_dag_in_cm', start_date=DEFAULT_DATE) as dag:
DummyOperator(task_id='op6')
self.assertEqual(dag.dag_id, 'creating_dag_in_cm')
self.assertEqual(dag.tasks[0].task_id, 'op6')
def test_dag_topological_sort(self):
dag = DAG(
'dag',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
# A -> B
# A -> C -> D
# ordered: B, D, C, A or D, B, C, A or D, C, B, A
with dag:
op1 = DummyOperator(task_id='A')
op2 = DummyOperator(task_id='B')
op3 = DummyOperator(task_id='C')
op4 = DummyOperator(task_id='D')
op1.set_upstream([op2, op3])
op3.set_upstream(op4)
topological_list = dag.topological_sort()
logging.info(topological_list)
tasks = [op2, op3, op4]
self.assertTrue(topological_list[0] in tasks)
tasks.remove(topological_list[0])
self.assertTrue(topological_list[1] in tasks)
tasks.remove(topological_list[1])
self.assertTrue(topological_list[2] in tasks)
tasks.remove(topological_list[2])
self.assertTrue(topological_list[3] == op1)
dag = DAG(
'dag',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
# C -> (A u B) -> D
# C -> E
# ordered: E | D, A | B, C
with dag:
op1 = DummyOperator(task_id='A')
op2 = DummyOperator(task_id='B')
op3 = DummyOperator(task_id='C')
op4 = DummyOperator(task_id='D')
op5 = DummyOperator(task_id='E')
op1.set_downstream(op3)
op2.set_downstream(op3)
op1.set_upstream(op4)
op2.set_upstream(op4)
op5.set_downstream(op3)
topological_list = dag.topological_sort()
logging.info(topological_list)
set1 = [op4, op5]
self.assertTrue(topological_list[0] in set1)
set1.remove(topological_list[0])
set2 = [op1, op2]
set2.extend(set1)
self.assertTrue(topological_list[1] in set2)
set2.remove(topological_list[1])
self.assertTrue(topological_list[2] in set2)
set2.remove(topological_list[2])
self.assertTrue(topological_list[3] in set2)
self.assertTrue(topological_list[4] == op3)
dag = DAG(
'dag',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
self.assertEquals(tuple(), dag.topological_sort())
def test_get_num_task_instances(self):
test_dag_id = 'test_get_num_task_instances_dag'
test_task_id = 'task_1'
test_dag = DAG(dag_id=test_dag_id, start_date=DEFAULT_DATE)
test_task = DummyOperator(task_id=test_task_id, dag=test_dag)
ti1 = TI(task=test_task, execution_date=DEFAULT_DATE)
ti1.state = None
ti2 = TI(task=test_task, execution_date=DEFAULT_DATE + datetime.timedelta(days=1))
ti2.state = State.RUNNING
ti3 = TI(task=test_task, execution_date=DEFAULT_DATE + datetime.timedelta(days=2))
ti3.state = State.QUEUED
ti4 = TI(task=test_task, execution_date=DEFAULT_DATE + datetime.timedelta(days=3))
ti4.state = State.RUNNING
session = settings.Session()
session.merge(ti1)
session.merge(ti2)
session.merge(ti3)
session.merge(ti4)
session.commit()
self.assertEqual(0, DAG.get_num_task_instances(test_dag_id, ['fakename'],
session=session))
self.assertEqual(4, DAG.get_num_task_instances(test_dag_id, [test_task_id],
session=session))
self.assertEqual(4, DAG.get_num_task_instances(test_dag_id,
['fakename', test_task_id], session=session))
self.assertEqual(1, DAG.get_num_task_instances(test_dag_id, [test_task_id],
states=[None], session=session))
self.assertEqual(2, DAG.get_num_task_instances(test_dag_id, [test_task_id],
states=[State.RUNNING], session=session))
self.assertEqual(3, DAG.get_num_task_instances(test_dag_id, [test_task_id],
states=[None, State.RUNNING], session=session))
self.assertEqual(4, DAG.get_num_task_instances(test_dag_id, [test_task_id],
states=[None, State.QUEUED, State.RUNNING], session=session))
session.close()
def test_render_template_field(self):
"""Tests if render_template from a field works"""
dag = DAG('test-dag',
start_date=DEFAULT_DATE)
with dag:
task = DummyOperator(task_id='op1')
result = task.render_template('', '{{ foo }}', dict(foo='bar'))
self.assertEqual(result, 'bar')
def test_render_template_field_macro(self):
""" Tests if render_template from a field works,
if a custom filter was defined"""
dag = DAG('test-dag',
start_date=DEFAULT_DATE,
user_defined_macros = dict(foo='bar'))
with dag:
task = DummyOperator(task_id='op1')
result = task.render_template('', '{{ foo }}', dict())
self.assertEqual(result, 'bar')
def test_user_defined_filters(self):
def jinja_udf(name):
return 'Hello %s' %name
dag = models.DAG('test-dag',
start_date=DEFAULT_DATE,
user_defined_filters=dict(hello=jinja_udf))
jinja_env = dag.get_template_env()
self.assertIn('hello', jinja_env.filters)
self.assertEqual(jinja_env.filters['hello'], jinja_udf)
def test_render_template_field_filter(self):
""" Tests if render_template from a field works,
if a custom filter was defined"""
def jinja_udf(name):
return 'Hello %s' %name
dag = DAG('test-dag',
start_date=DEFAULT_DATE,
user_defined_filters = dict(hello=jinja_udf))
with dag:
task = DummyOperator(task_id='op1')
result = task.render_template('', "{{ 'world' | hello}}", dict())
self.assertEqual(result, 'Hello world')
class DagStatTest(unittest.TestCase):
def test_dagstats_crud(self):
DagStat.create(dag_id='test_dagstats_crud')
session = settings.Session()
qry = session.query(DagStat).filter(DagStat.dag_id == 'test_dagstats_crud')
self.assertEqual(len(qry.all()), len(State.dag_states))
DagStat.set_dirty(dag_id='test_dagstats_crud')
res = qry.all()
for stat in res:
self.assertTrue(stat.dirty)
# create missing
DagStat.set_dirty(dag_id='test_dagstats_crud_2')
qry2 = session.query(DagStat).filter(DagStat.dag_id == 'test_dagstats_crud_2')
self.assertEqual(len(qry2.all()), len(State.dag_states))
dag = DAG(
'test_dagstats_crud',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
with dag:
op1 = DummyOperator(task_id='A')
now = datetime.datetime.now()
dr = dag.create_dagrun(
run_id='manual__' + now.isoformat(),
execution_date=now,
start_date=now,
state=State.FAILED,
external_trigger=False,
)
DagStat.update(dag_ids=['test_dagstats_crud'])
res = qry.all()
for stat in res:
if stat.state == State.FAILED:
self.assertEqual(stat.count, 1)
else:
self.assertEqual(stat.count, 0)
DagStat.update()
res = qry2.all()
for stat in res:
self.assertFalse(stat.dirty)
class DagRunTest(unittest.TestCase):
def create_dag_run(self, dag, state=State.RUNNING, task_states=None, execution_date=None):
now = datetime.datetime.now()
if execution_date is None:
execution_date = now
dag_run = dag.create_dagrun(
run_id='manual__' + now.isoformat(),
execution_date=execution_date,
start_date=now,
state=state,
external_trigger=False,
)
if task_states is not None:
session = settings.Session()
for task_id, state in task_states.items():
ti = dag_run.get_task_instance(task_id)
ti.set_state(state, session)
session.close()
return dag_run
def test_id_for_date(self):
run_id = models.DagRun.id_for_date(
datetime.datetime(2015, 1, 2, 3, 4, 5, 6, None))
self.assertEqual(
'scheduled__2015-01-02T03:04:05', run_id,
'Generated run_id did not match expectations: {0}'.format(run_id))
def test_dagrun_find(self):
session = settings.Session()
now = datetime.datetime.now()
dag_id1 = "test_dagrun_find_externally_triggered"
dag_run = models.DagRun(
dag_id=dag_id1,
run_id='manual__' + now.isoformat(),
execution_date=now,
start_date=now,
state=State.RUNNING,
external_trigger=True,
)
session.add(dag_run)
dag_id2 = "test_dagrun_find_not_externally_triggered"
dag_run = models.DagRun(
dag_id=dag_id2,
run_id='manual__' + now.isoformat(),
execution_date=now,
start_date=now,
state=State.RUNNING,
external_trigger=False,
)
session.add(dag_run)
session.commit()
self.assertEqual(1, len(models.DagRun.find(dag_id=dag_id1, external_trigger=True)))
self.assertEqual(0, len(models.DagRun.find(dag_id=dag_id1, external_trigger=False)))
self.assertEqual(0, len(models.DagRun.find(dag_id=dag_id2, external_trigger=True)))
self.assertEqual(1, len(models.DagRun.find(dag_id=dag_id2, external_trigger=False)))
def test_dagrun_success_when_all_skipped(self):
"""
Tests that a DAG run succeeds when all tasks are skipped
"""
dag = DAG(
dag_id='test_dagrun_success_when_all_skipped',
start_date=datetime.datetime(2017, 1, 1)
)
dag_task1 = ShortCircuitOperator(
task_id='test_short_circuit_false',
dag=dag,
python_callable=lambda: False)
dag_task2 = DummyOperator(
task_id='test_state_skipped1',
dag=dag)
dag_task3 = DummyOperator(
task_id='test_state_skipped2',
dag=dag)
dag_task1.set_downstream(dag_task2)
dag_task2.set_downstream(dag_task3)
initial_task_states = {
'test_short_circuit_false': State.SUCCESS,
'test_state_skipped1': State.SKIPPED,
'test_state_skipped2': State.SKIPPED,
}
dag_run = self.create_dag_run(dag=dag,
state=State.RUNNING,
task_states=initial_task_states)
updated_dag_state = dag_run.update_state()
self.assertEqual(State.SUCCESS, updated_dag_state)
def test_dagrun_success_conditions(self):
session = settings.Session()
dag = DAG(
'test_dagrun_success_conditions',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
# A -> B
# A -> C -> D
# ordered: B, D, C, A or D, B, C, A or D, C, B, A
with dag:
op1 = DummyOperator(task_id='A')
op2 = DummyOperator(task_id='B')
op3 = DummyOperator(task_id='C')
op4 = DummyOperator(task_id='D')
op1.set_upstream([op2, op3])
op3.set_upstream(op4)
dag.clear()
now = datetime.datetime.now()
dr = dag.create_dagrun(run_id='test_dagrun_success_conditions',
state=State.RUNNING,
execution_date=now,
start_date=now)
# op1 = root
ti_op1 = dr.get_task_instance(task_id=op1.task_id)
ti_op1.set_state(state=State.SUCCESS, session=session)
ti_op2 = dr.get_task_instance(task_id=op2.task_id)
ti_op3 = dr.get_task_instance(task_id=op3.task_id)
ti_op4 = dr.get_task_instance(task_id=op4.task_id)
# root is successful, but unfinished tasks
state = dr.update_state()
self.assertEqual(State.RUNNING, state)
# one has failed, but root is successful
ti_op2.set_state(state=State.FAILED, session=session)
ti_op3.set_state(state=State.SUCCESS, session=session)
ti_op4.set_state(state=State.SUCCESS, session=session)
state = dr.update_state()
self.assertEqual(State.SUCCESS, state)
def test_dagrun_deadlock(self):
session = settings.Session()
dag = DAG(
'text_dagrun_deadlock',
start_date=DEFAULT_DATE,
default_args={'owner': 'owner1'})
with dag:
op1 = DummyOperator(task_id='A')
op2 = DummyOperator(task_id='B')
op2.trigger_rule = TriggerRule.ONE_FAILED
op2.set_upstream(op1)
dag.clear()
now = datetime.datetime.now()
dr = dag.create_dagrun(run_id='test_dagrun_deadlock',
state=State.RUNNING,
execution_date=now,
start_date=now)
ti_op1 = dr.get_task_instance(task_id=op1.task_id)
ti_op1.set_state(state=State.SUCCESS, session=session)
ti_op2 = dr.get_task_instance(task_id=op2.task_id)
ti_op2.set_state(state=State.NONE, session=session)
dr.update_state()
self.assertEqual(dr.state, State.RUNNING)
ti_op2.set_state(state=State.NONE, session=session)
op2.trigger_rule = 'invalid'
dr.update_state()
self.assertEqual(dr.state, State.FAILED)
def test_get_task_instance_on_empty_dagrun(self):
"""
Make sure that a proper value is returned when a dagrun has no task instances
"""
dag = DAG(
dag_id='test_get_task_instance_on_empty_dagrun',
start_date=datetime.datetime(2017, 1, 1)
)
dag_task1 = ShortCircuitOperator(
task_id='test_short_circuit_false',
dag=dag,
python_callable=lambda: False)
session = settings.Session()
now = datetime.datetime.now()
# Don't use create_dagrun since it will create the task instances too which we
# don't want
dag_run = models.DagRun(
dag_id=dag.dag_id,
run_id='manual__' + now.isoformat(),
execution_date=now,
start_date=now,
state=State.RUNNING,
external_trigger=False,
)
session.add(dag_run)
session.commit()
ti = dag_run.get_task_instance('test_short_circuit_false')
self.assertEqual(None, ti)
def test_get_latest_runs(self):
session = settings.Session()
dag = DAG(
dag_id='test_latest_runs_1',
start_date=DEFAULT_DATE)
dag_1_run_1 = self.create_dag_run(dag,
execution_date=datetime.datetime(2015, 1, 1))
dag_1_run_2 = self.create_dag_run(dag,
execution_date=datetime.datetime(2015, 1, 2))
dagruns = models.DagRun.get_latest_runs(session)
session.close()
for dagrun in dagruns:
if dagrun.dag_id == 'test_latest_runs_1':
self.assertEqual(dagrun.execution_date, datetime.datetime(2015, 1, 2))
def test_is_backfill(self):
dag = DAG(dag_id='test_is_backfill', start_date=DEFAULT_DATE)
dagrun = self.create_dag_run(dag, execution_date=DEFAULT_DATE)
dagrun.run_id = BackfillJob.ID_PREFIX + '_sfddsffds'
dagrun2 = self.create_dag_run(dag, execution_date=DEFAULT_DATE + datetime.timedelta(days=1))
self.assertTrue(dagrun.is_backfill)
self.assertFalse(dagrun2.is_backfill)
class DagBagTest(unittest.TestCase):
def test_get_existing_dag(self):
"""
test that were're able to parse some example DAGs and retrieve them
"""
dagbag = models.DagBag(include_examples=True)
some_expected_dag_ids = ["example_bash_operator",
"example_branch_operator"]
for dag_id in some_expected_dag_ids:
dag = dagbag.get_dag(dag_id)
self.assertIsNotNone(dag)
self.assertEqual(dag_id, dag.dag_id)
self.assertGreaterEqual(dagbag.size(), 7)
def test_get_non_existing_dag(self):
"""
test that retrieving a non existing dag id returns None without crashing
"""
dagbag = models.DagBag(include_examples=True)
non_existing_dag_id = "non_existing_dag_id"
self.assertIsNone(dagbag.get_dag(non_existing_dag_id))
def test_process_file_that_contains_multi_bytes_char(self):
"""
test that we're able to parse file that contains multi-byte char
"""
from tempfile import NamedTemporaryFile
f = NamedTemporaryFile()
f.write('\u3042'.encode('utf8')) # write multi-byte char (hiragana)
f.flush()
dagbag = models.DagBag(include_examples=True)
self.assertEqual([], dagbag.process_file(f.name))
def test_zip(self):
"""
test the loading of a DAG within a zip file that includes dependencies
"""
dagbag = models.DagBag()
dagbag.process_file(os.path.join(TEST_DAGS_FOLDER, "test_zip.zip"))
self.assertTrue(dagbag.get_dag("test_zip_dag"))
@patch.object(DagModel,'get_current')
def test_get_dag_without_refresh(self, mock_dagmodel):
"""
Test that, once a DAG is loaded, it doesn't get refreshed again if it
hasn't been expired.
"""
dag_id = 'example_bash_operator'
mock_dagmodel.return_value = DagModel()
mock_dagmodel.return_value.last_expired = None
mock_dagmodel.return_value.fileloc = 'foo'
class TestDagBag(models.DagBag):
process_file_calls = 0
def process_file(self, filepath, only_if_updated=True, safe_mode=True):
if 'example_bash_operator.py' == os.path.basename(filepath):
TestDagBag.process_file_calls += 1
super(TestDagBag, self).process_file(filepath, only_if_updated, safe_mode)
dagbag = TestDagBag(include_examples=True)
processed_files = dagbag.process_file_calls
# Should not call process_file agani, since it's already loaded during init.
self.assertEqual(1, dagbag.process_file_calls)
self.assertIsNotNone(dagbag.get_dag(dag_id))
self.assertEqual(1, dagbag.process_file_calls)
def test_get_dag_fileloc(self):
"""
Test that fileloc is correctly set when we load example DAGs,
specifically SubDAGs.
"""
dagbag = models.DagBag(include_examples=True)
expected = {
'example_bash_operator': 'example_bash_operator.py',
'example_subdag_operator': 'example_subdag_operator.py',
'example_subdag_operator.section-1': 'subdags/subdag.py'
}
for dag_id, path in expected.items():
dag = dagbag.get_dag(dag_id)
self.assertTrue(
dag.fileloc.endswith('airflow/example_dags/' + path))
class TaskInstanceTest(unittest.TestCase):
def test_set_task_dates(self):
"""
Test that tasks properly take start/end dates from DAGs
"""
dag = DAG('dag', start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=10))
op1 = DummyOperator(task_id='op_1', owner='test')
self.assertTrue(op1.start_date is None and op1.end_date is None)
# dag should assign its dates to op1 because op1 has no dates
dag.add_task(op1)
self.assertTrue(
op1.start_date == dag.start_date and op1.end_date == dag.end_date)
op2 = DummyOperator(
task_id='op_2',
owner='test',
start_date=DEFAULT_DATE - datetime.timedelta(days=1),
end_date=DEFAULT_DATE + datetime.timedelta(days=11))
# dag should assign its dates to op2 because they are more restrictive
dag.add_task(op2)
self.assertTrue(
op2.start_date == dag.start_date and op2.end_date == dag.end_date)
op3 = DummyOperator(
task_id='op_3',
owner='test',
start_date=DEFAULT_DATE + datetime.timedelta(days=1),
end_date=DEFAULT_DATE + datetime.timedelta(days=9))
# op3 should keep its dates because they are more restrictive
dag.add_task(op3)
self.assertTrue(
op3.start_date == DEFAULT_DATE + datetime.timedelta(days=1))
self.assertTrue(
op3.end_date == DEFAULT_DATE + datetime.timedelta(days=9))
def test_set_dag(self):
"""
Test assigning Operators to Dags, including deferred assignment
"""
dag = DAG('dag', start_date=DEFAULT_DATE)
dag2 = DAG('dag2', start_date=DEFAULT_DATE)
op = DummyOperator(task_id='op_1', owner='test')
# no dag assigned
self.assertFalse(op.has_dag())
self.assertRaises(AirflowException, getattr, op, 'dag')
# no improper assignment
with self.assertRaises(TypeError):
op.dag = 1
op.dag = dag
# no reassignment
with self.assertRaises(AirflowException):
op.dag = dag2
# but assigning the same dag is ok
op.dag = dag
self.assertIs(op.dag, dag)
self.assertIn(op, dag.tasks)
def test_infer_dag(self):
dag = DAG('dag', start_date=DEFAULT_DATE)
dag2 = DAG('dag2', start_date=DEFAULT_DATE)
op1 = DummyOperator(task_id='test_op_1', owner='test')
op2 = DummyOperator(task_id='test_op_2', owner='test')
op3 = DummyOperator(task_id='test_op_3', owner='test', dag=dag)
op4 = DummyOperator(task_id='test_op_4', owner='test', dag=dag2)
# double check dags
self.assertEqual(
[i.has_dag() for i in [op1, op2, op3, op4]],
[False, False, True, True])
# can't combine operators with no dags
self.assertRaises(AirflowException, op1.set_downstream, op2)
# op2 should infer dag from op1
op1.dag = dag
op1.set_downstream(op2)
self.assertIs(op2.dag, dag)
# can't assign across multiple DAGs
self.assertRaises(AirflowException, op1.set_downstream, op4)
self.assertRaises(AirflowException, op1.set_downstream, [op3, op4])
def test_bitshift_compose_operators(self):
dag = DAG('dag', start_date=DEFAULT_DATE)
op1 = DummyOperator(task_id='test_op_1', owner='test')
op2 = DummyOperator(task_id='test_op_2', owner='test')
op3 = DummyOperator(task_id='test_op_3', owner='test')
op4 = DummyOperator(task_id='test_op_4', owner='test')
op5 = DummyOperator(task_id='test_op_5', owner='test')
# can't compose operators without dags
with self.assertRaises(AirflowException):
op1 >> op2
dag >> op1 >> op2 << op3
# make sure dag assignment carries through
# using __rrshift__
self.assertIs(op1.dag, dag)
self.assertIs(op2.dag, dag)
self.assertIs(op3.dag, dag)
# op2 should be downstream of both
self.assertIn(op2, op1.downstream_list)
self.assertIn(op2, op3.downstream_list)
# test dag assignment with __rlshift__
dag << op4
self.assertIs(op4.dag, dag)
# dag assignment with __rrshift__
dag >> op5
self.assertIs(op5.dag, dag)
@patch.object(DAG, 'concurrency_reached')
def test_requeue_over_concurrency(self, mock_concurrency_reached):
mock_concurrency_reached.return_value = True
dag = DAG(dag_id='test_requeue_over_concurrency', start_date=DEFAULT_DATE,
max_active_runs=1, concurrency=2)
task = DummyOperator(task_id='test_requeue_over_concurrency_op', dag=dag)
ti = TI(task=task, execution_date=datetime.datetime.now())
ti.run()
self.assertEqual(ti.state, models.State.NONE)
@patch.object(TI, 'pool_full')
def test_run_pooling_task(self, mock_pool_full):
"""
test that running task update task state as without running task.
(no dependency check in ti_deps anymore, so also -> SUCCESS)
"""
# Mock the pool out with a full pool because the pool doesn't actually exist
mock_pool_full.return_value = True
dag = models.DAG(dag_id='test_run_pooling_task')
task = DummyOperator(task_id='test_run_pooling_task_op', dag=dag,
pool='test_run_pooling_task_pool', owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
ti = TI(
task=task, execution_date=datetime.datetime.now())
ti.run()
self.assertEqual(ti.state, models.State.SUCCESS)
@patch.object(TI, 'pool_full')
def test_run_pooling_task_with_mark_success(self, mock_pool_full):
"""
test that running task with mark_success param update task state as SUCCESS
without running task.
"""
# Mock the pool out with a full pool because the pool doesn't actually exist
mock_pool_full.return_value = True
dag = models.DAG(dag_id='test_run_pooling_task_with_mark_success')
task = DummyOperator(
task_id='test_run_pooling_task_with_mark_success_op',
dag=dag,
pool='test_run_pooling_task_with_mark_success_pool',
owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
ti = TI(
task=task, execution_date=datetime.datetime.now())
ti.run(mark_success=True)
self.assertEqual(ti.state, models.State.SUCCESS)
def test_run_pooling_task_with_skip(self):
"""
test that running task which returns AirflowSkipOperator will end
up in a SKIPPED state.
"""
def raise_skip_exception():
raise AirflowSkipException
dag = models.DAG(dag_id='test_run_pooling_task_with_skip')
task = PythonOperator(
task_id='test_run_pooling_task_with_skip',
dag=dag,
python_callable=raise_skip_exception,
owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
ti = TI(
task=task, execution_date=datetime.datetime.now())
ti.run()
self.assertTrue(ti.state == models.State.SKIPPED)
def test_retry_delay(self):
"""
Test that retry delays are respected
"""
dag = models.DAG(dag_id='test_retry_handling')
task = BashOperator(
task_id='test_retry_handling_op',
bash_command='exit 1',
retries=1,
retry_delay=datetime.timedelta(seconds=3),
dag=dag,
owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
def run_with_error(ti):
try:
ti.run()
except AirflowException:
pass
ti = TI(
task=task, execution_date=datetime.datetime.now())
# first run -- up for retry
run_with_error(ti)
self.assertEqual(ti.state, State.UP_FOR_RETRY)
self.assertEqual(ti.try_number, 1)
# second run -- still up for retry because retry_delay hasn't expired
run_with_error(ti)
self.assertEqual(ti.state, State.UP_FOR_RETRY)
# third run -- failed
time.sleep(3)
run_with_error(ti)
self.assertEqual(ti.state, State.FAILED)
@patch.object(TI, 'pool_full')
def test_retry_handling(self, mock_pool_full):
"""
Test that task retries are handled properly
"""
# Mock the pool with a pool with slots open since the pool doesn't actually exist
mock_pool_full.return_value = False
dag = models.DAG(dag_id='test_retry_handling')
task = BashOperator(
task_id='test_retry_handling_op',
bash_command='exit 1',
retries=1,
retry_delay=datetime.timedelta(seconds=0),
dag=dag,
owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
def run_with_error(ti):
try:
ti.run()
except AirflowException:
pass
ti = TI(
task=task, execution_date=datetime.datetime.now())
# first run -- up for retry
run_with_error(ti)
self.assertEqual(ti.state, State.UP_FOR_RETRY)
self.assertEqual(ti.try_number, 1)
# second run -- fail
run_with_error(ti)
self.assertEqual(ti.state, State.FAILED)
self.assertEqual(ti.try_number, 2)
# Clear the TI state since you can't run a task with a FAILED state without
# clearing it first
dag.clear()
# third run -- up for retry
run_with_error(ti)
self.assertEqual(ti.state, State.UP_FOR_RETRY)
self.assertEqual(ti.try_number, 3)
# fourth run -- fail
run_with_error(ti)
self.assertEqual(ti.state, State.FAILED)
self.assertEqual(ti.try_number, 4)
def test_next_retry_datetime(self):
delay = datetime.timedelta(seconds=30)
max_delay = datetime.timedelta(minutes=60)
dag = models.DAG(dag_id='fail_dag')
task = BashOperator(
task_id='task_with_exp_backoff_and_max_delay',
bash_command='exit 1',
retries=3,
retry_delay=delay,
retry_exponential_backoff=True,
max_retry_delay=max_delay,
dag=dag,
owner='airflow',
start_date=datetime.datetime(2016, 2, 1, 0, 0, 0))
ti = TI(
task=task, execution_date=DEFAULT_DATE)
ti.end_date = datetime.datetime.now()
ti.try_number = 1
dt = ti.next_retry_datetime()
# between 30 * 2^0.5 and 30 * 2^1 (15 and 30)
self.assertEqual(dt, ti.end_date + datetime.timedelta(seconds=20.0))
ti.try_number = 4
dt = ti.next_retry_datetime()
# between 30 * 2^2 and 30 * 2^3 (120 and 240)
self.assertEqual(dt, ti.end_date + datetime.timedelta(seconds=181.0))
ti.try_number = 6
dt = ti.next_retry_datetime()
# between 30 * 2^4 and 30 * 2^5 (480 and 960)
self.assertEqual(dt, ti.end_date + datetime.timedelta(seconds=825.0))
ti.try_number = 9
dt = ti.next_retry_datetime()
self.assertEqual(dt, ti.end_date+max_delay)
ti.try_number = 50
dt = ti.next_retry_datetime()
self.assertEqual(dt, ti.end_date+max_delay)
def test_depends_on_past(self):