vgupta3 opened a new issue, #26760:
URL: https://github.com/apache/airflow/issues/26760

   ### Apache Airflow version
   
   2.4.0
   
   ### What happened
   
   When a task fails in a DAG, the on_failure_callback [registered while 
creating the 
dag](https://github.com/apache/airflow/blob/21c40cd4cc7830bc17a4180d8dd0fa2cc4ed9cea/airflow/models/dag.py#L387)
 is triggered using the context of a [random task 
instance](https://github.com/apache/airflow/blob/21c40cd4cc7830bc17a4180d8dd0fa2cc4ed9cea/airflow/models/dag.py#L1296).
   
   ### What you think should happen instead
   
   The expectation is that one of the task instances that caused the dag 
failure should be used instead of a random task instance.
   
   
   ### How to reproduce
   
   Run the below dag.
   
   ```python
   import datetime
   from airflow.models.dag import DAG
   from airflow.operators.python import PythonOperator
   from airflow.operators.empty import EmptyOperator
   
   def all_bad():
       raise Exception("I have failed")
   
   def all_good():
       print("ALL GOOD")
   
   def failure_callback_dag(context):
       print("Inside failure_callback_dag")
       print(context["task_instance"])
       print(context["task"])
   
   with DAG(
           dag_id = "test_dag",
           schedule_interval=None,
           start_date=datetime.datetime(2021, 1, 1),
           catchup=False,
           on_failure_callback=failure_callback_dag
       ) as dag:
   
       start = EmptyOperator(
           task_id="start"
       )
       
       fail = PythonOperator(
           provide_context = True,
           task_id = "fail",
           python_callable = all_bad
       )
   
       passs = PythonOperator(
           provide_context = True,
           task_id = "pass",
           python_callable = all_good
       )
   
       start >> [passs, fail]
   
   ```
   
   From the dag processor logs:
   
   The context is from the task instance that has succeeded.
   ```
   [2022-09-28T18:28:14.465+0000] {logging_mixin.py:117} INFO - 
[2022-09-28T18:28:14.463+0000] {dag.py:1292} INFO - Executing dag callback 
function: <function failure_callback_dag at 0x7fd17ca18560>
   [2022-09-28T18:28:14.943+0000] {logging_mixin.py:117} INFO - Inside 
failure_callback_dag
   [2022-09-28T18:28:14.944+0000] {logging_mixin.py:117} INFO - <TaskInstance: 
test_dag.pass manual__2022-09-28T18:27:59.612118+00:00 [success]>
   [2022-09-28T18:28:14.944+0000] {logging_mixin.py:117} INFO - 
<Task(PythonOperator): pass>
   ```
   
   ### Operating System
   
   Debian GNU/Linux 10 (buster)
   
   ### Versions of Apache Airflow Providers
   
   Default providers that are present in the official airflow docker image.
   
   ### Deployment
   
   Docker-Compose
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   Not sure if it is an expected behaviour, incase it is it needs to be 
documented in 
[Callbacks](https://airflow.apache.org/docs/apache-airflow/2.4.0/logging-monitoring/callbacks.html).
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


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