-
Notifications
You must be signed in to change notification settings - Fork 8
/
test_cleartasks.py
361 lines (304 loc) · 12.3 KB
/
test_cleartasks.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
#
# 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.
import datetime
import unittest
from airflow import settings
from airflow.models import DAG, TaskInstance as TI, TaskReschedule, clear_task_instances
from airflow.operators.dummy import DummyOperator
from airflow.sensors.python import PythonSensor
from airflow.utils.session import create_session
from airflow.utils.state import State
from airflow.utils.types import DagRunType
from tests.models import DEFAULT_DATE
from tests.test_utils import db
class TestClearTasks(unittest.TestCase):
def setUp(self) -> None:
db.clear_db_runs()
def tearDown(self):
db.clear_db_runs()
def test_clear_task_instances(self):
dag = DAG(
'test_clear_task_instances',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
task0 = DummyOperator(task_id='0', owner='test', dag=dag)
task1 = DummyOperator(task_id='1', owner='test', dag=dag, retries=2)
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
ti1 = TI(task=task1, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti0.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
ti0.run()
ti1.run()
with create_session() as session:
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session, dag=dag)
ti0.refresh_from_db()
ti1.refresh_from_db()
# Next try to run will be try 2
assert ti0.try_number == 2
assert ti0.max_tries == 1
assert ti1.try_number == 2
assert ti1.max_tries == 3
def test_clear_task_instances_external_executor_id(self):
dag = DAG(
'test_clear_task_instances_external_executor_id',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
task0 = DummyOperator(task_id='task0', owner='test', dag=dag)
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
ti0.state = State.SUCCESS
ti0.external_executor_id = "some_external_executor_id"
with create_session() as session:
session.add(ti0)
session.commit()
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session, dag=dag)
ti0.refresh_from_db()
assert ti0.state is None
assert ti0.external_executor_id is None
def test_clear_task_instances_without_task(self):
dag = DAG(
'test_clear_task_instances_without_task',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
task0 = DummyOperator(task_id='task0', owner='test', dag=dag)
task1 = DummyOperator(task_id='task1', owner='test', dag=dag, retries=2)
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
ti1 = TI(task=task1, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti0.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
ti0.run()
ti1.run()
# Remove the task from dag.
dag.task_dict = {}
assert not dag.has_task(task0.task_id)
assert not dag.has_task(task1.task_id)
with create_session() as session:
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session)
# When dag is None, max_tries will be maximum of original max_tries or try_number.
ti0.refresh_from_db()
ti1.refresh_from_db()
# Next try to run will be try 2
assert ti0.try_number == 2
assert ti0.max_tries == 1
assert ti1.try_number == 2
assert ti1.max_tries == 2
def test_clear_task_instances_without_dag(self):
dag = DAG(
'test_clear_task_instances_without_dag',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
task0 = DummyOperator(task_id='task_0', owner='test', dag=dag)
task1 = DummyOperator(task_id='task_1', owner='test', dag=dag, retries=2)
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
ti1 = TI(task=task1, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti0.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
ti0.run()
ti1.run()
with create_session() as session:
qry = session.query(TI).filter(TI.dag_id == dag.dag_id).all()
clear_task_instances(qry, session)
# When dag is None, max_tries will be maximum of original max_tries or try_number.
ti0.refresh_from_db()
ti1.refresh_from_db()
# Next try to run will be try 2
assert ti0.try_number == 2
assert ti0.max_tries == 1
assert ti1.try_number == 2
assert ti1.max_tries == 2
def test_clear_task_instances_with_task_reschedule(self):
"""Test that TaskReschedules are deleted correctly when TaskInstances are cleared"""
with DAG(
'test_clear_task_instances_with_task_reschedule',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
) as dag:
task0 = PythonSensor(task_id='0', python_callable=lambda: False, mode="reschedule")
task1 = PythonSensor(task_id='1', python_callable=lambda: False, mode="reschedule")
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
ti1 = TI(task=task1, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti0.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
ti0.run()
ti1.run()
with create_session() as session:
def count_task_reschedule(task_id):
return (
session.query(TaskReschedule)
.filter(
TaskReschedule.dag_id == dag.dag_id,
TaskReschedule.task_id == task_id,
TaskReschedule.execution_date == DEFAULT_DATE,
TaskReschedule.try_number == 1,
)
.count()
)
assert count_task_reschedule(ti0.task_id) == 1
assert count_task_reschedule(ti1.task_id) == 1
qry = session.query(TI).filter(TI.dag_id == dag.dag_id, TI.task_id == ti0.task_id).all()
clear_task_instances(qry, session, dag=dag)
assert count_task_reschedule(ti0.task_id) == 0
assert count_task_reschedule(ti1.task_id) == 1
def test_dag_clear(self):
dag = DAG(
'test_dag_clear', start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=10)
)
task0 = DummyOperator(task_id='test_dag_clear_task_0', owner='test', dag=dag)
ti0 = TI(task=task0, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti0.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
# Next try to run will be try 1
assert ti0.try_number == 1
ti0.run()
assert ti0.try_number == 2
dag.clear()
ti0.refresh_from_db()
assert ti0.try_number == 2
assert ti0.state == State.NONE
assert ti0.max_tries == 1
task1 = DummyOperator(task_id='test_dag_clear_task_1', owner='test', dag=dag, retries=2)
ti1 = TI(task=task1, execution_date=DEFAULT_DATE)
assert ti1.max_tries == 2
ti1.try_number = 1
# Next try will be 2
ti1.run()
assert ti1.try_number == 3
assert ti1.max_tries == 2
dag.clear()
ti0.refresh_from_db()
ti1.refresh_from_db()
# after clear dag, ti2 should show attempt 3 of 5
assert ti1.max_tries == 4
assert ti1.try_number == 3
# after clear dag, ti1 should show attempt 2 of 2
assert ti0.try_number == 2
assert ti0.max_tries == 1
def test_dags_clear(self):
# setup
session = settings.Session()
dags, tis = [], []
num_of_dags = 5
for i in range(num_of_dags):
dag = DAG(
'test_dag_clear_' + str(i),
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
ti = TI(
task=DummyOperator(task_id='test_task_clear_' + str(i), owner='test', dag=dag),
execution_date=DEFAULT_DATE,
)
dag.create_dagrun(
execution_date=ti.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
dags.append(dag)
tis.append(ti)
# test clear all dags
for i in range(num_of_dags):
tis[i].run()
assert tis[i].state == State.SUCCESS
assert tis[i].try_number == 2
assert tis[i].max_tries == 0
DAG.clear_dags(dags)
for i in range(num_of_dags):
tis[i].refresh_from_db()
assert tis[i].state == State.NONE
assert tis[i].try_number == 2
assert tis[i].max_tries == 1
# test dry_run
for i in range(num_of_dags):
tis[i].run()
assert tis[i].state == State.SUCCESS
assert tis[i].try_number == 3
assert tis[i].max_tries == 1
DAG.clear_dags(dags, dry_run=True)
for i in range(num_of_dags):
tis[i].refresh_from_db()
assert tis[i].state == State.SUCCESS
assert tis[i].try_number == 3
assert tis[i].max_tries == 1
# test only_failed
from random import randint
failed_dag_idx = randint(0, len(tis) - 1)
tis[failed_dag_idx].state = State.FAILED
session.merge(tis[failed_dag_idx])
session.commit()
DAG.clear_dags(dags, only_failed=True)
for i in range(num_of_dags):
tis[i].refresh_from_db()
if i != failed_dag_idx:
assert tis[i].state == State.SUCCESS
assert tis[i].try_number == 3
assert tis[i].max_tries == 1
else:
assert tis[i].state == State.NONE
assert tis[i].try_number == 3
assert tis[i].max_tries == 2
def test_operator_clear(self):
dag = DAG(
'test_operator_clear',
start_date=DEFAULT_DATE,
end_date=DEFAULT_DATE + datetime.timedelta(days=10),
)
op1 = DummyOperator(task_id='bash_op', owner='test', dag=dag)
op2 = DummyOperator(task_id='dummy_op', owner='test', dag=dag, retries=1)
op2.set_upstream(op1)
ti1 = TI(task=op1, execution_date=DEFAULT_DATE)
ti2 = TI(task=op2, execution_date=DEFAULT_DATE)
dag.create_dagrun(
execution_date=ti1.execution_date,
state=State.RUNNING,
run_type=DagRunType.SCHEDULED,
)
ti2.run()
# Dependency not met
assert ti2.try_number == 1
assert ti2.max_tries == 1
op2.clear(upstream=True)
ti1.run()
ti2.run(ignore_ti_state=True)
assert ti1.try_number == 2
# max_tries is 0 because there is no task instance in db for ti1
# so clear won't change the max_tries.
assert ti1.max_tries == 0
assert ti2.try_number == 2
# try_number (0) + retries(1)
assert ti2.max_tries == 1