-
Notifications
You must be signed in to change notification settings - Fork 14.3k
/
test_dagrun.py
2621 lines (2115 loc) · 89.3 KB
/
test_dagrun.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
#
# 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 datetime
from functools import reduce
from typing import Mapping
from unittest import mock
from unittest.mock import call
import pendulum
import pytest
from sqlalchemy.orm.session import Session
from airflow import settings
from airflow.callbacks.callback_requests import DagCallbackRequest
from airflow.decorators import setup, task, task_group, teardown
from airflow.exceptions import AirflowException
from airflow.models import (
DAG,
DagBag,
DagModel,
DagRun,
TaskInstance,
TaskInstance as TI,
clear_task_instances,
)
from airflow.models.baseoperator import BaseOperator
from airflow.models.dagrun import DagRunNote
from airflow.models.taskinstance import TaskInstanceNote
from airflow.models.taskmap import TaskMap
from airflow.models.taskreschedule import TaskReschedule
from airflow.operators.empty import EmptyOperator
from airflow.operators.python import ShortCircuitOperator
from airflow.serialization.serialized_objects import SerializedDAG
from airflow.stats import Stats
from airflow.utils import timezone
from airflow.utils.state import DagRunState, State, TaskInstanceState
from airflow.utils.trigger_rule import TriggerRule
from airflow.utils.types import DagRunType
from tests.models import DEFAULT_DATE as _DEFAULT_DATE
from tests.test_utils import db
from tests.test_utils.config import conf_vars
from tests.test_utils.mock_operators import MockOperator
DEFAULT_DATE = pendulum.instance(_DEFAULT_DATE)
class TestDagRun:
dagbag = DagBag(include_examples=True)
@staticmethod
def clean_db():
db.clear_db_runs()
db.clear_db_pools()
db.clear_db_dags()
db.clear_db_variables()
db.clear_db_datasets()
db.clear_db_xcom()
db.clear_db_task_fail()
def setup_class(self) -> None:
self.clean_db()
def teardown_method(self) -> None:
self.clean_db()
def create_dag_run(
self,
dag: DAG,
*,
task_states: Mapping[str, TaskInstanceState] | None = None,
execution_date: datetime.datetime | None = None,
is_backfill: bool = False,
state: DagRunState = DagRunState.RUNNING,
session: Session,
):
now = timezone.utcnow()
if execution_date is None:
execution_date = now
execution_date = pendulum.instance(execution_date)
if is_backfill:
run_type = DagRunType.BACKFILL_JOB
data_interval = dag.infer_automated_data_interval(execution_date)
else:
run_type = DagRunType.MANUAL
data_interval = dag.timetable.infer_manual_data_interval(run_after=execution_date)
dag_run = dag.create_dagrun(
run_type=run_type,
execution_date=execution_date,
data_interval=data_interval,
start_date=now,
state=state,
external_trigger=False,
)
if task_states is not None:
for task_id, task_state in task_states.items():
ti = dag_run.get_task_instance(task_id)
ti.set_state(task_state, session)
session.flush()
return dag_run
@pytest.mark.parametrize("state", [DagRunState.QUEUED, DagRunState.RUNNING])
def test_clear_task_instances_for_backfill_unfinished_dagrun(self, state, session):
now = timezone.utcnow()
dag_id = "test_clear_task_instances_for_backfill_dagrun"
dag = DAG(dag_id=dag_id, start_date=now)
dag_run = self.create_dag_run(dag, execution_date=now, is_backfill=True, state=state, session=session)
task0 = EmptyOperator(task_id="backfill_task_0", owner="test", dag=dag)
ti0 = TI(task=task0, run_id=dag_run.run_id)
ti0.run()
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session)
session.commit()
ti0.refresh_from_db()
dr0 = session.query(DagRun).filter(DagRun.dag_id == dag_id, DagRun.execution_date == now).first()
assert dr0.state == state
@pytest.mark.parametrize("state", [DagRunState.SUCCESS, DagRunState.FAILED])
def test_clear_task_instances_for_backfill_finished_dagrun(self, state, session):
now = timezone.utcnow()
dag_id = "test_clear_task_instances_for_backfill_dagrun"
dag = DAG(dag_id=dag_id, start_date=now)
dag_run = self.create_dag_run(dag, execution_date=now, is_backfill=True, state=state, session=session)
task0 = EmptyOperator(task_id="backfill_task_0", owner="test", dag=dag)
ti0 = TI(task=task0, run_id=dag_run.run_id)
ti0.run()
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session)
session.commit()
ti0.refresh_from_db()
dr0 = session.query(DagRun).filter(DagRun.dag_id == dag_id, DagRun.execution_date == now).first()
assert dr0.state == DagRunState.QUEUED
def test_dagrun_find(self, session):
now = timezone.utcnow()
dag_id1 = "test_dagrun_find_externally_triggered"
dag_run = DagRun(
dag_id=dag_id1,
run_id=dag_id1,
run_type=DagRunType.MANUAL,
execution_date=now,
start_date=now,
state=DagRunState.RUNNING,
external_trigger=True,
)
session.add(dag_run)
dag_id2 = "test_dagrun_find_not_externally_triggered"
dag_run = DagRun(
dag_id=dag_id2,
run_id=dag_id2,
run_type=DagRunType.MANUAL,
execution_date=now,
start_date=now,
state=DagRunState.RUNNING,
external_trigger=False,
)
session.add(dag_run)
session.commit()
assert 1 == len(DagRun.find(dag_id=dag_id1, external_trigger=True))
assert 1 == len(DagRun.find(run_id=dag_id1))
assert 2 == len(DagRun.find(run_id=[dag_id1, dag_id2]))
assert 2 == len(DagRun.find(execution_date=[now, now]))
assert 2 == len(DagRun.find(execution_date=now))
assert 0 == len(DagRun.find(dag_id=dag_id1, external_trigger=False))
assert 0 == len(DagRun.find(dag_id=dag_id2, external_trigger=True))
assert 1 == len(DagRun.find(dag_id=dag_id2, external_trigger=False))
def test_dagrun_find_duplicate(self, session):
now = timezone.utcnow()
dag_id = "test_dagrun_find_duplicate"
dag_run = DagRun(
dag_id=dag_id,
run_id=dag_id,
run_type=DagRunType.MANUAL,
execution_date=now,
start_date=now,
state=DagRunState.RUNNING,
external_trigger=True,
)
session.add(dag_run)
session.commit()
assert DagRun.find_duplicate(dag_id=dag_id, run_id=dag_id, execution_date=now) is not None
assert DagRun.find_duplicate(dag_id=dag_id, run_id=dag_id, execution_date=None) is not None
assert DagRun.find_duplicate(dag_id=dag_id, run_id=None, execution_date=now) is not None
assert DagRun.find_duplicate(dag_id=dag_id, run_id=None, execution_date=None) is None
def test_dagrun_success_when_all_skipped(self, session):
"""
Tests that a DAG run succeeds when all tasks are skipped
"""
dag = DAG(dag_id="test_dagrun_success_when_all_skipped", start_date=timezone.datetime(2017, 1, 1))
dag_task1 = ShortCircuitOperator(
task_id="test_short_circuit_false", dag=dag, python_callable=lambda: False
)
dag_task2 = EmptyOperator(task_id="test_state_skipped1", dag=dag)
dag_task3 = EmptyOperator(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": TaskInstanceState.SUCCESS,
"test_state_skipped1": TaskInstanceState.SKIPPED,
"test_state_skipped2": TaskInstanceState.SKIPPED,
}
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
dag_run.update_state()
assert DagRunState.SUCCESS == dag_run.state
def test_dagrun_not_stuck_in_running_when_all_tasks_instances_are_removed(self, session):
"""
Tests that a DAG run succeeds when all tasks are removed
"""
dag = DAG(dag_id="test_dagrun_success_when_all_skipped", start_date=timezone.datetime(2017, 1, 1))
dag_task1 = ShortCircuitOperator(
task_id="test_short_circuit_false", dag=dag, python_callable=lambda: False
)
dag_task2 = EmptyOperator(task_id="test_state_skipped1", dag=dag)
dag_task3 = EmptyOperator(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": TaskInstanceState.REMOVED,
"test_state_skipped1": TaskInstanceState.REMOVED,
"test_state_skipped2": TaskInstanceState.REMOVED,
}
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
dag_run.update_state()
assert DagRunState.SUCCESS == dag_run.state
def test_dagrun_success_conditions(self, 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 = EmptyOperator(task_id="A")
op2 = EmptyOperator(task_id="B")
op3 = EmptyOperator(task_id="C")
op4 = EmptyOperator(task_id="D")
op1.set_upstream([op2, op3])
op3.set_upstream(op4)
dag.clear()
now = pendulum.now("UTC")
dr = dag.create_dagrun(
run_id="test_dagrun_success_conditions",
state=DagRunState.RUNNING,
execution_date=now,
data_interval=dag.timetable.infer_manual_data_interval(run_after=now),
start_date=now,
)
# op1 = root
ti_op1 = dr.get_task_instance(task_id=op1.task_id)
ti_op1.set_state(state=TaskInstanceState.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
dr.update_state()
assert DagRunState.RUNNING == dr.state
# one has failed, but root is successful
ti_op2.set_state(state=TaskInstanceState.FAILED, session=session)
ti_op3.set_state(state=TaskInstanceState.SUCCESS, session=session)
ti_op4.set_state(state=TaskInstanceState.SUCCESS, session=session)
dr.update_state()
assert DagRunState.SUCCESS == dr.state
def test_dagrun_deadlock(self, session):
dag = DAG("text_dagrun_deadlock", start_date=DEFAULT_DATE, default_args={"owner": "owner1"})
with dag:
op1 = EmptyOperator(task_id="A")
op2 = EmptyOperator(task_id="B")
op2.trigger_rule = TriggerRule.ONE_FAILED
op2.set_upstream(op1)
dag.clear()
now = pendulum.now("UTC")
dr = dag.create_dagrun(
run_id="test_dagrun_deadlock",
state=DagRunState.RUNNING,
execution_date=now,
data_interval=dag.timetable.infer_manual_data_interval(run_after=now),
start_date=now,
session=session,
)
ti_op1: TI = dr.get_task_instance(task_id=op1.task_id, session=session)
ti_op2: TI = dr.get_task_instance(task_id=op2.task_id, session=session)
ti_op1.set_state(state=TaskInstanceState.SUCCESS, session=session)
ti_op2.set_state(state=None, session=session)
dr.update_state(session=session)
assert dr.state == DagRunState.RUNNING
ti_op2.set_state(state=None, session=session)
op2.trigger_rule = "invalid" # type: ignore
dr.update_state(session=session)
assert dr.state == DagRunState.FAILED
def test_dagrun_no_deadlock_with_shutdown(self, session):
dag = DAG("test_dagrun_no_deadlock_with_shutdown", start_date=DEFAULT_DATE)
with dag:
op1 = EmptyOperator(task_id="upstream_task")
op2 = EmptyOperator(task_id="downstream_task")
op2.set_upstream(op1)
dr = dag.create_dagrun(
run_id="test_dagrun_no_deadlock_with_shutdown",
state=DagRunState.RUNNING,
execution_date=DEFAULT_DATE,
data_interval=dag.timetable.infer_manual_data_interval(run_after=DEFAULT_DATE),
start_date=DEFAULT_DATE,
)
upstream_ti = dr.get_task_instance(task_id="upstream_task")
upstream_ti.set_state(TaskInstanceState.SHUTDOWN, session=session)
dr.update_state()
assert dr.state == DagRunState.RUNNING
def test_dagrun_no_deadlock_with_depends_on_past(self, session):
dag = DAG("test_dagrun_no_deadlock", start_date=DEFAULT_DATE)
with dag:
EmptyOperator(task_id="dop", depends_on_past=True)
EmptyOperator(task_id="tc", max_active_tis_per_dag=1)
dag.clear()
dr = dag.create_dagrun(
run_id="test_dagrun_no_deadlock_1",
state=DagRunState.RUNNING,
execution_date=DEFAULT_DATE,
data_interval=dag.timetable.infer_manual_data_interval(run_after=DEFAULT_DATE),
start_date=DEFAULT_DATE,
)
next_date = DEFAULT_DATE + datetime.timedelta(days=1)
dr2 = dag.create_dagrun(
run_id="test_dagrun_no_deadlock_2",
state=DagRunState.RUNNING,
execution_date=next_date,
data_interval=dag.timetable.infer_manual_data_interval(run_after=next_date),
start_date=next_date,
)
ti1_op1 = dr.get_task_instance(task_id="dop")
dr2.get_task_instance(task_id="dop")
ti2_op1 = dr.get_task_instance(task_id="tc")
dr.get_task_instance(task_id="tc")
ti1_op1.set_state(state=TaskInstanceState.RUNNING, session=session)
dr.update_state()
dr2.update_state()
assert dr.state == DagRunState.RUNNING
assert dr2.state == DagRunState.RUNNING
ti2_op1.set_state(state=TaskInstanceState.RUNNING, session=session)
dr.update_state()
dr2.update_state()
assert dr.state == DagRunState.RUNNING
assert dr2.state == DagRunState.RUNNING
def test_dagrun_success_callback(self, session):
def on_success_callable(context):
assert context["dag_run"].dag_id == "test_dagrun_success_callback"
dag = DAG(
dag_id="test_dagrun_success_callback",
start_date=datetime.datetime(2017, 1, 1),
on_success_callback=on_success_callable,
)
dag_task1 = EmptyOperator(task_id="test_state_succeeded1", dag=dag)
dag_task2 = EmptyOperator(task_id="test_state_succeeded2", dag=dag)
dag_task1.set_downstream(dag_task2)
initial_task_states = {
"test_state_succeeded1": TaskInstanceState.SUCCESS,
"test_state_succeeded2": TaskInstanceState.SUCCESS,
}
# Scheduler uses Serialized DAG -- so use that instead of the Actual DAG
dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag))
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
_, callback = dag_run.update_state()
assert DagRunState.SUCCESS == dag_run.state
# Callbacks are not added until handle_callback = False is passed to dag_run.update_state()
assert callback is None
def test_dagrun_failure_callback(self, session):
def on_failure_callable(context):
assert context["dag_run"].dag_id == "test_dagrun_failure_callback"
dag = DAG(
dag_id="test_dagrun_failure_callback",
start_date=datetime.datetime(2017, 1, 1),
on_failure_callback=on_failure_callable,
)
dag_task1 = EmptyOperator(task_id="test_state_succeeded1", dag=dag)
dag_task2 = EmptyOperator(task_id="test_state_failed2", dag=dag)
initial_task_states = {
"test_state_succeeded1": TaskInstanceState.SUCCESS,
"test_state_failed2": TaskInstanceState.FAILED,
}
dag_task1.set_downstream(dag_task2)
# Scheduler uses Serialized DAG -- so use that instead of the Actual DAG
dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag))
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
_, callback = dag_run.update_state()
assert DagRunState.FAILED == dag_run.state
# Callbacks are not added until handle_callback = False is passed to dag_run.update_state()
assert callback is None
def test_dagrun_update_state_with_handle_callback_success(self, session):
def on_success_callable(context):
assert context["dag_run"].dag_id == "test_dagrun_update_state_with_handle_callback_success"
dag = DAG(
dag_id="test_dagrun_update_state_with_handle_callback_success",
start_date=datetime.datetime(2017, 1, 1),
on_success_callback=on_success_callable,
)
DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session)
dag_task1 = EmptyOperator(task_id="test_state_succeeded1", dag=dag)
dag_task2 = EmptyOperator(task_id="test_state_succeeded2", dag=dag)
dag_task1.set_downstream(dag_task2)
initial_task_states = {
"test_state_succeeded1": TaskInstanceState.SUCCESS,
"test_state_succeeded2": TaskInstanceState.SUCCESS,
}
# Scheduler uses Serialized DAG -- so use that instead of the Actual DAG
dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag))
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
_, callback = dag_run.update_state(execute_callbacks=False)
assert DagRunState.SUCCESS == dag_run.state
# Callbacks are not added until handle_callback = False is passed to dag_run.update_state()
assert callback == DagCallbackRequest(
full_filepath=dag_run.dag.fileloc,
dag_id="test_dagrun_update_state_with_handle_callback_success",
run_id=dag_run.run_id,
is_failure_callback=False,
processor_subdir="/tmp/test",
msg="success",
)
def test_dagrun_update_state_with_handle_callback_failure(self, session):
def on_failure_callable(context):
assert context["dag_run"].dag_id == "test_dagrun_update_state_with_handle_callback_failure"
dag = DAG(
dag_id="test_dagrun_update_state_with_handle_callback_failure",
start_date=datetime.datetime(2017, 1, 1),
on_failure_callback=on_failure_callable,
)
DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session)
dag_task1 = EmptyOperator(task_id="test_state_succeeded1", dag=dag)
dag_task2 = EmptyOperator(task_id="test_state_failed2", dag=dag)
dag_task1.set_downstream(dag_task2)
initial_task_states = {
"test_state_succeeded1": TaskInstanceState.SUCCESS,
"test_state_failed2": TaskInstanceState.FAILED,
}
# Scheduler uses Serialized DAG -- so use that instead of the Actual DAG
dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag))
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
_, callback = dag_run.update_state(execute_callbacks=False)
assert DagRunState.FAILED == dag_run.state
# Callbacks are not added until handle_callback = False is passed to dag_run.update_state()
assert callback == DagCallbackRequest(
full_filepath=dag_run.dag.fileloc,
dag_id="test_dagrun_update_state_with_handle_callback_failure",
run_id=dag_run.run_id,
is_failure_callback=True,
processor_subdir="/tmp/test",
msg="task_failure",
)
def test_dagrun_set_state_end_date(self, session):
dag = DAG("test_dagrun_set_state_end_date", start_date=DEFAULT_DATE, default_args={"owner": "owner1"})
dag.clear()
now = pendulum.now("UTC")
dr = dag.create_dagrun(
run_id="test_dagrun_set_state_end_date",
state=DagRunState.RUNNING,
execution_date=now,
data_interval=dag.timetable.infer_manual_data_interval(now),
start_date=now,
)
# Initial end_date should be NULL
# DagRunState.SUCCESS and DagRunState.FAILED are all ending state and should set end_date
# DagRunState.RUNNING set end_date back to NULL
session.add(dr)
session.commit()
assert dr.end_date is None
dr.set_state(DagRunState.SUCCESS)
session.merge(dr)
session.commit()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_set_state_end_date").one()
assert dr_database.end_date is not None
assert dr.end_date == dr_database.end_date
dr.set_state(DagRunState.RUNNING)
session.merge(dr)
session.commit()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_set_state_end_date").one()
assert dr_database.end_date is None
dr.set_state(DagRunState.FAILED)
session.merge(dr)
session.commit()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_set_state_end_date").one()
assert dr_database.end_date is not None
assert dr.end_date == dr_database.end_date
def test_dagrun_update_state_end_date(self, session):
dag = DAG(
"test_dagrun_update_state_end_date", start_date=DEFAULT_DATE, default_args={"owner": "owner1"}
)
# A -> B
with dag:
op1 = EmptyOperator(task_id="A")
op2 = EmptyOperator(task_id="B")
op1.set_upstream(op2)
dag.clear()
now = pendulum.now("UTC")
dr = dag.create_dagrun(
run_id="test_dagrun_update_state_end_date",
state=DagRunState.RUNNING,
execution_date=now,
data_interval=dag.timetable.infer_manual_data_interval(now),
start_date=now,
)
# Initial end_date should be NULL
# DagRunState.SUCCESS and DagRunState.FAILED are all ending state and should set end_date
# DagRunState.RUNNING set end_date back to NULL
session.merge(dr)
session.commit()
assert dr.end_date is None
ti_op1 = dr.get_task_instance(task_id=op1.task_id)
ti_op1.set_state(state=TaskInstanceState.SUCCESS, session=session)
ti_op2 = dr.get_task_instance(task_id=op2.task_id)
ti_op2.set_state(state=TaskInstanceState.SUCCESS, session=session)
dr.update_state()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_update_state_end_date").one()
assert dr_database.end_date is not None
assert dr.end_date == dr_database.end_date
ti_op1.set_state(state=TaskInstanceState.RUNNING, session=session)
ti_op2.set_state(state=TaskInstanceState.RUNNING, session=session)
dr.update_state()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_update_state_end_date").one()
assert dr._state == DagRunState.RUNNING
assert dr.end_date is None
assert dr_database.end_date is None
ti_op1.set_state(state=TaskInstanceState.FAILED, session=session)
ti_op2.set_state(state=TaskInstanceState.FAILED, session=session)
dr.update_state()
dr_database = session.query(DagRun).filter(DagRun.run_id == "test_dagrun_update_state_end_date").one()
assert dr_database.end_date is not None
assert dr.end_date == dr_database.end_date
def test_get_task_instance_on_empty_dagrun(self, session):
"""
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=timezone.datetime(2017, 1, 1))
ShortCircuitOperator(task_id="test_short_circuit_false", dag=dag, python_callable=lambda: False)
now = timezone.utcnow()
# Don't use create_dagrun since it will create the task instances too which we
# don't want
dag_run = DagRun(
dag_id=dag.dag_id,
run_id="test_get_task_instance_on_empty_dagrun",
run_type=DagRunType.MANUAL,
execution_date=now,
start_date=now,
state=DagRunState.RUNNING,
external_trigger=False,
)
session.add(dag_run)
session.commit()
ti = dag_run.get_task_instance("test_short_circuit_false")
assert ti is None
def test_get_latest_runs(self, session):
dag = DAG(dag_id="test_latest_runs_1", start_date=DEFAULT_DATE)
self.create_dag_run(dag, execution_date=timezone.datetime(2015, 1, 1), session=session)
self.create_dag_run(dag, execution_date=timezone.datetime(2015, 1, 2), session=session)
dagruns = DagRun.get_latest_runs(session)
session.close()
for dagrun in dagruns:
if dagrun.dag_id == "test_latest_runs_1":
assert dagrun.execution_date == timezone.datetime(2015, 1, 2)
def test_removed_task_instances_can_be_restored(self, session):
def with_all_tasks_removed(dag):
return DAG(dag_id=dag.dag_id, start_date=dag.start_date)
dag = DAG("test_task_restoration", start_date=DEFAULT_DATE)
dag.add_task(EmptyOperator(task_id="flaky_task", owner="test"))
dagrun = self.create_dag_run(dag, session=session)
flaky_ti = dagrun.get_task_instances()[0]
assert "flaky_task" == flaky_ti.task_id
assert flaky_ti.state is None
dagrun.dag = with_all_tasks_removed(dag)
dagrun.verify_integrity()
flaky_ti.refresh_from_db()
assert flaky_ti.state is None
dagrun.dag.add_task(EmptyOperator(task_id="flaky_task", owner="test"))
dagrun.verify_integrity()
flaky_ti.refresh_from_db()
assert flaky_ti.state is None
def test_already_added_task_instances_can_be_ignored(self, session):
dag = DAG("triggered_dag", start_date=DEFAULT_DATE)
dag.add_task(EmptyOperator(task_id="first_task", owner="test"))
dagrun = self.create_dag_run(dag, session=session)
first_ti = dagrun.get_task_instances()[0]
assert "first_task" == first_ti.task_id
assert first_ti.state is None
# Lets assume that the above TI was added into DB by webserver, but if scheduler
# is running the same method at the same time it would find 0 TIs for this dag
# and proceeds further to create TIs. Hence mocking DagRun.get_task_instances
# method to return an empty list of TIs.
with mock.patch.object(DagRun, "get_task_instances") as mock_gtis:
mock_gtis.return_value = []
dagrun.verify_integrity()
first_ti.refresh_from_db()
assert first_ti.state is None
@pytest.mark.parametrize("state", State.task_states)
@mock.patch.object(settings, "task_instance_mutation_hook", autospec=True)
def test_task_instance_mutation_hook(self, mock_hook, session, state):
def mutate_task_instance(task_instance):
if task_instance.queue == "queue1":
task_instance.queue = "queue2"
else:
task_instance.queue = "queue1"
mock_hook.side_effect = mutate_task_instance
dag = DAG("test_task_instance_mutation_hook", start_date=DEFAULT_DATE)
dag.add_task(EmptyOperator(task_id="task_to_mutate", owner="test", queue="queue1"))
dagrun = self.create_dag_run(dag, session=session)
task = dagrun.get_task_instances()[0]
task.state = state
session.merge(task)
session.commit()
assert task.queue == "queue2"
dagrun.verify_integrity()
task = dagrun.get_task_instances()[0]
assert task.queue == "queue1"
@pytest.mark.parametrize(
"prev_ti_state, is_ti_success",
[
(TaskInstanceState.SUCCESS, True),
(TaskInstanceState.SKIPPED, True),
(TaskInstanceState.RUNNING, False),
(TaskInstanceState.FAILED, False),
(None, False),
],
)
def test_depends_on_past(self, session, prev_ti_state, is_ti_success):
dag_id = "test_depends_on_past"
dag = self.dagbag.get_dag(dag_id)
task = dag.tasks[0]
dag_run_1 = self.create_dag_run(
dag,
execution_date=timezone.datetime(2016, 1, 1, 0, 0, 0),
is_backfill=True,
session=session,
)
dag_run_2 = self.create_dag_run(
dag,
execution_date=timezone.datetime(2016, 1, 2, 0, 0, 0),
is_backfill=True,
session=session,
)
prev_ti = TI(task, run_id=dag_run_1.run_id)
ti = TI(task, run_id=dag_run_2.run_id)
prev_ti.set_state(prev_ti_state)
ti.set_state(TaskInstanceState.QUEUED)
ti.run()
assert (ti.state == TaskInstanceState.SUCCESS) == is_ti_success
@pytest.mark.parametrize(
"prev_ti_state, is_ti_success",
[
(TaskInstanceState.SUCCESS, True),
(TaskInstanceState.SKIPPED, True),
(TaskInstanceState.RUNNING, False),
(TaskInstanceState.FAILED, False),
(None, False),
],
)
def test_wait_for_downstream(self, session, prev_ti_state, is_ti_success):
dag_id = "test_wait_for_downstream"
dag = self.dagbag.get_dag(dag_id)
upstream, downstream = dag.tasks
# For ti.set_state() to work, the DagRun has to exist,
# Otherwise ti.previous_ti returns an unpersisted TI
dag_run_1 = self.create_dag_run(
dag,
execution_date=timezone.datetime(2016, 1, 1, 0, 0, 0),
is_backfill=True,
session=session,
)
dag_run_2 = self.create_dag_run(
dag,
execution_date=timezone.datetime(2016, 1, 2, 0, 0, 0),
is_backfill=True,
session=session,
)
prev_ti_downstream = TI(task=downstream, run_id=dag_run_1.run_id)
ti = TI(task=upstream, run_id=dag_run_2.run_id)
prev_ti = ti.get_previous_ti()
prev_ti.set_state(TaskInstanceState.SUCCESS)
assert prev_ti.state == TaskInstanceState.SUCCESS
prev_ti_downstream.set_state(prev_ti_state)
ti.set_state(TaskInstanceState.QUEUED)
ti.run()
assert (ti.state == TaskInstanceState.SUCCESS) == is_ti_success
@pytest.mark.parametrize("state", [DagRunState.QUEUED, DagRunState.RUNNING])
def test_next_dagruns_to_examine_only_unpaused(self, session, state):
"""
Check that "next_dagruns_to_examine" ignores runs from paused/inactive DAGs
and gets running/queued dagruns
"""
dag = DAG(dag_id="test_dags", start_date=DEFAULT_DATE)
EmptyOperator(task_id="dummy", dag=dag, owner="airflow")
orm_dag = DagModel(
dag_id=dag.dag_id,
has_task_concurrency_limits=False,
next_dagrun=DEFAULT_DATE,
next_dagrun_create_after=DEFAULT_DATE + datetime.timedelta(days=1),
is_active=True,
)
session.add(orm_dag)
session.flush()
dr = dag.create_dagrun(
run_type=DagRunType.SCHEDULED,
state=state,
execution_date=DEFAULT_DATE,
data_interval=dag.infer_automated_data_interval(DEFAULT_DATE),
start_date=DEFAULT_DATE if state == DagRunState.RUNNING else None,
session=session,
)
runs = DagRun.next_dagruns_to_examine(state, session).all()
assert runs == [dr]
orm_dag.is_paused = True
session.flush()
runs = DagRun.next_dagruns_to_examine(state, session).all()
assert runs == []
@mock.patch.object(Stats, "timing")
def test_no_scheduling_delay_for_nonscheduled_runs(self, stats_mock, session):
"""
Tests that dag scheduling delay stat is not called if the dagrun is not a scheduled run.
This case is manual run. Simple test for coherence check.
"""
dag = DAG(dag_id="test_dagrun_stats", start_date=DEFAULT_DATE)
dag_task = EmptyOperator(task_id="dummy", dag=dag)
initial_task_states = {
dag_task.task_id: TaskInstanceState.SUCCESS,
}
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
dag_run.update_state()
assert call(f"dagrun.{dag.dag_id}.first_task_scheduling_delay") not in stats_mock.mock_calls
@pytest.mark.parametrize(
"schedule_interval, expected",
[
("*/5 * * * *", True),
(None, False),
("@once", False),
],
)
def test_emit_scheduling_delay(self, session, schedule_interval, expected):
"""
Tests that dag scheduling delay stat is set properly once running scheduled dag.
dag_run.update_state() invokes the _emit_true_scheduling_delay_stats_for_finished_state method.
"""
dag = DAG(dag_id="test_emit_dag_stats", start_date=DEFAULT_DATE, schedule=schedule_interval)
dag_task = EmptyOperator(task_id="dummy", dag=dag, owner="airflow")
expected_stat_tags = {"dag_id": f"{dag.dag_id}", "run_type": DagRunType.SCHEDULED}
try:
info = dag.next_dagrun_info(None)
orm_dag_kwargs = {"dag_id": dag.dag_id, "has_task_concurrency_limits": False, "is_active": True}
if info is not None:
orm_dag_kwargs.update(
{
"next_dagrun": info.logical_date,
"next_dagrun_data_interval": info.data_interval,
"next_dagrun_create_after": info.run_after,
},
)
orm_dag = DagModel(**orm_dag_kwargs)
session.add(orm_dag)
session.flush()
dag_run = dag.create_dagrun(
run_type=DagRunType.SCHEDULED,
state=DagRunState.SUCCESS,
execution_date=dag.start_date,
data_interval=dag.infer_automated_data_interval(dag.start_date),
start_date=dag.start_date,
session=session,
)
ti = dag_run.get_task_instance(dag_task.task_id, session)
ti.set_state(TaskInstanceState.SUCCESS, session)
session.flush()
with mock.patch.object(Stats, "timing") as stats_mock:
dag_run.update_state(session)
metric_name = f"dagrun.{dag.dag_id}.first_task_scheduling_delay"
if expected:
true_delay = ti.start_date - dag_run.data_interval_end
sched_delay_stat_call = call(metric_name, true_delay, tags=expected_stat_tags)
sched_delay_stat_call_with_tags = call(
"dagrun.first_task_scheduling_delay", true_delay, tags=expected_stat_tags
)
assert (
sched_delay_stat_call in stats_mock.mock_calls
and sched_delay_stat_call_with_tags in stats_mock.mock_calls
)
else:
# Assert that we never passed the metric
sched_delay_stat_call = call(
metric_name,
mock.ANY,
)
assert sched_delay_stat_call not in stats_mock.mock_calls
finally:
# Don't write anything to the DB
session.rollback()
session.close()
def test_states_sets(self, session):
"""
Tests that adding State.failed_states and State.success_states work as expected.
"""
dag = DAG(dag_id="test_dagrun_states", start_date=DEFAULT_DATE)
dag_task_success = EmptyOperator(task_id="dummy", dag=dag)
dag_task_failed = EmptyOperator(task_id="dummy2", dag=dag)
initial_task_states = {
dag_task_success.task_id: TaskInstanceState.SUCCESS,
dag_task_failed.task_id: TaskInstanceState.FAILED,
}
dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session)
ti_success = dag_run.get_task_instance(dag_task_success.task_id)
ti_failed = dag_run.get_task_instance(dag_task_failed.task_id)
assert ti_success.state in State.success_states
assert ti_failed.state in State.failed_states
@pytest.mark.parametrize(
("run_type", "expected_tis"),
[
pytest.param(DagRunType.MANUAL, 1, id="manual"),
pytest.param(DagRunType.BACKFILL_JOB, 3, id="backfill"),
],
)
@mock.patch.object(Stats, "incr")
def test_verify_integrity_task_start_and_end_date(Stats_incr, session, run_type, expected_tis):
"""Test that tasks with specific dates are only created for backfill runs"""
with DAG("test", start_date=DEFAULT_DATE) as dag:
EmptyOperator(task_id="without")
EmptyOperator(task_id="with_start_date", start_date=DEFAULT_DATE + datetime.timedelta(1))
EmptyOperator(task_id="with_end_date", end_date=DEFAULT_DATE - datetime.timedelta(1))
dag_run = DagRun(
dag_id=dag.dag_id,
run_type=run_type,
execution_date=DEFAULT_DATE,
run_id=DagRun.generate_run_id(run_type, DEFAULT_DATE),
)
dag_run.dag = dag
session.add(dag_run)
session.flush()
dag_run.verify_integrity(session=session)
tis = dag_run.task_instances
assert len(tis) == expected_tis
Stats_incr.assert_any_call(
"task_instance_created_EmptyOperator", expected_tis, tags={"dag_id": "test", "run_type": run_type}
)
Stats_incr.assert_any_call(
"task_instance_created",
expected_tis,
tags={"dag_id": "test", "run_type": run_type, "task_type": "EmptyOperator"},
)
@pytest.mark.parametrize("is_noop", [True, False])
def test_expand_mapped_task_instance_at_create(is_noop, dag_maker, session):
with mock.patch("airflow.settings.task_instance_mutation_hook") as mock_mut: