Skip to content
This repository was archived by the owner on Apr 3, 2024. It is now read-only.

Commit e61b945

Browse files
authored
Merge pull request #66 from github/backfillkeyerror
Ensure backfill won't crash
2 parents 5946cf1 + 221aa50 commit e61b945

File tree

1 file changed

+21
-10
lines changed

1 file changed

+21
-10
lines changed

airflow/jobs/backfill_job.py

Lines changed: 21 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,14 @@
5050
from airflow.utils.types import DagRunType
5151

5252

53+
def _pop_running_ti(ti_status, reduced_key):
54+
try:
55+
ti_status.running.pop(reduced_key)
56+
except KeyError:
57+
# the task is not running tracking dict
58+
pass
59+
60+
5361
class BackfillJob(BaseJob):
5462
"""
5563
A backfill job consists of a dag or subdag for a specific time range. It
@@ -203,33 +211,39 @@ def _update_counters(self, ti_status, session=None):
203211
if filter_for_tis is not None:
204212
refreshed_tis = session.query(TI).filter(filter_for_tis).all()
205213

214+
self.log.debug("ti_status: %s", ti_status)
206215
for ti in refreshed_tis:
207216
# Here we remake the key by subtracting 1 to match in memory information
208217
reduced_key = ti.key.reduced
209218
if ti.state == State.SUCCESS:
210219
ti_status.succeeded.add(reduced_key)
211220
self.log.debug("Task instance %s succeeded. Don't rerun.", ti)
212-
ti_status.running.pop(reduced_key)
221+
self.log.debug("reduced_key: %s", reduced_key)
222+
_pop_running_ti(ti_status, reduced_key)
213223
continue
214224
if ti.state == State.SKIPPED:
215225
ti_status.skipped.add(reduced_key)
216226
self.log.debug("Task instance %s skipped. Don't rerun.", ti)
217-
ti_status.running.pop(reduced_key)
227+
self.log.debug("reduced_key: %s", reduced_key)
228+
_pop_running_ti(ti_status, reduced_key)
218229
continue
219230
if ti.state == State.FAILED:
220231
self.log.error("Task instance %s failed", ti)
232+
self.log.debug("reduced_key: %s", reduced_key)
221233
ti_status.failed.add(reduced_key)
222-
ti_status.running.pop(reduced_key)
234+
_pop_running_ti(ti_status, reduced_key)
223235
continue
224236
# special case: if the task needs to run again put it back
225237
if ti.state == State.UP_FOR_RETRY:
226238
self.log.warning("Task instance %s is up for retry", ti)
227-
ti_status.running.pop(reduced_key)
239+
self.log.debug("reduced_key: %s", reduced_key)
240+
_pop_running_ti(ti_status, reduced_key)
228241
ti_status.to_run[ti.key] = ti
229242
# special case: if the task needs to be rescheduled put it back
230243
elif ti.state == State.UP_FOR_RESCHEDULE:
231244
self.log.warning("Task instance %s is up for reschedule", ti)
232-
ti_status.running.pop(reduced_key)
245+
self.log.debug("reduced_key: %s", reduced_key)
246+
_pop_running_ti(ti_status, reduced_key)
233247
ti_status.to_run[ti.key] = ti
234248
# special case: The state of the task can be set to NONE by the task itself
235249
# when it reaches concurrency limits. It could also happen when the state
@@ -242,12 +256,9 @@ def _update_counters(self, ti_status, session=None):
242256
"reaching concurrency limits. Re-adding task to queue.",
243257
ti,
244258
)
259+
self.log.debug("reduced_key: %s", reduced_key)
245260
tis_to_be_scheduled.append(ti)
246-
try:
247-
ti_status.running.pop(reduced_key)
248-
except KeyError:
249-
# the task is not running
250-
pass
261+
_pop_running_ti(ti_status, reduced_key)
251262
ti_status.to_run[ti.key] = ti
252263

253264
# Batch schedule of task instances

0 commit comments

Comments
 (0)