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 2020/12/22 14:08:45 UTC

[GitHub] [airflow] leopoldhoudin opened a new issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

leopoldhoudin opened a new issue #13254:
URL: https://github.com/apache/airflow/issues/13254


   **Apache Airflow version**: 2.0.0
   
   **Environment**: 
   
   - **Cloud provider or hardware configuration**: N/A
   - **OS** (e.g. from /etc/os-release): custom Docker image (`FROM python:3.6`) and macOS Big Sur (11.0.1)
   - **Kernel** (e.g. `uname -a`): 
     - `Linux xxx 4.14.174+ #1 SMP x86_64 GNU/Linux`
     - `Darwin xxx 20.1.0 Darwin Kernel Version 20.1.0 rRELEASE_X86_64 x86_64`
   - **Install tools**:
   - **Others**:
   
   **What happened**:
   
   I may have mixed 2 different issues here, but this is what happened to me.
   
   I'm trying to use Airflow with the `airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend` and a `sql_alchemy_conn_secret` too, however, I have a `NameError` exception when attempting to run either `airflow scheduler` or `airflow webserver`:
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/usr/local/lib/python3.6/site-packages/airflow/__init__.py", line 34, in <module>
       from airflow import settings
     File "/usr/local/lib/python3.6/site-packages/airflow/settings.py", line 35, in <module>
       from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf  # NOQA F401
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 786, in <module>
       conf.read(AIRFLOW_CONFIG)
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 447, in read
       self._validate()
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 196, in _validate
       self._validate_config_dependencies()
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 224, in _validate_config_dependencies
       is_sqlite = "sqlite" in self.get('core', 'sql_alchemy_conn')
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 324, in get
       option = self._get_option_from_secrets(deprecated_key, deprecated_section, key, section)
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 342, in _get_option_from_secrets
       option = self._get_secret_option(section, key)
     File "/usr/local/lib/python3.6/site-packages/airflow/configuration.py", line 303, in _get_secret_option
       return _get_config_value_from_secret_backend(secrets_path)
   NameError: name '_get_config_value_from_secret_backend' is not defined
   ```
   
   **What you expected to happen**:
   
   A proper import and configuration creation.
   
   **How to reproduce it**:
   
   `airflow.cfg`:
   ```ini
   [core]
   
   # ...
   
   sql_alchemy_conn_secret = some-key
   
   # ...
   
   [secrets]
   
   backend = airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
   
   backend_kwargs = { ... }
   
   # ...
   
   ```
   
   **Anything else we need to know**:
   
   Here is the workaround I have for the moment, not sure it works all the way, and probably doesn't cover all edge cases, tho it kinda works for my setup:
   
   Move `get_custom_secret_backend` before (for me it's actually below `_get_config_value_from_secret_backend`): https://github.com/apache/airflow/blob/cc87caa0ce0b31aa29df7bbe90bdcc2426d80ff1/airflow/configuration.py#L794
   
   Then comment: https://github.com/apache/airflow/blob/cc87caa0ce0b31aa29df7bbe90bdcc2426d80ff1/airflow/configuration.py#L232-L236
   


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   I think this is an interesting one. Looks like circular dependency where the method is not yet defined because we are calling it during parsing of the python file. 


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

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



[GitHub] [airflow] kaxil edited a comment on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on issue #13254:
URL: https://github.com/apache/airflow/issues/13254#issuecomment-749823989


   This feels similar to https://github.com/apache/airflow/issues/11149#issuecomment-731398618
   
   Can you check my comment over there please @leopoldhoudin 
   
   If the error persist I will take a look again, but last time it had worked -- might have to verify if the code changed since then.


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

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



[GitHub] [airflow] potiuk closed issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #13254:
URL: https://github.com/apache/airflow/issues/13254


   


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   I pushed a possible fix in #13260 


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

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



[GitHub] [airflow] potiuk edited a comment on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on issue #13254:
URL: https://github.com/apache/airflow/issues/13254#issuecomment-750015222


   But the question is if  moving get_custom_secret_backend before conf.read() is a good idea ? The problem is that the 'get_custom_secret_backend' uses the conf variable and reads the configuration from it:  
   
   ```
   conf.getimport(section='secrets', key='backend')
   conf.get(section='secrets', key='backend_kwargs', fallback='{}')
   ```
   
   Then the `get_custom_secret_backend` can be actually called BEFORE 'secrets' section is parsed via conf! 
   
   This is the 'chicken-egg' problem I mentioned before. 
   
   We might assume of course that the 'secret' versions of the variables should be defined after [secrets] section in the conf. Which might make sense, but then it opens the doors to problems: 
   
   The behaviour will be different if  someone defines '[core'] section with `sql_aclhemy_conn' before the [secrets] or after.
   


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

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



[GitHub] [airflow] leopoldhoudin edited a comment on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
leopoldhoudin edited a comment on issue #13254:
URL: https://github.com/apache/airflow/issues/13254#issuecomment-750823841


   @potiuk works for me, and as a charm in production env! thx a lot for the quick reply and fix.
   I guess I'll have to use my own patch / a clone of your PR until the next release, isn't it? Do you have any idea on when this should fall?


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

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



[GitHub] [airflow] leopoldhoudin commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   @potiuk works for me, and as a charm in production env! thx a lot for the quick reply and fix.
   I guess I'll have to use my own patch until the next release, isn't it? Do you have any idea on when this should fall?


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   @leopoldhoudin -> can you please check if #13260 fixes the problem? 


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   @kaxil - i think we need to think and discuss together how to handle this case. There is this chicken-egg here i think 


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   But the question is if  moving get_custom_secret_backend before conf.read() is a good idea ? The problem is that the 'get_custom_secret_backend' uses the conf variable and reads the configuration from it:  
   
   ```
   conf.getimport(section='secrets', key='backend')
   conf.get(section='secrets', key='backend_kwargs', fallback='{}')
   ```
   
   Then the `get_custom_secret_backen` can be actually called BEFORE 'secrets' section is parsed via conf! 
   
   This is the 'chicken-egg' problem I mentioned before. 
   
   We might assume of course that the 'secret' versions of the variables should be defined after [secrets] section in the conf. Which might make sense, but then it opens the doors to problems: 
   
   The behaviour will be different if  someone defines '[core'] section with `sql_aclhemy_conn' before the [secrets] or after.
   


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

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



[GitHub] [airflow] leopoldhoudin commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Thanks for the quick reply @potiuk. Unfortunately, the PR doesn't fix the issue, for 2 reasons:
   1. Actually the `sql_alchemy_conn` configuration parameter is not read from the secrets backend as intended, but from Airflow's default configuration (fallback);
   2. It leads to the second error (which is why I had the second part in my "workaround"):
   
   <details><summary>Stacktrace</summary>
   
   ```bash
   Traceback (most recent call last):
     File "/some/path/airflow/venv/bin/airflow", line 5, in <module>
       from airflow.__main__ import main
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/__init__.py", line 34, in <module>
       from airflow import settings
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/settings.py", line 35, in <module>
       from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf  # NOQA F401
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/configuration.py", line 798, in <module>
       conf.read(AIRFLOW_CONFIG)
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/configuration.py", line 459, in read
       self._validate()
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/configuration.py", line 208, in _validate
       self._validate_config_dependencies()
     File "/some/path/airflow/venv/lib/python3.6/site-packages/airflow/configuration.py", line 239, in _validate_config_dependencies
       "error: cannot use sqlite with the {}".format(self.get('core', 'executor'))
   airflow.exceptions.AirflowConfigException: error: cannot use sqlite with the LocalExecutor
   ```
   
   </details>
   
   > Looks like circular dependency where the method is not yet defined because we are calling it during parsing of the python file.
   
   Completely, which is why I suggested in the "workaround" to move `get_custom_secret_backend` (which is the originally not found symbol) at an earlier location in the file, I guess anywhere before:
   
   https://github.com/apache/airflow/blob/cc87caa0ce0b31aa29df7bbe90bdcc2426d80ff1/airflow/configuration.py#L794
   
   would fix it.
   
   Yet the issue still exists around (see stack trace):
   
   https://github.com/apache/airflow/blob/cc87caa0ce0b31aa29df7bbe90bdcc2426d80ff1/airflow/configuration.py#L232-L236
   
   reads the default value for `sql_alchemy_conn` (i.e. `sqlite`), which is actually not true since the value of `sql_alchemy_conn` wasn't yet resolved.
   
   So I'm not sure here how to handle this edge case.
   
   I am actually puzzled on why I am the first to have this issue because I guess using a secrets backend probably means that you won't use either `DebugExecutor` nor `SequentialExecutor` (production environment)...
   


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Yeah. After looking closer - I think a better solution is simply to remove both _validate() methods from read() and read_dict() and simply explicitly call `conf._validate()`  at the end of configure.py
   
   Then we solve the problem once and for all. @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.

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   OK. the #13260  should fix it @leopoldhoudin. It is rather easy to apply locally, so it would be great to confirm if the fix works


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Thanks for verifying :). 
   
   I think we will be looking at releasing 2.0.1 in early January. Xmas/EOY time is a bit quiet, but we already have a few annoying 'teething problems' with 2.0 reported and some of them already fixed (some of them need discussion/fix) still. There are not many, but I think also there will be some users who will use the "quiet" period around Xmas Holidays to test migration to 2.0 and report some more issues they find. Most of big businesses have moratorium for production changes mid-Dec, beginning of Jan which results in more time to play with staging systems :).
   
   You can see all the opened (and closed as well) issues for 2.0.1 here:  https://github.com/apache/airflow/milestone/23
   
   We will likely decide on January 6th at our next DevCall (see calendar here: https://calendar.google.com/calendar/embed?src=c_dhdh3bdjc42c4ngtnpg7ovcs9g%40group.calendar.google.com )
   


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   It looks like you have somethng seriously wrong in your configuration -- looks like the line numbers reported do not match the line numbers from Airlfow installation. can you please remove/reinstall airflow from the scratch and see again? Preferably in a separate virtual environment?


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

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



[GitHub] [airflow] leopoldhoudin commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Hi @kaxil, yep it works moving `get_custom_secret_backend` before `conf.read(AIRFLOW_CONFIG)` as proposed by @potiuk in his first version of the PR.
   
   I'll ensure everything works in production environment tho during the day and revert asap.


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Maybe we can leave with that, it sounds like rather brittle thing to do.
   
   Another option would be that we make sure that 'validate" is callled at the end of the configure.py 


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

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



[GitHub] [airflow] leopoldhoudin commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Sorry, indeed, I may have copy/pasted a bad stack trace. Yet, started from a fresh env, and same behaviour arises:
   
   My step:
   ```
   virtualenv venv
   source venv/bin/activate
   pip install apache-airflow
   pip install apache-airflow-providers-google
   pip install apache-airflow-providers-postgres
   ```
   
   Create a `airflow.cfg` at current location with the following content:
   <details>
   <summary>airflow.cfg</summary>
   
   ```ini
   # -*- coding: utf-8 -*-
   #
   # Licensed to the Apache Software Foundation (ASF) under one
   # or more contributor license agreements.  See the NOTICE file
   # distributed with this work for additional information
   # regarding copyright ownership.  The ASF licenses this file
   # to you under the Apache License, Version 2.0 (the
   # "License"); you may not use this file except in compliance
   # with the License.  You may obtain a copy of the License at
   #
   #   http://www.apache.org/licenses/LICENSE-2.0
   #
   # Unless required by applicable law or agreed to in writing,
   # software distributed under the License is distributed on an
   # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
   # KIND, either express or implied.  See the License for the
   # specific language governing permissions and limitations
   # under the License.
   
   [core]
   # The folder where your airflow pipelines live, most likely a
   # subfolder in a code repository. This path must be absolute.
   dags_folder = workflows/
   
   # Users must supply an Airflow connection id that provides access to the storage
   # location.
   remote_log_conn_id =
   remote_base_log_folder =
   encrypt_s3_logs = False
   
   # Hostname by providing a path to a callable, which will resolve the hostname.
   # The format is "package:function".
   #
   # For example, default value "socket:getfqdn" means that result from getfqdn() of "socket"
   # package will be used as hostname.
   #
   # No argument should be required in the function specified.
   # If using IP address as hostname is preferred, use value ``airflow.utils.net:get_host_ip_address``
   hostname_callable = socket.getfqdn
   
   # Default timezone in case supplied date times are naive
   # can be utc (default), system, or any IANA timezone string (e.g. Europe/Amsterdam)
   default_timezone = utc
   
   # The executor class that airflow should use. Choices include
   # SequentialExecutor, LocalExecutor, CeleryExecutor, DaskExecutor, KubernetesExecutor
   executor = LocalExecutor
   
   # The SqlAlchemy connection string to the metadata database.
   # SqlAlchemy supports many different database engine, more information
   # their website
   sql_alchemy_conn_secret = sql_alchemy_conn
   
   # The encoding for the databases
   sql_engine_encoding = utf-8
   
   # If SqlAlchemy should pool database connections.
   sql_alchemy_pool_enabled = True
   
   # The SqlAlchemy pool size is the maximum number of database connections
   # in the pool. 0 indicates no limit.
   sql_alchemy_pool_size = 5
   
   # The maximum overflow size of the pool.
   # When the number of checked-out connections reaches the size set in pool_size,
   # additional connections will be returned up to this limit.
   # When those additional connections are returned to the pool, they are disconnected and discarded.
   # It follows then that the total number of simultaneous connections the pool will allow
   # is pool_size + max_overflow,
   # and the total number of "sleeping" connections the pool will allow is pool_size.
   # max_overflow can be set to -1 to indicate no overflow limit;
   # no limit will be placed on the total number of concurrent connections. Defaults to 10.
   sql_alchemy_max_overflow = 10
   
   # The SqlAlchemy pool recycle is the number of seconds a connection
   # can be idle in the pool before it is invalidated. This config does
   # not apply to sqlite. If the number of DB connections is ever exceeded,
   # a lower config value will allow the system to recover faster.
   sql_alchemy_pool_recycle = 1800
   
   # Check connection at the start of each connection pool checkout.
   # Typically, this is a simple statement like "SELECT 1".
   # More information here:
   # https://docs.sqlalchemy.org/en/13/core/pooling.html#disconnect-handling-pessimistic
   sql_alchemy_pool_pre_ping = True
   
   # The schema to use for the metadata database.
   # SqlAlchemy supports databases with the concept of multiple schemas.
   sql_alchemy_schema =
   
   # Import path for connect args in SqlAlchemy. Default to an empty dict.
   # This is useful when you want to configure db engine args that SqlAlchemy won't parse
   # in connection string.
   # See https://docs.sqlalchemy.org/en/13/core/engines.html#sqlalchemy.create_engine.params.connect_args
   # sql_alchemy_connect_args =
   
   # The amount of parallelism as a setting to the executor. This defines
   # the max number of task instances that should run simultaneously
   # on this airflow installation
   parallelism = 32
   
   # The number of task instances allowed to run concurrently by the scheduler
   dag_concurrency = 16
   
   # Are DAGs paused by default at creation
   dags_are_paused_at_creation = True
   
   # The maximum number of active DAG runs per DAG
   max_active_runs_per_dag = 16
   
   # Whether to load the DAG examples that ship with Airflow. It's good to
   # get started, but you probably want to set this to False in a production
   # environment
   load_examples = False
   
   # Whether to load the default connections that ship with Airflow. It's good to
   # get started, but you probably want to set this to False in a production
   # environment
   load_default_connections = False
   
   # Where your Airflow plugins are stored
   plugins_folder = airflow/plugins/
   
   # Secret key to save connection passwords in the db
   fernet_key_secret = fernet_key
   
   # Whether to disable pickling dags
   donot_pickle = False
   
   # How long before timing out a python file import
   dagbag_import_timeout = 30
   
   # How long before timing out a DagFileProcessor, which processes a dag file
   dag_file_processor_timeout = 50
   
   # The class to use for running task instances in a subprocess
   task_runner = StandardTaskRunner
   
   # If set, tasks without a ``run_as_user`` argument will be run with this user
   # Can be used to de-elevate a sudo user running Airflow when executing tasks
   default_impersonation =
   
   # What security module to use (for example kerberos)
   security =
   
   # If set to False enables some unsecure features like Charts and Ad Hoc Queries.
   # In 2.0 will default to True.
   secure_mode = False
   
   # Turn unit test mode on (overwrites many configuration options with test
   # values at runtime)
   unit_test_mode = False
   
   # Whether to enable pickling for xcom (note that this is insecure and allows for
   # RCE exploits). This will be deprecated in Airflow 2.0 (be forced to False).
   enable_xcom_pickling = True
   
   # When a task is killed forcefully, this is the amount of time in seconds that
   # it has to cleanup after it is sent a SIGTERM, before it is SIGKILLED
   killed_task_cleanup_time = 60
   
   # Whether to override params with dag_run.conf. If you pass some key-value pairs
   # through ``airflow dags backfill -c`` or
   # ``airflow dags trigger -c``, the key-value pairs will override the existing ones in params.
   dag_run_conf_overrides_params = False
   
   # Worker initialisation check to validate Metadata Database connection
   worker_precheck = False
   
   # When discovering DAGs, ignore any files that don't contain the strings ``DAG`` and ``airflow``.
   dag_discovery_safe_mode = True
   
   # The number of retries each task is going to have by default. Can be overridden at dag or task level.
   default_task_retries = 0
   
   # Whether to serialise DAGs and persist them in DB.
   # If set to True, Webserver reads from DB instead of parsing DAG files
   # More details: https://airflow.apache.org/docs/stable/dag-serialization.html
   store_serialized_dags = False
   
   # Updating serialized DAG can not be faster than a minimum interval to reduce database write rate.
   min_serialized_dag_update_interval = 30
   
   # Fetching serialized DAG can not be faster than a minimum interval to reduce database
   # read rate. This config controls when your DAGs are updated in the Webserver
   min_serialized_dag_fetch_interval = 10
   
   # Whether to persist DAG files code in DB.
   # If set to True, Webserver reads file contents from DB instead of
   # trying to access files in a DAG folder. Defaults to same as the
   # ``store_serialized_dags`` setting.
   # Example: store_dag_code = False
   # store_dag_code =
   
   # Maximum number of Rendered Task Instance Fields (Template Fields) per task to store
   # in the Database.
   # When Dag Serialization is enabled (``store_serialized_dags=True``), all the template_fields
   # for each of Task Instance are stored in the Database.
   # Keeping this number small may cause an error when you try to view ``Rendered`` tab in
   # TaskInstance view for older tasks.
   max_num_rendered_ti_fields_per_task = 30
   
   # On each dagrun check against defined SLAs
   check_slas = True
   
   # Path to custom XCom class that will be used to store and resolve operators results
   # Example: xcom_backend = path.to.CustomXCom
   xcom_backend = airflow.models.xcom.BaseXCom
   
   [logging]
   
   # Airflow can store logs remotely in AWS S3, Google Cloud Storage or Elastic Search.
   # Set this to True if you want to enable remote logging.
   remote_logging = False
   
   # The folder where airflow should store its log files
   # This path must be absolute
   base_log_folder = airflow/logs/
   
   # Log format for when Colored logs is enabled
   colored_log_format = [%%(blue)s%%(asctime)s%%(reset)s] %%(blue)s%%(filename)s:%%(reset)s%%(lineno)d %%(log_color)s%%(levelname)s%%(reset)s - %%(log_color)s%%(message)s%%(reset)s
   
   # Format of Log line
   log_format = [%%(asctime)s] %%(filename)s:%%(lineno)d %%(levelname)s - %%(message)s
   
   simple_log_format = %%(asctime)s %%(levelname)s - %%(message)s
   
   # Logging level
   logging_level = INFO
   
   # Logging level for Flask-appbuilder UI
   fab_logging_level = WARN
   
   # Logging class
   # Specify the class that will specify the logging configuration
   # This class has to be on the python classpath
   # Example: logging_config_class = my.path.default_local_settings.LOGGING_CONFIG
   logging_config_class =
   
   # Flag to enable/disable Colored logs in Console
   # Colour the logs when the controlling terminal is a TTY.
   colored_console_log = True
   
   colored_formatter_class = airflow.utils.log.colored_log.CustomTTYColoredFormatter
   
   # Log filename format
   log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log
   log_processor_filename_template = {{ filename }}.log
   dag_processor_manager_log_location = airflow/logs/dag_processor_manager/dag_processor_manager.log
   
   # Name of handler to read task instance logs.
   # Default to use task handler.
   task_log_reader = task
   
   [secrets]
   # Full class name of secrets backend to enable (will precede env vars and metastore in search path)
   # Example: backend = airflow.contrib.secrets.aws_systems_manager.SystemsManagerParameterStoreBackend
   backend = airflow.providers.google.cloud.secrets.secret_manager.CloudSecretManagerBackend
   
   # The backend_kwargs param is loaded into a dictionary and passed to __init__ of secrets backend class.
   # See documentation for the secrets backend you are using. JSON is expected.
   # Example for AWS Systems Manager ParameterStore:
   # ``{{"connections_prefix": "/airflow/connections", "profile_name": "default"}}``
   backend_kwargs = {"config_prefix": "airflow-config", "connections_prefix": "airflow-conn", "sep": "-"}
   
   [cli]
   # In what way should the cli access the API. The LocalClient will use the
   # database directly, while the json_client will use the api running on the
   # webserver
   api_client = airflow.api.client.local_client
   
   # If you set web_server_url_prefix, do NOT forget to append it here, ex:
   # ``endpoint_url = http://localhost:8080/myroot``
   # So api will look like: ``http://localhost:8080/myroot/api/experimental/...``
   endpoint_url = http://localhost:8080
   
   [debug]
   # Used only with DebugExecutor. If set to True DAG will fail with first
   # failed task. Helpful for debugging purposes.
   fail_fast = False
   
   [api]
   # How to authenticate users of the API. See
   # https://airflow.apache.org/docs/stable/security.html for possible values.
   # ("airflow.api.auth.backend.default" allows all requests for historic reasons)
   auth_backend = airflow.api.auth.backend.deny_all
   
   [operators]
   # The default owner assigned to each new operator, unless
   # provided explicitly or passed via ``default_args``
   default_owner = airflow
   default_cpus = 1
   default_ram = 512
   default_disk = 512
   default_gpus = 0
   
   [hive]
   # Default mapreduce queue for HiveOperator tasks
   default_hive_mapred_queue =
   
   [webserver]
   # The base url of your website as airflow cannot guess what domain or
   # cname you are using. This is used in automated emails that
   # airflow sends to point links to the right web server
   base_url = http://localhost:8080
   
   # Default timezone to display all dates in the RBAC UI, can be UTC, system, or
   # any IANA timezone string (e.g. Europe/Amsterdam). If left empty the
   # default value of core/default_timezone will be used
   # Example: default_ui_timezone = America/New_York
   default_ui_timezone = UTC
   
   # The ip specified when starting the web server
   web_server_host = 0.0.0.0
   
   # The port on which to run the web server
   web_server_port = 8080
   
   # Paths to the SSL certificate and key for the web server. When both are
   # provided SSL will be enabled. This does not change the web server port.
   web_server_ssl_cert =
   
   # Paths to the SSL certificate and key for the web server. When both are
   # provided SSL will be enabled. This does not change the web server port.
   web_server_ssl_key =
   
   # Number of seconds the webserver waits before killing gunicorn master that doesn't respond
   web_server_master_timeout = 120
   
   # Number of seconds the gunicorn webserver waits before timing out on a worker
   web_server_worker_timeout = 120
   
   # Number of workers to refresh at a time. When set to 0, worker refresh is
   # disabled. When nonzero, airflow periodically refreshes webserver workers by
   # bringing up new ones and killing old ones.
   worker_refresh_batch_size = 1
   
   # Number of seconds to wait before refreshing a batch of workers.
   worker_refresh_interval = 30
   
   # If set to True, Airflow will track files in plugins_folder directory. When it detects changes,
   # then reload the gunicorn.
   reload_on_plugin_change = False
   
   # Secret key used to run your flask app
   # It should be as random as possible
   secret_key_secret = secret_key
   
   # Number of workers to run the Gunicorn web server
   workers = 4
   
   # The worker class gunicorn should use. Choices include
   # sync (default), eventlet, gevent
   worker_class = sync
   
   # Log files for the gunicorn webserver. '-' means log to stderr.
   access_logfile = -
   
   # Log files for the gunicorn webserver. '-' means log to stderr.
   error_logfile = -
   
   # Expose the configuration file in the web server
   expose_config = False
   
   # Expose hostname in the web server
   expose_hostname = True
   
   # Expose stacktrace in the web server
   expose_stacktrace = True
   
   # Set to true to turn on authentication:
   # https://airflow.apache.org/security.html#web-authentication
   authenticate = True
   
   auth_backend = airflow.contrib.auth.backends.password_auth
   
   # Filter the list of dags by owner name (requires authentication to be enabled)
   filter_by_owner = False
   
   # Filtering mode. Choices include user (default) and ldapgroup.
   # Ldap group filtering requires using the ldap backend
   #
   # Note that the ldap server needs the "memberOf" overlay to be set up
   # in order to user the ldapgroup mode.
   owner_mode = user
   
   # Default DAG view. Valid values are:
   # tree, graph, duration, gantt, landing_times
   dag_default_view = tree
   
   # "Default DAG orientation. Valid values are:"
   # LR (Left->Right), TB (Top->Bottom), RL (Right->Left), BT (Bottom->Top)
   dag_orientation = TB
   
   # Puts the webserver in demonstration mode; blurs the names of Operators for
   # privacy.
   demo_mode = False
   
   # The amount of time (in secs) webserver will wait for initial handshake
   # while fetching logs from other worker machine
   log_fetch_timeout_sec = 5
   
   # Time interval (in secs) to wait before next log fetching.
   log_fetch_delay_sec = 2
   
   # Distance away from page bottom to enable auto tailing.
   log_auto_tailing_offset = 30
   
   # Animation speed for auto tailing log display.
   log_animation_speed = 1000
   
   # By default, the webserver shows paused DAGs. Flip this to hide paused
   # DAGs by default
   hide_paused_dags_by_default = False
   
   # Consistent page size across all listing views in the UI
   page_size = 100
   
   # Use FAB-based webserver with RBAC feature
   rbac = True
   
   # Define the color of navigation bar
   navbar_color = #fff
   
   # Default dagrun to show in UI
   default_dag_run_display_number = 25
   
   # Enable werkzeug ``ProxyFix`` middleware for reverse proxy
   enable_proxy_fix = False
   
   # Number of values to trust for ``X-Forwarded-For``.
   # More info: https://werkzeug.palletsprojects.com/en/0.16.x/middleware/proxy_fix/
   proxy_fix_x_for = 1
   
   # Number of values to trust for ``X-Forwarded-Proto``
   proxy_fix_x_proto = 1
   
   # Number of values to trust for ``X-Forwarded-Host``
   proxy_fix_x_host = 1
   
   # Number of values to trust for ``X-Forwarded-Port``
   proxy_fix_x_port = 1
   
   # Number of values to trust for ``X-Forwarded-Prefix``
   proxy_fix_x_prefix = 1
   
   # Set secure flag on session cookie
   cookie_secure = False
   
   # Set samesite policy on session cookie
   cookie_samesite = Strict
   
   # Default setting for wrap toggle on DAG code and TI log views.
   default_wrap = False
   
   # Allow the UI to be rendered in a frame
   x_frame_enabled = True
   
   # Send anonymous user activity to your analytics tool
   # choose from google_analytics, segment, or metarouter
   # analytics_tool =
   
   # Unique ID of your account in the analytics tool
   # analytics_id =
   
   # Update FAB permissions and sync security manager roles
   # on webserver startup
   update_fab_perms = True
   
   # The UI cookie lifetime in days
   session_lifetime_minutes = 60
   
   [email]
   email_backend = airflow.utils.email.send_email_smtp
   
   [scheduler]
   # Task instances listen for external kill signal (when you clear tasks
   # from the CLI or the UI), this defines the frequency at which they should
   # listen (in seconds).
   job_heartbeat_sec = 5
   
   # The scheduler constantly tries to trigger new tasks (look at the
   # scheduler section in the docs for more information). This defines
   # how often the scheduler should run (in seconds).
   scheduler_heartbeat_sec = 5
   
   # After how much time should the scheduler terminate in seconds
   # -1 indicates to run continuously (see also num_runs)
   run_duration = 41460
   
   # The number of times to try to schedule each DAG file
   # -1 indicates unlimited number
   num_runs = -1
   
   # The number of seconds to wait between consecutive DAG file processing
   processor_poll_interval = 1
   
   # after how much time (seconds) a new DAGs should be picked up from the filesystem
   min_file_process_interval = 0
   
   # How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes.
   dag_dir_list_interval = 300
   
   # How often should stats be printed to the logs. Setting to 0 will disable printing stats
   print_stats_interval = 30
   
   # If the last scheduler heartbeat happened more than scheduler_health_check_threshold
   # ago (in seconds), scheduler is considered unhealthy.
   # This is used by the health check in the "/health" endpoint
   scheduler_health_check_threshold = 30
   child_process_log_directory = airflow/logs/scheduler
   
   # Local task jobs periodically heartbeat to the DB. If the job has
   # not heartbeat in this many seconds, the scheduler will mark the
   # associated task instance as failed and will re-schedule the task.
   scheduler_zombie_task_threshold = 300
   
   # Turn off scheduler catchup by setting this to False.
   # Default behavior is unchanged and
   # Command Line Backfills still work, but the scheduler
   # will not do scheduler catchup if this is False,
   # however it can be set on a per DAG basis in the
   # DAG definition (catchup)
   catchup_by_default = False
   
   # This changes the batch size of queries in the scheduling main loop.
   # If this is too high, SQL query performance may be impacted by one
   # or more of the following:
   # - reversion to full table scan
   # - complexity of query predicate
   # - excessive locking
   # Additionally, you may hit the maximum allowable query length for your db.
   # Set this to 0 for no limit (not advised)
   max_tis_per_query = 512
   
   statsd_host = localhost
   statsd_port = 8125
   statsd_prefix = airflow
   
   # If you want to avoid send all the available metrics to StatsD,
   # you can configure an allow list of prefixes to send only the metrics that
   # start with the elements of the list (e.g: scheduler,executor,dagrun)
   statsd_allow_list =
   
   # The scheduler can run multiple threads in parallel to schedule dags.
   # This defines how many threads will run.
   parsing_processes = 2
   
   authenticate = False
   
   # Turn off scheduler use of cron intervals by setting this to False.
   # DAGs submitted manually in the web UI or with trigger_dag will still run.
   use_job_schedule = True
   
   # Allow externally triggered DagRuns for Execution Dates in the future
   # Only has effect if schedule_interval is set to None in DAG
   allow_trigger_in_future = False
   
   [metrics]
   
   # Statsd (https://github.com/etsy/statsd) integration settings
   statsd_on = False
   
   [admin]
   # UI to hide sensitive variable fields when set to True
   hide_sensitive_variable_fields = True
   
   [kubernetes]
   # The repository, tag and imagePullPolicy of the Kubernetes Image for the Worker to Run
   worker_container_repository =
   
   # Path to the YAML pod file. If set, all other kubernetes-related fields are ignored.
   # (This feature is experimental)
   pod_template_file =
   worker_container_tag =
   worker_container_image_pull_policy = IfNotPresent
   
   # If True, all worker pods will be deleted upon termination
   delete_worker_pods = True
   
   # If False (and delete_worker_pods is True),
   # failed worker pods will not be deleted so users can investigate them.
   delete_worker_pods_on_failure = False
   
   # Number of Kubernetes Worker Pod creation calls per scheduler loop
   worker_pods_creation_batch_size = 1
   
   # The Kubernetes namespace where airflow workers should be created. Defaults to ``default``
   namespace = default
   
   # Allows users to launch pods in multiple namespaces.
   # Will require creating a cluster-role for the scheduler
   multi_namespace_mode = False
   
   # Use the service account kubernetes gives to pods to connect to kubernetes cluster.
   # It's intended for clients that expect to be running inside a pod running on kubernetes.
   # It will raise an exception if called from a process not running in a kubernetes environment.
   in_cluster = True
   
   # Keyword parameters to pass while calling a kubernetes client core_v1_api methods
   # from Kubernetes Executor provided as a single line formatted JSON dictionary string.
   # List of supported params are similar for all core_v1_apis, hence a single config
   # variable for all apis.
   # See:
   # https://raw.githubusercontent.com/kubernetes-client/python/master/kubernetes/client/apis/core_v1_api.py
   # Note that if no _request_timeout is specified, the kubernetes client will wait indefinitely
   # for kubernetes api responses, which will cause the scheduler to hang.
   # The timeout is specified as [connect timeout, read timeout]
   kube_client_request_args =
   
   # Optional keyword arguments to pass to the ``delete_namespaced_pod`` kubernetes client
   # ``core_v1_api`` method when using the Kubernetes Executor.
   # This should be an object and can contain any of the options listed in the ``v1DeleteOptions``
   # class defined here:
   # https://github.com/kubernetes-client/python/blob/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/models/v1_delete_options.py#L19
   # Example: delete_option_kwargs = {{"grace_period_seconds": 10}}
   delete_option_kwargs =
   
   [kubernetes_node_selectors]
   
   # The Key-value pairs to be given to worker pods.
   # The worker pods will be scheduled to the nodes of the specified key-value pairs.
   # Should be supplied in the format: key = value
   
   [kubernetes_annotations]
   
   # The Key-value annotations pairs to be given to worker pods.
   # Should be supplied in the format: key = value
   
   [kubernetes_environment_variables]
   
   # The scheduler sets the following environment variables into your workers. You may define as
   # many environment variables as needed and the kubernetes launcher will set them in the launched workers.
   # Environment variables in this section are defined as follows
   # ``<environment_variable_key> = <environment_variable_value>``
   #
   # For example if you wanted to set an environment variable with value `prod` and key
   # ``ENVIRONMENT`` you would follow the following format:
   # ENVIRONMENT = prod
   #
   # Additionally you may override worker airflow settings with the ``AIRFLOW__<SECTION>__<KEY>``
   # formatting as supported by airflow normally.
   
   [kubernetes_secrets]
   
   # The scheduler mounts the following secrets into your workers as they are launched by the
   # scheduler. You may define as many secrets as needed and the kubernetes launcher will parse the
   # defined secrets and mount them as secret environment variables in the launched workers.
   # Secrets in this section are defined as follows
   # ``<environment_variable_mount> = <kubernetes_secret_object>=<kubernetes_secret_key>``
   #
   # For example if you wanted to mount a kubernetes secret key named ``postgres_password`` from the
   # kubernetes secret object ``airflow-secret`` as the environment variable ``POSTGRES_PASSWORD`` into
   # your workers you would follow the following format:
   # ``POSTGRES_PASSWORD = airflow-secret=postgres_credentials``
   #
   # Additionally you may override worker airflow settings with the ``AIRFLOW__<SECTION>__<KEY>``
   # formatting as supported by airflow normally.
   
   [kubernetes_labels]
   
   # The Key-value pairs to be given to worker pods.
   # The worker pods will be given these static labels, as well as some additional dynamic labels
   # to identify the task.
   # Should be supplied in the format: ``key = value``
   
   ```
   
   </details>
   
   Run the following command:
   
   ```bash
   AIRFLOW_CONFIG=./airflow.cfg airflow scheduler
   ```
   
   <details>
   <summary>Stack trace</summary>
   
   ```bash
   Traceback (most recent call last):
     File "/some/path/venv/bin/airflow", line 5, in <module>
       from airflow.__main__ import main
     File "/some/path/venv/lib/python3.6/site-packages/airflow/__init__.py", line 34, in <module>
       from airflow import settings
     File "/some/path/venv/lib/python3.6/site-packages/airflow/settings.py", line 35, in <module>
       from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf  # NOQA F401
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 794, in <module>
       conf.read(AIRFLOW_CONFIG)
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 455, in read
       self._validate()
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 204, in _validate
       self._validate_config_dependencies()
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 232, in _validate_config_dependencies
       is_sqlite = "sqlite" in self.get('core', 'sql_alchemy_conn')
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 332, in get
       option = self._get_option_from_secrets(deprecated_key, deprecated_section, key, section)
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 350, in _get_option_from_secrets
       option = self._get_secret_option(section, key)
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 311, in _get_secret_option
       return _get_config_value_from_secret_backend(secrets_path)
     File "/some/path/venv/lib/python3.6/site-packages/airflow/configuration.py", line 85, in _get_config_value_from_secret_backend
       secrets_client = get_custom_secret_backend()
   NameError: name 'get_custom_secret_backend' is not defined
   ```
   
   </details>
   
   Which now matches lines no...
   


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

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



[GitHub] [airflow] potiuk edited a comment on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on issue #13254:
URL: https://github.com/apache/airflow/issues/13254#issuecomment-750015222


   But the question is if  moving get_custom_secret_backend before conf.read() is a good idea ? The problem is that the 'get_custom_secret_backend' uses the conf variable and reads the configuration from it:  
   
   ```
   conf.getimport(section='secrets', key='backend')
   conf.get(section='secrets', key='backend_kwargs', fallback='{}')
   ```
   
   Then the `get_custom_secret_backend` can be actually called BEFORE 'secrets' section is parsed via conf! 
   
   This is the 'chicken-egg' problem I mentioned before. 
   
   The behaviour will be different if  someone reads '[core'] section with `sql_aclhemy_conn' before the [secrets] or after. In our case the _validate method already reads 'core.sql_alchemy_conn' during parsing and when [secrets] is probably not yet parsed.


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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #13254:
URL: https://github.com/apache/airflow/issues/13254#issuecomment-749557371


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


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

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



[GitHub] [airflow] kaxil commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   This feels similar to https://github.com/apache/airflow/issues/11149#issuecomment-731398618
   
   Can you check my comment over there please @leopoldhoudin 


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

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



[GitHub] [airflow] potiuk commented on issue #13254: Import error when using custom backend and sql_alchemy_conn_secret

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


   Hmm. After a closer look, I think it will not work :). It's  a bit of chicken-egg-problem where we are reading coonf using the secret backend configuration but at the same time we should have the configuration already read in order to initialize secret_backed. I think the only solution will be to avoid reading from the secret backend until it is initialized.


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

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