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/27 07:08:57 UTC

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

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