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 2022/01/03 21:21:22 UTC

[GitHub] [airflow] kazanzhy opened a new pull request #20642: Added Hook for Amazon RDS

kazanzhy opened a new pull request #20642:
URL: https://github.com/apache/airflow/pull/20642


   I've two different implementations of `RDSHook` and asking for your advice and suggestions. 
   The second one is not finished yet because it needs a lot of time. 
   Could I ask for your advice and suggestions?


-- 
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] kazanzhy commented on pull request #20642: Added Hook for Amazon RDS

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


   When I run `mypy airflow/providers/amazon/aws/hooks` on `main` I'm getting only two errors.
   But when I'm adding `boto3-stubs[rds]` to `settings.py` and rebuild the image, then I'm getting a lot of errors.


-- 
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 #20642: Added Hook for Amazon RDS

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


   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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):

Review comment:
       Get it. Fixed




-- 
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] kazanzhy commented on pull request #20642: Added Hook for Amazon RDS

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


   @potiuk 
   That's very interesting.
   A few days ago the same code passed all checks.
   That's a very interesting issue because I'm running locally
   `./breeze static-check all -- --from-ref HEAD^ --to-ref HEAD`
   And everything is OK.
   
   I see from CI/CD that some errors are raised by
   `airflow/providers/amazon/aws/hooks/base_aws.py`
   https://github.com/apache/airflow/runs/4998566528?check_suite_focus=true


-- 
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 commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: setup.py
##########
@@ -194,6 +194,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
     'redshift_connector~=2.0.888',
     'sqlalchemy_redshift~=0.8.6',
     pandas_requirement,
+    'boto3_stubs[rds]~=1.20.32',

Review comment:
       Yep. Lets leave it unpinned. 




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: docs/apache-airflow-providers-amazon/operators/rds.rst
##########
@@ -0,0 +1,19 @@
+ .. 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.
+
+RDS management operators

Review comment:
       Firstly I thought to add Hook and operators by one PR and then split.
   Removed




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,60 @@
+#
+# 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 RDS."""
+
+from typing import TYPE_CHECKING
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+if TYPE_CHECKING:
+    from mypy_boto3_rds import RDSClient
+
+
+class RdsHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using proper client from the boto3 library.
+
+    Hook attribute `client` has all methods that listed in documentation
+
+    .. seealso::
+        - https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+        - https://docs.aws.amazon.com/rds/index.html
+
+    Additional arguments (such as ``aws_conn_id`` or ``region_name``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str

Review comment:
       ```suggestion
   ```




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       hmm yeah i didn't scrutinize the `__getattr__` in the first pass but i see it now
   
   ok so you are just trying to make it so all boto3 methods work on the hook also.
   
   personally i do not think this good.   i see _no_ benefit to doing this because users can just use the `conn` cached property on AwsBaseHook.  (though `client` would have been a better name for the  attr). i do see a cost though because then we can't implement any of those methods in a different  way for example, and to me it introduces some duplication and, i think, confusion.  in this case we can very easily, without any cost, adhere to the "only one way to do something" principle.
   
   for auto-complete we can add a protocol class for autocomplete on the conn object.  or perhaps we can see if there is a boto3 typing stub library that we could use here (and possibly for other hooks as well)
   
   so to recap my suggestion is, don't use getattr, just direct users to use `conn` or perhaps better yet add property `client` that  returns `self.conn` then add a protocol classs (search the airflow repo for examples) where we stub out the methods for the client
   
   then that's it!
   
   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] dstandish commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')
+
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')
+    @mock_rds
+    def test_client_db_method_from_aws_rds_hook(self):
+        self._create_instances()
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        instances = hook.client.describe_db_instances()['DBInstances']
+        assert len(instances) == 2
+
+    @unittest.expectedFailure

Review comment:
       yeah it's not needed to test that in thiss hook since this hook is just a pass-through now -- life is easier this way




-- 
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 commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: docs/apache-airflow-providers-amazon/index.rst
##########
@@ -87,6 +87,7 @@ PIP package              Version required
 ``redshift_connector``   ``~=2.0.888``
 ``sqlalchemy_redshift``  ``~=0.8.6``
 ``watchtower``           ``~=2.0.1``
+``boto3_stubs[rds]``     ``>=1.20.46``

Review comment:
       No worries. It wll get overridden automatically - I plan to start preparing providers tomorrow!




-- 
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 #20642: Added Hook for Amazon RDS

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


   > I would really love if you could upvote and comment here:
   
   done
   
   > @kazanzhy @dstandish I think I managed to workaround the pip resolver issue with https://github.com/apache/airflow/pull/21824 - please rebase to latest main.
   
   thanks a bunch!
   
   


-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       I checked your suggestion and autocompletion works only for `hook.conn`.
   I think we should not override `get_conn` and I'll use just `hook.conn.some_method()` in my RDS operators
   




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')
+
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')
+    @mock_rds
+    def test_client_db_method_from_aws_rds_hook(self):
+        self._create_instances()
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        instances = hook.client.describe_db_instances()['DBInstances']
+        assert len(instances) == 2
+
+    @unittest.expectedFailure

Review comment:
       As I see `rds_mock` doesn't support some cluster operations.
   But as you wrote in later comments it's better doesn't complicate this test, so I move this mock to Operators` tests




-- 
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 commented on pull request #20642: Added Hook for Amazon RDS

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


   Just rebase to latest main and let's see.


-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):

Review comment:
       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] potiuk closed pull request #20642: Added Hook for Amazon RDS

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


   


-- 
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] kazanzhy commented on pull request #20642: Added Hook for Amazon RDS

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


   A lot of thanks that clarifying the way to implementation @dstandish 
   
   
   The solutions below don't work and it's expected 
   ```
   from typing import Type
   from botocore.client import BaseClient
   self.client: Type[BaseClient]
   
   import boto3
   rds_client = boto3.client('rds', 'us-east-1')
   self.client: type(rds_client)
   ```
   Did you mean the solution that I started to implement?


-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       Hi @kazanzhy 
   
   > I could finish the implementation, but I need some approval that this code is really needed and won't be deleted during the next reviews.
   
   I definitely appreciate your concerns @kazanzhy.
   
   I only suggested using `Protocol` as an alternative to duplicating all the client methods on the hook (if the goal is autocomplete).  I wouldn't say it's really necessary and I totally understand your reluctance to add all of this code just for autocomplete.
   
   I just took a look and it seems there is a good library we may be able to leverage here.
   
   Take a look here: https://github.com/vemel/boto3_stubs_docs
   
   I installed with `pip install 'boto3-stubs[s3,ec2,rds]'` and tried out hinting like this:
   ```python
       def get_conn(self) -> RDSClient:
           return super().get_conn()
   ```
   
   and it seemed to work.  Maybe this is a good solution here?
   
   What do you think?
   
   




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       That's awesome. A lot of thanks :)
   
   I think after this PR we could refactor Amazon provider in the following way:
   - add `client` attribute to the hook, which will be typed by some class from boto3_stub
   - refactor Operators to replace `hook.get_conn().[method]` by the `hook.client.[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] dstandish commented on pull request #20642: Added Hook for Amazon RDS

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


   finally we can merge.  thanks again to @potiuk for resolving the pip problem, and thank you @kazanzhy for your persistence and patience.


-- 
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] eladkal commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):

Review comment:
       Naming should be `RdsHook`
   see https://github.com/apache/airflow/issues/20296




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       hmm yeah i didn't scrutinize the `__getattr__` in the first pass but i see it now
   
   ok so you are just trying to make it so all boto3 methods work on the hook also.
   
   personally i do not think this good.   i see _no_ benefit to doing this because users can just use the `conn` cached property on AwsBaseHook.  (though `client` would have been a better name for the  attr).  in this case we can very easily, without any cost, adhere to the "only one way to do something" principle.
   
   for auto-complete we can add a protocol class for autocomplete on the conn object.  or perhaps we can see if there is a boto3 typing stub library that we could use here (and possibly for other hooks as well)
   
   so to recap my suggestion is, don't use getattr, just direct users to use `conn` or perhaps better yet add property `client` that  returns `self.conn` then add a protocol classs (search the airflow repo for examples) where we stub out the methods for the client
   
   then that's it!
   
   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] dstandish commented on pull request #20642: Added Hook for Amazon RDS

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


   @potiuk i'm a bit confused why we are seeing failures like this in this pr... https://github.com/apache/airflow/runs/5252964277?check_suite_focus=true#step:11:1533
   anything pop out to you re why?


-- 
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] kazanzhy edited a comment on pull request #20642: Added Hook for Amazon RDS

Posted by GitBox <gi...@apache.org>.
kazanzhy edited a comment on pull request #20642:
URL: https://github.com/apache/airflow/pull/20642#issuecomment-1045315483


   So there are three ways to install stubs for RDS.
   - `boto3-stubs[rds]` - ERRORS
   - `boto3-stubs-lite[rds]` - ERRORS
   - `mypy-boto3-rds` - OK
   
   Seems `boto3-stubs` overload something in botocore.
   I'll try to find the solution because it's more convenient to put all necessary libs inside `boto3-stubs` than list them.


-- 
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] eladkal commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -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.
+"""Interact with AWS RDS."""
+
+import sys
+from typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+if sys.version_info >= (3, 8):
+    from typing import Protocol
+else:
+    from typing_extensions import Protocol

Review comment:
       Why not using `from airflow.typing_compat import Protocol `?




-- 
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] kazanzhy commented on pull request #20642: Added Hook for Amazon RDS

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


   I've rebased to the latest main but seems something wrong with checks.
   
   I'm getting 213 errors on the latest main after I run this command in the breeze shell 
   `mypy --namespace-packages airflow/providers/amazon/aws/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] dstandish commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       Hi @kazanzhy 
   
   > I could finish the implementation, but I need some approval that this code is really needed and won't be deleted during the next reviews.
   
   I definitely appreciate your concerns @kazanzhy.
   
   I only suggested using `Protocol` as an alternative to duplicating all the client methods on the hook (if the goal is autocomplete).  I wouldn't say it's really necessary and I totally understand your reluctance to add all of this code just for autocomplete.
   
   I just took a look and it seems there is a good library we may be able to leverage here.
   
   Take a look here: https://github.com/vemel/boto3_stubs_docs
   
   I installed with `pip install 'boto3-stubs[s3,ec2,rds]'` and tried out hinting like this:
   ```python
   from mypy_boto3_rds import RDSClient
   ...
       def get_conn(self) -> RDSClient:
           return super().get_conn()
   ```
   
   and it seemed to work.  Maybe this is a good solution here?
   
   What do you think?
   
   




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')
+
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')
+    @mock_rds
+    def test_client_db_method_from_aws_rds_hook(self):
+        self._create_instances()
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        instances = hook.client.describe_db_instances()['DBInstances']
+        assert len(instances) == 2
+
+    @unittest.expectedFailure
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')

Review comment:
       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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -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.
+"""Interact with AWS RDS."""
+
+import sys
+from typing import Optional
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+if sys.version_info >= (3, 8):
+    from typing import Protocol
+else:
+    from typing_extensions import Protocol

Review comment:
       Thanks. Switched to it :)




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       I've partially implemented the protocol class and, unfortunately, doesn't see sense to continue.
   The resulting class will be more than one thousand rows that need to be supported and changed, based on new versions of the boto3 client.
   I could finish the implementation, but I need some approval that this code is really needed and won't be deleted during the next reviews.




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       > My motivation to use client is that it's more clear and convenient to use hook.client.some_method() than hook.conn.some_method().
   But I see that redshift operators use hook.get_conn().some_method() which is much more unclear in my opinion.
   
   Yeah I completely agree.
   
   > I think we should not override get_conn and I'll use just hook.conn.some_method() in my RDS operators
   
   Yeah I'm good with that.




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: docs/apache-airflow-providers-amazon/operators/rds.rst
##########
@@ -0,0 +1,19 @@
+ .. 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.
+
+RDS management operators

Review comment:
       is this file generated automatically?  if not it sseems we should remove it since you aren't adding any operators




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient

Review comment:
       Done.
   I thought that this hiding will disable auto-complete but it still works on Pycharm




-- 
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 commented on pull request #20642: Added Hook for Amazon RDS

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


   @kazanzhy  @dstandish  I think I managed to workaround the pip resolver issue with https://github.com/apache/airflow/pull/21824 - please rebase to latest main.


-- 
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 #20642: Added Hook for Amazon RDS

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


   


-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient

Review comment:
       i think you can / should put this behind a type checking guard
   
   e.g. like here https://github.com/apache/airflow/blob/602abe8394fafe7de54df7e73af56de848cdf617/airflow/lineage/backend.py#L21-L22
   
   and then you quote references to it




-- 
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 commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: setup.py
##########
@@ -194,6 +194,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
     'redshift_connector~=2.0.888',
     'sqlalchemy_redshift~=0.8.6',
     pandas_requirement,
+    'boto3_stubs[rds]~=1.20.32',

Review comment:
       Yep. Lets live it unpinned. 




-- 
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 commented on pull request #20642: Added Hook for Amazon RDS

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


   Some statuc checks and tests need fixing though 


-- 
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] eladkal commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: docs/apache-airflow-providers-amazon/index.rst
##########
@@ -87,6 +87,7 @@ PIP package              Version required
 ``redshift_connector``   ``~=2.0.888``
 ``sqlalchemy_redshift``  ``~=0.8.6``
 ``watchtower``           ``~=2.0.1``
+``boto3_stubs[rds]``     ``>=1.20.46``

Review comment:
       i think this file should not be edited as it created automatically by the process of prepare provider release
   cc @potiuk 




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: setup.py
##########
@@ -194,6 +194,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
     'redshift_connector~=2.0.888',
     'sqlalchemy_redshift~=0.8.6',
     pandas_requirement,
+    'boto3_stubs[rds]~=1.20.32',

Review comment:
       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] kazanzhy removed a comment on pull request #20642: Added Hook for Amazon RDS

Posted by GitBox <gi...@apache.org>.
kazanzhy removed a comment on pull request #20642:
URL: https://github.com/apache/airflow/pull/20642#issuecomment-1005276831


   A lot of thanks that clarifying the way to implementation @dstandish 
   
   
   The solutions below don't work and it's expected 
   ```
   from typing import Type
   from botocore.client import BaseClient
   self.client: Type[BaseClient]
   
   import boto3
   rds_client = boto3.client('rds', 'us-east-1')
   self.client: type(rds_client)
   ```
   Did you mean the solution that I started to implement?


-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       I've partially implemented the protocol class and, unfortunately, doesn't see sense to continue.
   The resulting class will be more than one thousand rows that need to be supported and changed, based on new versions of the boto3 client.
   I could finish the implementation, but I need some approval that this code is really needed and won't be deleted during the next reviews.




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       Probably you're right.
   My motivation to use `client` is that it's more clear and convenient to use `hook.client.some_method()` than `hook.conn.some_method()`.
   But I see that redshift operators use `hook.get_conn().some_method()` which is much more unclear in my opinion.




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       I completely agree with you. The `Alt` version will just add a lot of unuseful code.
   
   But in another way, I know that many developers don't like to override the `__getattr__` method.
   Actually, I've also got a recursion but fixed it with this PR https://github.com/apache/airflow/pull/20355




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       i don't recommend creating methods like this in hooks. what you're doing  here is essentially making an alias for the boto3 method.  but instead, the user can just do `client = hook.get_conn()` and calll the method directly.  airflow does not need to get in the way between user and library without good reason.
   
   so i would not recommend adding  any of these methods and i do not think you need the Alt version of this hook.  in any case you could add the first version first, and add additional methods later as the need arises.




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       A lot of thanks that clarifying the way to implementation @dstandish 
   
   The solutions below don't work and it's expected 
   ```
   from typing import Type
   from botocore.client import BaseClient
   self.client: Type[BaseClient]
   
   import boto3
   rds_client = boto3.client('rds', 'us-east-1')
   self.client: type(rds_client)
   ```
   Did you mean the solution that I started to implement?
   https://github.com/apache/airflow/pull/20642/files#diff-07892cfb7f2a919afec9185e36b417d75d5322d243096667ca1cf05c506d9533




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       A lot of thanks that clarifying the way to implementation @dstandish 
   
   The solutions below don't work and it's expected 
   ```
   from typing import Type
   from botocore.client import BaseClient
   self.client: Type[BaseClient]
   
   import boto3
   rds_client = boto3.client('rds', 'us-east-1')
   self.client: type(rds_client)
   ```
   Did you mean the solution that I started to implement?




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')
+
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')
+    @mock_rds
+    def test_client_db_method_from_aws_rds_hook(self):
+        self._create_instances()
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        instances = hook.client.describe_db_instances()['DBInstances']
+        assert len(instances) == 2
+
+    @unittest.expectedFailure

Review comment:
       why expected failure? what is  the purpose of this test?

##########
File path: setup.py
##########
@@ -194,6 +194,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version
     'redshift_connector~=2.0.888',
     'sqlalchemy_redshift~=0.8.6',
     pandas_requirement,
+    'boto3_stubs[rds]~=1.20.32',

Review comment:
       i think we probably don't need to pin the version here 
   
   cc @mik-laj @potiuk 

##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):

Review comment:
       should not inherit from unittest anymore we use pytest and sometimes it's incompatible with certain pytest features

##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):

Review comment:
       since you're not implementing methods anymore, just `conn` maybe it is unnecesssary to test these client methods -- since those are just boto features
   
   maybe it would be better to simply test that `hook.conn` returns the right client (namely an RDS client)
   
   e.g. i think this works
   
   ```python
   hook = RDSHook()
   conn = hook.conn
   assert conn.__class__.__name__ == 'RDS'
   ```
   in trying this out locally i had to patch env var with `AIRFLOW_CONN_AWS_DEFAULT=s3://@` but YMMV
   
   i'm updating test util `env_vars` to allow this [here](https://github.com/apache/airflow/pull/20818).
   
   then i think we  can  remove everything related to mock_rds

##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')

Review comment:
       think this will fail because no longer implementing this property

##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')
+
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')
+    @mock_rds
+    def test_client_db_method_from_aws_rds_hook(self):
+        self._create_instances()
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        instances = hook.client.describe_db_instances()['DBInstances']
+        assert len(instances) == 2
+
+    @unittest.expectedFailure
+    @unittest.skipIf(mock_rds is None, 'mock_rds package not present')

Review comment:
       i think nowadays we use `@pytest.mark.skipif`




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.py
##########
@@ -0,0 +1,165 @@
+#
+# 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 RDS."""
+
+from typing import Any, Callable, Dict, List, Optional, Union
+
+from airflow.exceptions import AirflowException
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RDSHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using the boto3 library.
+    Client methods could be used from hook object, like: hook.create_db_instance()
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All available methods are listed in documentation
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def __getattr__(self, item: str) -> Union[Callable, AirflowException]:
+        client = self.get_conn()
+        if hasattr(client, item):
+            self.log.info("Calling boto3 RDS client's method:", item)
+            return getattr(client, item)
+        else:
+            raise AirflowException("Boto3 RDS client doesn't have a method:", item)
+
+
+class RDSHookAlt(AwsBaseHook):
+    """
+    Interact with AWS RDS, using the boto3 library.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    All unimplemented methods available from boto3 RDS client -> hook.get_conn()
+    .. seealso::
+        https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    def describe_db_snapshots(

Review comment:
       On reflection, my suggestion re adding  a `client` attribute is probably a bad one actually.  
   
   We should just use the existing cached property `conn` (or get_conn which just returns it) rather than adding another property with a different name representing the same object. 
   
   I don't like `conn` as the property name because it is ambiguous.  In many places it is ussed to mean airflow Connection but here it means "the client" and it's what is returned by `get_conn` which in airflow by convention is the client object for the hook.
   
   But adding another property that refers to the same object is probably not the best choice.
   
   What do you think?
   
   So you can do like
   
   ```
   @property
   def conn(self) -> RDSClient:
       return super().conn
   ``` 
   
   (the only purpose of implementing is to get the autocomplete)
   
   operators can continue to use get_conn or conn.
   
   If you implement `conn` in this way, you might check to see if you also need to implement `get_conn` or if autocomplete already works with `get_conn` after implementing `conn`. 
   
   
   




-- 
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] kazanzhy commented on a change in pull request #20642: Added Hook for Amazon RDS

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



##########
File path: tests/providers/amazon/aws/hooks/test_rds.py
##########
@@ -0,0 +1,85 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import unittest
+
+import boto3
+
+from airflow.providers.amazon.aws.hooks.rds import RDSHook
+
+try:
+    from moto import mock_rds
+except ImportError:
+    mock_rds = None
+
+
+class TestRDSHook(unittest.TestCase):
+    @staticmethod
+    def _create_instances():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance1',
+            DBInstanceClass='db.m4.large',
+            Engine='postgres',
+        )
+        client.create_db_instance(
+            DBInstanceIdentifier='mydbinstance2',
+            DBInstanceClass='db.m4.large',
+            Engine='mysql',
+        )
+        if not client.describe_db_instances()['DBInstances']:
+            raise ValueError('AWS not properly mocked')
+
+    @staticmethod
+    def _create_clusters():
+        client = boto3.client('rds', region_name='us-east-1')
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster1',
+            Engine='postgres',
+        )
+        client.create_db_cluster(
+            DBClusterIdentifier='my-cluster2',
+            Engine='mysql',
+        )
+        if not client.describe_db_clusters()['DBClusters']:
+            raise ValueError('AWS not properly mocked')
+
+    def test_client_attribute_existence(self):
+        hook = RDSHook(aws_conn_id='aws_default', region_name='us-east-1')
+
+        assert hasattr(hook, 'client')

Review comment:
       Yeap. I'm going to fix and extend tests




-- 
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 #20642: Added Hook for Amazon RDS

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



##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient

Review comment:
       i think you can / should put this behind a type checking guard
   
   e.g. like here https://github.com/apache/airflow/blob/602abe8394fafe7de54df7e73af56de848cdf617/airflow/lineage/backend.py#L21-L22
   
   and then you quote references to it

##########
File path: airflow/providers/amazon/aws/hooks/rds.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.
+"""Interact with AWS RDS."""
+
+from mypy_boto3_rds import RDSClient
+
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class RdsHook(AwsBaseHook):
+    """
+    Interact with AWS RDS using proper client from the boto3 library.
+
+    Hook attribute `client` has all methods that listed in documentation
+
+    .. seealso::
+        - https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/rds.html
+        - https://docs.aws.amazon.com/rds/index.html
+
+    Additional arguments (such as ``aws_conn_id`` or ``region_name``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. seealso::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+
+    :param aws_conn_id: The Airflow connection used for AWS credentials.
+    :type aws_conn_id: str
+    """
+
+    def __init__(self, *args, **kwargs) -> None:
+        kwargs["client_type"] = "rds"
+        super().__init__(*args, **kwargs)
+
+    @property
+    def conn(self) -> RDSClient:

Review comment:
       then this becomes
   ```suggestion
       def conn(self) -> "RDSClient":
   ```




-- 
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] kazanzhy closed pull request #20642: Added Hook for Amazon RDS

Posted by GitBox <gi...@apache.org>.
kazanzhy closed pull request #20642:
URL: https://github.com/apache/airflow/pull/20642


   


-- 
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 commented on pull request #20642: Added Hook for Amazon RDS

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


   @kazanzhy @dstandish -  I would really love if you could upvote and comment here: https://github.com/pypa/pip/pull/10258#issuecomment-1025125753. 
   
   The problem is that we have no way currently to determine what is the root cause of the problem. The problem is that `pip` is not able to solve dependencies in a reasonable time and runs backtracking for hours - but it does not really give us the possibilty to get the root cause message that could allow us to determine how we could help `pip`. 
   
   I proposed to add optional timeout on `pip install` execution producing a "conflict" message if the timeout is reached. That could  give us at least a chance to understand what is the problem (someone released a new dependency that makes `pip` fail installing airflow from the scratch and we need to help `pip` to determine the right set of dependencies because it is unable to figure it out on its own).


-- 
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 #20642: Added Hook for Amazon RDS

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


   > So there are three ways to install stubs for RDS.
   boto3-stubs[rds] - ERRORS
   boto3-stubs-lite[rds] - ERRORS
   mypy-boto3-rds - OK
   Seems boto3-stubs overload something in botocore.
   I'll try to find the solution because it's more convenient to put all necessary libs inside boto3-stubs than list them.
   
   Nice! glad you figured it out.


-- 
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