Skip to content

Commit

Permalink
Fix CloudSecretsManagerBackend invalid connections_prefix
Browse files Browse the repository at this point in the history
  • Loading branch information
xinbinhuang committed Mar 25, 2020
1 parent 733d3d3 commit d760723
Show file tree
Hide file tree
Showing 5 changed files with 85 additions and 16 deletions.
35 changes: 28 additions & 7 deletions airflow/providers/google/cloud/secrets/secrets_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
"""
Objects relating to sourcing connections from GCP Secrets Manager
"""
import re
from typing import Optional

from cached_property import cached_property
Expand All @@ -26,12 +27,15 @@
from google.cloud.secretmanager_v1 import SecretManagerServiceClient

from airflow import version
from airflow.exceptions import AirflowException
from airflow.providers.google.cloud.utils.credentials_provider import (
_get_scopes, get_credentials_and_project_id,
)
from airflow.secrets import BaseSecretsBackend
from airflow.utils.log.logging_mixin import LoggingMixin

SECRET_ID_PATTERN = r"^[a-zA-Z0-9-_]*$"


class CloudSecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
"""
Expand All @@ -43,10 +47,11 @@ class CloudSecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
[secrets]
backend = airflow.providers.google.cloud.secrets.secrets_manager.CloudSecretsManagerBackend
backend_kwargs = {"connections_prefix": "airflow/connections"}
backend_kwargs = {"connections_prefix": "airflow-connections", "sep": "-"}
For example, if secret id is ``airflow/connections/smtp_default``, this would be accessible
if you provide ``{"connections_prefix": "airflow/connections"}`` and request conn_id ``smtp_default``.
For example, if the Secrets Manager secret id is ``airflow-connections-smtp_default``, this would be
accessiblen if you provide ``{"connections_prefix": "airflow-connections", "sep": "-"}`` and request
conn_id ``smtp_default``. The full secret id should follow the pattern "[a-zA-Z0-9-_]".
:param connections_prefix: Specifies the prefix of the secret to read to get Connections.
:type connections_prefix: str
Expand All @@ -55,20 +60,32 @@ class CloudSecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
:type gcp_key_path: str
:param gcp_scopes: Comma-separated string containing GCP scopes
:type gcp_scopes: str
:param sep: separator used to concatenate connections_prefix and conn_id. Default: "-"
:type sep: str
"""
def __init__(
self,
connections_prefix: str = "airflow/connections",
connections_prefix: str = "airflow-connections",
gcp_key_path: Optional[str] = None,
gcp_scopes: Optional[str] = None,
sep: str = "-",
**kwargs
):
self.connections_prefix = connections_prefix.rstrip("/")
super().__init__(**kwargs)
self.connections_prefix = connections_prefix
self.gcp_key_path = gcp_key_path
self.gcp_scopes = gcp_scopes
self.sep = sep
self.credentials: Optional[str] = None
self.project_id: Optional[str] = None
super().__init__(**kwargs)
if not self._is_valid_prefix_and_sep():
raise AirflowException(
f"`connections_prefix` and `sep` should follows that pattern {SECRET_ID_PATTERN}"
)

def _is_valid_prefix_and_sep(self) -> bool:
prefix = self.connections_prefix + self.sep
return bool(re.match(SECRET_ID_PATTERN, prefix))

@cached_property
def client(self) -> SecretManagerServiceClient:
Expand All @@ -93,7 +110,11 @@ def get_conn_uri(self, conn_id: str) -> Optional[str]:
:param conn_id: connection id
:type conn_id: str
"""
secret_id = self.build_path(connections_prefix=self.connections_prefix, conn_id=conn_id)
secret_id = self.build_path(
connections_prefix=self.connections_prefix,
conn_id=conn_id,
sep=self.sep
)
# always return the latest version of the secret
secret_version = "latest"
name = self.client.secret_version_path(self.project_id, secret_id, secret_version)
Expand Down
6 changes: 4 additions & 2 deletions airflow/secrets/base_secrets.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,16 +30,18 @@ def __init__(self, **kwargs):
pass

@staticmethod
def build_path(connections_prefix: str, conn_id: str) -> str:
def build_path(connections_prefix: str, conn_id: str, sep: str = "/") -> str:
"""
Given conn_id, build path for Secrets Backend
:param connections_prefix: prefix of the secret to read to get Connections
:type connections_prefix: str
:param conn_id: connection id
:type conn_id: str
:param sep: separator used to concatenate connections_prefix and conn_id. Default: "/"
:type sep: str
"""
return f"{connections_prefix}/{conn_id}"
return f"{connections_prefix}{sep}{conn_id}"

def get_conn_uri(self, conn_id: str) -> Optional[str]:
"""
Expand Down
5 changes: 4 additions & 1 deletion docs/howto/use-alternative-secrets-backend.rst
Original file line number Diff line number Diff line change
Expand Up @@ -155,14 +155,17 @@ Available parameters to ``backend_kwargs``:
* ``connections_prefix``: Specifies the prefix of the secret to read to get Connections.
* ``gcp_key_path``: Path to GCP Credential JSON file
* ``gcp_scopes``: Comma-separated string containing GCP scopes
* ``sep``: separator used to concatenate connections_prefix and conn_id. Default: "-"

Note: The full GCP Secrets Manager secret id should follow the pattern "[a-zA-Z0-9-_]".

Here is a sample configuration:

.. code-block:: ini
[secrets]
backend = airflow.providers.google.cloud.secrets.secrets_manager.CloudSecretsManagerBackend
backend_kwargs = {"connections_prefix": "airflow/connections"}
backend_kwargs = {"connections_prefix": "airflow-connections", "sep": "-"}
When ``gcp_key_path`` is not provided, it will use the Application Default Credentials in the current environment. You can set up the credentials with:

Expand Down
43 changes: 37 additions & 6 deletions tests/providers/google/cloud/secrets/test_secrets_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,21 +21,52 @@
from google.cloud.secretmanager_v1.types import AccessSecretVersionResponse
from parameterized import parameterized

from airflow.exceptions import AirflowException
from airflow.models import Connection
from airflow.providers.google.cloud.secrets.secrets_manager import CloudSecretsManagerBackend

CREDENTIALS = 'test-creds'
KEY_FILE = 'test-file.json'
PROJECT_ID = 'test-project-id'
CONNECTIONS_PREFIX = "test-connections"
SEP = '-'
CONN_ID = 'test-postgres'
CONN_URI = 'postgresql://airflow:airflow@host:5432/airflow'

MODULE_NAME = "airflow.providers.google.cloud.secrets.secrets_manager"


class TestCloudSecretsManagerBackend(TestCase):
def test_default_valid_and_sep(self):
backend = CloudSecretsManagerBackend()
self.assertTrue(backend._is_valid_prefix_and_sep())

@parameterized.expand([
"airflow/connections",
("colon:", "not:valid", ":"),
("slash/", "not/valid", "/"),
("space_with_char", "a b", ""),
("space_only", "", " ")
])
def test_raise_exception_with_invalid_prefix_sep(self, _, prefix, sep):
with self.assertRaises(AirflowException):
CloudSecretsManagerBackend(connections_prefix=prefix, sep=sep)

@parameterized.expand([
("dash-", "valid1", "-", True),
("underscore_", "isValid", "_", True),
("empty_string", "", "", True),
("space_prefix", " ", "", False),
("space_sep", "", " ", False),
("colon:", "not:valid", ":", False)
])
def test_is_valid_prefix_and_sep(self, _, prefix, sep, is_valid):
backend = CloudSecretsManagerBackend()
backend.connections_prefix = prefix
backend.sep = sep
self.assertEqual(backend._is_valid_prefix_and_sep(), is_valid)

@parameterized.expand([
"airflow-connections",
"connections",
"airflow"
])
Expand All @@ -51,10 +82,11 @@ def test_get_conn_uri(self, connections_prefix, mock_client_callable, mock_get_c
mock_client.access_secret_version.return_value = test_response

secrets_manager_backend = CloudSecretsManagerBackend(connections_prefix=connections_prefix)
secret_id = secrets_manager_backend.build_path(connections_prefix, CONN_ID, SEP)
returned_uri = secrets_manager_backend.get_conn_uri(conn_id=CONN_ID)
self.assertEqual(CONN_URI, returned_uri)
mock_client.secret_version_path.assert_called_once_with(
PROJECT_ID, f"{connections_prefix}/{CONN_ID}", "latest"
PROJECT_ID, secret_id, "latest"
)

@mock.patch(MODULE_NAME + ".get_credentials_and_project_id")
Expand All @@ -75,13 +107,12 @@ def test_get_conn_uri_non_existent_key(self, mock_client_callable, mock_get_cred
# The requested secret id or secret version does not exist
mock_client.access_secret_version.side_effect = NotFound('test-msg')

connections_prefix = "airflow/connections"

secrets_manager_backend = CloudSecretsManagerBackend(connections_prefix=connections_prefix)
secrets_manager_backend = CloudSecretsManagerBackend(connections_prefix=CONNECTIONS_PREFIX)
secret_id = secrets_manager_backend.build_path(CONNECTIONS_PREFIX, CONN_ID, SEP)
with self.assertLogs(secrets_manager_backend.log, level="ERROR") as log_output:
self.assertIsNone(secrets_manager_backend.get_conn_uri(conn_id=CONN_ID))
self.assertEqual([], secrets_manager_backend.get_connections(conn_id=CONN_ID))
self.assertRegex(
log_output.output[0],
f"GCP API Call Error \\(NotFound\\): Secret ID {connections_prefix}/{CONN_ID} not found"
f"GCP API Call Error \\(NotFound\\): Secret ID {secret_id} not found"
)
12 changes: 12 additions & 0 deletions tests/secrets/test_secrets_backends.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,10 @@
import os
import unittest

from parameterized import parameterized

from airflow.models import Connection
from airflow.secrets.base_secrets import BaseSecretsBackend
from airflow.secrets.environment_variables import EnvironmentVariablesBackend
from airflow.secrets.metastore import MetastoreBackend
from airflow.utils.session import create_session
Expand All @@ -37,6 +40,15 @@ def __init__(self, conn_id, variation: str):


class TestBaseSecretsBackend(unittest.TestCase):

@parameterized.expand([
('default', {"connections_prefix": "PREFIX", "conn_id": "ID"}, "PREFIX/ID"),
('with_sep', {"connections_prefix": "PREFIX", "conn_id": "ID", "sep": "-"}, "PREFIX-ID")
])
def test_build_path(self, _, kwargs, output):
build_path = BaseSecretsBackend.build_path
self.assertEqual(build_path(**kwargs), output)

def test_env_secrets_backend(self):
sample_conn_1 = SampleConn("sample_1", "A")
env_secrets_backend = EnvironmentVariablesBackend()
Expand Down

0 comments on commit d760723

Please sign in to comment.