You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/09/22 22:05:18 UTC

[GitHub] [airflow] Brooke-white opened a new pull request #18447: add RedshiftStatementHook, RedshiftOperator

Brooke-white opened a new pull request #18447:
URL: https://github.com/apache/airflow/pull/18447


   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   closes: #16355
   
   Adds `RedshiftStatementHook` and `RedshiftOperator`, which allow Airflow users to execute Redshift operations.
   
   `RedshiftStatementHook` is supported by the `redshift_connector` database driver which supports authenticating via IAM or Identity Provider. Integration with `sqlalchemy` is supported via `sqlalchemy-redshift`. 
   
   Unit tests have been added to:
   - Test Hook -- ensuring connection parameters flow through to `redshift_connector` as expected
   - Test Operator -- ensuring statements flow through to `redshift_connector.execute` method as expected
   
   Docs have been added to:
   - explain how to build a connection
   - provide an example DAG with create/insert/select statements
   - provide reference to `redshift_connector` docs for details about supported connection options
   
   
   ### Use case / motivation
   Please see #16355
   tl;dr: Provides support for IAM & Identity provider authentication as well as Redshift specific datatype support (e.g. GEOMETRY, SUPER, etc.).
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.md).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715267998



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""

Review comment:
       addressed in aa52988




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714439393



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""

Review comment:
       Is this comment correct? Or does it apply to the RedshiftHook?

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""

Review comment:
       Is this comment correct, Or does it only apply to the RedshiftHook?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r722744704



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)

Review comment:
       fixed in 2456aaf118ae3955809ccda187cd16e32e1172d6

##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,82 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+Database Authentication
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://awsuser:password@redshift-cluster-1.123456789.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=True
+
+IAM Authentication using AWS Profile
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default

Review comment:
       Thank you for noticing this detail! I've taken your advice and reworked these examples to use `Connection` and `get_uri()` in 
   6cfa652, so readers can see what the generated URL looks like. This approach is much more readable than the prior.
   
   Thanks for sharing re: SSM, it would definitely be useful in simplifying this use case :)

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,

Review comment:
       good catch -- thanks for noticing this. fixed in 
   d7899a7

##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +107,67 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(
+            conn_type='redshift', login='login', password='password', host='host', port=5439, schema="dev"
+        )
+
+        self.db_hook = RedshiftSQLHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        expected = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert x == expected
+
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn(self, mock_connect):
+        self.db_hook.get_conn()
+        mock_connect.assert_called_once_with(
+            user='login', password='password', host='host', port=5439, database='dev'
+        )
+
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn_extra(self, mock_connect):
+        self.connection.extra = json.dumps(
+            {
+                "iam": True,
+                "cluster_identifier": "my-test-cluster",
+                "profile": "default",
+            }
+        )
+        self.db_hook.get_conn()
+        mock_connect.assert_called_once_with(
+            user='login',
+            password='password',
+            host='host',
+            port=5439,
+            cluster_identifier="my-test-cluster",
+            profile="default",
+            database='dev',
+            iam=True,
+        )
+
+    @parameterized.expand(
+        [
+            ({}, {}, {}),
+            ({"login": "test"}, {}, {"user": "test"}),
+            ({}, {"user": "test"}, {"user": "test"}),
+            ({"login": "original"}, {"user": "overridden"}, {"user": "overridden"}),
+            ({"login": "test1"}, {"password": "test2"}, {"user": "test1", "password": "test2"}),
+        ],
+    )
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn_overrides_correctly(self, test_args, test_kwargs, expected_call_args, mock_connect):

Review comment:
       good point -- i've fixed this in 3821f40




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931488674


   @ashb, PostgresHook does not support authenticating via identity provider. That functionality is a big part of #16355. 
   
   IMO It would make more sense to have a separate hook for Redshift as it's a different product than Postgres. PostgresHook is using a driver designed for Postgres. RedshiftSQLHook would use a driver designed for Redshift.
   
   > Geometry and Super are returned as strings -- isn't that the default behaviour for pscopg already?
   
   Geometry can also be hex encoded under OID 3999, and requires manipulation by the driver before being returned to the user.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720474477



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       I reworked this in 1b22d59c1d3f25370b81c5f414a6e129a320e0ab. The overridden `get_uri()` method is removed in exchange for `_get_sqlalchemy_uri()` which contains this `user -> username` change. `_get_sqlalchemy_uri()` is used in the overriden `get_sqlalchemy_engine()` method.
   
   The `user -> username` key change is required for the sqlalchemy URL method, which expects `username`, while the driver itself expects `user`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720720951



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +133,91 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the schema. Additional connection
+    options can be passed to extra as a JSON string.

Review comment:
       I think it would be best to point to the connection howto at `_howto/connection:redshift` here and don't address connection details.
   
   One thing that's confusing about the wording here is that `schema` is ambiguous because you really mean `Connection` schema and redshift _database_.  And while you could update the wording here, I think better is to just make sure that usage is clear in the howto and point them there -- no need to duplicate.
   
   In the howto, I would add a simple example of connection instantiation in the most common one or two scenarios (e.g. db auth and iam).  Something that wasn't obvious to me is that when trying IAM is you actually have to use `db_user` and not `user`.  An example for this would help.
   
   Additionally, instead of forcing user to use conn attrs (in this case conn.schema) I would recommend allowing the user to connection params in either extra or conn attributes.
   
   So in this case where you have `conn.schema` required, better would be to allow `extra_dejson['database'] or conn.schema`.  And while you could add parameter validation logic for that, I think it's better to just defer to the library on that, especially since all but one are only _conditionally_ required.
   
   With this library, having ambiguity both with database vs schema and more importantly with user vs db_user vs login, I think it may be simplest for users to just dump everything in extra and therefore we should make sure to support that.
   
   So what does this boil down to in terms of changes...
   
   I think it's best to simply remove the `Connection.schema` requirement (and let the library complain when it doesn't have the right kwargs).  Then if conn.schema is not provided, but `database` is provided in extra, it will still work (and I think your code already supports this, once you remove the parameter validation).  And same with `conn.login` and `user` -- let extra override conn attr.  
   
   And could you please add tests that verify the connection parsing behavior -- i.e. what happens when passed in conn attr and conn extra, or when not in attr but yes in extra, and when passed in both.  
   
   Thanks for bearing with the iteration on this one... getting to know some more of the details as we go, and we're definitely getting closer.
   
   ---
   
   Separately.... I just want to mention that it might be helpful if, in redshift_connector, you raise a more helpful error message when the parameter combination is bad in a couple cases.  In particular when i tried iam with `user` (instead of `db_user` i got this:
   ```
       cache_key: str = IamHelper.get_credentials_cache_key(info)
     File "...python3.8/site-packages/redshift_connector/iam_helper.py", line 330, in get_credentials_cache_key
       return ";".join(
   TypeError: sequence item 0: expected str instance, NoneType found
   ```
   I think the real problem is that I need to use `db_user` instead of `user`. 
   
   Similarly when I omitted `database` I got this error:
   ```
   File "...python3.8/site-packages/redshift_connector/core.py", line 663, in __init__
       raise self.error
   redshift_connector.error.ProgrammingError: {'S': 'FATAL', 'C': '3D000', 'M': 'database "IAM:my_admin" does not exist', 'F': '/home/ec2-user/padb/src/pg/src/backend/utils/init/postinit.c', 'L': '517', 'R': 'LcInitPostgres'}
   ```
   And the problem was that I did not supply `database`.   Raising a more helpful error message in this case would make it more friendly for us to defer parameter validation to the library :) 
   
   Thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718046672



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       > In the next major release for this provider (which now that i think of it does not have to be in airflow 3.0, since providers have separate release schedule), we remove RedshiftHook and we are left with RedshiftSqlHook and RedshiftClusterHook
   
   It is not required that we delete deprecated classes in the next major release if maintenance is not burdensome. See discussion: https://apache-airflow.slack.com/archives/CCPRP7943/p1631032457154300




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-937374301


   This looks good to me.  Anyone else want to give it a second look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931507228


   Cool, makes sense then @dstandish!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-937374301


   This looks good to me.  Anyone else want to give it a second look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720147064



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",
+        params={'color': 'Red'},

Review comment:
       `params` is a BaseOperator thing -- I think you want `parameters`, and this:
   
   ```suggestion
           sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = :color;",
           parameters={'color': 'Red'},
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718837637



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User

Review comment:
       @mik-laj, in this hook, `login` and `schema` are [renamed](https://github.com/apache/airflow/pull/18447/files#diff-a6af6323bf6848d19c7d8066497ebebe20d221398642f5e1340b336c26eb2d87R167) to `User` and `Database` in the airflow UI.  Just wondering if this is something we encourage or not since I haven't seen this before.  To me, as someone who prefers using secrets backend to the airflow metastore, it would seem better to stick with `login` and `schema`, and use those uniformly, since those are the Connection attribute names, and those are what you'd have to use in secrets backend storage (e.g. if using a json backend) or when using Connection.get_uri to generate the airflow URI.
   

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,

Review comment:
       ```suggestion
           sql: Union[str, List[str]],
   ```

##########
File path: docs/apache-airflow-providers-amazon/operators/redshift.rst
##########
@@ -0,0 +1,96 @@
+ .. 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.
+
+.. _howto/operator:RedshiftSqlOperator:
+
+RedshiftSqlOperator
+===================
+
+.. contents::
+  :depth: 1
+  :local:
+
+Overview
+--------
+
+Use the :class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift>` to execute
+statements against an Amazon Redshift cluster.
+
+:class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift.RedshiftSqlOperator>` works together with
+: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
+""""""""""""""
+
+In the following code we are creating a table called "fruit".
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_create_table]
+    :end-before: [END howto_operator_redshift_create_table]
+
+Insert data into a table
+""""""""""""""""""""""""
+
+In the following code we insert a few sample rows into the "fruit" table.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_populate_table]
+    :end-before: [END howto_operator_redshift_populate_table]
+
+Fetching records from a table
+"""""""""""""""""""""""""""""
+
+Retrieving all records from the "fruit" table.

Review comment:
       alright so this may be nitpicky but ... it strikes me that the select * examples (in your example dag) might be confusing to a newcomer.   in reality this is not the way you would want to use this operator unless you did it with `handler` passed to `hook.run` (because the output of the select will not be captured anywhere).  you've written "retrieving" but i don't think that you actually retrieve them anywhere (e.g. neither printing to logs nor routing to csv -- i don't think any rows are actually fetched).
   
   just to be more realistic i would suggest simply converting them to `create table my_table AS select *`.  otherwise a newcomer might look at this and assume that the records selected here are actually sent somewhere.
   

##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +106,51 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftSQLHook(RedshiftSQLHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'

Review comment:
       I'm curious why you are making this subclass here when it does not appear to differ from RedshiftSQLHook




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714524245



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""

Review comment:
       It’s quite weird this functions is called exactly twice, once discarding the first element in the tuple, the other discarding the second. That seems to indicate this should be two separate functions instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r724408055



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       This isn't addressed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718029637



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       This sounds good. As @josh-fell mentioned,
   > Although, this suggestion would may warrant renaming the new operator to match.
   
   We can rename the operator to `RedshiftSqlOperator` as well so the naming is inline with `RedshiftSqlHook`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717846390



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member

Review comment:
       addressed in c9e182047b12474f00e9e5abca13a90364a44a0c




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720469019



##########
File path: airflow/providers/amazon/provider.yaml
##########
@@ -441,6 +447,8 @@ connection-types:
     connection-type: aws
   - hook-class-name: airflow.providers.amazon.aws.hooks.emr.EmrHook
     connection-type: emr
+  - hook-class-name: airflow.providers.amazon.aws.hooks.redshift.RedshiftSQLHook
+    connection-type: redshift+redshift_connector

Review comment:
       fixed in 1b22d59c1d3f25370b81c5f414a6e129a320e0ab




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r722690470



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,82 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+Database Authentication
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://awsuser:password@redshift-cluster-1.123456789.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=True
+
+IAM Authentication using AWS Profile
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default

Review comment:
       Ok so dealing with airflow URIs is a bit tricky.
   
   And in this case I think there's a small problem.
   
   Look at the handling of the `iam` parameter:
   
   ```python
   >>> from airflow.models.connection import Connection
   >>> c = Connection(uri='redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default')
   >>> c.extra_dejson
   {'database': 'dev', 'iam': 'True', 'db_user': 'awsuser', 'cluster_identifier': 'redshift-cluster-1', 'profile': 'default'}
   ```
   
   And if redshift connector gets `iam='True'` instead of `iam=True` it won't work.
   
   While we _could_ implement logic to handle this in the hook, we don't need to because there's a way to produce the URI such that we avoid this issue.
   
   We have a `get_uri` method on `Connection` that produces the URI from a connection object.  And when doing the standard URI encoding will lose fidelity (e.g. bool converted to string on reparsing) then it will use the alternative representation of extra:
   ```python
   >>> c = Connection(conn_type='redshift', extra=json.dumps({"database":"dev", "iam":True,"db_user":"awsuser", "cluster_identifier":"redshift-cluster-1", "profile":"default"}))
   >>> c.get_uri()
   'redshift://?__extra__=%7B%22database%22%3A+%22dev%22%2C+%22iam%22%3A+true%2C+%22db_user%22%3A+%22awsuser%22%2C+%22cluster_identifier%22%3A+%22redshift-cluster-1%22%2C+%22profile%22%3A+%22default%22%7D'
   ```
   
   It's ugly but it works.
   
   So bottom line I think here it would be to just show examples of how to define the connections using `Connection` instead of URI format.  It's the easiest way to produce the correct URI.  And you could also show the generated URI and how you produced it, or just point to the "generating an airflow URI" section in the corehowto / "managing connections" doc, where this is covered in detail.
   
   ---
   
   Side note I hope to implement support for json serialization broadly (i.e. as an alternative to airflow URI) along the lines done [here with SSM](https://github.com/apache/airflow/pull/18692), which will make this a little less painful.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720574828



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",
+        params={'color': 'Red'},

Review comment:
       fixed to use `parameters` in 66ba154c6e5fdc87e36eb97054cf0ac22fd295ca




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720144074



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       ```suggestion
   from airflow.compat.functools import cached_property
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720416109



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",

Review comment:
       please see my question [here](https://github.com/apache/airflow/pull/18447/files#r720411143)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720468726



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       fixed in ea1f49073f545e3c0d3eaa0733dc94a1ac206633




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720555874



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       I've brought `get_uri` back in 4589472, and removed `_get_sqlalchemy_uri` as it's unnecessarily confusing to have both :)
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720415219



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       yeah i think this may just be the wrong language (namely the use of `must`):
   
   > This connection must be initialized with the host, port, login, password
   
   i think they are all optional depending on the circumstance.  e.g. if you pass `cluster_identifier` as a kwarg (through extra) i don't think you would need host.  and if you are using iam you would not need password. and you don't need port, this i've verified.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714833401



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,70 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_s3_to_redshift_create_table]
+    setup__task_create_table = RedshiftOperator(
+        redshift_conn_id='redshift_default',

Review comment:
       You don't necessarily need to pass in the `redshift_conn_id` parameter to each `RedshiftOperator` task since the default value is being used here. 
   
   Or, if you would like to specify a non-default value for `redshift_conn_id` you could add it into the DAGs `default_args` since this arg is passed to every operator in the DAG.

##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,66 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+Login
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector. The following parameter is required:
+
+    * ``database``: Amazon Redshift database name.

Review comment:
       To be consistent with the connection field renaming in the `RedshiftStatementHook`, it might be more clear to rename "Login" to "User" as well as bring "Database" out of `Extra` and add as an explicit field.

##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,74 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(
+            login='login', password='password', host='host', port=5439, extra=json.dumps({"database": "dev"})
+        )

Review comment:
       Same comment here as in the `RedshiftStatementHook`, if the `schema` field is going to be renamed it seems more clear to change this to: 
   ```python
   self.connection = Connection(
          login='login', password='password', host='host', port=5439, schema="dev"})
   )
   ```

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),

Review comment:
       Since the `schema` field is being renamed to "Database" in the connection form, this should be:
   ```
   "database": conn.schema
   ```
   
   Otherwise, this may cause some confusion for users if there is a separate "Database" field (that is never used) but they have to input the desired database value in `Extra`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715269718



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']

Review comment:
       modified the aproach taken here in 4e2d43e. 
   
   The DB-API driver, redshift_connector, expects `user` while the function which builds the URL expects `username`. 

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']

Review comment:
       modified the approach taken here in 4e2d43e. 
   
   The DB-API driver, redshift_connector, expects `user` while the function which builds the URL expects `username`. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715269192



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""

Review comment:
       addressed in 6ef57a4 . broke this up into two functions




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714521670



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}

Review comment:
       [Do not set function argument default to a mutable value](https://docs.python-guide.org/writing/gotchas/#mutable-default-arguments).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931416466


   > as well as Redshift specific datatype support (e.g. GEOMETRY, SUPER, etc.).
   
   Geometry and Super are returned as strings -- isn't that the default behaviour for pscopg already?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714866413



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}

Review comment:
       Thanks for linking the explanation, I definitely wasn't aware of this behaviour. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720403036



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       The reason for this approach is that the method that builds the URL expects `username` while the DB-API driver expects `user`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720539871



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       > AFAIK, the only realon there is a get_uri method in hooks is for sqlalchemy
   
   In this case, the approach I took in 1b22d59 with `_get_sqlalchemy_uri` doesn't make sense.
   I thought I had seen `get_uri` mentioned for some other use in the docs somewhere but after taking another look I see that I was mistaken.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718004578



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       I think RedshiftSqlHook is an acceptable name, though I reached out on slack to see if anyone has some advice for how we could end up with `RedshiftHook` as the sql hook.

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       I think RedshiftSqlHook is an acceptable name, though I reached out on slack to see if anyone has some advice for how we could end up with `RedshiftHook` as the sql hook (which i think would be preferable if it can be done)

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   then in the next major release _for this provider_ (which now that i think of it does not have to be in 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   And those names we intend to stick with (i.e. ultimately we keep two hooks `RedshiftSqlHook` and `RedshiftClusterHook`)
   
   In the next major release _for this provider_ (which now that i think of it does not have to be in 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   And those names we intend to stick with (i.e. ultimately we keep two hooks `RedshiftSqlHook` and `RedshiftClusterHook`)
   
   In the next major release _for this provider_ (which now that i think of it does not have to be in airflow 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   

##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User

Review comment:
       @mik-laj, in this hook, `login` and `schema` are [renamed](https://github.com/apache/airflow/pull/18447/files#diff-a6af6323bf6848d19c7d8066497ebebe20d221398642f5e1340b336c26eb2d87R167) to `User` and `Database` in the airflow UI.  Just wondering if this is something we encourage or not since I haven't seen this before.  To me, as someone who prefers using secrets backend to the airflow metastore, it would seem better to stick with `login` and `schema`, and use those uniformly, since those are the Connection attribute names, and those are what you'd have to use in secrets backend storage (e.g. if using a json backend) or when using Connection.get_uri to generate the airflow URI.
   

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,

Review comment:
       ```suggestion
           sql: Union[str, List[str]],
   ```

##########
File path: docs/apache-airflow-providers-amazon/operators/redshift.rst
##########
@@ -0,0 +1,96 @@
+ .. 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.
+
+.. _howto/operator:RedshiftSqlOperator:
+
+RedshiftSqlOperator
+===================
+
+.. contents::
+  :depth: 1
+  :local:
+
+Overview
+--------
+
+Use the :class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift>` to execute
+statements against an Amazon Redshift cluster.
+
+:class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift.RedshiftSqlOperator>` works together with
+: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
+""""""""""""""
+
+In the following code we are creating a table called "fruit".
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_create_table]
+    :end-before: [END howto_operator_redshift_create_table]
+
+Insert data into a table
+""""""""""""""""""""""""
+
+In the following code we insert a few sample rows into the "fruit" table.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_populate_table]
+    :end-before: [END howto_operator_redshift_populate_table]
+
+Fetching records from a table
+"""""""""""""""""""""""""""""
+
+Retrieving all records from the "fruit" table.

Review comment:
       alright so this may be nitpicky but ... it strikes me that the select * examples (in your example dag) might be confusing to a newcomer.   in reality this is not the way you would want to use this operator unless you did it with `handler` passed to `hook.run` (because the output of the select will not be captured anywhere).  you've written "retrieving" but i don't think that you actually retrieve them anywhere (e.g. neither printing to logs nor routing to csv -- i don't think any rows are actually fetched).
   
   just to be more realistic i would suggest simply converting them to `create table my_table AS select *`.  otherwise a newcomer might look at this and assume that the records selected here are actually sent somewhere.
   

##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +106,51 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftSQLHook(RedshiftSQLHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'

Review comment:
       I'm curious why you are making this subclass here when it does not appear to differ from RedshiftSQLHook




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714442083



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg

Review comment:
       ```suggestion
           conn_kwargs: Dict = conn.extra_dejson
   ```
   
   Otherwise we're just deconstructing and reconstructing the same dictionary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720443372



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       I don't have much opinion, bit I don't really see what adding the default as a class property gives us.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-938821536


   Awesome work, congrats on your first merged pull request!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #18447:
URL: https://github.com/apache/airflow/pull/18447


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268973



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),

Review comment:
       addressed in 6ef57a4




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716879087



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       Would it be possible to include the hook under its current name, `RedshiftStatementHook`, so we don't need to wait for 3.0, and then add in some shim for backwards compatibility in the future? 
   
   i.e.
   ```python
   RedshiftHook = RedshiftStatementHook
   ```
   
   While RedshiftOperator would likely work with the Postgres hook, it would be lacking the primary feature requests in #16355, which include extended datatype support and authentication via IAM and Identity provider. 
   
   IMO, the authentication bits are especially important because they simplify an existing workflow (connecting to Redshift with temporary credentials), and open this integration up to a new group of users (those requiring authentication with Redshift via an identity provider (e.g. Okta, Azure)). 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716128423



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type

Review comment:
       i was gonna suggest cus i think it's easier to reason about without the `not`
   
   ```python
   conn_type = conn.conn_type if conn.conn_type else RedshiftStatementHook.conn_type
   ```
   
   but this is better, and I think still equivalent:
   
   ```suggestion
           conn_type = conn.conn_type or RedshiftStatementHook.conn_type
   ```
   
   

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(

Review comment:
       ok currently this code will make 2 or three trips to the secrets backend before it can create the redshift connection.  only one trip should be necessary.
   
   what I like to do is add a cached property `conn` which has the airflow connection object, so you don't have to make multiple trips.  
   
   then you might not need the `_get_conn_kwargs` since e.g. you could replace it with `self.conn.extra_dejson`

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @josh-fell what do you think about calling this `RedshiftHook` and renaming the existing one to `RedshiftManagementHook` or `RedshiftClusterHook`.  
   
   i think it was a mistake to call that one RedshiftHook -- it is not for using redshift but interacting with the API for cluster management tasks.
   
   and i'm sure this has been a source of confusion for redshift users over and over, and that running sql on redshift would be, by a long shot, the more commonly used hook.
   
   i recognize that this may require more work for us to implement, e.g. introducing a deprecation period, and having to wait for 3.0, but it would seem to be worth it.
   
   if it meant delaying the inclusion of this hook til 3.0 it would still be worth it IMO.  because the operator in this PR can still come in now and use postgres hook (which already works with redshift)
   

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,

Review comment:
       The redshift_connector library's connect function does not have port 5439 as a default (it's `None`) and you can connect without specifying.  so probably best to omit it here as well. 
   
   Probably it would be better to do `conn.port or None`
   
   perhaps better would be to only selectively add kwargs that have a meaningful value, and in this way to defer to the underlying library.
   

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type
+
+        return URL(
+            drivername=conn_type,
+            username=conn_params['user'],
+            password=conn_params['password'],
+            host=conn_params['host'],
+            port=conn_params['port'],
+            database=conn_params['database'],
+        ).__str__()
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        conn_kwargs = self._get_conn_kwargs()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params = self._get_conn_params()
+        conn_kwargs = self._get_conn_kwargs()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        conn: RedshiftConnection = redshift_connector.connect(**conn_kwargs)
+
+        return conn
+
+    def run(
+        self,
+        sql: Union[str, list],
+        autocommit: bool = False,
+        parameters: Optional[dict] = None,
+        handler: Optional[Callable] = None,
+    ):
+        """
+        Runs a command or a list of commands. Pass a list of sql
+        statements to the sql parameter to get them to execute
+        sequentially
+
+        :param sql: the sql statement to be executed (str) or a list of
+            sql statements to execute
+        :type sql: str or list
+        :param autocommit: What to set the connection's autocommit setting to
+            before executing the query.
+        :type autocommit: bool
+        :param parameters: The parameters to render the SQL query with.
+        :type parameters: dict or iterable
+        :param handler: The result handler which is called with the result of each statement.
+        :type handler: callable
+        :return: query results if handler was provided.
+        """
+        return super().run(sql, autocommit=False, parameters=parameters, handler=handler)

Review comment:
       you have hardcoded autocommit here, but you accept it as a param in this override.  you should ether use the param value or not accept the param.
   
   separately, assuming you resolve my first concern in this comment, it appears that your method override here will be exactly the same as the parent class, which suggest you should not override this method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931497586


   Some info on authenticating via identity provider using redshift_connector[1](https://aws.amazon.com/blogs/big-data/federated-api-access-to-amazon-redshift-using-an-amazon-redshift-connector-for-python/) [2](https://github.com/aws/amazon-redshift-python-driver/blob/master/tutorials/001%20-%20Connecting%20to%20Amazon%20Redshift.ipynb)
   
   The datatypes redshift_connector supports [3](https://github.com/aws/amazon-redshift-python-driver#supported-datatypes)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720470875



##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +106,47 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        self.db_hook = RedshiftSQLHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):

Review comment:
       added verification logic, and corresponding test in eac5cdacd75c0783f3418af24185f6e009a49c06




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r721602520



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +133,91 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the schema. Additional connection
+    options can be passed to extra as a JSON string.

Review comment:
       Thank you for taking the time to write this all up, I really appreciate all the time and effort you've put into this PR :)
   
   I've removed the parameter validation from the Hook, so we defer to the driver for that. Added examples to the connection docs for auth via IAM and Identity Provider. Test cases for the connection parsing behavior were added as well. 8f843d6
   
   Thank you for the feedback re: the driver's error messages when parameter combinations aren't valid. I'll pass this along to the Redshift driver team and see what we can do here. At the very least, improving the error messages & validation should be doable :). We *really* should be identifying these bad cases up front so users can receive a less cryptic error message rather than continuing through the code path with an incorrect configuration. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718025826



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   And those names we intend to stick with (i.e. ultimately we keep two hooks `RedshiftSqlHook` and `RedshiftClusterHook`)
   
   In the next major release _for this provider_ (which now that i think of it does not have to be in airflow 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720517813



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       hmm... not sure why you'd want to leave `get_uri` in place and add `_get_sqlalchemy_uri`... AFAIK, the only realon there is a `get_uri` method in hooks is for sqlalchemy.  so i'd just move your logic to get_uri...  but perhaps you had a reason i'm not seeing?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r722690470



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,82 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+Database Authentication
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://awsuser:password@redshift-cluster-1.123456789.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=True
+
+IAM Authentication using AWS Profile
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default

Review comment:
       Ok so dealing with airflow URIs is a bit tricky.
   
   And in this case I think there's a small problem.
   
   Look at the handling of the `iam` parameter:
   
   ```python
   >>> from airflow.models.connection import Connection
   >>> c = Connection(uri='redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default')
   >>> c.extra_dejson
   {'database': 'dev', 'iam': 'True', 'db_user': 'awsuser', 'cluster_identifier': 'redshift-cluster-1', 'profile': 'default'}
   ```
   
   And if redshift connector gets `iam='True'` instead of `iam=True` it won't work.
   
   While we _could_ implement logic to handle this in the hook, we don't need to because there's a way to produce the URI such that we avoid this issue.
   
   We have a `get_uri` method on `Connection` that produces the URI from a connection object.  And when doing the standard URI encoding will lose fidelity (e.g. bool converted to string on reparsing) then it will use the alternative representation of extra:
   ```python
   >>> c = Connection(conn_type='redshift', extra=json.dumps({"database":"dev", "iam":True,"db_user":"awsuser", "cluster_identifier":"redshift-cluster-1", "profile":"default"}))
   >>> c.get_uri()
   'redshift://?__extra__=%7B%22database%22%3A+%22dev%22%2C+%22iam%22%3A+true%2C+%22db_user%22%3A+%22awsuser%22%2C+%22cluster_identifier%22%3A+%22redshift-cluster-1%22%2C+%22profile%22%3A+%22default%22%7D'
   ```
   
   It's ugly but it works.
   
   So bottom line I think here it would be to just show examples of how to define the connections using `Connection` instead of URI format.  It's the easiest way to produce the correct URI.  And you could also show the generated URI and how you produced it as I've done above, or perhaps just point to the "generating an airflow URI" section in the corehowto / "managing connections" doc, where this is covered in detail.
   
   ---
   
   Side note I hope to implement support for json serialization broadly (i.e. as an alternative to airflow URI) along the lines done [here with SSM](https://github.com/apache/airflow/pull/18692), which would make this a little less painful.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716962976



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook Redshift hook
   
   Then there are two redshift hooks until 3.0
   
   After 3.0 the old one is renamed

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0, but in different modules
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       It's a good point re IAM auth.
   
   What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0, but in different modules
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftStatementHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')
+
+        return URL(drivername=conn_type, **conn_params).__str__()

Review comment:
       ```suggestion
           return str(URL(drivername=conn_type, **conn_params))
   ```
   
   I believe `str()` is preferred to calling `__str__()` directly

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class RedshiftOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,
+        autocommit: bool = True,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.redshift_conn_id = redshift_conn_id
+        self.sql = sql
+        self.autocommit = autocommit
+        self.parameters = parameters
+
+    def get_hook(self) -> RedshiftStatementHook:
+        """Create and return RedshiftStatementHook.
+        :return RedshiftStatementHook: A RedshiftStatementHook instance.
+        """
+        return RedshiftStatementHook(redshift_conn_id=self.redshift_conn_id)
+
+    def execute(self, context: dict) -> None:
+        """Execute a statement against Amazon Redshift"""
+        self.log.info(f"Executing statement: {self.sql}")
+        hook = self.get_hook()
+        hook.run(self.sql, autocommit=self.autocommit, parameters=self.parameters)

Review comment:
       please consider this an optional suggestion, but i noticed that a `handler` param was added to DBAPIHook.run
   
   you could add this param to this operator too, so it could be passed to hook.run

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)

Review comment:
       you could leave this out

##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,65 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement

Review comment:
       there's nothing in the example that is specific to IAM so perhaps best to remove that part of this comment.

##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       ```suggestion
           assert x == uri_shoudbe
   ```
   the convention with pytest is `assert actual == expected`

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member

Review comment:
       i think pylint is removed from airflow now...

##########
File path: tests/providers/amazon/aws/operators/test_redshift.py
##########
@@ -0,0 +1,57 @@
+#
+# 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
+from unittest import mock
+
+from airflow.models.dag import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils import timezone
+
+DEFAULT_DATE = timezone.datetime(2015, 1, 1)
+DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
+DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10]
+TEST_DAG_ID = 'unit_test_dag'
+
+
+class TestRedshiftOperator(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+        args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
+        dag = DAG(TEST_DAG_ID, default_args=args)
+        self.dag = dag
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator(self, mock_get_hook):
+        sql = """
+        CREATE TABLE IF NOT EXISTS test_airflow (
+            dummy VARCHAR(50)
+        );
+        """
+        operator = RedshiftOperator(task_id='redshift_operator', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator_test_multi(self, mock_get_hook):
+        sql = [
+            "CREATE TABLE IF NOT EXISTS test_airflow (dummy VARCHAR(50))",
+            "TRUNCATE TABLE test_airflow",
+            "INSERT INTO test_airflow VALUES ('X')",
+        ]
+        operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)

Review comment:
       ```suggestion
   import unittest
   from unittest import mock
   from unittest.mock import MagicMock
   
   from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
   
   
   class TestRedshiftOperator(unittest.TestCase):
       @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
       def test_redshift_operator(self, mock_get_hook):
           hook = MagicMock()
           mock_run = hook.run
           mock_get_hook.return_value = hook
           sql = MagicMock()
           operator = RedshiftOperator(task_id='test', sql=sql)
           operator.execute(None)
           mock_run.assert_called_once_with(
               sql,
               autocommit=True,
               parameters=None,
           )
   ```
   
   Ok so why have I suggested this rewriting...
   
   If you look at your test, the behavior of the operator is not verified.  `get_hook` will return a mock, and nothing that done to the mock will cause an error, and the calls are not verified.
   
   So for example, if I swap out the list of string like so it still passes:
   ```python
       @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
       def test_redshift_operator_test_multi(self, mock_get_hook):
           def fun():
               print('hello')
   
           sql = fun
           operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
           operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
   ```
   
   So the there is no difference between your test for str and the one for list[str]; and neither of them really verify the behavior of the operator.
   
   Further, you don't really need to verify that hook.run takes list of str or str -- that is behavior of DBAPIHook and presumably verified elsewhere.  We just care that hook.run is called with the values passed to the operator.
   
   So that's what I have tried to do here.
   
   And I called execute directly instead of run because it's simpler and means we can chop some of the boilerplate.
   
   You could improve upon my suggestion by parameterizing it such that it verifies that autocommit and parameters are forward to hook.run as appropriate

##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       ```suggestion
           assert x == uri_shoudbe
   ```
   the convention with pytest is `assert actual == expected`
   
   nit pick here, but would also be easier to read / more conventional if you called it  `expected` or `uri_expected` instead of `uri_shouldbe`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718771435



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       thanks to all for collaborating to find a good name for the hook and operator! :) 
   
   I've made the rename to `RedshiftSQLHook` and `RedshiftSQLOperator`, and placed `RedshiftSQLHook` in the same module as `RedshiftHook` as Kamil mentioned on slack in a5e8786




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720430065



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",

Review comment:
       as a fyi, this example was taken from the `PostgresOperator` example [here](https://github.com/apache/airflow/blob/ef037e702182e4370cb00c853c4fb0e246a0479c/airflow/providers/postgres/example_dags/example_postgres.py#L70) which does something very similar 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720470553



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       fixed in eac5cdacd75c0783f3418af24185f6e009a49c06, only schema is stated as required




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931503404


   @ashb I verified IAM works with postgres hook. and i verified that super is returned as string.
   
   But even so (and separate from the `geometry` data type, which seems like a valid point though I didn't verify), I think there's a case for adding a redshift hook using this connector.
   
   From a user experience perspective, the redshift connector appears to be easier to deal with.
   
   For example, to use IAM with postgreshook, unless your airflow user aws profile has access to generate cluster creds, you seem to need two separate airflow connections:
   
   ```python
   import os
   
   from airflow.models.connection import Connection
   from airflow.providers.postgres.hooks.postgres import PostgresHook
   
   os.environ['AIRFLOW_CONN_AWS_REDSHIFT_CLUSTER_CREDS_GEN'] = 'aws://aws_id:aws_key@?region=region-1'
   c = Connection(
       host='cluster-identifier.random-id.region-1.redshift.amazonaws.com',
       login='my_db_username',
       schema='my_database',
       extra=json.dumps({"iam": True, "redshift": True, "aws_conn_id": "aws_redshift_cluster_creds_gen"}),
   )
   hook = PostgresHook(connection=c)
   cnx = hook.get_conn()
   ```
   
   (commonly, your airflow user might not have that access, but you'd reference an aws airflow conn that does)
   
   And you'd need to use the cluster hostname `cluster-identifier.random-id.region-1.redshift.amazonaws.com` but with redshift connector the simple cluster identifier will do (which is just more convenient):
   
   ```python
   conn = redshift_connector.connect(
       iam=True,
       database='dev',
       db_user='awsuser',
       cluster_identifier='examplecluster',
       profile='default'
    )
   ```
   
   Beyond that, redshift does not behave 100% the same as postgres with regard to certain things.   For example external schemas don't show up in `information_schema.schemata`.  If you look in the [cursor code](https://github.com/aws/amazon-redshift-python-driver/blob/master/redshift_connector/cursor.py), where there are a lot of convenience methods for listing things like tables and primary keys, you can see 9 references to redshift-specific views (search for svv).
   
   These things built into tho redshift connector would seem to provide meaningful user value.  Though I'm unaware how how well tested this library is in the wild.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718837637



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User

Review comment:
       @mik-laj, in this hook, `login` and `schema` are [renamed](https://github.com/apache/airflow/pull/18447/files#diff-a6af6323bf6848d19c7d8066497ebebe20d221398642f5e1340b336c26eb2d87R167) to `User` and `Database` in the airflow UI.  Just wondering if this is something we encourage or not since I haven't seen this before.  To me, as someone who prefers using secrets backend to the airflow metastore, it would seem better to stick with `login` and `schema`, and use those uniformly, since those are the Connection attribute names, and those are what you'd have to use in secrets backend storage (e.g. if using a json backend) or when using Connection.get_uri to generate the airflow URI.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r719644330



##########
File path: docs/apache-airflow-providers-amazon/operators/redshift.rst
##########
@@ -0,0 +1,96 @@
+ .. 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.
+
+.. _howto/operator:RedshiftSqlOperator:
+
+RedshiftSqlOperator
+===================
+
+.. contents::
+  :depth: 1
+  :local:
+
+Overview
+--------
+
+Use the :class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift>` to execute
+statements against an Amazon Redshift cluster.
+
+:class:`RedshiftSqlOperator <airflow.providers.amazon.aws.operators.redshift.RedshiftSqlOperator>` works together with
+: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
+""""""""""""""
+
+In the following code we are creating a table called "fruit".
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_create_table]
+    :end-before: [END howto_operator_redshift_create_table]
+
+Insert data into a table
+""""""""""""""""""""""""
+
+In the following code we insert a few sample rows into the "fruit" table.
+
+.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_redshift.py
+    :language: python
+    :start-after: [START howto_operator_redshift_populate_table]
+    :end-before: [END howto_operator_redshift_populate_table]
+
+Fetching records from a table
+"""""""""""""""""""""""""""""
+
+Retrieving all records from the "fruit" table.

Review comment:
       thanks for the perspective. I agree this is confusing as written. I've reworked to use `create table my_table AS select *` as suggested. 1fd4ad631b24d6407c7d702176385b098d9584f9




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716962976



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook Redshift hook
   
   Then there are two redshift hooks until 3.0
   
   After 3.0 the old one is renamed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] JavierLopezT commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
JavierLopezT commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716684545



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       I agree with this. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720321218



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:

Review comment:
       There is an ongoing cleanup effort across all example DAGs to transition away from using `days_ago(n)` for `start_date` in deference for a static value since it's best practice.  We should have new DAGs contain a static `start_date` value; we've been using `datetime(2021, 1, 1)` but it doesn't matter what the value is really.  Sorry for not catching this earlier.

##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       If the connection requires `host`, `port`, `login` and `password`, it would be a good idea if there was logic in the `_get_conn_params()` method to verify and raise an exception if any are missing.

##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       Can the statement in `_get_conn_params()` for `user` assignment be changed to have the `conn_params` key named "username" instead of deleting the "user" and adding the "username" key here?
   
   From this:
   ```python
   if conn.login:
       conn_params['user'] = conn.login
   ```
   to:
   ```python
   if conn.login:
       conn_params['username'] = conn.login
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +106,47 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        self.db_hook = RedshiftSQLHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):

Review comment:
       If connection-parameter verification logic is added to the `RedshiftSQLHook._get_conn_params()` method, adding a  test validating the logic should be included here as well.

##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       Is `schema` (aka `database` in this connection's case) required as well?

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       Knowing that it's been the pattern in operators for a long time but it seems cleaner to not hardcode the default value for `redshift_conn_id` but use `RedshiftSQLHook.defaul_conn_name`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720411143



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",
+        params={'color': 'Red'},

Review comment:
       @ash a question on this.
   
   Are you saying that `params` is not meant for templating?  Or are you saying that in sql, we should prefer `parameters` _when possible_?
   
   E.g. I have used `params` e.g. like this:
   
   ```sql
   UNLOAD ('
   SELECT *
   FROM my_table a
   ')
   TO 's3://{{ params.bucket }}/{{ params.prefix }}/'
   ```
   
   You might not be able to use parameters for this kind of thing.  I had one other case where it was a massive union and I pulled the logic into a sql template generation script and it was certainly not possible in that case.
   
   I reckon what you are saying is that a better practice to use `parameters` whenever it's supported, and in this case it's supported so that's what the example should show.  And in other cases no issue with params in templates.   
   
   Side note, it's weird, i looked at BaseOperator and the docstring does not document `params` 🤔
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717847411



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,65 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement

Review comment:
       addressed in 7205a8cd7d0d05ed70dc7e5f87bf70d0227f0184




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714524245



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""

Review comment:
       It’s quite weird this function is called exactly twice, once discarding the first element in the tuple, the other discarding the second. This seems to indicate the function should be separated into two.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718025826



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   then in the next major release _for this provider_ (which now that i think of it does not have to be in 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720475825



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default

Review comment:
       fixed this in 1b22d59c1d3f25370b81c5f414a6e129a320e0ab. `get_uri()` is no longer overridden.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720142988



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",

Review comment:
       Doesn't this run the risk of an SQL injection attack?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720144828



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default

Review comment:
       Why is this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268559



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,66 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+Login
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector. The following parameter is required:
+
+    * ``database``: Amazon Redshift database name.

Review comment:
       addressed in 77deb1d




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720425991



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       > i think they are all optional depending on the circumstance. 
   this is correct, thank you both for identifying this mistake

##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       > i think they are all optional depending on the circumstance. 
   
   this is correct, thank you both for identifying this mistake




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718045977



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       > When using acronyms in CapWords, capitalize all the letters of the acronym. Thus HTTPServerError is better than HttpServerError.
   
   [PEP-8](https://www.python.org/dev/peps/pep-0008/)
   
   We should use `RedshiftSQLHook` instead of `RedshiftSqlHook`

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       > In the next major release for this provider (which now that i think of it does not have to be in airflow 3.0, since providers have separate release schedule), we remove RedshiftHook and we are left with RedshiftSqlHook and RedshiftClusterHook
   
   It is not required that we delete deprecated classes in the next major release if maintenance is not burdensome. See discussion: https://apache-airflow.slack.com/archives/CCPRP7943/p1631032457154300




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268661



##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,74 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(
+            login='login', password='password', host='host', port=5439, extra=json.dumps({"database": "dev"})
+        )

Review comment:
       addressed in 0aa5313




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r722690470



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,82 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+Database Authentication
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://awsuser:password@redshift-cluster-1.123456789.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=True
+
+IAM Authentication using AWS Profile
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default

Review comment:
       Ok so dealing with airflow URIs is a bit tricky.
   
   And in this case I think there's a small problem.
   
   Look at the handling of the `iam` parameter:
   
   ```python
   >>> from airflow.models.connection import Connection
   >>> c = Connection(uri='redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default')
   >>> c.extra_dejson
   {'database': 'dev', 'iam': 'True', 'db_user': 'awsuser', 'cluster_identifier': 'redshift-cluster-1', 'profile': 'default'}
   ```
   
   And if redshift connector gets `iam='True'` instead of `iam=True` it won't work.
   
   While we _could_ implement logic to handle this in the hook, we don't need to because there's a way to produce the URI such that we avoid this issue.
   
   We have a `get_uri` method on `Connection` that produces the URI from a connection object.  And when doing the standard URI encoding will lose fidelity (e.g. bool converted to string on reparsing) then it will use the alternative representation of extra:
   ```python
   >>> c = Connection(conn_type='redshift', extra=json.dumps({"database":"dev", "iam":True,"db_user":"awsuser", "cluster_identifier":"redshift-cluster-1", "profile":"default"}))
   >>> c.get_uri()
   'redshift://?__extra__=%7B%22database%22%3A+%22dev%22%2C+%22iam%22%3A+true%2C+%22db_user%22%3A+%22awsuser%22%2C+%22cluster_identifier%22%3A+%22redshift-cluster-1%22%2C+%22profile%22%3A+%22default%22%7D'
   ```
   
   It's ugly but it works.
   
   So bottom line I think here it would be to just show examples of how to define the connections using `Connection` instead of URI format.  It's the easiest way to produce the correct URI.  And you could also show the generated URI and how you produced it as I've done above, or perhaps just point to the "generating an airflow URI" section in the corehowto / "managing connections" doc, where this is covered in detail.
   
   ---
   
   Side note I hope to implement support for json serialization broadly (i.e. as an alternative to airflow URI) along the lines done [here with SSM](https://github.com/apache/airflow/pull/18692), which will make this a little less painful.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r724408055



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       This isn't addressed

##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       Same as https://github.com/apache/airflow/pull/18692#discussion_r721083266
   
   >The current try-except approach is intentional to keep the provider compatible with older Airflow versions. It shouldn't be changed unless you're going to bump the minimum Airflow requirement on this provider.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r722654639



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)

Review comment:
       ```suggestion
           or a list of str (sql statements)
   ```

##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,82 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+Database Authentication
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://awsuser:password@redshift-cluster-1.123456789.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=True
+
+IAM Authentication using AWS Profile
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default

Review comment:
       Ok so dealing with airflow URIs is a bit tricky.
   
   And in this case I think there's a small problem.
   
   Look at the handling of the `iam` parameter:
   
   ```python
   >>> from airflow.models.connection import Connection
   >>> c = Connection(uri='redshift://:@:/?database=dev&iam=True&db_user=awsuser&cluster_identifier=redshift-cluster-1&profile=default')
   >>> c.extra_dejson
   {'database': 'dev', 'iam': 'True', 'db_user': 'awsuser', 'cluster_identifier': 'redshift-cluster-1', 'profile': 'default'}
   ```
   
   And if redshift connector gets `iam='True'` instead of `iam=True` it won't work.
   
   While we _could_ implement logic to handle this in the hook, we don't need to because there's a way to produce the URI such that we avoid this issue.
   
   We have a `get_uri` method on `Connection` that produces the URI from a connection object.  And when doing the standard URI encoding will lose fidelity (e.g. bool converted to string on reparsing) then it will use the alternative representation of extra:
   ```python
   c = Connection(conn_type='redshift', extra=json.dumps({"database":"dev", "iam":True,"db_user":"awsuser", "cluster_identifier":"redshift-cluster-1", "profile":"default"}))
   c.get_uri()
   'redshift://?__extra__=%7B%22database%22%3A+%22dev%22%2C+%22iam%22%3A+true%2C+%22db_user%22%3A+%22awsuser%22%2C+%22cluster_identifier%22%3A+%22redshift-cluster-1%22%2C+%22profile%22%3A+%22default%22%7D'
   ```
   
   It's ugly but it works.
   
   So bottom line I think here it would be to just show examples of how to define the connections using `Connection` instead of URI format.  It's the easiest way to produce the correct URI.  And you could also show the generated URI and how you produced it, or just point to the "generating an airflow URI" section in the corehowto / "managing connections" doc, where this is covered in detail.
   
   ---
   
   Side note I hope to implement support for json serialization broadly (i.e. as an alternative to airflow URI) along the lines done [here with SSM](https://github.com/apache/airflow/pull/18692), which will make this a little less painful.
   
   

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,

Review comment:
       i don't think this type annotation is quite correct.  it might be `Optional[Union[Dict, Iterable]]`
   
   the point is that it could be a tuple or list, or a dict, as shown in the readme examples of the library.

##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +107,67 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(
+            conn_type='redshift', login='login', password='password', host='host', port=5439, schema="dev"
+        )
+
+        self.db_hook = RedshiftSQLHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        expected = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert x == expected
+
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn(self, mock_connect):
+        self.db_hook.get_conn()
+        mock_connect.assert_called_once_with(
+            user='login', password='password', host='host', port=5439, database='dev'
+        )
+
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn_extra(self, mock_connect):
+        self.connection.extra = json.dumps(
+            {
+                "iam": True,
+                "cluster_identifier": "my-test-cluster",
+                "profile": "default",
+            }
+        )
+        self.db_hook.get_conn()
+        mock_connect.assert_called_once_with(
+            user='login',
+            password='password',
+            host='host',
+            port=5439,
+            cluster_identifier="my-test-cluster",
+            profile="default",
+            database='dev',
+            iam=True,
+        )
+
+    @parameterized.expand(
+        [
+            ({}, {}, {}),
+            ({"login": "test"}, {}, {"user": "test"}),
+            ({}, {"user": "test"}, {"user": "test"}),
+            ({"login": "original"}, {"user": "overridden"}, {"user": "overridden"}),
+            ({"login": "test1"}, {"password": "test2"}, {"user": "test1", "password": "test2"}),
+        ],
+    )
+    @mock.patch('airflow.providers.amazon.aws.hooks.redshift.redshift_connector.connect')
+    def test_get_conn_overrides_correctly(self, test_args, test_kwargs, expected_call_args, mock_connect):

Review comment:
       ```suggestion
       def test_get_conn_overrides_correctly(self, conn_params, conn_extra, expected_call_args, mock_connect):
   ```
   hard to name this perfectly but test_args and test_kwargs... they're all "keywords args" but mainly the diff here is `extra` vs other conn params.  And this just seems a tad more clear.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717846629



##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       addressed in 01091dcace8f3c6efd57a2f7d144f7a9336ecbcd




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714523094



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params, conn_kwargs = self._get_conn_params()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        self.log.info(conn_kwargs)

Review comment:
       And if you _do_ want a log, this should probably be debug-level and have a more identifiable format, e.g.
   
   ```python
   self.log.debug("Redshift connection arguments: %s", conn_kwargs)
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720150211



##########
File path: airflow/providers/amazon/provider.yaml
##########
@@ -441,6 +447,8 @@ connection-types:
     connection-type: aws
   - hook-class-name: airflow.providers.amazon.aws.hooks.emr.EmrHook
     connection-type: emr
+  - hook-class-name: airflow.providers.amazon.aws.hooks.redshift.RedshiftSQLHook
+    connection-type: redshift+redshift_connector

Review comment:
       Is there a reason to not just have this as `redshift`? -- for example `AIRFLOW_CONN_MY_REDSHIFT=redshift://cluster/db` ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717847146



##########
File path: tests/providers/amazon/aws/operators/test_redshift.py
##########
@@ -0,0 +1,57 @@
+#
+# 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
+from unittest import mock
+
+from airflow.models.dag import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils import timezone
+
+DEFAULT_DATE = timezone.datetime(2015, 1, 1)
+DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
+DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10]
+TEST_DAG_ID = 'unit_test_dag'
+
+
+class TestRedshiftOperator(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+        args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
+        dag = DAG(TEST_DAG_ID, default_args=args)
+        self.dag = dag
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator(self, mock_get_hook):
+        sql = """
+        CREATE TABLE IF NOT EXISTS test_airflow (
+            dummy VARCHAR(50)
+        );
+        """
+        operator = RedshiftOperator(task_id='redshift_operator', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator_test_multi(self, mock_get_hook):
+        sql = [
+            "CREATE TABLE IF NOT EXISTS test_airflow (dummy VARCHAR(50))",
+            "TRUNCATE TABLE test_airflow",
+            "INSERT INTO test_airflow VALUES ('X')",
+        ]
+        operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)

Review comment:
       thank you for providing the re-write and very thorough explanation. i've addressed this in 7205a8cd7d0d05ed70dc7e5f87bf70d0227f0184, and added some parameterization for autocommit and parameters




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715267824



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,70 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class RedshiftOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,
+        autocommit: bool = True,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.redshift_conn_id = redshift_conn_id
+        self.sql = sql
+        self.redshift_conn_id = redshift_conn_id

Review comment:
       addressed in 1004e31




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717992712



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       How about something like `RedshiftSqlHook` instead of having two hooks named `RedshiftHook`? Otherwise I feel it may be confusing to users and seems relevant to the features being built with this PR as well as keeping to naming conventions. Although, this suggestion would may warrant renaming the new operator to match.  
   
   If `RedshiftSqlHook` is agreeable then maybe we don't need to rename it later either.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720411143



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",
+        params={'color': 'Red'},

Review comment:
       @ash a question on this.
   
   Are you saying that `params` is not meant for templating?  Or are you saying that in sql, we should prefer `parameters` _when possible_?
   
   E.g. I have used `params` e.g. like this:
   
   ```sql
   UNLOAD ('
   SELECT *
   FROM my_table a
   ')
   TO 's3://{{ params.bucket }}/{{ params.prefix }}/'
   ```
   
   I am pretty sure you can't use parameters for this kind of thing.  I had one other case where it was a massive union and I pulled the logic into a sql template generation script.
   
   I reckon what you are saying is that a better practice to use `parameters` whenever it's supported, and in this case it's supported so that's what the example should show.  And in other cases no issue with params in templates.   
   
   Side note, it's weird, i looked at BaseOperator and the docstring does not document `params` 🤔
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715617857



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector. The following parameter is required:

Review comment:
       This statement can be removed now - `The following parameter is required:`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720469474



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift+redshift_connector://awsuser:password@redshift-cluster-1.testing.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=False

Review comment:
       fixed in 1b22d59. host has been set to something more reasonable and SSL has been enabled.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716876359



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       Love the idea. Personally I'd vote for renaming the existing hook to `RedshiftClusterHook`.
   
   Getting to a point where users don't need to cross-pollinate their environment with the Postgres provider just to simply execute SQL in Redshift would be very beneficial. I agree that transition will take some time but it most certainly will be worth the effort.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268441



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,70 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_s3_to_redshift_create_table]
+    setup__task_create_table = RedshiftOperator(
+        redshift_conn_id='redshift_default',

Review comment:
       addressed in a88c781




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714833401



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,70 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_s3_to_redshift_create_table]
+    setup__task_create_table = RedshiftOperator(
+        redshift_conn_id='redshift_default',

Review comment:
       You don't necessarily need to pass in the `redshift_conn_id` parameter to each `RedshiftOperator` task since the default value is being used here. 
   
   Or, if you would like to specify a non-default value for `redshift_conn_id`, you could add it into the DAG `default_args` since this arg is passed to every operator in the DAG.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268347



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params, conn_kwargs = self._get_conn_params()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        self.log.info(conn_kwargs)

Review comment:
       addressed in 42d38c4




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715725337



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector. The following parameter is required:

Review comment:
       addressed in c4b0cd




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-933830460


   reverted 76a6e93 in c743bfc due to it the import approach causing an issue in CI workflow "Install and test provider packages and airflow on Airflow 2.1 files" [link to failing CI run](https://github.com/apache/airflow/pull/18447/checks?check_run_id=3795213662)
   
   ```bash
   Import all Airflow classes
     Searching for providers packages in:
     /usr/local/lib/python3.6/site-packages/airflow/providers
     
     Walking all packages in 
     ['/usr/local/lib/python3.6/site-packages/airflow/providers'] with prefix 
     airflow.providers.
   ...
    ERROR: There were some import errors
     
     ----------------------------------------
     Traceback (most recent call last):
       File "/opt/airflow/dev/import_all_classes.py", line 82, in import_all_classes
         _module = importlib.import_module(modinfo.name)
       File "/usr/local/lib/python3.6/importlib/__init__.py", line 126, in 
     import_module
         return _bootstrap._gcd_import(name, package, level)
       File "<frozen importlib._bootstrap>", line 994, in _gcd_import
       File "<frozen importlib._bootstrap>", line 971, in _find_and_load
       File "<frozen importlib._bootstrap>", line 955, in _find_and_load_unlocked
       File "<frozen importlib._bootstrap>", line 665, in _load_unlocked
       File "<frozen importlib._bootstrap_external>", line 678, in exec_module
       File "<frozen importlib._bootstrap>", line 219, in _call_with_frames_removed
       File "/usr/local/lib/python3.6/site-packages/airflow/providers/amazon/aws/exam
     ple_dags/example_redshift.py", line 26, in <module>
         from airflow.providers.amazon.aws.operators.redshift import 
     RedshiftSQLOperator
       File "/usr/local/lib/python3.6/site-packages/airflow/providers/amazon/aws/oper
     ators/redshift.py", line 21, in <module>
         from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
       File "/usr/local/lib/python3.6/site-packages/airflow/providers/amazon/aws/hook
     s/redshift.py", line 27, in <module>
         from airflow.compat.functools import cached_property
     ModuleNotFoundError: No module named 'airflow.compat'
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-925363487


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714445042



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params, conn_kwargs = self._get_conn_params()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        self.log.info(conn_kwargs)

Review comment:
       Is this log necessary? It looks like it might log the connection password from `conn_params`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-935300096


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720415219



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection

Review comment:
       yeah i think this may just be the wrong language:
   
   > This connection must be initialized with the host, port, login, password
   
   i think they are all optional depending on the circumstance.  e.g. if you pass `cluster_identifier` as a kwarg (through extra) i don't think you would need host.  and if you are using iam you would not need password. and you don't need port, this i've verified.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720586677



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)

Review comment:
       good catch, fixed in f57435b3b6a1e02907a232de28a6c62e7955b41e




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720413383



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       @ashb you have a take on this?  if you DRY it, then i think you won't see what the default is in IDE




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r724464795



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""
 
-from typing import List, Optional
+from typing import Dict, List, Optional, Union
 
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property

Review comment:
       Same as https://github.com/apache/airflow/pull/18692#discussion_r721083266
   
   >The current try-except approach is intentional to keep the provider compatible with older Airflow versions. It shouldn't be changed unless you're going to bump the minimum Airflow requirement on this provider.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717126873



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(

Review comment:
       cached property `conn` added, and `_get_conn_kwargs` removed in a1de44e

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type

Review comment:
       done in d90b760

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,

Review comment:
       addressed this in 6b3976b. the default port number of 5439 has been remove and I opted to selectively add kwargs as mentioned.

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type
+
+        return URL(
+            drivername=conn_type,
+            username=conn_params['user'],
+            password=conn_params['password'],
+            host=conn_params['host'],
+            port=conn_params['port'],
+            database=conn_params['database'],
+        ).__str__()
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        conn_kwargs = self._get_conn_kwargs()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params = self._get_conn_params()
+        conn_kwargs = self._get_conn_kwargs()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        conn: RedshiftConnection = redshift_connector.connect(**conn_kwargs)
+
+        return conn
+
+    def run(
+        self,
+        sql: Union[str, list],
+        autocommit: bool = False,
+        parameters: Optional[dict] = None,
+        handler: Optional[Callable] = None,
+    ):
+        """
+        Runs a command or a list of commands. Pass a list of sql
+        statements to the sql parameter to get them to execute
+        sequentially
+
+        :param sql: the sql statement to be executed (str) or a list of
+            sql statements to execute
+        :type sql: str or list
+        :param autocommit: What to set the connection's autocommit setting to
+            before executing the query.
+        :type autocommit: bool
+        :param parameters: The parameters to render the SQL query with.
+        :type parameters: dict or iterable
+        :param handler: The result handler which is called with the result of each statement.
+        :type handler: callable
+        :return: query results if handler was provided.
+        """
+        return super().run(sql, autocommit=False, parameters=parameters, handler=handler)

Review comment:
       removed the method override. The hard coded ` autocommit=False`was left over from testing, so not needed. 1efe5a9




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717131359



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftStatementHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')
+
+        return URL(drivername=conn_type, **conn_params).__str__()

Review comment:
       ```suggestion
           return str(URL(drivername=conn_type, **conn_params))
   ```
   
   I believe `str()` is preferred to calling `__str__()` directly

##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class RedshiftOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,
+        autocommit: bool = True,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.redshift_conn_id = redshift_conn_id
+        self.sql = sql
+        self.autocommit = autocommit
+        self.parameters = parameters
+
+    def get_hook(self) -> RedshiftStatementHook:
+        """Create and return RedshiftStatementHook.
+        :return RedshiftStatementHook: A RedshiftStatementHook instance.
+        """
+        return RedshiftStatementHook(redshift_conn_id=self.redshift_conn_id)
+
+    def execute(self, context: dict) -> None:
+        """Execute a statement against Amazon Redshift"""
+        self.log.info(f"Executing statement: {self.sql}")
+        hook = self.get_hook()
+        hook.run(self.sql, autocommit=self.autocommit, parameters=self.parameters)

Review comment:
       please consider this an optional suggestion, but i noticed that a `handler` param was added to DBAPIHook.run
   
   you could add this param to this operator too, so it could be passed to hook.run

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)

Review comment:
       you could leave this out

##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,65 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement

Review comment:
       there's nothing in the example that is specific to IAM so perhaps best to remove that part of this comment.

##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       ```suggestion
           assert x == uri_shoudbe
   ```
   the convention with pytest is `assert actual == expected`

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member

Review comment:
       i think pylint is removed from airflow now...

##########
File path: tests/providers/amazon/aws/operators/test_redshift.py
##########
@@ -0,0 +1,57 @@
+#
+# 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
+from unittest import mock
+
+from airflow.models.dag import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils import timezone
+
+DEFAULT_DATE = timezone.datetime(2015, 1, 1)
+DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
+DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10]
+TEST_DAG_ID = 'unit_test_dag'
+
+
+class TestRedshiftOperator(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+        args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
+        dag = DAG(TEST_DAG_ID, default_args=args)
+        self.dag = dag
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator(self, mock_get_hook):
+        sql = """
+        CREATE TABLE IF NOT EXISTS test_airflow (
+            dummy VARCHAR(50)
+        );
+        """
+        operator = RedshiftOperator(task_id='redshift_operator', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator_test_multi(self, mock_get_hook):
+        sql = [
+            "CREATE TABLE IF NOT EXISTS test_airflow (dummy VARCHAR(50))",
+            "TRUNCATE TABLE test_airflow",
+            "INSERT INTO test_airflow VALUES ('X')",
+        ]
+        operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)

Review comment:
       ```suggestion
   import unittest
   from unittest import mock
   from unittest.mock import MagicMock
   
   from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
   
   
   class TestRedshiftOperator(unittest.TestCase):
       @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
       def test_redshift_operator(self, mock_get_hook):
           hook = MagicMock()
           mock_run = hook.run
           mock_get_hook.return_value = hook
           sql = MagicMock()
           operator = RedshiftOperator(task_id='test', sql=sql)
           operator.execute(None)
           mock_run.assert_called_once_with(
               sql,
               autocommit=True,
               parameters=None,
           )
   ```
   
   Ok so why have I suggested this rewriting...
   
   If you look at your test, the behavior of the operator is not verified.  `get_hook` will return a mock, and nothing that done to the mock will cause an error, and the calls are not verified.
   
   So for example, if I swap out the list of string like so it still passes:
   ```python
       @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
       def test_redshift_operator_test_multi(self, mock_get_hook):
           def fun():
               print('hello')
   
           sql = fun
           operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
           operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
   ```
   
   So the there is no difference between your test for str and the one for list[str]; and neither of them really verify the behavior of the operator.
   
   Further, you don't really need to verify that hook.run takes list of str or str -- that is behavior of DBAPIHook and presumably verified elsewhere.  We just care that hook.run is called with the values passed to the operator.
   
   So that's what I have tried to do here.
   
   And I called execute directly instead of run because it's simpler and means we can chop some of the boilerplate.
   
   You could improve upon my suggestion by parameterizing it such that it verifies that autocommit and parameters are forward to hook.run as appropriate




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716962976



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720554894



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       reverted this change in 005531b, I agree this will be more transparent. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720561947



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)

Review comment:
       default is True below




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718004578



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       I think RedshiftSqlHook is an acceptable name, though I reached out on slack to see if anyone has some advice for how we could end up with `RedshiftHook` as the sql hook (which i think would be preferable if it can be done)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718013593



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       thank you for reaching out on slack, @dstandish. I think having two RedshiftHooks until 3.0, as you wrote on slack, would be a good path to getting this merged with consistent naming with the other database hooks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720468573



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""

Review comment:
       fixed in ea5b4c8dbe93388639ac9c0d2f02ac4c4a9f7bfd




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717992712



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       How about something like `RedshiftSqlHook` instead of having two hooks named `RedshiftHook`? Otherwise I feel it may be confusing to users and seems relevant to the features being built with this PR as well as keeping to naming conventions. Although, this suggestion would may warrant renaming the new operator to match.  
   
   If `RedshiftSqlHook` is agreeable then maybe we don't need to rename it later either.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717216128



##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       ```suggestion
           assert x == uri_shoudbe
   ```
   the convention with pytest is `assert actual == expected`
   
   nit pick here, but would also be easier to read / more conventional if you called it  `expected` or `uri_expected` instead of `uri_shouldbe`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716962976



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       It's a good point re IAM auth.
   
   What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0, but in different modules
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716962976



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       What if we just call the module redshift_sql but the hook RedshiftHook
   
   Then there are two redshift hooks until 3.0, but in different modules
   
   After 3.0 the old one is renamed
   
   And I guess we rename the new module at 3.0 too?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720469718



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:

Review comment:
       thanks for the heads up, fixed in 475dd8f2e1e7dc9b2ce0dfd7fcc28a679ae4cee9




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720142988



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",

Review comment:
       Doesn't this run the risk of an SQL injection attack?
   
   (Not likely in this specific case, but generally when I see using formatting in SQL strings I get _very_ nervous)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720150668



##########
File path: docs/apache-airflow-providers-amazon/connections/redshift.rst
##########
@@ -0,0 +1,68 @@
+.. 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.
+
+.. _howto/connection:redshift:
+
+Amazon Redshift Connection
+==========================
+
+The Redshift connection type enables integrations with Redshift.
+
+Authenticating to Amazon Redshift
+---------------------------------
+
+Authentication may be performed using any of the authentication methods supported by `redshift_connector <https://github.com/aws/amazon-redshift-python-driver>`_ such as via direct credentials, IAM authentication, or using an Identity Provider (IdP) plugin.
+
+Default Connection IDs
+-----------------------
+
+The default connection ID is ``redshift_default``.
+
+Configuring the Connection
+--------------------------
+
+
+User
+  Specify the username to use for authentication with Amazon Redshift.
+
+Password
+  Specify the password to use for authentication with Amazon Redshift.
+
+Host
+  Specify the Amazon Redshift hostname.
+
+Database
+  Specify the Amazon Redshift database name.
+
+Extra
+    Specify the extra parameters (as json dictionary) that can be used in
+    Amazon Redshift connection. For a complete list of supported parameters
+    please see the `documentation <https://github.com/aws/amazon-redshift-python-driver#connection-parameters>`_
+    for redshift_connector.
+
+
+When specifying the connection in environment variable you should specify
+it using URI syntax.
+
+Note that all components of the URI should be URL-encoded.
+
+Examples
+--------
+
+.. code-block:: bash
+
+  AIRFLOW_CONN_REDSHIFT_DEFAULT=redshift+redshift_connector://awsuser:password@redshift-cluster-1.testing.us-west-1.redshift.amazonaws.com:5439/?database=dev&ssl=False

Review comment:
       What is `testing` in this example? And we shouldn't encourage peole to turn SSL off.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720143599



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -15,10 +15,19 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-"""Interact with AWS Redshift, using the boto3 library."""
+"""Interact with AWS Redshift, using the boto3 or redshift_connector library."""

Review comment:
       ```suggestion
   """Interact with AWS Redshift clusters"""
   ```
   
   (It's not really that relevant to the users what libraries we use.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] josh-fell commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720333793



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       Knowing that it's been the pattern in operators for a long time but it seems cleaner to not hardcode the default value for `redshift_conn_id` but use `RedshiftSQLHook.default_conn_name`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white edited a comment on pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white edited a comment on pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#issuecomment-931497586


   Some info on authenticating via identity provider using redshift_connector [1](https://aws.amazon.com/blogs/big-data/federated-api-access-to-amazon-redshift-using-an-amazon-redshift-connector-for-python/) [2](https://github.com/aws/amazon-redshift-python-driver/blob/master/tutorials/001%20-%20Connecting%20to%20Amazon%20Redshift.ipynb)
   
   The datatypes redshift_connector supports [3](https://github.com/aws/amazon-redshift-python-driver#supported-datatypes)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720720951



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +133,91 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the schema. Additional connection
+    options can be passed to extra as a JSON string.

Review comment:
       I think it would be best to point to the connection howto at `_howto/connection:redshift` here and don't address connection details.
   
   One thing that's confusing about the wording here is that `schema` is ambiguous because you mean both `Connection` schema and redshift _database_ (but _not_ redshift schema).  And while you could update the wording here to specify, I think better is to just make sure that usage is clear in the howto and point them there -- no need to duplicate.
   
   In the howto, I would add a simple example of connection instantiation in the most common one or two scenarios (e.g. db auth and iam).  Something that wasn't obvious to me is that when trying IAM is you actually have to use `db_user` and not `user`.  An example for this would help.
   
   Additionally, instead of forcing user to use conn attrs (in this case conn.schema) I would recommend allowing the user to connection params in either extra or conn attributes.
   
   So in this case where you have `conn.schema` required, better would be to allow `extra_dejson['database'] or conn.schema`.  And while you could add parameter validation logic for that, I think it's better to just defer to the library on that, especially since all but one are only _conditionally_ required.
   
   With this library, having ambiguity both with database vs schema and more importantly with user vs db_user vs login, I think it may be simplest for users to just dump everything in extra and therefore we should make sure to support that.
   
   So what does this boil down to in terms of changes...
   
   I think it's best to simply remove the `Connection.schema` requirement (and let the library complain when it doesn't have the right kwargs).  Then if conn.schema is not provided, but `database` is provided in extra, it will still work (and I think your code already supports this, once you remove the parameter validation).  And same with `conn.login` and `user` -- let extra override conn attr.  
   
   And could you please add tests that verify the connection parsing behavior -- i.e. what happens when passed in conn attr and conn extra, or when not in attr but yes in extra, and when passed in both.  
   
   Thanks for bearing with the iteration on this one... getting to know some more of the details as we go, and we're definitely getting closer.
   
   ---
   
   Separately.... I just want to mention that it might be helpful if, in redshift_connector, you raise a more helpful error message when the parameter combination is bad in a couple cases.  In particular when i tried iam with `user` (instead of `db_user` i got this:
   ```
       cache_key: str = IamHelper.get_credentials_cache_key(info)
     File "...python3.8/site-packages/redshift_connector/iam_helper.py", line 330, in get_credentials_cache_key
       return ";".join(
   TypeError: sequence item 0: expected str instance, NoneType found
   ```
   I think the real problem is that I need to use `db_user` instead of `user`. 
   
   Similarly when I omitted `database` I got this error:
   ```
   File "...python3.8/site-packages/redshift_connector/core.py", line 663, in __init__
       raise self.error
   redshift_connector.error.ProgrammingError: {'S': 'FATAL', 'C': '3D000', 'M': 'database "IAM:my_admin" does not exist', 'F': '/home/ec2-user/padb/src/pg/src/backend/utils/init/postinit.c', 'L': '517', 'R': 'LcInitPostgres'}
   ```
   And the problem was that I did not supply `database`.   Raising a more helpful error message in this case would make it more friendly for us to defer parameter validation to the library :) 
   
   Thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r719643641



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,

Review comment:
       addressed in 5850f343ae2046ce1cb734eb08ece74aa3d53b62




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720514719



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       > I don't have much opinion, bit I don't really see what adding the default as a class property gives us.
   
   The class property thing is something defined in DbApiHook, just a way of abstracting common hook init boilerplate, but I'd agree it's probably not that helpful.
   
   Anyway, just cus the hook default conn id is `redshift_default`, there's no requirement that the operator has to use the same default.  So I think it's reasonable to duplicate the string default `redshift_conn_id='redshift_default'`.  And I think as a user it is more transparent and therefore easier to use.  But I'll leave it up to you @Brooke-white since you already made the change whether you want to leave it or put the literal back.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720149666



##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,67 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftSQLOperator` to authenticate with Amazon Redshift
+then execute a simple select statement
+"""
+# [START redshift_operator_howto_guide]
+from airflow import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftSQLOperator
+from airflow.utils.dates import days_ago
+
+with DAG(dag_id="example_redshift", start_date=days_ago(1), schedule_interval=None, tags=['example']) as dag:
+    # [START howto_operator_redshift_create_table]
+    setup__task_create_table = RedshiftSQLOperator(
+        task_id='setup__create_table',
+        sql="""
+            CREATE TABLE IF NOT EXISTS fruit (
+            fruit_id INTEGER,
+            name VARCHAR NOT NULL,
+            color VARCHAR NOT NULL
+            );
+        """,
+    )
+    # [END howto_operator_redshift_create_table]
+    # [START howto_operator_redshift_populate_table]
+    task_insert_data = RedshiftSQLOperator(
+        task_id='task_insert_data',
+        sql=[
+            "INSERT INTO fruit VALUES ( 1, 'Banana', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 2, 'Apple', 'Red');",
+            "INSERT INTO fruit VALUES ( 3, 'Lemon', 'Yellow');",
+            "INSERT INTO fruit VALUES ( 4, 'Grape', 'Purple');",
+            "INSERT INTO fruit VALUES ( 5, 'Pear', 'Green');",
+            "INSERT INTO fruit VALUES ( 6, 'Strawberry', 'Red');",
+        ],
+    )
+    # [END howto_operator_redshift_populate_table]
+    # [START howto_operator_redshift_get_all_rows]
+    task_get_all_table_data = RedshiftSQLOperator(
+        task_id='task_get_all_table_data', sql="CREATE TABLE more_fruit AS SELECT * FROM fruit;"
+    )
+    # [END howto_operator_redshift_get_all_rows]
+    # [START howto_operator_redshift_get_with_filter]
+    task_get_with_filter = RedshiftSQLOperator(
+        task_id='task_get_with_filter',
+        sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = '{{ params.color }}';",
+        params={'color': 'Red'},

Review comment:
       Oh that only works if the `paramstyle` is set on the connection. The default is:
   
   ```suggestion
           sql="CREATE TABLE filtered_fruit AS SELECT * FROM fruit WHERE color = %s;",
           parameterss=['Red'],
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720720951



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +133,91 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the schema. Additional connection
+    options can be passed to extra as a JSON string.

Review comment:
       I think it would be best to point to the connection howto at `_howto/connection:redshift` here and don't address connection details.
   
   One thing that's confusing about the wording here is that `schema` is ambiguous because you mean both `Connection` schema and redshift _database_ (but _not_ redshift schema).  And while you could update the wording here to specify, I think better is to just make sure that usage is clear in the howto and point them there -- no need to duplicate.
   
   In the howto, I would add a simple example of connection instantiation in the most common one or two scenarios (e.g. db auth and iam).  Something that wasn't obvious to me is that when trying IAM is you actually have to use `db_user` and not `user`.  An example for this would help.
   
   Additionally, instead of forcing user to use conn attrs (in this case conn.schema) I would recommend allowing the user to connection params in either extra or conn attributes.
   
   So in this case where you have `conn.schema` required, better would be to allow `extra_dejson['database'] or conn.schema`.  And while you could add parameter validation logic for that, I think it's better to just defer to the library on that, especially since all but one are only _conditionally_ required.
   
   With this library, having ambiguity both with database vs schema and more importantly with user vs db_user vs login, I think it may be simplest for users to just dump everything in extra and therefore we should make sure to support that.
   
   So what does this boil down to in terms of changes...
   
   I think it's best to simply remove the `Connection.schema` requirement (and let the library complain when it doesn't have the right kwargs).  Then if conn.schema is not provided, but `database` is provided in extra, it will still work (and I think your code already supports this, once you remove the parameter validation).  And same with `conn.login` and `user` -- let extra override conn attr.  
   
   The main work would be to please add tests that verify the connection parsing behavior -- i.e. what happens when passed in conn attr and conn extra, or when not in attr but yes in extra, and when passed in both.  
   
   Thanks for bearing with the iteration on this one... getting to know some more of the details as we go, and we're definitely getting closer.
   
   ---
   
   Separately.... I just want to mention that it might be helpful if, in redshift_connector, you raise a more helpful error message when the parameter combination is bad in a couple cases.  In particular when i tried iam with `user` (instead of `db_user` i got this:
   ```
       cache_key: str = IamHelper.get_credentials_cache_key(info)
     File "...python3.8/site-packages/redshift_connector/iam_helper.py", line 330, in get_credentials_cache_key
       return ";".join(
   TypeError: sequence item 0: expected str instance, NoneType found
   ```
   I think the real problem is that I need to use `db_user` instead of `user`. 
   
   Similarly when I omitted `database` I got this error:
   ```
   File "...python3.8/site-packages/redshift_connector/core.py", line 663, in __init__
       raise self.error
   redshift_connector.error.ProgrammingError: {'S': 'FATAL', 'C': '3D000', 'M': 'database "IAM:my_admin" does not exist', 'F': '/home/ec2-user/padb/src/pg/src/backend/utils/init/postinit.c', 'L': '517', 'R': 'LcInitPostgres'}
   ```
   And the problem was that I did not supply `database`.   Raising a more helpful error message in this case would make it more friendly for us to defer parameter validation to the library :) 
   
   Thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716999098



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       that sounds good to me, I'm happy to make this change if the others here agree with taking this route @josh-fell @JavierLopezT 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r715268254



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg

Review comment:
       addressed in 6ef57a4




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718025826



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       @mik-laj agreed with @josh-fell on this one, and i think i now agree with this too.
   
   namely we should do the following:
   * add RedshiftSqlHook to the existing `redshift` module (i.e. in the same module as the existing RedshiftHook)
   * deprecate RedshiftHook (renaming it to RedshiftClusterHook) -- this doesn't necessarily need to happen in this PR
   
   And those names we intend to stick with (i.e. ultimately we keep two hooks `RedshiftSqlHook` and `RedshiftClusterHook`)
   
   In the next major release _for this provider_ (which now that i think of it does not have to be in 3.0, since providers have separate release schedule), we remove `RedshiftHook` and we are left with `RedshiftSqlHook` and `RedshiftClusterHook`
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r716999098



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       that sounds good to me, I'm happy to make this change if the others here agree with taking this route @josh-fell @JavierLopezT 

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(

Review comment:
       cached property `conn` added, and `_get_conn_kwargs` removed in a1de44e

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type

Review comment:
       done in d90b760

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,

Review comment:
       addressed this in 6b3976b. the default port number of 5439 has been remove and I opted to selectively add kwargs as mentioned.

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,159 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+
+from typing import Callable, Dict, Optional, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.schema or '',
+        }
+
+        return conn_params
+
+    def _get_conn_kwargs(self) -> Dict:
+        """Helper method to retrieve connection kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        return conn.extra_dejson
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_type = RedshiftStatementHook.conn_type if not conn.conn_type else conn.conn_type
+
+        return URL(
+            drivername=conn_type,
+            username=conn_params['user'],
+            password=conn_params['password'],
+            host=conn_params['host'],
+            port=conn_params['port'],
+            database=conn_params['database'],
+        ).__str__()
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        conn_kwargs = self._get_conn_kwargs()
+        if engine_kwargs is None:
+            engine_kwargs = {}
+
+        if "connect_args" in engine_kwargs:
+            engine_kwargs["connect_args"] = {**conn_kwargs, **engine_kwargs["connect_args"]}
+        else:
+            engine_kwargs["connect_args"] = conn_kwargs
+
+        return super().get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
+
+    def get_conn(self) -> RedshiftConnection:
+        """Returns a redshift_connector.Connection object"""
+        conn_params = self._get_conn_params()
+        conn_kwargs = self._get_conn_kwargs()
+        conn_kwargs: Dict = {**conn_params, **conn_kwargs}
+        conn: RedshiftConnection = redshift_connector.connect(**conn_kwargs)
+
+        return conn
+
+    def run(
+        self,
+        sql: Union[str, list],
+        autocommit: bool = False,
+        parameters: Optional[dict] = None,
+        handler: Optional[Callable] = None,
+    ):
+        """
+        Runs a command or a list of commands. Pass a list of sql
+        statements to the sql parameter to get them to execute
+        sequentially
+
+        :param sql: the sql statement to be executed (str) or a list of
+            sql statements to execute
+        :type sql: str or list
+        :param autocommit: What to set the connection's autocommit setting to
+            before executing the query.
+        :type autocommit: bool
+        :param parameters: The parameters to render the SQL query with.
+        :type parameters: dict or iterable
+        :param handler: The result handler which is called with the result of each statement.
+        :type handler: callable
+        :return: query results if handler was provided.
+        """
+        return super().run(sql, autocommit=False, parameters=parameters, handler=handler)

Review comment:
       removed the method override. The hard coded ` autocommit=False`was left over from testing, so not needed. 1efe5a9

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftStatementHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')
+
+        return URL(drivername=conn_type, **conn_params).__str__()

Review comment:
       addressed in 949ff473e493a9af022924f5b1aea35ff5ff9af3

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member

Review comment:
       addressed in c9e182047b12474f00e9e5abca13a90364a44a0c

##########
File path: tests/providers/amazon/aws/hooks/test_redshift_statement.py
##########
@@ -0,0 +1,72 @@
+#
+# 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 json
+import unittest
+from unittest import mock
+
+from airflow.models import Connection
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class TestRedshiftStatementHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftStatementHook(RedshiftStatementHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'
+
+        self.db_hook = UnitTestRedshiftStatementHook()
+        self.db_hook.get_connection = mock.Mock()
+        self.db_hook.get_connection.return_value = self.connection
+
+    def test_get_uri(self):
+        uri_shouldbe = 'redshift+redshift_connector://login:password@host:5439/dev'
+        x = self.db_hook.get_uri()
+        assert uri_shouldbe == x

Review comment:
       addressed in 01091dcace8f3c6efd57a2f7d144f7a9336ecbcd

##########
File path: tests/providers/amazon/aws/operators/test_redshift.py
##########
@@ -0,0 +1,57 @@
+#
+# 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
+from unittest import mock
+
+from airflow.models.dag import DAG
+from airflow.providers.amazon.aws.operators.redshift import RedshiftOperator
+from airflow.utils import timezone
+
+DEFAULT_DATE = timezone.datetime(2015, 1, 1)
+DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
+DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10]
+TEST_DAG_ID = 'unit_test_dag'
+
+
+class TestRedshiftOperator(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+        args = {'owner': 'airflow', 'start_date': DEFAULT_DATE}
+        dag = DAG(TEST_DAG_ID, default_args=args)
+        self.dag = dag
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator(self, mock_get_hook):
+        sql = """
+        CREATE TABLE IF NOT EXISTS test_airflow (
+            dummy VARCHAR(50)
+        );
+        """
+        operator = RedshiftOperator(task_id='redshift_operator', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+
+    @mock.patch("airflow.providers.amazon.aws.operators.redshift.RedshiftOperator.get_hook")
+    def test_redshift_operator_test_multi(self, mock_get_hook):
+        sql = [
+            "CREATE TABLE IF NOT EXISTS test_airflow (dummy VARCHAR(50))",
+            "TRUNCATE TABLE test_airflow",
+            "INSERT INTO test_airflow VALUES ('X')",
+        ]
+        operator = RedshiftOperator(task_id='redshift_operator_test_multi', sql=sql, dag=self.dag)
+        operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)

Review comment:
       thank you for providing the re-write and very thorough explanation. i've addressed this in 7205a8cd7d0d05ed70dc7e5f87bf70d0227f0184, and added some parameterization for autocommit and parameters

##########
File path: airflow/providers/amazon/aws/example_dags/example_redshift.py
##########
@@ -0,0 +1,65 @@
+#
+# 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.
+"""
+This is an example dag for using `RedshiftOperator` to authenticate with Amazon Redshift
+using IAM authentication then executing a simple select statement

Review comment:
       addressed in 7205a8cd7d0d05ed70dc7e5f87bf70d0227f0184

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)

Review comment:
       addressed in ddd9d493




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720476596



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,73 @@
+#
+# 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 typing import List, Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift import RedshiftSQLHook
+
+
+class RedshiftSQLOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:RedshiftSQLOperator`
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Union[str, List[str]],
+        redshift_conn_id: str = 'redshift_default',

Review comment:
       removed hardcoded value in a02cdbc




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r720403036



##########
File path: airflow/providers/amazon/aws/hooks/redshift.py
##########
@@ -126,3 +135,101 @@ def create_cluster_snapshot(self, snapshot_identifier: str, cluster_identifier:
             ClusterIdentifier=cluster_identifier,
         )
         return response['Snapshot'] if response['Snapshot'] else None
+
+
+class RedshiftSQLHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(self.redshift_conn_id)  # type: ignore[attr-defined]
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftSQLHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')

Review comment:
       ~The reason for this approach is that the method that builds the URL expects `username` while the DB-API driver expects `user`.~ Scratch that, I'm going to rework this




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718013593



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       thank you for reaching out on slack, @dstandish. I think having two RedshiftHooks until 3.0, as you wrote on slack, would be a good path to getting this merged with consistent naming with the other database hooks.

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       This sounds good. As @josh-fell mentioned,
   > Although, this suggestion would may warrant renaming the new operator to match.
   
   We can rename the operator to `RedshiftSqlOperator` as well so the naming is inline with `RedshiftSqlHook`.

##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       thanks to all for collaborating to find a good name for the hook and operator! :) 
   
   I've made the rename to `RedshiftSQLHook` and `RedshiftSQLOperator`, and placed `RedshiftSQLHook` in the same module as `RedshiftHook` as Kamil mentioned on slack in a5e8786




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714444351



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']

Review comment:
       Is there a reason that this key can't be set to `username` originally, in the `_get_conn_params` function?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714386110



##########
File path: airflow/providers/amazon/aws/operators/redshift.py
##########
@@ -0,0 +1,70 @@
+#
+# 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 typing import Any, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.redshift_statement import RedshiftStatementHook
+
+
+class RedshiftOperator(BaseOperator):
+    """
+    Executes SQL Statements against an Amazon Redshift cluster
+
+    :param sql: the sql code to be executed
+    :type sql: Can receive a str representing a sql statement,
+        a list of str (sql statements)
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :type parameters: dict or iterable
+    :param autocommit: if True, each command is automatically committed.
+        (default value: False)
+    :type autocommit: bool
+    """
+
+    template_fields = ('sql',)
+    template_ext = ('.sql',)
+
+    def __init__(
+        self,
+        *,
+        sql: Any,
+        redshift_conn_id: str = 'redshift_default',
+        parameters: Optional[dict] = None,
+        autocommit: bool = True,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.redshift_conn_id = redshift_conn_id
+        self.sql = sql
+        self.redshift_conn_id = redshift_conn_id

Review comment:
       ```suggestion
           self.redshift_conn_id = redshift_conn_id
           self.sql = sql
   ```
   
   Removing duplicate line




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: Add RedshiftSQLHook, RedshiftSQLOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r719644641



##########
File path: tests/providers/amazon/aws/hooks/test_redshift.py
##########
@@ -103,3 +106,51 @@ def test_cluster_status_returns_available_cluster(self):
         hook = RedshiftHook(aws_conn_id='aws_default')
         status = hook.cluster_status('test_cluster')
         assert status == 'available'
+
+
+class TestRedshiftSQLHookConn(unittest.TestCase):
+    def setUp(self):
+        super().setUp()
+
+        self.connection = Connection(login='login', password='password', host='host', port=5439, schema="dev")
+
+        class UnitTestRedshiftSQLHook(RedshiftSQLHook):
+            conn_name_attr = "redshift_conn_id"
+            conn_type = 'redshift+redshift_connector'

Review comment:
       yes, the subclass is unnecessary. remove it in f6513ac03b06f2309f416749e215591e8443265c




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718045977



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       > When using acronyms in CapWords, capitalize all the letters of the acronym. Thus HTTPServerError is better than HttpServerError.
   
   [PEP-8](https://www.python.org/dev/peps/pep-0008/)
   
   We should use `RedshiftSQLHook` instead of `RedshiftSqlHook`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r718004578



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):

Review comment:
       I think RedshiftSqlHook is an acceptable name, though I reached out on slack to see if anyone has some advice for how we could end up with `RedshiftHook` as the sql hook.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717850202



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)

Review comment:
       addressed in ddd9d493




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Brooke-white commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
Brooke-white commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r717846133



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,131 @@
+#
+# 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.
+"""Execute statements against Amazon Redshift, using redshift_connector."""
+try:
+    from functools import cached_property
+except ImportError:
+    from cached_property import cached_property
+from typing import Dict, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift+redshift_connector'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    @cached_property
+    def conn(self):
+        return self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+    def _get_conn_params(self) -> Dict[str, Union[str, int]]:
+        """Helper method to retrieve connection args"""
+        conn = self.conn
+
+        conn_params: Dict[str, Union[str, int]] = {}
+
+        if conn.login:
+            conn_params['user'] = conn.login
+        if conn.password:
+            conn_params['password'] = conn.password
+        if conn.host:
+            conn_params['host'] = conn.host
+        if conn.port:
+            conn_params['port'] = conn.port
+        if conn.schema:
+            conn_params['database'] = conn.schema
+
+        return conn_params
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params = self._get_conn_params()
+
+        conn = self.conn
+
+        conn_type = conn.conn_type or RedshiftStatementHook.conn_type
+
+        if 'user' in conn_params:
+            conn_params['username'] = conn_params.pop('user')
+
+        return URL(drivername=conn_type, **conn_params).__str__()

Review comment:
       addressed in 949ff473e493a9af022924f5b1aea35ff5ff9af3




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] SamWheating commented on a change in pull request #18447: add RedshiftStatementHook, RedshiftOperator

Posted by GitBox <gi...@apache.org>.
SamWheating commented on a change in pull request #18447:
URL: https://github.com/apache/airflow/pull/18447#discussion_r714390481



##########
File path: airflow/providers/amazon/aws/hooks/redshift_statement.py
##########
@@ -0,0 +1,144 @@
+#
+# 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.
+"""Interact with AWS Redshift, using the boto3 library."""
+
+from typing import Callable, Dict, Optional, Tuple, Union
+
+import redshift_connector
+from redshift_connector import Connection as RedshiftConnection
+
+from airflow.hooks.dbapi import DbApiHook
+
+
+class RedshiftStatementHook(DbApiHook):
+    """
+    Execute statements against Amazon Redshift, using redshift_connector
+
+    This hook requires the redshift_conn_id connection. This connection must
+    be initialized with the host, port, login, password. Additional connection
+    options can be passed to extra as a JSON string.
+
+    :param redshift_conn_id: reference to
+        :ref:`Amazon Redshift connection id<howto/connection:redshift>`
+    :type redshift_conn_id: str
+
+    .. note::
+        get_sqlalchemy_engine() and get_uri() depend on sqlalchemy-amazon-redshift
+    """
+
+    conn_name_attr = 'redshift_conn_id'
+    default_conn_name = 'redshift_default'
+    conn_type = 'redshift'
+    hook_name = 'Amazon Redshift'
+    supports_autocommit = True
+
+    @staticmethod
+    def get_ui_field_behavior() -> Dict:
+        """Returns custom field behavior"""
+        return {
+            "hidden_fields": [],
+            "relabeling": {'login': 'User', 'schema': 'Database'},
+        }
+
+    def __init__(self, *args, **kwargs) -> None:
+        super().__init__(*args, **kwargs)
+
+    def _get_conn_params(self) -> Tuple[Dict[str, Union[str, int]], Dict]:
+        """Helper method to retrieve connection args and kwargs"""
+        conn = self.get_connection(
+            self.redshift_conn_id  # type: ignore[attr-defined]  # pylint: disable=no-member
+        )
+
+        conn_params: Dict[str, Union[str, int]] = {
+            "user": conn.login or '',
+            "password": conn.password or '',
+            "host": conn.host or '',
+            "port": conn.port or 5439,
+            "database": conn.extra_dejson.pop('database', ''),
+        }
+
+        conn_kwargs: Dict = {}
+
+        for param, arg in conn.extra_dejson.items():
+            conn_kwargs[param] = arg
+
+        return conn_params, conn_kwargs
+
+    def get_uri(self) -> str:
+        """
+        Override DbApiHook get_uri method for get_sqlalchemy_engine()
+
+        .. note::
+            Value passed to connection extra parameter will be excluded
+            from returned uri but passed to get_sqlalchemy_engine()
+            by default
+        """
+        from sqlalchemy.engine.url import URL
+
+        conn_params, _ = self._get_conn_params()
+        conn_params['username'] = conn_params['user']
+        del conn_params['user']
+
+        return URL(drivername='redshift+redshift_connector', **conn_params).__to_string__(hide_password=False)
+
+    def get_sqlalchemy_engine(self, engine_kwargs=None):
+        """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
+        _, conn_kwargs = self._get_conn_params()
+        if engine_kwargs is None:
+            engine_kwargs = {}

Review comment:
       ```suggestion
       def get_sqlalchemy_engine(self, engine_kwargs={}):
           """Overrides DbApiHook get_sqlalchemy_engine to pass redshift_connector specific kwargs"""
           _, conn_kwargs = self._get_conn_params()
   ```
   
   I see a similar pattern elsewhere in the codebase, but I'm not sure why. Wouldn't it be functionally identically to just set the default value of `engine_kwargs` to `{}`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org