Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
cb16569
Move out some exceptions to TaskSDK
ephraimbuddy Aug 14, 2025
6f497ee
fixup! Move out some exceptions to TaskSDK
ephraimbuddy Oct 3, 2025
f352b33
Fix provider RuntimeError change and fix for shared package modulenot…
ephraimbuddy Oct 14, 2025
a0fa853
fixup! Fix provider RuntimeError change and fix for shared package mo…
ephraimbuddy Oct 14, 2025
665fd5e
Fix sdk connection to catch RuntimeError instead of AirflowException
ephraimbuddy Oct 14, 2025
8b0a0d1
Use random str at task end to prevent duplicate task_id error so that…
ephraimbuddy Oct 14, 2025
a43d076
fixup! Use random str at task end to prevent duplicate task_id error …
ephraimbuddy Oct 14, 2025
c25d254
Use RuntimeError instead of AirflowNotFoundException for connections
ephraimbuddy Oct 16, 2025
3498f9f
fixup! fixup! Use RuntimeError instead of AirflowNotFoundException fo…
ephraimbuddy Oct 16, 2025
91ff96c
import tasktimeout properly
ephraimbuddy Oct 17, 2025
3d855ed
move TaskNotFound to sdk
ephraimbuddy Oct 28, 2025
7802e86
Apply suggestion from @ephraimbuddy
ephraimbuddy Oct 29, 2025
6105a09
Retain AirflowNotFoundException
ephraimbuddy Oct 30, 2025
2cca599
AirflowException now inherits from RuntimeError, update provider changes
ephraimbuddy Oct 30, 2025
43e8a89
fixup! AirflowException now inherits from RuntimeError, update provid…
ephraimbuddy Oct 30, 2025
fbccc23
Now that AirflowException inherits from Runtime, remove change of Air…
ephraimbuddy Nov 6, 2025
6461aff
Fix imports and exception use in providers
ephraimbuddy Nov 6, 2025
5110ec3
Explicitly cover for shared packages in exception
ephraimbuddy Nov 17, 2025
a5ce61e
Fix ruff broadexception
ephraimbuddy Nov 18, 2025
27cfbb5
fix mypy issue
ephraimbuddy Nov 18, 2025
27d9a6c
remove unintended file addition
ephraimbuddy Nov 19, 2025
be1feba
Remove left over cleanup
ephraimbuddy Nov 24, 2025
dfa8a6e
Return exception to AirflowException
ephraimbuddy Nov 24, 2025
79bd1eb
Use import_string
ephraimbuddy Nov 24, 2025
cdd460f
Fix static checks
ephraimbuddy Nov 24, 2025
87aff12
Revert subclassing AirflowException from Runtime
ephraimbuddy Nov 24, 2025
de216f9
Fix providers importing deprecated exceptions to use compat SDK
ephraimbuddy Nov 24, 2025
964bbca
fixup! Fix providers importing deprecated exceptions to use compat SDK
ephraimbuddy Nov 24, 2025
afcf1fa
Use ModuleNotFoundError in exception if airflow.sdk is not installed(…
ephraimbuddy Nov 25, 2025
0611b29
Fix Airflow 2 compat
ephraimbuddy Nov 25, 2025
cec855b
Use ValueError instead of RuntimeError were appropriate
ephraimbuddy Nov 25, 2025
efa0075
Update compat to have only the imports used in provider
ephraimbuddy Nov 25, 2025
b0b1b14
Update example dag import and fix mypy error
ephraimbuddy Nov 25, 2025
7d7cd20
fix conflict
ephraimbuddy Nov 25, 2025
9a5bf3d
Add news fragment item
ephraimbuddy Nov 26, 2025
8b77862
fixup! Add news fragment item
ephraimbuddy Nov 26, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
60 changes: 60 additions & 0 deletions airflow-core/newsfragments/54505.significant.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
Move task-level exception imports into the Task SDK

Airflow now sources task-facing exceptions (``AirflowSkipException``, ``TaskDeferred``, etc.) from
``airflow.sdk.exceptions``. ``airflow.exceptions`` still exposes the same exceptions, but they are
proxies that emit ``DeprecatedImportWarning`` so Dag authors can migrate before the shim is removed.

**What changed:**

- Runtime code now consistently raises the SDK versions of task-level exceptions.
- The Task SDK redefines these classes so workers no longer depend on ``airflow-core`` at runtime.
- ``airflow.providers.common.compat.sdk`` centralizes compatibility imports for providers.

**Behaviour changes:**

- Sensors and other helpers that validate user input now raise ``ValueError`` (instead of
``AirflowException``) when ``poke_interval``/ ``timeout`` arguments are invalid.
- Importing deprecated exception names from ``airflow.exceptions`` logs a warning directing users to
the SDK import path.

**Exceptions now provided by ``airflow.sdk.exceptions``:**

- ``AirflowException`` and ``AirflowNotFoundException``
- ``AirflowRescheduleException`` and ``AirflowSensorTimeout``
- ``AirflowSkipException``, ``AirflowFailException``, ``AirflowTaskTimeout``, ``AirflowTaskTerminated``
- ``TaskDeferred``, ``TaskDeferralTimeout``, ``TaskDeferralError``
- ``DagRunTriggerException`` and ``DownstreamTasksSkipped``
- ``AirflowDagCycleException`` and ``AirflowInactiveAssetInInletOrOutletException``
- ``ParamValidationError``, ``DuplicateTaskIdFound``, ``TaskAlreadyInTaskGroup``, ``TaskNotFound``, ``XComNotFound``

**Backward compatibility:**

- Existing Dags/operators that still import from ``airflow.exceptions`` continue to work, though
they log warnings.
- Providers can rely on ``airflow.providers.common.compat.sdk`` to keep one import path that works
across supported Airflow versions.

**Migration:**

- Update custom operators, sensors, and extensions to import exception classes from
``airflow.sdk.exceptions`` (or from the provider compat shim).
- Adjust custom validation code to expect ``ValueError`` for invalid sensor arguments if it
previously caught ``AirflowException``.

* Types of change

* [ ] Dag changes
* [ ] Config changes
* [ ] API changes
* [ ] CLI changes
* [x] Behaviour changes
* [ ] Plugin changes
* [ ] Dependency changes
* [x] Code interface changes

* Migration rules needed

* Import task-level exceptions such as ``AirflowSkipException``, ``TaskDeferred``,
``AirflowFailException``, etc. from ``airflow.sdk.exceptions`` instead of ``airflow.exceptions``.
* Update custom sensors/operators that validated arguments by catching ``AirflowException`` to
expect ``ValueError`` for invalid ``poke_interval`` / ``timeout`` inputs.
5 changes: 3 additions & 2 deletions airflow-core/src/airflow/dag_processing/dagbag.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,8 @@
AirflowClusterPolicyError,
AirflowClusterPolicySkipDag,
AirflowClusterPolicyViolation,
AirflowDagCycleException,
AirflowDagDuplicatedIdException,
AirflowException,
AirflowTaskTimeout,
UnknownExecutorException,
)
from airflow.executors.executor_loader import ExecutorLoader
Expand Down Expand Up @@ -119,6 +117,8 @@ def timeout(seconds=1, error_message="Timeout"):
def handle_timeout(signum, frame):
"""Log information and raises AirflowTaskTimeout."""
log.error("Process timed out, PID: %s", str(os.getpid()))
from airflow.sdk.exceptions import AirflowTaskTimeout

raise AirflowTaskTimeout(error_message)

try:
Expand Down Expand Up @@ -588,6 +588,7 @@ def bag_dag(self, dag: DAG):
except Exception as e:
self.log.exception(e)
raise AirflowClusterPolicyError(e)
from airflow.sdk.exceptions import AirflowDagCycleException

try:
prev_dag = self.dags.get(dag.dag_id)
Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/dag_processing/processor.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
)
from airflow.configuration import conf
from airflow.dag_processing.dagbag import DagBag
from airflow.exceptions import TaskNotFound
from airflow.sdk.exceptions import TaskNotFound
from airflow.sdk.execution_time.comms import (
ConnectionResult,
DeleteVariable,
Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/example_dags/example_skip_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,9 @@

import pendulum

from airflow.exceptions import AirflowSkipException
from airflow.providers.standard.operators.empty import EmptyOperator
from airflow.sdk import DAG, BaseOperator, TriggerRule
from airflow.sdk.exceptions import AirflowSkipException

if TYPE_CHECKING:
from airflow.sdk import Context
Expand Down
Loading
Loading