2121
2222from __future__ import annotations
2323
24- from datetime import datetime
2524from http import HTTPStatus
2625from typing import TYPE_CHECKING , NamedTuple
2726
27+ from airflow .sdk .exceptions import AirflowException , AirflowNotFoundException
28+
2829if TYPE_CHECKING :
2930 from airflow .models import DagRun
30- from airflow .utils .state import DagRunState
31-
32-
33- class AirflowException (Exception ):
34- """
35- Base class for all Airflow's errors.
36-
37- Each custom exception should be derived from this class.
38- """
39-
40- status_code = HTTPStatus .INTERNAL_SERVER_ERROR
41-
42- def serialize (self ):
43- cls = self .__class__
44- return f"{ cls .__module__ } .{ cls .__name__ } " , (str (self ),), {}
4531
4632
4733class AirflowDagCycleException (AirflowException ):
@@ -54,12 +40,6 @@ class AirflowBadRequest(AirflowException):
5440 status_code = HTTPStatus .BAD_REQUEST
5541
5642
57- class AirflowNotFoundException (AirflowException ):
58- """Raise when the requested object/resource is not available in the system."""
59-
60- status_code = HTTPStatus .NOT_FOUND
61-
62-
6343class AirflowConfigException (AirflowException ):
6444 """Raise when there is configuration problem."""
6545
@@ -84,13 +64,6 @@ class InvalidStatsNameException(AirflowException):
8464 """Raise when name of the stats is invalid."""
8565
8666
87- # Important to inherit BaseException instead of AirflowException->Exception, since this Exception is used
88- # to explicitly interrupt ongoing task. Code that does normal error-handling should not treat
89- # such interrupt as an error that can be handled normally. (Compare with KeyboardInterrupt)
90- class AirflowTaskTimeout (BaseException ):
91- """Raise when the task execution times-out."""
92-
93-
9467class AirflowOptionalProviderFeatureException (AirflowException ):
9568 """Raise by providers when imports are missing for optional provider features."""
9669
@@ -250,44 +223,6 @@ class VariableNotUnique(AirflowException):
250223 """Raise when multiple values are found for the same variable name."""
251224
252225
253- # TODO: workout this to correct place https://github.com/apache/airflow/issues/44353
254- class DagRunTriggerException (AirflowException ):
255- """
256- Signal by an operator to trigger a specific Dag Run of a dag.
257-
258- Special exception raised to signal that the operator it was raised from wishes to trigger
259- a specific Dag Run of a dag. This is used in the ``TriggerDagRunOperator``.
260- """
261-
262- def __init__ (
263- self ,
264- * ,
265- trigger_dag_id : str ,
266- dag_run_id : str ,
267- conf : dict | None ,
268- logical_date : datetime | None ,
269- reset_dag_run : bool ,
270- skip_when_already_exists : bool ,
271- wait_for_completion : bool ,
272- allowed_states : list [str | DagRunState ],
273- failed_states : list [str | DagRunState ],
274- poke_interval : int ,
275- deferrable : bool ,
276- ):
277- super ().__init__ ()
278- self .trigger_dag_id = trigger_dag_id
279- self .dag_run_id = dag_run_id
280- self .conf = conf
281- self .logical_date = logical_date
282- self .reset_dag_run = reset_dag_run
283- self .skip_when_already_exists = skip_when_already_exists
284- self .wait_for_completion = wait_for_completion
285- self .allowed_states = allowed_states
286- self .failed_states = failed_states
287- self .poke_interval = poke_interval
288- self .deferrable = deferrable
289-
290-
291226# The try/except handling is needed after we moved all k8s classes to cncf.kubernetes provider
292227# These two exceptions are used internally by Kubernetes Executor but also by PodGenerator, so we need
293228# to leave them here in case older version of cncf.kubernetes provider is used to run KubernetesPodOperator
@@ -354,6 +289,9 @@ class UnknownExecutorException(ValueError):
354289 "AirflowSkipException" : "airflow.sdk.exceptions.AirflowSkipException" ,
355290 "AirflowFailException" : "airflow.sdk.exceptions.AirflowFailException" ,
356291 "AirflowSensorTimeout" : "airflow.sdk.exceptions.AirflowSensorTimeout" ,
292+ "AirflowTaskTimeout" : "airflow.sdk.exceptions.AirflowTaskTimeout" ,
293+ "DagRunTriggerException" : "airflow.sdk.exceptions.DagRunTriggerException" ,
294+ "AirflowNotFoundException" : "airflow.sdk.exceptions.AirflowNotFoundException" ,
357295}
358296
359297
0 commit comments