Skip to content

Commit 0a30706

Browse files
authored
Use AirflowProviderDeprecationWarning in providers (#30975)
* Use `AirflowProviderDeprecationWarning` in providers Now that Providers minimum Airflow version is 2.4.0 we can replace `DeprecationWarning` with `AirflowProviderDeprecationWarning`
1 parent eb7b98d commit 0a30706

File tree

91 files changed

+362
-285
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

91 files changed

+362
-285
lines changed

β€Žairflow/providers/amazon/aws/hooks/s3.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@
4040
from boto3.s3.transfer import S3Transfer, TransferConfig
4141
from botocore.exceptions import ClientError
4242

43-
from airflow.exceptions import AirflowException
43+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
4444
from airflow.providers.amazon.aws.exceptions import S3HookUriParseFailure
4545
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
4646
from airflow.providers.amazon.aws.utils.tags import format_tags
@@ -74,7 +74,7 @@ def wrapper(*args, **kwargs) -> T:
7474
"s3 conn_type, and the associated schema field, is deprecated."
7575
" Please use aws conn_type instead, and specify `bucket_name`"
7676
" in `service_config.s3` within `extras`.",
77-
DeprecationWarning,
77+
AirflowProviderDeprecationWarning,
7878
stacklevel=2,
7979
)
8080
bound_args.arguments["bucket_name"] = self.conn_config.schema

β€Žairflow/providers/amazon/aws/operators/batch.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@
2929
from typing import TYPE_CHECKING, Any, Sequence
3030

3131
from airflow.compat.functools import cached_property
32-
from airflow.exceptions import AirflowException
32+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
3333
from airflow.models import BaseOperator
3434
from airflow.providers.amazon.aws.hooks.batch_client import BatchClientHook
3535
from airflow.providers.amazon.aws.links.batch import (
@@ -163,7 +163,7 @@ def __init__(
163163
self.container_overrides = overrides
164164
warnings.warn(
165165
"Parameter `overrides` is deprecated, Please use `container_overrides` instead.",
166-
DeprecationWarning,
166+
AirflowProviderDeprecationWarning,
167167
stacklevel=2,
168168
)
169169

β€Žairflow/providers/amazon/aws/operators/eks.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@
2323

2424
from botocore.exceptions import ClientError, WaiterError
2525

26-
from airflow import AirflowException
26+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
2727
from airflow.models import BaseOperator
2828
from airflow.providers.amazon.aws.hooks.eks import EksHook
2929

@@ -691,7 +691,7 @@ def __init__(
691691
"Currently the default for this parameter is `False` but in a future release the default "
692692
"will be changed to `True`. To ensure pods are not deleted in the future you will need to "
693693
"set `is_delete_operator_pod=False` explicitly.",
694-
DeprecationWarning,
694+
AirflowProviderDeprecationWarning,
695695
stacklevel=2,
696696
)
697697
is_delete_operator_pod = False

β€Žairflow/providers/amazon/aws/operators/emr.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@
2222
from typing import TYPE_CHECKING, Any, Sequence
2323
from uuid import uuid4
2424

25-
from airflow.exceptions import AirflowException
25+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
2626
from airflow.models import BaseOperator
2727
from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook
2828
from airflow.providers.amazon.aws.links.emr import EmrClusterLink
@@ -486,7 +486,7 @@ def __init__(
486486
warnings.warn(
487487
f"Parameter `{self.__class__.__name__}.max_tries` is deprecated and will be removed "
488488
"in a future release. Please use method `max_polling_attempts` instead.",
489-
DeprecationWarning,
489+
AirflowProviderDeprecationWarning,
490490
stacklevel=2,
491491
)
492492
if max_polling_attempts and max_polling_attempts != max_tries:

β€Žairflow/providers/amazon/aws/operators/sagemaker.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424
from botocore.exceptions import ClientError
2525

2626
from airflow.compat.functools import cached_property
27-
from airflow.exceptions import AirflowException
27+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
2828
from airflow.models import BaseOperator
2929
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
3030
from airflow.providers.amazon.aws.hooks.sagemaker import SageMakerHook
@@ -201,7 +201,7 @@ def __init__(
201201
warnings.warn(
202202
"Action 'increment' on job name conflict has been deprecated for performance reasons."
203203
"The alternative to 'fail' is now 'timestamp'.",
204-
DeprecationWarning,
204+
AirflowProviderDeprecationWarning,
205205
stacklevel=2,
206206
)
207207
self.action_if_job_exists = action_if_job_exists
@@ -486,7 +486,7 @@ def __init__(
486486
warnings.warn(
487487
"Action 'increment' on job name conflict has been deprecated for performance reasons."
488488
"The alternative to 'fail' is now 'timestamp'.",
489-
DeprecationWarning,
489+
AirflowProviderDeprecationWarning,
490490
stacklevel=2,
491491
)
492492
self.action_if_job_exists = action_if_job_exists
@@ -718,7 +718,7 @@ def __init__(
718718
warnings.warn(
719719
"Action 'increment' on job name conflict has been deprecated for performance reasons."
720720
"The alternative to 'fail' is now 'timestamp'.",
721-
DeprecationWarning,
721+
AirflowProviderDeprecationWarning,
722722
stacklevel=2,
723723
)
724724
self.action_if_job_exists = action_if_job_exists

β€Žairflow/providers/amazon/aws/secrets/secrets_manager.py

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
from urllib.parse import unquote
2626

2727
from airflow.compat.functools import cached_property
28+
from airflow.exceptions import AirflowProviderDeprecationWarning
2829
from airflow.providers.amazon.aws.utils import trim_none_values
2930
from airflow.secrets import BaseSecretsBackend
3031
from airflow.utils.log.logging_mixin import LoggingMixin
@@ -149,7 +150,7 @@ def __init__(
149150
"The `full_url_mode` kwarg is deprecated. Going forward, the `SecretsManagerBackend`"
150151
" will support both URL-encoded and JSON-encoded secrets at the same time. The encoding"
151152
" of the secret will be determined automatically.",
152-
DeprecationWarning,
153+
AirflowProviderDeprecationWarning,
153154
stacklevel=2,
154155
)
155156

@@ -159,7 +160,7 @@ def __init__(
159160
" migrating away from URL-encoding secret values for JSON secrets."
160161
" To remove this warning, make sure your JSON secrets are *NOT* URL-encoded, and then"
161162
" remove this kwarg from backend_kwargs.",
162-
DeprecationWarning,
163+
AirflowProviderDeprecationWarning,
163164
stacklevel=2,
164165
)
165166
self.are_secret_values_urlencoded = kwargs.pop("are_secret_values_urlencoded", None)

β€Žairflow/providers/amazon/aws/transfers/base.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import warnings
2222
from typing import Sequence
2323

24+
from airflow.exceptions import AirflowProviderDeprecationWarning
2425
from airflow.models import BaseOperator
2526
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2627
from airflow.utils.types import NOTSET, ArgNotSet
@@ -60,7 +61,7 @@ def __init__(
6061
) -> None:
6162
super().__init__(**kwargs)
6263
if not isinstance(aws_conn_id, ArgNotSet):
63-
warnings.warn(_DEPRECATION_MSG, DeprecationWarning, stacklevel=3)
64+
warnings.warn(_DEPRECATION_MSG, AirflowProviderDeprecationWarning, stacklevel=3)
6465
self.source_aws_conn_id = aws_conn_id
6566
else:
6667
self.source_aws_conn_id = source_aws_conn_id

β€Žairflow/providers/amazon/aws/utils/connection_wrapper.py

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@
2525
from botocore.config import Config
2626

2727
from airflow.compat.functools import cached_property
28-
from airflow.exceptions import AirflowException
28+
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
2929
from airflow.providers.amazon.aws.utils import trim_none_values
3030
from airflow.utils.log.logging_mixin import LoggingMixin
3131
from airflow.utils.log.secrets_masker import mask_secret
@@ -168,7 +168,7 @@ def __post_init__(self, conn: Connection):
168168
f"{self.conn_repr} has connection type 's3', "
169169
"which has been replaced by connection type 'aws'. "
170170
"Please update your connection to have `conn_type='aws'`.",
171-
DeprecationWarning,
171+
AirflowProviderDeprecationWarning,
172172
stacklevel=2,
173173
)
174174
elif self.conn_type != "aws":
@@ -186,7 +186,7 @@ def __post_init__(self, conn: Connection):
186186
warnings.warn(
187187
"'session_kwargs' in extra config is deprecated and will be removed in a future releases. "
188188
f"Please specify arguments passed to boto3 Session directly in {self.conn_repr} extra.",
189-
DeprecationWarning,
189+
AirflowProviderDeprecationWarning,
190190
stacklevel=2,
191191
)
192192

@@ -242,7 +242,7 @@ def __post_init__(self, conn: Connection):
242242
warnings.warn(
243243
f"Host {conn.host} specified in the connection is not used."
244244
" Please, set it on extra['endpoint_url'] instead",
245-
DeprecationWarning,
245+
AirflowProviderDeprecationWarning,
246246
stacklevel=2,
247247
)
248248

@@ -251,7 +251,7 @@ def __post_init__(self, conn: Connection):
251251
warnings.warn(
252252
"extra['host'] is deprecated and will be removed in a future release."
253253
" Please set extra['endpoint_url'] instead",
254-
DeprecationWarning,
254+
AirflowProviderDeprecationWarning,
255255
stacklevel=2,
256256
)
257257
else:
@@ -385,7 +385,7 @@ def _get_assume_role_configs(
385385
"Constructing 'role_arn' from extra['aws_account_id'] and extra['aws_iam_role'] is deprecated"
386386
f" and will be removed in a future releases."
387387
f" Please set 'role_arn' in {self.conn_repr} extra.",
388-
DeprecationWarning,
388+
AirflowProviderDeprecationWarning,
389389
stacklevel=3,
390390
)
391391
role_arn = f"arn:aws:iam::{aws_account_id}:role/{aws_iam_role}"
@@ -413,7 +413,7 @@ def _get_assume_role_configs(
413413
warnings.warn(
414414
"'external_id' in extra config is deprecated and will be removed in a future releases. "
415415
f"Please set 'ExternalId' in 'assume_role_kwargs' in {self.conn_repr} extra.",
416-
DeprecationWarning,
416+
AirflowProviderDeprecationWarning,
417417
stacklevel=3,
418418
)
419419
assume_role_kwargs["ExternalId"] = external_id
@@ -436,7 +436,7 @@ def _parse_s3_config(
436436
warnings.warn(
437437
"Use local credentials file is never documented and well tested. "
438438
"Obtain credentials by this way deprecated and will be removed in a future releases.",
439-
DeprecationWarning,
439+
AirflowProviderDeprecationWarning,
440440
stacklevel=4,
441441
)
442442

β€Žairflow/providers/apache/drill/operators/drill.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import warnings
2121
from typing import Sequence
2222

23+
from airflow.exceptions import AirflowProviderDeprecationWarning
2324
from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator
2425

2526

@@ -49,6 +50,6 @@ def __init__(self, *, drill_conn_id: str = "drill_default", **kwargs) -> None:
4950
warnings.warn(
5051
"""This class is deprecated.
5152
Please use `airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator`.""",
52-
DeprecationWarning,
53+
AirflowProviderDeprecationWarning,
5354
stacklevel=2,
5455
)

β€Žairflow/providers/apache/druid/operators/druid_check.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
import warnings
2121

22+
from airflow.exceptions import AirflowProviderDeprecationWarning
2223
from airflow.providers.common.sql.operators.sql import SQLCheckOperator
2324

2425

@@ -32,7 +33,7 @@ def __init__(self, druid_broker_conn_id: str = "druid_broker_default", **kwargs)
3233
warnings.warn(
3334
"""This class is deprecated.
3435
Please use `airflow.providers.common.sql.operators.sql.SQLCheckOperator`.""",
35-
DeprecationWarning,
36+
AirflowProviderDeprecationWarning,
3637
stacklevel=2,
3738
)
3839
super().__init__(conn_id=druid_broker_conn_id, **kwargs)

0 commit comments

Comments
 (0)