Skip to content

Fix DatabricksWorkflowTaskGroup leaking TaskGroupContext on internal exception#66582

Merged
eladkal merged 6 commits into
apache:mainfrom
moomindani:providers/42164-databricks-workflow-taskgroup-cleanup
May 8, 2026
Merged

Fix DatabricksWorkflowTaskGroup leaking TaskGroupContext on internal exception#66582
eladkal merged 6 commits into
apache:mainfrom
moomindani:providers/42164-databricks-workflow-taskgroup-cleanup

Conversation

@moomindani
Copy link
Copy Markdown
Contributor

@moomindani moomindani commented May 8, 2026

DatabricksWorkflowTaskGroup.exit can raise mid-execution (for example when a child task does not implement _convert_to_databricks_workflow_task). Without try/finally, super().exit never runs and the workflow group is left on the global TaskGroupContext stack, breaking later DAGs with Cannot mix TaskGroups from different DAGs.

Wrap the body of __exit__ in try/finally so super().exit is always invoked and the context stack is cleaned up.

Picks up @ayudovin's earlier work in #55891 (closed by author with explicit permission to take over) and adds a regression test that exercises the bug path directly: an EmptyOperator (which lacks _convert_to_databricks_workflow_task) is added inside the workflow body, which makes __exit__ raise mid-way; the test then asserts that TaskGroupContext._context is empty, which fails on unfixed code and passes on fixed code.

closes: #42164


Was generative AI tooling used to co-author this PR?
  • Yes — Claude Code (Opus 4.7)

Generated-by: Claude Code (Opus 4.7) following the guidelines


  • Read the Pull Request Guidelines for more information. Note: commit author/co-author name and email in commits become permanently public when merged.
  • For fundamental code changes, an Airflow Improvement Proposal (AIP) is needed.
  • When adding dependency, check compliance with the ASF 3rd Party License Policy.
  • For significant user-facing changes create newsfragment: {pr_number}.significant.rst, in airflow-core/newsfragments. You can add this file in a follow-up commit after the PR is created so you know the PR number.

ayudovin and others added 6 commits May 8, 2026 15:44
The previous test raised an AirflowException from inside the workflow body,
but super().__exit__ is reached even on the unfixed code in that path
(the for-loop over tasks is empty), so the test passed without the fix.

Replace it with a regression test that triggers the raise inside
DatabricksWorkflowTaskGroup.__exit__ itself (via an EmptyOperator that
does not implement _convert_to_databricks_workflow_task) and asserts
TaskGroupContext is empty afterward — fails on unfixed code, passes on
fixed code. Also drops two unused imports introduced earlier.
Copy link
Copy Markdown
Contributor

@eladkal eladkal left a comment

Choose a reason for hiding this comment

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

LGTM

@eladkal eladkal merged commit 0a03470 into apache:main May 8, 2026
345 of 348 checks passed
arpitrathore pushed a commit to arpitrathore/airflow that referenced this pull request May 9, 2026
…l exception (apache#66582)

* Removing resources properly for DatabricksWorkflowGroup

* Add the unit test to check that super().__exit()__ called even exception happen

* remove unnecessary comment

* Fix unit tests

* Fix unit tests

* Strengthen test to actually exercise the bug

The previous test raised an AirflowException from inside the workflow body,
but super().__exit__ is reached even on the unfixed code in that path
(the for-loop over tasks is empty), so the test passed without the fix.

Replace it with a regression test that triggers the raise inside
DatabricksWorkflowTaskGroup.__exit__ itself (via an EmptyOperator that
does not implement _convert_to_databricks_workflow_task) and asserts
TaskGroupContext is empty afterward — fails on unfixed code, passes on
fixed code. Also drops two unused imports introduced earlier.

---------

Co-authored-by: artsiomyudovin <a.yudovin6695@gmail.com>
jason810496 pushed a commit to jason810496/airflow that referenced this pull request May 11, 2026
…l exception (apache#66582)

* Removing resources properly for DatabricksWorkflowGroup

* Add the unit test to check that super().__exit()__ called even exception happen

* remove unnecessary comment

* Fix unit tests

* Fix unit tests

* Strengthen test to actually exercise the bug

The previous test raised an AirflowException from inside the workflow body,
but super().__exit__ is reached even on the unfixed code in that path
(the for-loop over tasks is empty), so the test passed without the fix.

Replace it with a regression test that triggers the raise inside
DatabricksWorkflowTaskGroup.__exit__ itself (via an EmptyOperator that
does not implement _convert_to_databricks_workflow_task) and asserts
TaskGroupContext is empty afterward — fails on unfixed code, passes on
fixed code. Also drops two unused imports introduced earlier.

---------

Co-authored-by: artsiomyudovin <a.yudovin6695@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

TaskGroupContext not removing resources properly if DatabricksWorkflowGroup raises inside __exit__ method

3 participants