ashb closed pull request #4082: [AIRFLOW-2865] Call success_callback before 
updating task state
URL: https://github.com/apache/incubator-airflow/pull/4082
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/models.py b/airflow/models.py
index 8071585323..1b05f3c1a5 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -1671,14 +1671,6 @@ def signal_handler(signum, frame):
             self.handle_failure(e, test_mode, context)
             raise
 
-        # Recording SUCCESS
-        self.end_date = timezone.utcnow()
-        self.set_duration()
-        if not test_mode:
-            session.add(Log(self.state, self))
-            session.merge(self)
-        session.commit()
-
         # Success callback
         try:
             if task.on_success_callback:
@@ -1687,6 +1679,12 @@ def signal_handler(signum, frame):
             self.log.error("Failed when executing success callback")
             self.log.exception(e3)
 
+        # Recording SUCCESS
+        self.end_date = timezone.utcnow()
+        self.set_duration()
+        if not test_mode:
+            session.add(Log(self.state, self))
+            session.merge(self)
         session.commit()
 
     @provide_session
diff --git a/tests/models.py b/tests/models.py
index e7ca343ca1..c07ca19b4c 100644
--- a/tests/models.py
+++ b/tests/models.py
@@ -2469,6 +2469,40 @@ def test_set_duration_empty_dates(self):
         ti.set_duration()
         self.assertIsNone(ti.duration)
 
+    def test_success_callbak_no_race_condition(self):
+        class CallbackWrapper(object):
+            def wrap_task_instance(self, ti):
+                self.task_id = ti.task_id
+                self.dag_id = ti.dag_id
+                self.execution_date = ti.execution_date
+                self.task_state_in_callback = ""
+                self.callback_ran = False
+
+            def success_handler(self, context):
+                self.callback_ran = True
+                session = settings.Session()
+                temp_instance = session.query(TI).filter(
+                    TI.task_id == self.task_id).filter(
+                    TI.dag_id == self.dag_id).filter(
+                    TI.execution_date == self.execution_date).one()
+                self.task_state_in_callback = temp_instance.state
+        cw = CallbackWrapper()
+        dag = DAG('test_success_callbak_no_race_condition', 
start_date=DEFAULT_DATE,
+                  end_date=DEFAULT_DATE + datetime.timedelta(days=10))
+        task = DummyOperator(task_id='op', email='[email protected]',
+                             on_success_callback=cw.success_handler, dag=dag)
+        ti = TI(task=task, execution_date=datetime.datetime.now())
+        ti.state = State.RUNNING
+        session = settings.Session()
+        session.merge(ti)
+        session.commit()
+        cw.wrap_task_instance(ti)
+        ti._run_raw_task()
+        self.assertTrue(cw.callback_ran)
+        self.assertEqual(cw.task_state_in_callback, State.RUNNING)
+        ti.refresh_from_db()
+        self.assertEqual(ti.state, State.SUCCESS)
+
 
 class ClearTasksTest(unittest.TestCase):
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to