Skip to content
This repository was archived by the owner on Mar 15, 2023. It is now read-only.

Commit c5322f4

Browse files
eviziteiwyndhblb
authored andcommitted
[AIRFLOW-2865] Call success_callback before updating task state (apache#4082)
In cases where the success callback takes variable time, it's possible for it to interrupted by the heartbeat process. This is because the heartbeat process looks for tasks that are no longer in the "running" state but are still executing and reaps them. This commit reverses the order of callback invocation and state updating so that the "SUCCESS" state for the task isn't committed to the database until after the success callback has finished.
1 parent f4cb395 commit c5322f4

File tree

2 files changed

+40
-8
lines changed

2 files changed

+40
-8
lines changed

airflow/models.py

+6-8
Original file line numberDiff line numberDiff line change
@@ -1697,14 +1697,6 @@ def signal_handler(signum, frame):
16971697
self.handle_failure(e, test_mode, context)
16981698
raise
16991699

1700-
# Recording SUCCESS
1701-
self.end_date = timezone.utcnow()
1702-
self.set_duration()
1703-
if not test_mode:
1704-
session.add(Log(self.state, self))
1705-
session.merge(self)
1706-
session.commit()
1707-
17081700
# Success callback
17091701
try:
17101702
if task.on_success_callback:
@@ -1713,6 +1705,12 @@ def signal_handler(signum, frame):
17131705
self.log.error("Failed when executing success callback")
17141706
self.log.exception(e3)
17151707

1708+
# Recording SUCCESS
1709+
self.end_date = timezone.utcnow()
1710+
self.set_duration()
1711+
if not test_mode:
1712+
session.add(Log(self.state, self))
1713+
session.merge(self)
17161714
session.commit()
17171715

17181716
@provide_session

tests/models.py

+34
Original file line numberDiff line numberDiff line change
@@ -2559,6 +2559,40 @@ def test_set_duration_empty_dates(self):
25592559
ti.set_duration()
25602560
self.assertIsNone(ti.duration)
25612561

2562+
def test_success_callbak_no_race_condition(self):
2563+
class CallbackWrapper(object):
2564+
def wrap_task_instance(self, ti):
2565+
self.task_id = ti.task_id
2566+
self.dag_id = ti.dag_id
2567+
self.execution_date = ti.execution_date
2568+
self.task_state_in_callback = ""
2569+
self.callback_ran = False
2570+
2571+
def success_handler(self, context):
2572+
self.callback_ran = True
2573+
session = settings.Session()
2574+
temp_instance = session.query(TI).filter(
2575+
TI.task_id == self.task_id).filter(
2576+
TI.dag_id == self.dag_id).filter(
2577+
TI.execution_date == self.execution_date).one()
2578+
self.task_state_in_callback = temp_instance.state
2579+
cw = CallbackWrapper()
2580+
dag = DAG('test_success_callbak_no_race_condition', start_date=DEFAULT_DATE,
2581+
end_date=DEFAULT_DATE + datetime.timedelta(days=10))
2582+
task = DummyOperator(task_id='op', email='test@test.test',
2583+
on_success_callback=cw.success_handler, dag=dag)
2584+
ti = TI(task=task, execution_date=datetime.datetime.now())
2585+
ti.state = State.RUNNING
2586+
session = settings.Session()
2587+
session.merge(ti)
2588+
session.commit()
2589+
cw.wrap_task_instance(ti)
2590+
ti._run_raw_task()
2591+
self.assertTrue(cw.callback_ran)
2592+
self.assertEqual(cw.task_state_in_callback, State.RUNNING)
2593+
ti.refresh_from_db()
2594+
self.assertEqual(ti.state, State.SUCCESS)
2595+
25622596

25632597
class ClearTasksTest(unittest.TestCase):
25642598

0 commit comments

Comments
 (0)