You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2022/12/05 19:22:04 UTC

[airflow] branch main updated: AWS Secrets Manager Backend - major update (#27920)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 8f0265d0d9 AWS Secrets Manager Backend - major update (#27920)
8f0265d0d9 is described below

commit 8f0265d0d9079a8abbd7b895ada418908d8b9909
Author: Daniel Reeves <31...@users.noreply.github.com>
AuthorDate: Mon Dec 5 14:21:54 2022 -0500

    AWS Secrets Manager Backend - major update (#27920)
---
 .../amazon/aws/secrets/secrets_manager.py          | 244 ++++-----------------
 .../img/aws-secrets-manager-json.png               | Bin 0 -> 86477 bytes
 .../img/aws-secrets-manager-uri.png                | Bin 0 -> 60559 bytes
 .../img/aws-secrets-manager.png                    | Bin 28060 -> 0 bytes
 .../secrets-backends/aws-secrets-manager.rst       |  86 +++-----
 .../amazon/aws/secrets/test_secrets_manager.py     |  49 ++---
 6 files changed, 98 insertions(+), 281 deletions(-)

diff --git a/airflow/providers/amazon/aws/secrets/secrets_manager.py b/airflow/providers/amazon/aws/secrets/secrets_manager.py
index 684a8d4afb..8b50864baa 100644
--- a/airflow/providers/amazon/aws/secrets/secrets_manager.py
+++ b/airflow/providers/amazon/aws/secrets/secrets_manager.py
@@ -18,21 +18,16 @@
 """Objects relating to sourcing secrets from AWS Secrets Manager"""
 from __future__ import annotations
 
-import ast
 import json
 import warnings
-from typing import TYPE_CHECKING, Any
-from urllib.parse import unquote, urlencode
+from typing import Any
+from urllib.parse import unquote
 
 from airflow.compat.functools import cached_property
 from airflow.providers.amazon.aws.utils import get_airflow_version, trim_none_values
 from airflow.secrets import BaseSecretsBackend
 from airflow.utils.log.logging_mixin import LoggingMixin
 
-if TYPE_CHECKING:
-    # Avoid circular import problems when instantiating the backend during configuration.
-    from airflow.models.connection import Connection
-
 
 class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
     """
@@ -74,7 +69,7 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
     .. code-block:: python
 
         possible_words_for_conn_fields = {
-            "login": ["user", "username", "login", "user_name"],
+            "login": ["login", "user", "username", "user_name"],
             "password": ["password", "pass", "key"],
             "host": ["host", "remote_host", "server"],
             "port": ["port"],
@@ -96,12 +91,6 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         If set to None (null value in the configuration), requests for configurations will not be sent to
         AWS Secrets Manager. If you don't want a config_prefix, set it as an empty string
     :param sep: separator used to concatenate secret_prefix and secret_id. Default: "/"
-    :param full_url_mode: if True, the secrets must be stored as one conn URI in just one field per secret.
-        If False (set it as false in backend_kwargs), you can store the secret using different
-        fields (password, user...).
-    :param are_secret_values_urlencoded: If True, and full_url_mode is False, then the values are assumed to
-        be URL-encoded and will be decoded before being passed into a Connection object. This option is
-        ignored when full_url_mode is True.
     :param extra_conn_words: for using just when you set full_url_mode as false and store
         the secrets in different fields of secrets manager. You can add more words for each connection
         part beyond the default ones. The extra words to be searched should be passed as a dict of lists,
@@ -115,8 +104,6 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         variables_prefix: str = "airflow/variables",
         config_prefix: str = "airflow/config",
         sep: str = "/",
-        full_url_mode: bool = True,
-        are_secret_values_urlencoded: bool | None = None,
         extra_conn_words: dict[str, list[str]] | None = None,
         **kwargs,
     ):
@@ -134,27 +121,29 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         else:
             self.config_prefix = config_prefix
         self.sep = sep
-        self.full_url_mode = full_url_mode
 
-        if are_secret_values_urlencoded is None:
-            self.are_secret_values_urlencoded = True
-        else:
+        if kwargs.pop("full_url_mode", None) is not None:
             warnings.warn(
-                "The `secret_values_are_urlencoded` kwarg only exists to assist in migrating away from"
-                " URL-encoding secret values when `full_url_mode` is False. It will be considered deprecated"
-                " when values are not required to be URL-encoded by default.",
+                "The `full_url_mode` kwarg is deprecated. Going forward, the `SecretsManagerBackend`"
+                " will support both URL-encoded and JSON-encoded secrets at the same time. The encoding"
+                " of the secret will be determined automatically.",
                 DeprecationWarning,
                 stacklevel=2,
             )
-            if full_url_mode and not are_secret_values_urlencoded:
-                warnings.warn(
-                    "The `secret_values_are_urlencoded` kwarg for the SecretsManagerBackend is only used"
-                    " when `full_url_mode` is False. When `full_url_mode` is True, the secret needs to be"
-                    " URL-encoded.",
-                    UserWarning,
-                    stacklevel=2,
-                )
-            self.are_secret_values_urlencoded = are_secret_values_urlencoded
+
+        if kwargs.get("are_secret_values_urlencoded") is not None:
+            warnings.warn(
+                "The `secret_values_are_urlencoded` is deprecated. This kwarg only exists to assist in"
+                " migrating away from URL-encoding secret values for JSON secrets."
+                " To remove this warning, make sure your JSON secrets are *NOT* URL-encoded, and then"
+                " remove this kwarg from backend_kwargs.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            self.are_secret_values_urlencoded = kwargs.pop("are_secret_values_urlencoded", None)
+        else:
+            self.are_secret_values_urlencoded = False
+
         self.extra_conn_words = extra_conn_words or {}
 
         self.profile_name = kwargs.get("profile_name", None)
@@ -185,59 +174,10 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         session = SessionFactory(conn=conn_config).create_session()
         return session.client(service_name="secretsmanager", **client_kwargs)
 
-    @staticmethod
-    def _format_uri_with_extra(secret, conn_string: str) -> str:
-        try:
-            extra_dict = secret["extra"]
-        except KeyError:
-            return conn_string
-
-        extra = json.loads(extra_dict)  # this is needed because extra_dict is a string and we need a dict
-        conn_string = f"{conn_string}?{urlencode(extra)}"
-
-        return conn_string
-
-    def get_connection(self, conn_id: str) -> Connection | None:
-        if not self.full_url_mode:
-            # Avoid circular import problems when instantiating the backend during configuration.
-            from airflow.models.connection import Connection
-
-            secret_string = self._get_secret(self.connections_prefix, conn_id)
-            secret_dict = self._deserialize_json_string(secret_string)
-
-            if not secret_dict:
-                return None
-
-            if "extra" in secret_dict and isinstance(secret_dict["extra"], str):
-                secret_dict["extra"] = self._deserialize_json_string(secret_dict["extra"])
-
-            data = self._standardize_secret_keys(secret_dict)
-
-            if self.are_secret_values_urlencoded:
-                data = self._remove_escaping_in_secret_dict(secret=data, conn_id=conn_id)
-
-            port: int | None = None
-
-            if data["port"] is not None:
-                port = int(data["port"])
-
-            return Connection(
-                conn_id=conn_id,
-                login=data["user"],
-                password=data["password"],
-                host=data["host"],
-                port=port,
-                schema=data["schema"],
-                conn_type=data["conn_type"],
-                extra=data["extra"],
-            )
-
-        return super().get_connection(conn_id=conn_id)
-
     def _standardize_secret_keys(self, secret: dict[str, Any]) -> dict[str, Any]:
         """Standardize the names of the keys in the dict. These keys align with"""
         possible_words_for_conn_fields = {
-            "user": ["user", "username", "login", "user_name"],
+            "login": ["login", "user", "username", "user_name"],
             "password": ["password", "pass", "key"],
             "host": ["host", "remote_host", "server"],
             "port": ["port"],
@@ -247,6 +187,9 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         }
 
         for conn_field, extra_words in self.extra_conn_words.items():
+            if conn_field == "user":
+                # Support `user` for backwards compatibility.
+                conn_field = "login"
             possible_words_for_conn_fields[conn_field].extend(extra_words)
 
         conn_d: dict[str, Any] = {}
@@ -258,96 +201,16 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
 
         return conn_d
 
-    def get_uri_from_secret(self, secret: dict[str, str]) -> str:
-        conn_d: dict[str, str] = {k: v if v else "" for k, v in self._standardize_secret_keys(secret).items()}
-        conn_string = "{conn_type}://{user}:{password}@{host}:{port}/{schema}".format(**conn_d)
-        return self._format_uri_with_extra(secret, conn_string)
-
-    def _deserialize_json_string(self, value: str | None) -> dict[Any, Any] | None:
-        if not value:
-            return None
-        try:
-            # Use ast.literal_eval for backwards compatibility.
-            # Previous version of this code had a comment saying that using json.loads caused errors.
-            # This likely means people were using dict reprs instead of valid JSONs.
-            res: dict[str, Any] = json.loads(value)
-        except json.JSONDecodeError:
-            try:
-                res = ast.literal_eval(value) if value else None
-                warnings.warn(
-                    f"In future versions, `{type(self).__name__}` will only support valid JSONs, not dict"
-                    " reprs. Please make sure your secret is a valid JSON."
-                )
-            except ValueError:  # 'malformed node or string: ' error, for empty conns
-                return None
-
-        return res
-
-    def _remove_escaping_in_secret_dict(self, secret: dict[str, Any], conn_id: str) -> dict[str, Any]:
-        # When ``unquote(v) == v``, then removing unquote won't affect the user, regardless of
-        # whether or not ``v`` is URL-encoded. For example, "foo bar" is not URL-encoded. But
-        # because decoding it doesn't affect the value, then it will migrate safely when
-        # ``unquote`` gets removed.
-        #
-        # When parameters are URL-encoded, but decoding is idempotent, we need to warn the user
-        # to un-escape their secrets. For example, if "foo%20bar" is a URL-encoded string, then
-        # decoding is idempotent because ``unquote(unquote("foo%20bar")) == unquote("foo%20bar")``.
-        #
-        # In the rare situation that value is URL-encoded but the decoding is _not_ idempotent,
-        # this causes a major issue. For example, if "foo%2520bar" is URL-encoded, then decoding is
-        # _not_ idempotent because ``unquote(unquote("foo%2520bar")) != unquote("foo%2520bar")``
-        #
-        # This causes a problem for migration because if the user decodes their value, we cannot
-        # infer that is the case by looking at the decoded value (from our vantage point, it will
-        # look to be URL-encoded.)
-        #
-        # So when this uncommon situation occurs, the user _must_ adjust the configuration and set
-        # ``parameters_are_urlencoded`` to False to migrate safely. In all other cases, we do not
-        # need the user to adjust this object to migrate; they can transition their secrets with
-        # the default configuration.
-
-        warn_user = False
-        idempotent = True
-
+    def _remove_escaping_in_secret_dict(self, secret: dict[str, Any]) -> dict[str, Any]:
+        """Un-escape secret values that are URL-encoded"""
         for k, v in secret.copy().items():
-
             if k == "extra" and isinstance(v, dict):
                 # The old behavior was that extras were _not_ urlencoded inside the secret.
-                # If they were urlencoded (e.g. "foo%20bar"), then they would be re-urlencoded
-                # (e.g. "foo%20bar" becomes "foo%2520bar") and then unquoted once when parsed.
                 # So we should just allow the extra dict to remain as-is.
                 continue
 
             elif v is not None:
-                v_unquoted = unquote(v)
-                if v != v_unquoted:
-                    secret[k] = unquote(v)
-                    warn_user = True
-
-                    # Check to see if decoding is idempotent.
-                    if v_unquoted == unquote(v_unquoted):
-                        idempotent = False
-
-        if warn_user:
-            msg = (
-                "When full_url_mode=False, URL-encoding secret values is deprecated. In future versions, "
-                f"this value will not be un-escaped. For the conn_id {conn_id!r}, please remove the "
-                "URL-encoding.\n\n"
-                "This warning was raised because the SecretsManagerBackend detected that this "
-                "connection was URL-encoded."
-            )
-            if idempotent:
-                msg = f" Once the values for conn_id {conn_id!r} are decoded, this warning will go away."
-            if not idempotent:
-                msg += (
-                    " In addition to decoding the values for your connection, you must also set"
-                    " secret_values_are_urlencoded=False for your config variable"
-                    " secrets.backend_kwargs because this connection's URL encoding is not idempotent."
-                    " For more information, see:"
-                    " https://airflow.apache.org/docs/apache-airflow-providers-amazon/stable/secrets-backends"
-                    "/aws-secrets-manager.html#url-encoding-of-secrets-when-full-url-mode-is-false"
-                )
-            warnings.warn(msg, DeprecationWarning, stacklevel=2)
+                secret[k] = unquote(v)
 
         return secret
 
@@ -360,38 +223,27 @@ class SecretsManagerBackend(BaseSecretsBackend, LoggingMixin):
         if self.connections_prefix is None:
             return None
 
-        if self.full_url_mode:
-            return self._get_secret(self.connections_prefix, conn_id)
+        secret = self._get_secret(self.connections_prefix, conn_id)
+
+        if secret is not None and secret.strip().startswith("{"):
+            # Before Airflow 2.3, the AWS SecretsManagerBackend added support for JSON secrets.
+            #
+            # The way this was implemented differs a little from how Airflow's core API handle JSON secrets.
+            #
+            # The most notable difference is that SecretsManagerBackend supports extra aliases for the
+            # Connection parts, e.g. "users" is allowed instead of "login".
+            #
+            # This means we need to deserialize then re-serialize the secret if it's a JSON, potentially
+            # renaming some keys in the process.
+
+            secret_dict = json.loads(secret)
+            standardized_secret_dict = self._standardize_secret_keys(secret_dict)
+            if self.are_secret_values_urlencoded:
+                standardized_secret_dict = self._remove_escaping_in_secret_dict(standardized_secret_dict)
+            standardized_secret = json.dumps(standardized_secret_dict)
+            return standardized_secret
         else:
-            warnings.warn(
-                f"In future versions, `{type(self).__name__}.get_conn_value` will return a JSON string when"
-                " full_url_mode is False, not a URI.",
-                DeprecationWarning,
-            )
-
-        # It is very rare for user code to get to this point, since:
-        #
-        # - When full_url_mode is True, the previous statement returns.
-        # - When full_url_mode is False, get_connection() does not call
-        #   `get_conn_value`. Additionally, full_url_mode defaults to True.
-        #
-        # So the code would have to be calling `get_conn_value` directly, and
-        # the user would be using a non-default setting.
-        #
-        # As of Airflow 2.3.0, get_conn_value() is allowed to return a JSON
-        # string in the base implementation. This is a way to deprecate this
-        # behavior gracefully.
-
-        secret_string = self._get_secret(self.connections_prefix, conn_id)
-
-        secret = self._deserialize_json_string(secret_string)
-        connection = None
-
-        # These lines will check if we have with some denomination stored an username, password and host
-        if secret:
-            connection = self.get_uri_from_secret(secret)
-
-        return connection
+            return secret
 
     def get_conn_uri(self, conn_id: str) -> str | None:
         """
diff --git a/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-json.png b/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-json.png
new file mode 100644
index 0000000000..6ed7150858
Binary files /dev/null and b/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-json.png differ
diff --git a/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-uri.png b/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-uri.png
new file mode 100644
index 0000000000..bc6796f9da
Binary files /dev/null and b/docs/apache-airflow-providers-amazon/img/aws-secrets-manager-uri.png differ
diff --git a/docs/apache-airflow-providers-amazon/img/aws-secrets-manager.png b/docs/apache-airflow-providers-amazon/img/aws-secrets-manager.png
deleted file mode 100644
index 5c721eeec0..0000000000
Binary files a/docs/apache-airflow-providers-amazon/img/aws-secrets-manager.png and /dev/null differ
diff --git a/docs/apache-airflow-providers-amazon/secrets-backends/aws-secrets-manager.rst b/docs/apache-airflow-providers-amazon/secrets-backends/aws-secrets-manager.rst
index 68127befa0..7d1190a904 100644
--- a/docs/apache-airflow-providers-amazon/secrets-backends/aws-secrets-manager.rst
+++ b/docs/apache-airflow-providers-amazon/secrets-backends/aws-secrets-manager.rst
@@ -16,7 +16,7 @@
     under the License.
 
 AWS Secrets Manager Backend
-^^^^^^^^^^^^^^^^^^^^^^^^^^^
+===========================
 
 To enable Secrets Manager, specify :py:class:`~airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend`
 as the ``backend`` in  ``[secrets]`` section of ``airflow.cfg``. These ``backend_kwargs`` are parsed as JSON, hence Python
@@ -28,7 +28,7 @@ Here is a sample configuration:
 
     [secrets]
     backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
-    backend_kwargs = {"connections_prefix": "airflow/connections", "variables_prefix": "airflow/variables", "profile_name": "default", "full_url_mode": false}
+    backend_kwargs = {"connections_prefix": "airflow/connections", "variables_prefix": "airflow/variables", "profile_name": "default"}
 
 To authenticate you can either supply arguments listed in
 :ref:`Amazon Webservices Connection Extra config <howto/connection:aws:configuring-the-connection>` or set
@@ -42,25 +42,39 @@ To authenticate you can either supply arguments listed in
 
 
 Storing and Retrieving Connections
-""""""""""""""""""""""""""""""""""
-You can store the different values for a secret in two forms: storing the conn URI in one field (default mode) or using different
-fields in Amazon Secrets Manager (setting ``full_url_mode`` as ``false`` in the backend config), as follows:
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
-.. image:: /img/aws-secrets-manager.png
+There are two ways to store Airflow connections in AWS Secrets Manager.
 
-By default you must use some of the following words for each kind of field:
+Storing a Connection as a URI
+"""""""""""""""""""""""""""""
 
-* For storing passwords, valid key names are password, pass and key
-* Users: user, username, login, user_name
-* Host: host, remote_host, server
-* Port: port
-* You should also specify the type of connection, which can be done naming the key as conn_type, conn_id,
-  connection_type or engine. Valid values for this field are postgres, mysql, snowflake, google_cloud, mongo...
-* For the extra value of the connections, a field called extra must exists. Please note this extra field
-  should be a valid JSON.
+You can store the connection as an Airflow connection URI:
+
+.. image:: /img/aws-secrets-manager-uri.png
+
+Note that every field is assumed to be URL-encoded if you store the connection as a URI.
+For example, if you want to use the value ``my password``, in the URI, it should be represented as ``my%20password``.
+
+Storing a Connection as a JSON
+""""""""""""""""""""""""""""""
+
+You can also store the connection as a JSON with the appropriate key-value pairs:
+
+.. image:: /img/aws-secrets-manager-json.png
+
+Using Airflow default names for connection fields is encouraged, but different aliases are allowed for each field:
+
+* Connection type: ``conn_type``, ``conn_id``, ``connection_type``, ``engine``
+* Login: ``login``, ``user``, ``username``, ``user_name``
+* Password: ``password``, ``pass``, ``key``
+* Host: ``host``, ``remote_host``, ``server``
+* Port: ``port``
+* Extra: ``extra``. Note that this extra field should be a valid JSON.
+
+More words can be added to the list using the parameter ``extra_conn_words`` in the configuration. This
+parameter has to be a dict of lists with the following optional keys: user, password, host, schema, conn_type.
 
-However, more words can be added to the list using the parameter ``extra_conn_words`` in the configuration. This
-parameter has to be a dict of lists with the following optional keys: user, password, host, schema, conn_type
 As an example, 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``. This can be done through the AWS web
 console or through Amazon CLI as shown below:
@@ -69,7 +83,7 @@ console or through Amazon CLI as shown below:
 
     aws secretsmanager put-secret-value \
         --secret-id airflow/connections/smtp_default \
-        --secret-string '{"user": "nice_user","pass": "this_is_the_password","host": "ec2.8399.com","port": "999"}'
+        --secret-string '{"login": "nice_user", "password": "this_is_the_password", "host": "ec2.8399.com", "port": "999"}'
 
 Verify that you can get the secret:
 
@@ -80,7 +94,7 @@ Verify that you can get the secret:
         "ARN": "arn:aws:secretsmanager:us-east-2:314524341751:secret:airflow/connections/smtp_default-7meuul",
         "Name": "airflow/connections/smtp_default",
         "VersionId": "34f90eff-ea21-455a-9c8f-5ee74b21be672",
-        "SecretString": "{\n  \"user\":\"nice_user\",\n  \"pass\":\"this_is_the_password\"\n,
+        "SecretString": "{\n  \"login\":\"nice_user\",\n  \"password\":\"this_is_the_password\"\n,
         \n  \"host\":\"ec2.8399.com\"\n,\n  \"port\":\"999\"\n}\n",
         "VersionStages": [
             "AWSCURRENT"
@@ -90,42 +104,14 @@ Verify that you can get the secret:
 
 If you don't want to use any ``connections_prefix`` for retrieving connections, set it as an empty string ``""`` in the configuration.
 
-URL-Encoding of Secrets When Full URL Mode is False
-"""""""""""""""""""""""""""""""""""""""""""""""""""
-
-Previous versions of the Amazon provider package required values in the AWS secret to be URL-encoded when the setting ``full_url_mode`` is ``false``.
-This behavior is now deprecated, and will be removed at a future date.
-
-In most cases, you should not have any issues migrating your secrets to not being URL-encoded in advance of the deprecation.
-Simply decoding your secret values will work, and no further changes are required.
-
-In rare circumstances, the ``DeprecationWarning`` will tell you to add a new parameter to your ``backend_kwargs``.
-This warning occurs when decoding is not idempotent.
-A decoding is idempotent when decoding it once using the ``urllib.parse.unquote`` function is equivalent to decoding it two or more times using that function.
-For example:
-
-* If ``"foo%20bar"`` is a URL-encoded value, then decoding is idempotent because ``unquote(unquote("foo%20bar")) == unquote("foo%20bar")``
-* If ``"foo%2520bar"`` is a URL-encoded value, then decoding is _not_ idempotent because ``unquote(unquote("foo%2520bar")) != unquote("foo%2520bar")``
-
-Setting ``secret_values_are_urlencoded`` to ``false`` will force the ``SecretsManagerBackend`` to stop treating secret values as being URL-encoded.
-
-.. code-block:: ini
-
-    [secrets]
-    backend = airflow.providers.amazon.aws.secrets.secrets_manager.SecretsManagerBackend
-    backend_kwargs = {"connections_prefix": "airflow/connections", "full_url_mode": false, "secret_values_are_urlencoded": false}
-
-
-Note that if ``full_url_mode`` is ``true``, it is still necessary to URL-encode the entire secret.
-
 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``.
 
 Optional lookup
-"""""""""""""""
+^^^^^^^^^^^^^^^
 
 Optionally connections, variables, or config may be looked up exclusive of each other or in any combination.
 This will prevent requests being sent to AWS Secrets Manager for the excluded type.
@@ -141,7 +127,7 @@ For example, if you want to set parameter ``connections_prefix`` to ``"airflow/c
     backend_kwargs = {"connections_prefix": "airflow/connections", "variables_prefix": null, "profile_name": "default"}
 
 Example of storing Google Secrets in AWS Secrets Manager
-""""""""""""""""""""""""""""""""""""""""""""""""""""""""
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 For connecting to a google cloud conn, all the fields must be in the extra field, and their names follow the pattern
 ``extra_google_cloud_platform__value``. For example:
 
diff --git a/tests/providers/amazon/aws/secrets/test_secrets_manager.py b/tests/providers/amazon/aws/secrets/test_secrets_manager.py
index 1488ffe3b2..5d875b69d8 100644
--- a/tests/providers/amazon/aws/secrets/test_secrets_manager.py
+++ b/tests/providers/amazon/aws/secrets/test_secrets_manager.py
@@ -52,14 +52,14 @@ class TestSecretsManagerBackend:
         assert "postgresql://airflow:airflow@host:5432/airflow" == returned_uri
 
     @pytest.mark.parametrize(
-        "full_url_mode, login, host",
+        "are_secret_values_urlencoded, login, host",
         [
-            (False, "is url encoded", "not%20idempotent"),
-            (True, "is%20url%20encoded", "not%2520idempotent"),
+            (True, "is url encoded", "not%20idempotent"),
+            (False, "is%20url%20encoded", "not%2520idempotent"),
         ],
     )
     @mock_secretsmanager
-    def test_get_connection_broken_field_mode_url_encoding(self, full_url_mode, login, host):
+    def test_get_connection_broken_field_mode_url_encoding(self, are_secret_values_urlencoded, login, host):
         secret_id = "airflow/connections/test_postgres"
         create_param = {
             "Name": secret_id,
@@ -78,16 +78,13 @@ class TestSecretsManagerBackend:
             ),
         }
 
-        secrets_manager_backend = SecretsManagerBackend(full_url_mode=full_url_mode)
+        secrets_manager_backend = SecretsManagerBackend(
+            are_secret_values_urlencoded=are_secret_values_urlencoded
+        )
         secrets_manager_backend.client.create_secret(**create_param)
         secrets_manager_backend.client.put_secret_value(**param)
 
-        if full_url_mode:
-            conn = secrets_manager_backend.get_connection(conn_id="test_postgres")
-        else:
-            warning_match = r"When full_url_mode=False, URL-encoding secret values is deprecated\..+"
-            with pytest.warns(DeprecationWarning, match=warning_match):
-                conn = secrets_manager_backend.get_connection(conn_id="test_postgres")
+        conn = secrets_manager_backend.get_connection(conn_id="test_postgres")
 
         assert conn.login == login
         assert conn.password == "not url encoded"
@@ -139,13 +136,9 @@ class TestSecretsManagerBackend:
         secrets_manager_backend.client.create_secret(**create_param)
         secrets_manager_backend.client.put_secret_value(**param)
 
-        warning_match = (
-            r"In future versions, `SecretsManagerBackend\.get_conn_value` will return a JSON string when "
-            r"full_url_mode is False, not a URI\."
-        )
-        with pytest.warns(DeprecationWarning, match=warning_match):
-            returned_uri = secrets_manager_backend.get_conn_value(conn_id="test_postgres")
-        assert "postgresql://airflow:airflow@host:5432/airflow" == returned_uri
+        conn = secrets_manager_backend.get_connection(conn_id="test_postgres")
+        returned_uri = conn.get_uri()
+        assert "postgres://airflow:airflow@host:5432/airflow" == returned_uri
 
     @mock_secretsmanager
     def test_get_conn_value_broken_field_mode_extra_words_added(self):
@@ -166,23 +159,9 @@ class TestSecretsManagerBackend:
         secrets_manager_backend.client.create_secret(**create_param)
         secrets_manager_backend.client.put_secret_value(**param)
 
-        warning_match = (
-            r"In future versions, `SecretsManagerBackend\.get_conn_value` will return a JSON string when "
-            r"full_url_mode is False, not a URI\."
-        )
-        with pytest.warns(DeprecationWarning, match=warning_match):
-            returned_uri = secrets_manager_backend.get_conn_value(conn_id="test_postgres")
-        assert "postgresql://airflow:airflow@host:5432/airflow" == returned_uri
-
-    @mock_secretsmanager
-    def test_format_uri_with_extra(self):
-        secret = {"extra": '{"key1": "value1", "key2": "value2"}'}
-        conn_string = "CS"
-        secrets_manager_backend = SecretsManagerBackend()
-
-        conn_string_with_extra = secrets_manager_backend._format_uri_with_extra(secret, conn_string)
-
-        assert conn_string_with_extra == "CS?key1=value1&key2=value2"
+        conn = secrets_manager_backend.get_connection(conn_id="test_postgres")
+        returned_uri = conn.get_uri()
+        assert "postgres://airflow:airflow@host:5432/airflow" == returned_uri
 
     @mock_secretsmanager
     def test_get_conn_value_non_existent_key(self):