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 2022/10/24 02:27:24 UTC

[GitHub] [airflow] Taragolis opened a new pull request, #23560: Add advanced secrets backend configurations

Taragolis opened a new pull request, #23560:
URL: https://github.com/apache/airflow/pull/23560

   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   At that moment Airflow have only predefined order for secrets backend
   1. Alternative secrets backend (if defined)
   2. Environment Variables Secrets backend
   3. Metastore Backend
   
   ~This PR add ability to change priority for Alternative secrets backend in relative of Environment Variables and Metastore~
   ~So it make possible for check variables/connections in Environment Variables / Metastore before Alternative secrets backend~
   
   
   This PR add advanced configurations for Secrets Backends which allow
   
   * Configure more than one alternative secrets backend.
   * Change search ordering of secrets backends.
   * Turn off built-in backends (environment variables or the metastore database).
   
   By providing in `[secrets] backends_config` JSON configuration 
   
   ```json
   [
     {
       "backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"
     },
     {
       "backend":"airflow.secrets.local_filesystem.LocalFilesystemBackend",
       "backend_kwargs":{
         "variables_file_path":"/local_backend/variables.yaml",
         "connections_file_path":"/local_backend/connections.yaml",
         "profile_name":"default"
       }
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/airflow/connections",
         "variables_prefix": "/airflow/variables",
         "profile_name":"default"
       }
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/another/connections",
         "variables_prefix": "/another/variables",
         "profile_name":"awesome",
         "region_name": "ap-southeast-1"
       }
     },
     {
       "backend":"airflow.secrets.metastore.MetastoreBackend"
     }
   ]
   ```
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   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 a newsfragement file, named `{pr_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1272662947

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] commented on pull request #23560: Add advanced secrets backend configurations

Posted by github-actions.
github-actions[bot] commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1402861209

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r869192353


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -38,8 +38,16 @@ Search path
 When looking up a connection/variable, 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.
+.. versionchanged:: 2.3.1

Review Comment:
   No problem I will change to 2.4.0. 
   
   I confused that some of the features added in the path versions (2.2.4, 2.2.5, etc) however I didn't investigate much, probably it was some bug fixes



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1162287832

   ok @Taragolis i thought about it some more.  
   
   i am thinking maybe we're better off stcking closer to your first approach and just making search path orderable.  
   
   i think originally the problem concern was with the language "low / medium / high" which did not have an obvious meaning.  so we suggested making it take an abritrary list.  but as you have pointed out, there are additional complexities, and i think these may tip the balance back to the earlier approach.  namely, airflow provides support for grabbing certain "secret" config values from secrets backend.  and this logic assumes there's _one_ custom secrets backend.  and given that, as well as the difficulties you've encountered with trying to implement deprecation of the old values, it seems it may not be worth the complexity tradeoff to go with the "arbitrary" approach, and instead go the more limited route.
   
    i think that simply being able to provide an ordering will itself be a huge value.  and for those users who really need to implement multiple external secrets backends, they can still do so by adding a custom secrets backend that hits multiple sources.
   
   so i would suggest let's just provide some mechanism for ordering.  i can think of two approaches.  probably the simplest is to add an optional key `backend_ordering_sequence` which could take a value in `[0, 1, 2]`.  alternatively, it could be an optional key `search_path` which would take a comma-separated `List[str]` of class names  e.g. `SSMParameterStoreBackend, MetastoreBackend, EnvVariablesBackend`.
   
   thanks


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913089337


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):

Review Comment:
   Why do we need this class? Is not just normal "list" enough?



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
ashb commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1176023187

   Thanks for sticking with it @Taragolis !


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1153575675

   After a bit of time I've change initial implementation to customisable configuration.
   
   Might be so customisable, so user can turn off Metastore DB and Env Variables as secrets backend. It could be easily reverted to behaviour, that if user not explicitly set this backend (for changing order) it would implicit add in the end of backend list
   
   cc: @potiuk @kaxil @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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r870419105


##########
airflow/configuration.py:
##########
@@ -1260,18 +1260,28 @@ def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
 def initialize_secrets_backends() -> List[BaseSecretsBackend]:
     """
     * import secrets backend classes
+    * sort by prioritisation
     * instantiate them and return them in a list
     """
     backend_list = []
 
+    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
+        secrets_backend_cls = import_string(class_name)
+        backend_list.append(secrets_backend_cls())
+
     custom_secret_backend = get_custom_secret_backend()
 
     if custom_secret_backend is not None:
-        backend_list.append(custom_secret_backend)
+        backend_priority = conf.get('secrets', 'backend_priority', fallback='') or 'HIGH'
 
-    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
-        secrets_backend_cls = import_string(class_name)
-        backend_list.append(secrets_backend_cls())
+        # Do enum validation here, because other enums checks run after initialise custom secrets backends
+        if backend_priority not in CustomSecretsBackendPriority.priorities():
+            raise AirflowConfigException(
+                f"`['secrets'] backend_priority` should not be {backend_priority!r}. "
+                f"Possible values: {', '.join(CustomSecretsBackendPriority.priorities())}."
+            )

Review Comment:
   ```suggestion
           if backend_priority not in CustomSecretsBackendPriority.__members__:
               raise AirflowConfigException(
                   f"`['secrets'] backend_priority` should not be {backend_priority!r}. "
                   f"Possible values: {', '.join(CustomSecretsBackendPriority.__members__)}."
               )
   ```



##########
airflow/secrets/__init__.py:
##########
@@ -22,10 +22,26 @@
     * Metastore database
     * AWS SSM Parameter store
 """
-__all__ = ['BaseSecretsBackend', 'DEFAULT_SECRETS_SEARCH_PATH']
+__all__ = ['BaseSecretsBackend', 'DEFAULT_SECRETS_SEARCH_PATH', "CustomSecretsBackendPriority"]
+from enum import IntEnum
+from functools import lru_cache
 
 from airflow.secrets.base_secrets import BaseSecretsBackend
 
+
+class CustomSecretsBackendPriority(IntEnum):
+    """Custom Secrets Backend Priority"""
+
+    HIGH = 0
+    MEDIUM = 1
+    LOW = 2
+
+    @classmethod
+    @lru_cache(maxsize=None)
+    def priorities(cls):
+        return {e.name for e in cls}

Review Comment:
   an we can get rid of 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901030614


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Secrets Backends itself this is exactly "two ways of doing things" - it initialised outside of configuration and it ignores all validations, most possible for avoid Chicken or the egg. So it is already kind of exception in airflow.configurations
   
   The idea of this PR extends current implementation Secrets Backend - make it more flexible, not rewrite entire configurations of Airflow and deprecation model.
   
   My current proposal it is focus on implementation of Advanced Secrets Backend rather that try solve all issues of configurations related only to SB. Or you think that this actually blocker?



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r900855610


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   I agree that better support only one way of configuration secrets backends.
   However I think old way in any cases should be supported until it entirely remove from Airflow (3.0.0 ?) 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r914257501


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -73,6 +74,47 @@ the example below.
     $ airflow config get-value secrets backend
     airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
 
+Advanced Configuration
+""""""""""""""""""""""
+
+.. versionadded:: 2.4.0
+
+You can provide ``backends_config`` with json-list which contains json-objects of required backends for:
+
+* Configure more than one alternative secrets backend.
+* Change search ordering of secrets backends.
+* Turn off built-in backends (environment variables or the metastore database).
+
+Object should contains mandatory **backend** attribute with fully qualified class name of the backend you want to enable
+and optionally contains **backend_kwargs** with json-object which will be passed as kwargs to the ``__init__``
+method of your secrets backend.
+
+Here's a basic example of how to configure to use Environment Variables first,
+next :ref:`SSM Parameter Store <ssm_parameter_store_secrets>`, and then metastore.:
+
+.. code-block:: json
+
+    [
+      {
+        "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
+      },
+      {
+        "backend": "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
+        "backend_kwargs": {
+          "connections_prefix": "/airflow/connections",
+          "profile_name": "default"
+        }
+      },
+      {
+        "backend": "airflow.secrets.metastore.MetastoreBackend"
+      }
+    ]
+
+.. warning::
+
+    If you specify ``backends_config`` options than Airflow will ignore values
+    in ``backend`` and ``backend_kwargs`` options.

Review Comment:
   If I configure `AIRFLOW__CORE__SQL_ALCHEMY_CONN` and `AIRFLOW__DATABASE__SQL_ALCHEMY_CONN` in airflow 2.3.x then `AIRFLOW__CORE__SQL_ALCHEMY_CONN` would ignored, am I right?
   
   Anyway if we decide deprecate old values in some new version then better remove information about old configuration with warning if user use old configurations.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913105349


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   Oh... default value for current implementation - "not defined", null or None and fallback to current implementation.
   
   However if we decide to keep only one way to configure backends in this case we might set as default:
   ```ini
   backends_config = [{"backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"}, {"backend":"airflow.secrets.metastore.MetastoreBackend"}]
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1173975828

   yeah @potiuk I suggested that if we add this new way of configuring, we have to deprecate the old way.  @Taragolis has indicated that doing so is difficult / problematic / infeasible.  and there is the chicken egg problem.  and then there is the issue where if you're searching external backend for config values, well if you can support multiple multiple external backends that gets a bit messy.   all of which has brought be to the thinking that we ought to keep it simple and just make the search path orderable, more similar to the original PR.  really, the big win here will be being able to check env vars first.  _Indeed, maybe we should have done that from the beginning (env var -> external -> metastore)... but anyway..._  and if you really need multiple external backends you can wrap them in custom class.  that's my thinking anyway.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913580713


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -73,6 +74,47 @@ the example below.
     $ airflow config get-value secrets backend
     airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
 
+Advanced Configuration
+""""""""""""""""""""""
+
+.. versionadded:: 2.4.0
+
+You can provide ``backends_config`` with json-list which contains json-objects of required backends for:
+
+* Configure more than one alternative secrets backend.
+* Change search ordering of secrets backends.
+* Turn off built-in backends (environment variables or the metastore database).
+
+Object should contains mandatory **backend** attribute with fully qualified class name of the backend you want to enable
+and optionally contains **backend_kwargs** with json-object which will be passed as kwargs to the ``__init__``
+method of your secrets backend.
+
+Here's a basic example of how to configure to use Environment Variables first,
+next :ref:`SSM Parameter Store <ssm_parameter_store_secrets>`, and then metastore.:
+
+.. code-block:: json
+
+    [
+      {
+        "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
+      },
+      {
+        "backend": "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
+        "backend_kwargs": {
+          "connections_prefix": "/airflow/connections",
+          "profile_name": "default"
+        }
+      },
+      {
+        "backend": "airflow.secrets.metastore.MetastoreBackend"
+      }
+    ]

Review Comment:
   (This is very much a suggestion/talking point, not a "please do it this way")



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901003363


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   This probably it not a big deal actually. Need to check it a bit later. 
   Seems like when secret backends initialised first time it can't get any values, so it initialised with default backend



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r900879673


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:

Review Comment:
   This helper class for
   
   - read single config of backend
      - separate classmethod for current configuration
      - separate classmethod for other cases
   - compare backends config
   - initialise backend class



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901002985


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Approach in general fine only one thing that could be a problem to do it in the same way for secrets backend
   
   https://github.com/apache/airflow/blob/b692517ce3aafb276e9d23570e9734c30a5f3d1f/airflow/configuration.py#L1600-L1602
   
   Secrets Backend initialised before config validated



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124051714

   I don't think this is the right way to do this.  The enums don't really have any intuitive connection to the priority.  Why don't we just allow users to optionally specify a list of backend classes (which would imply a search ordering).
   
   And if only one class is supplied, then  we just insert it in the front.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901817872


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:
+    """Secrets Backend Config dataclass helper."""
+
+    backend: str
+    backend_kwargs: Dict[str, Any] = field(default_factory=dict)
+
+    @classmethod
+    def from_dict(cls, d: Dict[str, Any]) -> 'SecretsBackendConfig':
+        """
+        Read Secret Backend Config from dictionary
+
+        Ignores all unexpected keywords
+        """
+        return cls(d['backend'], d.get('backend_kwargs', {}))
 
+    @classmethod
+    def from_config(cls) -> Optional['SecretsBackendConfig']:
+        """Try to get ``SecretsBackendConfig`` from airflow config [secrets] section"""
+        secrets_backend = conf.get(section='secrets', key='backend', fallback=None)
 
-def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
-    """Get Secret Backend if defined in airflow.cfg"""
-    secrets_backend_cls = conf.getimport(section='secrets', key='backend')
+        if not secrets_backend:
+            return None
 
-    if secrets_backend_cls:
         try:
-            backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
-            alternative_secrets_config_dict = json.loads(backends)
+            secrets_backend_kwargs: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
+            secrets_backend_kwargs = json.loads(secrets_backend_kwargs)
         except JSONDecodeError:
-            alternative_secrets_config_dict = {}
+            secrets_backend_kwargs = {}
+
+        return cls(secrets_backend, secrets_backend_kwargs)
+
+    def initialize(self) -> BaseSecretsBackend:
+        """Initialize Secrets Backend."""
+        return import_string(self.backend)(**self.backend_kwargs)
+
+
+class UniqueSecretsBackendsConfigs(UserList):

Review Comment:
   I can't think of any valid reason why a user would want to specify duplicates of (class, kwargs) so it feels like we should treat that case as an error and throw an exception during validate time.
   
   And since this is used in exactly one location, a custom class here feels like huge overkill -- essentially all we need is `if len(set(backends)) != len(backends): raise AirflowConfigException`.



##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -73,6 +74,47 @@ the example below.
     $ airflow config get-value secrets backend
     airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
 
+Advanced Configuration
+""""""""""""""""""""""
+
+.. versionadded:: 2.4.0
+
+You can provide ``backends_config`` with json-list which contains json-objects of required backends for:
+
+* Configure more than one alternative secrets backend.
+* Change search ordering of secrets backends.
+* Turn off built-in backends (environment variables or the metastore database).
+
+Object should contains mandatory **backend** attribute with fully qualified class name of the backend you want to enable
+and optionally contains **backend_kwargs** with json-object which will be passed as kwargs to the ``__init__``
+method of your secrets backend.
+
+Here's a basic example of how to configure to use Environment Variables first,
+next :ref:`SSM Parameter Store <ssm_parameter_store_secrets>`, and then metastore.:
+
+.. code-block:: json
+
+    [
+      {
+        "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
+      },
+      {
+        "backend": "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
+        "backend_kwargs": {
+          "connections_prefix": "/airflow/connections",
+          "profile_name": "default"
+        }
+      },
+      {
+        "backend": "airflow.secrets.metastore.MetastoreBackend"
+      }
+    ]

Review Comment:
   How about this as a simpler-for-users syntax:
   
   
   ```suggestion
       [
         "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
         [
           "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
           {
             "connections_prefix": "/airflow/connections",
             "profile_name": "default"
           }
         ],
         "airflow.secrets.metastore.MetastoreBackend"
       ]
   ```



##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   There is precedent for handling deprecations specially, checkout https://github.com/apache/airflow/blob/05c542dfa8eee9b4cdca4e9370f459ce807354b2/airflow/configuration.py#L338-L362
   
   So deprecate_config doesn't have to handle this if it's a more complex situation.
   
   



##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -73,6 +74,47 @@ the example below.
     $ airflow config get-value secrets backend
     airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
 
+Advanced Configuration
+""""""""""""""""""""""
+
+.. versionadded:: 2.4.0
+
+You can provide ``backends_config`` with json-list which contains json-objects of required backends for:
+
+* Configure more than one alternative secrets backend.
+* Change search ordering of secrets backends.
+* Turn off built-in backends (environment variables or the metastore database).
+
+Object should contains mandatory **backend** attribute with fully qualified class name of the backend you want to enable
+and optionally contains **backend_kwargs** with json-object which will be passed as kwargs to the ``__init__``
+method of your secrets backend.
+
+Here's a basic example of how to configure to use Environment Variables first,
+next :ref:`SSM Parameter Store <ssm_parameter_store_secrets>`, and then metastore.:
+
+.. code-block:: json
+
+    [
+      {
+        "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
+      },
+      {
+        "backend": "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
+        "backend_kwargs": {
+          "connections_prefix": "/airflow/connections",
+          "profile_name": "default"
+        }
+      },
+      {
+        "backend": "airflow.secrets.metastore.MetastoreBackend"
+      }
+    ]
+
+.. warning::
+
+    If you specify ``backends_config`` options than Airflow will ignore values
+    in ``backend`` and ``backend_kwargs`` options.

Review Comment:
   Silently ignoring config is a recipie for confusion -- we should make this an error.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913081767


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I think it woudl be nice to add the default configuration as comment. It would give the user an opportunity to figure out how to do it without reaching to documentation. 
   
   There are two kinds of users, those who love reading the docs, and those who prefer to explore. Comment (or even default value written here) would actually be nice one for the latter group.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r914254348


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -73,6 +74,47 @@ the example below.
     $ airflow config get-value secrets backend
     airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
 
+Advanced Configuration
+""""""""""""""""""""""
+
+.. versionadded:: 2.4.0
+
+You can provide ``backends_config`` with json-list which contains json-objects of required backends for:
+
+* Configure more than one alternative secrets backend.
+* Change search ordering of secrets backends.
+* Turn off built-in backends (environment variables or the metastore database).
+
+Object should contains mandatory **backend** attribute with fully qualified class name of the backend you want to enable
+and optionally contains **backend_kwargs** with json-object which will be passed as kwargs to the ``__init__``
+method of your secrets backend.
+
+Here's a basic example of how to configure to use Environment Variables first,
+next :ref:`SSM Parameter Store <ssm_parameter_store_secrets>`, and then metastore.:
+
+.. code-block:: json
+
+    [
+      {
+        "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend"
+      },
+      {
+        "backend": "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
+        "backend_kwargs": {
+          "connections_prefix": "/airflow/connections",
+          "profile_name": "default"
+        }
+      },
+      {
+        "backend": "airflow.secrets.metastore.MetastoreBackend"
+      }
+    ]

Review Comment:
   One benefit with dictionary (json-object) we could add additional configuration and do not worry about actual position in list, however side effect a bit longer json-string in configuration which could make users unhappy.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1227596908

   > @potiuk I'm worried about that if we add this part right now that we might add additional workaround on secrets backend.
   > 
   > Might be better try to proper configure Secrets Backends first?
   
   Right. Better. Agree.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913081767


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I think it woudl be nice to add the default configuration in config (either as comment or (better) as the config itself. It would give the user an opportunity to figure out how to do it without reaching to documentation. 
   
   There are two kinds of users, those who love reading the docs, and those who prefer to explore. Comment (or even default value written here) would actually be nice one for the latter group.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913105349


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   Oh... default value for this PR implementation - "not defined", null or None and fallback to current implementation.
   
   However if we decide to keep only one way to configure backends in this case we might set as default:
   ```ini
   backends_config = [{"backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"}, {"backend":"airflow.secrets.metastore.MetastoreBackend"}]
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1173940941

   Sorry @dstandish @Taragolis I know I promised to take a look before :( . 
   
   Just one comment and question - I amnot sure if the extra class is needed at all? I think maybe  because I do not understand a sequence of initialization. I think this is (historically) a bit convoluted so maybe  it might be good to posisbly explain this sequence here @Taragolis in a comment, i.e what happens in what sequnce ? I think this might save us some mental effort in the future when we decide to unbundle Secrets from config (It's kinda chicken-egg currently as some configs can be read from secrets and secret config is read from .. config  and I still have not wrapped my head around how to decouple those two). 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1125353155

   I am very much in favour of the last proposal by @dstandish.
   
   The Low/High/Medium is super unintuitive, and we might as well make it explicit and generic as Daniel explained.  I would even modify it slightly to simply make it single json array with both - classes and params:
   
   ```
   secrets = [
      {
         "backend": "airflow.smth.Backend", 
        "init-kwargs": { 
             "arg1": "",
             "arg2: True,
            .....
         } 
   ]
   ```
   
   This is explicit, localized (args next to class), fully customizable and much easier to reason about.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901133246


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Nothing personal but let me explain how it looks like for my side. I invest some my time to check how it possible to do it right now with same approach and found that withouts hacks in runtime it is hardly possible but you've told me:
   
   > So maybe this means you can't copy the existing "upgrade" examples exactly -- maybe you have to do something a little different, a little more difficult
   
   Which more like for me `I don't know how to do it, but think better and do it better` - that is last thing that I or someone else expects see on review (both on open source or commercial development) . If you have suggestion how to do it better right now without rewrite a lot of stuff - feel free to suggest.
   
   ---
   
   Right now Secrets Backends ignore "better approach", implicit upgrades values and do not track it. I think everyone sleep well
   
   https://github.com/apache/airflow/blob/2936759256273a66cf6343e451d81013a95ec47e/airflow/configuration.py#L1506-L1510
   
   ---
   
   Just for reduce your or someone else time for investigate how to do it "in right way right now".
   
   ### Specify directly AirflowConfigParser.deprecated_options
   
   You can't just placed in AirflowConfigParser.deprecated_options something like `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` because we need to check that **[secrets] backend** defined and upgrades required.
   
   ```python
   
       def _upgrade_secrets_backend(self):
           old_backend = self.get(section='secrets', key='backend', fallback='')
           if old_backend:
               ...
   ```
   
   Code above always will return warning even if user do not define anything in **[secrets] backend**
   _DeprecationWarning: The backend option in [secrets] has been renamed to backends_config - the old setting has been used, but please update your config._
   
   _Possible solution?_ Define in deprecation config `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` right after upgrade. Which might be unclear for developers, because they should keep it in their mind that something outside could change AirflowConfigParser.deprecated_options
   
   ### Deprecate [secrets] backend_kwargs
   
   It is not possible right now by define something in AirflowConfigParser.
   
   _Possible solution?_ Show warning that this config doesn't use anymore and anywhere on upgrade. However right now we also could warn users with custom message on init secrets backend
   
   ### Warning show that only one option deprecated
   
   We need to inform user by this message
   ```
   DeprecationWarning: The backend and backend_kwargs options in [secrets] has been moved to backends_config - the old setting has been used, but please update your config.
   ```
   
   But now we can get only predefined
   ```python
               warnings.warn(
                   f'The {deprecated_name} option in [{deprecated_section}] has been moved to the {key} option '
                   f'in [{section}] - the old setting has been used, but please update your config.',
                   DeprecationWarning,
                   stacklevel=3,
               )
   ```
   
   _Possible solution?_ Rewrite AirflowConfigParser.deprecated_options and methods which check and show this warning to make it more generic.
   
   ---
   
   I'm absolutely agree with your idea that we need to have only one approach with deprecated values.
   But right now Secrets Backend doesn't use this approach in anyway and without changes in AirflowConfigParser it is hardly possible.
   
   And this PR only about extends functional of Secrets Backend not changes in AirflowConfigParser. If we want to change AirflowConfigParser better to open separate PR in this case if we need to revert back changes we would revert "Add advanced secrets backend configurations" or "Make AirflowConfigParser more generic with deprecation  warning" but not both.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1176156917

   > Thanks for sticking with it @Taragolis !
   
   Yep. Persistency is often key to success in OSS :)


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] boring-cyborg[bot] commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1120238635

   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] kaxil commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
kaxil commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1125864832

   >Only one tricky things, probably we need to some how define is secrets backend support store configurations. For example:
   airflow.secrets.environment_variables.EnvironmentVariablesBackend and airflow.secrets.metastore.MetastoreBackend doesn't support by it by design, and we probably need to skip it.
   
   Yup, making it explicit in docs might be sufficient but showing a warning somewhere in logs/webserver might be even better


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913113058


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):

Review Comment:
   Unfortunately not enough in this PR implementation (just what we have right now). It is fine for current implementation which assume that if length of backends list is `2` than backends not initialised 
   
   https://github.com/apache/airflow/blob/e2fd41f7b14adef2c3a88dde14d088b5ef93b460/airflow/configuration.py#L1499
   
   Might be better (just an idea) create separate class for initialise secrets backends which care about initialise backends itself and some additional methods inside this class, like this
   
   https://github.com/apache/airflow/blob/e2fd41f7b14adef2c3a88dde14d088b5ef93b460/airflow/configuration.py#L114-L127
   
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913080158


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,

Review Comment:
   Agree.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r900879673


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:

Review Comment:
   This helper class for
   
   - read single config of backend
      - separate classmethod for current configuration
      - separate classmethod for other cases
   - compare backends config
   - initialise backend class with config



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1164779057

   > airflow provides support for grabbing certain "secret" config values from secrets backend. and this logic assumes there's one custom secrets backend. and this logic assumes there's one custom secrets backend. and given that, as well as the difficulties you've encountered with trying to implement deprecation of the old values, it seems it may not be worth the complexity tradeoff to go with the "arbitrary" approach, and instead go the more limited route
   
   I add some changes when switch to the list. Now it works in the same way that secrets backends work for Variables and Connections. Env and Metastore just not implements `get_config` method so it always return None and try to next backend in the list
   https://github.com/apache/airflow/blob/68e0741b8cd0554948be4db7946d8c7afaa5b4d0/airflow/configuration.py#L125-L140
   
   >and for those users who really need to implement multiple external secrets backends, they can still do so by adding a custom secrets backend that hits multiple sources.
   
   It is not so long if compare to current. If we do not want to provide ability drop EnvVar and Metastore we could simply add 
   in L1603 `backend_configs.extend(default_configs)` and remove from L1602
   
   https://github.com/apache/airflow/blob/68e0741b8cd0554948be4db7946d8c7afaa5b4d0/airflow/configuration.py#L1601-L1605
   
   And it only add to the list if user not specified it yet
   
   current implementation
   ```ini
   [secrets]
   backend = "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend"
   backend_kwargs ={"connections_prefix":"/airflow/connections","variables_prefix":"/airflow/variables","profile_name":"default"}
   ```
   
   proposal
   ```ini
   [secrets]
   # It could be just backends
   backends_config = [{"backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend","backend_kwargs":{"connections_prefix":"/airflow/connections","variables_prefix":"/airflow/variables","profile_name":"default"}}]
   ```
   
   The only one difference if user now specify backend=airflow.secrets.metastore.MetastoreBackend than result
   now `[MetastoreBackend, EnvironmentVariablesBackend, MetastoreBackend]` with current proposal `[MetastoreBackend, EnvironmentVariablesBackend]`


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] fhoda commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
fhoda commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124266571

   @Taragolis are your users trying to change the preferred connection location for all the dags in an environment at once or just a specific DAG that they might be working on? 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] closed pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #23560: Add advanced secrets backend configurations
URL: https://github.com/apache/airflow/pull/23560


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1288325212

   Just wondering @Taragolis - should it really be closed? Are you still working on it ?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124357318

   @dstandish I thought about custom mixin secret backend which allow to use different existed secret backends initially. And found that implements prioritisation much easier rather than move all logic to newly created class. Especially I do not know how it would work with `_SECRET` configurations
   
   Example: need create secret backend which mixin Env and AWS SSM, what happen when we try to find `awesome_variable` or `super_connection_id` which only defined in Metastore:
   1. Check in ENV (Alternative Backend)
   2. Check in SSM (Alternative Backend)
   3. Check in ENV (Default Backend)
   4. Check in Metastore (Default Backend)
   
   Another sample, implement custom backend which mixin Env, Metastore, SSM (in this order), in this case we call if variable/connection undefined
   
   1. Check in ENV (Alternative Backend)
   2. Check in Metastore (Alternative Backend)
   3. Check in SSM (Alternative Backend)
   4. Check in ENV (Default Backend)
   5. Check in Metastore (Default Backend)
   
   About flexible / explicit I have probably couple questions:
   * Should it allow to users turn off some default backends at all?
   * Should it allow to users define more than one alternate backends?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r869145208


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -38,8 +38,16 @@ Search path
 When looking up a connection/variable, 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.
+.. versionchanged:: 2.3.1

Review Comment:
   Should be 2.4.0. New features are added to the next minor release



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r870415769


##########
airflow/configuration.py:
##########
@@ -1260,18 +1260,28 @@ def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
 def initialize_secrets_backends() -> List[BaseSecretsBackend]:
     """
     * import secrets backend classes
+    * sort by prioritisation
     * instantiate them and return them in a list
     """
     backend_list = []
 
+    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
+        secrets_backend_cls = import_string(class_name)
+        backend_list.append(secrets_backend_cls())
+
     custom_secret_backend = get_custom_secret_backend()
 
     if custom_secret_backend is not None:
-        backend_list.append(custom_secret_backend)
+        backend_priority = conf.get('secrets', 'backend_priority', fallback='') or 'HIGH'
 
-    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
-        secrets_backend_cls = import_string(class_name)
-        backend_list.append(secrets_backend_cls())
+        # Do enum validation here, because other enums checks run after initialise custom secrets backends
+        if backend_priority not in CustomSecretsBackendPriority.priorities():
+            raise AirflowConfigException(
+                f"`['secrets'] backend_priority` should not be {backend_priority!r}. "
+                f"Possible values: {', '.join(CustomSecretsBackendPriority.priorities())}."
+            )
+
+        backend_list.insert(CustomSecretsBackendPriority[backend_priority], custom_secret_backend)

Review Comment:
   I wonder if we should have a `conf.getenum` for this pattern. If not, an `upper` call here would save some unnecessary user frustration, at least.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r914251589


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Thanks for suggestion!
   I don't think I got how it actually work but I need check also this way.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] raphaelauv commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
raphaelauv commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1175365294

   Feel like it's getting over complicated
   
   We need 2 things 
   
   Be able to define multiple secret backend where we can set the default order
   
   BUT ALSO
   
   Be able to choose which secret backend is use depending the cases at code level , no declarative configuration will ever be enough flexible
   
   So we need new parameters at dag level
   
   default_dag_secret_backend for dag
   
   and
   
   default_operator_secret_backend for task
   
   That way things can stay magical at airflow configuration level
   
   And very precise at dag code level
   
   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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913107498


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I realized that we cannot do that actually -  because lack of the `backend_congigs` determines if we should use backend and backend_kwargs - by adding default we would break compatibiliyt if someone generates the default config and moves their "backend/backend_kwarg" - it's mabe not very proabable scenario but might happen



##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I realized that we cannot do that actually -  because lack of the `backend_configs` determines if we should use backend and backend_kwargs - by adding default we would break compatibiliyt if someone generates the default config and moves their "backend/backend_kwarg" - it's mabe not very proabable scenario but might happen



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1221434358

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1288629683

   @potiuk Not right now. I have a plan to have a break of my full-time projects in December so I would enough time work on this (or something similar like slightly rewrite conf -> secrets backends conf integration)
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] closed pull request #23560: Add advanced secrets backend configurations

Posted by github-actions.
github-actions[bot] closed pull request #23560: Add advanced secrets backend configurations
URL: https://github.com/apache/airflow/pull/23560


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901151998


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   > Which more like for me I don't know how to do it, but think better and do it better - that is last thing that I or someone else expects see on review (both on open source or commercial development) . If you have suggestion how to do it better right now without rewrite a lot of stuff - feel free to suggest.
   
   I'm just saying on the face of it, it seems it should be possible to update the config approach for secrets backend without keeping the old one live.  And no, I did not investigate exactly how you should do this.
   
   > I'm absolutely agree with your idea that we need to have only one approach with deprecated values.
   But right now Secrets Backend doesn't use this approach in anyway and without changes in AirflowConfigParser it is hardly possible.
   
   What i'm saying is that we should not maintain two config styles concurrently, that the old one should be deprecated, and the nuts and bolts of exactly how I'm less concerned with.
   
   I may be able to explore solutions next week.  In the meantime who knows maybe someone else will support your approach. 
   
   Thanks



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] raphaelauv commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
raphaelauv commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1120241297

   https://github.com/apache/airflow/pull/19332


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913123196


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
    > This might happen that someone will set the three of them, and backends_config should get precedence
   
   I can't find the good way how to do it right now with `AirflowConfigParser.deprecated_options`, because if we put this configs as deprecated and try to check is user actually put something in this config the result will - deprecation warning every time even if user do not use that.
   
   I think I do not try only one option - get this configs without fallback, in this case probably we only have to catch exception and no deprecation warnings raised in case if user do not set anything. It just my assuming.
   
   Another interesting moment it would a first time when one config option replace two options but right now all deprecation expected one-to-one 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913110782


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   HAving al those warnings will give a transition path for the users so that in 3 we could remove the old configs (and then default value for backend_config would make perfect sense.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r914260195


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:
+    """Secrets Backend Config dataclass helper."""
+
+    backend: str
+    backend_kwargs: Dict[str, Any] = field(default_factory=dict)
+
+    @classmethod
+    def from_dict(cls, d: Dict[str, Any]) -> 'SecretsBackendConfig':
+        """
+        Read Secret Backend Config from dictionary
+
+        Ignores all unexpected keywords
+        """
+        return cls(d['backend'], d.get('backend_kwargs', {}))
 
+    @classmethod
+    def from_config(cls) -> Optional['SecretsBackendConfig']:
+        """Try to get ``SecretsBackendConfig`` from airflow config [secrets] section"""
+        secrets_backend = conf.get(section='secrets', key='backend', fallback=None)
 
-def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
-    """Get Secret Backend if defined in airflow.cfg"""
-    secrets_backend_cls = conf.getimport(section='secrets', key='backend')
+        if not secrets_backend:
+            return None
 
-    if secrets_backend_cls:
         try:
-            backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
-            alternative_secrets_config_dict = json.loads(backends)
+            secrets_backend_kwargs: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
+            secrets_backend_kwargs = json.loads(secrets_backend_kwargs)
         except JSONDecodeError:
-            alternative_secrets_config_dict = {}
+            secrets_backend_kwargs = {}
+
+        return cls(secrets_backend, secrets_backend_kwargs)
+
+    def initialize(self) -> BaseSecretsBackend:
+        """Initialize Secrets Backend."""
+        return import_string(self.backend)(**self.backend_kwargs)
+
+
+class UniqueSecretsBackendsConfigs(UserList):

Review Comment:
   Actually I've tried to solve couple issues
   
   1. Set only unique values - I think no reason to check backend if it already added
   2. Inform user if this backend already exists.
   3. If we decide to deprecate old values (initially I think keep both), than it would be easy to add in the end default backend, and if user already add it - nothing bad happen
   
   Actually this class didn't solve second issue -there are no warnings in service logs. I think something suppress it.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1175574656

   @potiuk @ashb @dstandish I converted this PR to draft and I will focus on other things related to secrets backends, a do not when I found a time for that, I hope soon.
   
   And after that I think we could return to ordering/configuration Secrets Backends.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r900877600


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:
+    """Secrets Backend Config dataclass helper."""
+
+    backend: str
+    backend_kwargs: Dict[str, Any] = field(default_factory=dict)
+
+    @classmethod
+    def from_dict(cls, d: Dict[str, Any]) -> 'SecretsBackendConfig':
+        """
+        Read Secret Backend Config from dictionary
+
+        Ignores all unexpected keywords
+        """
+        return cls(d['backend'], d.get('backend_kwargs', {}))
 
+    @classmethod
+    def from_config(cls) -> Optional['SecretsBackendConfig']:
+        """Try to get ``SecretsBackendConfig`` from airflow config [secrets] section"""
+        secrets_backend = conf.get(section='secrets', key='backend', fallback=None)
 
-def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
-    """Get Secret Backend if defined in airflow.cfg"""
-    secrets_backend_cls = conf.getimport(section='secrets', key='backend')
+        if not secrets_backend:
+            return None
 
-    if secrets_backend_cls:
         try:
-            backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
-            alternative_secrets_config_dict = json.loads(backends)
+            secrets_backend_kwargs: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
+            secrets_backend_kwargs = json.loads(secrets_backend_kwargs)
         except JSONDecodeError:
-            alternative_secrets_config_dict = {}
+            secrets_backend_kwargs = {}
+
+        return cls(secrets_backend, secrets_backend_kwargs)
+
+    def initialize(self) -> BaseSecretsBackend:
+        """Initialize Secrets Backend."""
+        return import_string(self.backend)(**self.backend_kwargs)
+
+
+class UniqueSecretsBackendsConfigs(UserList):

Review Comment:
   This helper only remove backends with the same configuration. 
   I think there is no reason check again backend which return `None` again because in most case the result in most case would be same, only show warning that this Backend with exactly the same configuration already in list of configuration
   
   ---
   
   Example:
   
   ```json
   [
     {
       "backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/airflow/connections",
         "variables_prefix": "/airflow/variables",
         "profile_name":"default"
       }
     },
     {
       "backend":"airflow.secrets.metastore.MetastoreBackend"
     },
     {
       "backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/another/connections",
         "variables_prefix": "/another/variables",
         "profile_name":"awesome",
         "region_name": "ap-southeast-1"
       }
     },
     {
       "backend":"airflow.secrets.metastore.MetastoreBackend"
     }
   ]
   ```
   
   keep only this one 
   
   
   ```json
   [
     {
       "backend":"airflow.secrets.environment_variables.EnvironmentVariablesBackend"
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/airflow/connections",
         "variables_prefix": "/airflow/variables",
         "profile_name":"default"
       }
     },
     {
       "backend":"airflow.secrets.metastore.MetastoreBackend"
     },
     {
       "backend":"airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend",
       "backend_kwargs":{
         "connections_prefix":"/another/connections",
         "variables_prefix": "/another/variables",
         "profile_name":"awesome",
         "region_name": "ap-southeast-1"
       }
     }
   ]
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124201085

   @kaxil 
   > but I would like to know more about your use-case. Can you expand on it please?
   
   Major use cases - lazy developers :-D. Usual cases is local development and QA struggling
   
   In local development usual we want to use the same connections as  we use in remote dev and depend on project it could be 10 in other it could be 50. So we use secret backend to share between developers but as usual developer wanted re-define some certain connections (e.g. use their own snapshot of database). At that moment it really hardly possible, because secret backend always have a high priority.
   
   The current solutions:
   1. dump of secrets backend for remote dev -> change it manually -> create new prefix -> repeat as soon as sources in remote changed
   2. do not use secrets backends in remote dev / tests -> dump metadata db -> share across developers -> restore secrets -> repeat as sources in remote airflow changed
   
   For tests also the same, QA wanted to test in different environment (test, prod, pre-prod), but replace some connections
   
   Personally for me it would be nice if Environment Variables in search path have bigger priority but it is not possible to do this changes in core of Airflow, at least until Apache Airflow 3.y.z
   
   So my idea is give ability to change priority for secrets backends with minimum changes in Airflow and do not create breaking changes
   
   > Would [#19251 (comment)](https://github.com/apache/airflow/issues/19251#issuecomment-958992113) solve your issue? 
   
   I think it could solve but required to implement the same things in every new project. (I'm also lazy developer)
   
   @dstandish
   
   > I don't think this is the right way to do this.
   
   I agree with you and also think it not the best solution. First of all because my solution expected that:
   1. Exactly two default secrets backends
   2. Zero or one alternative secrets backend
   
   If in one day in Airflow this changed - solution which I suggest wouldn't work as 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124340443

   > @Taragolis are your users trying to change the preferred connection location for all the dags in an environment at once or just a specific DAG that they might be working on?
   
   In all DAGs in environment. However only for specific environments it applied.
   Last case
   * dev, staging, prod use their own connections (and variables) stored in secret backend and no connections in env and metabase
   * local-dev use connections from dev, except couple connections
   * pre-prod-test use connection from prod, except couple of them


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1125343762

   @Taragolis we should not care what backends users decide to use. they can specify the class name, and airflow will just import it, so it doesn't matter to airflow whether it's custom or built in.
   
   here's what i think is a simple and workable approach. 
   
   if it’s one class, insert at front; (this is current behavior, hence backward compatible)
   
   if list of classes, assume that you’re specifying all backends you want searched and in that order.
   
   it's a little trickier is to decide how to supply backend_kwargs.  but i think we can just require list of dicts.  and what we could do is, check the class list first, and if it is list of string, then enforce that backend kwargs must be list of dict and same number of elements.  if it's just a single class name (backcompat) then backend kwargs, if supplied, should be dictionary.
   
   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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r870692237


##########
airflow/configuration.py:
##########
@@ -1260,18 +1260,28 @@ def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
 def initialize_secrets_backends() -> List[BaseSecretsBackend]:
     """
     * import secrets backend classes
+    * sort by prioritisation
     * instantiate them and return them in a list
     """
     backend_list = []
 
+    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
+        secrets_backend_cls = import_string(class_name)
+        backend_list.append(secrets_backend_cls())
+
     custom_secret_backend = get_custom_secret_backend()
 
     if custom_secret_backend is not None:
-        backend_list.append(custom_secret_backend)
+        backend_priority = conf.get('secrets', 'backend_priority', fallback='') or 'HIGH'
 
-    for class_name in DEFAULT_SECRETS_SEARCH_PATH:
-        secrets_backend_cls = import_string(class_name)
-        backend_list.append(secrets_backend_cls())
+        # Do enum validation here, because other enums checks run after initialise custom secrets backends
+        if backend_priority not in CustomSecretsBackendPriority.priorities():
+            raise AirflowConfigException(
+                f"`['secrets'] backend_priority` should not be {backend_priority!r}. "
+                f"Possible values: {', '.join(CustomSecretsBackendPriority.priorities())}."
+            )
+
+        backend_list.insert(CustomSecretsBackendPriority[backend_priority], custom_secret_backend)

Review Comment:
   added `upper`



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] github-actions[bot] commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1344925482

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901024555


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   There are quite a few findings with use `deprecated_values` and `upgraded_values` for Secrets Backends
   
   1. We need to deprecate in once two configs `[secrets] backend` and `[secrets] backend_kwargs` by `[secrets] backends_config`. I think it is not possible right now.
   
   2. Current and new config expected different types. New JSON List, Current just a string. So warning would be 
   ```airflow/airflow/configuration.py:552: DeprecationWarning: The backend option in [secrets] has been renamed to backends_config - the old setting has been used, but please update your config.```
   
   3. We want to upgrade config so it should be done before
   https://github.com/apache/airflow/blob/b692517ce3aafb276e9d23570e9734c30a5f3d1f/airflow/configuration.py#L1601-L1602
   
   4. One side effect about transformation old value to new one, we need to call config.get(section='secrets', key='backends_config') and we don't know where this value stored in `backend` or in `backends_config`. 
   
   I would rather for now add Depreciation warning in `initialize_secrets_backends()` and after think about how to solve all findings which listed above
   https://github.com/apache/airflow/blob/68e0741b8cd0554948be4db7946d8c7afaa5b4d0/airflow/configuration.py#L1596-L1600
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913110114


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   One more thing - I think we shoudl raise a deprecation warning when backend/backend_kwargs is used and maybe even provide a detailed information how backends_config should look like in this case? We should have enough info to work-out what should be the replacement?
   
   Also another warning should be raised when 'backends_config` is defined and either of the `backend_kwargs`, `backend` are set. This **might** happen that someone will set the three of them, and backends_config should get precedence, but it should also raise warnig to the user to remove the two old settings - this might be a mistake on the side of configuring user, and in the worst case it should lead to cleaer and less ambiguous config eventually.
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1174010308

   @potiuk @dstandish tl;dr: After tried to change something in secrets backends I found that might be better to try make this mechanism more transparent first at least for developers and after that we could go back to backend order/configurations.
   I don't think that is possible completely solve chicken-egg unfortunately.
   
   ---
   
   In each case if we change order or add ability to configure multiple backends we got some false positive warnings.
   
   **Variables expected that Metastore always last when created Variable in UI**
   
   https://github.com/apache/airflow/blob/e2fd41f7b14adef2c3a88dde14d088b5ef93b460/airflow/models/variable.py#L226-L237
   
   ---
   
   Some additional findings
   
   **Connections and Variables have methods for lookup values which might be better to move in some SecretsBackendsClass**
   
   https://github.com/apache/airflow/blob/e2fd41f7b14adef2c3a88dde14d088b5ef93b460/airflow/models/variable.py#L257-L275
   
   https://github.com/apache/airflow/blob/e2fd41f7b14adef2c3a88dde14d088b5ef93b460/airflow/models/connection.py#L410-L430
   
   **List variables/connections use different approach rather than get**
   
   Get variables/connections stored in metastore - use `airflow.secrets.metastore.MetastoreBackend`
   List variables/connections outside of `airflow.secrets.metastore.MetastoreBackend`
   
   I think most of (probably all) secrets backends which implemented in airflow or in community providers could implemented List operation. Which make possible to show this variables/connections in UI/CLI
   
   **No information for end-user where received variable/connection actually stored**
   
   Personally spend additional time last Friday just because I assume that we do not create specific connection in secrets backend.
   
   **Implicit ignore for end-users any json.Decoding errors in backend_kwargs**
   
   `¯\_(ツ)_/¯`


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901001809


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   right and what i am telling you is that you need to initiate deprecation properly, using the framework we have for this kind of thing.   for example see here: https://github.com/apache/airflow/blob/main/airflow/configuration.py#L171
   
   the option for `base_log_folder` was moved from `core` to `logging` -- we don't keep the deprecated values around in the old location.  we only keep the new location.  and we handle "upgrading" deprecated approaches in AirflowConfigParser.  see also methods `_upgrade_auth_backends` and `_upgrade_postgres_metastore_conn`.   so it seems to me the right approach is to formally remove it from the config immediately as a deprecation and add the logic to "upgrade" or "convert" the deprecated legacy values.
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901104423


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   I'm not proposing `rewrite entire configurations of Airflow and deprecation model` or `solve all issues of configurations related only to SB`.  I'm essentially saying, when you want to change configuration style, you _change_ it; you don't add a _another_ style.
   
   I understand that with SB there are complexities with being also able to take "secret" config items from it, and I understand that unfortunately it's a bit chicken egg and confusing and it's not my favorite --- but it's already like that.  So maybe this means you can't copy the existing "upgrade" examples exactly -- maybe you have to do something a little different, a little more difficult.  But the fundamental principle here is not to "multiply" configuration approaches but to just _change_ the configuration approach, and to maintain backcompat with deprecation warning for a time.  I am pretty confident you can do that.
   
   But keep in mind, I'm not the boss, right?  This is open source.  And if you disagree with my thinking, and if no one else is chimes in here, one thing you can always do is start a discussion on the dev list, starting a thread like `[DISCUSS] To add or update secrets backend configuration approach` and you can state your case and see if you find support.
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901133246


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Nothing personal but let me explain how it looks like for my side. I invest some my time to check how it possible to do it right now with same approach and found that withouts hacks in runtime it is hardly possible but you've told me:
   
   > So maybe this means you can't copy the existing "upgrade" examples exactly -- maybe you have to do something a little different, a little more difficult
   
   Which more like for me `I don't know how to do it, but think better and do it better` - that is last thing that I or someone else expects see on review (both on open source or commercial development) . If you have suggestion how to do it better right now without rewrite a lot of stuff - feel free to suggest.
   
   ---
   
   Right now Secrets Backends ignore "better approach", implicit upgrades values and do not track it. I think everyone sleep well
   
   https://github.com/apache/airflow/blob/2936759256273a66cf6343e451d81013a95ec47e/airflow/configuration.py#L1506-L1510
   
   ---
   
   Just for reduce your or someone else time for investigate how to do it "in right way right now".
   
   ### Specify directly AirflowConfigParser.deprecated_options
   
   You can't just placed in AirflowConfigParser.deprecated_options something like `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` because we need to check that **[secrets] backend** defined and upgrades required.
   
   ```python
   
       def _upgrade_secrets_backend(self):
           old_backend = self.get(section='secrets', key='backend', fallback='')
           if old_backend:
               ...
   ```
   
   Code above always will return warning even if user do not define anything in **[secrets] backend**
   _DeprecationWarning: The backend option in [secrets] has been renamed to backends_config - the old setting has been used, but please update your config._
   
   _Possible solution?_ Define in deprecation config `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` right after upgrade. Which might be unclear for developers, because they should keep it in their mind that something outside could change AirflowConfigParser.deprecated_options
   
   ### Deprecate [secrets] backend_kwargs
   
   It is not possible right now by define something in AirflowConfigParser.
   
   _Possible solution?_ Show warning that this config doesn't use anymore and anywhere on upgrade. However right now we also could warn users with custom message on init secrets backend
   
   ### Waring show that only one option deprecated
   
   We need to inform user by this message
   ```
   DeprecationWarning: The backend and backend_kwargs options in [secrets] has been moved to backends_config - the old setting has been used, but please update your config.
   ```
   
   But now we can get only predefined
   ```
               warnings.warn(
                   f'The {deprecated_name} option in [{deprecated_section}] has been moved to the {key} option '
                   f'in [{section}] - the old setting has been used, but please update your config.',
                   DeprecationWarning,
                   stacklevel=3,
               )
   ```
   
   _Possible solution?_ Rewrite AirflowConfigParser.deprecated_options and methods which check and show this warning to make it more generic.
   
   ---
   
   I'm absolutely agree with your idea that we need to have only one approach with deprecated values.
   But right now Secrets Backend doesn't use this approach in anyway and without changes in AirflowConfigParser it is hardly possible.
   
   And this PR only about extends functional of Secrets Backend not changes in AirflowConfigParser. If we want to change AirflowConfigParser better to open separate PR in this case if we need to revert back changes we would revert "Add advanced secrets backend configurations" or "Make AirflowConfigParser more generic with deprecation  warning" but not both.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1121476593

   It's an interesting approach but I am not sure if this is the best approach. @kaxil  - 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
dstandish commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1124277005

   And perhaps i should add...  i fully support adding prioritization.  i just don't like "encoding" that prioritization as low / medium high -- that's the only part i think doesn't make sense, let's just be fully explicit and fully flexible about the ordering.
   
   I floated flexble ordering such as thes when i initially contributed secrets backend long ago.  But this is the way we went.  I think it makes sense to add the flexibility.
   
   In the meantime, yeah, as others have indicated, you can implement your own "composite" secrets backend.  It's not hard to do.  Your backend can first check in env vars then check external backend.  Then you have what you need.  I have done precisely this before at a company.
   
   Happy to support this if we can make it more explicit / flexible.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add prioritization for secrets backend

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1125798842

   @potiuk @dstandish 
   
   Nice idea! Just to clarify some moments
   
   ---
   
   1. We want to use list of dictionary, or just dictionary (json-object) because since Python 3.7 should keep initial order of dictionary keys (CPython since 3.6)?
   
   ```python
   secrets = {
       "awesome.secrets.Backend": {
           "key": "value",
           "key2": "another value",
       },
       "super.secrets.Backend": {},
       "airflow.secrets.environment_variables.EnvironmentVariablesBackend": {},
       "airflow.secrets.metastore.MetastoreBackend": {},
   }
   ```
   
   ---
   
   2. If user doesn't set up every alternative secrets backends than use default one?
   
   ```python
   secrets = {
       "airflow.secrets.environment_variables.EnvironmentVariablesBackend": {},
       "airflow.secrets.metastore.MetastoreBackend": {},
   }
   ```
   
   or
   
   ```python
   secrets = [
      {
         "backend": "airflow.secrets.environment_variables.EnvironmentVariablesBackend", 
      },
      {
         "backend": "airflow.secrets.metastore.MetastoreBackend", 
      },
   ]
   ```
   
   ---
   
   3. If user setup alternative secrets backend by current parameters
   ```ini
   [secrets]
   backend = awesome.secrets.Backend
   backend_kwargs = '{"key": "value", "key2": "another value"}'
   ```
   We need to convert to appropriate object with same current order than uses now: Alternative backend, Env Var, Metastore.
   
   But should we show some kind of depreciation warning?
   
   ---
   
   4. If user uses configurations which required get values from secret backend, e.g. `[database] sql_alchemy_conn_secret` we should try to [get config](https://github.com/apache/airflow/blob/2111d73a9277c8e036279f2cc59f146270ef8e5b/airflow/secrets/base_secrets.py#L141) in each secrets backend in same order that this backends defined until found value or raise an error in case of config not exists.
   
   Only one tricky things, probably we need to some how define is secrets backend support store configurations. For example: 
   `airflow.secrets.environment_variables.EnvironmentVariablesBackend` and `airflow.secrets.metastore.MetastoreBackend` doesn't support by it by design, and we probably need to skip it.
   
   ---
   
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1227336148

   @potiuk I'm worried about that if we add this part right now that we might add additional workaround on secrets backend.
   
   Might be better try to proper configure Secrets Backends first?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1227127534

   Hey @Taragolis - are you continuing this one ? I think there are quite a few users who would benefit from that one and if we could get it on time for 2.4.0 that would be fantastic :D


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913081767


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I think it woudl be nice to add the default way of configuration as config. It would give the user an opportunity to figure out how to do it without reaching to documentation. 
   
   There are two kinds of users, those who love reading the docs, and those who prefer to explore. Comment (or even default value written here would actually be nice one for the latter group.



##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   I think it woudl be nice to add the default way of configuration as config. It would give the user an opportunity to figure out how to do it without reaching to documentation. 
   
   There are two kinds of users, those who love reading the docs, and those who prefer to explore. Comment (or even default value written here) would actually be nice one for the latter group.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ashb commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r913581516


##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,
+# order of secrets backend and turn off built-in backends.
+# Expected JSON list of objects. See
+# https://airflow.apache.org/docs/apache-airflow/stable/security/secrets/secrets-backend/index.html#advanced-configuration
+# for more details.
+backends_config =

Review Comment:
   https://github.com/apache/airflow/pull/23560#discussion_r902364807



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
Taragolis commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901133246


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   Nothing personal but let me explain how it looks like for my side. I invest some my time to check how it possible to do it right now with same approach and found that withouts hacks in runtime it is hardly possible but you've told me:
   
   > So maybe this means you can't copy the existing "upgrade" examples exactly -- maybe you have to do something a little different, a little more difficult
   
   Which more like for me `I don't know how to do it, but think better and do it better` - that is last thing that I or someone else expects see on review (both on open source or commercial development) . If you have suggestion how to do it better right now without rewrite a lot of stuff - feel free to suggest.
   
   ---
   
   Right now Secrets Backends ignore "better approach", implicit upgrades values and do not track it. I think everyone sleep well
   
   https://github.com/apache/airflow/blob/2936759256273a66cf6343e451d81013a95ec47e/airflow/configuration.py#L1506-L1510
   
   ---
   
   Just for reduce your or someone else time for investigate how to do it "in right way right now".
   
   ### Specify directly AirflowConfigParser.deprecated_options
   
   You can't just placed in AirflowConfigParser.deprecated_options something like `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` because we need to check that **[secrets] backend** defined and upgrades required.
   
   ```python
   
       def _upgrade_secrets_backend(self):
           old_backend = self.get(section='secrets', key='backend', fallback='')
           if old_backend:
               ...
   ```
   
   Code above always will return warning even if user do not define anything in **[secrets] backend**
   _DeprecationWarning: The backend option in [secrets] has been renamed to backends_config - the old setting has been used, but please update your config._
   
   _Possible solution?_ Define in deprecation config `('secrets', 'backends_config'): ('secrets', 'backend', '2.4.0')` right after upgrade. Which might be unclear for developers, because they should keep it in their mind that something outside could change AirflowConfigParser.deprecated_options
   
   ### Deprecate [secrets] backend_kwargs
   
   It is not possible right now by define something in AirflowConfigParser.
   
   _Possible solution?_ Show warning that this config doesn't use anymore and anywhere on upgrade. However right now we also could warn users with custom message on init secrets backend
   
   ### Waring show that only one option deprecated
   
   We need to inform user by this message
   ```
   DeprecationWarning: The backend and backend_kwargs options in [secrets] has been moved to backends_config - the old setting has been used, but please update your config.
   ```
   
   But now we can get only predefined
   ```python
               warnings.warn(
                   f'The {deprecated_name} option in [{deprecated_section}] has been moved to the {key} option '
                   f'in [{section}] - the old setting has been used, but please update your config.',
                   DeprecationWarning,
                   stacklevel=3,
               )
   ```
   
   _Possible solution?_ Rewrite AirflowConfigParser.deprecated_options and methods which check and show this warning to make it more generic.
   
   ---
   
   I'm absolutely agree with your idea that we need to have only one approach with deprecated values.
   But right now Secrets Backend doesn't use this approach in anyway and without changes in AirflowConfigParser it is hardly possible.
   
   And this PR only about extends functional of Secrets Backend not changes in AirflowConfigParser. If we want to change AirflowConfigParser better to open separate PR in this case if we need to revert back changes we would revert "Add advanced secrets backend configurations" or "Make AirflowConfigParser more generic with deprecation  warning" but not both.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r900491214


##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:
+    """Secrets Backend Config dataclass helper."""
+
+    backend: str
+    backend_kwargs: Dict[str, Any] = field(default_factory=dict)
+
+    @classmethod
+    def from_dict(cls, d: Dict[str, Any]) -> 'SecretsBackendConfig':
+        """
+        Read Secret Backend Config from dictionary
+
+        Ignores all unexpected keywords
+        """
+        return cls(d['backend'], d.get('backend_kwargs', {}))
 
+    @classmethod
+    def from_config(cls) -> Optional['SecretsBackendConfig']:
+        """Try to get ``SecretsBackendConfig`` from airflow config [secrets] section"""
+        secrets_backend = conf.get(section='secrets', key='backend', fallback=None)
 
-def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
-    """Get Secret Backend if defined in airflow.cfg"""
-    secrets_backend_cls = conf.getimport(section='secrets', key='backend')
+        if not secrets_backend:
+            return None
 
-    if secrets_backend_cls:
         try:
-            backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
-            alternative_secrets_config_dict = json.loads(backends)
+            secrets_backend_kwargs: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
+            secrets_backend_kwargs = json.loads(secrets_backend_kwargs)
         except JSONDecodeError:
-            alternative_secrets_config_dict = {}
+            secrets_backend_kwargs = {}
+
+        return cls(secrets_backend, secrets_backend_kwargs)
+
+    def initialize(self) -> BaseSecretsBackend:
+        """Initialize Secrets Backend."""
+        return import_string(self.backend)(**self.backend_kwargs)
+
+
+class UniqueSecretsBackendsConfigs(UserList):

Review Comment:
   do we really need this? if user provides duplicate backends, can we just let it be however they've configured it?



##########
airflow/config_templates/default_airflow.cfg:
##########
@@ -422,6 +422,13 @@ backend =
 # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
 backend_kwargs =
 
+# Advanced secrets backend configuration, allow to user configure more than one secret backend,

Review Comment:
   same comment as elsewhere; better have just one way then "a way" and "an advanced way"



##########
airflow/configuration.py:
##########
@@ -1487,47 +1500,118 @@ def set(*args, **kwargs) -> None:
     conf.set(*args, **kwargs)
 
 
-def ensure_secrets_loaded() -> List[BaseSecretsBackend]:
-    """
-    Ensure that all secrets backends are loaded.
-    If the secrets_backend_list contains only 2 default backends, reload it.
-    """
-    # Check if the secrets_backend_list contains only 2 default backends
-    if len(secrets_backend_list) == 2:
-        return initialize_secrets_backends()
-    return secrets_backend_list
+class DefaultSecretsBackend(UserList):
+    """List Container which use for store default secrets backends."""
+
+
+@dataclass(frozen=True)
+class SecretsBackendConfig:

Review Comment:
   if we provide only one way of configuring, and we "upgrade" legacy config for backward compatibility, maybe we don't need this either?



##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   rather than supporting two ways of configuration, I think it would be better to just support one way and deprecate the old way.  if you look at AirflowConfigParser, you'll find things concerning "deprecated_values" and "upgraded_values"
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] dstandish commented on a diff in pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #23560:
URL: https://github.com/apache/airflow/pull/23560#discussion_r901027524


##########
docs/apache-airflow/security/secrets/secrets-backend/index.rst:
##########
@@ -101,8 +143,8 @@ Roll your own secrets backend
 A secrets backend is a subclass of :py:class:`airflow.secrets.BaseSecretsBackend` and must implement either
 :py:meth:`~airflow.secrets.BaseSecretsBackend.get_connection` or :py:meth:`~airflow.secrets.BaseSecretsBackend.get_conn_value`.
 
-After writing your backend class, provide the fully qualified class name in the ``backend`` key in the ``[secrets]``
-section of ``airflow.cfg``.
+After writing your backend class, provide the fully qualified class name in the ``backend`` key or provide

Review Comment:
   > I would rather for now add Depreciation warning in initialize_secrets_backends() and after think about how to solve all findings which listed above
   
   i understand the impulse but this does not make sense to me. it is hard to think through backcompat and deprecation and UX -- this is one of the challenges with open source.  we should not have two ways of doing things and should not defer til "later" these problems.  the right time to deal with deprecation of an "old" way of configuration is when you add the "new" way of configuration.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on pull request #23560: Add advanced secrets backend configurations

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on PR #23560:
URL: https://github.com/apache/airflow/pull/23560#issuecomment-1162443583

   I also think going back to only exposing ordering is a good path forward. I lean more toward being text based, maybe even more generalized than class names (e.g. `[env, db, backend]`, validated against an enum or something), but I'm not picky here at this point.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org