From faa8f543d5f7bf7113076dab5f579493ec2860ec Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Thu, 20 Jul 2023 08:20:23 +0200 Subject: [PATCH] Store config description in Airflow configuration object (#32669) We would like to use the config.yml approach as our main source of truth for airflow configuration. So far it has been split amongst multiple files: * confg.yml -> descroption * default_airflow.cfg -> JINJA template to keep cofiguration, examples and description (used to generate airflow.cfg when airflow starts) * default_test.cfg -> storing test-only configuration used in some test cases - used to generate unittest.cfg * scripts/in_container/airflow_ci.cfg where dCI-specific configuration overwrote the unittest.cfg This change consolidates it all into unified appraoch, where all configuration information is retrieved from .yml files stored in config_templates. No more additional template files processed by JINJA, no more special CI versions of it, no more unittestdb.cfg file where such configuration would be generated, no more unittestdb to be used separately for tests. * The default_*.cfg files were not real configuration files, becuase they were really JINJA templates and it got peoeple confused when copying the files. This change leaves the file empty with the comment that instructs the user how they can get the default configuration. * The default_airflow.cfg is gone and instead, we have a way to show defaults via `airflow config list --defaults` command * Unittest config is generated on-the-flight using defaults stored in confg_templates. constituing a single place where they need to be changed for the tests to use them * internally, description of the configuration is stored in AirflowConfigurationParser and used by the parser to generate the default configuration when needed. * we got rid of `{{{{` in templated config defaults by marking the templates with ``is_template`` and getting rid of processing those entries with regular formatting when generating the default values. This only concerns defaults from config.yml. Reading those configuration entries from file is unaffected. This change aims to be 100% backwards compatible with the previous implementation when it comes to functionality, even if internals changed. It also does not add provider-specific changes that are coming separately. The only changes visible to the user are: * generated airflow.cfg is slightly more readable and displays names of variables that can be used to override each configuration (which is very useful for copy&pasting) * user are advised, instead of copying the default_airflow.cfg to use `airflow config list --defaults` to generate production config. This configuration has all the entries commented out, so that they can selectively uncomment and change the values they want. This is now promoted as "best practice" in the documentation. --- .github/mergeable.yml | 5 - .gitignore | 2 - .pre-commit-config.yaml | 8 - .rat-excludes | 1 - Dockerfile.ci | 1 - STATIC_CODE_CHECKS.rst | 2 - TESTING.rst | 33 +- airflow/cli/cli_config.py | 37 +- airflow/cli/commands/config_command.py | 11 +- airflow/config_templates/config.yml | 34 +- .../config_templates/config.yml.schema.json | 5 + airflow/config_templates/default_airflow.cfg | 1513 +---------------- airflow/config_templates/default_test.cfg | 68 - airflow/config_templates/unit_tests.cfg | 102 ++ airflow/configuration.py | 703 +++++--- airflow/utils/db.py | 4 +- .../src/airflow_breeze/pre_commit_ids.py | 1 - docs/apache-airflow/howto/index.rst | 1 - docs/apache-airflow/howto/set-config.rst | 27 +- docs/apache-airflow/howto/use-test-config.rst | 36 - docs/apache-airflow/redirects.txt | 3 + docs/conf.py | 4 +- images/breeze/output-commands-hash.txt | 2 +- images/breeze/output-commands.svg | 108 +- images/breeze/output_static-checks.svg | 154 +- .../ci/pre_commit/pre_commit_yaml_to_cfg.py | 179 -- scripts/docker/entrypoint_ci.sh | 1 - scripts/in_container/airflow_ci.cfg | 59 - tests/cli/commands/test_config_command.py | 138 +- tests/cli/test_cli_parser.py | 17 +- tests/config_templates/deprecated.cfg | 9 - tests/config_templates/deprecated_cmd.cfg | 10 - tests/config_templates/deprecated_secret.cfg | 10 - tests/config_templates/empty.cfg | 9 - tests/conftest.py | 15 +- tests/core/test_config_templates.py | 98 -- tests/core/test_configuration.py | 151 +- 37 files changed, 1139 insertions(+), 2422 deletions(-) delete mode 100644 airflow/config_templates/default_test.cfg create mode 100644 airflow/config_templates/unit_tests.cfg delete mode 100644 docs/apache-airflow/howto/use-test-config.rst delete mode 100755 scripts/ci/pre_commit/pre_commit_yaml_to_cfg.py delete mode 100644 scripts/in_container/airflow_ci.cfg delete mode 100644 tests/core/test_config_templates.py diff --git a/.github/mergeable.yml b/.github/mergeable.yml index e3101c6087c35..dc774f2b81594 100644 --- a/.github/mergeable.yml +++ b/.github/mergeable.yml @@ -39,8 +39,3 @@ mergeable: changed: file: 'package.json' files: ['yarn.lock'] - # If config.yml is updated, so should default_airflow.cfg - - do: dependent - changed: - file: 'config.yml' - files: ['default_airflow.cfg'] diff --git a/.gitignore b/.gitignore index a6528d887d24f..6ee528b9ce88d 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,5 @@ # Airflow configuration airflow.cfg -unittests.cfg airflow_login.py dbinit.py initdb.py @@ -8,7 +7,6 @@ secrets.py # Airflow sqlite databases airflow.db -unittests.db # Airflow temporary artifacts airflow/git_version diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 11ce7b2f28543..ff8085d03033a 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -618,14 +618,6 @@ repos: pass_filenames: true additional_dependencies: ['click', 'rich>=12.4.4', 'pyyaml'] require_serial: true - - id: check-airflow-config-yaml-consistent - name: Check consistency between config.yml and default_config.cfg - language: python - entry: ./scripts/ci/pre_commit/pre_commit_yaml_to_cfg.py - files: config\.yml$|default_airflow\.cfg$|default\.cfg$ - pass_filenames: false - require_serial: true - additional_dependencies: ['pyyaml', 'packaging'] - id: check-boring-cyborg-configuration name: Checks for Boring Cyborg configuration consistency language: python diff --git a/.rat-excludes b/.rat-excludes index 7d98d4e348652..cab48fa0eede1 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -38,7 +38,6 @@ metastore_db .*md5 .*zip .*lock -unittests.cfg logs .bash_aliases venv diff --git a/Dockerfile.ci b/Dockerfile.ci index 88e4316559856..23a24d26dd46a 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -837,7 +837,6 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then unset AIRFLOW__CORE__UNIT_TEST_MODE mkdir -pv "${AIRFLOW_HOME}/logs/" - cp -f "${IN_CONTAINER_DIR}/airflow_ci.cfg" "${AIRFLOW_HOME}/unittests.cfg" # Change the default worker_concurrency for tests export AIRFLOW__CELERY__WORKER_CONCURRENCY=8 diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst index 4bcba4bb55e29..1729c8ba8e615 100644 --- a/STATIC_CODE_CHECKS.rst +++ b/STATIC_CODE_CHECKS.rst @@ -146,8 +146,6 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-aiobotocore-optional | Check if aiobotocore is an optional dependency only | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ -| check-airflow-config-yaml-consistent | Check consistency between config.yml and default_config.cfg | | -+-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-airflow-provider-compatibility | Check compatibility of Providers with Airflow | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-apache-license-rat | Check if licenses are OK for Apache | | diff --git a/TESTING.rst b/TESTING.rst index 94b4ff7438590..09b7e9d31491c 100644 --- a/TESTING.rst +++ b/TESTING.rst @@ -55,8 +55,37 @@ Follow the guidelines when writing unit tests: tests, so we run Pytest with ``--disable-warnings`` but instead we have ``pytest-capture-warnings`` plugin that overrides ``recwarn`` fixture behaviour. -**NOTE:** We plan to convert all unit tests to standard "asserts" semi-automatically, but this will be done later -in Airflow 2.0 development phase. That will include setUp/tearDown/context managers and decorators. + +.. note:: + + We are in the process of converting all unit tests to standard "asserts" and pytest fixtures + so if you find some tests that are still using classic setUp/tearDown approach or unittest asserts, feel + free to convert them to pytest. + +Airflow configuration for unit tests +------------------------------------ + +Some of the unit tests require special configuration set as the ``default``. This is done automatically by +adding ``AIRFLOW__CORE__UNIT_TEST_MODE=True`` to the environment variables in Pytest auto-used +fixture. This in turn makes Airflow load test configuration from the file +``airflow/config_templates/unit_tests.cfg``. Test configuration from there replaces the original +defaults from ``airflow/config_templates/config.yml``. If you want to add some test-only configuration, +as default for all tests you should add the value to this file. + +You can also of course override the values in individual test by patching environment variables following +the usual ``AIRFLOW__SECTION__KEY`` pattern or ``conf_vars`` context manager. + +.. note:: + + The test configuration for Airflow before July 2023 was automatically generated in a file named + ``AIRFLOW_HOME/unittest.cfg``. The template for it was stored in "config_templates" next to the yaml file. + However writing the file was only done for the first time you run airflow and you had to manually + maintain the file. It was pretty arcane knowledge, and this generated file in {AIRFLOW_HOME} + has been overwritten in the Breeze environment with another CI-specific file. Using ``unit_tests.cfg`` + as a single source of the configuration for tests - coming from Airflow sources + rather than from {AIRFLOW_HOME} is much more convenient and it is automatically used by pytest. + + The unittest.cfg file generated in {AIRFLOW_HOME} will no longer be used and can be removed. Airflow test types ------------------ diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index 66b959051363b..79327a4701acc 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -311,6 +311,31 @@ def string_lower_type(val): ARG_MARK_SUCCESS = Arg( ("-m", "--mark-success"), help="Mark jobs as succeeded without running them", action="store_true" ) +ARG_INCLUDE_DESCRIPTIONS = Arg( + ("-d", "--include-descriptions"), + help="Show descriptions for the configuration variables", + action="store_true", +) +ARG_INCLUDE_EXAMPLES = Arg( + ("-e", "--include-examples"), help="Show examples for the configuration variables", action="store_true" +) +ARG_INCLUDE_SOURCES = Arg( + ("-s", "--include-sources"), help="Show source of the configuration variable", action="store_true" +) +ARG_INCLUDE_ENV_VARS = Arg( + ("-V", "--include-env-vars"), help="Show environment variable for each option", action="store_true" +) +ARG_COMMENT_OUT_EVERYTHING = Arg( + ("-c", "--comment-out-everything"), + help="Comment out all configuration options. Useful as starting point for new installation", + action="store_true", +) +ARG_DEFAULTS = Arg( + ("-a", "--defaults"), + help="Show only defaults - do not include local configuration, sources," + " includes descriptions, examples, variables. Comment out everything.", + action="store_true", +) ARG_VERBOSE = Arg(("-v", "--verbose"), help="Make logging output more verbose", action="store_true") ARG_LOCAL = Arg(("-l", "--local"), help="Run the task using the LocalExecutor", action="store_true") ARG_DONOT_PICKLE = Arg( @@ -2002,7 +2027,17 @@ class GroupCommand(NamedTuple): name="list", help="List options for the configuration", func=lazy_load_command("airflow.cli.commands.config_command.show_config"), - args=(ARG_OPTIONAL_SECTION, ARG_COLOR, ARG_VERBOSE), + args=( + ARG_OPTIONAL_SECTION, + ARG_COLOR, + ARG_INCLUDE_DESCRIPTIONS, + ARG_INCLUDE_EXAMPLES, + ARG_INCLUDE_SOURCES, + ARG_INCLUDE_ENV_VARS, + ARG_COMMENT_OUT_EVERYTHING, + ARG_DEFAULTS, + ARG_VERBOSE, + ), ), ) diff --git a/airflow/cli/commands/config_command.py b/airflow/cli/commands/config_command.py index f8d8cac0cb86c..8d664417237da 100644 --- a/airflow/cli/commands/config_command.py +++ b/airflow/cli/commands/config_command.py @@ -30,7 +30,16 @@ def show_config(args): """Show current application configuration.""" with io.StringIO() as output: - conf.write(output, section=args.section) + conf.write( + output, + section=args.section, + include_examples=args.include_examples or args.defaults, + include_descriptions=args.include_descriptions or args.defaults, + include_sources=args.include_sources and not args.defaults, + include_env_vars=args.include_env_vars or args.defaults, + comment_out_everything=args.comment_out_everything or args.defaults, + only_defaults=args.defaults, + ) code = output.getvalue() if should_use_colors(args): code = pygments.highlight(code=code, formatter=get_terminal_formatter(), lexer=IniLexer()) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 4dede4af281ac..0629cb5d11959 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -44,10 +44,10 @@ core: default: "airflow.utils.net.getfqdn" might_contain_dag_callable: description: | - A callable to check if a python file has airflow dags defined or not - with argument as: `(file_path: str, zip_file: zipfile.ZipFile | None = None)` - return True if it has dags otherwise False - If this is not provided, Airflow uses its own heuristic rules. + A callable to check if a python file has airflow dags defined or not + with argument as: `(file_path: str, zip_file: zipfile.ZipFile | None = None)` + return True if it has dags otherwise False + If this is not provided, Airflow uses its own heuristic rules. version_added: 2.6.0 type: string example: ~ @@ -288,8 +288,8 @@ core: default: "0" default_task_retry_delay: description: | - The number of seconds each task is going to wait by default between retries. Can be overridden at - dag or task level. + The number of seconds each task is going to wait by default between retries. Can be overridden at + dag or task level. version_added: 2.4.0 type: integer example: ~ @@ -681,7 +681,7 @@ logging: description: | The remote_task_handler_kwargs param is loaded into a dictionary and passed to __init__ of remote task handler and it overrides the values provided by Airflow config. For example if you set - `delete_local_logs=False` and you provide ``{{"delete_local_copy": true}}``, then the local + `delete_local_logs=False` and you provide ``{"delete_local_copy": true}``, then the local log files will be deleted after they are uploaded to remote location. version_added: 2.6.0 type: string @@ -804,7 +804,8 @@ logging: Specify prefix pattern like mentioned below with stream handler TaskHandlerWithCustomFormatter version_added: 2.0.0 type: string - example: "{{ti.dag_id}}-{{ti.task_id}}-{{execution_date}}-{{try_number}}" + example: "{ti.dag_id}-{ti.task_id}-{execution_date}-{try_number}" + is_template: true default: "" log_filename_template: description: | @@ -812,16 +813,18 @@ logging: version_added: 2.0.0 type: string example: ~ - default: "dag_id={{{{ ti.dag_id }}}}/run_id={{{{ ti.run_id }}}}/task_id={{{{ ti.task_id }}}}/\ - {{%% if ti.map_index >= 0 %%}}map_index={{{{ ti.map_index }}}}/{{%% endif %%}}\ - attempt={{{{ try_number }}}}.log" + is_template: true + default: "dag_id={{ ti.dag_id }}/run_id={{ ti.run_id }}/task_id={{ ti.task_id }}/\ + {%% if ti.map_index >= 0 %%}map_index={{ ti.map_index }}/{%% endif %%}\ + attempt={{ try_number }}.log" log_processor_filename_template: description: | Formatting for how airflow generates file names for log version_added: 2.0.0 type: string example: ~ - default: "{{{{ filename }}}}.log" + is_template: true + default: "{{ filename }}.log" dag_processor_manager_log_location: description: | Full path of dag_processor_manager logfile. @@ -1071,7 +1074,7 @@ secrets: 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"}}`` + ``{"connections_prefix": "/airflow/connections", "profile_name": "default"}`` version_added: 1.10.10 type: string sensitive: true @@ -1825,7 +1828,7 @@ webserver: default: "True" auth_rate_limit: description: | - Rate limit for authentication endpoints. + Rate limit for authentication endpoints. version_added: 2.6.0 type: string example: ~ @@ -2688,7 +2691,8 @@ elasticsearch: version_added: 1.10.4 type: string example: ~ - default: "{{dag_id}}-{{task_id}}-{{run_id}}-{{map_index}}-{{try_number}}" + is_template: true + default: "{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}" end_of_log_mark: description: | Used to mark the end of a log stream for a task diff --git a/airflow/config_templates/config.yml.schema.json b/airflow/config_templates/config.yml.schema.json index d800ade7d18b3..9bc321acc1500 100644 --- a/airflow/config_templates/config.yml.schema.json +++ b/airflow/config_templates/config.yml.schema.json @@ -72,6 +72,11 @@ "sensitive": { "type": "boolean", "description": "When true, this option is sensitive and can be specified using AIRFLOW__{section}___{name}__SECRET or AIRFLOW__{section}___{name}_CMD environment variables. See: airflow.configuration.AirflowConfigParser.sensitive_config_values" + }, + "is_template": { + "type": "boolean", + "description": "When true, the {VARS} have no special meaning there - they won't be expanded with env vars/local/global variables.", + "default": false } }, "required": [ diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 8046fbc18ad81..f0f9c1b60c823 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -1,4 +1,3 @@ -# # 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 @@ -16,1507 +15,31 @@ # specific language governing permissions and limitations # under the License. -# This is the template for Airflow's default configuration. When Airflow is -# imported, it looks for a configuration file at $AIRFLOW_HOME/airflow.cfg. If -# it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# airflow.cfg instead. - - -# ----------------------- TEMPLATE BEGINS HERE ----------------------- - -[core] -# The folder where your airflow pipelines live, most likely a -# subfolder in a code repository. This path must be absolute. -dags_folder = {AIRFLOW_HOME}/dags - -# Hostname by providing a path to a callable, which will resolve the hostname. -# The format is "package.function". # -# For example, default value "airflow.utils.net.getfqdn" means that result from patched -# version of socket.getfqdn() - see https://github.com/python/cpython/issues/49254. +# NOTE: # -# 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 = airflow.utils.net.getfqdn - -# A callable to check if a python file has airflow dags defined or not -# with argument as: `(file_path: str, zip_file: zipfile.ZipFile | None = None)` -# return True if it has dags otherwise False -# If this is not provided, Airflow uses its own heuristic rules. -might_contain_dag_callable = airflow.utils.file.might_contain_dag_via_default_heuristic - -# 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``, ``CeleryKubernetesExecutor`` or the -# full import path to the class when using a custom executor. -executor = SequentialExecutor - -# The auth manager class that airflow should use. Full import path to the auth manager class. -auth_manager = airflow.auth.managers.fab.fab_auth_manager.FabAuthManager - -# This defines the maximum number of task instances that can run concurrently per scheduler in -# Airflow, regardless of the worker count. Generally this value, multiplied by the number of -# schedulers in your cluster, is the maximum number of task instances with the running -# state in the metadata database. -parallelism = 32 - -# The maximum number of task instances allowed to run concurrently in each DAG. To calculate -# the number of tasks that is running concurrently for a DAG, add up the number of running -# tasks for all DAG runs of the DAG. This is configurable at the DAG level with ``max_active_tasks``, -# which is defaulted as ``max_active_tasks_per_dag``. +# IF YOU ARE LOOKING FOR DEFAULT CONFIGURATION FILE HERE - LOOK NO MORE. READ EXPLANATION BELOW! # -# An example scenario when this would be useful is when you want to stop a new dag with an early -# start date from stealing all the executor slots in a cluster. -max_active_tasks_per_dag = 16 - -# Are DAGs paused by default at creation -dags_are_paused_at_creation = True - -# The maximum number of active DAG runs per DAG. The scheduler will not create more DAG runs -# if it reaches the limit. This is configurable at the DAG level with ``max_active_runs``, -# which is defaulted as ``max_active_runs_per_dag``. -max_active_runs_per_dag = 16 - -# The name of the method used in order to start Python processes via the multiprocessing module. -# This corresponds directly with the options available in the Python docs: -# https://docs.python.org/3/library/multiprocessing.html#multiprocessing.set_start_method. -# Must be one of the values returned by: -# https://docs.python.org/3/library/multiprocessing.html#multiprocessing.get_all_start_methods. -# Example: mp_start_method = fork -# mp_start_method = - -# 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 = True - -# Path to the folder containing Airflow plugins -plugins_folder = {AIRFLOW_HOME}/plugins - -# Should tasks be executed via forking of the parent process ("False", -# the speedier option) or by spawning a new python process ("True" slow, -# but means plugin changes picked up by tasks straight away) -execute_tasks_new_python_interpreter = False - -# Secret key to save connection passwords in the db -fernet_key = {FERNET_KEY} - -# Whether to disable pickling dags -donot_pickle = True - -# How long before timing out a python file import -dagbag_import_timeout = 30.0 - -# Should a traceback be shown in the UI for dagbag import errors, -# instead of just the exception message -dagbag_import_error_tracebacks = True - -# If tracebacks are shown, how many entries from the traceback should be shown -dagbag_import_error_traceback_depth = 2 - -# 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. -# Choices include StandardTaskRunner, CgroupTaskRunner or the full import path to the class -# when using a custom task runner. -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 = - -# 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). -enable_xcom_pickling = False - -# What classes can be imported during deserialization. This is a multi line value. -# The individual items will be parsed as regexp. Python built-in classes (like dict) -# are always allowed. Bare "." will be replaced so you can set airflow.* . -allowed_deserialization_classes = airflow\..* - -# 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 = True - -# If enabled, Airflow will only scan files containing both ``DAG`` and ``airflow`` (case-insensitive). -dag_discovery_safe_mode = True - -# The pattern syntax used in the ".airflowignore" files in the DAG directories. Valid values are -# ``regexp`` or ``glob``. -dag_ignore_file_syntax = regexp - -# The number of retries each task is going to have by default. Can be overridden at dag or task level. -default_task_retries = 0 - -# The number of seconds each task is going to wait by default between retries. Can be overridden at -# dag or task level. -default_task_retry_delay = 300 - -# The maximum delay (in seconds) each task is going to wait by default between retries. -# This is a global setting and cannot be overridden at task or DAG level. -max_task_retry_delay = 86400 - -# The weighting method used for the effective total priority weight of the task -default_task_weight_rule = downstream - -# The default task execution_timeout value for the operators. Expected an integer value to -# be passed into timedelta as seconds. If not specified, then the value is considered as None, -# meaning that the operators are never timed out by default. -default_task_execution_timeout = - -# Updating serialized DAG can not be faster than a minimum interval to reduce database write rate. -min_serialized_dag_update_interval = 30 - -# If True, serialized DAGs are compressed before writing to DB. -# Note: this will disable the DAG dependencies view -compress_serialized_dags = False - -# 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 - -# Maximum number of Rendered Task Instance Fields (Template Fields) per task to store -# in the Database. -# 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 - -# By default Airflow plugins are lazily-loaded (only loaded when required). Set it to ``False``, -# if you want to load plugins whenever 'airflow' is invoked via cli or loaded from module. -lazy_load_plugins = True - -# By default Airflow providers are lazily-discovered (discovery and imports happen only when required). -# Set it to False, if you want to discover providers whenever 'airflow' is invoked via cli or -# loaded from module. -lazy_discover_providers = True - -# Hide sensitive Variables or Connection extra json keys from UI and task logs when set to True +# This file used to have something that was similar to the default Airflow configuration but it was +# really just a template. It was used to generate the final configuration and it was confusing +# if you copied it to your configuration and some of values were wrong. # -# (Connection passwords are always hidden in logs) -hide_sensitive_var_conn_fields = True - -# A comma-separated list of extra sensitive keywords to look for in variables names or connection's -# extra JSON. -sensitive_var_conn_names = - -# Task Slot counts for ``default_pool``. This setting would not have any effect in an existing -# deployment where the ``default_pool`` is already created. For existing deployments, users can -# change the number of slots using Webserver, API or the CLI -default_pool_task_slot_count = 128 - -# The maximum list/dict length an XCom can push to trigger task mapping. If the pushed list/dict has a -# length exceeding this value, the task pushing the XCom will be failed automatically to prevent the -# mapped tasks from clogging the scheduler. -max_map_length = 1024 - -# The default umask to use for process when run in daemon mode (scheduler, worker, etc.) +# The first time you run Airflow, it will create a file called ``airflow.cfg`` in +# your ``$AIRFLOW_HOME`` directory (``~/airflow`` by default). This is in order to make it easy to +# "play" with airflow configuration. # -# This controls the file-creation mode mask which determines the initial value of file permission bits -# for newly created files. +# However, for production case you are advised to generate the configuration using command line: # -# This value is treated as an octal-integer. -daemon_umask = 0o077 - -# Class to use as dataset manager. -# Example: dataset_manager_class = airflow.datasets.manager.DatasetManager -# dataset_manager_class = - -# Kwargs to supply to dataset manager. -# Example: dataset_manager_kwargs = {{"some_param": "some_value"}} -# dataset_manager_kwargs = - -# (experimental) Whether components should use Airflow Internal API for DB connectivity. -database_access_isolation = False - -# (experimental) Airflow Internal API url. Only used if [core] database_access_isolation is True. -# Example: internal_api_url = http://localhost:8080 -# internal_api_url = - -# The ability to allow testing connections across Airflow UI, API and CLI. -# Supported options: Disabled, Enabled, Hidden. Default: Disabled -# Disabled - Disables the test connection functionality and disables the Test Connection button in UI. -# Enabled - Enables the test connection functionality and shows the Test Connection button in UI. -# Hidden - Disables the test connection functionality and hides the Test Connection button in UI. -# Before setting this to Enabled, make sure that you review the users who are able to add/edit -# connections and ensure they are trusted. Connection testing can be done maliciously leading to -# undesired and insecure outcomes. For more information on capabilities of users, see the documentation: -# https://airflow.apache.org/docs/apache-airflow/stable/security/index.html#capabilities-of-authenticated-ui-users -test_connection = Disabled - -[database] -# Path to the ``alembic.ini`` file. You can either provide the file path relative -# to the Airflow home directory or the absolute path if it is located elsewhere. -alembic_ini_file_path = alembic.ini - -# The SqlAlchemy connection string to the metadata database. -# SqlAlchemy supports many different database engines. -# More information here: -# http://airflow.apache.org/docs/apache-airflow/stable/howto/set-up-database.html#database-uri -sql_alchemy_conn = sqlite:///{AIRFLOW_HOME}/airflow.db - -# Extra engine specific keyword args passed to SQLAlchemy's create_engine, as a JSON-encoded value -# Example: sql_alchemy_engine_args = {{"arg1": True}} -# sql_alchemy_engine_args = - -# The encoding for the databases -sql_engine_encoding = utf-8 - -# Collation for ``dag_id``, ``task_id``, ``key``, ``external_executor_id`` columns -# in case they have different encoding. -# By default this collation is the same as the database collation, however for ``mysql`` and ``mariadb`` -# the default is ``utf8mb3_bin`` so that the index sizes of our index keys will not exceed -# the maximum size of allowed index when collation is set to ``utf8mb4`` variant -# (see https://github.com/apache/airflow/pull/17603#issuecomment-901121618). -# sql_engine_collation_for_ids = - -# 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/14/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. Defaults 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/14/core/engines.html#sqlalchemy.create_engine.params.connect_args -# Example: sql_alchemy_connect_args = {{"timeout": 30}} -# sql_alchemy_connect_args = - -# 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 = True - -# Number of times the code should be retried in case of DB Operational Errors. -# Not all transactions will be retried as it can cause undesired state. -# Currently it is only used in ``DagFileProcessor.process_file`` to retry ``dagbag.sync_to_db``. -max_db_retries = 3 - -# Whether to run alembic migrations during Airflow start up. Sometimes this operation can be expensive, -# and the users can assert the correct version through other means (e.g. through a Helm chart). -# Accepts "True" or "False". -check_migrations = True - -[logging] -# The folder where airflow should store its log files. -# This path must be absolute. -# There are a few existing configurations that assume this is set to the default. -# If you choose to override this you may need to update the dag_processor_manager_log_location and -# dag_processor_manager_log_location settings as well. -base_log_folder = {AIRFLOW_HOME}/logs - -# 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 - -# Users must supply an Airflow connection id that provides access to the storage -# location. Depending on your remote logging service, this may only be used for -# reading logs, not writing them. -remote_log_conn_id = - -# Whether the local log files for GCS, S3, WASB and OSS remote logging should be deleted after -# they are uploaded to the remote location. -delete_local_logs = False - -# Path to Google Credential JSON file. If omitted, authorization based on `the Application Default -# Credentials -# `__ will -# be used. -google_key_path = - -# Storage bucket URL for remote logging -# S3 buckets should start with "s3://" -# Cloudwatch log groups should start with "cloudwatch://" -# GCS buckets should start with "gs://" -# WASB buckets should start with "wasb" just to help Airflow select correct handler -# Stackdriver logs should start with "stackdriver://" -remote_base_log_folder = - -# The remote_task_handler_kwargs param is loaded into a dictionary and passed to __init__ of remote -# task handler and it overrides the values provided by Airflow config. For example if you set -# `delete_local_logs=False` and you provide ``{{"delete_local_copy": true}}``, then the local -# log files will be deleted after they are uploaded to remote location. -# Example: remote_task_handler_kwargs = {{"delete_local_copy": true}} -remote_task_handler_kwargs = - -# Use server-side encryption for logs stored in S3 -encrypt_s3_logs = False - -# Logging level. +# airflow config list --defaults # -# Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``. -logging_level = INFO - -# Logging level for celery. If not set, it uses the value of logging_level +# This command will produce the output that you can copy to your configuration file and edit. +# It will contain all the default configuration options, with examples, nicely commented out +# so you need only un-comment and modify those that you want to change. +# This way you can easily keep track of all the configuration options that you changed from default +# and you can also easily upgrade your installation to new versions of Airflow when they come out and +# automatically use the defaults for existing options if they changed there. # -# Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``. -celery_logging_level = - -# Logging level for Flask-appbuilder UI. +# You can redirect it to your configuration file and edit it: # -# Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``. -fab_logging_level = WARNING - -# 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 - -# 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 -colored_formatter_class = airflow.utils.log.colored_log.CustomTTYColoredFormatter - -# 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 - -# Where to send dag parser logs. If "file", logs are sent to log files defined by child_process_log_directory. -dag_processor_log_target = file - -# Format of Dag Processor Log line -dag_processor_log_format = [%%(asctime)s] [SOURCE:DAG_PROCESSOR] {{%%(filename)s:%%(lineno)d}} %%(levelname)s - %%(message)s -log_formatter_class = airflow.utils.log.timezone_aware.TimezoneAware - -# An import path to a function to add adaptations of each secret added with -# `airflow.utils.log.secrets_masker.mask_secret` to be masked in log messages. The given function -# is expected to require a single parameter: the secret to be adapted. It may return a -# single adaptation of the secret or an iterable of adaptations to each be masked as secrets. -# The original secret will be masked as well as any adaptations returned. -# Example: secret_mask_adapter = urllib.parse.quote -secret_mask_adapter = - -# Specify prefix pattern like mentioned below with stream handler TaskHandlerWithCustomFormatter -# Example: task_log_prefix_template = {{ti.dag_id}}-{{ti.task_id}}-{{execution_date}}-{{try_number}} -task_log_prefix_template = - -# Formatting for how airflow generates file names/paths for each task run. -log_filename_template = dag_id={{{{ ti.dag_id }}}}/run_id={{{{ ti.run_id }}}}/task_id={{{{ ti.task_id }}}}/{{%% if ti.map_index >= 0 %%}}map_index={{{{ ti.map_index }}}}/{{%% endif %%}}attempt={{{{ try_number }}}}.log - -# Formatting for how airflow generates file names for log -log_processor_filename_template = {{{{ filename }}}}.log - -# Full path of dag_processor_manager logfile. -dag_processor_manager_log_location = {AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log - -# Name of handler to read task instance logs. -# Defaults to use ``task`` handler. -task_log_reader = task - -# A comma\-separated list of third-party logger names that will be configured to print messages to -# consoles\. -# Example: extra_logger_names = connexion,sqlalchemy -extra_logger_names = - -# When you start an airflow worker, airflow starts a tiny web server -# subprocess to serve the workers local log files to the airflow main -# web server, who then builds pages and sends them to users. This defines -# the port on which the logs are served. It needs to be unused, and open -# visible from the main web server to connect into the workers. -worker_log_server_port = 8793 - -# Port to serve logs from for triggerer. See worker_log_server_port description -# for more info. -trigger_log_server_port = 8794 - -# We must parse timestamps to interleave logs between trigger and task. To do so, -# we need to parse timestamps in log files. In case your log format is non-standard, -# you may provide import path to callable which takes a string log line and returns -# the timestamp (datetime.datetime compatible). -# Example: interleave_timestamp_parser = path.to.my_func -# interleave_timestamp_parser = - -# Permissions in the form or of octal string as understood by chmod. The permissions are important -# when you use impersonation, when logs are written by a different user than airflow. The most secure -# way of configuring it in this case is to add both users to the same group and make it the default -# group of both users. Group-writeable logs are default in airflow, but you might decide that you are -# OK with having the logs other-writeable, in which case you should set it to `0o777`. You might -# decide to add more security if you do not use impersonation and change it to `0o755` to make it -# only owner-writeable. You can also make it just readable only for owner by changing it to `0o700` if -# all the access (read/write) for your logs happens from the same user. -# Example: file_task_handler_new_folder_permissions = 0o775 -file_task_handler_new_folder_permissions = 0o775 - -# Permissions in the form or of octal string as understood by chmod. The permissions are important -# when you use impersonation, when logs are written by a different user than airflow. The most secure -# way of configuring it in this case is to add both users to the same group and make it the default -# group of both users. Group-writeable logs are default in airflow, but you might decide that you are -# OK with having the logs other-writeable, in which case you should set it to `0o666`. You might -# decide to add more security if you do not use impersonation and change it to `0o644` to make it -# only owner-writeable. You can also make it just readable only for owner by changing it to `0o600` if -# all the access (read/write) for your logs happens from the same user. -# Example: file_task_handler_new_file_permissions = 0o664 -file_task_handler_new_file_permissions = 0o664 - -# By default Celery sends all logs into stderr. -# If enabled any previous logging handlers will get *removed*. -# With this option AirFlow will create new handlers -# and send low level logs like INFO and WARNING to stdout, -# while sending higher severity logs to stderr. -celery_stdout_stderr_separation = False - -[metrics] - -# StatsD (https://github.com/etsy/statsd) integration settings. -# If you want to avoid emitting all the available metrics, you can configure an -# allow list of prefixes (comma separated) to send only the metrics that start -# with the elements of the list (e.g: "scheduler,executor,dagrun") -metrics_allow_list = - -# If you want to avoid emitting all the available metrics, you can configure a -# block list of prefixes (comma separated) to filter out metrics that start with -# the elements of the list (e.g: "scheduler,executor,dagrun"). -# If metrics_allow_list and metrics_block_list are both configured, metrics_block_list is ignored. -metrics_block_list = - -# Enables sending metrics to StatsD. -statsd_on = False -statsd_host = localhost -statsd_port = 8125 -statsd_prefix = airflow - -# A function that validate the StatsD stat name, apply changes to the stat name if necessary and return -# the transformed stat name. -# -# The function should have the following signature: -# def func_name(stat_name: str) -> str: -stat_name_handler = - -# To enable datadog integration to send airflow metrics. -statsd_datadog_enabled = False - -# List of datadog tags attached to all metrics(e.g: key1:value1,key2:value2) -statsd_datadog_tags = - -# Set to False to disable metadata tags for some of the emitted metrics -statsd_datadog_metrics_tags = True - -# If you want to utilise your own custom StatsD client set the relevant -# module path below. -# Note: The module path must exist on your PYTHONPATH for Airflow to pick it up -# statsd_custom_client_path = - -# If you want to avoid sending all the available metrics tags to StatsD, -# you can configure a block list of prefixes (comma separated) to filter out metric tags -# that start with the elements of the list (e.g: "job_id,run_id") -# Example: statsd_disabled_tags = job_id,run_id,dag_id,task_id -statsd_disabled_tags = job_id,run_id - -# To enable sending Airflow metrics with StatsD-Influxdb tagging convention. -statsd_influxdb_enabled = False - -# Enables sending metrics to OpenTelemetry. -otel_on = False -otel_host = localhost -otel_port = 8889 -otel_prefix = airflow -otel_interval_milliseconds = 60000 - -# If True, all metrics are also emitted to the console. Defaults to False. -otel_debugging_on = False - -[secrets] -# Full class name of secrets backend to enable (will precede env vars and metastore in search path) -# Example: backend = airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend -backend = - -# 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 = - -[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] -# Enables the deprecated experimental API. Please note that these APIs do not have access control. -# The authenticated user has full access. -# -# .. warning:: -# -# This `Experimental REST API `__ is -# deprecated since version 2.0. Please consider using -# `the Stable REST API `__. -# For more information on migration, see -# `RELEASE_NOTES.rst `_ -enable_experimental_api = False - -# Comma separated list of auth backends to authenticate users of the API. See -# https://airflow.apache.org/docs/apache-airflow/stable/security/api.html for possible values. -# ("airflow.api.auth.backend.default" allows all requests for historic reasons) -auth_backends = airflow.api.auth.backend.session - -# Used to set the maximum page limit for API requests. If limit passed as param -# is greater than maximum page limit, it will be ignored and maximum page limit value -# will be set as the limit -maximum_page_limit = 100 - -# Used to set the default page limit when limit param is zero or not provided in API -# requests. Otherwise if positive integer is passed in the API requests as limit, the -# smallest number of user given limit or maximum page limit is taken as limit. -fallback_page_limit = 100 - -# The intended audience for JWT token credentials used for authorization. This value must match on the client and server sides. If empty, audience will not be tested. -# Example: google_oauth2_audience = project-id-random-value.apps.googleusercontent.com -google_oauth2_audience = - -# Path to Google Cloud Service Account key file (JSON). If omitted, authorization based on -# `the Application Default Credentials -# `__ will -# be used. -# Example: google_key_path = /files/service-account-json -google_key_path = - -# Used in response to a preflight request to indicate which HTTP -# headers can be used when making the actual request. This header is -# the server side response to the browser's -# Access-Control-Request-Headers header. -access_control_allow_headers = - -# Specifies the method or methods allowed when accessing the resource. -access_control_allow_methods = - -# Indicates whether the response can be shared with requesting code from the given origins. -# Separate URLs with space. -access_control_allow_origins = - -# Indicates whether the *xcomEntries* endpoint supports the *deserialize* -# flag. If set to False, setting this flag in a request would result in a -# 400 Bad Request error. -enable_xcom_deserialize_support = False - -[lineage] -# what lineage backend to use -backend = - -[openlineage] - -# This section applies settings for OpenLineage integration. -# For backwards compatibility with `openlineage-python` one can still use -# `openlineage.yml` file or `OPENLINEAGE_` environment variables. However, below -# configuration takes precedence over those. -# More in documentation - https://openlineage.io/docs/client/python#configuration. -# Set this to true if you don't want OpenLineage to emit events. -disabled = False - -# OpenLineage namespace -# Example: namespace = food_delivery -# namespace = - -# Comma-separated paths to custom OpenLineage extractors. -# Example: extractors = full.path.to.ExtractorClass;full.path.to.AnotherExtractorClass -extractors = - -# Path to YAML config. This provides backwards compatibility to pass config as -# `openlineage.yml` file. -config_path = - -# OpenLineage Client transport configuration. It should contain type -# and additional options per each type. -# -# Currently supported types are: -# -# * HTTP -# * Kafka -# * Console -# Example: transport = {{"type": "http", "url": "http://localhost:5000"}} -transport = - -# If disabled, OpenLineage events do not contain source code of particular -# operators, like PythonOperator. -# disable_source_code = - -[atlas] -sasl_enabled = False -host = -port = 21000 -username = -password = - -[operators] -# The default owner assigned to each new operator, unless -# provided explicitly or passed via ``default_args`` -default_owner = airflow - -# The default value of attribute "deferrable" in operators and sensors. -default_deferrable = false -default_cpus = 1 -default_ram = 512 -default_disk = 512 -default_gpus = 0 - -# Default queue that tasks get assigned to and that worker listen on. -default_queue = default - -# Is allowed to pass additional/unused arguments (args, kwargs) to the BaseOperator operator. -# If set to False, an exception will be thrown, otherwise only the console message will be displayed. -allow_illegal_arguments = False - -[hive] -# Default mapreduce queue for HiveOperator tasks -default_hive_mapred_queue = - -# Template for mapred_job_name in HiveOperator, supports the following named parameters -# hostname, dag_id, task_id, execution_date -# mapred_job_name_template = - -[webserver] -# Path of webserver config file used for configuring the webserver parameters -config_file = {AIRFLOW_HOME}/webserver_config.py - -# 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 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 = - -# The type of backend used to store web session data, can be 'database' or 'securecookie' -# Example: session_backend = securecookie -session_backend = database - -# 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 = 6000 - -# 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. However, when running -# more than 1 instances of webserver, make sure all of them use the same ``secret_key`` otherwise -# one of them will error with "CSRF session token is missing". -# The webserver key is also used to authorize requests to Celery workers when logs are retrieved. -# The token generated using the secret key has a short expiry time though - make sure that time on -# ALL the machines that you run airflow components on is synchronized (for example using ntpd) -# otherwise you might get "forbidden" errors when the logs are accessed. -secret_key = {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. Note when using gevent you might also want to set the -# "_AIRFLOW_PATCH_GEVENT" environment variable to "1" to make sure gevent patching is done as -# early as possible. -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 = - - -# Access log format for gunicorn webserver. -# default format is %%(h)s %%(l)s %%(u)s %%(t)s "%%(r)s" %%(s)s %%(b)s "%%(f)s" "%%(a)s" -# documentation - https://docs.gunicorn.org/en/stable/settings.html#access-log-format -access_logformat = - -# Expose the configuration file in the web server. Set to "non-sensitive-only" to show all values -# except those that have security implications. "True" shows all values. "False" hides the -# configuration completely. -expose_config = False - -# Expose hostname in the web server -expose_hostname = False - -# Expose stacktrace in the web server -expose_stacktrace = False - -# Default DAG view. Valid values are: ``grid``, ``graph``, ``duration``, ``gantt``, ``landing_times`` -dag_default_view = grid - -# Default DAG orientation. Valid values are: -# ``LR`` (Left->Right), ``TB`` (Top->Bottom), ``RL`` (Right->Left), ``BT`` (Bottom->Top) -dag_orientation = LR - -# Sorting order in grid view. Valid values are: ``topological``, ``hierarchical_alphabetical`` -grid_view_sorting_order = topological - -# 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 - -# 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 = Lax - -# 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 = - -# 'Recent Tasks' stats will show for old DagRuns if set -show_recent_stats_for_completed_runs = True - -# Update FAB permissions and sync security manager roles -# on webserver startup -update_fab_perms = True - -# The UI cookie lifetime in minutes. User will be logged out from UI after -# ``session_lifetime_minutes`` of non-activity -session_lifetime_minutes = 43200 - -# Sets a custom page title for the DAGs overview page and site title for all pages -# instance_name = - -# Whether the custom page title for the DAGs overview page contains any Markup language -instance_name_has_markup = False - -# How frequently, in seconds, the DAG data will auto-refresh in graph or grid view -# when auto-refresh is turned on -auto_refresh_interval = 3 - -# Boolean for displaying warning for publicly viewable deployment -warn_deployment_exposure = True - -# Comma separated string of view events to exclude from dag audit view. -# All other events will be added minus the ones passed here. -# The audit logs in the db will not be affected by this parameter. -audit_view_excluded_events = gantt,landing_times,tries,duration,calendar,graph,grid,tree,tree_data - -# Comma separated string of view events to include in dag audit view. -# If passed, only these events will populate the dag audit view. -# The audit logs in the db will not be affected by this parameter. -# Example: audit_view_included_events = dagrun_cleared,failed -# audit_view_included_events = - -# Boolean for running SwaggerUI in the webserver. -enable_swagger_ui = True - -# Boolean for running Internal API in the webserver. -run_internal_api = False - -# Boolean for enabling rate limiting on authentication endpoints. -auth_rate_limited = True - -# Rate limit for authentication endpoints. -auth_rate_limit = 5 per 40 second - -# The caching algorithm used by the webserver. Must be a valid hashlib function name. -# Example: caching_hash_method = sha256 -caching_hash_method = md5 - -[email] - -# Configuration email backend and whether to -# send email alerts on retry or failure -# Email backend to use -email_backend = airflow.utils.email.send_email_smtp - -# Email connection to use -email_conn_id = smtp_default - -# Whether email alerts should be sent when a task is retried -default_email_on_retry = True - -# Whether email alerts should be sent when a task failed -default_email_on_failure = True - -# File that will be used as the template for Email subject (which will be rendered using Jinja2). -# If not set, Airflow uses a base template. -# Example: subject_template = /path/to/my_subject_template_file -# subject_template = - -# File that will be used as the template for Email content (which will be rendered using Jinja2). -# If not set, Airflow uses a base template. -# Example: html_content_template = /path/to/my_html_content_template_file -# html_content_template = - -# Email address that will be used as sender address. -# It can either be raw email or the complete address in a format ``Sender Name `` -# Example: from_email = Airflow -# from_email = - -[smtp] - -# If you want airflow to send emails on retries, failure, and you want to use -# the airflow.utils.email.send_email_smtp function, you have to configure an -# smtp server here -smtp_host = localhost -smtp_starttls = True -smtp_ssl = False -# Example: smtp_user = airflow -# smtp_user = -# Example: smtp_password = airflow -# smtp_password = -smtp_port = 25 -smtp_mail_from = airflow@example.com -smtp_timeout = 30 -smtp_retry_limit = 5 - -[sentry] - -# Sentry (https://docs.sentry.io) integration. Here you can supply -# additional configuration options based on the Python platform. See: -# https://docs.sentry.io/error-reporting/configuration/?platform=python. -# Unsupported options: ``integrations``, ``in_app_include``, ``in_app_exclude``, -# ``ignore_errors``, ``before_breadcrumb``, ``transport``. -# Enable error reporting to Sentry -sentry_on = false -sentry_dsn = - -# Dotted path to a before_send function that the sentry SDK should be configured to use. -# before_send = - -[local_kubernetes_executor] - -# This section only applies if you are using the ``LocalKubernetesExecutor`` in -# ``[core]`` section above -# Define when to send a task to ``KubernetesExecutor`` when using ``LocalKubernetesExecutor``. -# When the queue of a task is the value of ``kubernetes_queue`` (default ``kubernetes``), -# the task is executed via ``KubernetesExecutor``, -# otherwise via ``LocalExecutor`` -kubernetes_queue = kubernetes - -[celery_kubernetes_executor] - -# This section only applies if you are using the ``CeleryKubernetesExecutor`` in -# ``[core]`` section above -# Define when to send a task to ``KubernetesExecutor`` when using ``CeleryKubernetesExecutor``. -# When the queue of a task is the value of ``kubernetes_queue`` (default ``kubernetes``), -# the task is executed via ``KubernetesExecutor``, -# otherwise via ``CeleryExecutor`` -kubernetes_queue = kubernetes - -[celery] - -# This section only applies if you are using the CeleryExecutor in -# ``[core]`` section above -# The app name that will be used by celery -celery_app_name = airflow.providers.celery.executors.celery_executor - -# The concurrency that will be used when starting workers with the -# ``airflow celery worker`` command. This defines the number of task instances that -# a worker will take, so size up your workers based on the resources on -# your worker box and the nature of your tasks -worker_concurrency = 16 - -# The maximum and minimum concurrency that will be used when starting workers with the -# ``airflow celery worker`` command (always keep minimum processes, but grow -# to maximum if necessary). Note the value should be max_concurrency,min_concurrency -# Pick these numbers based on resources on worker box and the nature of the task. -# If autoscale option is available, worker_concurrency will be ignored. -# http://docs.celeryproject.org/en/latest/reference/celery.bin.worker.html#cmdoption-celery-worker-autoscale -# Example: worker_autoscale = 16,12 -# worker_autoscale = - -# Used to increase the number of tasks that a worker prefetches which can improve performance. -# The number of processes multiplied by worker_prefetch_multiplier is the number of tasks -# that are prefetched by a worker. A value greater than 1 can result in tasks being unnecessarily -# blocked if there are multiple workers and one worker prefetches tasks that sit behind long -# running tasks while another worker has unutilized processes that are unable to process the already -# claimed blocked tasks. -# https://docs.celeryproject.org/en/stable/userguide/optimizing.html#prefetch-limits -worker_prefetch_multiplier = 1 - -# Specify if remote control of the workers is enabled. -# In some cases when the broker does not support remote control, Celery creates lots of -# ``.*reply-celery-pidbox`` queues. You can prevent this by setting this to false. -# However, with this disabled Flower won't work. -# https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/index.html#broker-overview -worker_enable_remote_control = true - -# The Celery broker URL. Celery supports RabbitMQ, Redis and experimentally -# a sqlalchemy database. Refer to the Celery documentation for more information. -broker_url = redis://redis:6379/0 - -# The Celery result_backend. When a job finishes, it needs to update the -# metadata of the job. Therefore it will post a message on a message bus, -# or insert it into a database (depending of the backend) -# This status is used by the scheduler to update the state of the task -# The use of a database is highly recommended -# When not specified, sql_alchemy_conn with a db+ scheme prefix will be used -# http://docs.celeryproject.org/en/latest/userguide/configuration.html#task-result-backend-settings -# Example: result_backend = db+postgresql://postgres:airflow@postgres/airflow -# result_backend = - -# Optional configuration dictionary to pass to the Celery result backend SQLAlchemy engine. -# Example: result_backend_sqlalchemy_engine_options = {{"pool_recycle": 1800}} -result_backend_sqlalchemy_engine_options = - -# Celery Flower is a sweet UI for Celery. Airflow has a shortcut to start -# it ``airflow celery flower``. This defines the IP that Celery Flower runs on -flower_host = 0.0.0.0 - -# The root URL for Flower -# Example: flower_url_prefix = /flower -flower_url_prefix = - -# This defines the port that Celery Flower runs on -flower_port = 5555 - -# Securing Flower with Basic Authentication -# Accepts user:password pairs separated by a comma -# Example: flower_basic_auth = user1:password1,user2:password2 -flower_basic_auth = - -# How many processes CeleryExecutor uses to sync task state. -# 0 means to use max(1, number of cores - 1) processes. -sync_parallelism = 0 - -# Import path for celery configuration options -celery_config_options = airflow.providers.celery.executors.default_celery.DEFAULT_CELERY_CONFIG -ssl_active = False - -# Path to the client key. -ssl_key = - -# Path to the client certificate. -ssl_cert = - -# Path to the CA certificate. -ssl_cacert = - -# Celery Pool implementation. -# Choices include: ``prefork`` (default), ``eventlet``, ``gevent`` or ``solo``. -# See: -# https://docs.celeryproject.org/en/latest/userguide/workers.html#concurrency -# https://docs.celeryproject.org/en/latest/userguide/concurrency/eventlet.html -pool = prefork - -# The number of seconds to wait before timing out ``send_task_to_executor`` or -# ``fetch_celery_task_state`` operations. -operation_timeout = 1.0 - -# Celery task will report its status as 'started' when the task is executed by a worker. -# This is used in Airflow to keep track of the running tasks and if a Scheduler is restarted -# or run in HA mode, it can adopt the orphan tasks launched by previous SchedulerJob. -task_track_started = True - -# The Maximum number of retries for publishing task messages to the broker when failing -# due to ``AirflowTaskTimeout`` error before giving up and marking Task as failed. -task_publish_max_retries = 3 - -# Worker initialisation check to validate Metadata Database connection -worker_precheck = False - -[celery_broker_transport_options] - -# This section is for specifying options which can be passed to the -# underlying celery broker transport. See: -# http://docs.celeryproject.org/en/latest/userguide/configuration.html#std:setting-broker_transport_options -# The visibility timeout defines the number of seconds to wait for the worker -# to acknowledge the task before the message is redelivered to another worker. -# Make sure to increase the visibility timeout to match the time of the longest -# ETA you're planning to use. -# visibility_timeout is only supported for Redis and SQS celery brokers. -# See: -# https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/redis.html#visibility-timeout -# Example: visibility_timeout = 21600 -# visibility_timeout = - -# The sentinel_kwargs parameter allows passing additional options to the Sentinel client. -# In a typical scenario where Redis Sentinel is used as the broker and Redis servers are -# password-protected, the password needs to be passed through this parameter. Although its -# type is string, it is required to pass a string that conforms to the dictionary format. -# See: -# https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/redis.html#configuration -# Example: sentinel_kwargs = {{"password": "password_for_redis_server"}} -# sentinel_kwargs = - -[dask] - -# This section only applies if you are using the DaskExecutor in -# [core] section above -# The IP address and port of the Dask cluster's scheduler. -cluster_address = 127.0.0.1:8786 - -# Path to a CA certificate file encoded in PEM format to access a secured Dask scheduler. -tls_ca = - -# Path to a certificate file for the client, encoded in PEM format. -tls_cert = - -# Path to a key file for the client, encoded in PEM format. -tls_key = - -[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 - -# The frequency (in seconds) at which the LocalTaskJob should send heartbeat signals to the -# scheduler to notify it's still alive. If this value is set to 0, the heartbeat interval will default -# to the value of scheduler_zombie_task_threshold. -local_task_job_heartbeat_sec = 0 - -# The number of times to try to schedule each DAG file -# -1 indicates unlimited number -num_runs = -1 - -# Controls how long the scheduler will sleep between loops, but if there was nothing to do -# in the loop. i.e. if it scheduled something then it will start the next loop -# iteration straight away. -scheduler_idle_sleep_time = 1 - -# Number of seconds after which a DAG file is parsed. The DAG file is parsed every -# ``min_file_process_interval`` number of seconds. Updates to DAGs are reflected after -# this interval. Keeping this number low will increase CPU usage. -min_file_process_interval = 30 - -# How often (in seconds) to check for stale DAGs (DAGs which are no longer present in -# the expected files) which should be deactivated, as well as datasets that are no longer -# referenced and should be marked as orphaned. -parsing_cleanup_interval = 60 - -# How long (in seconds) to wait after we have re-parsed a DAG file before deactivating stale -# DAGs (DAGs which are no longer present in the expected files). The reason why we need -# this threshold is to account for the time between when the file is parsed and when the -# DAG is loaded. The absolute maximum that this could take is `dag_file_processor_timeout`, -# but when you have a long timeout configured, it results in a significant delay in the -# deactivation of stale dags. -stale_dag_threshold = 50 - -# 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 - -# How often (in seconds) should pool usage stats be sent to StatsD (if statsd_on is enabled) -pool_metrics_interval = 5.0 - -# 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 and in `airflow jobs check` CLI -# for SchedulerJob. -scheduler_health_check_threshold = 30 - -# When you start a scheduler, airflow starts a tiny web server -# subprocess to serve a health check if this is set to True -enable_health_check = False - -# When you start a scheduler, airflow starts a tiny web server -# subprocess to serve a health check on this port -scheduler_health_check_server_port = 8974 - -# How often (in seconds) should the scheduler check for orphaned tasks and SchedulerJobs -orphaned_tasks_check_interval = 300.0 -child_process_log_directory = {AIRFLOW_HOME}/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 - -# How often (in seconds) should the scheduler check for zombie tasks. -zombie_detection_interval = 10.0 - -# 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 = True - -# Setting this to True will make first task instance of a task -# ignore depends_on_past setting. A task instance will be considered -# as the first task instance of a task when there is no task instance -# in the DB with an execution_date earlier than it., i.e. no manual marking -# success will be needed for a newly added task to be scheduled. -ignore_first_depends_on_past_by_default = True - -# This changes the batch size of queries in the scheduling main loop. -# If this is too high, SQL query performance may be impacted by -# complexity of query predicate, and/or 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 - -# Should the scheduler issue ``SELECT ... FOR UPDATE`` in relevant queries. -# If this is set to False then you should not run more than a single -# scheduler at once -use_row_level_locking = True - -# Max number of DAGs to create DagRuns for per scheduler loop. -max_dagruns_to_create_per_loop = 10 - -# How many DagRuns should a scheduler examine (and lock) when scheduling -# and queuing tasks. -max_dagruns_per_loop_to_schedule = 20 - -# Should the Task supervisor process perform a "mini scheduler" to attempt to schedule more tasks of the -# same DAG. Leaving this on will mean tasks in the same DAG execute quicker, but might starve out other -# dags in some circumstances -schedule_after_task_execution = True - -# The scheduler reads dag files to extract the airflow modules that are going to be used, -# and imports them ahead of time to avoid having to re-do it for each parsing process. -# This flag can be set to False to disable this behavior in case an airflow module needs to be freshly -# imported each time (at the cost of increased DAG parsing time). -parsing_pre_import_modules = True - -# The scheduler can run multiple processes in parallel to parse dags. -# This defines how many processes will run. -parsing_processes = 2 - -# One of ``modified_time``, ``random_seeded_by_host`` and ``alphabetical``. -# The scheduler will list and sort the dag files to decide the parsing order. +# airflow config list --defaults > "${AIRFLOW_HOME}/airflow.cfg" # -# * ``modified_time``: Sort by modified time of the files. This is useful on large scale to parse the -# recently modified DAGs first. -# * ``random_seeded_by_host``: Sort randomly across multiple Schedulers but with same order on the -# same host. This is useful when running with Scheduler in HA mode where each scheduler can -# parse different DAG files. -# * ``alphabetical``: Sort by filename -file_parsing_sort_mode = modified_time - -# Whether the dag processor is running as a standalone process or it is a subprocess of a scheduler -# job. -standalone_dag_processor = False - -# Only applicable if `[scheduler]standalone_dag_processor` is true and callbacks are stored -# in database. Contains maximum number of callbacks that are fetched during a single loop. -max_callbacks_per_loop = 20 - -# Only applicable if `[scheduler]standalone_dag_processor` is true. -# Time in seconds after which dags, which were not updated by Dag Processor are deactivated. -dag_stale_not_seen_duration = 600 - -# 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 - -# How often to check for expired trigger requests that have not run yet. -trigger_timeout_check_interval = 15 - -# Amount of time a task can be in the queued state before being retried or set to failed. -task_queued_timeout = 600.0 - -# How often to check for tasks that have been in the queued state for -# longer than `[scheduler] task_queued_timeout`. -task_queued_timeout_check_interval = 120.0 - -# The run_id pattern used to verify the validity of user input to the run_id parameter when -# triggering a DAG. This pattern cannot change the pattern used by scheduler to generate run_id -# for scheduled DAG runs or DAG runs triggered without changing the run_id parameter. -allowed_run_id_pattern = ^[A-Za-z0-9_.~:+-]+$ - -[triggerer] -# How many triggers a single Triggerer will run at once, by default. -default_capacity = 1000 - -# How often to heartbeat the Triggerer job to ensure it hasn't been killed. -job_heartbeat_sec = 5 - -[kerberos] -ccache = /tmp/airflow_krb5_ccache - -# gets augmented with fqdn -principal = airflow -reinit_frequency = 3600 -kinit_path = kinit -keytab = airflow.keytab - -# Allow to disable ticket forwardability. -forwardable = True - -# Allow to remove source IP from token, useful when using token behind NATted Docker host. -include_ip = True - -[elasticsearch] -# Elasticsearch host -host = - -# Format of the log_id, which is used to query for a given tasks logs -log_id_template = {{dag_id}}-{{task_id}}-{{run_id}}-{{map_index}}-{{try_number}} - -# Used to mark the end of a log stream for a task -end_of_log_mark = end_of_log - -# Qualified URL for an elasticsearch frontend (like Kibana) with a template argument for log_id -# Code will construct log_id using the log_id template from the argument above. -# NOTE: scheme will default to https if one is not provided -# Example: frontend = http://localhost:5601/app/kibana#/discover?_a=(columns:!(message),query:(language:kuery,query:'log_id: "{{log_id}}"'),sort:!(log.offset,asc)) -frontend = - -# Write the task logs to the stdout of the worker, rather than the default files -write_stdout = False - -# Instead of the default log formatter, write the log lines as JSON -json_format = False - -# Log fields to also attach to the json output, if enabled -json_fields = asctime, filename, lineno, levelname, message - -# The field where host name is stored (normally either `host` or `host.name`) -host_field = host - -# The field where offset is stored (normally either `offset` or `log.offset`) -offset_field = offset - -# Comma separated list of index patterns to use when searching for logs (default: `_all`). -# Example: index_patterns = something-* -index_patterns = _all - -[elasticsearch_configs] -use_ssl = False -verify_certs = True - -[kubernetes_executor] -# Kwargs to override the default urllib3 Retry used in the kubernetes API client -# Example: api_client_retry_configuration = {{ "total": 3, "backoff_factor": 0.5 }} -api_client_retry_configuration = - -# Flag to control the information added to kubernetes executor logs for better traceability -logs_task_metadata = False - -# Path to the YAML pod file that forms the basis for KubernetesExecutor workers. -pod_template_file = - -# The repository of the Kubernetes Image for the Worker to Run -worker_container_repository = - -# The tag of the Kubernetes Image for the Worker to Run -worker_container_tag = - -# The Kubernetes namespace where airflow workers should be created. Defaults to ``default`` -namespace = default - -# 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. -# This only prevents removal of worker pods where the worker itself failed, -# not when the task it ran failed. -delete_worker_pods_on_failure = False - -# Number of Kubernetes Worker Pod creation calls per scheduler loop. -# Note that the current default of "1" will only launch a single pod -# per-heartbeat. It is HIGHLY recommended that users increase this -# number to match the tolerance of their kubernetes cluster for -# better performance. -worker_pods_creation_batch_size = 1 - -# Allows users to launch pods in multiple namespaces. -# Will require creating a cluster-role for the scheduler, -# or use multi_namespace_mode_namespace_list configuration. -multi_namespace_mode = False - -# If multi_namespace_mode is True while scheduler does not have a cluster-role, -# give the list of namespaces where the scheduler will schedule jobs -# Scheduler needs to have the necessary permissions in these namespaces. -multi_namespace_mode_namespace_list = - -# 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 - -# When running with in_cluster=False change the default cluster_context or config_file -# options to Kubernetes client. Leave blank these to use default behaviour like ``kubectl`` has. -# cluster_context = - -# Path to the kubernetes configfile to be used when ``in_cluster`` is set to False -# config_file = - -# 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/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/api/core_v1_api.py -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 = - -# Enables TCP keepalive mechanism. This prevents Kubernetes API requests to hang indefinitely -# when idle connection is time-outed on services like cloud load balancers or firewalls. -enable_tcp_keepalive = True - -# When the `enable_tcp_keepalive` option is enabled, TCP probes a connection that has -# been idle for `tcp_keep_idle` seconds. -tcp_keep_idle = 120 - -# When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond -# to a keepalive probe, TCP retransmits the probe after `tcp_keep_intvl` seconds. -tcp_keep_intvl = 30 - -# When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond -# to a keepalive probe, TCP retransmits the probe `tcp_keep_cnt number` of times before -# a connection is considered to be broken. -tcp_keep_cnt = 6 - -# Set this to false to skip verifying SSL certificate of Kubernetes python client. -verify_ssl = True - -# How often in seconds to check for task instances stuck in "queued" status without a pod -worker_pods_queued_check_interval = 60 - -# Path to a CA certificate to be used by the Kubernetes client to verify the server's SSL certificate. -ssl_ca_cert = - -[sensors] -# Sensor default timeout, 7 days by default (7 * 24 * 60 * 60). -default_timeout = 604800 diff --git a/airflow/config_templates/default_test.cfg b/airflow/config_templates/default_test.cfg deleted file mode 100644 index 523f52cb69a04..0000000000000 --- a/airflow/config_templates/default_test.cfg +++ /dev/null @@ -1,68 +0,0 @@ -# -# 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. - - -# This is the template for Airflow's unit test configuration. When Airflow runs -# unit tests, it looks for a configuration file at $AIRFLOW_HOME/unittests.cfg. -# If it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# unittests.cfg instead. - - -# ----------------------- TEMPLATE BEGINS HERE ----------------------- - -[core] -unit_test_mode = True -dags_folder = {TEST_DAGS_FOLDER} -plugins_folder = {TEST_PLUGINS_FOLDER} -dags_are_paused_at_creation = False -fernet_key = {FERNET_KEY} -killed_task_cleanup_time = 5 -allowed_deserialization_classes = airflow\..* - tests\..* - -[database] -sql_alchemy_conn = sqlite:///{AIRFLOW_HOME}/unittests.db - -[logging] -celery_logging_level = WARN - -[api] -auth_backends = airflow.api.auth.backend.default - -[hive] -default_hive_mapred_queue = airflow - -[smtp] -smtp_user = airflow -smtp_password = airflow - -[celery] -broker_url = sqla+mysql://airflow:airflow@localhost:3306/airflow -result_backend = db+mysql://airflow:airflow@localhost:3306/airflow - -[scheduler] -job_heartbeat_sec = 1 -schedule_after_task_execution = False -scheduler_health_check_server_port = 8794 -dag_dir_list_interval = 0 - -[elasticsearch] -log_id_template = {{dag_id}}-{{task_id}}-{{run_id}}-{{map_index}}-{{try_number}} diff --git a/airflow/config_templates/unit_tests.cfg b/airflow/config_templates/unit_tests.cfg new file mode 100644 index 0000000000000..540c3f3be42ba --- /dev/null +++ b/airflow/config_templates/unit_tests.cfg @@ -0,0 +1,102 @@ +# 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. + +# NOTE FOR CONTRIBUTORS: +# +# Values specified in this file are automatically used in our unit tests +# run by pytest and override default airflow configuration values provided by config.yml. +# +# These configuration settings should provide consistent environment to run tests - +# no matter if you are in Breeze env or use local venv or even run tests in the CI environment. +# +# If you want to have all unit tests to get some default configuration value, you should set it here. +# +# You can instead use ``conf_vars`` context manager or override the ``AIRFLOW__
__`` environment +# variable to set the configuration value for a specific test or a group of tests. +# +# The test configuration is loaded via setting AIRFLOW__CORE__UNIT_TEST_MODE=True +# in a pytest fixture in tests/conftest.py. This in turn triggers reloading of the configuration +# from this file after cleaning the respective configuration retrieved during initialization +# of configuration. See ``load_test_config`` function in ``airflow/config.py`` for details. +# + + +[core] +# We want to read dags from the test dags folder +dags_folder = {TEST_DAGS_FOLDER} +# we want to read plugins from example_dags/plugins folder +plugins_folder = {TEST_PLUGINS_FOLDER} +# we want to read fernet key generated dynamically in load_test_config +fernet_key = {FERNET_KEY} +# for tests we use local executor by default +executor = LocalExecutor +# We do not want dags to be paused at creation +dags_are_paused_at_creation = False +# We want to load examples +load_examples = True +# We want pickling to be enabled in tests +donot_pickle = False +# No default impersonation in tests - tests that run impersonation should set this explicitly +default_impersonation = +# We want to use unit test mode (though technically it is not needed as we set it in pytest fixture +# via AIRFLOW__CORE__UNIT_TEST_MODE environment variable) +unit_test_mode = True +# We want to use a shorter timeout for task cleanup +killed_task_cleanup_time = 5 +# We only allow our own classes to be deserialized in tests +allowed_deserialization_classes = airflow\..* tests\..* +[database] +# we want to have default connections loaded in unit tests +load_default_connections = True + +[logging] +# celery tests rely on it being set +celery_logging_level = INFO + +[smtp] +# Used as default values for SMTP unit tests +smtp_user = airflow +smtp_password = airflow +smtp_mail_from = airflow@example.com + +[api] +auth_backends = airflow.api.auth.backend.default + +[hive] +# Hive uses the configuration below to run the tests +default_hive_mapred_queue: airflow + +[celery] +# used to test that we can load celery config from airflow.cfg +broker_url = amqp://guest:guest@rabbitmq:5672/ +# and same for result backend +result_backend = db+mysql://airflow:airflow@localhost:3306/airflow + +[scheduler] +# Those values are set so that during unit tests things run faster than usual +job_heartbeat_sec = 1 +scheduler_heartbeat_sec = 5 +parsing_processes = 2 +dag_dir_list_interval = 0 + +[example_section] +# This section is used to test coercions of configuration values retrieval +string_value = 21600 +only_bool_value = true +only_float_value = 12.0 +only_integer_value = 21600 +only_string_value = this is a test diff --git a/airflow/configuration.py b/airflow/configuration.py index 8cb5971cd5e80..c8e19651d5b05 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -18,6 +18,7 @@ import datetime import functools +import io import json import logging import multiprocessing @@ -30,15 +31,15 @@ import warnings from base64 import b64encode from collections import OrderedDict - -# Ignored Mypy on configparser because it thinks the configparser module has no _UNSET attribute -from configparser import _UNSET, ConfigParser, NoOptionError, NoSectionError # type: ignore -from contextlib import contextmanager, suppress +from configparser import ConfigParser, NoOptionError, NoSectionError +from contextlib import contextmanager from json.decoder import JSONDecodeError +from threading import Lock from typing import IO, Any, Dict, Iterable, Pattern, Set, Tuple, Union from urllib.parse import urlsplit import re2 +from packaging.version import parse as parse_version from typing_extensions import overload from airflow.auth.managers.base_auth_manager import BaseAuthManager @@ -64,6 +65,8 @@ ENV_VAR_PREFIX = "AIRFLOW__" +EMPTY_SET: Set[tuple[str, str]] = set() # noqa: UP006 + def _parse_sqlite_version(s: str) -> tuple[int, ...]: match = _SQLITE3_VERSION_PATTERN.match(s) @@ -131,35 +134,155 @@ def _get_config_value_from_secret_backend(config_key: str) -> str | None: ) +def _is_template(configuration_description: dict[str, dict[str, Any]], section: str, key: str) -> bool: + """ + Check if the config is a template. + + :param configuration_description: description of configuration + :param section: section + :param key: key + :return: True if the config is a template + """ + return configuration_description.get(section, {}).get(key, {}).get("is_template", False) + + def _default_config_file_path(file_name: str) -> str: templates_dir = os.path.join(os.path.dirname(__file__), "config_templates") return os.path.join(templates_dir, file_name) -def default_config_yaml() -> dict[str, Any]: +def retrieve_configuration_description() -> dict[str, dict[str, Any]]: """ - Read Airflow configs from YAML file. + Read Airflow configuration description from YAML file. :return: Python dictionary containing configs & their info """ + base_configuration_description: dict[str, dict[str, Any]] = {} with open(_default_config_file_path("config.yml")) as config_file: - return yaml.safe_load(config_file) + base_configuration_description.update(yaml.safe_load(config_file)) + return base_configuration_description class AirflowConfigParser(ConfigParser): - """Custom Airflow Configparser supporting defaults and deprecated options.""" + """ + Custom Airflow Configparser supporting defaults and deprecated options. + + This is a subclass of ConfigParser that supports defaults and deprecated options. + + The defaults are stored in the ``_default_values ConfigParser. The configuration description keeps + description of all the options available in Airflow (description follow config.yaml.schema). + + :param default_config: default configuration (in the form of ini file). + :param configuration_description: description of configuration to use + """ + + def __init__( + self, + default_config: str | None = None, + *args, + **kwargs, + ): + super().__init__(*args, **kwargs) + self.configuration_description = retrieve_configuration_description() + self.upgraded_values = {} + # For those who would like to use a different data structure to keep defaults: + # We have to keep the default values in a ConfigParser rather than in any other + # data structure, because the values we have might contain %% which are ConfigParser + # interpolation placeholders. The _default_values config parser will interpolate them + # properly when we call get() on it. + self._default_values = create_default_config_parser(self.configuration_description) + if default_config is not None: + self._update_defaults_from_string(default_config) + self._update_logging_deprecated_template_to_one_from_defaults() + self.is_validated = False + self._suppress_future_warnings = False + self._interpolation_lock = Lock() + + def _update_logging_deprecated_template_to_one_from_defaults(self): + default = self.get_default_value("logging", "log_filename_template") + if default: + # Tuple does not support item assignment, so we have to create a new tuple and replace it + original_replacement = self.deprecated_values["logging"]["log_filename_template"] + self.deprecated_values["logging"]["log_filename_template"] = ( + original_replacement[0], + default, + original_replacement[2], + ) + + def is_template(self, section: str, key) -> bool: + """ + Return whether the value is templated. + + :param section: section of the config + :param key: key in the section + :return: True if the value is templated + """ + if self.configuration_description is None: + return False + return _is_template(self.configuration_description, section, key) + + def _update_defaults_from_string(self, config_string: str): + """ + The defaults in _default_values are updated based on values in config_string ("ini" format). + Note that those values are not validated and cannot contain variables because we are using + regular config parser to load them. This method is used to test the config parser in unit tests. + + :param config_string: ini-formatted config string + """ + parser = ConfigParser() + parser.read_string(config_string) + for section in parser.sections(): + if section not in self._default_values.sections(): + self._default_values.add_section(section) + errors = False + for key, value in parser.items(section): + if not self.is_template(section, key) and "{" in value: + errors = True + log.error( + f"The {section}.{key} value {value} read from string contains " + "variable. This is not supported" + ) + self._default_values.set(section, key, value) + if errors: + raise Exception( + f"The string config passed as default contains variables. " + f"This is not supported. String config: {config_string}" + ) + + def get_default_value(self, section: str, key: str, fallback: Any = None, raw=False, **kwargs) -> Any: + """ + Retrieves default value from default config parser. + + This will retrieve the default value from the default config parser. Optionally a raw, stored + value can be retrieved by setting skip_interpolation to True. This is useful for example when + we want to write the default value to a file, and we don't want the interpolation to happen + as it is going to be done later when the config is read. + + :param section: section of the config + :param key: key to use + :param fallback: fallback value to use + :param raw: if raw, then interpolation will be reversed + :param kwargs: other args + :return: + """ + value = self._default_values.get(section, key, fallback=fallback, **kwargs) + if raw and value is not None: + return value.replace("%", "%%") + return value # These configuration elements can be fetched as the stdout of commands # following the "{section}__{name}_cmd" pattern, the idea behind this # is to not store password on boxes in text files. # These configs can also be fetched from Secrets backend # following the "{section}__{name}__secret" pattern - @functools.cached_property def sensitive_config_values(self) -> Set[tuple[str, str]]: # noqa: UP006 - default_config = default_config_yaml() + if self.configuration_description is None: + return EMPTY_SET.copy() # we can't use set() here because set is defined below # ¯\_(ツ)_/¯ flattened = { - (s, k): item for s, s_c in default_config.items() for k, item in s_c.get("options").items() + (s, k): item + for s, s_c in self.configuration_description.items() + for k, item in s_c.get("options").items() # type: ignore[union-attr] } sensitive = {(section, key) for (section, key), v in flattened.items() if v.get("sensitive") is True} depr_option = {self.deprecated_options[x][:-1] for x in sensitive if x in self.deprecated_options} @@ -285,6 +408,7 @@ def inversed_deprecated_sections(self): "logging": { "log_filename_template": ( re2.compile(re2.escape("{{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log")), + # The actual replacement value will be updated after defaults are loaded from config.yml "XX-set-after-default-config-loaded-XX", "3.0", ), @@ -321,38 +445,151 @@ def inversed_deprecated_sections(self): upgraded_values: dict[tuple[str, str], str] """Mapping of (section,option) to the old value that was upgraded""" - # This method transforms option names on every read, get, or set operation. - # This changes from the default behaviour of ConfigParser from lower-casing - # to instead be case-preserving - def optionxform(self, optionstr: str) -> str: - return optionstr + def get_sections_including_defaults(self) -> list[str]: + """ + Retrieves all sections from the configuration parser, including sections defined by built-in defaults. - def __init__(self, default_config: str | None = None, *args, **kwargs): - super().__init__(*args, **kwargs) - self.upgraded_values = {} + :return: list of section names + """ + my_own_sections = self.sections() + all_sections_from_defaults = list(self.configuration_description.keys()) + for section in my_own_sections: + if section not in all_sections_from_defaults: + all_sections_from_defaults.append(section) + return all_sections_from_defaults + + def get_options_including_defaults(self, section: str) -> list[str]: + """ + Retrieves all possible option from the configuration parser for the section given, + including options defined by built-in defaults. - self.airflow_defaults = ConfigParser(*args, **kwargs) - if default_config is not None: - self.airflow_defaults.read_string(default_config) - # Set the upgrade value based on the current loaded default - default = self.airflow_defaults.get("logging", "log_filename_template", fallback=None) - if default: - replacement = self.deprecated_values["logging"]["log_filename_template"] - self.deprecated_values["logging"]["log_filename_template"] = ( - replacement[0], - default, - replacement[2], - ) - else: - # In case of tests it might not exist - with suppress(KeyError): - del self.deprecated_values["logging"]["log_filename_template"] - else: - with suppress(KeyError): - del self.deprecated_values["logging"]["log_filename_template"] + :return: list of option names for the section given + """ + my_own_options = self.options(section) if self.has_section(section) else [] + all_options_from_defaults = list( + self.configuration_description.get(section, {}).get("options", {}).keys() + ) + for option in my_own_options: + if option not in all_options_from_defaults: + all_options_from_defaults.append(option) + return all_options_from_defaults - self.is_validated = False - self._suppress_future_warnings = False + def optionxform(self, optionstr: str) -> str: + """ + This method transforms option names on every read, get, or set operation. + This changes from the default behaviour of ConfigParser from lower-casing + to instead be case-preserving. + + :param optionstr: + :return: + """ + return optionstr + + def write( # type: ignore[override] + self, + file: IO[str], + section: str | None = None, + include_examples: bool = True, + include_descriptions: bool = True, + include_sources: bool = True, + include_env_vars: bool = True, + comment_out_everything: bool = False, + hide_sensitive_values: bool = False, + extra_spacing: bool = True, + only_defaults: bool = False, + **kwargs: Any, + ) -> None: + """ + Writes configuration with comments and examples to a file. + + :param file: file to write to + :param section: section of the config to write, defaults to all sections + :param include_examples: Include examples in the output + :param include_descriptions: Include descriptions in the output + :param include_sources: Include the source of each config option + :param include_env_vars: Include environment variables corresponding to each config option + :param hide_sensitive_values: Include sensitive values in the output + :param extra_spacing: Add extra spacing before examples and after variables + :param only_defaults: Only include default values when writing the config, not the actual values + """ + from airflow import __version__ as airflow_version + + sources_dict = {} + if include_sources: + sources_dict = self.as_dict(display_source=True) + if self._default_values is None: + raise RuntimeError("Cannot write default config, no default config set") + if self.configuration_description is None: + raise RuntimeError("Cannot write default config, no default configuration description set") + for section_to_write in self.get_sections_including_defaults(): + section_config_description = self.configuration_description.get(section_to_write, {}) + if section_to_write != section and section is not None: + continue + if self._default_values.has_section(section_to_write) or self.has_section(section_to_write): + needs_separation = False + file.write(f"[{section_to_write}]\n") + section_description = section_config_description.get("description") + if section_description and include_descriptions: + for line in section_description.splitlines(): + file.write(f"# {line}\n") + file.write("\n") + for key in self.get_options_including_defaults(section_to_write): + option_config_description = ( + section_config_description.get("options", {}).get(key, {}) + if section_config_description + else {} + ) + version_added = option_config_description.get("version_added") + if version_added is not None and parse_version(version_added) > parse_version( + parse_version(airflow_version).base_version + ): + # skip if option is going to be added in the future version + continue + description = option_config_description.get("description") + needs_separation = False + if description and include_descriptions: + for line in description.splitlines(): + file.write(f"# {line}\n") + needs_separation = True + example = option_config_description.get("example") + if example is not None and include_examples: + if extra_spacing: + file.write("#\n") + file.write(f"# Example: {key} = {example}\n") + needs_separation = True + if include_sources and sources_dict: + sources_section = sources_dict.get(section_to_write) + value_with_source = sources_section.get(key) if sources_section else None + if value_with_source is None: + file.write("#\n# Source: not defined\n") + else: + file.write(f"#\n# Source: {value_with_source[1]}\n") + needs_separation = True + if include_env_vars: + file.write(f"#\n# Variable: AIRFLOW__{section_to_write.upper()}__{key.upper()}\n") + if extra_spacing: + file.write("#\n") + needs_separation = True + if self._default_values is None: + default_value = None + else: + default_value = self.get_default_value(section_to_write, key, raw=True) + if only_defaults: + value = default_value + else: + value = self.get(section_to_write, key, fallback=default_value, raw=True) + if value is None: + file.write(f"# {key} = \n") + else: + if comment_out_everything: + file.write(f"# {key} = {value}\n") + else: + file.write(f"{key} = {value}\n") + if needs_separation: + file.write("\n") + if include_descriptions and not needs_separation: + # extra separation between sections in case last option did not need it + file.write("\n") def validate(self): self._validate_sqlite3_version() @@ -437,8 +674,8 @@ def _validate_enums(self): """Validate that enum type config has an accepted value.""" for (section_key, option_key), enum_options in self.enums_options.items(): if self.has_option(section_key, option_key): - value = self.get(section_key, option_key) - if value not in enum_options: + value = self.get(section_key, option_key, fallback=None) + if value and value not in enum_options: raise AirflowConfigException( f"`[{section_key}] {option_key}` should not be " f"{value!r}. Possible values: {', '.join(enum_options)}." @@ -561,17 +798,18 @@ def get_mandatory_value(self, section: str, key: str, **kwargs) -> str: return value @overload # type: ignore[override] - def get(self, section: str, key: str, fallback: str = ..., **kwargs) -> str: # type: ignore[override] + def get(self, section: str, key: str, fallback: str = ..., **kwargs) -> str: ... @overload # type: ignore[override] - def get(self, section: str, key: str, **kwargs) -> str | None: # type: ignore[override] + def get(self, section: str, key: str, **kwargs) -> str | None: ... - def get( # type: ignore[override, misc] + def get( # type: ignore[override,misc] self, section: str, key: str, + suppress_warnings: bool = False, _extra_stacklevel: int = 0, **kwargs, ) -> str | None: @@ -667,10 +905,10 @@ def get( # type: ignore[override, misc] return option # ...then the default config - if self.airflow_defaults.has_option(section, key) or "fallback" in kwargs: - return expand_env_var(self.airflow_defaults.get(section, key, **kwargs)) - - log.warning("section/key [%s/%s] not found in config", section, key) + if self.get_default_value(section, key) is not None or "fallback" in kwargs: + return expand_env_var(self.get_default_value(section, key, **kwargs)) + if not suppress_warnings: + log.warning("section/key [%s/%s] not found in config", section, key) raise AirflowConfigException(f"section/key [{section}/{key}] not found in config") @@ -825,27 +1063,20 @@ def getimport(self, section: str, key: str, **kwargs) -> Any: ) def getjson( - self, section: str, key: str, fallback=_UNSET, **kwargs + self, section: str, key: str, fallback=None, **kwargs ) -> dict | list | str | int | float | None: """ Return a config value parsed from a JSON string. ``fallback`` is *not* JSON parsed but used verbatim when no config value is given. """ - # get always returns the fallback value as a string, so for this if - # someone gives us an object we want to keep that - default = _UNSET - if fallback is not _UNSET: - default = fallback - fallback = _UNSET - try: - data = self.get(section=section, key=key, fallback=fallback, _extra_stacklevel=1, **kwargs) + data = self.get(section=section, key=key, fallback=None, _extra_stacklevel=1, **kwargs) except (NoSectionError, NoOptionError): - return default + data = None - if not data: - return default if default is not _UNSET else None + if data is None or data == "": + return fallback try: return json.loads(data) @@ -896,19 +1127,33 @@ def read( ): super().read(filenames=filenames, encoding=encoding) - # The RawConfigParser defines "Mapping" from abc.collections is not subscriptable - so we have - # to use Dict here. def read_dict( # type: ignore[override] self, dictionary: dict[str, dict[str, Any]], source: str = "" ): + """ + We define a different signature here to add better type hints and checking. + + :param dictionary: dictionary to read from + :param source: source to be used to store the configuration + :return: + """ super().read_dict(dictionary=dictionary, source=source) def has_option(self, section: str, option: str) -> bool: + """ + Check if option is defined. + + Uses self.get() to avoid reimplementing the priority order of config variables + (env, config, cmd, defaults). + + :param section: section to get option from + :param option: option to get + :return: + """ try: - # Using self.get() to avoid reimplementing the priority order - # of config variables (env, config, cmd, defaults) - # UNSET to avoid logging a warning about missing values - self.get(section, option, fallback=_UNSET, _extra_stacklevel=1) + value = self.get(section, option, fallback=None, _extra_stacklevel=1, suppress_warnings=True) + if value is None: + return False return True except (NoOptionError, NoSectionError): return False @@ -923,8 +1168,8 @@ def remove_option(self, section: str, option: str, remove_default: bool = True): if super().has_option(section, option): super().remove_option(section, option) - if self.airflow_defaults.has_option(section, option) and remove_default: - self.airflow_defaults.remove_option(section, option) + if self.get_default_value(section, option) is not None and remove_default: + self._default_values.remove_option(section, option) def getsection(self, section: str) -> ConfigOptionsDictType | None: """ @@ -934,10 +1179,10 @@ def getsection(self, section: str) -> ConfigOptionsDictType | None: :param section: section from the config """ - if not self.has_section(section) and not self.airflow_defaults.has_section(section): + if not self.has_section(section) and not self._default_values.has_section(section): return None - if self.airflow_defaults.has_section(section): - _section: ConfigOptionsDictType = OrderedDict(self.airflow_defaults.items(section)) + if self._default_values.has_section(section): + _section: ConfigOptionsDictType = OrderedDict(self._default_values.items(section)) else: _section = OrderedDict() @@ -971,29 +1216,6 @@ def getsection(self, section: str) -> ConfigOptionsDictType | None: _section[key] = False return _section - def write( # type: ignore[override] - self, fp: IO, space_around_delimiters: bool = True, section: str | None = None - ) -> None: - # This is based on the configparser.RawConfigParser.write method code to add support for - # reading options from environment variables. - # Various type ignores below deal with less-than-perfect RawConfigParser superclass typing - if space_around_delimiters: - delimiter = f" {self._delimiters[0]} " # type: ignore[attr-defined] - else: - delimiter = self._delimiters[0] # type: ignore[attr-defined] - if self._defaults: # type: ignore - self._write_section( # type: ignore[attr-defined] - fp, self.default_section, self._defaults.items(), delimiter # type: ignore[attr-defined] - ) - sections = ( - {section: dict(self.getsection(section))} # type: ignore[arg-type] - if section - else self._sections # type: ignore[attr-defined] - ) - for sect in sections: - item_section: ConfigOptionsDictType = self.getsection(sect) # type: ignore[assignment] - self._write_section(fp, sect, item_section.items(), delimiter) # type: ignore[attr-defined] - def as_dict( self, display_source: bool = False, @@ -1044,14 +1266,15 @@ def as_dict( ) config_sources: ConfigSourcesType = {} - configs = [ - ("default", self.airflow_defaults), + configs: Iterable[tuple[str, ConfigParser]] = [ + ("default", self._default_values), ("airflow.cfg", self), ] self._replace_config_with_display_sources( config_sources, configs, + self.configuration_description if self.configuration_description else {}, display_source, raw, self.deprecated_options, @@ -1212,7 +1435,7 @@ def _filter_by_source( if not getter_opt: continue # Check to see that there is a default value - if not self.airflow_defaults.has_option(section, key): + if self.get_default_value(section, key) is None: continue # Check to see if bare setting is the same as defaults if display_source: @@ -1220,13 +1443,14 @@ def _filter_by_source( opt, source = config_sources[section][key] # type: ignore else: opt = config_sources[section][key] - if opt == self.airflow_defaults.get(section, key): + if opt == self.get_default_value(section, key): del config_sources[section][key] @staticmethod def _replace_config_with_display_sources( config_sources: ConfigSourcesType, configs: Iterable[tuple[str, ConfigParser]], + configuration_description: dict[str, dict[str, Any]], display_source: bool, raw: bool, deprecated_options: dict[tuple[str, str], tuple[str, str, str]], @@ -1235,10 +1459,12 @@ def _replace_config_with_display_sources( include_secret: bool, ): for source_name, config in configs: - for section in config.sections(): + sections = config.sections() + for section in sections: AirflowConfigParser._replace_section_config_with_display_sources( config, config_sources, + configuration_description, display_source, raw, section, @@ -1277,7 +1503,9 @@ def _deprecated_variable_is_set(deprecated_section: str, deprecated_key: str) -> @staticmethod def _deprecated_command_is_set_in_config( - deprecated_section: str, deprecated_key: str, configs: Iterable[tuple[str, ConfigParser]] + deprecated_section: str, + deprecated_key: str, + configs: Iterable[tuple[str, ConfigParser]], ) -> bool: return AirflowConfigParser._deprecated_value_is_set_in_config( deprecated_section=deprecated_section, deprecated_key=deprecated_key + "_cmd", configs=configs @@ -1292,7 +1520,9 @@ def _deprecated_variable_command_is_set(deprecated_section: str, deprecated_key: @staticmethod def _deprecated_secret_is_set_in_config( - deprecated_section: str, deprecated_key: str, configs: Iterable[tuple[str, ConfigParser]] + deprecated_section: str, + deprecated_key: str, + configs: Iterable[tuple[str, ConfigParser]], ) -> bool: return AirflowConfigParser._deprecated_value_is_set_in_config( deprecated_section=deprecated_section, deprecated_key=deprecated_key + "_secret", configs=configs @@ -1316,6 +1546,7 @@ def suppress_future_warnings(self): def _replace_section_config_with_display_sources( config: ConfigParser, config_sources: ConfigSourcesType, + configuration_description: dict[str, dict[str, Any]], display_source: bool, raw: bool, section: str, @@ -1329,7 +1560,7 @@ def _replace_section_config_with_display_sources( sect = config_sources.setdefault(section, OrderedDict()) if isinstance(config, AirflowConfigParser): with config.suppress_future_warnings(): - items = config.items(section=section, raw=raw) + items: Iterable[tuple[str, Any]] = config.items(section=section, raw=raw) else: items = config.items(section=section, raw=raw) for k, val in items: @@ -1372,22 +1603,46 @@ def _replace_section_config_with_display_sources( def load_test_config(self): """ - Load the unit test configuration. + Uses test configuration rather than the configuration coming from airflow defaults. - Note: this is not reversible. + When running tests we use special the unit_test configuration to avoid accidental modifications and + different behaviours when running the tests. Values for those test configuration are stored in + the "unit_tests.cfg" configuration file in the ``airflow/config_templates`` folder + and you need to change values there if you want to make some specific configuration to be used """ - # remove all sections, falling back to defaults + # We need those globals before we run "get_all_expansion_variables" because this is where + # the variables are expanded from in the configuration + global FERNET_KEY, AIRFLOW_HOME + from cryptography.fernet import Fernet + + unit_test_config_file = pathlib.Path(__file__).parent / "config_templates" / "unit_tests.cfg" + unit_test_config = unit_test_config_file.read_text() + self.remove_all_read_configurations() + with io.StringIO(unit_test_config) as test_config_file: + self.read_file(test_config_file) + # set fernet key to a random value + global FERNET_KEY + FERNET_KEY = Fernet.generate_key().decode() + self.expand_all_configuration_values() + log.info("Unit test configuration loaded from 'config_unit_tests.cfg'") + + def expand_all_configuration_values(self): + """Expands all configuration values using global and local variables defined in this module.""" + all_vars = get_all_expansion_variables() for section in self.sections(): - self.remove_section(section) - - # then read test config + for key, value in self.items(section): + if value is not None: + if self.has_option(section, key): + self.remove_option(section, key) + if self.is_template(section, key) or not isinstance(value, str): + self.set(section, key, value) + else: + self.set(section, key, value.format(**all_vars)) - path = _default_config_file_path("default_test.cfg") - log.info("Reading default test configuration from %s", path) - self.read_string(_parameterized_config_from_template("default_test.cfg")) - # then read any "custom" test settings - log.info("Reading test configuration from %s", TEST_CONFIG_FILE) - self.read(TEST_CONFIG_FILE) + def remove_all_read_configurations(self): + """Removes all read configurations, leaving only default values in the config.""" + for section in self.sections(): + self.remove_section(section) @staticmethod def _warn_deprecate( @@ -1414,7 +1669,9 @@ def __getstate__(self): for name in [ "_sections", "is_validated", - "airflow_defaults", + "configuration_description", + "upgraded_values", + "_default_values", ] } @@ -1430,7 +1687,7 @@ def get_airflow_home() -> str: return expand_env_var(os.environ.get("AIRFLOW_HOME", "~/airflow")) -def get_airflow_config(airflow_home) -> str: +def get_airflow_config(airflow_home: str) -> str: """Get Path to airflow.cfg path.""" airflow_config_var = os.environ.get("AIRFLOW_CONFIG") if airflow_config_var is None: @@ -1438,34 +1695,8 @@ def get_airflow_config(airflow_home) -> str: return expand_env_var(airflow_config_var) -def _parameterized_config_from_template(filename) -> str: - TEMPLATE_START = "# ----------------------- TEMPLATE BEGINS HERE -----------------------\n" - - path = _default_config_file_path(filename) - with open(path) as fh: - for line in fh: - if line != TEMPLATE_START: - continue - return parameterized_config(fh.read().strip()) - raise RuntimeError(f"Template marker not found in {path!r}") - - -def parameterized_config(template) -> str: - """ - Generates configuration from provided template & variables defined in current scope. - - :param template: a config content templated with {{variables}} - """ - all_vars = {k: v for d in [globals(), locals()] for k, v in d.items()} - return template.format(**all_vars) - - -def get_airflow_test_config(airflow_home) -> str: - """Get path to unittests.cfg.""" - if "AIRFLOW_TEST_CONFIG" not in os.environ: - return os.path.join(airflow_home, "unittests.cfg") - # It will never return None - return expand_env_var(os.environ["AIRFLOW_TEST_CONFIG"]) # type: ignore[return-value] +def get_all_expansion_variables() -> dict[str, Any]: + return {k: v for d in [globals(), locals()] for k, v in d.items()} def _generate_fernet_key() -> str: @@ -1474,84 +1705,109 @@ def _generate_fernet_key() -> str: return Fernet.generate_key().decode() -def initialize_config() -> AirflowConfigParser: - """ - Load the Airflow config files. - - Called for you automatically as part of the Airflow boot process. +def create_default_config_parser(configuration_description: dict[str, dict[str, Any]]) -> ConfigParser: """ - global FERNET_KEY, AIRFLOW_HOME, WEBSERVER_CONFIG + Creates default config parser based on configuration description. - default_config = _parameterized_config_from_template("default_airflow.cfg") + It creates ConfigParser with all default values retrieved from the configuration description and + expands all the variables from the global and local variables defined in this module. - local_conf = AirflowConfigParser(default_config=default_config) - - if local_conf.getboolean("core", "unit_test_mode"): - # Load test config only - if not os.path.isfile(TEST_CONFIG_FILE): - from cryptography.fernet import Fernet - - log.info("Creating new Airflow config file for unit tests in: %s", TEST_CONFIG_FILE) - pathlib.Path(AIRFLOW_HOME).mkdir(parents=True, exist_ok=True) - - FERNET_KEY = Fernet.generate_key().decode() + :param configuration_description: configuration description - retrieved from config.yaml files + following the schema defined in "config.yml.schema.json" in the config_templates folder. + :return: Default Config Parser that can be used to read configuration values from. + """ + parser = ConfigParser() + all_vars = get_all_expansion_variables() + for section, section_desc in configuration_description.items(): + parser.add_section(section) + options = section_desc["options"] + for key in options: + default_value = options[key]["default"] + is_template = options[key].get("is_template", False) + if default_value is not None: + if is_template or not isinstance(default_value, str): + parser.set(section, key, default_value) + else: + parser.set(section, key, default_value.format(**all_vars)) + return parser - with open(TEST_CONFIG_FILE, "w") as file: - cfg = _parameterized_config_from_template("default_test.cfg") - file.write(cfg) - make_group_other_inaccessible(TEST_CONFIG_FILE) - local_conf.load_test_config() - else: - # Load normal config - if not os.path.isfile(AIRFLOW_CONFIG): - from cryptography.fernet import Fernet +def load_standard_airflow_configuration(airflow_config_parser: AirflowConfigParser): + """ + Loads standard airflow configuration. - log.info("Creating new Airflow config file in: %s", AIRFLOW_CONFIG) - pathlib.Path(AIRFLOW_HOME).mkdir(parents=True, exist_ok=True) + In case it finds that the configuration file is missing, it will create it and write the default + configuration values there, based on defaults passed, and will add the comments and examples + from the default configuration. - FERNET_KEY = Fernet.generate_key().decode() + :param airflow_config_parser: parser to which the configuration will be loaded + """ + global FERNET_KEY, AIRFLOW_HOME + if not os.path.isfile(AIRFLOW_CONFIG): + from cryptography.fernet import Fernet + + log.info("Creating new Airflow config file in: %s", AIRFLOW_CONFIG) + pathlib.Path(AIRFLOW_HOME).mkdir(parents=True, exist_ok=True) + FERNET_KEY = Fernet.generate_key().decode() + if airflow_config_parser._default_values and airflow_config_parser.configuration_description: with open(AIRFLOW_CONFIG, "w") as file: - file.write(default_config) - make_group_other_inaccessible(AIRFLOW_CONFIG) - - log.info("Reading the config from %s", AIRFLOW_CONFIG) - - local_conf.read(AIRFLOW_CONFIG) - - if local_conf.has_option("core", "AIRFLOW_HOME"): - msg = ( - "Specifying both AIRFLOW_HOME environment variable and airflow_home " - "in the config file is deprecated. Please use only the AIRFLOW_HOME " - "environment variable and remove the config file entry." - ) - if "AIRFLOW_HOME" in os.environ: - warnings.warn(msg, category=DeprecationWarning) - elif local_conf.get("core", "airflow_home") == AIRFLOW_HOME: - warnings.warn( - "Specifying airflow_home in the config file is deprecated. As you " - "have left it at the default value you should remove the setting " - "from your airflow.cfg and suffer no change in behaviour.", - category=DeprecationWarning, + airflow_config_parser.write( + file, + include_sources=False, + include_env_vars=True, + extra_spacing=True, + only_defaults=True, ) - else: - # there - AIRFLOW_HOME = local_conf.get("core", "airflow_home") # type: ignore[assignment] - warnings.warn(msg, category=DeprecationWarning) + make_group_other_inaccessible(AIRFLOW_CONFIG) + log.info("Reading the config from %s", AIRFLOW_CONFIG) + airflow_config_parser.read(AIRFLOW_CONFIG) + if airflow_config_parser.has_option("core", "AIRFLOW_HOME"): + msg = ( + "Specifying both AIRFLOW_HOME environment variable and airflow_home " + "in the config file is deprecated. Please use only the AIRFLOW_HOME " + "environment variable and remove the config file entry." + ) + if "AIRFLOW_HOME" in os.environ: + warnings.warn(msg, category=DeprecationWarning) + elif airflow_config_parser.get("core", "airflow_home") == AIRFLOW_HOME: + warnings.warn( + "Specifying airflow_home in the config file is deprecated. As you " + "have left it at the default value you should remove the setting " + "from your airflow.cfg and suffer no change in behaviour.", + category=DeprecationWarning, + ) + else: + # there + AIRFLOW_HOME = airflow_config_parser.get("core", "airflow_home") # type: ignore[assignment] + warnings.warn(msg, category=DeprecationWarning) - # They _might_ have set unit_test_mode in the airflow.cfg, we still - # want to respect that and then load the unittests.cfg - if local_conf.getboolean("core", "unit_test_mode"): - local_conf.load_test_config() - WEBSERVER_CONFIG = local_conf.get("webserver", "config_file") +def initialize_config() -> AirflowConfigParser: + """ + Load the Airflow config files. + + Called for you automatically as part of the Airflow boot process. + """ + global WEBSERVER_CONFIG + airflow_config_parser = AirflowConfigParser() + if airflow_config_parser.getboolean("core", "unit_test_mode"): + airflow_config_parser.load_test_config() + else: + load_standard_airflow_configuration(airflow_config_parser) + # If the user set unit_test_mode in the airflow.cfg, we still + # want to respect that and then load the default unit test configuration + # file on top of it. + if airflow_config_parser.getboolean("core", "unit_test_mode"): + airflow_config_parser.load_test_config() + + WEBSERVER_CONFIG = airflow_config_parser.get("webserver", "config_file") if not os.path.isfile(WEBSERVER_CONFIG): import shutil log.info("Creating new FAB webserver config file in: %s", WEBSERVER_CONFIG) shutil.copy(_default_config_file_path("default_webserver_config.py"), WEBSERVER_CONFIG) - return local_conf + return airflow_config_parser def make_group_other_inaccessible(file_path: str): @@ -1761,39 +2017,6 @@ def initialize_auth_manager() -> BaseAuthManager: return auth_manager_cls() -@functools.lru_cache(maxsize=None) -def _DEFAULT_CONFIG() -> str: - path = _default_config_file_path("default_airflow.cfg") - with open(path) as fh: - return fh.read() - - -@functools.lru_cache(maxsize=None) -def _TEST_CONFIG() -> str: - path = _default_config_file_path("default_test.cfg") - with open(path) as fh: - return fh.read() - - -_deprecated = { - "DEFAULT_CONFIG": _DEFAULT_CONFIG, - "TEST_CONFIG": _TEST_CONFIG, - "TEST_CONFIG_FILE_PATH": functools.partial(_default_config_file_path, "default_test.cfg"), - "DEFAULT_CONFIG_FILE_PATH": functools.partial(_default_config_file_path, "default_airflow.cfg"), -} - - -def __getattr__(name): - if name in _deprecated: - warnings.warn( - f"{__name__}.{name} is deprecated and will be removed in future", - DeprecationWarning, - stacklevel=2, - ) - return _deprecated[name]() - raise AttributeError(f"module {__name__} has no attribute {name}") - - # Setting AIRFLOW_HOME and AIRFLOW_CONFIG from environment variables, using # "~/airflow" and "$AIRFLOW_HOME/airflow.cfg" respectively as defaults. AIRFLOW_HOME = get_airflow_home() @@ -1818,8 +2041,6 @@ def __getattr__(name): else: TEST_PLUGINS_FOLDER = os.path.join(AIRFLOW_HOME, "plugins") -TEST_CONFIG_FILE = get_airflow_test_config(AIRFLOW_HOME) - SECRET_KEY = b64encode(os.urandom(16)).decode("utf-8") FERNET_KEY = "" # Set only if needed when generating a new file WEBSERVER_CONFIG = "" # Set by initialize_config diff --git a/airflow/utils/db.py b/airflow/utils/db.py index f4c2ae8b1bd00..d6c5664a96f4b 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -911,8 +911,8 @@ def synchronize_log_template(*, session: Session = NEW_SESSION) -> None: # If we have an empty table, and the default values exist, we will seed the # table with values from pre 2.3.0, so old logs will still be retrievable. if not stored: - is_default_log_id = elasticsearch_id == conf.airflow_defaults.get("elasticsearch", "log_id_template") - is_default_filename = filename == conf.airflow_defaults.get("logging", "log_filename_template") + is_default_log_id = elasticsearch_id == conf.get_default_value("elasticsearch", "log_id_template") + is_default_filename = filename == conf.get_default_value("logging", "log_filename_template") if is_default_log_id and is_default_filename: session.add( LogTemplate( diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py index ed7a06d4a4666..0ef8ef65b908e 100644 --- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py +++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py @@ -27,7 +27,6 @@ "black", "blacken-docs", "check-aiobotocore-optional", - "check-airflow-config-yaml-consistent", "check-airflow-provider-compatibility", "check-apache-license-rat", "check-base-operator-partial-arguments", diff --git a/docs/apache-airflow/howto/index.rst b/docs/apache-airflow/howto/index.rst index 617975d2958de..15a5249147f08 100644 --- a/docs/apache-airflow/howto/index.rst +++ b/docs/apache-airflow/howto/index.rst @@ -47,7 +47,6 @@ configuring an Airflow environment. variable run-behind-proxy run-with-systemd - use-test-config define-extra-link email-config dynamic-dag-generation diff --git a/docs/apache-airflow/howto/set-config.rst b/docs/apache-airflow/howto/set-config.rst index b443cff9c81ae..14da8198d5d93 100644 --- a/docs/apache-airflow/howto/set-config.rst +++ b/docs/apache-airflow/howto/set-config.rst @@ -21,8 +21,31 @@ Setting Configuration Options ============================= The first time you run Airflow, it will create a file called ``airflow.cfg`` in -your ``$AIRFLOW_HOME`` directory (``~/airflow`` by default). This file contains Airflow's configuration and you -can edit it to change any of the settings. You can also set options with environment variables by using this format: +your ``$AIRFLOW_HOME`` directory (``~/airflow`` by default). This is in order to make it easy to +"play" with airflow configuration. + +However, for production case you are advised to generate the configuration using command line: + +.. code-block:: bash + + airflow config list --defaults + +This command will produce the output that you can copy to your configuration file and edit. + +It will contain all the default configuration options, with examples, nicely commented out +so you need only un-comment and modify those that you want to change. +This way you can easily keep track of all the configuration options that you changed from default +and you can also easily upgrade your installation to new versions of Airflow when they come out and +automatically use the defaults for existing options if they changed there. + +You can redirect it to your configuration file and edit it: + +.. code-block:: bash + + airflow config list --defaults > "${AIRFLOW_HOME}/airflow.cfg" + + +You can also set options with environment variables by using this format: :envvar:`AIRFLOW__{SECTION}__{KEY}` (note the double underscores). For example, the metadata database connection string can either be set in ``airflow.cfg`` like this: diff --git a/docs/apache-airflow/howto/use-test-config.rst b/docs/apache-airflow/howto/use-test-config.rst deleted file mode 100644 index 1c7731efe597d..0000000000000 --- a/docs/apache-airflow/howto/use-test-config.rst +++ /dev/null @@ -1,36 +0,0 @@ - .. 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. - - - -Using the Test Mode Configuration -================================= - -Airflow has a fixed set of "test mode" configuration options. You can load these -at any time by calling ``airflow.configuration.load_test_config()``. Please **note** that this -operation is **not reversible**. - -Some options for example, ``DAG_FOLDER``, are loaded before you have a chance to call ``load_test_config()``. -In order to eagerly load the test configuration, set ``test_mode`` in ``airflow.cfg``: - -.. code-block:: ini - - [tests] - unit_test_mode = True - -Due to Airflow's automatic environment variable expansion (see :doc:`set-config`), you can also set the -environment variable ``AIRFLOW__CORE__UNIT_TEST_MODE`` to temporarily overwrite ``airflow.cfg``. diff --git a/docs/apache-airflow/redirects.txt b/docs/apache-airflow/redirects.txt index 9d8a5527fbbd9..01deeddc87172 100644 --- a/docs/apache-airflow/redirects.txt +++ b/docs/apache-airflow/redirects.txt @@ -141,3 +141,6 @@ python-api-ref.rst public-airflow-interface.rst # Typos howto/define_extra_link.rst howto/define-extra-link.rst + +# Use test config (it's not a howto for users but a howto for developers so we redirect it back to index) +howto/use-test-config.rst index.rst diff --git a/docs/conf.py b/docs/conf.py index 9f1645e42599a..a8939833fd6b3 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -43,7 +43,7 @@ from packaging.version import parse as parse_version import airflow -from airflow.configuration import AirflowConfigParser, default_config_yaml +from airflow.configuration import AirflowConfigParser, retrieve_configuration_description sys.path.append(str(Path(__file__).parent / "exts")) @@ -419,7 +419,7 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): for deprecated_section, deprecated_key, since_version in deprecated: deprecated_options[deprecated_section][deprecated_key] = section, key, since_version - configs = default_config_yaml() + configs = retrieve_configuration_description() # We want the default/example we show in the docs to reflect the value _after_ # the config has been templated, not before diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index a0bd58e32e2c3..a678af7996c67 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -62,7 +62,7 @@ setup:version:be116d90a21c2afe01087f7609774e1e setup:cdf58a9c27af2877dc5a658ed0a1d99c shell:87e7bdcebe1180395adfec86b2a065f5 start-airflow:f7216a8126ecf14b033e2ea677e1a105 -static-checks:bb1248670000271a9e011c12747079c1 +static-checks:f9ec0d7edaba84180403d95469d94ea0 testing:docker-compose-tests:0c810047fc66a0cfe91119e2d08b3507 testing:helm-tests:8e491da2e01ebd815322c37562059d77 testing:integration-tests:486e4d91449ecdb7630ef2a470d705a3 diff --git a/images/breeze/output-commands.svg b/images/breeze/output-commands.svg index ea72db06740eb..11b7038cecc51 100644 --- a/images/breeze/output-commands.svg +++ b/images/breeze/output-commands.svg @@ -35,8 +35,8 @@ .breeze-help-r1 { fill: #c5c8c6;font-weight: bold } .breeze-help-r2 { fill: #c5c8c6 } .breeze-help-r3 { fill: #d0b344;font-weight: bold } -.breeze-help-r4 { fill: #68a0b3;font-weight: bold } -.breeze-help-r5 { fill: #868887 } +.breeze-help-r4 { fill: #868887 } +.breeze-help-r5 { fill: #68a0b3;font-weight: bold } .breeze-help-r6 { fill: #98a84b;font-weight: bold } .breeze-help-r7 { fill: #8d7b39 } @@ -217,59 +217,59 @@ -Usage: breeze [OPTIONSCOMMAND [ARGS]... +Usage: breeze [OPTIONS] COMMAND [ARGS]... -╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.8< | 3.9 | 3.10 | 3.11) -[default: 3.8]                                               ---backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] ---postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] ---mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] ---mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] ---integrationIntegration(s) to enable when running (can be more than one).                             -(all | all-testable | cassandra | celery | kafka | kerberos | mongo | otel | pinot |      -statsd | statsd | trino)                                                                  ---forward-credentials-fForward local credentials to container when running. ---db-reset-dReset DB when entering the container. ---max-timeMaximum time that the command should take - if it takes longer, the command will fail. -(INTEGER RANGE)                                                                        ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT) -[default: autodetect]                                          -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Developer commands ─────────────────────────────────────────────────────────────────────────────────────────────────╮ -start-airflow     Enter breeze environment and starts all Airflow components in the tmux session. Compile assets   -if contents of www directory changed.                                                            -static-checks     Run static checks.                                                                               -build-docs         -down              Stop running breeze environment.                                                                 -shell             Enter breeze environment. this is the default command use when no other is selected.             -exec              Joins the interactive shell of running airflow container.                                        -compile-www-assetsCompiles www assets.                                                                             -cleanup           Cleans the cache of parameters, docker cache and optionally built CI/PROD images.                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Testing commands ───────────────────────────────────────────────────────────────────────────────────────────────────╮ -testing        Tools that developers can use to run tests                                                          -k8s            Tools that developers use to run Kubernetes tests                                                   -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Image commands ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ -ci-image         Tools that developers can use to manually manage CI images                                        -prod-image       Tools that developers can use to manually manage PROD images                                      -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Release management commands ────────────────────────────────────────────────────────────────────────────────────────╮ -release-management     Tools that release managers can use to prepare and manage Airflow releases                  -sbom                   Tools that release managers can use to prepare sbom information                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Other commands ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ -setup     Tools that developers can use to configure Breeze                                                        -ci        Tools that CI workflows use to cleanup/manage CI environment                                             -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Basic flags ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--python-pPython major/minor version used in Airflow image for images.(>3.8< | 3.9 | 3.10 | 3.11) +[default: 3.8]                                               +--backend-bDatabase backend to use.(>sqlite< | mysql | postgres | mssql)[default: sqlite] +--postgres-version-PVersion of Postgres used.(>11< | 12 | 13 | 14 | 15)[default: 11] +--mysql-version-MVersion of MySQL used.(>5.7< | 8)[default: 5.7] +--mssql-version-SVersion of MsSQL used.(>2017-latest< | 2019-latest)[default: 2017-latest] +--integrationIntegration(s) to enable when running (can be more than one).                             +(all | all-testable | cassandra | celery | kafka | kerberos | mongo | otel | pinot |      +statsd | statsd | trino)                                                                  +--forward-credentials-fForward local credentials to container when running. +--db-reset-dReset DB when entering the container. +--max-timeMaximum time that the command should take - if it takes longer, the command will fail. +(INTEGER RANGE)                                                                        +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT) +[default: autodetect]                                          +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Developer commands ─────────────────────────────────────────────────────────────────────────────────────────────────╮ +start-airflow     Enter breeze environment and starts all Airflow components in the tmux session. Compile assets   +if contents of www directory changed.                                                            +static-checks     Run static checks.                                                                               +build-docs         +down              Stop running breeze environment.                                                                 +shell             Enter breeze environment. this is the default command use when no other is selected.             +exec              Joins the interactive shell of running airflow container.                                        +compile-www-assetsCompiles www assets.                                                                             +cleanup           Cleans the cache of parameters, docker cache and optionally built CI/PROD images.                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Testing commands ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +testing        Tools that developers can use to run tests                                                          +k8s            Tools that developers use to run Kubernetes tests                                                   +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Image commands ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +ci-image         Tools that developers can use to manually manage CI images                                        +prod-image       Tools that developers can use to manually manage PROD images                                      +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Release management commands ────────────────────────────────────────────────────────────────────────────────────────╮ +release-management     Tools that release managers can use to prepare and manage Airflow releases                  +sbom                   Tools that release managers can use to prepare sbom information                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Other commands ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +setup     Tools that developers can use to configure Breeze                                                        +ci        Tools that CI workflows use to cleanup/manage CI environment                                             +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/images/breeze/output_static-checks.svg b/images/breeze/output_static-checks.svg index da730e9da7029..6220e1db62e07 100644 --- a/images/breeze/output_static-checks.svg +++ b/images/breeze/output_static-checks.svg @@ -1,4 +1,4 @@ - + - + @@ -264,12 +264,9 @@ - - - - Command: static-checks + Command: static-checks @@ -280,80 +277,79 @@ -Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... +Usage: breeze static-checks [OPTIONS] [PRECOMMIT_ARGS]... Run static checks. -╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---type-tType(s) of the static checks to run.                                              -(all | black | blacken-docs | check-aiobotocore-optional |                        -check-airflow-config-yaml-consistent | check-airflow-provider-compatibility |     -check-apache-license-rat | check-base-operator-partial-arguments |                -check-base-operator-usage | check-boring-cyborg-configuration |                   -check-breeze-top-dependencies-limited | check-builtin-literals |                  -check-changelog-has-no-duplicates | check-core-deprecation-classes |              -check-daysago-import-from-utils | check-decorated-operator-implements-custom-name -| check-deferrable-default-value | check-docstring-param-types |                  -check-example-dags-urls | check-executables-have-shebangs |                       -check-extra-packages-references | check-extras-order |                            -check-for-inclusive-language | check-hooks-apply |                                -check-incorrect-use-of-LoggingMixin | check-init-decorator-arguments |            -check-lazy-logging | check-links-to-example-dags-do-not-use-hardcoded-versions |  -check-merge-conflict | check-newsfragments-are-valid |                            -check-no-airflow-deprecation-in-providers | check-no-providers-in-core-examples | -check-no-relative-imports | check-only-new-session-with-provide-session |         -check-persist-credentials-disabled-in-github-workflows |                          -check-pre-commit-information-consistent | check-provide-create-sessions-imports | -check-provider-yaml-valid | check-providers-init-file-missing |                   -check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |         -check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |  -check-start-date-not-used-in-defaults | check-system-tests-present |              -check-system-tests-tocs | check-tests-unittest-testcase |                         -check-urlparse-usage-in-code | check-usage-of-re2-over-re | check-xml | codespell -| compile-www-assets | compile-www-assets-dev |                                   -create-missing-init-py-files-tests | debug-statements | detect-private-key |      -doctoc | end-of-file-fixer | fix-encoding-pragma | flynt | identity |             -insert-license | lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart -| lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending | mypy-core -| mypy-dev | mypy-docs | mypy-providers | pretty-format-json | python-no-log-warn -| replace-bad-characters | rst-backticks | ruff | shellcheck |                    -trailing-whitespace | ts-compile-format-lint-www | update-black-version |         -update-breeze-cmd-output | update-breeze-readme-config-hash |                     -update-common-sql-api-stubs | update-er-diagram | update-extras |                 -update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |             -update-installed-providers-to-be-sorted | update-local-yml-file |                 -update-migration-references | update-providers-dependencies |                     -update-spelling-wordlist-to-be-sorted | update-supported-versions |               -update-vendored-in-k8s-json-schema | update-version | yamllint)                   ---show-diff-on-failure-sShow diff for files modified by the checks. ---initialize-environmentInitialize environment before running checks. ---max-initialization-attemptsMaximum number of attempts to initialize environment before giving up. -(INTEGER RANGE)                                                        -[default: 3; 1<=x<=10]                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Selecting files to run the checks on ───────────────────────────────────────────────────────────────────────────────╮ ---file-fList of files to run the checks on.(PATH) ---all-files-aRun checks on all files. ---commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually     -exclusive with --last-commit.                                                                 -(TEXT)                                                                                        ---last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. ---only-my-changes-mRun checks for commits belonging to my PR only: for all commits between merge base to `main`  -branch and HEAD of your branch.                                                               -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Building image before running checks ───────────────────────────────────────────────────────────────────────────────╮ ---skip-image-checkSkip checking if the CI image is up to date. Useful if you run non-image checks only ---force-buildForce image build no matter if it is determined as needed. ---image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) -[default: latest]                                                               ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Pre-commit flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--type-tType(s) of the static checks to run.                                              +(all | black | blacken-docs | check-aiobotocore-optional |                        +check-airflow-provider-compatibility | check-apache-license-rat |                 +check-base-operator-partial-arguments | check-base-operator-usage |               +check-boring-cyborg-configuration | check-breeze-top-dependencies-limited |       +check-builtin-literals | check-changelog-has-no-duplicates |                      +check-core-deprecation-classes | check-daysago-import-from-utils |                +check-decorated-operator-implements-custom-name | check-deferrable-default-value  +| check-docstring-param-types | check-example-dags-urls |                         +check-executables-have-shebangs | check-extra-packages-references |               +check-extras-order | check-for-inclusive-language | check-hooks-apply |           +check-incorrect-use-of-LoggingMixin | check-init-decorator-arguments |            +check-lazy-logging | check-links-to-example-dags-do-not-use-hardcoded-versions |  +check-merge-conflict | check-newsfragments-are-valid |                            +check-no-airflow-deprecation-in-providers | check-no-providers-in-core-examples | +check-no-relative-imports | check-only-new-session-with-provide-session |         +check-persist-credentials-disabled-in-github-workflows |                          +check-pre-commit-information-consistent | check-provide-create-sessions-imports | +check-provider-yaml-valid | check-providers-init-file-missing |                   +check-providers-subpackages-init-file-exist | check-pydevd-left-in-code |         +check-revision-heads-map | check-safe-filter-usage-in-html | check-setup-order |  +check-start-date-not-used-in-defaults | check-system-tests-present |              +check-system-tests-tocs | check-tests-unittest-testcase |                         +check-urlparse-usage-in-code | check-usage-of-re2-over-re | check-xml | codespell +| compile-www-assets | compile-www-assets-dev |                                   +create-missing-init-py-files-tests | debug-statements | detect-private-key |      +doctoc | end-of-file-fixer | fix-encoding-pragma | flynt | identity |             +insert-license | lint-chart-schema | lint-css | lint-dockerfile | lint-helm-chart +| lint-json-schema | lint-markdown | lint-openapi | mixed-line-ending | mypy-core +| mypy-dev | mypy-docs | mypy-providers | pretty-format-json | python-no-log-warn +| replace-bad-characters | rst-backticks | ruff | shellcheck |                    +trailing-whitespace | ts-compile-format-lint-www | update-black-version |         +update-breeze-cmd-output | update-breeze-readme-config-hash |                     +update-common-sql-api-stubs | update-er-diagram | update-extras |                 +update-in-the-wild-to-be-sorted | update-inlined-dockerfile-scripts |             +update-installed-providers-to-be-sorted | update-local-yml-file |                 +update-migration-references | update-providers-dependencies |                     +update-spelling-wordlist-to-be-sorted | update-supported-versions |               +update-vendored-in-k8s-json-schema | update-version | yamllint)                   +--show-diff-on-failure-sShow diff for files modified by the checks. +--initialize-environmentInitialize environment before running checks. +--max-initialization-attemptsMaximum number of attempts to initialize environment before giving up. +(INTEGER RANGE)                                                        +[default: 3; 1<=x<=10]                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Selecting files to run the checks on ───────────────────────────────────────────────────────────────────────────────╮ +--file-fList of files to run the checks on.(PATH) +--all-files-aRun checks on all files. +--commit-ref-rRun checks for this commit reference only (can be any git commit-ish reference). Mutually     +exclusive with --last-commit.                                                                 +(TEXT)                                                                                        +--last-commit-cRun checks for all files in last commit. Mutually exclusive with --commit-ref. +--only-my-changes-mRun checks for commits belonging to my PR only: for all commits between merge base to `main`  +branch and HEAD of your branch.                                                               +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building image before running checks ───────────────────────────────────────────────────────────────────────────────╮ +--skip-image-checkSkip checking if the CI image is up to date. Useful if you run non-image checks only +--force-buildForce image build no matter if it is determined as needed. +--image-tag-tTag of the image which is used to run the image (implies --mount-sources=skip).(TEXT) +[default: latest]                                                               +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/scripts/ci/pre_commit/pre_commit_yaml_to_cfg.py b/scripts/ci/pre_commit/pre_commit_yaml_to_cfg.py deleted file mode 100755 index 45aff2c2a12b0..0000000000000 --- a/scripts/ci/pre_commit/pre_commit_yaml_to_cfg.py +++ /dev/null @@ -1,179 +0,0 @@ -#!/usr/bin/env python -# -# 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. -""" -Module to convert Airflow configs in config.yml to default_airflow.cfg file -""" -from __future__ import annotations - -import os -import re -from pathlib import Path - -import yaml -from packaging.version import parse as parse_version - -FILE_HEADER = """# -# 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. - -# This is the template for Airflow's default configuration. When Airflow is -# imported, it looks for a configuration file at $AIRFLOW_HOME/airflow.cfg. If -# it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# airflow.cfg instead. - - -# ----------------------- TEMPLATE BEGINS HERE ----------------------- -""" - - -def read_default_config_yaml(file_path: str) -> dict: - """ - Read Airflow configs from YAML file - - :param file_path: Full path to config.yaml - - :return: Python dictionary containing configs & their info - """ - - with open(file_path) as config_file: - return yaml.safe_load(config_file) - - -def write_config(yaml_config_file_path: str, default_cfg_file_path: str): - """ - Write config to default_airflow.cfg file - - :param yaml_config_file_path: Full path to config.yaml - :param default_cfg_file_path: Full path to default_airflow.cfg - """ - print(f"Converting {yaml_config_file_path} to {default_cfg_file_path}") - with open(default_cfg_file_path, "w") as configfile: - configfile.writelines(FILE_HEADER) - config_yaml = read_default_config_yaml(yaml_config_file_path) - - for section_name, section in config_yaml.items(): - _write_section(configfile, section_name, section) - - -def _write_section(configfile, section_name, section): - configfile.write(f"\n[{section_name}]\n") - section_description = None - if section["description"] is not None: - section_description = list( - filter(lambda x: (x is not None) or x != "", section["description"].splitlines()) - ) - if section_description: - configfile.write("\n") - for single_line_desc in section_description: - if single_line_desc == "": - configfile.write("#\n") - else: - configfile.write(f"# {single_line_desc}\n") - for idx, (option_name, option) in enumerate(section["options"].items()): - _write_option(configfile, idx, option_name, option) - - -def _write_option(configfile, idx, option_name, option): - option_description = None - version_added = option["version_added"] - if version_added is not None and parse_version(version_added) > airflow_version: - # skip if option is going to be added in the future version - return - if option["description"] is not None: - option_description = list(filter(lambda x: x is not None, option["description"].splitlines())) - - if option_description: - if idx != 0: - configfile.write("\n") - for single_line_desc in option_description: - if single_line_desc == "": - configfile.write("#\n") - else: - configfile.write(f"# {single_line_desc}\n") - - if option["example"]: - if not str(option_name).endswith("_template"): - option["example"] = option["example"].replace("{", "{{").replace("}", "}}") - configfile.write(f"# Example: {option_name} = {option['example']}\n") - - if option["default"] is not None: - if not isinstance(option["default"], str): - raise Exception( - f'Key "default" in element with name="{option_name}" has an invalid type. ' - f"Current type: {type(option['default'])}" - ) - # Remove trailing whitespace on empty string - if option["default"]: - value = " " + option["default"] - else: - value = "" - configfile.write(f"{option_name} ={value}\n") - else: - configfile.write(f"# {option_name} =\n") - - -if __name__ == "__main__": - airflow_config_dir = os.path.join( - os.path.dirname(__file__), os.pardir, os.pardir, os.pardir, "airflow", "config_templates" - ) - airflow_default_config_path = os.path.join(airflow_config_dir, "default_airflow.cfg") - airflow_config_yaml_file_path = os.path.join(airflow_config_dir, "config.yml") - - airflow_version = parse_version( - re.search( # type: ignore[union-attr,arg-type] - r"__version__ = \"([0-9\.]*)(\.dev[0-9]*)?\"", - (Path(__file__).parents[3] / "airflow" / "__init__.py").read_text(), - ).groups(0)[0] - ) - - write_config( - yaml_config_file_path=airflow_config_yaml_file_path, default_cfg_file_path=airflow_default_config_path - ) - - providers_dir = os.path.join( - os.path.dirname(__file__), os.pardir, os.pardir, os.pardir, "airflow", "providers" - ) - for root, dir_names, file_names in os.walk(providers_dir): - for file_name in file_names: - if ( - root.endswith("config_templates") - and file_name == "config.yml" - and os.path.isfile(os.path.join(root, "default_config.cfg")) - ): - write_config( - yaml_config_file_path=os.path.join(root, "config.yml"), - default_cfg_file_path=os.path.join(root, "default_config.cfg"), - ) diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh index 287bcbca0e09e..237c97a300f8d 100755 --- a/scripts/docker/entrypoint_ci.sh +++ b/scripts/docker/entrypoint_ci.sh @@ -262,7 +262,6 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then unset AIRFLOW__CORE__UNIT_TEST_MODE mkdir -pv "${AIRFLOW_HOME}/logs/" - cp -f "${IN_CONTAINER_DIR}/airflow_ci.cfg" "${AIRFLOW_HOME}/unittests.cfg" # Change the default worker_concurrency for tests export AIRFLOW__CELERY__WORKER_CONCURRENCY=8 diff --git a/scripts/in_container/airflow_ci.cfg b/scripts/in_container/airflow_ci.cfg deleted file mode 100644 index 57863bd9c69f6..0000000000000 --- a/scripts/in_container/airflow_ci.cfg +++ /dev/null @@ -1,59 +0,0 @@ -# -# 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] -dags_folder = ~/airflow/dags -executor = LocalExecutor -sql_alchemy_conn = # overridden by the startup scripts -#sql_engine_collation_for_ids = overridden by the startup scripts -unit_test_mode = True -load_examples = True -donot_pickle = False -dags_are_paused_at_creation = False -default_impersonation = -fernet_key = af7CN0q6ag5U3g08IsPsw3K45U7Xa0axgVFhoh-3zB8= - -[database] -load_default_connections = True - -[hive] -default_hive_mapred_queue = airflow - -[logging] -base_log_folder = ~/airflow/logs - -[smtp] -smtp_user = airflow -smtp_password = airflow -smtp_mail_from = airflow@example.com - -[celery] -broker_url = amqp://guest:guest@rabbitmq:5672/ -result_backend = # overridden by startup scripts - - -[celery_broker_transport_options] -visibility_timeout = 21600 -_test_only_bool = True -_test_only_float = 12.0 -_test_only_string = this is a test - -[scheduler] -job_heartbeat_sec = 1 -scheduler_heartbeat_sec = 5 -parsing_processes = 2 diff --git a/tests/cli/commands/test_config_command.py b/tests/cli/commands/test_config_command.py index f5f90bd2f3476..f5b427f0cfca8 100644 --- a/tests/cli/commands/test_config_command.py +++ b/tests/cli/commands/test_config_command.py @@ -37,7 +37,14 @@ def setup_class(cls): def test_cli_show_config_should_write_data(self, mock_conf, mock_stringio): config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) mock_conf.write.assert_called_once_with( - mock_stringio.return_value.__enter__.return_value, section=None + mock_stringio.return_value.__enter__.return_value, + section=None, + include_examples=False, + include_descriptions=False, + include_sources=False, + include_env_vars=False, + comment_out_everything=False, + only_defaults=False, ) @mock.patch("airflow.cli.commands.config_command.io.StringIO") @@ -47,16 +54,141 @@ def test_cli_show_config_should_write_data_specific_section(self, mock_conf, moc self.parser.parse_args(["config", "list", "--section", "core", "--color", "off"]) ) mock_conf.write.assert_called_once_with( - mock_stringio.return_value.__enter__.return_value, section="core" + mock_stringio.return_value.__enter__.return_value, + section="core", + include_examples=False, + include_descriptions=False, + include_sources=False, + include_env_vars=False, + comment_out_everything=False, + only_defaults=False, ) @conf_vars({("core", "testkey"): "test_value"}) def test_cli_show_config_should_display_key(self): with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) - assert "[core]" in temp_stdout.getvalue() + output = temp_stdout.getvalue() + assert "[core]" in output assert "testkey = test_value" in temp_stdout.getvalue() + def test_cli_show_config_should_only_show_comments_when_no_defaults(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert all(not line.startswith("#") or line.endswith("= ") for line in lines if line) + + def test_cli_show_config_shows_descriptions(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--include-descriptions"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + # comes from metrics description + assert all(not line.startswith("# Source: ") for line in lines if line) + assert any(line.startswith("# StatsD") for line in lines if line) + assert all(not line.startswith("# Example:") for line in lines if line) + assert all(not line.startswith("# Variable:") for line in lines if line) + + def test_cli_show_config_shows_examples(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--include-examples"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert all(not line.startswith("# Source: ") for line in lines if line) + assert all(not line.startswith("# StatsD") for line in lines if line) + assert any(line.startswith("# Example:") for line in lines if line) + assert all(not line.startswith("# Variable:") for line in lines if line) + + def test_cli_show_config_shows_variables(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--include-env-vars"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert all(not line.startswith("# Source: ") for line in lines if line) + assert all(not line.startswith("# StatsD") for line in lines if line) + assert all(not line.startswith("# Example:") for line in lines if line) + assert any(line.startswith("# Variable:") for line in lines if line) + + def test_cli_show_config_shows_sources(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--include-sources"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("# Source: ") for line in lines if line) + assert all(not line.startswith("# StatsD") for line in lines if line) + assert all(not line.startswith("# Example:") for line in lines if line) + assert all(not line.startswith("# Variable:") for line in lines if line) + + def test_cli_show_config_defaults(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--defaults"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert all(not line.startswith("# Source: ") for line in lines if line) + assert any(line.startswith("# StatsD") for line in lines if line) + assert any(not line.startswith("# Example:") for line in lines if line) + assert any(not line.startswith("# Example:") for line in lines if line) + assert any(line.startswith("# Variable:") for line in lines if line) + assert any(line.startswith("# task_runner = StandardTaskRunner") for line in lines if line) + + @conf_vars({("core", "task_runner"): "test-runner"}) + def test_cli_show_config_defaults_not_show_conf_changes(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--defaults"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("# task_runner = StandardTaskRunner") for line in lines if line) + + @mock.patch("os.environ", {"AIRFLOW__CORE__TASK_RUNNER": "test-env-runner"}) + def test_cli_show_config_defaults_do_not_show_env_changes(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--defaults"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("# task_runner = StandardTaskRunner") for line in lines if line) + + @conf_vars({("core", "task_runner"): "test-runner"}) + def test_cli_show_changed_defaults_when_overridden_in_conf(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("task_runner = test-runner") for line in lines if line) + + @mock.patch("os.environ", {"AIRFLOW__CORE__TASK_RUNNER": "test-env-runner"}) + def test_cli_show_changed_defaults_when_overridden_in_env(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("task_runner = test-env-runner") for line in lines if line) + + def test_cli_comment_out_everything(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config( + self.parser.parse_args(["config", "list", "--color", "off", "--comment-out-everything"]) + ) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert all( + line.startswith("#") or line.strip() == "" or line.startswith("[") for line in lines if line + ) + class TestCliConfigGetValue: @classmethod diff --git a/tests/cli/test_cli_parser.py b/tests/cli/test_cli_parser.py index 378ebd4e715e6..7cac8f7ad1d1d 100644 --- a/tests/cli/test_cli_parser.py +++ b/tests/cli/test_cli_parser.py @@ -198,13 +198,26 @@ def test_positive_int(self): cli_config.positive_int(allow_zero=False)("0") cli_config.positive_int(allow_zero=True)("-1") - def test_dag_parser_celery_command_require_celery_executor(self): + @pytest.mark.parametrize( + "command", + [ + ["celery"], + ["celery", "--help"], + ["celery", "worker", "--help"], + ["celery", "worker"], + ["celery", "flower", "--help"], + ["celery", "flower"], + ["celery", "stop_worker", "--help"], + ["celery", "stop_worker"], + ], + ) + def test_dag_parser_require_celery_executor(self, command): with conf_vars({("core", "executor"): "SequentialExecutor"}), contextlib.redirect_stderr( io.StringIO() ) as stderr: parser = cli_parser.get_parser() with pytest.raises(SystemExit): - parser.parse_args(["celery"]) + parser.parse_args(command) stderr = stderr.getvalue() assert ( "airflow command error: argument GROUP_OR_COMMAND: celery subcommand " diff --git a/tests/config_templates/deprecated.cfg b/tests/config_templates/deprecated.cfg index cab0b6e7fa2d5..1a79045424816 100644 --- a/tests/config_templates/deprecated.cfg +++ b/tests/config_templates/deprecated.cfg @@ -15,14 +15,5 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - - -# This is the template for Airflow's unit test configuration. When Airflow runs -# unit tests, it looks for a configuration file at $AIRFLOW_HOME/unittests.cfg. -# If it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# unittests.cfg instead. [core] sql_alchemy_conn = mysql:// diff --git a/tests/config_templates/deprecated_cmd.cfg b/tests/config_templates/deprecated_cmd.cfg index ff2a9268dd691..dbe819fbb63f8 100644 --- a/tests/config_templates/deprecated_cmd.cfg +++ b/tests/config_templates/deprecated_cmd.cfg @@ -15,15 +15,5 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - - -# This is the template for Airflow's unit test configuration. When Airflow runs -# unit tests, it looks for a configuration file at $AIRFLOW_HOME/unittests.cfg. -# If it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# unittests.cfg instead. - [core] sql_alchemy_conn_cmd = echo -n "postgresql://" diff --git a/tests/config_templates/deprecated_secret.cfg b/tests/config_templates/deprecated_secret.cfg index 6c4cda091af6e..ca2f5aa637199 100644 --- a/tests/config_templates/deprecated_secret.cfg +++ b/tests/config_templates/deprecated_secret.cfg @@ -15,15 +15,5 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - - -# This is the template for Airflow's unit test configuration. When Airflow runs -# unit tests, it looks for a configuration file at $AIRFLOW_HOME/unittests.cfg. -# If it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# unittests.cfg instead. - [core] sql_alchemy_conn_secret = secret_path diff --git a/tests/config_templates/empty.cfg b/tests/config_templates/empty.cfg index 36906c07cfde6..217e5db960782 100644 --- a/tests/config_templates/empty.cfg +++ b/tests/config_templates/empty.cfg @@ -15,12 +15,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - - -# This is the template for Airflow's unit test configuration. When Airflow runs -# unit tests, it looks for a configuration file at $AIRFLOW_HOME/unittests.cfg. -# If it doesn't exist, Airflow uses this template to generate it by replacing -# variables in curly braces with their global values from configuration.py. - -# Users should not modify this file; they should customize the generated -# unittests.cfg instead. diff --git a/tests/conftest.py b/tests/conftest.py index 9fdda1467cf73..b2ef0b1e6e903 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -22,6 +22,7 @@ import sys from contextlib import ExitStack, suppress from datetime import datetime, timedelta +from pathlib import Path from typing import TYPE_CHECKING import pytest @@ -32,9 +33,10 @@ from itsdangerous import URLSafeSerializer assert "airflow" not in sys.modules, "No airflow module can be imported before these lines" -tests_directory = os.path.dirname(os.path.realpath(__file__)) +AIRFLOW_TESTS_DIR = Path(os.path.dirname(os.path.realpath(__file__))).resolve() +AIRFLOW_SOURCES_ROOT_DIR = AIRFLOW_TESTS_DIR.parent.parent -os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = os.path.join(tests_directory, "dags") +os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = os.fspath(AIRFLOW_TESTS_DIR / "dags") os.environ["AIRFLOW__CORE__UNIT_TEST_MODE"] = "True" os.environ["AWS_DEFAULT_REGION"] = os.environ.get("AWS_DEFAULT_REGION") or "us-east-1" os.environ["CREDENTIALS_DIR"] = os.environ.get("CREDENTIALS_DIR") or "/files/airflow-breeze-config/keys" @@ -882,6 +884,15 @@ def _clear_db(request): """Clear DB before each test module run.""" if not request.config.option.db_cleanup: return + from airflow.configuration import conf + + sql_alchemy_conn = conf.get("database", "sql_alchemy_conn") + if sql_alchemy_conn.startswith("sqlite"): + sql_alchemy_file = sql_alchemy_conn.replace("sqlite:///", "") + if not os.path.exists(sql_alchemy_file): + print(f"The sqlite file `{sql_alchemy_file}` does not exist. Attempt to initialize it.") + initial_db_init() + dist_option = getattr(request.config.option, "dist", "no") if dist_option != "no" or hasattr(request.config, "workerinput"): # Skip if pytest-xdist detected (controller or worker) diff --git a/tests/core/test_config_templates.py b/tests/core/test_config_templates.py deleted file mode 100644 index 574bcf9537ede..0000000000000 --- a/tests/core/test_config_templates.py +++ /dev/null @@ -1,98 +0,0 @@ -# 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. -from __future__ import annotations - -import configparser -import os - -import pytest - -from tests.test_utils import AIRFLOW_MAIN_FOLDER - -CONFIG_TEMPLATES_FOLDER = os.path.join(AIRFLOW_MAIN_FOLDER, "airflow", "config_templates") - -DEFAULT_AIRFLOW_SECTIONS = [ - "core", - "database", - "logging", - "metrics", - "secrets", - "cli", - "debug", - "api", - "lineage", - "openlineage", - "atlas", - "operators", - "hive", - "webserver", - "email", - "smtp", - "sentry", - "local_kubernetes_executor", - "celery_kubernetes_executor", - "celery", - "celery_broker_transport_options", - "dask", - "scheduler", - "triggerer", - "kerberos", - "elasticsearch", - "elasticsearch_configs", - "kubernetes_executor", - "sensors", -] - -DEFAULT_TEST_SECTIONS = [ - "core", - "database", - "logging", - "api", - "hive", - "smtp", - "celery", - "scheduler", - "elasticsearch", -] - - -class TestAirflowCfg: - @pytest.mark.parametrize("filename", ["default_airflow.cfg", "default_test.cfg"]) - def test_should_be_ascii_file(self, filename: str): - with open(os.path.join(CONFIG_TEMPLATES_FOLDER, filename), "rb") as f: - content = f.read().decode("ascii") - assert content - - @pytest.mark.parametrize( - "filename, expected_sections", - [ - ( - "default_airflow.cfg", - DEFAULT_AIRFLOW_SECTIONS, - ), - ( - "default_test.cfg", - DEFAULT_TEST_SECTIONS, - ), - ], - ) - def test_should_be_ini_file(self, filename: str, expected_sections): - filepath = os.path.join(CONFIG_TEMPLATES_FOLDER, filename) - config = configparser.ConfigParser() - config.read(filepath) - - assert expected_sections == config.sections() diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py index 90aa7959ea6a8..aa7820cf37b5a 100644 --- a/tests/core/test_configuration.py +++ b/tests/core/test_configuration.py @@ -36,11 +36,10 @@ AirflowConfigException, AirflowConfigParser, conf, - default_config_yaml, expand_env_var, get_airflow_config, get_airflow_home, - parameterized_config, + get_all_expansion_variables, run_command, ) from tests.test_utils.config import conf_vars @@ -61,6 +60,16 @@ def restore_env(): yield +def parameterized_config(template) -> str: + """ + Generates configuration from provided template & variables defined in current scope. + + :param template: a config content templated with {{variables}} + """ + all_vars = get_all_expansion_variables() + return template.format(**all_vars) + + @mock.patch.dict( "os.environ", { @@ -566,12 +575,19 @@ def test_kubernetes_environment_variables_section(self): "kubernetes_environment_variables" ) - def test_broker_transport_options(self): - section_dict = conf.getsection("celery_broker_transport_options") - assert isinstance(section_dict["visibility_timeout"], int) - assert isinstance(section_dict["_test_only_bool"], bool) - assert isinstance(section_dict["_test_only_float"], float) - assert isinstance(section_dict["_test_only_string"], str) + @pytest.mark.parametrize( + "key, type", + [ + ("string_value", int), # Coercion happens here + ("only_bool_value", bool), + ("only_float_value", float), + ("only_integer_value", int), + ("only_string_value", str), + ], + ) + def test_config_value_types(self, key, type): + section_dict = conf.getsection("example_section") + assert isinstance(section_dict[key], type) def test_auth_backends_adds_session(self): test_conf = AirflowConfigParser(default_config="") @@ -608,7 +624,7 @@ def test_command_from_env(self): # sensitive_config_values and therefore should return 'OK' from the environment variable's # echo command, and must not return 'NOT OK' from the configuration assert test_cmdenv_conf.get("testcmdenv", "itsacommand") == "OK" - # AIRFLOW__TESTCMDENV__NOTACOMMAND_CMD maps to no entry in sensitive_config_values and therefore + # AIRFLOW__TESTCMDENV__xNOTACOMMAND_CMD maps to no entry in sensitive_config_values and therefore # the option should return 'OK' from the configuration, and must not return 'NOT OK' from # the environment variable's echo command assert test_cmdenv_conf.get("testcmdenv", "notacommand") == "OK" @@ -687,11 +703,20 @@ def test_config_override_original_when_empty_envvar_is_provided(self): @mock.patch.dict("os.environ", {"AIRFLOW__CORE__DAGS_FOLDER": "/tmp/test_folder"}) def test_write_should_respect_env_variable(self): + parser = AirflowConfigParser() with io.StringIO() as string_file: - conf.write(string_file) + parser.write(string_file) content = string_file.getvalue() assert "dags_folder = /tmp/test_folder" in content + @mock.patch.dict("os.environ", {"AIRFLOW__CORE__DAGS_FOLDER": "/tmp/test_folder"}) + def test_write_with_only_defaults_should_not_respect_env_variable(self): + parser = AirflowConfigParser() + with io.StringIO() as string_file: + parser.write(string_file, only_defaults=True) + content = string_file.getvalue() + assert "dags_folder = /tmp/test_folder" not in content + def test_run_command(self): write = r'sys.stdout.buffer.write("\u1000foo".encode("utf8"))' @@ -734,15 +759,15 @@ def test_as_dict_works_without_sensitive_cmds(self): conf_materialize_cmds = conf.as_dict(display_sensitive=True, raw=True, include_cmds=True) conf_maintain_cmds = conf.as_dict(display_sensitive=True, raw=True, include_cmds=False) - assert "sql_alchemy_conn" in conf_materialize_cmds["core"] - assert "sql_alchemy_conn_cmd" not in conf_materialize_cmds["core"] + assert "sql_alchemy_conn" in conf_materialize_cmds["database"] + assert "sql_alchemy_conn_cmd" not in conf_materialize_cmds["database"] - assert "sql_alchemy_conn" in conf_maintain_cmds["core"] - assert "sql_alchemy_conn_cmd" not in conf_maintain_cmds["core"] + assert "sql_alchemy_conn" in conf_maintain_cmds["database"] + assert "sql_alchemy_conn_cmd" not in conf_maintain_cmds["database"] assert ( - conf_materialize_cmds["core"]["sql_alchemy_conn"] - == conf_maintain_cmds["core"]["sql_alchemy_conn"] + conf_materialize_cmds["database"]["sql_alchemy_conn"] + == conf_maintain_cmds["database"]["sql_alchemy_conn"] ) def test_as_dict_respects_sensitive_cmds(self): @@ -763,13 +788,13 @@ def test_as_dict_respects_sensitive_cmds(self): assert "sql_alchemy_conn" in conf_materialize_cmds["database"] assert "sql_alchemy_conn_cmd" not in conf_materialize_cmds["database"] - if conf_conn == test_conf.airflow_defaults["database"]["sql_alchemy_conn"]: + if conf_conn == test_conf._default_values["database"]["sql_alchemy_conn"]: assert conf_materialize_cmds["database"]["sql_alchemy_conn"] == "my-super-secret-conn" assert "sql_alchemy_conn_cmd" in conf_maintain_cmds["database"] assert conf_maintain_cmds["database"]["sql_alchemy_conn_cmd"] == "echo -n my-super-secret-conn" - if conf_conn == test_conf.airflow_defaults["database"]["sql_alchemy_conn"]: + if conf_conn == test_conf._default_values["database"]["sql_alchemy_conn"]: assert "sql_alchemy_conn" not in conf_maintain_cmds["database"] else: assert "sql_alchemy_conn" in conf_maintain_cmds["database"] @@ -1449,6 +1474,93 @@ def test_future_warning_only_for_code_ref(self, key): assert "your `conf.get*` call to use the new name" in str(w.message) assert w.category == FutureWarning + def test_as_dict_raw(self): + test_conf = AirflowConfigParser() + raw_dict = test_conf.as_dict(raw=True) + assert "%%" in raw_dict["logging"]["log_format"] + + def test_as_dict_not_raw(self): + test_conf = AirflowConfigParser() + raw_dict = test_conf.as_dict(raw=False) + assert "%%" not in raw_dict["logging"]["log_format"] + + def test_default_value_raw(self): + test_conf = AirflowConfigParser() + log_format = test_conf.get_default_value("logging", "log_format", raw=True) + assert "%%" in log_format + + def test_default_value_not_raw(self): + test_conf = AirflowConfigParser() + log_format = test_conf.get_default_value("logging", "log_format", raw=False) + assert "%%" not in log_format + + def test_default_value_raw_with_fallback(self): + test_conf = AirflowConfigParser() + log_format = test_conf.get_default_value("logging", "missing", fallback="aa %%", raw=True) + assert "%%" in log_format + + def test_default_value_not_raw_with_fallback(self): + test_conf = AirflowConfigParser() + log_format = test_conf.get_default_value("logging", "missing", fallback="aa %%", raw=False) + # Note that fallback is never interpolated so we expect the value passed as-is + assert "%%" in log_format + + def test_written_defaults_are_raw_for_defaults(self): + test_conf = AirflowConfigParser() + with io.StringIO() as f: + test_conf.write(f, only_defaults=True) + string_written = f.getvalue() + assert "%%(asctime)s" in string_written + + def test_written_defaults_are_raw_for_non_defaults(self): + test_conf = AirflowConfigParser() + with io.StringIO() as f: + test_conf.write(f) + string_written = f.getvalue() + assert "%%(asctime)s" in string_written + + def test_get_sections_including_defaults(self): + airflow_cfg = AirflowConfigParser() + airflow_cfg.remove_all_read_configurations() + default_sections = airflow_cfg.get_sections_including_defaults() + assert "core" in default_sections + assert "test-section" not in default_sections + airflow_cfg.add_section("test-section") + airflow_cfg.set("test-section", "test-key", "test-value") + all_sections_including_defaults = airflow_cfg.get_sections_including_defaults() + assert "core" in all_sections_including_defaults + assert "test-section" in all_sections_including_defaults + airflow_cfg.add_section("core") + airflow_cfg.set("core", "new-test-key", "test-value") + all_sections_including_defaults = airflow_cfg.get_sections_including_defaults() + assert "core" in all_sections_including_defaults + assert "test-section" in all_sections_including_defaults + assert len([section for section in all_sections_including_defaults if section == "core"]) == 1 + + def test_get_options_including_defaults(self): + airflow_cfg = AirflowConfigParser() + airflow_cfg.remove_all_read_configurations() + default_options = airflow_cfg.get_options_including_defaults("core") + assert "task_runner" in default_options + assert "StandardTaskRunner" == airflow_cfg.get("core", "task_runner") + assert "test-key" not in default_options + no_options = airflow_cfg.get_options_including_defaults("test-section") + assert no_options == [] + airflow_cfg.add_section("test-section") + airflow_cfg.set("test-section", "test-key", "test-value") + test_section_options = airflow_cfg.get_options_including_defaults("test-section") + assert "test-key" in test_section_options + assert "StandardTaskRunner" == airflow_cfg.get("core", "task_runner") + airflow_cfg.add_section("core") + airflow_cfg.set("core", "new-test-key", "test-value") + airflow_cfg.set("core", "task_runner", "test-runner") + all_core_options_including_defaults = airflow_cfg.get_options_including_defaults("core") + assert "new-test-key" in all_core_options_including_defaults + assert "dags_folder" in all_core_options_including_defaults + assert "test-value" == airflow_cfg.get("core", "new-test-key") + assert "test-runner" == airflow_cfg.get("core", "task_runner") + assert len([option for option in all_core_options_including_defaults if option == "task_runner"]) == 1 + def test_sensitive_values(): from airflow.settings import conf @@ -1471,8 +1583,7 @@ def test_sensitive_values(): ("database", "sql_alchemy_engine_args"), ("core", "sql_alchemy_conn"), } - default_config = default_config_yaml() - all_keys = {(s, k) for s, v in default_config.items() for k in v.get("options")} + all_keys = {(s, k) for s, v in conf.configuration_description.items() for k in v.get("options")} suspected_sensitive = {(s, k) for (s, k) in all_keys if k.endswith(("password", "kwargs"))} exclude_list = { ("kubernetes_executor", "delete_option_kwargs"),