Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add two operators in AWS Providers: RedshiftResumeClusterOperator and RedshiftPauseClusterOperator #19665

Merged
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
Show all changes
29 commits
Select commit Hold shift + click to select a range
580dd93
This commit adds new features to the airflow aws redshift module. Sim…
Nov 17, 2021
7a921c5
Add missing requirements for Static checks. Add license to all python…
Nov 18, 2021
0edb98d
Adding changes for pre-commit success
Nov 18, 2021
9911c72
Adding `ClusterStates` Enum to redshift hook to avoid having magic st…
Nov 20, 2021
a68cbbf
Adding documentation to existing AWS operator docs
Nov 22, 2021
b20a38e
Moving the cast to `ClusterStates` Enum to inside the `cluster_status…
Nov 22, 2021
81f2f75
Fixed logging to redshift sensor to show the actual value of the ENUM…
Nov 22, 2021
61b2935
Fixed logging to redshift sensor to show the actual value of the ENUM…
Nov 29, 2021
3b3f1e0
Adding `seealso` to Operator documentation
Nov 30, 2021
b151337
Update airflow/providers/amazon/aws/operators/redshift_resume_cluster.py
dbarrundiag Nov 30, 2021
ec92273
Update airflow/providers/amazon/aws/operators/redshift_pause_cluster.py
dbarrundiag Nov 30, 2021
3ce29da
Removing unused `check_interval` from operator
Nov 30, 2021
456cb73
Merge remote-tracking branch 'origin/redshift-resume-pause-cluster-op…
Nov 30, 2021
4d6e402
Update airflow/providers/amazon/aws/sensors/redshift.py
dbarrundiag Nov 30, 2021
57c4c35
Update docs/apache-airflow-providers-amazon/operators/redshift.rst
dbarrundiag Nov 30, 2021
e03031f
Update docs/apache-airflow-providers-amazon/operators/redshift.rst
dbarrundiag Nov 30, 2021
92cfcf2
Update tests/providers/amazon/aws/sensors/test_redshift.py
dbarrundiag Nov 30, 2021
06a7e3d
Rolling back changes to not use enum and move all Operators into one …
Dec 7, 2021
d20662a
Merge remote-tracking branch 'origin/redshift-resume-pause-cluster-op…
Dec 7, 2021
e72bec0
Rolling back changes to not use enum and move all Operators into one …
Dec 7, 2021
e659253
Rolling back changes to not use enum and move all Operators into one …
Dec 7, 2021
b303dc7
Rolling back changes to not use enum and move all Operators into one …
Dec 7, 2021
6c63fed
Update airflow/providers/amazon/aws/operators/redshift.py
dbarrundiag Dec 7, 2021
3ac8f33
Update airflow/providers/amazon/aws/operators/redshift.py
dbarrundiag Dec 7, 2021
50becc6
Removing no longer required subclass
Dec 7, 2021
653d6b2
Merge remote-tracking branch 'origin/redshift-resume-pause-cluster-op…
Dec 7, 2021
f012c29
Add unittests for TestPauseClusterOperator and TestResumeClusterOperator
Dec 9, 2021
a7ff6ca
Clean unittest for redshift Operators
Dec 9, 2021
f4ac7a4
Small fixups
Dec 9, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
25 changes: 21 additions & 4 deletions airflow/providers/amazon/aws/hooks/redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,16 @@
# specific language governing permissions and limitations
# under the License.
"""Interact with AWS Redshift clusters."""

import time
from typing import Dict, List, Optional, Union

try:
from functools import cached_property
except ImportError:
from cached_property import cached_property

from enum import Enum

import redshift_connector
from redshift_connector import Connection as RedshiftConnection
from sqlalchemy import create_engine
Expand All @@ -33,6 +35,21 @@
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook


class ClusterStates(Enum):
"""Contains the possible State values of a Redshift Cluster."""

AVAILABLE = 'available'
CREATING = 'creating'
DELETING = 'deleting'
RESUMING = 'resuming'
MODIFYING = 'modifying'
PAUSED = 'paused'
REBOOTING = 'rebooting'
RENAMING = 'renaming'
RESIZING = 'resizing'
NONEXISTENT = 'nonexistent'


class RedshiftHook(AwsBaseHook):
"""
Interact with AWS Redshift, using the boto3 library
Expand All @@ -52,7 +69,7 @@ def __init__(self, *args, **kwargs) -> None:
super().__init__(*args, **kwargs)

# TODO: Wrap create_cluster_snapshot
def cluster_status(self, cluster_identifier: str) -> str:
def cluster_status(self, cluster_identifier: str) -> ClusterStates:
"""
Return status of a cluster

Expand All @@ -65,9 +82,9 @@ def cluster_status(self, cluster_identifier: str) -> str:
"""
try:
response = self.get_conn().describe_clusters(ClusterIdentifier=cluster_identifier)['Clusters']
return response[0]['ClusterStatus'] if response else None
return ClusterStates(response[0]['ClusterStatus']) if response else None
except self.get_conn().exceptions.ClusterNotFoundFault:
return 'cluster_not_found'
dstandish marked this conversation as resolved.
Show resolved Hide resolved
return ClusterStates.NONEXISTENT

def delete_cluster(
self,
Expand Down
58 changes: 58 additions & 0 deletions airflow/providers/amazon/aws/operators/redshift_pause_cluster.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
#
# 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 airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.redshift import ClusterStates, RedshiftHook


class RedshiftPauseClusterOperator(BaseOperator):
"""
Pause an AWS Redshift Cluster using boto3.
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved

:param cluster_identifier: id of the AWS Redshift Cluster
:type cluster_identifier: str
:param aws_conn_id: aws connection to use
:type aws_conn_id: str
:param check_interval: time in seconds that the job should wait in
between each instance state checks until operation is completed
:type check_interval: float
"""

template_fields = ("cluster_identifier",)
ui_color = "#eeaa11"
ui_fgcolor = "#ffffff"

def __init__(
self,
*,
cluster_identifier: str,
aws_conn_id: str = "aws_default",
check_interval: float = 15,
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
**kwargs,
):
super().__init__(**kwargs)
self.cluster_identifier = cluster_identifier
self.aws_conn_id = aws_conn_id
self.check_interval = check_interval

def execute(self, context):
redshift_hook = RedshiftHook(aws_conn_id=self.aws_conn_id)
self.log.info("Pausing Redshift cluster %s", self.cluster_identifier)
cluster_state = redshift_hook.cluster_status(cluster_identifier=self.cluster_identifier)
if cluster_state == ClusterStates.AVAILABLE:
redshift_hook.get_conn().pause_cluster(ClusterIdentifier=self.cluster_identifier)
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
58 changes: 58 additions & 0 deletions airflow/providers/amazon/aws/operators/redshift_resume_cluster.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
#
# 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 airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.redshift import ClusterStates, RedshiftHook


class RedshiftResumeClusterOperator(BaseOperator):
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
"""
Resume an AWS Redshift Cluster using boto3.
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved

:param cluster_identifier: id of the AWS Redshift Cluster
:type cluster_identifier: str
:param aws_conn_id: aws connection to use
:type aws_conn_id: str
:param check_interval: time in seconds that the job should wait in
between each instance state checks until operation is completed
:type check_interval: float
"""

template_fields = ("cluster_identifier",)
ui_color = "#eeaa11"
ui_fgcolor = "#ffffff"

def __init__(
self,
*,
cluster_identifier: str,
aws_conn_id: str = "aws_default",
check_interval: float = 15,
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
**kwargs,
):
super().__init__(**kwargs)
self.cluster_identifier = cluster_identifier
self.aws_conn_id = aws_conn_id
self.check_interval = check_interval

def execute(self, context):
redshift_hook = RedshiftHook(aws_conn_id=self.aws_conn_id)
self.log.info("Starting Redshift cluster %s", self.cluster_identifier)
cluster_state = redshift_hook.cluster_status(cluster_identifier=self.cluster_identifier)
if cluster_state == ClusterStates.PAUSED:
redshift_hook.get_conn().resume_cluster(ClusterIdentifier=self.cluster_identifier)
13 changes: 8 additions & 5 deletions airflow/providers/amazon/aws/sensors/redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# under the License.
from typing import Optional

from airflow.providers.amazon.aws.hooks.redshift import RedshiftHook
from airflow.providers.amazon.aws.hooks.redshift import ClusterStates, RedshiftHook
from airflow.sensors.base import BaseSensorOperator


Expand All @@ -28,7 +28,7 @@ class AwsRedshiftClusterSensor(BaseSensorOperator):
:param cluster_identifier: The identifier for the cluster being pinged.
:type cluster_identifier: str
:param target_status: The cluster status desired.
:type target_status: str
:type target_status: ClusterStates
"""

template_fields = ('cluster_identifier', 'target_status')
Expand All @@ -37,18 +37,21 @@ def __init__(
self,
*,
cluster_identifier: str,
target_status: str = 'available',
target_status: ClusterStates = ClusterStates.AVAILABLE,
aws_conn_id: str = 'aws_default',
**kwargs,
):
super().__init__(**kwargs)
self.cluster_identifier = cluster_identifier
self.target_status = target_status
self.target_status = (
target_status if isinstance(target_status, ClusterStates) else ClusterStates(str(target_status))
)

self.aws_conn_id = aws_conn_id
self.hook: Optional[RedshiftHook] = None

dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
def poke(self, context):
self.log.info('Poking for status : %s\nfor cluster %s', self.target_status, self.cluster_identifier)
self.log.info('Poking for status : %s\nfor cluster %s', self.target_status.value, self.cluster_identifier)
return self.get_hook().cluster_status(self.cluster_identifier) == self.target_status

def get_hook(self) -> RedshiftHook:
Expand Down
2 changes: 2 additions & 0 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,8 @@ operators:
- integration-name: Amazon Redshift
python-modules:
- airflow.providers.amazon.aws.operators.redshift
- airflow.providers.amazon.aws.operators.redshift_pause_cluster
- airflow.providers.amazon.aws.operators.redshift_resume_cluster

sensors:
- integration-name: Amazon Athena
Expand Down
47 changes: 28 additions & 19 deletions docs/apache-airflow-providers-amazon/operators/redshift.rst
Original file line number Diff line number Diff line change
Expand Up @@ -15,17 +15,19 @@
specific language governing permissions and limitations
under the License.

.. _howto/operator:RedshiftSQLOperator:
Amazon Redshift Operators
=================================================

RedshiftSQLOperator
===================
`Amazon Redshift <https://aws.amazon.com/redshift/>`__ is a fully managed,
petabyte-scale data warehouse service in the cloud. You can start with just a few hundred gigabytes
of data and scale to a petabyte or more. This enables you to use your data to acquire new insights
for your business and customers.

.. contents::
:depth: 1
:local:
Airflow provides operators to create and interact with the Redshift clusters and compute infrastructure.

Overview
--------
RedshiftSQLOperator
^^^^^^^^^^^^^^^^^^^^^^^^^^
.. _howto/operator:RedshiftSQLOperator:

Use the :class:`RedshiftSQLOperator <airflow.providers.amazon.aws.operators.redshift>` to execute
statements against an Amazon Redshift cluster.
Expand All @@ -34,16 +36,6 @@ statements against an Amazon Redshift cluster.
:class:`RedshiftSQLHook <airflow.providers.amazon.aws.hooks.redshift.RedshiftSQLHook>` to establish
connections with Amazon Redshift.


example_redshift.py
-------------------

Purpose
"""""""

This is a basic example dag for using :class:`RedshiftSQLOperator <airflow.providers.amazon.aws.operators.redshift>`
to execute statements against an Amazon Redshift cluster.

Create a table
""""""""""""""

Expand Down Expand Up @@ -86,11 +78,28 @@ parameters into SQL statements.
:end-before: [END howto_operator_redshift_get_with_filter]

The complete RedshiftSQLOperator DAG
------------------------------------
"""""""""""""""""""""""""""""""""""""""""""

All together, here is our DAG:

.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
:language: python
:start-after: [START redshift_operator_howto_guide]
:end-before: [END redshift_operator_howto_guide]


.. _howto/operator:RedshiftResumeClusterOperator:
Resume a Redshift Cluster
"""""""""""""""""""""""""""""""""""""""""""
To resume an existing AWS Redshift Cluster you can use
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
:class:`~airflow.providers.amazon.aws.operators.redshift_resume_cluster.RedshiftResumeClusterOperator`.

This Operator leverages the AWS CLI `resume-cluster <https://docs.aws.amazon.com/cli/latest/reference/redshift/resume-cluster.html>`__ API

.. _howto/operator:RedshiftPauseClusterOperator:
Pause a Redshift Cluster
"""""""""""""""""""""""""""""""""""""""""""
To pause an existing AWS Redshift Cluster you can use
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
:class:`~airflow.providers.amazon.aws.operators.redshift_pause_cluster.RedshiftPauseClusterOperator`.

This Operator leverages the AWS CLI `pause-cluster <https://docs.aws.amazon.com/cli/latest/reference/redshift/pause-cluster.html>`__ API
6 changes: 3 additions & 3 deletions tests/providers/amazon/aws/hooks/test_redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@

from airflow.models import Connection
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.aws.hooks.redshift import RedshiftHook, RedshiftSQLHook
from airflow.providers.amazon.aws.hooks.redshift import ClusterStates, RedshiftHook, RedshiftSQLHook

try:
from moto import mock_redshift
Expand Down Expand Up @@ -98,15 +98,15 @@ def test_cluster_status_returns_cluster_not_found(self):
self._create_clusters()
hook = RedshiftHook(aws_conn_id='aws_default')
status = hook.cluster_status('test_cluster_not_here')
assert status == 'cluster_not_found'
assert status == ClusterStates.NONEXISTENT

@unittest.skipIf(mock_redshift is None, 'mock_redshift package not present')
@mock_redshift
def test_cluster_status_returns_available_cluster(self):
self._create_clusters()
hook = RedshiftHook(aws_conn_id='aws_default')
status = hook.cluster_status('test_cluster')
assert status == 'available'
assert status == ClusterStates.AVAILABLE


class TestRedshiftSQLHookConn(unittest.TestCase):
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
#
# 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.
#

import unittest

import boto3

from airflow.providers.amazon.aws.operators.redshift_pause_cluster import RedshiftPauseClusterOperator

try:
from moto import mock_redshift
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
except ImportError:
mock_redshift = None


class TestPauseClusterOperator(unittest.TestCase):
dbarrundiag marked this conversation as resolved.
Show resolved Hide resolved
@staticmethod
def _create_clusters():
client = boto3.client('redshift', region_name='us-east-1')
client.create_cluster(
ClusterIdentifier='test_cluster_to_pause',
NodeType='dc1.large',
MasterUsername='admin',
MasterUserPassword='mock_password',
)
client.create_cluster(
ClusterIdentifier='test_cluster_to_resume',
NodeType='dc1.large',
MasterUsername='admin',
MasterUserPassword='mock_password',
)
if not client.describe_clusters()['Clusters']:
raise ValueError('AWS not properly mocked')

def test_init(self):
redshift_operator = RedshiftPauseClusterOperator(
task_id="task_test",
cluster_identifier="test_cluster",
aws_conn_id="aws_conn_test",
check_interval=3,
)
assert redshift_operator.task_id == "task_test"
assert redshift_operator.cluster_identifier == "test_cluster"
assert redshift_operator.aws_conn_id == "aws_conn_test"
assert redshift_operator.check_interval == 3
Loading