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 all 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
84 changes: 83 additions & 1 deletion airflow/providers/amazon/aws/operators/redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
from typing import Dict, Iterable, Optional, Union

from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
from airflow.providers.amazon.aws.hooks.redshift import RedshiftHook, RedshiftSQLHook


class RedshiftSQLOperator(BaseOperator):
Expand Down Expand Up @@ -71,3 +71,85 @@ def execute(self, context: dict) -> None:
self.log.info(f"Executing statement: {self.sql}")
hook = self.get_hook()
hook.run(self.sql, autocommit=self.autocommit, parameters=self.parameters)


class RedshiftResumeClusterOperator(BaseOperator):
"""
Resume a paused AWS Redshift Cluster
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:RedshiftResumeClusterOperator`
: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
"""

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

def __init__(
self,
*,
cluster_identifier: str,
aws_conn_id: str = "aws_default",
**kwargs,
):
super().__init__(**kwargs)
self.cluster_identifier = cluster_identifier
self.aws_conn_id = aws_conn_id

def execute(self, context):
redshift_hook = RedshiftHook(aws_conn_id=self.aws_conn_id)
cluster_state = redshift_hook.cluster_status(cluster_identifier=self.cluster_identifier)
if cluster_state == 'paused':
self.log.info("Starting Redshift cluster %s", self.cluster_identifier)
redshift_hook.get_conn().resume_cluster(ClusterIdentifier=self.cluster_identifier)
else:
self.log.warning(
"Unable to resume cluster since cluster is currently in status: %s", cluster_state
)


class RedshiftPauseClusterOperator(BaseOperator):
"""
Pause an AWS Redshift Cluster if it has status `available`.
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:RedshiftPauseClusterOperator`
: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
"""

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

def __init__(
self,
*,
cluster_identifier: str,
aws_conn_id: str = "aws_default",
**kwargs,
):
super().__init__(**kwargs)
self.cluster_identifier = cluster_identifier
self.aws_conn_id = aws_conn_id

def execute(self, context):
redshift_hook = RedshiftHook(aws_conn_id=self.aws_conn_id)
cluster_state = redshift_hook.cluster_status(cluster_identifier=self.cluster_identifier)
if cluster_state == 'available':
self.log.info("Pausing Redshift cluster %s", self.cluster_identifier)
redshift_hook.get_conn().pause_cluster(ClusterIdentifier=self.cluster_identifier)
else:
self.log.warning(
"Unable to pause cluster since cluster is currently in status: %s", cluster_state
)
22 changes: 22 additions & 0 deletions docs/apache-airflow-providers-amazon/operators/redshift.rst
Original file line number Diff line number Diff line change
Expand Up @@ -94,3 +94,25 @@ All together, here is our DAG:
: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 a 'paused' AWS Redshift Cluster you can use
:class:`RedshiftResumeClusterOperator <airflow.providers.amazon.aws.operators.redshift>`

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 'available' AWS Redshift Cluster you can use
:class:`RedshiftPauseClusterOperator <airflow.providers.amazon.aws.operators.redshift>`
This Operator leverages the AWS CLI
`pause-cluster <https://docs.aws.amazon.com/cli/latest/reference/redshift/pause-cluster.html>`__ API
66 changes: 65 additions & 1 deletion tests/providers/amazon/aws/operators/test_redshift.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,11 @@

from parameterized import parameterized

from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
from airflow.providers.amazon.aws.operators.redshift import (
RedshiftPauseClusterOperator,
RedshiftResumeClusterOperator,
RedshiftSQLOperator,
)


class TestRedshiftSQLOperator(unittest.TestCase):
Expand All @@ -42,3 +46,63 @@ def test_redshift_operator(self, test_autocommit, test_parameters, mock_get_hook
autocommit=test_autocommit,
parameters=test_parameters,
)


class TestResumeClusterOperator:
def test_init(self):
redshift_operator = RedshiftResumeClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
assert redshift_operator.task_id == "task_test"
assert redshift_operator.cluster_identifier == "test_cluster"
assert redshift_operator.aws_conn_id == "aws_conn_test"

@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.cluster_status")
@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.get_conn")
def test_resume_cluster_is_called_when_cluster_is_paused(self, mock_get_conn, mock_cluster_status):
mock_cluster_status.return_value = 'paused'
redshift_operator = RedshiftResumeClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
redshift_operator.execute(None)
mock_get_conn.return_value.resume_cluster.assert_called_once_with(ClusterIdentifier='test_cluster')

@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.cluster_status")
@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.get_conn")
def test_resume_cluster_not_called_when_cluster_is_not_paused(self, mock_get_conn, mock_cluster_status):
mock_cluster_status.return_value = 'available'
redshift_operator = RedshiftResumeClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
redshift_operator.execute(None)
mock_get_conn.return_value.resume_cluster.assert_not_called()


class TestPauseClusterOperator:
def test_init(self):
redshift_operator = RedshiftPauseClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
assert redshift_operator.task_id == "task_test"
assert redshift_operator.cluster_identifier == "test_cluster"
assert redshift_operator.aws_conn_id == "aws_conn_test"

@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.cluster_status")
@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.get_conn")
def test_pause_cluster_is_called_when_cluster_is_available(self, mock_get_conn, mock_cluster_status):
mock_cluster_status.return_value = 'available'
redshift_operator = RedshiftPauseClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
redshift_operator.execute(None)
mock_get_conn.return_value.pause_cluster.assert_called_once_with(ClusterIdentifier='test_cluster')

@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.cluster_status")
@mock.patch("airflow.providers.amazon.aws.hooks.redshift.RedshiftHook.get_conn")
def test_pause_cluster_not_called_when_cluster_is_not_available(self, mock_get_conn, mock_cluster_status):
mock_cluster_status.return_value = 'paused'
redshift_operator = RedshiftPauseClusterOperator(
task_id="task_test", cluster_identifier="test_cluster", aws_conn_id="aws_conn_test"
)
redshift_operator.execute(None)
mock_get_conn.return_value.pause_cluster.assert_not_called()