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 2022/01/21 14:04:05 UTC

[GitHub] [airflow] yuqian90 opened a new issue #21017: Opening "Task Instance Details" page in web UI unintentionally resets TaskInstance.max_tries

yuqian90 opened a new issue #21017:
URL: https://github.com/apache/airflow/issues/21017


   ### Apache Airflow version
   
   2.2.3 (latest released)
   
   ### What happened
   
   `TaskInstance.max_tries` is a cumulative value that's stored in the database. Whenever a failed `TaskInstance` is cleared, its `max_tries` is incremented by `task.retries - 1`. However, the increased `max_tries` value of a `TaskInstance` is reset to the original value (`task.retries`) whenever a user opens the `Task Instance Details` page in the web UI. This is because `www/views.py` queries `TaskInstance` and then calls `refresh_from_task()` on it in a few places, which changes the attributes of the `TaskInstance`. The `session` used in `www/views.py` is then auto committed and the updated values such as `max_tries` are saved to the database.
   
   This looks like a bug. So far the only effect I noticed is `max_tries` gets reset causing sensors in "reschedule" mode to timeout prematurely. It happens because of the following lines in `sensors/base.py`. When `max_tries` is reset, this code always gets the `first_try_number` 1 which may have happened a long time ago and thinks the sensor times out.
   
   ```
           if self.reschedule:
   
               # If reschedule, use the start date of the first try (first try can be either the very
               # first execution of the task, or the first execution after the task was cleared.)
               first_try_number = context['ti'].max_tries - self.retries + 1
               task_reschedules = TaskReschedule.find_for_task_instance(
                   context['ti'], try_number=first_try_number
               )
   ```
   
   
   ### What you expected to happen
   
   Opening "Task Instance Details" page should not make any changes to the state of the TaskInstance in the database.
   
   ### How to reproduce
   
   Create a DAG like this:
   ```
   from datetime import datetime
   from airflow import DAG
   from airflow.sensors.python import PythonSensor
   
   with DAG(
       dag_id="reschedule_clear",
       start_date=datetime(2021, 1, 1),
       catchup=False,
       schedule_interval="@daily",
   ) as dag:
       task_1 = PythonSensor(
           task_id="task_1",
           python_callable=lambda: False,
           mode="reschedule",
           timeout=90,
           retries=5,
       )
   ```
   
   ```
   State of the TaskInstance after it failed for the first time:
   
   airflow=# select max_tries, try_number from task_instance ;
    max_tries | try_number
   -----------+------------
            5 |          1
   (1 row)
   
   After the failed TaskInstance was cleared:
   
   airflow=# select max_tries, try_number from task_instance ;
    max_tries | try_number
   -----------+------------
            6 |          1
   (1 row)
   
   After the TaskInstance failed for a second time:
   
   airflow=# select max_tries, try_number from task_instance ;
    max_tries | try_number
   -----------+------------
            6 |          2
   (1 row)
   
   
   After the TaskInstance is cleared a second time:
   
   airflow=# select max_tries, try_number from task_instance ;
    max_tries | try_number
   -----------+------------
            7 |          2
   (1 row)
   
   
   After user opens the "Task Instance Details" page in Web UI:
   
   airflow=# select max_tries, try_number from task_instance ;
    max_tries | try_number
   -----------+------------
            5 |          2
   (1 row)
   ```
   
   ### Operating System
   
   Any
   
   ### Versions of Apache Airflow Providers
   
   Any
   
   ### Deployment
   
   Virtualenv installation
   
   ### Deployment details
   
   Any
   
   ### Anything else
   
   Every time a user opens "Task Instance Details" page.
   
   ### 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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] jedcunningham closed issue #21017: Opening "Task Instance Details" page in web UI unintentionally resets TaskInstance.max_tries

Posted by GitBox <gi...@apache.org>.
jedcunningham closed issue #21017:
URL: https://github.com/apache/airflow/issues/21017


   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org