Skip to content

Commit

Permalink
[AIRFLOW-6792] Remove _operator/_hook/_sensor in providers package an…
Browse files Browse the repository at this point in the history
…d add tests (#7412)
  • Loading branch information
mik-laj committed Feb 17, 2020
1 parent d83ce26 commit 9cbd7de
Show file tree
Hide file tree
Showing 64 changed files with 220 additions and 218 deletions.
6 changes: 3 additions & 3 deletions airflow/contrib/hooks/aws_dynamodb_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,14 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb_hook`."""
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb`."""

import warnings

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

warnings.warn(
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb_hook`.",
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_dynamodb`.",
DeprecationWarning, stacklevel=2
)
18 changes: 15 additions & 3 deletions airflow/contrib/hooks/aws_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,26 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_hook`."""
"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws`."""

import warnings

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

warnings.warn(
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.aws_hook`.",
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws`.",
DeprecationWarning, stacklevel=2
)


class AwsHook(AwsBaseHook):
"""
This class is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`.
"""
def __init__(self, *args, **kwargs):
warnings.warn(
"This class is deprecated. Please use `airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`.",
DeprecationWarning, stacklevel=2
)
super().__init__(*args, **kwargs)
6 changes: 3 additions & 3 deletions airflow/contrib/operators/gcs_to_gdrive_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,15 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""This module is deprecated. Please use `airflow.providers.google.suite.operators.gcs_to_gdrive_operator`."""
"""This module is deprecated. Please use `airflow.providers.google.suite.operators.gcs_to_gdrive`."""

import warnings

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

warnings.warn(
"This module is deprecated. "
"Please use `airflow.providers.google.suite.operators.gcs_to_gdrive_operator`.",
"Please use `airflow.providers.google.suite.operators.gcs_to_gdrive.",
DeprecationWarning, stacklevel=2
)
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/athena.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
"""
from time import sleep

from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AWSAthenaHook(AwsHook):
class AWSAthenaHook(AwsBaseHook):
"""
Interact with AWS Athena to run, poll queries and return query results
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
This module contains the AWS DynamoDB hook
"""
from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsDynamoDBHook(AwsHook):
class AwsDynamoDBHook(AwsBaseHook):
"""
Interact with AWS DynamoDB.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
from airflow.hooks.base_hook import BaseHook


class AwsHook(BaseHook):
class AwsBaseHook(BaseHook):
"""
Interact with AWS.
This class is a thin wrapper around the boto3 python library.
Expand Down
10 changes: 5 additions & 5 deletions airflow/providers/amazon/aws/hooks/batch_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
from typing_extensions import Protocol, runtime_checkable

from airflow import AirflowException, LoggingMixin
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook

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

@property
def hook(self) -> AwsHook:
def hook(self) -> AwsBaseHook:
"""
An AWS API connection manager (wraps boto3)
:return: the connected hook to AWS
:rtype: AwsHook
:rtype: AwsBaseHook
"""
if self._hook is None:
self._hook = AwsHook(aws_conn_id=self.aws_conn_id)
self._hook = AwsBaseHook(aws_conn_id=self.aws_conn_id)
return self._hook

@property
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/cloud_formation.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
"""
from botocore.exceptions import ClientError

from airflow.contrib.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AWSCloudFormationHook(AwsHook):
class AWSCloudFormationHook(AwsBaseHook):
"""
Interact with AWS CloudFormation.
"""
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/datasync.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,10 @@
import time

from airflow.exceptions import AirflowBadRequest, AirflowException, AirflowTaskTimeout
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AWSDataSyncHook(AwsHook):
class AWSDataSyncHook(AwsBaseHook):
"""
Interact with AWS DataSync.
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/emr.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@
# under the License.

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class EmrHook(AwsHook):
class EmrHook(AwsBaseHook):
"""
Interact with AWS EMR. emr_conn_id is only necessary for using the
create_job_flow method.
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/glue_catalog.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
"""
This module contains AWS Glue Catalog Hook
"""
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsGlueCatalogHook(AwsHook):
class AwsGlueCatalogHook(AwsBaseHook):
"""
Interact with AWS Glue Catalog
Expand Down
6 changes: 3 additions & 3 deletions airflow/providers/amazon/aws/hooks/kinesis.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
"""
This module contains AWS Firehose hook
"""
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsFirehoseHook(AwsHook):
class AwsFirehoseHook(AwsBaseHook):
"""
Interact with AWS Kinesis Firehose.
Expand All @@ -40,7 +40,7 @@ def __init__(self, delivery_stream, region_name=None, *args, **kwargs):

def get_conn(self):
"""
Returns AwsHook connection object.
Returns AWS connection object.
"""

self.conn = self.get_client_type('firehose', self.region_name)
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/lambda_function.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
"""
This module contains AWS Lambda hook
"""
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsLambdaHook(AwsHook):
class AwsLambdaHook(AwsBaseHook):
"""
Interact with AWS Lambda
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/logs.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
functionality for interacting with AWS CloudWatch.
"""

from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsLogsHook(AwsHook):
class AwsLogsHook(AwsBaseHook):
"""
Interact with AWS CloudWatch Logs
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
Interact with AWS Redshift, using the boto3 library.
"""

from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class RedshiftHook(AwsHook):
class RedshiftHook(AwsBaseHook):
"""
Interact with AWS Redshift, using the boto3 library
"""
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
from botocore.exceptions import ClientError

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.utils.helpers import chunks


Expand Down Expand Up @@ -58,7 +58,7 @@ def has_arg(name):
return wrapper


class S3Hook(AwsHook):
class S3Hook(AwsBaseHook):
"""
Interact with AWS S3, using the boto3 library.
"""
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/sagemaker.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
from botocore.exceptions import ClientError

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.aws.hooks.logs import AwsLogsHook
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
from airflow.utils import timezone
Expand Down Expand Up @@ -121,7 +121,7 @@ def secondary_training_status_message(job_description, prev_description):
return '\n'.join(status_strs)


class SageMakerHook(AwsHook):
class SageMakerHook(AwsBaseHook):
"""
Interact with Amazon SageMaker.
"""
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/sns.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
"""
import json

from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class AwsSnsHook(AwsHook):
class AwsSnsHook(AwsBaseHook):
"""
Interact with Amazon Simple Notification Service.
"""
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/hooks/sqs.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@
"""
This module contains AWS SQS hook
"""
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class SQSHook(AwsHook):
class SQSHook(AwsBaseHook):
"""
Interact with Amazon Simple Queue Service.
"""
Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/amazon/aws/operators/dynamodb_to_s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
from boto.compat import json # type: ignore

from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.aws_dynamodb_hook import AwsDynamoDBHook
from airflow.providers.amazon.aws.hooks.aws_dynamodb import AwsDynamoDBHook
from airflow.providers.amazon.aws.hooks.s3 import S3Hook


Expand Down
6 changes: 3 additions & 3 deletions airflow/providers/amazon/aws/operators/ecs.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.aws.hooks.logs import AwsLogsHook
from airflow.typing_compat import Protocol
from airflow.utils.decorators import apply_defaults
Expand Down Expand Up @@ -234,8 +234,8 @@ def _check_success_task(self):
format(container.get('reason', '').lower()))

def get_hook(self):
"""Create and return an AwsHook."""
return AwsHook(
"""Create and return an AwsBaseHook."""
return AwsBaseHook(
aws_conn_id=self.aws_conn_id
)

Expand Down
2 changes: 1 addition & 1 deletion airflow/providers/amazon/aws/operators/hive_to_dynamodb.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import json

from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.aws_dynamodb_hook import AwsDynamoDBHook
from airflow.providers.amazon.aws.hooks.aws_dynamodb import AwsDynamoDBHook
from airflow.providers.apache.hive.hooks.hive import HiveServer2Hook
from airflow.utils.decorators import apply_defaults

Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/operators/sagemaker_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# under the License.

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.aws.operators.sagemaker_base import SageMakerBaseOperator
from airflow.utils.decorators import apply_defaults

Expand Down Expand Up @@ -99,7 +99,7 @@ def create_integer_fields(self):
def expand_role(self):
if 'Model' not in self.config:
return
hook = AwsHook(self.aws_conn_id)
hook = AwsBaseHook(self.aws_conn_id)
config = self.config['Model']
if 'ExecutionRoleArn' in config:
config['ExecutionRoleArn'] = hook.expand_role(config['ExecutionRoleArn'])
Expand Down
4 changes: 2 additions & 2 deletions airflow/providers/amazon/aws/operators/sagemaker_model.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# under the License.

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.aws_hook import AwsHook
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.aws.operators.sagemaker_base import SageMakerBaseOperator
from airflow.utils.decorators import apply_defaults

Expand Down Expand Up @@ -48,7 +48,7 @@ def __init__(self,

def expand_role(self):
if 'ExecutionRoleArn' in self.config:
hook = AwsHook(self.aws_conn_id)
hook = AwsBaseHook(self.aws_conn_id)
self.config['ExecutionRoleArn'] = hook.expand_role(self.config['ExecutionRoleArn'])

def execute(self, context):
Expand Down
Loading

0 comments on commit 9cbd7de

Please sign in to comment.