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/04/08 01:24:42 UTC

[GitHub] [airflow] kaxil opened a new pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

kaxil opened a new pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186
 
 
   Allow retrieving Airflow Connections and Variables from AWS Secrets Manager (https://aws.amazon.com/secrets-manager/)
   
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405435244
 
 

 ##########
 File path: airflow/providers/amazon/aws/secrets/secrets_manager.py
 ##########
 @@ -0,0 +1,129 @@
+#
+# 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.
+"""
+Objects relating to sourcing secrests from AWS Secrets Manager
 
 Review comment:
   ```suggestion
   Objects relating to sourcing secrets from 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


With regards,
Apache Git Services

[GitHub] [airflow] BasPH commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
BasPH commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#issuecomment-611004418
 
 
   @ashb 
   > AWS has more than one way of storing "secrets"? (I.e. the SSM backend is something else in AWS right?)
   
   Yea there's the secrets manager and the parameter store (under the systems manager service). Both do the same thing, but the secrets manager costs money to store secrets while the parameter store does not, so everybody uses the parameter store (as far as I know).

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil closed pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil closed pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186
 
 
   

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405434064
 
 

 ##########
 File path: airflow/providers/amazon/aws/secrets/secrets_manager.py
 ##########
 @@ -0,0 +1,129 @@
+#
+# 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.
+"""
+Objects relating to sourcing secrests from AWS Secrets Manager
+"""
+
+from typing import Optional
+
+import boto3
+from cached_property import cached_property
+
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection or Variables from AWS Secrets Manager
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, if secrets prefix is ``airflow/connections/smtp_default``, this would be accessible
+    if you provide ``{"connections_prefix": "airflow/connections"}`` and request conn_id ``smtp_default``.
+    And if variables prefix is ``airflow/variables/hello``, this would be accessible
+    if you provide ``{"variables_prefix": "airflow/variables"}`` and request variable key ``hello``.
+
+    You can also pass additional keyword arguments like ``aws_secret_access_key``, ``aws_access_key_id``
+    or ``region_name`` to this class and they would be passed on to Boto3 client.
+
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+    :type connections_prefix: str
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+    :type variables_prefix: str
+    :param profile_name: The name of a profile to use. If not given, then the default profile is used.
+    :type profile_name: str
+    :param sep: separator used to concatenate secret_prefix and secret_id. Default: "/"
+    :type sep: str
+    """
+
+    def __init__(
+        self,
+        connections_prefix: str = 'airflow/connections',
+        variables_prefix: str = 'airflow/variables',
+        profile_name: Optional[str] = None,
+        sep: str = "/",
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_prefix = connections_prefix.rstrip("/")
+        self.variables_prefix = variables_prefix.rstrip('/')
+        self.profile_name = profile_name
+        self.sep = sep
+        self.kwargs = kwargs
+
+    @cached_property
+    def client(self):
+        """
+        Create a Secrets Manager client
+        """
+        session = boto3.session.Session(
+            profile_name=self.profile_name,
+        )
+        return session.client(service_name="secretsmanager", **self.kwargs)
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get Connection Value
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self._get_secret(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> Optional[str]:
+        """
+        Get Airflow Variable from Environment Variable
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        return self._get_secret(self.variables_prefix, key)
+
+    def _get_secret(self, path_prefix: str, secret_id: str) -> Optional[str]:
+        """
+        Get secret value from Secrets Manager
+
+        :param path_prefix: Prefix for the Path to get Secret
+        :type path_prefix: str
+        :param secret_id: Secret Key
+        :type secret_id: str
+        """
+        secrets_path = self.build_path(path_prefix, secret_id, self.sep)
+        try:
+            response = self.client.get_secret_value(
+                SecretId=secrets_path,
+            )
+            if 'SecretString' in response:
+                secret_value = response['SecretString']
+            else:
+                secret_value = None
 
 Review comment:
   ```suggestion
               secret_value = response.get('SecretString')
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#issuecomment-610885788
 
 
   AWS has more than one way of storing "secrets"? (I.e. the SSM backend is something else in AWS right?)

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405432075
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -87,6 +87,69 @@ you would want to store your Variable at ``/airflow/variables/hello``.
 
 Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
 
+AWS Secrets Manager Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Secrets Manager, specify :py:class:`~airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+    backend_kwargs = {"connections_prefix": "airflow/connections", "variables_prefix": "airflow/variables", "profile_name": "default"}
+
+To authenticate you can either supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``, set
+environment variables like ``AWS_ACCESS_KEY_ID``, ``AWS_SECRET_ACCESS_KEY`` or pass those to the client as follows:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+    backend_kwargs = {"connections_prefix":"airflow/connections","aws_access_key_id": "AKI4Z78ADFSAFOZ3R",
 
 Review comment:
   Don't recommend this approach. We can leave the code to support it, but I think encouraging people to put these passwords in config files is bad.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405451542
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -87,6 +87,69 @@ you would want to store your Variable at ``/airflow/variables/hello``.
 
 Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
 
+AWS Secrets Manager Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Secrets Manager, specify :py:class:`~airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+    backend_kwargs = {"connections_prefix": "airflow/connections", "variables_prefix": "airflow/variables", "profile_name": "default"}
+
+To authenticate you can either supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``, set
+environment variables like ``AWS_ACCESS_KEY_ID``, ``AWS_SECRET_ACCESS_KEY`` or pass those to the client as follows:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+    backend_kwargs = {"connections_prefix":"airflow/connections","aws_access_key_id": "AKI4Z78ADFSAFOZ3R",
 
 Review comment:
   Updated

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405592574
 
 

 ##########
 File path: airflow/providers/amazon/aws/secrets/secrets_manager.py
 ##########
 @@ -0,0 +1,125 @@
+#
+# 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.
+"""
+Objects relating to sourcing secrets from AWS Secrets Manager
+"""
+
+from typing import Optional
+
+import boto3
+from cached_property import cached_property
+
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection or Variables from AWS Secrets Manager
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, if secrets prefix is ``airflow/connections/smtp_default``, this would be accessible
+    if you provide ``{"connections_prefix": "airflow/connections"}`` and request conn_id ``smtp_default``.
+    And if variables prefix is ``airflow/variables/hello``, this would be accessible
+    if you provide ``{"variables_prefix": "airflow/variables"}`` and request variable key ``hello``.
+
+    You can also pass additional keyword arguments like ``aws_secret_access_key``, ``aws_access_key_id``
+    or ``region_name`` to this class and they would be passed on to Boto3 client.
+
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+    :type connections_prefix: str
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+    :type variables_prefix: str
+    :param profile_name: The name of a profile to use. If not given, then the default profile is used.
+    :type profile_name: str
+    :param sep: separator used to concatenate secret_prefix and secret_id. Default: "/"
+    :type sep: str
+    """
+
+    def __init__(
+        self,
+        connections_prefix: str = 'airflow/connections',
+        variables_prefix: str = 'airflow/variables',
+        profile_name: Optional[str] = None,
+        sep: str = "/",
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_prefix = connections_prefix.rstrip("/")
+        self.variables_prefix = variables_prefix.rstrip('/')
+        self.profile_name = profile_name
+        self.sep = sep
+        self.kwargs = kwargs
+
+    @cached_property
+    def client(self):
+        """
+        Create a Secrets Manager client
+        """
+        session = boto3.session.Session(
+            profile_name=self.profile_name,
+        )
 
 Review comment:
   Actually no real reason except that it would allow adding new keyword arguments without showing much in "git diff"

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil merged pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186
 
 
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil opened a new pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil opened a new pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186
 
 
   Allow retrieving Airflow Connections and Variables from AWS Secrets Manager (https://aws.amazon.com/secrets-manager/)
   
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] BasPH commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
BasPH commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405580226
 
 

 ##########
 File path: airflow/providers/amazon/aws/secrets/secrets_manager.py
 ##########
 @@ -0,0 +1,125 @@
+#
+# 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.
+"""
+Objects relating to sourcing secrets from AWS Secrets Manager
+"""
+
+from typing import Optional
+
+import boto3
+from cached_property import cached_property
+
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection or Variables from AWS Secrets Manager
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, if secrets prefix is ``airflow/connections/smtp_default``, this would be accessible
+    if you provide ``{"connections_prefix": "airflow/connections"}`` and request conn_id ``smtp_default``.
+    And if variables prefix is ``airflow/variables/hello``, this would be accessible
+    if you provide ``{"variables_prefix": "airflow/variables"}`` and request variable key ``hello``.
+
+    You can also pass additional keyword arguments like ``aws_secret_access_key``, ``aws_access_key_id``
+    or ``region_name`` to this class and they would be passed on to Boto3 client.
+
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+    :type connections_prefix: str
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+    :type variables_prefix: str
+    :param profile_name: The name of a profile to use. If not given, then the default profile is used.
+    :type profile_name: str
+    :param sep: separator used to concatenate secret_prefix and secret_id. Default: "/"
+    :type sep: str
+    """
+
+    def __init__(
+        self,
+        connections_prefix: str = 'airflow/connections',
+        variables_prefix: str = 'airflow/variables',
+        profile_name: Optional[str] = None,
+        sep: str = "/",
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_prefix = connections_prefix.rstrip("/")
+        self.variables_prefix = variables_prefix.rstrip('/')
+        self.profile_name = profile_name
+        self.sep = sep
+        self.kwargs = kwargs
+
+    @cached_property
+    def client(self):
+        """
+        Create a Secrets Manager client
+        """
+        session = boto3.session.Session(
+            profile_name=self.profile_name,
+        )
 
 Review comment:
   Not related to the change, but why is it formatted like this? There's plenty of space to put it on a single line?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#discussion_r405431426
 
 

 ##########
 File path: airflow/providers/amazon/aws/secrets/secrets_manager.py
 ##########
 @@ -0,0 +1,129 @@
+#
+# 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.
+"""
+Objects relating to sourcing secrests from AWS Secrets Manager
+"""
+
+from typing import Optional
+
+import boto3
+from cached_property import cached_property
+
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection or Variables from AWS Secrets Manager
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, if secrets prefix is ``airflow/connections/smtp_default``, this would be accessible
+    if you provide ``{"connections_prefix": "airflow/connections"}`` and request conn_id ``smtp_default``.
+    And if variables prefix is ``airflow/variables/hello``, this would be accessible
+    if you provide ``{"variables_prefix": "airflow/variables"}`` and request variable key ``hello``.
+
+    You can also pass additional keyword arguments like ``aws_secret_access_key``, ``aws_access_key_id``
+    or ``region_name`` to this class and they would be passed on to Boto3 client.
+
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+    :type connections_prefix: str
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+    :type variables_prefix: str
+    :param profile_name: The name of a profile to use. If not given, then the default profile is used.
+    :type profile_name: str
+    :param sep: separator used to concatenate secret_prefix and secret_id. Default: "/"
+    :type sep: str
+    """
+
+    def __init__(
+        self,
+        connections_prefix: str = 'airflow/connections',
+        variables_prefix: str = 'airflow/variables',
+        profile_name: Optional[str] = None,
+        sep: str = "/",
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_prefix = connections_prefix.rstrip("/")
+        self.variables_prefix = variables_prefix.rstrip('/')
+        self.profile_name = profile_name
+        self.sep = sep
+        self.kwargs = kwargs
+
+    @cached_property
+    def client(self):
+        """
+        Create a Secrets Manager client
+        """
+        session = boto3.session.Session(
+            profile_name=self.profile_name,
+        )
+        return session.client(service_name="secretsmanager", **self.kwargs)
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get Connection Value
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self._get_secret(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> Optional[str]:
+        """
+        Get Airflow Variable from Environment Variable
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        return self._get_secret(self.variables_prefix, key)
+
+    def _get_secret(self, path_prefix: str, secret_id: str) -> Optional[str]:
+        """
+        Get secret value from Secrets Manager
+
+        :param path_prefix: Prefix for the Path to get Secret
+        :type path_prefix: str
+        :param secret_id: Secret Key
+        :type secret_id: str
+        """
+        secrets_path = self.build_path(path_prefix, secret_id, self.sep)
+        try:
+            response = self.client.get_secret_value(
+                SecretId=secrets_path,
+            )
+            if 'SecretString' in response:
+                secret_value = response['SecretString']
+            else:
+                secret_value = None
+            return secret_value
+        except self.client.exceptions.ResourceNotFoundException:
+            self.log.info(
 
 Review comment:
   Do we want to log this every time? It seems overkill at info level. _Maybe_.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #8186: Add support for AWS Secrets Manager as Secrets Backend
URL: https://github.com/apache/airflow/pull/8186#issuecomment-610881086
 
 
   CI failure is unrelated

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


With regards,
Apache Git Services