Skip to content

Conversation

@amoghrajesh
Copy link
Contributor

@amoghrajesh amoghrajesh commented Dec 20, 2024

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.

image (7)

We can use the above state change diagram and handle the retry state while handling failed state. 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.

Approach chosen

  • We define a new state: FAIL_WITHOUT_RETRY, if the task runner sends a request that it wants to FAIL_WITHOUT_RETRY, we do not need to retry, this is done so that we prioritise retry over no retry. The cases where we should retry is far more than when we shouldn't.
  • FAIL_WITHOUT_RETRY is sent out for cases such as AirflowFailException or AirflowSensorTimeout.

Handling on task runner and execution side

  1. The task is executed normally, and in cases when there is a failure, it is handled in this way:
    For AirflowFailException, AirflowSensorTimeout, AirflowTaskTerminated, send FAIL_WITHOUT_RETRY
    For any other case, send FAIL.

  2. If FAILED state is sent, we attempt the retry.

Handling at execution API

  1. For the case when we receive FAILED,
        elif ti_patch_payload.state == State.FAILED:
            if _is_eligible_to_retry(previous_state, try_number, max_tries):
                updated_state = State.UP_FOR_RETRY
            else:
                updated_state = State.FAILED
  1. For case where we receive FAIL_WITHOUT_RETRY, just fail.

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)

TI details with max_tries
image (4)

Try number in grid view
image (5)

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

Ti detiails with 0 max_tries:
image

Try number in grid view
image

============

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

^ Add meaningful description above
Read the Pull Request Guidelines for more information.
In case of fundamental code changes, an Airflow Improvement Proposal (AIP) is needed.
In case of a new dependency, check compliance with the ASF 3rd Party License Policy.
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.

@amoghrajesh amoghrajesh requested review from ashb and kaxil and removed request for ephraimbuddy and pierrejeambrun December 20, 2024 10:26
@amoghrajesh amoghrajesh added the area:task-execution-interface-aip72 AIP-72: Task Execution Interface (TEI) aka Task SDK label Dec 20, 2024
@amoghrajesh
Copy link
Contributor Author

If we agree on the approach, I will work on the tests.

@amoghrajesh amoghrajesh self-assigned this Dec 23, 2024
@amoghrajesh
Copy link
Contributor Author

amoghrajesh commented Dec 23, 2024

It probably might also be a good idea to de couple the entire payload construction out of TaskState. We might need to handle future cases where "fail" ti state has specific attributes like fail_stop for example.

Somewhat like:

class TaskState(BaseModel):
    """
    Update a task's state.

    If a process exits without sending one of these the state will be derived from the exit code:
    - 0 = SUCCESS
    - anything else = FAILED
    """

    state: TerminalTIState
    end_date: datetime | None = None
    type: Literal["TaskState"] = "TaskState"

class FailTask(BaseModel):
    """
    Update a task's state to failed. Inherits TaskState to be able to define attributes specific to
    failure state.
    """

    state: TerminalTIState.FAILED
    task_retries: int | None = None
    fail_stop: bool = False
    type: Literal["FailTask"] = "FailTask"

Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove task_retries from the payloads

@amoghrajesh amoghrajesh requested a review from ashb December 26, 2024 11:30
@ashb
Copy link
Member

ashb commented Dec 27, 2024

This still doesn't feel right to me @amoghrajesh -- I think "should_fail" as a property on the task state, and all the plumbing we need to set through the layers is a sign that the design isn't right yet.

Spit-balling here, but what if we had a different state, so instead of sending TerminalTIState.FAILED we sent TerminalTIState.FAIL_WITHOUT_RETRY or something. By doing that it would mean we wouldn't need a new object process<->supervisor comms, nor _fail_request_sent as that would be handled by the existing terminal state mechanism

@amoghrajesh amoghrajesh force-pushed the AIP72-retries branch 2 times, most recently from 835de51 to 70c7c73 Compare December 27, 2024 16:18
@amoghrajesh
Copy link
Contributor Author

@ashb tested out with retries as 0, 3, None. Works as expected.
image

@amoghrajesh amoghrajesh requested a review from ashb December 27, 2024 16:24
Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Few comments/nits, then LGTM!

@amoghrajesh
Copy link
Contributor Author

Resolved all conversations. Merging this PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:task-execution-interface-aip72 AIP-72: Task Execution Interface (TEI) aka Task SDK area:task-sdk

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Handle Task retries in Task SDK

3 participants