Asquator commented on code in PR #61274:
URL: https://github.com/apache/airflow/pull/61274#discussion_r2775447876


##########
airflow-core/src/airflow/models/dagrun.py:
##########
@@ -1410,24 +1416,44 @@ def notify_dagrun_state_changed(self, msg: str):
         # or LocalTaskJob, so we don't want to "falsely advertise" we notify 
about that
 
     @provide_session
-    def get_last_ti(self, dag: SerializedDAG, session: Session = NEW_SESSION) 
-> TI | None:
-        """Get Last TI from the dagrun to build and pass Execution context 
object from server to then run callbacks."""
+    def get_first_ti_causing_failure(self, dag: SerializedDAG, session: 
Session = NEW_SESSION) -> TI | None:  
+        """  
+        Get the first task instance that would cause a leaf task to fail the 
run.
+        """
+
         tis = self.get_task_instances(session=session)
-        # tis from a dagrun may not be a part of dag.partial_subset,
-        # since dag.partial_subset is a subset of the dag.
-        # This ensures that we will only use the accessible TI
-        # context for the callback.
+
+        failed_leaf_tis = [  
+            ti for ti in self._tis_for_dagrun_state(dag=dag, tis=tis)  
+            if ti.state in State.failed_states  
+        ]
+          
+        if not failed_leaf_tis:
+            return None  
+
         if dag.partial:
-            tis = [ti for ti in tis if not ti.state == State.NONE]
-        # filter out removed tasks
-        tis = natsorted(
-            (ti for ti in tis if ti.state != TaskInstanceState.REMOVED),
-            key=lambda ti: ti.task_id,
-        )
-        if not tis:
-            return None
-        ti = tis[-1]  # get last TaskInstance of DagRun
-        return ti
+            tis = [
+                ti for ti in tis if not ti.state in (
+                    State.NONE, TaskInstanceState.REMOVED
+                )
+            ]
+
+        # Collect all task IDs on failure paths
+        failure_path_task_ids = set()
+        for failed_leaf in failed_leaf_tis:
+            leaf_task = dag.get_task(failed_leaf.task_id)
+            upstream_ids = leaf_task.get_flat_relative_ids(upstream=True)
+            failure_path_task_ids.update(upstream_ids)
+            failure_path_task_ids.add(failed_leaf.task_id)
+
+        # Find failed tasks on possible failure paths
+        failed_on_paths = [  
+            ti for ti in tis
+            if ti.task_id in failure_path_task_ids and ti.state == 
State.FAILED  
+        ]
+          
+        return min(failed_on_paths, key=lambda ti: ti.end_date, default=None)

Review Comment:
   > Why shouldn't we return multiple tasks if a few of them were running 
concurrently?
   
   Unfortunately, the current `Context` has to provide at least one TI under 
the `"ti"` key, even on DAG failure callbacks. I don't know whether it's good 
or bad, probably bad, but I decided to adhere to this rule for back comp. I 
think the context user can still access other tasks in the DAG, can't he?
   
   > And why filter by end_date and not start_date?
   
   A good point. Logically, I wanted to return the task that ended last and 
could cause the DAG to fail. On the second thought, maybe it will be wiser to 
filter by `start_date` since many tasks will be failed manually after the DAG 
fails.



-- 
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.

To unsubscribe, e-mail: [email protected]

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

Reply via email to