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/08/05 16:13:38 UTC

[GitHub] [airflow] pdebelak opened a new issue, #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

pdebelak opened a new issue, #25555:
URL: https://github.com/apache/airflow/issues/25555

   ### Apache Airflow version
   
   main (development)
   
   ### What happened
   
   When airflow is loading its configuration, it creates a new secrets backend instance for each configuration backend it loads from secrets and then additionally creates a global secrets backend instance that is used in `ensure_secrets_loaded` which code outside of the configuration file uses. This can cause issues with the vault backend (and possibly others, not sure) since logging in to vault can be an expensive operation server-side and each instance of the vault secrets backend needs to re-login to use its internal client.
   
   ### What you think should happen instead
   
   Ideally, airflow would attempt to create a single secrets backend instance and re-use this. This can possibly be patched in the vault secrets backend, but instead I think updating the `configuration` module to cache the secrets backend would be preferable since it would then apply to any secrets backend.
   
   ### How to reproduce
   
   Use the hashicorp vault secrets backend and store some configuration in `X_secret` values. See that it logs in more than you'd expect.
   
   ### Operating System
   
   Ubuntu 18.04
   
   ### Versions of Apache Airflow Providers
   
   ```
   apache-airflow==2.3.0
   apache-airflow-providers-hashicorp==2.2.0
   hvac==0.11.2
   ```
   
   ### Deployment
   
   Official Apache Airflow Helm Chart
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


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

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


[GitHub] [airflow] potiuk commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206807395

   Did you test if it is really needed (i.e. is Vault client not caching connection internally) ? 


-- 
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 issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206715673

   Have you seen the cache on `custom_secrets_backend(secrets_backend_cls)` just few lines below?
    
   ```
   def get_custom_secret_backend() -> Optional[BaseSecretsBackend]:
       """Get Secret Backend if defined in airflow.cfg"""
       secrets_backend_cls = conf.getimport(section='secrets', key='backend')
   
       return _custom_secrets_backend(secrets_backend_cls)
   
   
   @functools.lru_cache(maxsize=2)
   def _custom_secrets_backend(secrets_backend_cls) -> Optional[BaseSecretsBackend]:
       if secrets_backend_cls:
   
   ```


-- 
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] pdebelak commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
pdebelak commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206713869

   I don't believe it does when looking up secrets for configuration values. If you look at [`_get_config_value_from_secrets_backend`](https://github.com/apache/airflow/blob/cade49845459af57f2ba32dfe6a6ab144bbb8f98/airflow/configuration.py#L114-L127) which is called in a few places inside `AirflowConfigParser` it calls [`get_custom_secret_backend`](https://github.com/apache/airflow/blob/cade49845459af57f2ba32dfe6a6ab144bbb8f98/airflow/configuration.py#L1534-L1538) which directly instantiates a new secrets backend instance rather than using [`ensure_secrets_loaded`](https://github.com/apache/airflow/blob/cade49845459af57f2ba32dfe6a6ab144bbb8f98/airflow/configuration.py#L1523-L1531) which is used in other parts of the code and looks up the instance inside from the [`secrets_backend_list`](https://github.com/apache/airflow/blob/cade49845459af57f2ba32dfe6a6ab144bbb8f98/airflow/configuration.py#L1639).
   
   I assume (perhaps wrongly) that this is either an oversight or due to some circular issue where `conf` depends on secrets and the `secrets` depend on conf. That's why I thought using an `lru_cache` around a function that creates the instance from the class name might be an effective way to share an instance between the configuration lookup and variables/connections.


-- 
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] pdebelak commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
pdebelak commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206703100

   I understand that the instance will not be used across processes, but aren't there cases where the same process will either need to look up multiple secret configuration values (for example, the webserver secret key and the sql alchemy conn) or need to look up both a secret configuration value and a variable or connection? My understanding is that this is the case sometimes, but I could be wrong.
   
   It's also possible that even though it might help in a few cases it isn't worth the code change/potential bugs in the cases where a bad value gets cached for some reason which I totally understand.


-- 
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 issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206753457

   Yeah. This is messy. But do you know what happens if you put that cache in case you have recursive retrieval ?  how will it behave ? Seems that extracting this did not change the recursion. Do. you know how lru_cache() works when it is hit several times during the same method call stack?


-- 
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 closed issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values
URL: https://github.com/apache/airflow/issues/25555


-- 
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 issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206706535

   > I understand that the instance will not be used across processes, but aren't there cases where the same process will either need to look up multiple secret configuration values (for example, the webserver secret key and the sql alchemy conn) or need to look up both a secret configuration value and a variable or connection? My understanding is that this is the case sometimes, but I could be wrong.
   
   This already happens. The Secret Backend client is reused in single process. 


-- 
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] pdebelak commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
pdebelak commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206724560

   That is the change from my PR at https://github.com/apache/airflow/pull/25556. Looks like I was accidentally linking to the git sha from my commit where I added caching there (sorry). If you look at [main](https://github.com/apache/airflow/blob/main/airflow/configuration.py#L1534-L1546) it looks like this:
   
   ```python
   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 secrets_backend_cls:
           try:
               backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
               alternative_secrets_config_dict = json.loads(backends)
           except JSONDecodeError:
               alternative_secrets_config_dict = {}
   
           return secrets_backend_cls(**alternative_secrets_config_dict)
       return None
   ```


-- 
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 issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206628897

   This is not possible due to distributed nature of Airlfow. Secret client can be potentially use by scheduler and by multiuple workers - even if you just use a Local Executor you have potentially many independent processes and you cannot share client between the processes (even if they are forked, it is problematic, but when they are spawned it's impossible). 
   
   Solving the problem would require basically to implement some kind of agent that would shield connection from Airflow. Lucklily Vault has a ready-to use solution like that solves your problem - you just need to deploy it https://www.vaultproject.io/docs/agent/caching


-- 
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] pdebelak commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
pdebelak commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206870454

   `hvac` doesn't appear to be doing any global caching of login credentials that I can see, it [posts to the api every time you call `login`](https://github.com/hvac/hvac/blob/develop/hvac/adapters.py#L179). I added the following test to `tests/core/test_configuration.py` locally and it fails on the second call to `mock_hvac.Client.assert_called_once()` in `main` indicating that multiple hvac clients (that each will need to login) are being created:
   
   ```python
       @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac")
       @conf_vars(
           {
               ("secrets", "backend"): "airflow.providers.hashicorp.secrets.vault.VaultBackend",
               ("secrets", "backend_kwargs"): '{"url": "http://127.0.0.1:8200", "token": "token"}',
           }
       )
       def test_config_from_secret_backend_caches_instance(self, mock_hvac):
           """Get Config Value from a Secret Backend caches the instance"""
   
           test_config = '''[test]
   sql_alchemy_conn_secret = sql_alchemy_conn
   secret_key_secret = secret_key
   '''
           test_config_default = '''[test]
   sql_alchemy_conn = airflow
   secret_key = airflow
   '''
   
           mock_client = mock.MagicMock()
           mock_hvac.Client.return_value = mock_client
   
           def fake_read_secret(path, mount_point, version):
               if path.endswith('sql_alchemy_conn'):
                   return {
                       'request_id': '2d48a2ad-6bcb-e5b6-429d-da35fdf31f56',
                       'lease_id': '',
                       'renewable': False,
                       'lease_duration': 0,
                       'data': {
                           'data': {'value': 'fake_conn'},
                           'metadata': {
                               'created_time': '2020-03-28T02:10:54.301784Z',
                               'deletion_time': '',
                               'destroyed': False,
                               'version': 1,
                           },
                       },
                       'wrap_info': None,
                       'warnings': None,
                       'auth': None,
                   }
               if path.endswith('secret_key'):
                   return {
                       'request_id': '2d48a2ad-6bcb-e5b6-429d-da35fdf31f56',
                       'lease_id': '',
                       'renewable': False,
                       'lease_duration': 0,
                       'data': {
                           'data': {'value': 'fake_key'},
                           'metadata': {
                               'created_time': '2020-03-28T02:10:54.301784Z',
                               'deletion_time': '',
                               'destroyed': False,
                               'version': 1,
                           },
                       },
                       'wrap_info': None,
                       'warnings': None,
                       'auth': None,
                   }
   
           mock_client.secrets.kv.v2.read_secret_version.side_effect = fake_read_secret
   
           test_conf = AirflowConfigParser(default_config=parameterized_config(test_config_default))
           test_conf.read_string(test_config)
           test_conf.sensitive_config_values = test_conf.sensitive_config_values | {
               ('test', 'sql_alchemy_conn'),
               ('test', 'secret_key'),
           }
   
           assert 'fake_conn' == test_conf.get('test', 'sql_alchemy_conn')
           mock_hvac.Client.assert_called_once()
           assert 'fake_key' == test_conf.get('test', 'secret_key')
           mock_hvac.Client.assert_called_once()
   ```


-- 
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] pdebelak commented on issue #25555: Airflow doesn't re-use a secrets backend instance when loading configuration values

Posted by GitBox <gi...@apache.org>.
pdebelak commented on issue #25555:
URL: https://github.com/apache/airflow/issues/25555#issuecomment-1206788354

   Yeah, the recursion between conf and secrets isn't eliminated by this change. I thought it seemed relatively safe to cache based on the key of `secrets_backend_cls` based on the assumption that if `secrets_backend_cls` is set, the other values needed for this secret should be set as well and won't change (not sure if this is a safe assumption or under what circumstances, if any, `conf.get` might return different value for the same key). The safest approach would be to cache on things that don't depend on `conf` at all like so:
   
   ```python
   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 secrets_backend_cls:
           try:
               backends: Any = conf.get(section='secrets', key='backend_kwargs', fallback='{}')
               alternative_secrets_config_dict = json.loads(backends)
           except JSONDecodeError:
               alternative_secrets_config_dict = {}
   
           return _custom_secrets_backend(secrets_backend_cls, alternative_secrets_config_dict)
       return None
   
   
   @functools.lru_cache
   def _custom_secrets_backend(secrets_backend_cls, alternative_secrets_config_dict):
       """Separate function to create secrets backend instance to allow caching"""
       return secrets_backend_cls(**alternative_secrets_config_dict)
   ```
   
   That way the cache only cares about the values from conf, without using `conf` itself so if the `conf` is changed in a way that would result in a new secrets backend configuration we'd get a new secrets backend instance.
   
   An `lru_cache`d function in general can be called many times in the same call stack, even with recursion. In fact, people often demonstrate it with a recursive example like:
   
   ```
   >>> import functools
   >>> @functools.lru_cache
   ... def fibonacci(num):
   ...     print(f'Calculating fibonacci for {num}')
   ...     if num < 2:
   ...             return num
   ...     return fibonacci(num - 1) + fibonacci(num - 2)
   ... 
   >>> [fibonacci(n) for n in range(10)]
   Calculating fibonacci for 0
   Calculating fibonacci for 1
   Calculating fibonacci for 2
   Calculating fibonacci for 3
   Calculating fibonacci for 4
   Calculating fibonacci for 5
   Calculating fibonacci for 6
   Calculating fibonacci for 7
   Calculating fibonacci for 8
   Calculating fibonacci for 9
   [0, 1, 1, 2, 3, 5, 8, 13, 21, 34]
   >>> [fibonacci(n) for n in range(10)]
   [0, 1, 1, 2, 3, 5, 8, 13, 21, 34]
   ```
   
   If you are asking about thread safety and `lru_cache`, the implementation of `lru_cache` is thread safe in the sense that it uses a `threading.RLock` for non-thread-safe things (and uses an `RLock` rather than a regular `Lock` to avoid deadlocking), but it unfortunately doesn't use that lock when filling the cache, so if you have two threads both getting a cache miss they might both call the underlying function so there's no hard guarantee that it will only be called once. I think airflow is mostly multiprocess rather than threaded, but it should at the very least be no worse than the current situation to add caching, but in some situations it would be 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