From b0bc9cbd8458853d3ae4fe916bd1b87b557aa523 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Sat, 28 Mar 2020 13:35:51 +0000 Subject: [PATCH] Get Airflow Variables from AWS Systems Manager Parameter Store (#7945) (cherry picked from commit 7239d9a82dbb3b9bdf27b531daa70338af9dd796) --- .../contrib/secrets/aws_systems_manager.py | 30 ++++++++++++++++-- .../howto/use-alternative-secrets-backend.rst | 14 ++++++++- .../secrets/test_aws_systems_manager.py | 31 +++++++++++++++++++ 3 files changed, 72 insertions(+), 3 deletions(-) diff --git a/airflow/contrib/secrets/aws_systems_manager.py b/airflow/contrib/secrets/aws_systems_manager.py index 2912d4c5ef87e0..971ad18acaf4e3 100644 --- a/airflow/contrib/secrets/aws_systems_manager.py +++ b/airflow/contrib/secrets/aws_systems_manager.py @@ -29,7 +29,7 @@ class SystemsManagerParameterStoreBackend(BaseSecretsBackend, LoggingMixin): """ - Retrieves Connection object from AWS SSM Parameter Store + Retrieves Connection or Variables from AWS SSM Parameter Store Configurable via ``airflow.cfg`` like so: @@ -41,15 +41,19 @@ class SystemsManagerParameterStoreBackend(BaseSecretsBackend, LoggingMixin): For example, if ssm path is ``/airflow/connections/smtp_default``, this would be accessible if you provide ``{"connections_prefix": "/airflow/connections"}`` and request conn_id ``smtp_default``. + And if ssm path is ``/airflow/variables/hello``, this would be accessible + if you provide ``{"variables_prefix": "/airflow/variables"}`` and request conn_id ``hello``. """ def __init__( self, connections_prefix='/airflow/connections', # type: str + variables_prefix='/airflow/variables', # type: str profile_name=None, # type: Optional[str] **kwargs ): self.connections_prefix = connections_prefix.rstrip("/") + self.variables_prefix = variables_prefix.rstrip('/') self.profile_name = profile_name super(SystemsManagerParameterStoreBackend, self).__init__(**kwargs) @@ -71,7 +75,29 @@ def get_conn_uri(self, conn_id): :rtype: str """ - ssm_path = self.build_path(self.connections_prefix, conn_id) + return self._get_secret(self.connections_prefix, conn_id) + + def get_variable(self, key): + # type: (str) -> Optional[str] + """ + Get Airflow Variable from Environment Variable + + :param key: Variable Key + :return: Variable Value + """ + return self._get_secret(self.variables_prefix, key) + + def _get_secret(self, path_prefix, secret_id): + # type: (str, str) -> Optional[str] + """ + Get secret value from Parameter Store. + + :param path_prefix: Prefix for the Path to get Secret + :type path_prefix: str + :param secret_id: Secret Key + :type secret_id: str + """ + ssm_path = self.build_path(path_prefix, secret_id) try: response = self.client.get_parameter( Name=ssm_path, WithDecryption=False diff --git a/docs/howto/use-alternative-secrets-backend.rst b/docs/howto/use-alternative-secrets-backend.rst index 34810c165e85fd..786ea0af3fd832 100644 --- a/docs/howto/use-alternative-secrets-backend.rst +++ b/docs/howto/use-alternative-secrets-backend.rst @@ -65,8 +65,12 @@ Here is a sample configuration: .. code-block:: ini [secrets] + backend = airflow.contrib.secrets.aws_systems_manager.SystemsManagerParameterStoreBackend - backend_kwargs = {"connections_prefix": "/airflow/connections", "profile_name": "default"} + backend_kwargs = {"connections_prefix": "/airflow/connections", "variables_prefix": "/airflow/variables", "profile_name": "default"} + +Storing and Retrieving Connections +"""""""""""""""""""""""""""""""""" If you have set ``connections_prefix`` as ``/airflow/connections``, then for a connection id of ``smtp_default``, you would want to store your connection at ``/airflow/connections/smtp_default``. @@ -76,6 +80,14 @@ Optionally you can supply a profile name to reference aws profile, e.g. defined The value of the SSM parameter must be the :ref:`connection URI representation ` of the connection object. +Storing and Retrieving Variables +"""""""""""""""""""""""""""""""" + +If you have set ``variables_prefix`` as ``/airflow/variables``, then for an Variable key of ``hello``, +you would want to store your Variable at ``/airflow/variables/hello``. + +Optionally you can supply a profile name to reference aws profile, e.g. defined in ``~/.aws/config``. + .. _hashicorp_vault_secrets: Hashicorp Vault Secrets Backend diff --git a/tests/contrib/secrets/test_aws_systems_manager.py b/tests/contrib/secrets/test_aws_systems_manager.py index b6616eef2662b4..975e29860a5cf1 100644 --- a/tests/contrib/secrets/test_aws_systems_manager.py +++ b/tests/contrib/secrets/test_aws_systems_manager.py @@ -65,3 +65,34 @@ def test_get_conn_uri_non_existent_key(self): self.assertIsNone(ssm_backend.get_conn_uri(conn_id=conn_id)) self.assertEqual([], ssm_backend.get_connections(conn_id=conn_id)) + + @mock_ssm + def test_get_variable(self): + param = { + 'Name': '/airflow/variables/hello', + 'Type': 'String', + 'Value': 'world' + } + + ssm_backend = SystemsManagerParameterStoreBackend() + ssm_backend.client.put_parameter(**param) + + returned_uri = ssm_backend.get_variable('hello') + self.assertEqual('world', returned_uri) + + @mock_ssm + def test_get_variable_non_existent_key(self): + """ + Test that if Variable key is not present in SSM, + SystemsManagerParameterStoreBackend.get_variables should return None + """ + param = { + 'Name': '/airflow/variables/hello', + 'Type': 'String', + 'Value': 'world' + } + + ssm_backend = SystemsManagerParameterStoreBackend() + ssm_backend.client.put_parameter(**param) + + self.assertIsNone(ssm_backend.get_variable("test_mysql"))