amoghrajesh opened a new pull request, #45106:
URL: https://github.com/apache/airflow/pull/45106

   <!--
    Licensed to the Apache Software Foundation (ASF) under one
    or more contributor license agreements.  See the NOTICE file
    distributed with this work for additional information
    regarding copyright ownership.  The ASF licenses this file
    to you under the Apache License, Version 2.0 (the
    "License"); you may not use this file except in compliance
    with the License.  You may obtain a copy of the License at
   
      http://www.apache.org/licenses/LICENSE-2.0
   
    Unless required by applicable law or agreed to in writing,
    software distributed under the License is distributed on an
    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    KIND, either express or implied.  See the License for the
    specific language governing permissions and limitations
    under the License.
    -->
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   closes: #44351
   
   "Retries" are majorly handled in airflow 2.x in here: 
https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L3082-L3101.
   
   The idea here is that in case a task is retry able, defined by 
https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L1054-L1073,
 the task is marked as "up_for_retry". Rest of the part is taken care by the 
scheduler loop normally if the ti state is marked correctly.
   
   Coming to task sdk, we cannot perform validations such as 
https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L1054-L1073
 in the task runner / sdk side because we do not have/ should not have access 
to the database. Instead of having API handler and states for "up_for_retry", 
we can handle it when we are handling failures - which we do by calling the 
https://github.com/apache/airflow/blob/main/airflow/api_fastapi/execution_api/routes/task_instances.py#L160-L212
 API endpoint. If we can send in enough data to the api handler in the 
execution API, we should be able to handle the cases of retry well.
   
   ### What needs to be done for porting this to `task_sdk`?
   1. Defining "try_number", "max_retries" for task instances ---> not needed 
because this is handled already in the scheduler side of things / parsing time 
and not at execution time, so we do not need to handle it. It is handled here 
https://github.com/apache/airflow/blob/main/airflow/models/dagrun.py#L1445-L1471
 when a dag run is created and it is initialised with the initial values: 
max_tries(https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L1809)
 and 
try_number(https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L1808)
   
   2. We need to have a mechanism that can send a signal from the task runner 
if retries are defined. We will send this in this fashion:
   task runner informs the supervisor while failing that it needs to retry -> 
supervisor sends a normal request to the client (but with task_retries defined) 
-> client sends a normal API request (TITerminalStatePayload) to the execution 
API but with task_retries
   
   3. At the execution API, we receive the request and perform a check to check 
if the Ti is eligible for retry, if it is, we mark it as "up_for_retry", the 
rest of things are taken care by the scheduler.
   
   
   ### Testing results
   Right now the PR is meant to handle `BaseException` -- will extend to all 
other eligible TI exceptions in follow ups.
   
   
   #### Scenario 1: With retries = 3 defined. 
   
   DAG:
   ```
   import sys
   from time import sleep
   
   from airflow import DAG
   from airflow.providers.standard.operators.python import PythonOperator
   from datetime import datetime, timedelta
   from airflow.exceptions import AirflowTaskTimeout
   
   
   def print_hello():
       1//0
   
   with DAG(
       dag_id="abcd",
       schedule=None,
       catchup=False,
       tags=["demo"],
   ) as dag:
       hello_task = PythonOperator(
           task_id="say_hello",
           python_callable=print_hello,
           retries=3
       )
   ```
   Rightly marked as "up_for_retry"
   ![image 
(3)](https://github.com/user-attachments/assets/a53ca5c1-4e86-421f-bcd4-80f41ed61816)
   
   TI details with max_tries
   ![image 
(4)](https://github.com/user-attachments/assets/54f494b1-555d-438a-b5af-b3c203163bdb)
   
   Try number in grid view
   ![image 
(5)](https://github.com/user-attachments/assets/fd47718d-3a56-45be-9cc9-d3fc4d595170)
   
   
   #### Scenario 2: With retries not defined. 
   
   DAG:
   ```
   import sys
   from time import sleep
   
   from airflow import DAG
   from airflow.providers.standard.operators.python import PythonOperator
   from datetime import datetime, timedelta
   from airflow.exceptions import AirflowTaskTimeout
   
   
   def print_hello():
       1//0
   
   with DAG(
       dag_id="abcd",
       schedule=None,
       catchup=False,
       tags=["demo"],
   ) as dag:
       hello_task = PythonOperator(
           task_id="say_hello",
           python_callable=print_hello,
       )
   ```
   
   Rightly marked as "failed"
   <img width="1724" alt="image" 
src="https://github.com/user-attachments/assets/d5cce08c-39b1-499d-a87d-43d2cc2e074b";
 />
   
   Ti detiails with 0 max_tries:
   <img width="1724" alt="image" 
src="https://github.com/user-attachments/assets/9a862cc5-6310-4f33-a5cc-a0759bc72b1f";
 />
   
   Try number in grid view
   <img width="1724" alt="image" 
src="https://github.com/user-attachments/assets/6b0811d7-d01e-4bb4-92e4-98f679e6f935";
 />
   
   ============
   
   ### Pending:
   
   - [ ] UT coverage for execution API for various scenarios
   - [ ] UT coverage for supervisor and task_runner, client
   - [ ] Extending to various other scenarios when retry is needed -- eg: 
AirflowTaskTimeout / AirflowException
   
   
   <!-- Please keep an empty line above the dashes. -->
   ---
   **^ Add meaningful description above**
   Read the **[Pull Request 
Guidelines](https://github.com/apache/airflow/blob/main/contributing-docs/05_pull_requests.rst#pull-request-guidelines)**
 for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal 
([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals))
 is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party 
License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a 
newsfragment file, named `{pr_number}.significant.rst` or 
`{issue_number}.significant.rst`, in 
[newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


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