diff --git a/airflow/exceptions.py b/airflow/exceptions.py index 163a9ca626619..fd4fbf6758f07 100644 --- a/airflow/exceptions.py +++ b/airflow/exceptions.py @@ -461,7 +461,7 @@ class TaskDeferralTimeout(AirflowException): # 2) if you have new provider, both provider and pod generator will throw the # "airflow.providers.cncf.kubernetes" as it will be imported here from the provider. try: - from airflow.providers.cncf.kubernetes.pod_generator import PodMutationHookException + from airflow.providers.cncf.kubernetes.exceptions import PodMutationHookException except ImportError: class PodMutationHookException(AirflowException): # type: ignore[no-redef] @@ -469,7 +469,7 @@ class PodMutationHookException(AirflowException): # type: ignore[no-redef] try: - from airflow.providers.cncf.kubernetes.pod_generator import PodReconciliationError + from airflow.providers.cncf.kubernetes.exceptions import PodReconciliationError except ImportError: class PodReconciliationError(AirflowException): # type: ignore[no-redef] diff --git a/providers/src/airflow/providers/cncf/kubernetes/exceptions.py b/providers/src/airflow/providers/cncf/kubernetes/exceptions.py new file mode 100644 index 0000000000000..c0b6ad83a3fdc --- /dev/null +++ b/providers/src/airflow/providers/cncf/kubernetes/exceptions.py @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +from airflow.exceptions import ( + AirflowException, +) + + +class PodMutationHookException(AirflowException): + """Raised when exception happens during Pod Mutation Hook execution.""" + + +class PodReconciliationError(AirflowException): + """Raised when an error is encountered while trying to merge pod configs.""" diff --git a/providers/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py b/providers/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py index cfd31cda894a2..482f99725b58f 100644 --- a/providers/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py +++ b/providers/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py @@ -61,13 +61,13 @@ from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.executors.base_executor import BaseExecutor from airflow.executors.executor_constants import KUBERNETES_EXECUTOR +from airflow.providers.cncf.kubernetes.exceptions import PodMutationHookException, PodReconciliationError from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import ( ADOPTED, POD_EXECUTOR_DONE_KEY, ) from airflow.providers.cncf.kubernetes.kube_config import KubeConfig from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import annotations_to_key -from airflow.providers.cncf.kubernetes.pod_generator import PodMutationHookException, PodReconciliationError from airflow.stats import Stats from airflow.utils.event_scheduler import EventScheduler from airflow.utils.log.logging_mixin import remove_escape_codes diff --git a/providers/src/airflow/providers/cncf/kubernetes/pod_generator.py b/providers/src/airflow/providers/cncf/kubernetes/pod_generator.py index ad4123eacc0c0..b90fa715333bf 100644 --- a/providers/src/airflow/providers/cncf/kubernetes/pod_generator.py +++ b/providers/src/airflow/providers/cncf/kubernetes/pod_generator.py @@ -39,9 +39,9 @@ from airflow.exceptions import ( AirflowConfigException, - AirflowException, ) from airflow.providers.cncf.kubernetes.backcompat import get_logical_date_key +from airflow.providers.cncf.kubernetes.exceptions import PodMutationHookException, PodReconciliationError from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import ( POD_NAME_MAX_LENGTH, add_unique_suffix, @@ -58,14 +58,6 @@ MAX_LABEL_LEN = 63 -class PodMutationHookException(AirflowException): - """Raised when exception happens during Pod Mutation Hook execution.""" - - -class PodReconciliationError(AirflowException): - """Raised when an error is encountered while trying to merge pod configs.""" - - def make_safe_label_value(string: str) -> str: """ Normalize a provided label to be of valid length and characters. diff --git a/tests/core/test_exceptions.py b/tests/core/test_exceptions.py new file mode 100644 index 0000000000000..b0ebce05e90d6 --- /dev/null +++ b/tests/core/test_exceptions.py @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import sys + + +class TestExceptions: + def setup_method(self): + self.old_modules = dict(sys.modules) + + def teardown_method(self): + # Remove any new modules imported during the test run. This lets us + # import the same source files for more than one test. + for mod in [m for m in sys.modules if m not in self.old_modules]: + del sys.modules[mod] + + def test_pod_mutation_hook_exceptions_compatibility( + self, + ): + from airflow.exceptions import ( + PodMutationHookException as CoreMutationHookException, + ) + from airflow.providers.cncf.kubernetes.exceptions import ( + PodMutationHookException as ProviderMutationHookException, + ) + from airflow.providers.cncf.kubernetes.pod_generator import ( + PodMutationHookException as ProviderGeneratorMutationHookException, + ) + + assert ProviderMutationHookException == CoreMutationHookException + assert ProviderMutationHookException == ProviderGeneratorMutationHookException + + def test_pod_reconciliation_error_exceptions_compatibility( + self, + ): + from airflow.exceptions import ( + PodReconciliationError as CoreReconciliationError, + ) + from airflow.providers.cncf.kubernetes.exceptions import ( + PodReconciliationError as ProviderReconciliationError, + ) + from airflow.providers.cncf.kubernetes.pod_generator import ( + PodReconciliationError as ProviderGeneratorReconciliationError, + ) + + assert ProviderReconciliationError == CoreReconciliationError + assert ProviderReconciliationError == ProviderGeneratorReconciliationError