Skip to content

Conversation

@kaxil
Copy link
Member

@kaxil kaxil commented Aug 29, 2025

closes #55045

Clearing a DAG with running tasks causes them to get stuck in RESTARTING state indefinitely, with continuous heartbeat timeout events. This is a regression in Airflow 3.x where the scheduler fails to process executor completion events for RESTARTING tasks. This change also preserves logs.

Error Symptoms

  • Tasks remain in RESTARTING state after clearing
  • Continuous heartbeat timeout events: 404 Not Found or 409 Conflict
  • Tasks never transition to retry or completion
  • Executor reports SUCCESS but task state doesn't update

Root Cause

The scheduler's _process_executor_events() function has two gatekeeping mechanisms that filter which executor events get processed:

  1. First filter (lines 769-775): Only processes events for certain states
  2. Second filter (lines 859-864): Validates task instance state matches executor event

Both filters were missing TaskInstanceState.RESTARTING, causing the scheduler to ignore when tasks in RESTARTING state were successfully terminated by the executor.

Historical Context

The RESTARTING state was introduced in Airflow 2.2.0 (PR #16681) to handle cleared running tasks gracefully:

  • Prevent cleared tasks from transitioning to FAILED (like SHUTDOWN did)
  • Allow tasks to be retried instead of marked as failed
  • Originally relied on "zombie detection" mechanism in Airflow 2.x

In Airflow 3.x, the zombie detection mechanism was removed with the transition from Job-based to Task SDK architecture, but the scheduler was never updated to handle RESTARTING tasks in executor events.

Why not immediately set state to None

The RESTARTING state serves a critical purpose in the clearing workflow:

  1. Race condition prevention: When clear_task_instances() is called, it immediately:

    • Generates a new ti.id via prepare_db_for_next_try()
    • Sets state to RESTARTING
    • Commits it to DB
  2. Running task process continues: The task process keeps running with the old ti.id and attempts heartbeating, which fails with 404 Not Found since the database now has a new ti.id.

  3. Executor termination: The executor eventually kills the running process and reports SUCCESS for the terminated task.

  4. Scheduler coordination: The RESTARTING state signals to the scheduler that this task was intentionally cleared and should be retried (not failed) when the executor confirms termination.

Setting the state directly to None during clearing would lose this coordination mechanism and could lead to race conditions where the scheduler processes a "completed" task before the executor has terminated the running process.

Future Considerations

This fix resolves the immediate regression but raises design questions for future improvements:

  1. Callbacks: Should on_retry_callback be triggered for cleared tasks?
  2. Email notifications: Should email_on_retry be sent?

These questions are deferred to maintain scope and can be addressed in follow-up discussions if needed.

Logs are preserved with this change:
image

image

… state

Clearing a DAG with running tasks causes them to get stuck in RESTARTING state
indefinitely with continuous heartbeat timeout events. This was a regression in
Airflow 3.x where the scheduler failed to process executor completion events for
RESTARTING tasks.

The scheduler's _process_executor_events() function has two gatekeeping mechanisms
that filter executor events, both of which were missing TaskInstanceState.RESTARTING.
This caused the scheduler to ignore when tasks in RESTARTING state were successfully
terminated by the executor.

Changes:
- Add RESTARTING to executor event processing filters in scheduler
- Add special handling for successful termination of RESTARTING tasks
- Set RESTARTING tasks to None (scheduled) state with adjusted max_tries
- Update task lifecycle diagram to reflect RESTARTING → None transition
- Add regression test for max_tries edge case with proper mock spec

This ensures cleared running tasks can retry immediately without getting stuck,
consistent with clearing behavior for non-running tasks.

Fixes apache#55045
@kaxil kaxil requested review from XD-DENG and ashb as code owners August 29, 2025 22:08
@boring-cyborg boring-cyborg bot added area:Scheduler including HA (high availability) scheduler kind:documentation labels Aug 29, 2025
@kaxil kaxil requested review from ephraimbuddy and rawwar August 29, 2025 22:08
@rawwar
Copy link
Contributor

rawwar commented Aug 30, 2025

Tested this locally and it works.

@eladkal eladkal added the backport-to-v3-1-test Mark PR with this label to backport to v3-1-test branch label Aug 30, 2025
@eladkal eladkal added this to the Airflow 3.0.7 milestone Aug 30, 2025
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.

This will work, but is there any way we could do this without relying on executor events in the future?

@eladkal
Copy link
Contributor

eladkal commented Aug 30, 2025

Should on_retry_callback be triggered for cleared tasks?

In Airflow 2 when I cleared a task I did not expect on_retry_callback to be executed. I don't think I've seen anyone expecting this. Why would this be an issue specifically now in Airflow 3?

@kaxil
Copy link
Member Author

kaxil commented Aug 30, 2025

This will work, but is there any way we could do this without relying on executor events in the future?

Yeah, possible to improve in future. Currently, we mainly need it to only run after the current task attempt is killed. Otherwise they could run in parallel i.e new attempt can start running before the first one is fully terminated.

@kaxil kaxil merged commit f9adaad into apache:main Aug 30, 2025
62 checks passed
@kaxil kaxil deleted the fix-clearing-when-running branch August 30, 2025 08:39
@github-actions
Copy link

Backport failed to create: v3-0-test. View the failure log Run details

Status Branch Result
v3-0-test Commit Link

You can attempt to backport this manually by running:

cherry_picker f9adaad v3-0-test

This should apply the commit to the v3-0-test branch and leave the commit in conflict state marking
the files that need manual conflict resolution.

After you have resolved the conflicts, you can continue the backport process by running:

cherry_picker --continue

@kaxil
Copy link
Member Author

kaxil commented Aug 30, 2025

Should on_retry_callback be triggered for cleared tasks?

In Airflow 2 when I cleared a task I did not expect on_retry_callback to be executed. I don't think I've seen anyone expecting this. Why would this be an issue specifically now in Airflow 3?

Mainly cause in 2.x, it first went to UP_FOR_RETRY state after RESTARTING so on_retry_callback could have been executed if set.

mangal-vairalkar pushed a commit to mangal-vairalkar/airflow that referenced this pull request Aug 30, 2025
nothingmin pushed a commit to nothingmin/airflow that referenced this pull request Sep 2, 2025
@kaxil kaxil modified the milestones: Airflow 3.0.7, Airflow 3.1.0 Sep 13, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Sep 30, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 1, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 2, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 3, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 4, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 5, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 5, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 7, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 8, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 9, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 10, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 11, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 12, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 14, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 15, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 17, 2025
abdulrahman305 bot pushed a commit to abdulrahman305/airflow that referenced this pull request Oct 19, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:Scheduler including HA (high availability) scheduler backport-to-v3-1-test Mark PR with this label to backport to v3-1-test branch kind:documentation

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Clearing a dag in the running state leads to tasks getting stuck in restarting, and emit heartbeat timeout events continuously.

4 participants