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

Commit 6294208

Browse files
eviziteiashb
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 fc785a5 commit 6294208

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
@@ -1686,14 +1686,6 @@ def signal_handler(signum, frame):
16861686
self.handle_failure(e, test_mode, context)
16871687
raise
16881688

1689-
# Recording SUCCESS
1690-
self.end_date = timezone.utcnow()
1691-
self.set_duration()
1692-
if not test_mode:
1693-
session.add(Log(self.state, self))
1694-
session.merge(self)
1695-
session.commit()
1696-
16971689
# Success callback
16981690
try:
16991691
if task.on_success_callback:
@@ -1702,6 +1694,12 @@ def signal_handler(signum, frame):
17021694
self.log.error("Failed when executing success callback")
17031695
self.log.exception(e3)
17041696

1697+
# Recording SUCCESS
1698+
self.end_date = timezone.utcnow()
1699+
self.set_duration()
1700+
if not test_mode:
1701+
session.add(Log(self.state, self))
1702+
session.merge(self)
17051703
session.commit()
17061704

17071705
@provide_session

tests/models.py

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

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

25622596
class ClearTasksTest(unittest.TestCase):
25632597

0 commit comments

Comments
 (0)