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/03/16 23:24:36 UTC

[GitHub] [airflow] kaxil opened a new pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secret Backend

kaxil opened a new pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secret Backend
URL: https://github.com/apache/airflow/pull/7741
 
 
   Add support for HashiCorp Vault as Secret Backend
   
   ---
   Issue link: WILL BE INSERTED BY [boring-cyborg](https://github.com/kaxil/boring-cyborg)
   
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Commit message/PR title starts with `[AIRFLOW-NNNN]`. AIRFLOW-NNNN = JIRA ID<sup>*</sup>
   - [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).
   
   <sup>*</sup> For document-only changes commit message can start with `[AIRFLOW-XXXX]`.
   
   ---
   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] kaxil commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secret Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secret Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599802092
 
 
   cc @dstandish 

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393376426
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,156 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github'). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    """
+    def __init__(
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
 
 Review comment:
   Can you also add GCP? This is a very often used cloud provider for Airflow.  We have operators for [31 Google service](https://airflow.readthedocs.io/en/latest/operators-and-hooks-ref.html#id13). The code will look similar to the following.
   ```python
       elif self.auth_type == "gcp":
           credentials = self.get_gcp_credentialss()
           self._client.auth.gcp.configure(credentials=credentials, mount_point=self.mount_point):
       else:
   
   
   
   def get_gcp_credentialss():
       if self.gcp_key_path:
           # Get credentials from a JSON file.
           if key_path.endswith('.json'):
               self.log.debug('Getting connection using JSON key file %s', key_path)
               credentials = (
                   google.oauth2.service_account.Credentials.from_service_account_file(
                       key_path, scopes=self.scopes)
               )
           elif key_path.endswith('.p12'):
               raise AirflowException(
                   'Legacy P12 key file are not supported, use a JSON key file.'
               )
           else:
               raise AirflowException('Unrecognised extension for key file.')
       else:
           self.log.debug(
               'Getting connection using `google.auth.default()` since no key file is defined.'
           )
           credentials, _ = google.auth.default(scopes=self.scopes)
   
       return credentials
   ```
   
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394033810
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   Do we need to do that though? I would suggest changing the `get_conn_uri` method on VaultSecrets or AwsSsmBackend to a generic `get_secret` method as follows:
   
   ```python
       def get_secret(
           self, secret_key: str, path: Optional[str] = None, mount_point: Optional[str] = None
       ) -> Optional[str]:
           """
           Get secret value from Vault
   
           :param secret_key: Key of the Secret to retrieve
           :type secret_key: str
           :param path: Path of the secret to read
           :type path: str
           :param mount_point: The "path" the secret engine was mounted on
           :type mount_point: str
           """
           path = path if path else self.path
           mount_point = mount_point if mount_point else self.mount_point
   
           self.log.debug("Path: %s", path)
           self.log.debug("Mount Point: %s", mount_point)
           self.log.debug("Retrieving the secret for the key: %s", secret_key)
   
           if self.kv_engine_version == 1:
               response = self.client.secrets.kv.v1.read_secret(
                   path=self.path, mount_point=self.mount_point
               )
           else:
               response = self.client.secrets.kv.v2.read_secret_version(
                   path=self.path, mount_point=self.mount_point)
   
           if not response:
               return None
   
           return_data = response["data"] if self.kv_engine_version == 1 else response["data"]["data"]
           return return_data.get(secret_key)
   ```
   
   This method would allow overwriting the **path** and **mount_point** and hence we can use this method in Operator/xcom
   
   I think we can figure out later when we actually implement that generic method.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393601898
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default Airflow will search environment variables first and metastore
+database second.
+
+If you enable an alternative secrets backend, it will be searched first, followed by environment variables,
+then metastore.  This search ordering is not configurable.
+
+.. _secrets_backend_configuration:
+
+Configuration
+^^^^^^^^^^^^^
+
+The ``[secrets]`` section has the following options:
+
+.. code-block:: ini
+
+    [secrets]
+    backend =
+    backend_kwargs =
+
+Set ``backend`` to the fully qualified class name of the backend you want to enable.
+
+You can provide ``backend_kwargs`` with json and it will be passed as kwargs to the ``__init__`` method of
+your secrets backend.
+
+See :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>` for an example configuration.
+
+.. _ssm_parameter_store_secrets:
+
+AWS SSM Parameter Store Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend
+    backend_kwargs = {"prefix": "/airflow", "profile_name": "default"}
+
+If you have set your prefix as ``/airflow``, then for a connection id of ``smtp_default``, you would want to
+store your connection at ``/airflow/AIRFLOW_CONN_SMTP_DEFAULT``.
+
+Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
+
+The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
+
+.. _hashicorp_vault_secrets:
+
+Hashicorp Vault Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Hashicorp vault to retrieve connection, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
 
 Review comment:
   This is normal for vault I guess? (i.e. no `AIRFLOW_` prefix here because this is expected?)

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  because raising an exception when not found is probably more intuitive than returning an empty list from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393602738
 
 

 ##########
 File path: setup.py
 ##########
 @@ -551,6 +554,7 @@ def do_setup():
             'grpc': grpc,
             'hdfs': hdfs,
             'hive': hive,
+            'hvac': hvac,
 
 Review comment:
   This name isn't very descriptive if you don't already know what the module acronym does (and I can't see any docs) about extras? How about calling this extra `hasicorp`?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394019612
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default Airflow will search environment variables first and metastore
+database second.
+
+If you enable an alternative secrets backend, it will be searched first, followed by environment variables,
+then metastore.  This search ordering is not configurable.
+
+.. _secrets_backend_configuration:
+
+Configuration
+^^^^^^^^^^^^^
+
+The ``[secrets]`` section has the following options:
+
+.. code-block:: ini
+
+    [secrets]
+    backend =
+    backend_kwargs =
+
+Set ``backend`` to the fully qualified class name of the backend you want to enable.
+
+You can provide ``backend_kwargs`` with json and it will be passed as kwargs to the ``__init__`` method of
+your secrets backend.
+
+See :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>` for an example configuration.
+
+.. _ssm_parameter_store_secrets:
+
+AWS SSM Parameter Store Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend
+    backend_kwargs = {"prefix": "/airflow", "profile_name": "default"}
+
+If you have set your prefix as ``/airflow``, then for a connection id of ``smtp_default``, you would want to
+store your connection at ``/airflow/AIRFLOW_CONN_SMTP_DEFAULT``.
+
+Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
+
+The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
+
+.. _hashicorp_vault_secrets:
+
+Hashicorp Vault Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Hashicorp vault to retrieve connection, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
+key to ``backend_kwargs``:
+
+.. code-block:: bash
+
+    export VAULT_ADDR="http://127.0.0.1:8200"
+
+If you have set your path as ``airflow/connections``, then for a connection id of ``smtp_default``, you would want to
+store your secret as:
+
+.. code-block:: bash
+
+    vault kv put secret/airflow/connections smtp_default=postgresql://airflow:airflow@host:5432/airflow
+
+The value of the Vault key must be the :ref:`airflow connection URI representation <generating_connection_uri>`
 
 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] ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394657351
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Right now this seems to be _all_ connections in a single path, with different values for each of them. I would have expected that each connection is it's own secret instead.
   
   (I'm guessing Vault lets you limit access based on path, but not on key?)

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395308823
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
+        elif self.auth_type == "ldap":
+            _client.auth.ldap.login(
+                username=self.username, password=self.password)
+        elif self.auth_type == "userpass":
+            _client.auth_userpass(username=self.username, password=self.password)
+        elif self.auth_type == "approle":
+            _client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+        elif self.auth_type == "github":
+            _client.auth.github.login(token=self.token)
+        elif self.auth_type == "gcp":
+            credentials = self._get_gcp_credentials()
+            _client.auth.gcp.configure(credentials=credentials)
+        else:
+            raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+        if _client.is_authenticated():
+            return _client
+        else:
+            raise VaultError("Vault Authentication Error!")
+
+    def build_path(self, conn_id: str):
+        """
+        Given conn_id, build path for Vault Secret
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self.connections_path + "/" + conn_id
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        self.log.debug("Path: %s", self.connections_path)
+        self.log.debug("Mount Point: %s", self.mount_point)
 
 Review comment:
   Removed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393976739
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   With the example as @kaxil has it in the docs it's already "namespaced" as `airflow/connections/`

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393559730
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default airflow will search environment variables first and metastore
 
 Review comment:
   Fixed

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214?src=pr&el=desc) will **decrease** coverage by `0.34%`.
   > The diff coverage is `68%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   86.59%   -0.35%     
   ==========================================
     Files         915      916       +1     
     Lines       44172    44262      +90     
   ==========================================
   - Hits        38403    38328      -75     
   - Misses       5769     5934     +165
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/amazon/aws/hooks/s3.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9zMy5weQ==) | `96.52% <100%> (+0.06%)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `60.81% <60.81%> (ø)` | |
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <83.33%> (+7.93%)` | :arrow_up: |
   | [airflow/providers/amazon/aws/secrets/ssm.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9zZWNyZXRzL3NzbS5weQ==) | `94.28% <84.61%> (+27.61%)` | :arrow_up: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/security/kerberos.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWN1cml0eS9rZXJiZXJvcy5weQ==) | `30.43% <0%> (-45.66%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0%> (-45.08%)` | :arrow_down: |
   | [...viders/cncf/kubernetes/operators/kubernetes\_pod.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvY25jZi9rdWJlcm5ldGVzL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZC5weQ==) | `69.69% <0%> (-25.26%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0%> (-23.53%)` | :arrow_down: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...78f3938](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394000355
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   > Or we could change the prefix argument to connections_prefix now to future proof it a bit?
   
   I think that seems like a good idea, unless you want  connection to merely be a special case of secret, i.e. in same "namespace".
   
   so e.g. ...
   if ssm param is `/airflow/smtp_default=my://uri` and prefix is `/airflow`, then `get_secret` pulls the raw value `my://uri` and get_connections pulls the same raw value _and_ tries to parse the URI and load as connection....
   
   if i had to choose, i would probably say, do as you've suggested -- put them on different prefixes, and change to `connections_prefix` 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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393601194
 
 

 ##########
 File path: airflow/secrets/__init__.py
 ##########
 @@ -50,7 +51,7 @@ def __init__(self, **kwargs):
         pass
 
     @abstractmethod
-    def get_connections(self, conn_id) -> List[Connection]:
+    def get_connections(self, conn_id) -> Optional[List[Connection]]:
 
 Review comment:
   I don't think this should be changed -- return an empty list instead of none?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395298801
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
 
 Review comment:
   We should validat token is not None -- it lead to some hard to debug issues when I was testing 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


With regards,
Apache Git Services

[GitHub] [airflow] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394000355
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   > Or we could change the prefix argument to connections_prefix now to future proof it a bit?
   
   I think that seems like a good idea, unless you want  connection to merely be a special case of secret, i.e. in same "namespace".
   
   so e.g. ...
   if ssm param is `/airflow/smtp_default=my://uri` and prefix is `/airflow`, then `get_secret` pulls the raw value `my://uri` and get_connections pulls the same raw value _and_ tries to parse the URI and load as connection....
   
   if i had to choose, i would probably say, do as you've suggested and change to `connections_prefix` 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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395308906
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
 
 Review comment:
   Updated and added test

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393471028
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
 
 Review comment:
   do we need to add these docstring type hints now that we are python 3?

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214&el=desc) will **decrease** coverage by `0.34%`.
   > The diff coverage is `68.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   86.59%   -0.35%     
   ==========================================
     Files         915      916       +1     
     Lines       44172    44262      +90     
   ==========================================
   - Hits        38403    38328      -75     
   - Misses       5769     5934     +165     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `60.81% <60.81%> (ø)` | |
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <83.33%> (+7.93%)` | :arrow_up: |
   | [airflow/providers/amazon/aws/secrets/ssm.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9zZWNyZXRzL3NzbS5weQ==) | `94.28% <84.61%> (+27.61%)` | :arrow_up: |
   | [airflow/providers/amazon/aws/hooks/s3.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9zMy5weQ==) | `96.52% <100.00%> (+0.06%)` | :arrow_up: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/security/kerberos.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWN1cml0eS9rZXJiZXJvcy5weQ==) | `30.43% <0.00%> (-45.66%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0.00%> (-45.08%)` | :arrow_down: |
   | [...viders/cncf/kubernetes/operators/kubernetes\_pod.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvY25jZi9rdWJlcm5ldGVzL29wZXJhdG9ycy9rdWJlcm5ldGVzX3BvZC5weQ==) | `69.69% <0.00%> (-25.26%)` | :arrow_down: |
   | [airflow/kubernetes/refresh\_config.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3JlZnJlc2hfY29uZmlnLnB5) | `50.98% <0.00%> (-23.53%)` | :arrow_down: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...78f3938](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395299828
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
+        elif self.auth_type == "ldap":
+            _client.auth.ldap.login(
+                username=self.username, password=self.password)
+        elif self.auth_type == "userpass":
+            _client.auth_userpass(username=self.username, password=self.password)
+        elif self.auth_type == "approle":
+            _client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+        elif self.auth_type == "github":
+            _client.auth.github.login(token=self.token)
+        elif self.auth_type == "gcp":
+            credentials = self._get_gcp_credentials()
+            _client.auth.gcp.configure(credentials=credentials)
+        else:
+            raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+        if _client.is_authenticated():
+            return _client
+        else:
+            raise VaultError("Vault Authentication Error!")
+
+    def build_path(self, conn_id: str):
+        """
+        Given conn_id, build path for Vault Secret
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self.connections_path + "/" + conn_id
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        self.log.debug("Path: %s", self.connections_path)
+        self.log.debug("Mount Point: %s", self.mount_point)
 
 Review comment:
   Do we need to log this everytime (even if it is debug it still seems verbose. to have it on every secret fetch 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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r406725554
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
 
 Review comment:
   The last time I tried, there were some errors but I didn't dig into it.
   
   I will try again today/tomorrow.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
BasPH commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393490521
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
 
 Review comment:
   I don't think it's good practice to log the secret value.

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393509505
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
 
 Review comment:
   Yes. These are for rendered docs - sphinx doesn't currently pick up type hints sadly

----------------------------------------------------------------
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] mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393372282
 
 

 ##########
 File path: docs/howto/connection/index.rst
 ##########
 @@ -179,6 +179,41 @@ Optionally you can supply a profile name to reference aws profile, e.g. defined
 
 The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
 
+.. _hashicorp_vault_secrets:
+
+Hashicorp Value Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
+key to ``backend_kwargs``:
+
+.. code-block:: bash
+
+    export VAULT_ADDR="http://127.0.0.1:8200"
+
+If you have set your path as ``airflow``, then for a connection id of ``smtp_default``, you would want to
 
 Review comment:
   Before adding information about a secret backend on this page, we only had the configuration described, which can be changed during runtime. This was information that a normal Airflow user used (User guide). We now also have information that is very specific and is not used by normal users during everyday use (Administrator Guide). Would you section this information on a new page? This page should only contain references to the new page. 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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-600776150
 
 
   > @kaxil you added `airflow/providers/hashicorp/secrets/vault.py`
   > say that in future there will be a `HashicorpVaultHook`. Where would that go?
   > https://issues.apache.org/jira/browse/AIRFLOW-3669
   > 
   > Maybe I'm getting this wrong but I thought the providers package is for hooks/operators/sensor ?
   
   That would go under `airflow/providers/hashicorp/operator/`
   
   providers packages are for everything related to that specific provider

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


With regards,
Apache Git Services

[GitHub] [airflow] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393967490
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   for background, the thinking was just to keep things uniform between env vars and creds store so you could copy paste and not think about it or have to change the keys in text editor. 
   
   but this obvs has benefit of readability and simplicity, when looking at the creds store in isolation alone.
   
   best to be consistent in any case...
   
   one little thing... if @potiuk's idea materializes -- generic arbitrary secret...  would a prefix be helpful to distinguish between types?  or would it just pull with same convention / prefix, get secret is raw value, get connection pulls same key but attempts to parse as connection....

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393532739
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   Good point, will update

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214?src=pr&el=desc) will **decrease** coverage by `22.34%`.
   > The diff coverage is `64.63%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7741       +/-   ##
   ===========================================
   - Coverage   86.93%   64.59%   -22.35%     
   ===========================================
     Files         915      915               
     Lines       44172    44249       +77     
   ===========================================
   - Hits        38403    28581     -9822     
   - Misses       5769    15668     +9899
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/amazon/aws/hooks/s3.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9zMy5weQ==) | `96.52% <100%> (+0.06%)` | :arrow_up: |
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `93.02% <100%> (+3.27%)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `60.81% <60.81%> (ø)` | |
   | [...low/contrib/operators/wasb\_delete\_blob\_operator.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy93YXNiX2RlbGV0ZV9ibG9iX29wZXJhdG9yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/hooks/vertica\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2hvb2tzL3ZlcnRpY2FfaG9vay5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvX19pbml0X18ucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/mssql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9tc3NxbF9ob29rLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [...viders/docker/example\_dags/example\_docker\_swarm.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZG9ja2VyL2V4YW1wbGVfZGFncy9leGFtcGxlX2RvY2tlcl9zd2FybS5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/webhdfs\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy93ZWJoZGZzX2hvb2sucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/emr\_base\_sensor.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvZW1yX2Jhc2Vfc2Vuc29yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | ... and [496 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...78f3938](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393967490
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   the thinking was just to keep things uniform between env vars and creds store so you could copy paste and not think about it or have to change the keys in text editor. 
   
   but this obvs has benefit of readability and simplicity, when looking at the creds store in isolation alone.
   
   one little thing... if @potiuk's idea materializes -- generic arbitrary secret...  would a prefix be helpful to distinguish between types?  or would it just pull with same convention / prefix, get secret is raw value, get connection pulls same key but attempts to parse as connection....

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then airflow will not continue to search env vars and metastore (as is the documented behavior); it will just stop here and raise.
   
   see [get_connections](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62) function.
   
   to preserve expected behavior re search path, when no value is found, the instance method `get_connections` has to return an empty list.
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394266851
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,202 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
 
 Review comment:
   Fixed

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/029c84e5527b6db6bdbdbe026f455da325bedef3&el=desc) will **decrease** coverage by `0.98%`.
   > The diff coverage is `62.33%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.96%   85.98%   -0.99%     
   ==========================================
     Files         920      921       +1     
     Lines       44436    44513      +77     
   ==========================================
   - Hits        38644    38273     -371     
   - Misses       5792     6240     +448     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <ø> (ø)` | |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `62.33% <62.33%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0.00%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/api/auth/backend/kerberos\_auth.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9hcGkvYXV0aC9iYWNrZW5kL2tlcmJlcm9zX2F1dGgucHk=) | `28.16% <0.00%> (-54.93%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | ... and [18 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [029c84e...e0932a1](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394656677
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   `test_postgres` here is in the duplicated in the key name isn't it? To make it easy to support storing in other forms (such as a k/v per attribute, i.e. not needing to URI parse) we could store that as 
   ```suggestion
                   'data': {'conn_uri': 'postgresql://airflow:airflow@host:5432/airflow'},
   ```
   
   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


With regards,
Apache Git Services

[GitHub] [airflow] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r406643318
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
 
 Review comment:
   hey... i just stumbled onto this https://pypi.org/project/sphinx-autodoc-typehints/
   
   > This extension allows you to use Python 3 annotations for documenting acceptable argument types and return value types of functions. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394655248
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "connections_path": "airflow/connections",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        returned_uri = test_client.get_conn_uri(conn_id="test_postgres")
 
 Review comment:
   Is it worth adding an assertion to check that `mock_client.secrets.kv.v2.read_secret_version` is called with args we expect (path, secret name etc.)

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393603600
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
 
 Review comment:
   ```suggestion
       Retrieves Connection object from Hashicorp Vault
   ```

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/029c84e5527b6db6bdbdbe026f455da325bedef3?src=pr&el=desc) will **decrease** coverage by `22.33%`.
   > The diff coverage is `70.65%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7741       +/-   ##
   ===========================================
   - Coverage   86.96%   64.62%   -22.34%     
   ===========================================
     Files         920      920               
     Lines       44436    44455       +19     
   ===========================================
   - Hits        38644    28730     -9914     
   - Misses       5792    15725     +9933
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `93.02% <ø> (-4.66%)` | :arrow_down: |
   | [...oviders/google/cloud/operators/natural\_language.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9uYXR1cmFsX2xhbmd1YWdlLnB5) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/providers/google/cloud/hooks/automl.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2F1dG9tbC5weQ==) | `88.52% <ø> (-1.03%)` | :arrow_down: |
   | [...w/providers/google/cloud/hooks/natural\_language.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL25hdHVyYWxfbGFuZ3VhZ2UucHk=) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/providers/google/suite/hooks/sheets.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL3N1aXRlL2hvb2tzL3NoZWV0cy5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [...oogle/marketing\_platform/hooks/campaign\_manager.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9ob29rcy9jYW1wYWlnbl9tYW5hZ2VyLnB5) | `88.88% <ø> (-0.35%)` | :arrow_down: |
   | [...rflow/providers/google/cloud/hooks/bigquery\_dts.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2JpZ3F1ZXJ5X2R0cy5weQ==) | `79.31% <ø> (-1.34%)` | :arrow_down: |
   | [...providers/google/cloud/operators/text\_to\_speech.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy90ZXh0X3RvX3NwZWVjaC5weQ==) | `100% <ø> (ø)` | :arrow_up: |
   | [airflow/providers/google/cloud/operators/gcs.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9nY3MucHk=) | `85.71% <ø> (ø)` | :arrow_up: |
   | [...ers/google/cloud/example\_dags/example\_cloud\_sql.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2V4YW1wbGVfZGFncy9leGFtcGxlX2Nsb3VkX3NxbC5weQ==) | `0% <ø> (-100%)` | :arrow_down: |
   | ... and [528 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [029c84e...e39f971](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393993686
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   Ohhh good point. Let's address that in the future I think?
   
   Or we could change the `prefix` argument to `connections_prefix` now to future proof it a bit?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393402748
 
 

 ##########
 File path: docs/howto/connection/index.rst
 ##########
 @@ -179,6 +179,41 @@ Optionally you can supply a profile name to reference aws profile, e.g. defined
 
 The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
 
+.. _hashicorp_vault_secrets:
+
+Hashicorp Value Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
+key to ``backend_kwargs``:
+
+.. code-block:: bash
+
+    export VAULT_ADDR="http://127.0.0.1:8200"
+
+If you have set your path as ``airflow``, then for a connection id of ``smtp_default``, you would want to
 
 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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393957584
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   I had a talk with @dstandish and he agrees we should update AWS SSM. He is busy today, so I might make the change, if I don't get to it today, he might do it tomorrow.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393618414
 
 

 ##########
 File path: airflow/secrets/__init__.py
 ##########
 @@ -50,7 +51,7 @@ def __init__(self, **kwargs):
         pass
 
     @abstractmethod
-    def get_connections(self, conn_id) -> List[Connection]:
+    def get_connections(self, conn_id) -> Optional[List[Connection]]:
 
 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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394664957
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   So all the connections would live in a single path `airflow/connections` and it accepts a key_value store.
   
   Key I think should be **conn_id**.
   
   Storing them as following:
   
   ```
   vault kv put airflow/connections conn_type=postgresql user=airflow password=airflow host=host port=5432 schema=airflow
   ```
   
   would create separate keys:
   
   ```
   ❯ vault kv put airflow/connections conn_type=mysql user=airflow password=airflow host=host port=5432 schema=airflow
   
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   ❯ vault kv get airflow/connections
   ====== Metadata ======
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   
   ====== Data ======
   Key          Value
   ---          -----
   conn_type    mysql
   host         host
   password     airflow
   port         5432
   schema       airflow
   user         airflow
   ```
   
   which doesn't make sense as they are not linked to each other. I think what we have currently is fine and allows us to retrieve all the connections in a path.

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394658516
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   `conn_uri` looks redundant given we are already in `connections` path.
   
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393381763
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,156 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github'). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    """
+    def __init__(
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
 
 Review comment:
   Sure

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/029c84e5527b6db6bdbdbe026f455da325bedef3?src=pr&el=desc) will **decrease** coverage by `0.83%`.
   > The diff coverage is `62.33%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.96%   86.13%   -0.84%     
   ==========================================
     Files         920      921       +1     
     Lines       44436    44513      +77     
   ==========================================
   - Hits        38644    38341     -303     
   - Misses       5792     6172     +380
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <ø> (ø)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `62.33% <62.33%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50% <0%> (-50%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50% <0%> (-50%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0%> (-46.67%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0%> (-45.08%)` | :arrow_down: |
   | ... and [13 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [029c84e...e0932a1](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393972573
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,192 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/connections``, this would be accessible if you
+    provide ``{"path": "airflow/connections"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
 
 Review comment:
   ```suggestion
           super().__init__(**kwrgs)
   ```

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395294730
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Agree with what you have suggested on further reading and deep dive.
   
   Updated: https://github.com/apache/airflow/pull/7741/commits/2311f937bcf14899314fe7bebf073da254f41d02

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394929982
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   By having them all in a single path does it restrict the ability to apply different policies to each key?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393559672
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
 
 Review comment:
   Fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393618459
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
 
 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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i think airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393971110
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   no i agree but what i mean is like it could be like `AIRFLOW_SECRET_BLAH` and `AIRFLOW_CONN_BLAH` in theory.... and allow it to work on env vars too... and both could coexist at same prefix `/airflow`....
   
   alternative could be that get secret and get connection pull from same `AIRFLOW_CONN_{}` in env vars, and just one tries to parse as connection, one does not.
   
   just a thought, a hypothetical
   

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214?src=pr&el=desc) will **decrease** coverage by `22.34%`.
   > The diff coverage is `64.63%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7741       +/-   ##
   ===========================================
   - Coverage   86.93%   64.59%   -22.35%     
   ===========================================
     Files         915      915               
     Lines       44172    44249       +77     
   ===========================================
   - Hits        38403    28581     -9822     
   - Misses       5769    15668     +9899
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/amazon/aws/hooks/s3.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9zMy5weQ==) | `96.52% <100%> (+0.06%)` | :arrow_up: |
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `93.02% <100%> (+3.27%)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `60.81% <60.81%> (ø)` | |
   | [...low/contrib/operators/wasb\_delete\_blob\_operator.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy93YXNiX2RlbGV0ZV9ibG9iX29wZXJhdG9yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/hooks/vertica\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2hvb2tzL3ZlcnRpY2FfaG9vay5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvX19pbml0X18ucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/mssql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9tc3NxbF9ob29rLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [...viders/docker/example\_dags/example\_docker\_swarm.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZG9ja2VyL2V4YW1wbGVfZGFncy9leGFtcGxlX2RvY2tlcl9zd2FybS5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/webhdfs\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy93ZWJoZGZzX2hvb2sucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/emr\_base\_sensor.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvZW1yX2Jhc2Vfc2Vuc29yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | ... and [496 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...78f3938](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search, first in env vars, then in metastore (as is the documented behavior); it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it (have to update the other backends), and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  because raising an exception when not found is probably more intuitive than returning an empty list, from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393971110
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   no i agree but what i mean is like it could be like `AIRFLOW_SECRET_BLAH` and `AIRFLOW_CONN_BLAH` in theory.... and allow it to work on env vars too... and both could coexist at same prefix `/airflow`....
   
   alternative could be that get secret and get connection pull from same `AIRFLOW_CONN_{}` in env vars, and just one tries to parse as connection, one does not.
   
   just a thought, a hypothetical
   
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393600514
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
 
 Review comment:
   ```suggestion
       @cached_property
       def client(self) -> hvac.Client:
   ```
   
   And then we don't need `self._client` or the `if` in this fn.

----------------------------------------------------------------
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] mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393376426
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,156 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github'). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    """
+    def __init__(
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
 
 Review comment:
   Can you also add GCP? This is a very often used cloud provider for Airflow.  We have operators for [31 Google service](https://airflow.readthedocs.io/en/latest/operators-and-hooks-ref.html#id13). The code will look similar to the following.
   ```
       elif self.auth_type == "gcp":
           credentials = self.get_gcp_credentialss()
           self._client.auth.gcp.configure(credentials=credentials, mount_point=self.mount_point):
       else:
   
   
   
   def get_gcp_credentialss():
       if self.gcp_key_path:
           # Get credentials from a JSON file.
           if key_path.endswith('.json'):
               self.log.debug('Getting connection using JSON key file %s', key_path)
               credentials = (
                   google.oauth2.service_account.Credentials.from_service_account_file(
                       key_path, scopes=self.scopes)
               )
           elif key_path.endswith('.p12'):
               raise AirflowException(
                   'Legacy P12 key file are not supported, use a JSON key file.'
               )
           else:
               raise AirflowException('Unrecognised extension for key file.')
       else:
           self.log.debug(
               'Getting connection using `google.auth.default()` since no key file is defined.'
           )
           credentials, _ = google.auth.default(scopes=self.scopes)
   
       return credentials
   ```
   
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393652095
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+from airflow.secrets import get_connections
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        returned_uri = test_client.get_conn_uri(conn_id="test_postgres")
+        self.assertEqual('postgresql://airflow:airflow@host:5432/airflow', returned_uri)
+
+    @mock.patch.dict('os.environ', {
+        'AIRFLOW_CONN_TEST_MYSQL': 'mysql://airflow:airflow@host:5432/airflow',
+    })
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri_non_existent_key(self, mock_hvac):
+        """
+        Test that if the key with connection ID is not present in Vault, VaultClient.get_connections
+        should return None and fallback to the environment variable if it is set
+        """
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        # Response does not contain the requested key
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        self.assertIsNone(test_client.get_conn_uri(conn_id="test_mysql"))
+        self.assertIsNone(test_client.get_connections(conn_id="test_mysql"))
+
+        # Fallback to connection defined in Environment Variable
 
 Review comment:
   Fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394661300
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "connections_path": "airflow/connections",
 
 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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394019548
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,192 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/connections``, this would be accessible if you
+    provide ``{"path": "airflow/connections"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
 
 Review comment:
   Fixed

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


With regards,
Apache Git Services

[GitHub] [airflow] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393967490
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   for background, the thinking was just to keep things uniform between env vars and creds store so you could copy paste and not think about it or have to change the keys in text editor. 
   
   but this obvs just using the conn id has benefit of readability and simplicity, when looking at the creds store in isolation alone.
   
   best to be consistent in any case...
   
   one little thing... if @potiuk's idea materializes -- generic arbitrary secret...  would a prefix be helpful to distinguish between types?  or would it just pull with same convention / prefix, get secret is raw value, get connection pulls same key but attempts to parse as connection....

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214&el=desc) will **decrease** coverage by `0.78%`.
   > The diff coverage is `60.81%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   86.15%   -0.79%     
   ==========================================
     Files         915      916       +1     
     Lines       44172    44262      +90     
   ==========================================
   - Hits        38403    38134     -269     
   - Misses       5769     6128     +359     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <ø> (+7.93%)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `60.81% <60.81%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0.00%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0.00%> (-45.08%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...78f3938](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393956556
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,192 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
 
 Review comment:
   This is out of date now I think.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393604138
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   Why do we have this if we have a path already? This seems a bit odd -- for instance I'd expect tit to be something like `airflow/connections/mysql_default`.

----------------------------------------------------------------
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] codecov-io commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214&el=desc) will **decrease** coverage by `0.77%`.
   > The diff coverage is `77.58%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   86.15%   -0.78%     
   ==========================================
     Files         915      916       +1     
     Lines       44172    44234      +62     
   ==========================================
   - Hits        38403    38112     -291     
   - Misses       5769     6122     +353     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `77.58% <77.58%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0.00%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0.00%> (-45.08%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | ... and [12 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...1995fe8](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394661262
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "connections_path": "airflow/connections",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        returned_uri = test_client.get_conn_uri(conn_id="test_postgres")
 
 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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  raising an exception when not found is probably more intuitive from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393628852
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default Airflow will search environment variables first and metastore
+database second.
+
+If you enable an alternative secrets backend, it will be searched first, followed by environment variables,
+then metastore.  This search ordering is not configurable.
+
+.. _secrets_backend_configuration:
+
+Configuration
+^^^^^^^^^^^^^
+
+The ``[secrets]`` section has the following options:
+
+.. code-block:: ini
+
+    [secrets]
+    backend =
+    backend_kwargs =
+
+Set ``backend`` to the fully qualified class name of the backend you want to enable.
+
+You can provide ``backend_kwargs`` with json and it will be passed as kwargs to the ``__init__`` method of
+your secrets backend.
+
+See :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>` for an example configuration.
+
+.. _ssm_parameter_store_secrets:
+
+AWS SSM Parameter Store Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend
+    backend_kwargs = {"prefix": "/airflow", "profile_name": "default"}
+
+If you have set your prefix as ``/airflow``, then for a connection id of ``smtp_default``, you would want to
+store your connection at ``/airflow/AIRFLOW_CONN_SMTP_DEFAULT``.
+
+Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
+
+The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
+
+.. _hashicorp_vault_secrets:
+
+Hashicorp Vault Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Hashicorp vault to retrieve connection, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
 
 Review comment:
   This paragraph explains how you can pass values to the Vault client. i.e. by setting Environment Variables for Vault.
   
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393603370
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+from airflow.secrets import get_connections
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        returned_uri = test_client.get_conn_uri(conn_id="test_postgres")
+        self.assertEqual('postgresql://airflow:airflow@host:5432/airflow', returned_uri)
+
+    @mock.patch.dict('os.environ', {
+        'AIRFLOW_CONN_TEST_MYSQL': 'mysql://airflow:airflow@host:5432/airflow',
+    })
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri_non_existent_key(self, mock_hvac):
+        """
+        Test that if the key with connection ID is not present in Vault, VaultClient.get_connections
+        should return None and fallback to the environment variable if it is set
+        """
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        # Response does not contain the requested key
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        self.assertIsNone(test_client.get_conn_uri(conn_id="test_mysql"))
+        self.assertIsNone(test_client.get_connections(conn_id="test_mysql"))
+
+        # Fallback to connection defined in Environment Variable
 
 Review comment:
   Isn't this test (re)testing the base secrets interface?

----------------------------------------------------------------
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] mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393376426
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,156 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github'). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    """
+    def __init__(
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
 
 Review comment:
   Can you also add GCP? This is a very often used cloud provider for Airflow.  We have operators for [31 Google service](https://airflow.readthedocs.io/en/latest/operators-and-hooks-ref.html#id13). The code will look similar to the following.
   ```python
       elif self.auth_type == "gcp":
           credentials = self.get_gcp_credentialss()
           self._client.auth.gcp.configure(credentials=credentials):
       else:
   
   
   
   def get_gcp_credentialss():
       if self.gcp_key_path:
           # Get credentials from a JSON file.
           if key_path.endswith('.json'):
               self.log.debug('Getting connection using JSON key file %s', key_path)
               credentials = (
                   google.oauth2.service_account.Credentials.from_service_account_file(
                       key_path, scopes=self.scopes)
               )
           elif key_path.endswith('.p12'):
               raise AirflowException(
                   'Legacy P12 key file are not supported, use a JSON key file.'
               )
           else:
               raise AirflowException('Unrecognised extension for key file.')
       else:
           self.log.debug(
               'Getting connection using `google.auth.default()` since no key file is defined.'
           )
           credentials, _ = google.auth.default(scopes=self.scopes)
   
       return credentials
   ```
   
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore (as is the documented behavior); it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  because raising an exception when not found is probably more intuitive than returning an empty list from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore (as is the documented behavior); it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it (have to update the other backends), and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  because raising an exception when not found is probably more intuitive than returning an empty list, from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394658547
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Yes, that seems to be the level Vault polices operate at: https://www.vaultproject.io/docs/concepts/policies/#policy-syntax

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394934494
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   I have worked with Google KMS and access was granted on a path 
   Level too.
   
   You generally grant a team permission to a path or mounting point, so they can add all rheir secrets in that path.

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then airflow will not continue to search env vars and metastore (as is the documented behavior); it will just stop here and raise.
   
   see [get_connections](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62) function.
   
   to preserve expected behavior re search path, i think you need to return an empty list.
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394660515
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Remember: I'm not familiar with Vault so may be making stupid assumptions.

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393652598
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+from airflow.secrets import get_connections
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        returned_uri = test_client.get_conn_uri(conn_id="test_postgres")
+        self.assertEqual('postgresql://airflow:airflow@host:5432/airflow', returned_uri)
+
+    @mock.patch.dict('os.environ', {
+        'AIRFLOW_CONN_TEST_MYSQL': 'mysql://airflow:airflow@host:5432/airflow',
+    })
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri_non_existent_key(self, mock_hvac):
+        """
+        Test that if the key with connection ID is not present in Vault, VaultClient.get_connections
+        should return None and fallback to the environment variable if it is set
+        """
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        # Response does not contain the requested key
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'AIRFLOW_CONN_TEST_POSTGRES': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "path": "test_path",
+            "auth_type": "token",
+            "url": "http://127.0.0.1:8200",
+            "token": "s.7AU0I51yv1Q1lxOIg1F3ZRAS"
+        }
+
+        test_client = VaultSecrets(**kwargs)
+        self.assertIsNone(test_client.get_conn_uri(conn_id="test_mysql"))
+        self.assertIsNone(test_client.get_connections(conn_id="test_mysql"))
+
+        # Fallback to connection defined in Environment Variable
 
 Review comment:
   The test didn't previously exist to check this behavior in the base too. This is now added and merged, so will remove

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394654715
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
+                'metadata': {'created_time': '2020-03-16T21:01:43.331126Z',
+                             'deletion_time': '',
+                             'destroyed': False,
+                             'version': 1}},
+            'wrap_info': None,
+            'warnings': None,
+            'auth': None
+        }
+
+        kwargs = {
+            "connections_path": "airflow/connections",
 
 Review comment:
   Not _super_ important, but this doesn't line up with the example in the docs.

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393954612
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   @dstandish what do you think of making the same change to the AWS SSM provider (before we include it in a replace)

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/029c84e5527b6db6bdbdbe026f455da325bedef3&el=desc) will **decrease** coverage by `0.84%`.
   > The diff coverage is `65.43%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.96%   86.11%   -0.85%     
   ==========================================
     Files         920      921       +1     
     Lines       44436    44468      +32     
   ==========================================
   - Hits        38644    38295     -349     
   - Misses       5792     6173     +381     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/secrets/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWNyZXRzL19faW5pdF9fLnB5) | `97.67% <ø> (ø)` | |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `65.43% <65.43%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0.00%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0.00%> (-55.56%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0.00%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0.00%> (-45.08%)` | :arrow_down: |
   | ... and [39 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [029c84e...e39f971](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394197368
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   The main reason for having a "dedicated" way of getting connections is so that we could have the way of listing the connections - checking if an individual connection exists could just try and read the secret and parse it, but if we want to support list then we need a way to tell connections from other 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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394664957
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   So all the connections would live in a single path `airflow/connections` and it accepts a key_value store.
   
   Key I think should be **conn_id**.
   
   Storing them as following:
   
   ```
   vault kv put airflow/connections conn_type=postgresql user=airflow password=airflow host=host port=5432 schema=airflow
   ```
   
   would create separate keys:
   
   ```
   ❯ vault kv put airflow/connections conn_type=mysql user=airflow password=airflow host=host port=5432 schema=airflow
   
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   ❯ vault kv get airflow/connections
   ====== Metadata ======
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   
   ====== Data ======
   Key          Value
   ---          -----
   conn_type    mysql
   host         host
   password     airflow
   port         5432
   schema       airflow
   user         airflow
   ```
   
   which doesn't make sense as there are not linked to each other

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395299575
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
+        elif self.auth_type == "ldap":
+            _client.auth.ldap.login(
+                username=self.username, password=self.password)
+        elif self.auth_type == "userpass":
+            _client.auth_userpass(username=self.username, password=self.password)
+        elif self.auth_type == "approle":
+            _client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+        elif self.auth_type == "github":
+            _client.auth.github.login(token=self.token)
+        elif self.auth_type == "gcp":
+            credentials = self._get_gcp_credentials()
+            _client.auth.gcp.configure(credentials=credentials)
+        else:
+            raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+        if _client.is_authenticated():
+            return _client
+        else:
+            raise VaultError("Vault Authentication Error!")
+
+    def build_path(self, conn_id: str):
+        """
+        Given conn_id, build path for Vault Secret
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self.connections_path + "/" + conn_id
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        self.log.debug("Path: %s", self.connections_path)
 
 Review comment:
   Probably should log this after we've built the path.

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `ConnectionNotFound` error, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  raising an exception when not found is probably more intuitive from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395308865
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,214 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import InvalidPath, VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param connections_path: Specifies the path of the secret to read to get Connections.
+    :type connections_path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        connections_path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        kv_engine_version: int = 2,
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__(**kwargs)
+        self.connections_path = connections_path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.kv_engine_version = kv_engine_version
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+
+    @cached_property
+    def client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+
+        _client = hvac.Client(url=self.url, **self.kwargs)
+        if self.auth_type == "token":
+            _client.token = self.token
+        elif self.auth_type == "ldap":
+            _client.auth.ldap.login(
+                username=self.username, password=self.password)
+        elif self.auth_type == "userpass":
+            _client.auth_userpass(username=self.username, password=self.password)
+        elif self.auth_type == "approle":
+            _client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+        elif self.auth_type == "github":
+            _client.auth.github.login(token=self.token)
+        elif self.auth_type == "gcp":
+            credentials = self._get_gcp_credentials()
+            _client.auth.gcp.configure(credentials=credentials)
+        else:
+            raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+        if _client.is_authenticated():
+            return _client
+        else:
+            raise VaultError("Vault Authentication Error!")
+
+    def build_path(self, conn_id: str):
+        """
+        Given conn_id, build path for Vault Secret
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        return self.connections_path + "/" + conn_id
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        self.log.debug("Path: %s", self.connections_path)
 
 Review comment:
   Removed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394266783
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 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] ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394658839
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   `conn_uri` is there because you need _something_ as the key, don't you?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393373474
 
 

 ##########
 File path: docs/howto/connection/index.rst
 ##########
 @@ -179,6 +179,41 @@ Optionally you can supply a profile name to reference aws profile, e.g. defined
 
 The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
 
+.. _hashicorp_vault_secrets:
+
+Hashicorp Value Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
+key to ``backend_kwargs``:
+
+.. code-block:: bash
+
+    export VAULT_ADDR="http://127.0.0.1:8200"
+
+If you have set your path as ``airflow``, then for a connection id of ``smtp_default``, you would want to
 
 Review comment:
   Sure, will move

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393560292
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   Also added a test case to capture that fallback on env vars

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394194861
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   I think mount_point is a Vault-specific concept, but path is probably generic enough a concept.

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393971110
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   no i agree but what i mean is like it could be like `AIRFLOW_SECRET_BLAH` and `AIRFLOW_CONN_BLAH` in theory.... and allow it to work on env vars too... and same prefix....
   
   just a thought, a hypothetical

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393402774
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,156 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github'). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    """
+    def __init__(
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
 
 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] ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r395209173
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Does G kms just have a single key at each path, or does it also have keys like valut does?

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   i think the desirable behavior is to return an empty list.
   
   but perhaps better yet would be to define a `ConnectionNotFound` error, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214?src=pr&el=desc) will **decrease** coverage by `0.77%`.
   > The diff coverage is `77.58%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   86.15%   -0.78%     
   ==========================================
     Files         915      916       +1     
     Lines       44172    44234      +62     
   ==========================================
   - Hits        38403    38112     -291     
   - Misses       5769     6122     +353
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `77.58% <77.58%> (ø)` | |
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.51% <0%> (-72.16%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0%> (-64.16%)` | :arrow_down: |
   | [airflow/kubernetes/volume\_mount.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZV9tb3VudC5weQ==) | `44.44% <0%> (-55.56%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `50% <0%> (-50%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50% <0%> (-50%)` | :arrow_down: |
   | [airflow/kubernetes/volume.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3ZvbHVtZS5weQ==) | `52.94% <0%> (-47.06%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0%> (-46.67%)` | :arrow_down: |
   | [airflow/kubernetes/pod\_launcher.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9rdWJlcm5ldGVzL3BvZF9sYXVuY2hlci5weQ==) | `47.18% <0%> (-45.08%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55% <0%> (-45%)` | :arrow_down: |
   | ... and [12 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...1995fe8](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore; it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` error, raise it, and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  raising an exception when not found is probably more intuitive from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393470882
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 Review comment:
   @kaxil if you raise here, then i'm pretty sure airflow will not continue to search in env vars then in metastore (as is the documented behavior); it will just stop here and raise.
   
   i think to preserve expected behavior re search path, you need to return an empty list.
   
   but perhaps better yet would be to define a `AirflowConnectionNotFound` exception, raise it (have to update the other backends), and then in [the static get connections function](https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62), catch and ignore.  because raising an exception when not found is probably more intuitive than returning an empty list from the secrets backend class's perspective.
   
   see https://github.com/apache/airflow/blob/master/airflow/secrets/__init__.py#L62
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393970026
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   I think in most cases 1 prefix might be fine. Just `/airflow` might do. When we implement generic secret, we can provide user a way to override/specify prefix/entire path from the operator/hook/xcom etc

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393984195
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   but then what if you add in `get_secret`.  do you have to modify backends to have two prefixes -- one for connections, and one for 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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393618321
 
 

 ##########
 File path: setup.py
 ##########
 @@ -551,6 +554,7 @@ def do_setup():
             'grpc': grpc,
             'hdfs': hdfs,
             'hive': hive,
+            'hvac': hvac,
 
 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] ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393974473
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default Airflow will search environment variables first and metastore
+database second.
+
+If you enable an alternative secrets backend, it will be searched first, followed by environment variables,
+then metastore.  This search ordering is not configurable.
+
+.. _secrets_backend_configuration:
+
+Configuration
+^^^^^^^^^^^^^
+
+The ``[secrets]`` section has the following options:
+
+.. code-block:: ini
+
+    [secrets]
+    backend =
+    backend_kwargs =
+
+Set ``backend`` to the fully qualified class name of the backend you want to enable.
+
+You can provide ``backend_kwargs`` with json and it will be passed as kwargs to the ``__init__`` method of
+your secrets backend.
+
+See :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>` for an example configuration.
+
+.. _ssm_parameter_store_secrets:
+
+AWS SSM Parameter Store Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable SSM parameter store, specify :py:class:`~airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.amazon.aws.secrets.ssm.AwsSsmSecretsBackend
+    backend_kwargs = {"prefix": "/airflow", "profile_name": "default"}
+
+If you have set your prefix as ``/airflow``, then for a connection id of ``smtp_default``, you would want to
+store your connection at ``/airflow/AIRFLOW_CONN_SMTP_DEFAULT``.
+
+Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``.
+
+The value of the SSM parameter must be the :ref:`airflow connection URI representation <generating_connection_uri>` of the connection object.
+
+.. _hashicorp_vault_secrets:
+
+Hashicorp Vault Secrets Backend
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+To enable Hashicorp vault to retrieve connection, specify :py:class:`~airflow.providers.hashicorp.secrets.vault.VaultSecrets`
+as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``.
+
+Here is a sample configuration:
+
+.. code-block:: ini
+
+    [secrets]
+    backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+    backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+You can also set and pass values to Vault client by setting environment variables. All the
+environment variables listed at https://www.vaultproject.io/docs/commands/#environment-variables are supported.
+
+Hence, if you set ``VAULT_ADDR`` environment variable like below, you do not need to pass ``url``
+key to ``backend_kwargs``:
+
+.. code-block:: bash
+
+    export VAULT_ADDR="http://127.0.0.1:8200"
+
+If you have set your path as ``airflow/connections``, then for a connection id of ``smtp_default``, you would want to
+store your secret as:
+
+.. code-block:: bash
+
+    vault kv put secret/airflow/connections smtp_default=postgresql://airflow:airflow@host:5432/airflow
+
+The value of the Vault key must be the :ref:`airflow connection URI representation <generating_connection_uri>`
 
 Review comment:
   ```suggestion
   The value of the Vault key must be the :ref:`connection URI representation <generating_connection_uri>`
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393559896
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,197 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> str:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
+        client = self.get_client()
+
+        self.log.debug("Path: %s", self.path)
+        self.log.debug("Mount Point: %s", self.mount_point)
+        self.log.debug("Retrieving the secret for Connection ID: %s", connection_id)
+
+        response = client.secrets.kv.v2.read_secret_version(
+            path=self.path, mount_point=self.mount_point)
+        return_data = response["data"]["data"]
+
+        try:
+            value = return_data[connection_id]
+            self.log.debug("Value of the secret: %s", return_data)
+        except KeyError:
 
 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] codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-599830481
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=h1) Report
   > Merging [#7741](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/ae035cdb69c76eb7070629d00ded294510df1214?src=pr&el=desc) will **decrease** coverage by `22.39%`.
   > The diff coverage is `80%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7741/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7741      +/-   ##
   ==========================================
   - Coverage   86.93%   64.54%   -22.4%     
   ==========================================
     Files         915      915              
     Lines       44172    44221      +49     
   ==========================================
   - Hits        38403    28541    -9862     
   - Misses       5769    15680    +9911
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/providers/amazon/aws/hooks/s3.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYW1hem9uL2F3cy9ob29rcy9zMy5weQ==) | `96.52% <100%> (+0.06%)` | :arrow_up: |
   | [airflow/providers/hashicorp/secrets/vault.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvaGFzaGljb3JwL3NlY3JldHMvdmF1bHQucHk=) | `77.58% <77.58%> (ø)` | |
   | [...low/contrib/operators/wasb\_delete\_blob\_operator.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy93YXNiX2RlbGV0ZV9ibG9iX29wZXJhdG9yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/hooks/vertica\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL2hvb2tzL3ZlcnRpY2FfaG9vay5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/\_\_init\_\_.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvX19pbml0X18ucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/mssql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9tc3NxbF9ob29rLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [...viders/docker/example\_dags/example\_docker\_swarm.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZG9ja2VyL2V4YW1wbGVfZGFncy9leGFtcGxlX2RvY2tlcl9zd2FybS5weQ==) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/hooks/webhdfs\_hook.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy93ZWJoZGZzX2hvb2sucHk=) | `0% <0%> (-100%)` | :arrow_down: |
   | [airflow/contrib/sensors/emr\_base\_sensor.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL3NlbnNvcnMvZW1yX2Jhc2Vfc2Vuc29yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | [...irflow/contrib/operators/slack\_webhook\_operator.py](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree#diff-YWlyZmxvdy9jb250cmliL29wZXJhdG9ycy9zbGFja193ZWJob29rX29wZXJhdG9yLnB5) | `0% <0%> (-100%)` | :arrow_down: |
   | ... and [493 more](https://codecov.io/gh/apache/airflow/pull/7741/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=footer). Last update [ae035cd...1995fe8](https://codecov.io/gh/apache/airflow/pull/7741?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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] dephusluke commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dephusluke commented on issue #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#issuecomment-600774328
 
 
   @kaxil you added `airflow/providers/hashicorp/secrets/vault.py`
   say that in future there will be a `HashicorpVaultHook`. Where would that go?
   https://issues.apache.org/jira/browse/AIRFLOW-3669
   
   Maybe I'm getting this wrong but I thought the providers package is for hooks/operators/sensor ?
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
BasPH commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393491014
 
 

 ##########
 File path: docs/howto/use-alternative-secrets-backend.rst
 ##########
 @@ -0,0 +1,130 @@
+ .. 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.
+
+
+Alternative secrets backend
+---------------------------
+
+In addition to retrieving connections from environment variables or the metastore database, you can enable
+an alternative secrets backend to retrieve connections,
+such as :ref:`AWS SSM Parameter Store <ssm_parameter_store_secrets>`,
+:ref:`Hashicorp Vault Secrets<hashicorp_vault_secrets>` or you can :ref:`roll your own <roll_your_own_secrets_backend>`.
+
+Search path
+^^^^^^^^^^^
+When looking up a connection, by default airflow will search environment variables first and metastore
 
 Review comment:
   ```suggestion
   When looking up a connection, by default Airflow will search environment variables first and metastore
   ```

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394659919
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   Current method:
   
   ```
   vault kv put airflow/connections smtp_default=smtps://user:host@relay.example.com:465
   vault kv put airflow/connections postgres_default=postgresql://airflow:airflow@host:5432/airflow
   ```
   
   My proposal:
   
   ```
   vault kv put airflow/connections/smtp_default conn_uri=smtps://user:host@relay.example.com:465
   vault kv put airflow/connections conn_uri=postgresql://airflow:airflow@host:5432/airflow
   ```
   
   And conn_uri is there so we could do this in the future:
   
   ```
   vault kv put airflow/connections conn_type=postgresql user=airflow password=airflow host=host port=5432 schema=airflow
   ```

----------------------------------------------------------------
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] dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393984195
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   but then what if you add in `get_secret`.  do you have to modify backends to have two prefixes -- one for connections, and one for secrets?
   
   if `get_secret` is added, is any namespacing at all required?  if so, do we namespace within prefix? or do we have to use distinct prefixes?

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393629466
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,196 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import CONN_ENV_PREFIX, BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault Secrets
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
+    provide ``{"path": "airflow"}`` and request conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
+        url: Optional[str] = None,
+        auth_type: str = 'token',
+        mount_point: str = 'secret',
+        token: Optional[str] = None,
+        username: Optional[str] = None,
+        password: Optional[str] = None,
+        role_id: Optional[str] = None,
+        secret_id: Optional[str] = None,
+        gcp_key_path: Optional[str] = None,
+        gcp_scopes: Optional[str] = None,
+        **kwargs
+    ):
+        super().__init__()
+        self.path = path.rstrip('/')
+        self.url = url
+        self.auth_type = auth_type
+        self.kwargs = kwargs
+        self.token = token
+        self.username = username
+        self.password = password
+        self.role_id = role_id
+        self.secret_id = secret_id
+        self.mount_point = mount_point
+        self.gcp_key_path = gcp_key_path
+        self.gcp_scopes = gcp_scopes
+        self._client: Optional[hvac.Client] = None
+
+    def get_client(self) -> hvac.Client:
+        """
+        Return an authenticated Hashicorp Vault client
+        """
+        if not self._client:
+            self._client = hvac.Client(url=self.url, **self.kwargs)
+            if self.auth_type == "token":
+                self._client.token = self.token
+            elif self.auth_type == "ldap":
+                self._client.auth.ldap.login(
+                    username=self.username, password=self.password)
+            elif self.auth_type == "userpass":
+                self._client.auth_userpass(username=self.username, password=self.password)
+            elif self.auth_type == "approle":
+                self._client.auth_approle(role_id=self.role_id, secret_id=self.secret_id)
+            elif self.auth_type == "github":
+                self._client.auth.github.login(token=self.token)
+            elif self.auth_type == "gcp":
+                credentials = self._get_gcp_credentials()
+                self._client.auth.gcp.configure(credentials=credentials)
+            else:
+                raise AirflowException(f"Authentication type '{self.auth_type}' not supported")
+
+            if self._client.is_authenticated():
+                return self._client
+            else:
+                raise VaultError("Vault Authentication Error!")
+        else:
+            return self._client
+
+    def get_conn_uri(self, conn_id: str) -> Optional[str]:
+        """
+        Get secret value from Vault. Store the secret in the form of URI
+
+        :param conn_id: connection id
+        :type conn_id: str
+        """
+        connection_id = f"{CONN_ENV_PREFIX}{conn_id}".upper()
 
 Review comment:
   This was done to keep it consistent with how AWS SSM has implemented i.e. store keys as "AIRFLOW_CONN_{CONN_ID}" format.
   
   But I agree it is redundant and does not make sense.  I have updated it in https://github.com/apache/airflow/pull/7741/commits/d59b45aace618e5d373eb02f421f53bc4352062b

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394249601
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,202 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path":"connections","url":"http://127.0.0.1:8200","mount_point":"airflow"}
+
+    For example, if your keys are under ``connections`` path in ``airflow`` mount_point, this
+    would be accessible if you provide ``{"path": "connections"}`` and request
+    conn_id ``smtp_default``.
+
+    :param path: Specifies the path of the secret to read.
+    :type path: str
+    :param url: Base URL for the Vault instance being addressed.
+    :type url: str
+    :param auth_type: Authentication Type for Vault (one of 'token', 'ldap', 'userpass', 'approle',
+        'github', 'gcp). Default is ``token``.
+    :type auth_type: str
+    :param mount_point: The "path" the secret engine was mounted on. (Default: ``secret``)
+    :type mount_point: str
+    :param token: Authentication token to include in requests sent to Vault.
+        (for ``token`` and ``github`` auth_type)
+    :type token: str
+    :param kv_engine_version: Select the version of the engine to run (``1`` or ``2``, default: ``2``)
+    :type kv_engine_version: int
+    :param username: Username for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type username: str
+    :param password: Password for Authentication (for ``ldap`` and ``userpass`` auth_type)
+    :type password: str
+    :param role_id: Role ID for Authentication (for ``approle`` auth_type)
+    :type role_id: str
+    :param secret_id: Secret ID for Authentication (for ``approle`` auth_type)
+    :type secret_id: str
+    :param gcp_key_path: Path to GCP Credential JSON file (for ``gcp`` auth_type)
+    :type gcp_key_path: str
+    :param gcp_scopes: Comma-separated string containing GCP scopes (for ``gcp`` auth_type)
+    :type gcp_scopes: str
+    """
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        path: str,
 
 Review comment:
   ```suggestion
           connections_path: str,
   ```

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r393959245
 
 

 ##########
 File path: airflow/providers/hashicorp/secrets/vault.py
 ##########
 @@ -0,0 +1,192 @@
+#
+# 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 connections from Hashicorp Vault
+"""
+from typing import List, Optional
+
+import hvac
+from cached_property import cached_property
+from hvac.exceptions import VaultError
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.secrets import BaseSecretsBackend
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class VaultSecrets(BaseSecretsBackend, LoggingMixin):
+    """
+    Retrieves Connection object from Hashicorp Vault
+
+    Configurable via ``airflow.cfg`` as follows:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.hashicorp.secrets.vault.VaultSecrets
+        backend_kwargs = {"path": "airflow/connections", "url": "http://127.0.0.1:8200"}
+
+    For example, if your keys are under ``airflow/AIRFLOW_CONN_SMTP_DEFAULT``, this would be accessible if you
 
 Review comment:
   Fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394657719
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   I.e. instead of this
   
   ```
   vault kv put airflow/connections smtp_default=postgresql://airflow:airflow@host:5432/airflow
   ```
   
   I'm proposing we do
   
   ```
   vault kv put airflow/connections/smtp_default
   conn_uri=postgresql://airflow:airflow@host:5432/airflow
   ```

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741
 
 
   

----------------------------------------------------------------
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 #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7741: [AIRFLOW-7076] Add support for HashiCorp Vault as Secrets Backend
URL: https://github.com/apache/airflow/pull/7741#discussion_r394664957
 
 

 ##########
 File path: tests/providers/hashicorp/secrets/test_vault.py
 ##########
 @@ -0,0 +1,137 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from unittest import TestCase, mock
+
+from hvac.exceptions import VaultError
+
+from airflow.providers.hashicorp.secrets.vault import VaultSecrets
+
+
+class TestVaultSecrets(TestCase):
+
+    @mock.patch("airflow.providers.hashicorp.secrets.vault.hvac")
+    def test_get_conn_uri(self, mock_hvac):
+        mock_client = mock.MagicMock()
+        mock_hvac.Client.return_value = mock_client
+        mock_client.secrets.kv.v2.read_secret_version.return_value = {
+            'request_id': '94011e25-f8dc-ec29-221b-1f9c1d9ad2ae',
+            'lease_id': '',
+            'renewable': False,
+            'lease_duration': 0,
+            'data': {
+                'data': {'test_postgres': 'postgresql://airflow:airflow@host:5432/airflow'},
 
 Review comment:
   So all the connections would live in a single path `airflow/connections` and it accepts a key_value store.
   
   Key I think should be **conn_id**.
   
   Storing them as following:
   
   ```
   vault kv put airflow/connections conn_type=postgresql user=airflow password=airflow host=host port=5432 schema=airflow
   ```
   
   would create separate keys:
   
   ```
   ❯ vault kv put airflow/connections conn_type=mysql user=airflow password=airflow host=host port=5432 schema=airflow
   
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   ❯ vault kv get airflow/connections
   ====== Metadata ======
   Key              Value
   ---              -----
   created_time     2020-03-18T22:00:51.665274Z
   deletion_time    n/a
   destroyed        false
   version          2
   
   ====== Data ======
   Key          Value
   ---          -----
   conn_type    mysql
   host         host
   password     airflow
   port         5432
   schema       airflow
   user         airflow
   ```
   
   which doesn't make sense as they are not linked to each other. I think what we have currently is fine

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