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 2020/08/31 06:35:21 UTC

[GitHub] [airflow] dyon78 opened a new pull request #10655: [AIRFLOW-10645] secrets manager as a hook

dyon78 opened a new pull request #10655:
URL: https://github.com/apache/airflow/pull/10655


    Make sure you have checked _all_ steps below.
    
    10645 Jira
    * [x]  My PR addresses the following [Airflow Jira](https://issues.apache.org/jira/browse/AIRFLOW/) issues and references them in the PR title. For example, "[AIRFLOW-10645] My Airflow PR"
      
      * https://issues.apache.org/jira/browse/AIRFLOW-10645
      * In case you are fixing a typo in the documentation you can prepend your commit with [AIRFLOW-XXX], code changes always need a Jira issue.
      * In case you are proposing a fundamental code change, you need to create an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)).
      * In case you are adding a dependency, check if the license complies with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
    
   > ### Description
    * [x]  Here are some details about my PR, including screenshots of any UI changes:
    
   Adding AWS Secrets Manager as a hook
   
    
    ### Tests
    * [x]  My PR adds  unit tests 
    
    ### Commits
    * [x]  My commits all reference Jira issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
      
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"
   
   


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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       Added one more function to return secrets as dict




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       You are right, thanks - got 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.

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:
+        """
+        Create queue using connection object
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])

Review comment:
       Added tests for binary secrets




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

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



[GitHub] [airflow] houqp merged pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   


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

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



[GitHub] [airflow] houqp commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:
+        """
+        Create queue using connection object
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])

Review comment:
       could you add a test for binary secret as well?

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       might be better to just return the raw secret bytes here and let caller do the json.loads manually instead. there is no guarantee stored secret string will be a valid json object, especially for SecretBinary.




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.

Review comment:
       ```suggestion
           reflecting format it stored in the AWS Secrets Manager
   
           :param secret_name: the secret id in the AWS Secrets Manager.
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, secret_value)

Review comment:
       ```suggestion
           secret = hook.get_secret(secret_name)
           self.assertEqual(secret, secret_value)
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, secret_value)
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_dict(self):

Review comment:
       ```suggestion
       def test_get_secret_dict(self):
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, secret_value)
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_dict(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret_as_dict(secret_name)
+        self.assertEqual(secrets, json.loads(secret_value))

Review comment:
       ```suggestion
           secret = hook.get_secret_as_dict(secret_name)
           self.assertEqual(secret, json.loads(secret_value))
   ```

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.

Review comment:
       ```suggestion
           Retrieve secret value from AWS Secrets Manager in a dict representation
   
           :param secret_name: the secret id in the AWS Secrets Manager.
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):

Review comment:
       ```suggestion
       def test_get_secret_string(self):
   ```

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.

Review comment:
       ```suggestion
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, secret_value)
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_dict(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret_as_dict(secret_name)
+        self.assertEqual(secrets, json.loads(secret_value))
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_binary(self):

Review comment:
       ```suggestion
       def test_get_secret_binary(self):
   ```

##########
File path: tests/providers/amazon/aws/hooks/test_secrets_manager.py
##########
@@ -0,0 +1,88 @@
+#
+# 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 base64
+import json
+
+from airflow.providers.amazon.aws.hooks.secrets_manager import SecretsManagerHook
+
+try:
+    from moto import mock_secretsmanager
+except ImportError:
+    mock_secretsmanager = None
+
+
+class TestSecretsManagerHook(unittest.TestCase):
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_conn_returns_a_boto3_connection(self):
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+        self.assertIsNotNone(hook.get_conn())
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_string(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, secret_value)
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_dict(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value = '{"user": "test"}'
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretString': secret_value,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret_as_dict(secret_name)
+        self.assertEqual(secrets, json.loads(secret_value))
+
+    @unittest.skipIf(mock_secretsmanager is None, 'mock_secretsmanager package not present')
+    @mock_secretsmanager
+    def test_get_secrets_binary(self):
+        secret_name = "arn:aws:secretsmanager:us-east-2:999999999999:secret:db_cluster-YYYYYYY"
+        secret_value_binary = base64.b64encode(b'{"username": "test"}')
+        hook = SecretsManagerHook(aws_conn_id='aws_default')
+
+        param = {
+            'SecretId': secret_name,
+            'SecretBinary': secret_value_binary,
+        }
+
+        hook.get_conn().put_secret_value(**param)
+
+        secrets = hook.get_secret(secret_name)
+        self.assertEqual(secrets, base64.b64decode(secret_value_binary))

Review comment:
       ```suggestion
           secret = hook.get_secret(secret_name)
           self.assertEqual(secret, base64.b64decode(secret_value_binary))
   ```




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       An idea what @houqp suggested and I follow is to make it as low level as possible and do not do preprocessing so caller always have flexibility what to do with Secrets. If we need secrets as a dict I would add wrapper on top of existing function that returns `str` that'll return it as a `dict`. What about this?




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       For ['SecretString'] it's `str` according to `type(sm.get_secrets(secret_name))`. Since we put responsibility on how to handle output result to caller I've changed return type to `Union[str, bytes]` 

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       For ['SecretString'] it's `str` according to `type(hook.get_secrets(secret_name))`. Since we put responsibility on how to handle output result to caller I've changed return type to `Union[str, bytes]` 




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

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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #10655: [AIRFLOW-10645] secrets manager as a hook

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


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


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

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



[GitHub] [airflow] JavierLopezT commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       What about something like this for string secrets (not binary):
   ```
   try:
       secret = json.loads(secret)
   except JSONDecodeError:
       pass
   ```
   That way if its a JSON we get directly the dict.




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

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



[GitHub] [airflow] houqp commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       yeah, it would be much better to add a special wrapper for it. For secrets that the caller is sure to not be stored in JSON format, there is no point incurring this extra doomed to fail json decoding at runtime.




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

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



[GitHub] [airflow] JavierLopezT commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       @houqp In which cases a secret won't be a valid json object? 




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       base64.b64decode returns `bytes`, not `str` - we already have such type of discussion here and I had to change it to `bytes`, docs about it - https://docs.python.org/3/library/base64.html.
   
   Applied another suggested changes




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

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



[GitHub] [airflow] dyon78 commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   > Can you add another test that tests that an invalid json during `get_secret_as_dict` does not fail i.e. is None, please.
   
   As the function has been changed I believe this test is not needed anymore 


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

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



[GitHub] [airflow] kaxil commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       we just need to add a debug log before the "pass"




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       I agree, especially accounting that we follow approach not to do much processing here and giving flwxibility to do it for caller.

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       I agree, especially accounting that we follow approach not to do much processing here and giving flwxibility to do it for caller. Changed accordingly.

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       I agree, especially accounting that we follow approach not to do much processing here and giving flexibility to do it for caller. Changed accordingly.




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

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



[GitHub] [airflow] JavierLTPromofarma commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       @houqp In which cases a secret would not be a valid json object? 




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       For ['SecretString`] it's `str` according to `type(sm.get_secrets(secret_name_test))`. Since we put responsibility on how to handle output result to caller I've changed return type to `Union[str, bytes]` 




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       ```suggestion
               self.log.error('Unable to parse secrets as a dict: %s', secret_name)
   ```
   I think that should be `error` or `warning`? @kaxil WDYT?




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

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



[GitHub] [airflow] houqp commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       `base64.b64decode` returns `bytes` not str, i am guessing get_secret_value_response['SecretString'] is also a `bytes` object, could you confirm whether that's the case?




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       base64.b64decode returns `bytes`, not `str` - we already have such type of discussion here and and I had to change it to `bytes`, docs about it - https://docs.python.org/3/library/base64.html.




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

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



[GitHub] [airflow] mik-laj commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10655:
URL: https://github.com/apache/airflow/pull/10655#issuecomment-686771887


   @houqp I don't have experience with AWS. I am focusing on GCP. @feluelle Can I ask for a review?


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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       For ['SecretString`] it is `str` according to `type(sm.get_secrets(secret_name_test))`. Since we put responsibility on how to handle output result to caller I've changed return type to `Union[str, bytes]` 




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

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



[GitHub] [airflow] JavierLopezT commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       @houqp In which cases a secret won't be a valid json object? How would that secret stored in AWS Secrets Manager? 




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

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



[GitHub] [airflow] JavierLTPromofarma commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   Would you think it would be useful to add this line at the end? 
   `secret_dict = ast.literal_eval(secret)`
   That will return the dictionary instead of a string and you don't have to make the literal_eval in your script. 


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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       Actually as this functionality (json.load) has its own function I think we should not try-except at all.

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       Actually as this functionality (json.load) has its own function now I think we should not try-except at all.




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       ```suggestion
       def get_secret(self, secret_name: str) -> str:
   ```
   This is only returning one secret, correct? And it will be decoded to string if it is a `SecretBinary`. So it is always of type `str`.

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       ^ Please also update the docs according that.
   
   I think the title is not correct? Same in `get_secrets_as_dict`.

##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Create queue using connection object
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secrets_as_dict(self, secret_name: str) -> Optional[dict]:

Review comment:
       ```suggestion
       def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
   ```
   Same here.




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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       @houqp @JavierLopezT @dyon78 WDYT?




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

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



[GitHub] [airflow] dyon78 commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   @JavierLopezT , @kaxil , @mik-laj , @feluelle please re-review


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

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



[GitHub] [airflow] feluelle commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,73 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secret(self, secret_name: str) -> Union[str, bytes]:
+        """
+        Retrieve secret value from AWS Secrets Manager as a str or bytes
+        reflecting format it stored in the AWS Secrets Manager
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: Union[str, bytes] with the information about the secrets
+        :rtype: Union[str, bytes]
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        get_secret_value_response = self.get_conn().get_secret_value(SecretId=secret_name)
+        if 'SecretString' in get_secret_value_response:
+            secret = get_secret_value_response['SecretString']
+        else:
+            secret = base64.b64decode(get_secret_value_response['SecretBinary'])
+        return secret
+
+    def get_secret_as_dict(self, secret_name: str) -> Optional[dict]:
+        """
+        Retrieve secret value from AWS Secrets Manager in a dict representation
+        :param secret_name: name of the secrets.
+        :type secret_name: str
+        :return: dict with the information about the secrets
+        :rtype: dict
+        """
+        # Depending on whether the secret is a string or binary, one of
+        # these fields will be populated.
+        secret = None
+        try:
+            secret = json.loads(self.get_secret(secret_name))
+        except json.JSONDecodeError:
+            self.log.debug('Unable to parse secrets as a dict: %s', secret_name)

Review comment:
       The `json.load` is the whole point in calling this function.




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

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



[GitHub] [airflow] JavierLTPromofarma edited a comment on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   Do you think it would be useful to add this line at the end? 
   `secret_dict = ast.literal_eval(secret)`
   That will return the dictionary instead of a string and you don't have to make the literal_eval in your script. 


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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       yes, returning raw secrets now




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       An idea what @houqp suggested and I follow is to make it as low level as possible and do not do preprocessing so caller always have flexibility what to do with Secrets. If we need secrets as a dict I would add wrapper on top of existing function that returns `str` that'll return it as a `dict` so we'll have two functions at least. What about this?




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

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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   Awesome work, congrats on your first merged pull request!
   


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

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



[GitHub] [airflow] JavierLTPromofarma removed a comment on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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


   Do you think it would be useful to add this line at the end? 
   `secret_dict = ast.literal_eval(secret)`
   That will return the dictionary instead of a string and you don't have to make the literal_eval in your script. 


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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       An idea what @houqp suggested and I follow is to make it as low level as possible and do not do preprocessing so caller always have flexibility what to do with Secrets. If we need secrets as a dict I would add wrapper on top of existing function that returns `str` that'll return dict. What about this?




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,53 @@
+#
+# 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 base64
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> str:

Review comment:
       For ['SecretString'] it's `str` according to `type(sm.get_secrets(secret_name_test))`. Since we put responsibility on how to handle output result to caller I've changed return type to `Union[str, bytes]` 




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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       base64.b64decode returns `bytes`, not `str` - we already have such type of discussion here and and I had to change it to `bytes`, docs about it - https://docs.python.org/3/library/base64.html.
   
   Applied another suggested changes




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

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



[GitHub] [airflow] houqp commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,54 @@
+#
+# 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 base64
+import json
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> dict:

Review comment:
       we store a lot of none json secret values in secret managers, for example, simple database passwords without the quote, private certificate keys, etc. You can store raw binary secret as well, which by definition won't be in json format.




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

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



[GitHub] [airflow] mik-laj commented on pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10655:
URL: https://github.com/apache/airflow/pull/10655#issuecomment-686772675


   @potiuk I know you wrote hooks for GCP Secret Manager and Hashicorp Vault. What do you think about this change?


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

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



[GitHub] [airflow] dyon78 commented on a change in pull request #10655: [AIRFLOW-10645] Add AWS Secrets Manager Hook

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



##########
File path: airflow/providers/amazon/aws/hooks/secrets_manager.py
##########
@@ -0,0 +1,72 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import base64
+import json
+from typing import Union, Optional
+from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
+
+
+class SecretsManagerHook(AwsBaseHook):
+    """
+    Interact with Amazon SecretsManager Service.
+
+    Additional arguments (such as ``aws_conn_id``) may be specified and
+    are passed down to the underlying AwsBaseHook.
+
+    .. see also::
+        :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook`
+    """
+
+    def __init__(self, *args, **kwargs):
+        super().__init__(client_type='secretsmanager', *args, **kwargs)
+
+    def get_secrets(self, secret_name: str) -> Union[str, bytes]:

Review comment:
       base64.b64decode returns `bytes`, not `str` - we already have such type of discussion here and and I had to change it to `bytes`, docs about it - https://docs.python.org/3/library/base64.html.
   Applied another suggested changes




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

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