You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2018/08/07 22:18:44 UTC

[GitHub] gwax closed pull request #2482: AIRFLOW-1460 clear "REMOVED" tis on DagRun update

gwax closed pull request #2482: AIRFLOW-1460 clear "REMOVED" tis on DagRun update
URL: https://github.com/apache/incubator-airflow/pull/2482
 
 
   

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 d1f8e59fe3..ff059b4fcb 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -4255,6 +4255,7 @@ def update_state(self, session=None):
             # skip in db?
             if ti.state == State.REMOVED:
                 tis.remove(ti)
+                session.delete(ti)
             else:
                 ti.task = dag.get_task(ti.task_id)
 
diff --git a/tests/models.py b/tests/models.py
index cf2734b746..b8157a664b 100644
--- a/tests/models.py
+++ b/tests/models.py
@@ -487,6 +487,34 @@ def test_dagrun_success_conditions(self):
         state = dr.update_state()
         self.assertEqual(State.FAILED, state)
 
+    def test_dagrun_clear_removed(self):
+        session = settings.Session()
+
+        dag = DAG(
+            'test_dagrun_clear_removed',
+            start_date=DEFAULT_DATE,
+            default_args={'owner': 'owner1'})
+
+        with dag:
+            op = DummyOperator(task_id='A')
+
+        dag.clear()
+
+        now = datetime.datetime.now()
+        dr = dag.create_dagrun(run_id='test_dagrun_clear_removed',
+                               state=State.RUNNING,
+                               execution_date=now,
+                               start_date=now)
+
+        ti_op = dr.get_task_instance(task_id=op.task_id)
+        ti_op.set_state(state=State.REMOVED, session=session)
+
+        self.assertEqual(
+            dr.get_task_instance(task_id=op.task_id).state,
+            State.REMOVED)
+        dr.update_state()
+        self.assertIsNone(dr.get_task_instance(task_id=op.task_id))
+
     def test_get_task_instance_on_empty_dagrun(self):
         """
         Make sure that a proper value is returned when a dagrun has no task instances


 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services