-
Notifications
You must be signed in to change notification settings - Fork 14.4k
/
test_base.py
912 lines (771 loc) · 35.3 KB
/
test_base.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
#
# 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
from datetime import timedelta
from unittest.mock import Mock, patch
import pytest
import time_machine
from airflow.exceptions import AirflowException, AirflowRescheduleException, AirflowSensorTimeout
from airflow.executors.debug_executor import DebugExecutor
from airflow.executors.executor_constants import (
CELERY_EXECUTOR,
CELERY_KUBERNETES_EXECUTOR,
DASK_EXECUTOR,
DEBUG_EXECUTOR,
KUBERNETES_EXECUTOR,
LOCAL_EXECUTOR,
LOCAL_KUBERNETES_EXECUTOR,
SEQUENTIAL_EXECUTOR,
)
from airflow.executors.local_executor import LocalExecutor
from airflow.executors.sequential_executor import SequentialExecutor
from airflow.models import TaskReschedule
from airflow.models.xcom import XCom
from airflow.operators.empty import EmptyOperator
from airflow.providers.celery.executors.celery_executor import CeleryExecutor
from airflow.providers.celery.executors.celery_kubernetes_executor import CeleryKubernetesExecutor
from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubernetesExecutor
from airflow.providers.cncf.kubernetes.executors.local_kubernetes_executor import (
LocalKubernetesExecutor,
)
from airflow.sensors.base import BaseSensorOperator, PokeReturnValue, poke_mode_only
from airflow.ti_deps.deps.ready_to_reschedule import ReadyToRescheduleDep
from airflow.utils import timezone
from airflow.utils.context import Context
from airflow.utils.state import State
from airflow.utils.timezone import datetime
from tests.test_utils import db
DEFAULT_DATE = datetime(2015, 1, 1)
TEST_DAG_ID = "unit_test_dag"
DUMMY_OP = "dummy_op"
SENSOR_OP = "sensor_op"
DEV_NULL = "dev/null"
class DummySensor(BaseSensorOperator):
def __init__(self, return_value=False, **kwargs):
super().__init__(**kwargs)
self.return_value = return_value
def poke(self, context: Context):
return self.return_value
class DummySensorWithXcomValue(BaseSensorOperator):
def __init__(self, return_value=False, xcom_value=None, **kwargs):
super().__init__(**kwargs)
self.xcom_value = xcom_value
self.return_value = return_value
def poke(self, context: Context):
return PokeReturnValue(self.return_value, self.xcom_value)
class TestBaseSensor:
@staticmethod
def clean_db():
db.clear_db_runs()
db.clear_db_task_reschedule()
db.clear_db_xcom()
@pytest.fixture(autouse=True)
def _auto_clean(self, dag_maker):
"""(auto use)"""
self.clean_db()
yield
self.clean_db()
@pytest.fixture
def make_sensor(self, dag_maker):
"""Create a DummySensor and associated DagRun"""
def _make_sensor(return_value, task_id=SENSOR_OP, **kwargs):
poke_interval = "poke_interval"
timeout = "timeout"
if poke_interval not in kwargs:
kwargs[poke_interval] = 0
if timeout not in kwargs:
kwargs[timeout] = 0
with dag_maker(TEST_DAG_ID):
if "xcom_value" in kwargs:
sensor = DummySensorWithXcomValue(task_id=task_id, return_value=return_value, **kwargs)
else:
sensor = DummySensor(task_id=task_id, return_value=return_value, **kwargs)
dummy_op = EmptyOperator(task_id=DUMMY_OP)
sensor >> dummy_op
return sensor, dag_maker.create_dagrun()
return _make_sensor
@classmethod
def _run(cls, task, **kwargs):
task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True, **kwargs)
def test_ok(self, make_sensor):
sensor, dr = make_sensor(True)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SUCCESS
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_fail(self, make_sensor):
sensor, dr = make_sensor(False)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.FAILED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_soft_fail(self, make_sensor):
sensor, dr = make_sensor(False, soft_fail=True)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SKIPPED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_soft_fail_with_retries(self, make_sensor):
sensor, dr = make_sensor(
return_value=False, soft_fail=True, retries=1, retry_delay=timedelta(milliseconds=1)
)
# first run times out and task instance is skipped
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SKIPPED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_ok_with_reschedule(self, make_sensor, time_machine):
sensor, dr = make_sensor(return_value=None, poke_interval=10, timeout=25, mode="reschedule")
sensor.poke = Mock(side_effect=[False, False, True])
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
# verify task is re-scheduled, i.e. state set to NONE
assert ti.state == State.UP_FOR_RESCHEDULE
# verify task start date is the initial one
assert ti.start_date == date1
# verify one row in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 1
assert task_reschedules[0].start_date == date1
assert task_reschedules[0].reschedule_date == date1 + timedelta(seconds=sensor.poke_interval)
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# second poke returns False and task is re-scheduled
time_machine.coordinates.shift(sensor.poke_interval)
date2 = date1 + timedelta(seconds=sensor.poke_interval)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
# verify task is re-scheduled, i.e. state set to NONE
assert ti.state == State.UP_FOR_RESCHEDULE
# verify task start date is the initial one
assert ti.start_date == date1
# verify two rows in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 2
assert task_reschedules[1].start_date == date2
assert task_reschedules[1].reschedule_date == date2 + timedelta(seconds=sensor.poke_interval)
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# third poke returns True and task succeeds
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SUCCESS
# verify task start date is the initial one
assert ti.start_date == date1
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_fail_with_reschedule(self, make_sensor, time_machine):
sensor, dr = make_sensor(return_value=False, poke_interval=10, timeout=5, mode="reschedule")
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.UP_FOR_RESCHEDULE
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# second poke returns False, timeout occurs
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.FAILED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_soft_fail_with_reschedule(self, make_sensor, time_machine):
sensor, dr = make_sensor(
return_value=False, poke_interval=10, timeout=5, soft_fail=True, mode="reschedule"
)
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.UP_FOR_RESCHEDULE
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# second poke returns False, timeout occurs
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SKIPPED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_ok_with_reschedule_and_retry(self, make_sensor, time_machine):
sensor, dr = make_sensor(
return_value=None,
poke_interval=10,
timeout=5,
retries=1,
retry_delay=timedelta(seconds=10),
mode="reschedule",
)
sensor.poke = Mock(side_effect=[False, False, False, True])
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.UP_FOR_RESCHEDULE
# verify one row in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 1
assert task_reschedules[0].start_date == date1
assert task_reschedules[0].reschedule_date == date1 + timedelta(seconds=sensor.poke_interval)
assert task_reschedules[0].try_number == 1
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# second poke timesout and task instance is failed
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.FAILED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# Task is cleared
sensor.clear()
# third poke returns False and task is rescheduled again
date3 = date1 + timedelta(seconds=sensor.poke_interval) * 2 + sensor.retry_delay
time_machine.coordinates.shift(sensor.poke_interval + sensor.retry_delay.total_seconds())
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.UP_FOR_RESCHEDULE
# verify one row in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 1
assert task_reschedules[0].start_date == date3
assert task_reschedules[0].reschedule_date == date3 + timedelta(seconds=sensor.poke_interval)
assert task_reschedules[0].try_number == 2
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# fourth poke return True and task succeeds
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SUCCESS
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
@pytest.mark.parametrize("mode", ["poke", "reschedule"])
def test_should_include_ready_to_reschedule_dep(self, mode):
sensor = DummySensor(task_id="a", return_value=True, mode=mode)
deps = sensor.deps
assert ReadyToRescheduleDep() in deps
def test_invalid_mode(self):
with pytest.raises(AirflowException):
DummySensor(task_id="a", mode="foo")
def test_ok_with_custom_reschedule_exception(self, make_sensor):
sensor, dr = make_sensor(return_value=None, mode="reschedule")
date1 = timezone.utcnow()
date2 = date1 + timedelta(seconds=60)
date3 = date1 + timedelta(seconds=120)
sensor.poke = Mock(
side_effect=[AirflowRescheduleException(date2), AirflowRescheduleException(date3), True]
)
# first poke returns False and task is re-scheduled
with time_machine.travel(date1, tick=False):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
# verify task is re-scheduled, i.e. state set to NONE
assert ti.state == State.UP_FOR_RESCHEDULE
# verify one row in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 1
assert task_reschedules[0].start_date == date1
assert task_reschedules[0].reschedule_date == date2
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# second poke returns False and task is re-scheduled
with time_machine.travel(date2, tick=False):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
# verify task is re-scheduled, i.e. state set to NONE
assert ti.state == State.UP_FOR_RESCHEDULE
# verify two rows in task_reschedule table
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 2
assert task_reschedules[1].start_date == date2
assert task_reschedules[1].reschedule_date == date3
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
# third poke returns True and task succeeds
with time_machine.travel(date3, tick=False):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SUCCESS
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_reschedule_with_test_mode(self, make_sensor):
sensor, dr = make_sensor(return_value=None, poke_interval=10, timeout=25, mode="reschedule")
sensor.poke = Mock(side_effect=[False])
# poke returns False and AirflowRescheduleException is raised
date1 = timezone.utcnow()
with time_machine.travel(date1, tick=False):
self._run(sensor, test_mode=True)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
# in test mode state is not modified
assert ti.state == State.NONE
# in test mode no reschedule request is recorded
task_reschedules = TaskReschedule.find_for_task_instance(ti)
assert len(task_reschedules) == 0
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
def test_sensor_with_invalid_poke_interval(self):
negative_poke_interval = -10
non_number_poke_interval = "abcd"
positive_poke_interval = 10
with pytest.raises(AirflowException):
DummySensor(
task_id="test_sensor_task_1",
return_value=None,
poke_interval=negative_poke_interval,
timeout=25,
)
with pytest.raises(AirflowException):
DummySensor(
task_id="test_sensor_task_2",
return_value=None,
poke_interval=non_number_poke_interval,
timeout=25,
)
DummySensor(
task_id="test_sensor_task_3", return_value=None, poke_interval=positive_poke_interval, timeout=25
)
def test_sensor_with_invalid_timeout(self):
negative_timeout = -25
non_number_timeout = "abcd"
positive_timeout = 25
with pytest.raises(AirflowException):
DummySensor(
task_id="test_sensor_task_1", return_value=None, poke_interval=10, timeout=negative_timeout
)
with pytest.raises(AirflowException):
DummySensor(
task_id="test_sensor_task_2", return_value=None, poke_interval=10, timeout=non_number_timeout
)
DummySensor(
task_id="test_sensor_task_3", return_value=None, poke_interval=10, timeout=positive_timeout
)
def test_sensor_with_exponential_backoff_off(self):
sensor = DummySensor(
task_id=SENSOR_OP, return_value=None, poke_interval=5, timeout=60, exponential_backoff=False
)
started_at = timezone.utcnow() - timedelta(seconds=10)
def run_duration():
return (timezone.utcnow - started_at).total_seconds()
assert sensor._get_next_poke_interval(started_at, run_duration, 1) == sensor.poke_interval
assert sensor._get_next_poke_interval(started_at, run_duration, 2) == sensor.poke_interval
def test_sensor_with_exponential_backoff_on(self):
sensor = DummySensor(
task_id=SENSOR_OP, return_value=None, poke_interval=5, timeout=60, exponential_backoff=True
)
with patch("airflow.utils.timezone.utcnow") as mock_utctime:
mock_utctime.return_value = DEFAULT_DATE
started_at = timezone.utcnow() - timedelta(seconds=10)
def run_duration():
return (timezone.utcnow - started_at).total_seconds()
interval1 = sensor._get_next_poke_interval(started_at, run_duration, 1)
interval2 = sensor._get_next_poke_interval(started_at, run_duration, 2)
assert interval1 >= 0
assert interval1 <= sensor.poke_interval
assert interval2 >= sensor.poke_interval
assert interval2 > interval1
@pytest.mark.parametrize("poke_interval", [0, 0.1, 0.9, 1, 2, 3])
def test_sensor_with_exponential_backoff_on_and_small_poke_interval(self, poke_interval):
"""Test that sensor works correctly when poke_interval is small and exponential_backoff is on"""
sensor = DummySensor(
task_id=SENSOR_OP,
return_value=None,
poke_interval=poke_interval,
timeout=60,
exponential_backoff=True,
)
with patch("airflow.utils.timezone.utcnow") as mock_utctime:
mock_utctime.return_value = DEFAULT_DATE
started_at = timezone.utcnow() - timedelta(seconds=10)
def run_duration():
return (timezone.utcnow - started_at).total_seconds()
intervals = [
sensor._get_next_poke_interval(started_at, run_duration, retry_number)
for retry_number in range(1, 10)
]
for i in range(0, len(intervals) - 1):
# intervals should be increasing or equals
assert intervals[i] <= intervals[i + 1]
if poke_interval > 0:
# check if the intervals are increasing after some retries when poke_interval > 0
assert intervals[0] < intervals[-1]
else:
# check if the intervals are equal after some retries when poke_interval == 0
assert intervals[0] == intervals[-1]
def test_sensor_with_exponential_backoff_on_and_max_wait(self):
sensor = DummySensor(
task_id=SENSOR_OP,
return_value=None,
poke_interval=10,
timeout=60,
exponential_backoff=True,
max_wait=timedelta(seconds=30),
)
with patch("airflow.utils.timezone.utcnow") as mock_utctime:
mock_utctime.return_value = DEFAULT_DATE
started_at = timezone.utcnow() - timedelta(seconds=10)
def run_duration():
return (timezone.utcnow - started_at).total_seconds()
for idx, expected in enumerate([2, 6, 13, 30, 30, 30, 30, 30]):
assert sensor._get_next_poke_interval(started_at, run_duration, idx) == expected
@pytest.mark.backend("mysql")
def test_reschedule_poke_interval_too_long_on_mysql(self, make_sensor):
with pytest.raises(AirflowException) as ctx:
make_sensor(poke_interval=863998946, mode="reschedule", return_value="irrelevant")
assert str(ctx.value) == (
"Cannot set poke_interval to 863998946.0 seconds in reschedule mode "
"since it will take reschedule time over MySQL's TIMESTAMP limit."
)
@pytest.mark.backend("mysql")
def test_reschedule_date_too_late_on_mysql(self, make_sensor):
sensor, _ = make_sensor(poke_interval=60 * 60 * 24, mode="reschedule", return_value=False)
# A few hours until TIMESTAMP's limit, the next poke will take us over.
with time_machine.travel(datetime(2038, 1, 19, tzinfo=timezone.utc), tick=False):
with pytest.raises(AirflowSensorTimeout) as ctx:
self._run(sensor)
assert str(ctx.value) == (
"Cannot reschedule DAG unit_test_dag to 2038-01-20T00:00:00+00:00 "
"since it is over MySQL's TIMESTAMP storage limit."
)
def test_reschedule_and_retry_timeout(self, make_sensor, time_machine):
"""
Test mode="reschedule", retries and timeout configurations interact correctly.
Given a sensor configured like this:
poke_interval=5
timeout=10
retries=2
retry_delay=timedelta(seconds=3)
If the second poke raises RuntimeError, all other pokes return False, this is how it should
behave:
00:00 Returns False try_number=1, max_tries=2, state=up_for_reschedule
00:05 Raises RuntimeError try_number=2, max_tries=2, state=up_for_retry
00:08 Returns False try_number=2, max_tries=2, state=up_for_reschedule
00:13 Raises AirflowSensorTimeout try_number=3, max_tries=2, state=failed
And then the sensor is cleared at 00:19. It should behave like this:
00:19 Returns False try_number=3, max_tries=4, state=up_for_reschedule
00:24 Returns False try_number=3, max_tries=4, state=up_for_reschedule
00:26 Returns False try_number=3, max_tries=4, state=up_for_reschedule
00:31 Raises AirflowSensorTimeout, try_number=4, max_tries=4, state=failed
"""
sensor, dr = make_sensor(
return_value=None,
poke_interval=5,
timeout=10,
retries=2,
retry_delay=timedelta(seconds=3),
mode="reschedule",
)
sensor.poke = Mock(side_effect=[False, RuntimeError, False, False, False, False, False, False])
def assert_ti_state(try_number, max_tries, state):
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.try_number == try_number
assert ti.max_tries == max_tries
assert ti.state == state
break
else:
self.fail("sensor not found")
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
assert_ti_state(1, 2, State.UP_FOR_RESCHEDULE)
# second poke raises RuntimeError and task instance retries
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(RuntimeError):
self._run(sensor)
assert_ti_state(2, 2, State.UP_FOR_RETRY)
# third poke returns False and task is rescheduled again
time_machine.coordinates.shift(sensor.retry_delay + timedelta(seconds=1))
self._run(sensor)
assert_ti_state(2, 2, State.UP_FOR_RESCHEDULE)
# fourth poke times out and raises AirflowSensorTimeout
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
assert_ti_state(3, 2, State.FAILED)
# Clear the failed sensor
sensor.clear()
time_machine.coordinates.shift(20)
for _ in range(3):
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
assert_ti_state(3, 4, State.UP_FOR_RESCHEDULE)
# Last poke times out and raises AirflowSensorTimeout
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
assert_ti_state(4, 4, State.FAILED)
def test_reschedule_and_retry_timeout_and_silent_fail(self, make_sensor, time_machine):
"""
Test mode="reschedule", silent_fail=True then retries and timeout configurations interact correctly.
Given a sensor configured like this:
poke_interval=5
timeout=10
retries=2
retry_delay=timedelta(seconds=3)
silent_fail=True
If the second poke raises RuntimeError, all other pokes return False, this is how it should
behave:
00:00 Returns False try_number=1, max_tries=2, state=up_for_reschedule
00:05 Raises RuntimeError try_number=1, max_tries=2, state=up_for_reschedule
00:08 Returns False try_number=1, max_tries=2, state=up_for_reschedule
00:13 Raises AirflowSensorTimeout try_number=2, max_tries=2, state=failed
And then the sensor is cleared at 00:19. It should behave like this:
00:19 Returns False try_number=2, max_tries=3, state=up_for_reschedule
00:24 Returns False try_number=2, max_tries=3, state=up_for_reschedule
00:26 Returns False try_number=2, max_tries=3, state=up_for_reschedule
00:31 Raises AirflowSensorTimeout, try_number=3, max_tries=3, state=failed
"""
sensor, dr = make_sensor(
return_value=None,
poke_interval=5,
timeout=10,
retries=2,
retry_delay=timedelta(seconds=3),
mode="reschedule",
silent_fail=True,
)
sensor.poke = Mock(side_effect=[False, RuntimeError, False, False, False, False, False, False])
def assert_ti_state(try_number, max_tries, state):
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.try_number == try_number
assert ti.max_tries == max_tries
assert ti.state == state
break
else:
self.fail("sensor not found")
# first poke returns False and task is re-scheduled
date1 = timezone.utcnow()
time_machine.move_to(date1, tick=False)
self._run(sensor)
assert_ti_state(1, 2, State.UP_FOR_RESCHEDULE)
# second poke raises RuntimeError and task instance is re-scheduled again
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
assert_ti_state(1, 2, State.UP_FOR_RESCHEDULE)
# third poke returns False and task is rescheduled again
time_machine.coordinates.shift(sensor.retry_delay + timedelta(seconds=1))
self._run(sensor)
assert_ti_state(1, 2, State.UP_FOR_RESCHEDULE)
# fourth poke times out and raises AirflowSensorTimeout
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
assert_ti_state(2, 2, State.FAILED)
# Clear the failed sensor
sensor.clear()
time_machine.coordinates.shift(20)
for _ in range(3):
time_machine.coordinates.shift(sensor.poke_interval)
self._run(sensor)
assert_ti_state(2, 3, State.UP_FOR_RESCHEDULE)
# Last poke times out and raises AirflowSensorTimeout
time_machine.coordinates.shift(sensor.poke_interval)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
assert_ti_state(3, 3, State.FAILED)
def test_sensor_with_xcom(self, make_sensor):
xcom_value = "TestValue"
sensor, dr = make_sensor(True, xcom_value=xcom_value)
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.SUCCESS
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
actual_xcom_value = XCom.get_one(
key="return_value", task_id=SENSOR_OP, dag_id=dr.dag_id, run_id=dr.run_id
)
assert actual_xcom_value == xcom_value
def test_sensor_with_xcom_fails(self, make_sensor):
xcom_value = "TestValue"
sensor, dr = make_sensor(False, xcom_value=xcom_value)
with pytest.raises(AirflowSensorTimeout):
self._run(sensor)
tis = dr.get_task_instances()
assert len(tis) == 2
for ti in tis:
if ti.task_id == SENSOR_OP:
assert ti.state == State.FAILED
if ti.task_id == DUMMY_OP:
assert ti.state == State.NONE
actual_xcom_value = XCom.get_one(
key="return_value", task_id=SENSOR_OP, dag_id=dr.dag_id, run_id=dr.run_id
)
assert actual_xcom_value is None
@pytest.mark.parametrize(
"executor_cls_mode",
[
(CELERY_EXECUTOR, CeleryExecutor, "poke"),
(CELERY_KUBERNETES_EXECUTOR, CeleryKubernetesExecutor, "poke"),
(DEBUG_EXECUTOR, DebugExecutor, "reschedule"),
(KUBERNETES_EXECUTOR, KubernetesExecutor, "poke"),
(LOCAL_EXECUTOR, LocalExecutor, "poke"),
(LOCAL_KUBERNETES_EXECUTOR, LocalKubernetesExecutor, "poke"),
(SEQUENTIAL_EXECUTOR, SequentialExecutor, "poke"),
(DASK_EXECUTOR, DebugExecutor, "poke"),
],
ids=[
CELERY_EXECUTOR,
CELERY_KUBERNETES_EXECUTOR,
DEBUG_EXECUTOR,
KUBERNETES_EXECUTOR,
LOCAL_EXECUTOR,
LOCAL_KUBERNETES_EXECUTOR,
SEQUENTIAL_EXECUTOR,
DASK_EXECUTOR,
],
)
def test_prepare_for_execution(self, executor_cls_mode):
"""
Should change mode of the task to reschedule if using DEBUG_EXECUTOR
"""
executor_name, executor_cls, mode = executor_cls_mode
sensor = DummySensor(
task_id=SENSOR_OP,
return_value=None,
poke_interval=10,
timeout=60,
exponential_backoff=True,
max_wait=timedelta(seconds=30),
)
with patch("airflow.configuration.conf.get") as get, patch(
"airflow.executors.executor_loader.ExecutorLoader.load_executor"
) as load_executor:
get.return_value = executor_name
load_executor.return_value = executor_cls
task = sensor.prepare_for_execution()
assert task.mode == mode
@poke_mode_only
class DummyPokeOnlySensor(BaseSensorOperator):
def __init__(self, poke_changes_mode=False, **kwargs):
self.mode = kwargs["mode"]
super().__init__(**kwargs)
self.poke_changes_mode = poke_changes_mode
self.return_value = True
def poke(self, context: Context):
if self.poke_changes_mode:
self.change_mode("reschedule")
return self.return_value
def change_mode(self, mode):
self.mode = mode
class TestPokeModeOnly:
def test_poke_mode_only_allows_poke_mode(self):
try:
sensor = DummyPokeOnlySensor(task_id="foo", mode="poke", poke_changes_mode=False)
except ValueError:
self.fail("__init__ failed with mode='poke'.")
try:
sensor.poke({})
except ValueError:
self.fail("poke failed without changing mode from 'poke'.")
try:
sensor.change_mode("poke")
except ValueError:
self.fail("class method failed without changing mode from 'poke'.")
def test_poke_mode_only_bad_class_method(self):
sensor = DummyPokeOnlySensor(task_id="foo", mode="poke", poke_changes_mode=False)
with pytest.raises(ValueError, match="Cannot set mode to 'reschedule'. Only 'poke' is acceptable"):
sensor.change_mode("reschedule")
def test_poke_mode_only_bad_init(self):
with pytest.raises(ValueError, match="Cannot set mode to 'reschedule'. Only 'poke' is acceptable"):
DummyPokeOnlySensor(task_id="foo", mode="reschedule", poke_changes_mode=False)
def test_poke_mode_only_bad_poke(self):
sensor = DummyPokeOnlySensor(task_id="foo", mode="poke", poke_changes_mode=True)
with pytest.raises(ValueError, match="Cannot set mode to 'reschedule'. Only 'poke' is acceptable"):
sensor.poke({})