This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new cb979f9  Get Airflow configs with sensitive data from 
CloudSecretManagerBackend (#11024)
cb979f9 is described below

commit cb979f9f213bb3c9835a3dc924f84a07f5387378
Author: Kaxil Naik <[email protected]>
AuthorDate: Tue Sep 22 08:17:58 2020 +0100

    Get Airflow configs with sensitive data from CloudSecretManagerBackend 
(#11024)
---
 .../providers/google/cloud/secrets/secret_manager.py | 14 ++++++++++++++
 .../google-cloud-secret-manager-backend.rst          |  2 ++
 docs/security/secrets/secrets-backend/index.rst      |  3 +++
 .../google/cloud/secrets/test_secret_manager.py      | 20 ++++++++++++++++++++
 4 files changed, 39 insertions(+)

diff --git a/airflow/providers/google/cloud/secrets/secret_manager.py 
b/airflow/providers/google/cloud/secrets/secret_manager.py
index fed5e0e..ffc7b4d 100644
--- a/airflow/providers/google/cloud/secrets/secret_manager.py
+++ b/airflow/providers/google/cloud/secrets/secret_manager.py
@@ -57,6 +57,9 @@ class CloudSecretManagerBackend(BaseSecretsBackend, 
LoggingMixin):
     :type connections_prefix: str
     :param variables_prefix: Specifies the prefix of the secret to read to get 
Variables.
     :type variables_prefix: str
+    :param config_prefix: Specifies the prefix of the secret to read to get 
Airflow Configurations
+        containing secrets.
+    :type config_prefix: str
     :param gcp_key_path: Path to Google Cloud Service Account key file (JSON). 
Mutually exclusive with
         gcp_keyfile_dict. use default credentials in the current environment 
if not provided.
     :type gcp_key_path: str
@@ -75,6 +78,7 @@ class CloudSecretManagerBackend(BaseSecretsBackend, 
LoggingMixin):
         self,
         connections_prefix: str = "airflow-connections",
         variables_prefix: str = "airflow-variables",
+        config_prefix: str = "airflow-config",
         gcp_keyfile_dict: Optional[dict] = None,
         gcp_key_path: Optional[str] = None,
         gcp_scopes: Optional[str] = None,
@@ -85,6 +89,7 @@ class CloudSecretManagerBackend(BaseSecretsBackend, 
LoggingMixin):
         super().__init__(**kwargs)
         self.connections_prefix = connections_prefix
         self.variables_prefix = variables_prefix
+        self.config_prefix = config_prefix
         self.sep = sep
         if not self._is_valid_prefix_and_sep():
             raise AirflowException(
@@ -129,6 +134,15 @@ class CloudSecretManagerBackend(BaseSecretsBackend, 
LoggingMixin):
         """
         return self._get_secret(self.variables_prefix, key)
 
+    def get_config(self, key: str) -> Optional[str]:
+        """
+        Get Airflow Configuration
+
+        :param key: Configuration Option Key
+        :return: Configuration Option Value
+        """
+        return self._get_secret(self.config_prefix, key)
+
     def _get_secret(self, path_prefix: str, secret_id: str) -> Optional[str]:
         """
         Get secret value from the SecretManager based on prefix.
diff --git 
a/docs/security/secrets/secrets-backend/google-cloud-secret-manager-backend.rst 
b/docs/security/secrets/secrets-backend/google-cloud-secret-manager-backend.rst
index bbca274..1435d73 100644
--- 
a/docs/security/secrets/secrets-backend/google-cloud-secret-manager-backend.rst
+++ 
b/docs/security/secrets/secrets-backend/google-cloud-secret-manager-backend.rst
@@ -116,11 +116,13 @@ The name of the secret must fit the following formats:
 
  * for connection: ``[variable_prefix][sep][connection_name]``
  * for variable: ``[connections_prefix][sep][variable_name]``
+ * for Airflow config: ``[config_prefix][sep][config_name]``
 
 where:
 
  * ``connections_prefix`` - fixed value defined in the ``connections_prefix`` 
parameter in backend configuration. Default: ``airflow-connections``.
  * ``variable_prefix`` - fixed value defined in the ``variable_prefix`` 
parameter in backend configuration. Default: ``airflow-variables``.
+ * ``config_prefix`` - fixed value defined in the ``config_prefix`` parameter 
in backend configuration. Default: ``airflow-config``.
  * ``sep`` - fixed value defined in the ``sep`` parameter in backend 
configuration. Default: ``-``.
 
 The Cloud Secrets Manager secret name should follow the pattern 
``^[a-zA-Z0-9-_]*$``.
diff --git a/docs/security/secrets/secrets-backend/index.rst 
b/docs/security/secrets/secrets-backend/index.rst
index 81526d7..0c03708 100644
--- a/docs/security/secrets/secrets-backend/index.rst
+++ b/docs/security/secrets/secrets-backend/index.rst
@@ -31,6 +31,9 @@ such as :ref:`Google Cloud Secret 
Manager<google_cloud_secret_manager_backend>`,
     The Airflow UI only shows connections and variables stored in the Metadata 
DB and not via any other method.
     If you use an alternative secrets backend, check inside your backend to 
view the values of your variables and connections.
 
+You can also get Airflow configurations with sensitive data from the Secrets 
Backend.
+See :doc:`../../../howto/set-config` for more details.
+
 Search path
 ^^^^^^^^^^^
 When looking up a connection/variable, by default Airflow will search 
environment variables first and metastore
diff --git a/tests/providers/google/cloud/secrets/test_secret_manager.py 
b/tests/providers/google/cloud/secrets/test_secret_manager.py
index 4b6c981..a11fc3e 100644
--- a/tests/providers/google/cloud/secrets/test_secret_manager.py
+++ b/tests/providers/google/cloud/secrets/test_secret_manager.py
@@ -36,6 +36,8 @@ CONN_ID = 'test-postgres'
 CONN_URI = 'postgresql://airflow:airflow@host:5432/airflow'
 VAR_KEY = 'hello'
 VAR_VALUE = 'world'
+CONFIG_KEY = 'sql_alchemy_conn'
+CONFIG_VALUE = 'postgresql://airflow:airflow@host:5432/airflow'
 
 MODULE_NAME = "airflow.providers.google.cloud.secrets.secret_manager"
 CLIENT_MODULE_NAME = 
"airflow.providers.google.cloud._internal_client.secret_manager_client"
@@ -149,6 +151,24 @@ class TestCloudSecretManagerBackend(TestCase):
         self.assertEqual(VAR_VALUE, returned_uri)
         mock_client.secret_version_path.assert_called_once_with(PROJECT_ID, 
secret_id, "latest")
 
+    @parameterized.expand(["airflow-config", "config", "airflow"])
+    @mock.patch(MODULE_NAME + ".get_credentials_and_project_id")
+    @mock.patch(CLIENT_MODULE_NAME + ".SecretManagerServiceClient")
+    def test_get_config(self, config_prefix, mock_client_callable, 
mock_get_creds):
+        mock_get_creds.return_value = CREDENTIALS, PROJECT_ID
+        mock_client = mock.MagicMock()
+        mock_client_callable.return_value = mock_client
+
+        test_response = AccessSecretVersionResponse()
+        test_response.payload.data = CONFIG_VALUE.encode("UTF-8")
+        mock_client.access_secret_version.return_value = test_response
+
+        secrets_manager_backend = 
CloudSecretManagerBackend(config_prefix=config_prefix)
+        secret_id = secrets_manager_backend.build_path(config_prefix, 
CONFIG_KEY, SEP)
+        returned_val = secrets_manager_backend.get_config(CONFIG_KEY)
+        self.assertEqual(CONFIG_VALUE, returned_val)
+        mock_client.secret_version_path.assert_called_once_with(PROJECT_ID, 
secret_id, "latest")
+
     @parameterized.expand(["airflow-variables", "variables", "airflow"])
     @mock.patch(MODULE_NAME + ".get_credentials_and_project_id")
     @mock.patch(CLIENT_MODULE_NAME + ".SecretManagerServiceClient")

Reply via email to