Skip to content

Conversation

@ashb
Copy link
Member

@ashb ashb commented Aug 26, 2022

This one was a fun one to track down, and was only a problem when the scheduler "expanded" the first mapped task (making it SKIPPED).

  • The scheduler looks at add_one and marks it as SKIPPED.
  • unfinished_tis contains add_one__1, and _are_premature_tis changes the state of add_one__1, (which just so happens to be a leaf task), and the check on line 584 essentially gets confused, as no one envisaged the states changing!

In a reverse of how this normally plays out, if the mini_scheduler in the LocalTaskJob was disabled then this example DAG would deadlock every time. (Since that mini scheduler only operates on a partial DAG it can't ever change the whole DagRun state.)

Fixes #25200

Ephraim had the idea of the fix, I just optomized it to not check the (possibly) large list everytime but only when a state was changed.


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

@ashb ashb requested review from XD-DENG and kaxil as code owners August 26, 2022 20:25
@ashb ashb added this to the Airflow 2.3.5 milestone Aug 26, 2022
@ashb
Copy link
Member Author

ashb commented Aug 26, 2022

/cc @collinmcnulty @ldacey I think this should fix the issue if you want to try it out.

@uranusjr uranusjr self-assigned this Aug 29, 2022
@ashb ashb force-pushed the fix-mapped-leaf-task-skip-dagrun-deadlock branch from 510ded2 to ab64e0f Compare August 30, 2022 11:15
ashb and others added 2 commits August 30, 2022 20:05
This one was a fun one to track down, and was only a problem when the
scheduler "expanded" the first mapped task (making it SKIPPED).

- The scheduler looks at `add_one` and marks it as SKIPPED.
- `unfinished_tis` contains add_one_1, and `_are_premature_tis` changes
  the state of `add_one__1`, (which just so happens to be a leaf task),
  and the check on line 584 essentially gets confused, as no one
  envisaged the states changing!

In a reverse of how this normally plays out, if the mini_scheduler in
the LocalTaskJob was disabled then this example DAG would deadlock every
time. (Since that mini scheduler only operates on a partial DAG it can't
ever change the whole DagRun state.)

Co-authored-by: Ephraim Anierobi <splendidzigy24@gmail.com>
@uranusjr uranusjr force-pushed the fix-mapped-leaf-task-skip-dagrun-deadlock branch from 940cb22 to d98ca51 Compare August 30, 2022 12:05
@ashb
Copy link
Member Author

ashb commented Aug 30, 2022

Mypy is confusing me:

airflow/ti_deps/deps/trigger_rule_dep.py:149: error: Cannot determine type of
"trigger_rule"  [has-type]
            trigger_rule = cast(TR, task.trigger_rule)

Can't get a "good" fix, having to add a type-ignore. (I'm getting conflicting behaviour. I can fix it either for all files, or for a single file, but not both)

@uranusjr uranusjr self-requested a review as a code owner August 31, 2022 07:27
)

self.trigger_rule = TriggerRule(trigger_rule)
self.trigger_rule: TriggerRule = TriggerRule(trigger_rule)
Copy link
Member Author

Choose a reason for hiding this comment

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

Mypy really can't work this out? :o

Copy link
Member

Choose a reason for hiding this comment

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

I think it has something to do with either (or a combination of) Enum having a custom metaclass, TriggerRule does multi-inheritance, and one of the parents is a built-in.

@ashb ashb merged commit 5697e9f into apache:main Sep 1, 2022
@ashb ashb deleted the fix-mapped-leaf-task-skip-dagrun-deadlock branch September 1, 2022 12:03
@ashb ashb modified the milestones: Airflow 2.3.5, Airflow 2.4.0 Sep 8, 2022
@ephraimbuddy ephraimbuddy added the type:bug-fix Changelog: Bug Fixes label Sep 13, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

type:bug-fix Changelog: Bug Fixes

Projects

None yet

Development

Successfully merging this pull request may close these issues.

DAG Run fails when chaining multiple empty mapped tasks

3 participants