Skip to content

Conversation

@karenbraganz
Copy link
Collaborator

In issue #51301, it was reported that failure callbacks do not run for task instances that get stuck in queued and fail in Airflow 2.10.5. This is happening due to the changes introduced in PR #43520 . In this PR, logic was introduced to requeue tasks that get stuck in queued (up to two times by default) before failing them.

Previously, the executor's fail method was called when the task needed to be failed after max requeue attempts. This was replaced by the task instance's set_state method in the PR ti.set_state(TaskInstanceState.FAILED, session=session). Without the executor's fail method being called, failure callbacks will not be executed for such task instances. Therefore, I changed the code to call the executor's fail method instead in Airflow 3.

I have created PR #53038 to address this issue separately in Airflow 2.11.


^ 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 airflow-core/newsfragments.

@boring-cyborg boring-cyborg bot added the area:Scheduler including HA (high availability) scheduler label Jul 16, 2025
@karenbraganz karenbraganz self-assigned this Jul 16, 2025
@karenbraganz karenbraganz marked this pull request as ready for review July 16, 2025 22:04
Copy link
Contributor

@Nataneljpwd Nataneljpwd left a comment

Choose a reason for hiding this comment

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

Looks good!
Quick and simple solution which addresses the issue well

@karenbraganz karenbraganz added this to the Airflow 3.0.4 milestone Jul 18, 2025
Copy link
Member

@potiuk potiuk left a comment

Choose a reason for hiding this comment

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

LGTM - but others who are more executor-oriented can verify it @ashb @amoghrajesh @o-nikolas ?

@karenbraganz
Copy link
Collaborator Author

I was testing this PR out by repeatedly getting tasks stuck in queued. Initially, the task instances would get stuck in queued, fail, and the callbacks would run. After a few rounds of this, the below exception was raised:
image

According to the code, the session is not committed until after _maybe_requeue_stuck_ti completes running. Any idea what would cause the TaskInstance object to be detached from a session? As mentioned before, the code works as expected most of the time. This happens intermittently.

@Nataneljpwd
Copy link
Contributor

@karenbraganz it could be due to the expundge happening here

https://github.com/apache/airflow/blob/main/airflow-core%2Fsrc%2Fairflow%2Fjobs%2Fscheduler_job_runner.py#L1286

This detaches all the instance objects from the session, I couldn't follow the flow as I did not have time but I think it might be related, or at least it is a good place to start investigating from.

@karenbraganz
Copy link
Collaborator Author

@Nataneljpwd The session being expunged in the link you provided is not the same session being used in _maybe_requeue_stuck_ti. The function _maybe_requeue_stuck_ti uses the session that is passed to it when it is called in _handle_tasks_stuck_in_queued. This session is created in _handle_tasks_stuck_in_queued and is not committed until after _maybe_requeue_stuck_ti is completed, so I did not expect the TaskInstance to be detached from the session in _maybe_requeue_stuck_ti. Please let me know if I am mistaken about something.

@Nataneljpwd
Copy link
Contributor

@karenbraganz, I think that the issue might be that if there is more than 1 stuck task, it commits before it changed or requed all tasks, and so the other tasks are lost, and we can see the commit here, I would try moving the commit out of the loop and see if it changes anything, nevertheless, I would do it either way as we batch requests to the db that way which is better, but I think it might resolve the issue, thoug, I do not see that expire on commit is configured anywhere

@karenbraganz
Copy link
Collaborator Author

@Nataneljpwd I will try moving it out of the loop and run the tests again.

@ashb
Copy link
Member

ashb commented Jul 29, 2025

@karenbraganz A similar "detached instance error" was just fixed by Kaxil, but maybe we need another fix like #53838 (in a separate PR please)

@karenbraganz
Copy link
Collaborator Author

I moved session.commit() outside the loop and re-ran the repeated tests overnight. The first 45 task instances got stuck and failed as expected, but the 46th task instance that got stuck experienced the DetachedInstanceError again.

I will take a look at Kaxil's fix next.

@karenbraganz
Copy link
Collaborator Author

@ashb looks like eager loading the attributes is the solution. Still need to implement and test this out to confirm that it works.

Is there a reason why you want this in a separate PR? I would be eager loading attributes that are only used in this PR, so I thought it would be more appropriate to add that code in this PR itself.

For example, ti.dag_model.relative_fileloc is one of the attributes I would be eager loading, and this attribute is directly used in the code I am adding in this PR.

@ashb
Copy link
Member

ashb commented Jul 29, 2025

@karenbraganz Ah, if this PR is where you first access them, then yes making them eager load here is probably the right thing to do.

However one thing we need to consider is if eagerloading them is going to have performance impact on a rarely used case (I don't have the context on this path or PR to say either way), and if it might better to load it from the DB in some other way only when needed.

@karenbraganz karenbraganz changed the title Call executor fail method for stuck in queued tasks Allow failure callbacks for stuck in queued TIs that fail Jul 29, 2025
@karenbraganz
Copy link
Collaborator Author

Confirmed that eager loading resolves the issue. Working through the correct way to implement this since the attributes will only be used if the task is failed and has a failure callback.

@karenbraganz
Copy link
Collaborator Author

I was able to find a workaround for the DetachedInstanceError that does not involve eager loading. Right before the TaskCallbackRequest is created (where the error occurs), I added a condition to check whether the ti is detached. If it is, it will be merged into the session. I found that this prevents the issue from occurring without having to eager load any attributes.

I have implemented this in my latest commit. Please let me know if there are any objections to this workaround.

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.

LGTM. We'll merge this and backport it for 3.0.5 (Was too late for 3.0.4, and even though we are having an RC2 I don't want to introduce more code changes than we have to in order to get that release out)

@ashb ashb added the backport-to-v3-1-test Mark PR with this label to backport to v3-1-test branch label Aug 6, 2025
Copy link
Contributor

@o-nikolas o-nikolas left a comment

Choose a reason for hiding this comment

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

Nice, this looks great now! Thanks for sticking with it :)

@ashb ashb force-pushed the tqt-callbacks-af3 branch from ee70e81 to 4e88a5b Compare August 12, 2025 09:22
@ashb ashb merged commit 6da77b1 into apache:main Aug 12, 2025
57 checks passed
@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 6da77b1 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

ashb pushed a commit that referenced this pull request Aug 12, 2025
…#53435)

In issue #51301, it was reported that failure callbacks do not run for task instances that get stuck in queued and fail in Airflow 2.10.5. This is happening due to the changes introduced in PR #43520 . In this PR, logic was introduced to requeue tasks that get stuck in queued (up to two times by default) before failing them.

Previously, the executor's fail method was called when the task needed to be failed after max requeue attempts. This was replaced by the task instance's set_state method in the PR ti.set_state(TaskInstanceState.FAILED, session=session). Without the executor's fail method being called, failure callbacks will not be executed for such task instances. Therefore, I changed the code to call the executor's fail method instead in Airflow 3.
(cherry picked from commit 6da77b1)

Co-authored-by: Karen Braganza <karenbraganza15@gmail.com>
ashb pushed a commit that referenced this pull request Aug 12, 2025
…#53435)

In issue #51301, it was reported that failure callbacks do not run for task instances that get stuck in queued and fail in Airflow 2.10.5. This is happening due to the changes introduced in PR #43520 . In this PR, logic was introduced to requeue tasks that get stuck in queued (up to two times by default) before failing them.

Previously, the executor's fail method was called when the task needed to be failed after max requeue attempts. This was replaced by the task instance's set_state method in the PR ti.set_state(TaskInstanceState.FAILED, session=session). Without the executor's fail method being called, failure callbacks will not be executed for such task instances. Therefore, I changed the code to call the executor's fail method instead in Airflow 3.
(cherry picked from commit 6da77b1)

Co-authored-by: Karen Braganza <karenbraganza15@gmail.com>
ashb pushed a commit that referenced this pull request Aug 13, 2025
…#53435)

In issue #51301, it was reported that failure callbacks do not run for task instances that get stuck in queued and fail in Airflow 2.10.5. This is happening due to the changes introduced in PR #43520 . In this PR, logic was introduced to requeue tasks that get stuck in queued (up to two times by default) before failing them.

Previously, the executor's fail method was called when the task needed to be failed after max requeue attempts. This was replaced by the task instance's set_state method in the PR ti.set_state(TaskInstanceState.FAILED, session=session). Without the executor's fail method being called, failure callbacks will not be executed for such task instances. Therefore, I changed the code to call the executor's fail method instead in Airflow 3.
(cherry picked from commit 6da77b1)

Co-authored-by: Karen Braganza <karenbraganza15@gmail.com>
kaxil pushed a commit that referenced this pull request Aug 13, 2025
…#53435) (#54401)

In issue #51301, it was reported that failure callbacks do not run for task instances that get stuck in queued and fail in Airflow 2.10.5. This is happening due to the changes introduced in PR #43520 . In this PR, logic was introduced to requeue tasks that get stuck in queued (up to two times by default) before failing them.

Previously, the executor's fail method was called when the task needed to be failed after max requeue attempts. This was replaced by the task instance's set_state method in the PR ti.set_state(TaskInstanceState.FAILED, session=session). Without the executor's fail method being called, failure callbacks will not be executed for such task instances. Therefore, I changed the code to call the executor's fail method instead in Airflow 3.
(cherry picked from commit 6da77b1)

Co-authored-by: Karen Braganza <karenbraganza15@gmail.com>
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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants