Skip to content

Commit 0a3d02c

Browse files
committed
Retain AirflowNotFoundException
1 parent e7c54a2 commit 0a3d02c

File tree

19 files changed

+35
-34
lines changed

19 files changed

+35
-34
lines changed

airflow-core/src/airflow/exceptions.py

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -27,17 +27,13 @@
2727
if TYPE_CHECKING:
2828
from airflow.models import DagRun
2929

30-
from airflow.sdk.exceptions import AirflowException
30+
from airflow.sdk.exceptions import AirflowException, AirflowNotFoundException
3131

3232

3333
class TaskNotFound(AirflowException):
3434
"""Raise when a Task is not available in the system."""
3535

3636

37-
class AirflowNotFoundException(AirflowException):
38-
"""Raise when the requested object/resource is not available in the system."""
39-
40-
4137
class AirflowBadRequest(AirflowException):
4238
"""Raise when the application or server cannot handle the request."""
4339

providers/amazon/src/airflow/providers/amazon/aws/hooks/athena_sql.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -119,7 +119,7 @@ def conn_config(self) -> AwsConnectionWrapper:
119119
raise ValueError(
120120
f"Encountered non-JSON in `extra` field for connection {self.aws_conn_id!r}."
121121
)
122-
except (AirflowNotFoundException, RuntimeError):
122+
except AirflowNotFoundException:
123123
connection = athena_conn
124124
connection.conn_type = "aws"
125125
self.log.warning(

providers/amazon/src/airflow/providers/amazon/aws/hooks/base_aws.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -620,7 +620,7 @@ def conn_config(self) -> AwsConnectionWrapper:
620620
if self.aws_conn_id:
621621
try:
622622
connection = self.get_connection(self.aws_conn_id)
623-
except (AirflowNotFoundException, RuntimeError):
623+
except AirflowNotFoundException:
624624
self.log.warning(
625625
"Unable to find AWS Connection ID '%s', switching to empty.", self.aws_conn_id
626626
)

providers/amazon/src/airflow/providers/amazon/aws/hooks/emr.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -112,7 +112,7 @@ def create_job_flow(self, job_flow_overrides: dict[str, Any]) -> dict[str, Any]:
112112
if self.emr_conn_id:
113113
try:
114114
emr_conn = self.get_connection(self.emr_conn_id)
115-
except (AirflowNotFoundException, RuntimeError):
115+
except AirflowNotFoundException:
116116
warnings.warn(
117117
f"Unable to find {self.hook_name} Connection ID {self.emr_conn_id!r}, "
118118
"using an empty initial configuration. If you want to get rid of this warning "

providers/apache/spark/src/airflow/providers/apache/spark/hooks/spark_sql.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ def __init__(
104104

105105
try:
106106
conn = self.get_connection(conn_id)
107-
except (AirflowNotFoundException, RuntimeError):
107+
except AirflowNotFoundException:
108108
conn = None
109109
if conn:
110110
options = conn.extra_dejson

providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/hooks/kubernetes.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,7 +208,7 @@ def get_connection(cls, conn_id: str) -> Connection:
208208
"""
209209
try:
210210
return super().get_connection(conn_id) # type: ignore[return-value]
211-
except (AirflowNotFoundException, RuntimeError):
211+
except AirflowNotFoundException:
212212
if conn_id == cls.default_conn_name:
213213
return Connection(conn_id=cls.default_conn_name)
214214
raise

providers/cncf/kubernetes/tests/unit/cncf/kubernetes/hooks/test_kubernetes.py

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -448,9 +448,7 @@ def test_missing_default_connection_is_ok(self, remove_default_conn, sdk_connect
448448

449449
# meanwhile, asking for non-default should still fail if it doesn't exist
450450
hook = KubernetesHook("some_conn")
451-
with pytest.raises(
452-
(AirflowNotFoundException, RuntimeError), match="The conn_id `some_conn` isn't defined"
453-
):
451+
with pytest.raises(AirflowNotFoundException, match="The conn_id `some_conn` isn't defined"):
454452
hook.conn_extras
455453

456454
@patch("kubernetes.config.kube_config.KubeConfigLoader")

providers/edge3/src/airflow/providers/edge3/example_dags/integration_test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ def connection():
129129
try:
130130
conn = BaseHook.get_connection("integration_test")
131131
print(f"Got connection {conn}")
132-
except (AirflowNotFoundException, RuntimeError):
132+
except AirflowNotFoundException:
133133
print("Connection not found... but also OK.")
134134

135135
@task_group(prefix_group_id=False)

providers/edge3/src/airflow/providers/edge3/example_dags/win_test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -303,7 +303,7 @@ def connection():
303303
try:
304304
conn = BaseHook.get_connection("integration_test")
305305
print(f"Got connection {conn}")
306-
except (AirflowNotFoundException, RuntimeError):
306+
except AirflowNotFoundException:
307307
print("Connection 'integration_test' not found... but also OK.")
308308

309309
command = CmdOperator(task_id="command", command="echo Hello World")

providers/google/src/airflow/providers/google/cloud/log/gcs_task_handler.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ def hook(self) -> GCSHook | None:
9393
if conn_id:
9494
try:
9595
return GCSHook(gcp_conn_id=conn_id)
96-
except (AirflowNotFoundException, RuntimeError):
96+
except AirflowNotFoundException:
9797
pass
9898
return None
9999

0 commit comments

Comments
 (0)