Skip to content

Commit 9cbd7de

Browse files
authored
[AIRFLOW-6792] Remove _operator/_hook/_sensor in providers package and add tests (#7412)
1 parent d83ce26 commit 9cbd7de

File tree

64 files changed

+220
-218
lines changed

Some content is hidden

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

64 files changed

+220
-218
lines changed

β€Žairflow/contrib/hooks/aws_dynamodb_hook.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,14 +15,14 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb_hook`."""
18+
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb`."""
1919

2020
import warnings
2121

2222
# pylint: disable=unused-import
23-
from airflow.providers.amazon.aws.hooks.aws_dynamodb_hook import AwsDynamoDBHook # noqa
23+
from airflow.providers.amazon.aws.hooks.aws_dynamodb import AwsDynamoDBHook # noqa
2424

2525
warnings.warn(
26-
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb_hook`.",
26+
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb`.",
2727
DeprecationWarning, stacklevel=2
2828
)

β€Žairflow/contrib/hooks/aws_hook.py

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,14 +15,26 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_hook`."""
18+
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws`."""
1919

2020
import warnings
2121

2222
# pylint: disable=unused-import
23-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook, _parse_s3_config, boto3 # noqa
23+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook, _parse_s3_config, boto3 # noqa
2424

2525
warnings.warn(
26-
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_hook`.",
26+
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws`.",
2727
DeprecationWarning, stacklevel=2
2828
)
29+
30+
31+
class AwsHook(AwsBaseHook):
32+
"""
33+
This class is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`.
34+
"""
35+
def __init__(self, *args, **kwargs):
36+
warnings.warn(
37+
"This class is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`.",
38+
DeprecationWarning, stacklevel=2
39+
)
40+
super().__init__(*args, **kwargs)

β€Žairflow/contrib/operators/gcs_to_gdrive_operator.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,15 +15,15 @@
1515
# KIND, either express or implied. See the License for the
1616
# specific language governing permissions and limitations
1717
# under the License.
18-
"""This module is deprecated. Please use `airflow.providers.google.suite.operators.gcs_to_gdrive_operator`."""
18+
"""This module is deprecated. Please use `airflow.providers.google.suite.operators.gcs_to_gdrive`."""
1919

2020
import warnings
2121

2222
# pylint: disable=unused-import
23-
from airflow.providers.google.suite.operators.gcs_to_gdrive_operator import GCSToGoogleDriveOperator # noqa
23+
from airflow.providers.google.suite.operators.gcs_to_gdrive import GCSToGoogleDriveOperator # noqa
2424

2525
warnings.warn(
2626
"This module is deprecated. "
27-
"Please use `airflow.providers.google.suite.operators.gcs_to_gdrive_operator`.",
27+
"Please use `airflow.providers.google.suite.operators.gcs_to_gdrive.",
2828
DeprecationWarning, stacklevel=2
2929
)

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@
2121
"""
2222
from time import sleep
2323

24-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
24+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2525

2626

27-
class AWSAthenaHook(AwsHook):
27+
class AWSAthenaHook(AwsBaseHook):
2828
"""
2929
Interact with AWS Athena to run, poll queries and return query results
3030

β€Žairflow/providers/amazon/aws/hooks/aws_dynamodb_hook.py renamed to β€Žairflow/providers/amazon/aws/hooks/aws_dynamodb.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@
2121
This module contains the AWS DynamoDB hook
2222
"""
2323
from airflow.exceptions import AirflowException
24-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
24+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2525

2626

27-
class AwsDynamoDBHook(AwsHook):
27+
class AwsDynamoDBHook(AwsBaseHook):
2828
"""
2929
Interact with AWS DynamoDB.
3030

β€Žairflow/providers/amazon/aws/hooks/aws_hook.py renamed to β€Žairflow/providers/amazon/aws/hooks/base_aws.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@
3434
from airflow.hooks.base_hook import BaseHook
3535

3636

37-
class AwsHook(BaseHook):
37+
class AwsBaseHook(BaseHook):
3838
"""
3939
Interact with AWS.
4040
This class is a thin wrapper around the boto3 python library.

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@
3636
from typing_extensions import Protocol, runtime_checkable
3737

3838
from airflow import AirflowException, LoggingMixin
39-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
39+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
4040

4141
# Add exceptions to pylint for the boto3 protocol only; ideally the boto3 library could provide
4242
# protocols for all their dynamically generated classes (try to migrate this to a PR on botocore).
@@ -218,19 +218,19 @@ def __init__(
218218
self.status_retries = status_retries or self.STATUS_RETRIES
219219
self.aws_conn_id = aws_conn_id
220220
self.region_name = region_name
221-
self._hook = None # type: Union[AwsHook, None]
221+
self._hook = None # type: Union[AwsBaseHook, None]
222222
self._client = None # type: Union[AwsBatchProtocol, botocore.client.BaseClient, None]
223223

224224
@property
225-
def hook(self) -> AwsHook:
225+
def hook(self) -> AwsBaseHook:
226226
"""
227227
An AWS API connection manager (wraps boto3)
228228
229229
:return: the connected hook to AWS
230-
:rtype: AwsHook
230+
:rtype: AwsBaseHook
231231
"""
232232
if self._hook is None:
233-
self._hook = AwsHook(aws_conn_id=self.aws_conn_id)
233+
self._hook = AwsBaseHook(aws_conn_id=self.aws_conn_id)
234234
return self._hook
235235

236236
@property

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@
2121
"""
2222
from botocore.exceptions import ClientError
2323

24-
from airflow.contrib.hooks.aws_hook import AwsHook
24+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2525

2626

27-
class AWSCloudFormationHook(AwsHook):
27+
class AWSCloudFormationHook(AwsBaseHook):
2828
"""
2929
Interact with AWS CloudFormation.
3030
"""

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,10 @@
2222
import time
2323

2424
from airflow.exceptions import AirflowBadRequest, AirflowException, AirflowTaskTimeout
25-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
25+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2626

2727

28-
class AWSDataSyncHook(AwsHook):
28+
class AWSDataSyncHook(AwsBaseHook):
2929
"""
3030
Interact with AWS DataSync.
3131

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,10 @@
1717
# under the License.
1818

1919
from airflow.exceptions import AirflowException
20-
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
20+
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
2121

2222

23-
class EmrHook(AwsHook):
23+
class EmrHook(AwsBaseHook):
2424
"""
2525
Interact with AWS EMR. emr_conn_id is only necessary for using the
2626
create_job_flow method.

0 commit comments

Comments
 (0)