ashb commented on a change in pull request #16182:
URL: https://github.com/apache/airflow/pull/16182#discussion_r646408397



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1417,6 +1420,37 @@ def _clean_tis_without_dagrun(self, session):
                     raise
             guard.commit()
 
+    @provide_session
+    def _missing_dag_file_cleanup(self, session: Session = None):
+        """Fails task instances and DagRuns of DAGs that no longer exist in 
the dag folder"""
+        states_to_check = State.unfinished - frozenset([State.NONE, 
State.SHUTDOWN])
+        tis = session.query(TI).filter(TI.state.in_(states_to_check)).all()
+        missing_dags = {}
+        dag_runs = set()
+        for ti in tis:
+            dag = self.dagbag.dags.get(ti.dag_id, None)
+            if not dag:
+                continue

Review comment:
       Loading the dag is an "expensive" operation, so this should be reworked 
to add a new method on dagbag: `has_dag` that does the following:
   
   - Checks the "local" cache, if it exists, and is not older than the 
confiured cache time, return True
   - Else check in the DB -- but crucially, only check for the row existing, 
don't load the full DagModel object, and crucially don't load the SerializedDag.

##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1417,6 +1420,37 @@ def _clean_tis_without_dagrun(self, session):
                     raise
             guard.commit()
 
+    @provide_session
+    def _missing_dag_file_cleanup(self, session: Session = None):
+        """Fails task instances and DagRuns of DAGs that no longer exist in 
the dag folder"""
+        states_to_check = State.unfinished - frozenset([State.NONE, 
State.SHUTDOWN])
+        tis = session.query(TI).filter(TI.state.in_(states_to_check)).all()
+        missing_dags = {}
+        dag_runs = set()
+        for ti in tis:
+            dag = self.dagbag.dags.get(ti.dag_id, None)
+            if not dag:
+                continue
+            if not os.path.exists(dag.fileloc):

Review comment:
       This isn't going to work for the case where the _file_ still exists, but 
that file no longer defines this DAG -- i.e. where you have more than one dag 
in a file and you delete one of them.
   
   To fix this properly we might have to fix 
https://github.com/apache/airflow/pull/11462 correctly.
   

##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1417,6 +1420,37 @@ def _clean_tis_without_dagrun(self, session):
                     raise
             guard.commit()
 
+    @provide_session
+    def _missing_dag_file_cleanup(self, session: Session = None):
+        """Fails task instances and DagRuns of DAGs that no longer exist in 
the dag folder"""
+        states_to_check = State.unfinished - frozenset([State.NONE, 
State.SHUTDOWN])
+        tis = session.query(TI).filter(TI.state.in_(states_to_check)).all()
+        missing_dags = {}
+        dag_runs = set()
+        for ti in tis:
+            dag = self.dagbag.dags.get(ti.dag_id, None)
+            if not dag:
+                continue

Review comment:
       And an optimization: if the dag is already in `missing_dags`, we don't 
need to check again here :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to