Configuration Reference

This page contains the list of all the available Airflow configurations that you can set in airflow.cfg file or using environment variables.

Use the same configuration across all the Airflow components. While each component does not require all, some configurations need to be same otherwise they would not work as expected. A good example for that is secret_key which should be same on the Webserver and Worker to allow Webserver to fetch logs from Worker.

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.

Note

For more information on setting the configuration, see Setting Configuration Options

[api]

access_control_allow_headers

New in version 2.1.0.

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.

Type

string

Default

''

Environment Variable

AIRFLOW__API__ACCESS_CONTROL_ALLOW_HEADERS

access_control_allow_methods

New in version 2.1.0.

Specifies the method or methods allowed when accessing the resource.

Type

string

Default

''

Environment Variable

AIRFLOW__API__ACCESS_CONTROL_ALLOW_METHODS

access_control_allow_origins

New in version 2.2.0.

Indicates whether the response can be shared with requesting code from the given origins. Separate URLs with space.

Type

string

Default

''

Environment Variable

AIRFLOW__API__ACCESS_CONTROL_ALLOW_ORIGINS

auth_backends

New in version 2.3.0.

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)

Type

string

Default

airflow.api.auth.backend.session

Environment Variable

AIRFLOW__API__AUTH_BACKENDS

enable_experimental_api

New in version 2.0.0.

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

Type

boolean

Default

False

Environment Variable

AIRFLOW__API__ENABLE_EXPERIMENTAL_API

fallback_page_limit

New in version 2.0.0.

Used to set the default page limit when limit is zero. A default limit of 100 is set on OpenApi spec. However, this particular default limit only work when limit is set equal to zero(0) from API requests. If no limit is supplied, the OpenApi spec default is used.

Type

integer

Default

100

Environment Variable

AIRFLOW__API__FALLBACK_PAGE_LIMIT

google_key_path

New in version 2.0.0.

Path to Google Cloud Service Account key file (JSON). If omitted, authorization based on the Application Default Credentials will be used.

Type

string

Default

''

Environment Variable

AIRFLOW__API__GOOGLE_KEY_PATH

Example

/files/service-account-json

google_oauth2_audience

New in version 2.0.0.

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.

Type

string

Default

''

Environment Variable

AIRFLOW__API__GOOGLE_OAUTH2_AUDIENCE

Example

project-id-random-value.apps.googleusercontent.com

maximum_page_limit

New in version 2.0.0.

Used to set the maximum page limit for API requests

Type

integer

Default

100

Environment Variable

AIRFLOW__API__MAXIMUM_PAGE_LIMIT

access_control_allow_origin (Deprecated)

Deprecated since version 2.2.0: The option has been moved to api.access_control_allow_origins

auth_backend (Deprecated)

Deprecated since version 2.3.0: The option has been moved to api.auth_backends

[atlas]

host

Type

string

Default

''

Environment Variable

AIRFLOW__ATLAS__HOST

password

Type

string

Default

''

Environment Variables

AIRFLOW__ATLAS__PASSWORD

AIRFLOW__ATLAS__PASSWORD_CMD

AIRFLOW__ATLAS__PASSWORD_SECRET

port

Type

string

Default

21000

Environment Variable

AIRFLOW__ATLAS__PORT

sasl_enabled

Type

string

Default

False

Environment Variable

AIRFLOW__ATLAS__SASL_ENABLED

username

Type

string

Default

''

Environment Variable

AIRFLOW__ATLAS__USERNAME

[celery]

This section only applies if you are using the CeleryExecutor in [core] section above

broker_url

The Celery broker URL. Celery supports RabbitMQ, Redis and experimentally a sqlalchemy database. Refer to the Celery documentation for more information.

Type

string

Default

redis://redis:6379/0

Environment Variables

AIRFLOW__CELERY__BROKER_URL

AIRFLOW__CELERY__BROKER_URL_CMD

AIRFLOW__CELERY__BROKER_URL_SECRET

celery_app_name

The app name that will be used by celery

Type

string

Default

airflow.executors.celery_executor

Environment Variable

AIRFLOW__CELERY__CELERY_APP_NAME

celery_config_options

Import path for celery configuration options

Type

string

Default

airflow.config_templates.default_celery.DEFAULT_CELERY_CONFIG

Environment Variable

AIRFLOW__CELERY__CELERY_CONFIG_OPTIONS

flower_basic_auth

New in version 1.10.2.

Securing Flower with Basic Authentication Accepts user:password pairs separated by a comma

Type

string

Default

''

Environment Variables

AIRFLOW__CELERY__FLOWER_BASIC_AUTH

AIRFLOW__CELERY__FLOWER_BASIC_AUTH_CMD

AIRFLOW__CELERY__FLOWER_BASIC_AUTH_SECRET

Example

user1:password1,user2:password2

flower_host

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

Type

string

Default

0.0.0.0

Environment Variable

AIRFLOW__CELERY__FLOWER_HOST

flower_port

This defines the port that Celery Flower runs on

Type

string

Default

5555

Environment Variable

AIRFLOW__CELERY__FLOWER_PORT

flower_url_prefix

The root URL for Flower

Type

string

Default

''

Environment Variable

AIRFLOW__CELERY__FLOWER_URL_PREFIX

Example

/flower

operation_timeout

New in version 1.10.8.

The number of seconds to wait before timing out send_task_to_executor or fetch_celery_task_state operations.

Type

float

Default

1.0

Environment Variable

AIRFLOW__CELERY__OPERATION_TIMEOUT

pool

New in version 1.10.4.

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

Type

string

Default

prefork

Environment Variable

AIRFLOW__CELERY__POOL

result_backend

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 http://docs.celeryproject.org/en/latest/userguide/configuration.html#task-result-backend-settings

Type

string

Default

db+postgresql://postgres:airflow@postgres/airflow

Environment Variables

AIRFLOW__CELERY__RESULT_BACKEND

AIRFLOW__CELERY__RESULT_BACKEND_CMD

AIRFLOW__CELERY__RESULT_BACKEND_SECRET

ssl_active

Type

string

Default

False

Environment Variable

AIRFLOW__CELERY__SSL_ACTIVE

ssl_cacert

Type

string

Default

''

Environment Variable

AIRFLOW__CELERY__SSL_CACERT

ssl_cert

Type

string

Default

''

Environment Variable

AIRFLOW__CELERY__SSL_CERT

ssl_key

Type

string

Default

''

Environment Variable

AIRFLOW__CELERY__SSL_KEY

stalled_task_timeout

New in version 2.3.1.

Time in seconds after which tasks queued in celery are assumed to be stalled, and are automatically rescheduled. Adopted tasks will instead use the task_adoption_timeout setting if specified. When set to 0, automatic clearing of stalled tasks is disabled.

Type

integer

Default

0

Environment Variable

AIRFLOW__CELERY__STALLED_TASK_TIMEOUT

sync_parallelism

New in version 1.10.3.

How many processes CeleryExecutor uses to sync task state. 0 means to use max(1, number of cores - 1) processes.

Type

string

Default

0

Environment Variable

AIRFLOW__CELERY__SYNC_PARALLELISM

task_adoption_timeout

New in version 2.0.0.

Time in seconds after which adopted tasks which are queued in celery are assumed to be stalled, and are automatically rescheduled. This setting does the same thing as stalled_task_timeout but applies specifically to adopted tasks only. When set to 0, the stalled_task_timeout setting also applies to adopted tasks.

Type

integer

Default

600

Environment Variable

AIRFLOW__CELERY__TASK_ADOPTION_TIMEOUT

task_publish_max_retries

New in version 2.0.0.

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.

Type

integer

Default

3

Environment Variable

AIRFLOW__CELERY__TASK_PUBLISH_MAX_RETRIES

task_track_started

New in version 2.0.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.

Type

boolean

Default

True

Environment Variable

AIRFLOW__CELERY__TASK_TRACK_STARTED

worker_autoscale

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

Type

string

Default

None

Environment Variable

AIRFLOW__CELERY__WORKER_AUTOSCALE

Example

16,12

worker_concurrency

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

Type

string

Default

16

Environment Variable

AIRFLOW__CELERY__WORKER_CONCURRENCY

worker_enable_remote_control

New in version 2.3.0.

Specify if remote control of the workers is enabled. When using Amazon SQS as the broker, 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.

Type

boolean

Default

true

Environment Variable

AIRFLOW__CELERY__WORKER_ENABLE_REMOTE_CONTROL

worker_precheck

New in version 2.0.0.

Worker initialisation check to validate Metadata Database connection

Type

string

Default

False

Environment Variable

AIRFLOW__CELERY__WORKER_PRECHECK

worker_prefetch_multiplier

New in version 2.0.0.

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

Type

integer

Default

1

Environment Variable

AIRFLOW__CELERY__WORKER_PREFETCH_MULTIPLIER

worker_umask

New in version 2.0.0.

Umask that will be used when starting workers with the airflow celery worker in daemon mode. This control the file-creation mode mask which determines the initial value of file permission bits for newly created files.

Type

string

Default

0o077

Environment Variable

AIRFLOW__CELERY__WORKER_UMASK

default_queue (Deprecated)

Deprecated since version 2.1.0: The option has been moved to operators.default_queue

worker_log_server_port (Deprecated)

Deprecated since version 2.2.0: The option has been moved to logging.worker_log_server_port

[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

visibility_timeout

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: http://docs.celeryproject.org/en/master/userguide/configuration.html#std:setting-broker_transport_options

Type

string

Default

None

Environment Variable

AIRFLOW__CELERY_BROKER_TRANSPORT_OPTIONS__VISIBILITY_TIMEOUT

Example

21600

[celery_kubernetes_executor]

This section only applies if you are using the CeleryKubernetesExecutor in [core] section above

kubernetes_queue

New in version 2.0.0.

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

Type

string

Default

kubernetes

Environment Variable

AIRFLOW__CELERY_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE

[cli]

api_client

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

Type

string

Default

airflow.api.client.local_client

Environment Variable

AIRFLOW__CLI__API_CLIENT

endpoint_url

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

Type

string

Default

http://localhost:8080

Environment Variable

AIRFLOW__CLI__ENDPOINT_URL

[core]

check_slas

New in version 1.10.8.

On each dagrun check against defined SLAs

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__CHECK_SLAS

compress_serialized_dags

New in version 2.3.0.

If True, serialized DAGs are compressed before writing to DB. Note: this will disable the DAG dependencies view

Type

string

Default

False

Environment Variable

AIRFLOW__CORE__COMPRESS_SERIALIZED_DAGS

dag_discovery_safe_mode

New in version 1.10.3.

When discovering DAGs, ignore any files that don’t contain the strings DAG and airflow.

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__DAG_DISCOVERY_SAFE_MODE

dag_file_processor_timeout

New in version 1.10.6.

How long before timing out a DagFileProcessor, which processes a dag file

Type

string

Default

50

Environment Variable

AIRFLOW__CORE__DAG_FILE_PROCESSOR_TIMEOUT

dag_ignore_file_syntax

New in version 2.3.0.

The pattern syntax used in the “.airflowignore” files in the DAG directories. Valid values are regexp or glob.

Type

string

Default

regexp

Environment Variable

AIRFLOW__CORE__DAG_IGNORE_FILE_SYNTAX

dag_run_conf_overrides_params

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.

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__DAG_RUN_CONF_OVERRIDES_PARAMS

dagbag_import_error_traceback_depth

New in version 2.0.0.

If tracebacks are shown, how many entries from the traceback should be shown

Type

integer

Default

2

Environment Variable

AIRFLOW__CORE__DAGBAG_IMPORT_ERROR_TRACEBACK_DEPTH

dagbag_import_error_tracebacks

New in version 2.0.0.

Should a traceback be shown in the UI for dagbag import errors, instead of just the exception message

Type

boolean

Default

True

Environment Variable

AIRFLOW__CORE__DAGBAG_IMPORT_ERROR_TRACEBACKS

dagbag_import_timeout

How long before timing out a python file import

Type

float

Default

30.0

Environment Variable

AIRFLOW__CORE__DAGBAG_IMPORT_TIMEOUT

dags_are_paused_at_creation

Are DAGs paused by default at creation

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION

dags_folder

The folder where your airflow pipelines live, most likely a subfolder in a code repository. This path must be absolute.

Type

string

Default

{AIRFLOW_HOME}/dags

Environment Variable

AIRFLOW__CORE__DAGS_FOLDER

default_impersonation

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

Type

string

Default

''

Environment Variable

AIRFLOW__CORE__DEFAULT_IMPERSONATION

default_pool_task_slot_count

New in version 2.2.0.

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

Type

string

Default

128

Environment Variable

AIRFLOW__CORE__DEFAULT_POOL_TASK_SLOT_COUNT

default_task_execution_timeout

New in version 2.3.0.

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.

Type

integer

Default

''

Environment Variable

AIRFLOW__CORE__DEFAULT_TASK_EXECUTION_TIMEOUT

default_task_retries

New in version 1.10.6.

The number of retries each task is going to have by default. Can be overridden at dag or task level.

Type

string

Default

0

Environment Variable

AIRFLOW__CORE__DEFAULT_TASK_RETRIES

default_task_weight_rule

New in version 2.2.0.

The weighting method used for the effective total priority weight of the task

Type

string

Default

downstream

Environment Variable

AIRFLOW__CORE__DEFAULT_TASK_WEIGHT_RULE

default_timezone

Default timezone in case supplied date times are naive can be utc (default), system, or any IANA timezone string (e.g. Europe/Amsterdam)

Type

string

Default

utc

Environment Variable

AIRFLOW__CORE__DEFAULT_TIMEZONE

donot_pickle

Whether to disable pickling dags

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__DONOT_PICKLE

enable_xcom_pickling

Whether to enable pickling for xcom (note that this is insecure and allows for RCE exploits).

Type

string

Default

False

Environment Variable

AIRFLOW__CORE__ENABLE_XCOM_PICKLING

execute_tasks_new_python_interpreter

New in version 2.0.0.

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)

Type

boolean

Default

False

Environment Variable

AIRFLOW__CORE__EXECUTE_TASKS_NEW_PYTHON_INTERPRETER

executor

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.

Type

string

Default

SequentialExecutor

Environment Variable

AIRFLOW__CORE__EXECUTOR

fernet_key

Secret key to save connection passwords in the db

Type

string

Default

{FERNET_KEY}

Environment Variables

AIRFLOW__CORE__FERNET_KEY

AIRFLOW__CORE__FERNET_KEY_CMD

AIRFLOW__CORE__FERNET_KEY_SECRET

hide_sensitive_var_conn_fields

New in version 2.1.0.

Hide sensitive Variables or Connection extra json keys from UI and task logs when set to True

(Connection passwords are always hidden in logs)

Type

boolean

Default

True

Environment Variable

AIRFLOW__CORE__HIDE_SENSITIVE_VAR_CONN_FIELDS

hostname_callable

Hostname by providing a path to a callable, which will resolve the hostname. The format is “package.function”.

For example, default value “socket.getfqdn” means that result from getfqdn() of “socket” package will be used as hostname.

No argument should be required in the function specified. If using IP address as hostname is preferred, use value airflow.utils.net.get_host_ip_address

Type

string

Default

socket.getfqdn

Environment Variable

AIRFLOW__CORE__HOSTNAME_CALLABLE

killed_task_cleanup_time

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

Type

string

Default

60

Environment Variable

AIRFLOW__CORE__KILLED_TASK_CLEANUP_TIME

lazy_discover_providers

New in version 2.0.0.

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.

Type

boolean

Default

True

Environment Variable

AIRFLOW__CORE__LAZY_DISCOVER_PROVIDERS

lazy_load_plugins

New in version 2.0.0.

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.

Type

boolean

Default

True

Environment Variable

AIRFLOW__CORE__LAZY_LOAD_PLUGINS

load_examples

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

Type

string

Default

True

Environment Variable

AIRFLOW__CORE__LOAD_EXAMPLES

max_active_runs_per_dag

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.

Type

string

Default

16

Environment Variable

AIRFLOW__CORE__MAX_ACTIVE_RUNS_PER_DAG

max_active_tasks_per_dag

New in version 2.2.0.

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.

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.

Type

string

Default

16

Environment Variable

AIRFLOW__CORE__MAX_ACTIVE_TASKS_PER_DAG

max_map_length

New in version 2.3.0.

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.

Type

integer

Default

1024

Environment Variable

AIRFLOW__CORE__MAX_MAP_LENGTH

max_num_rendered_ti_fields_per_task

New in version 1.10.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.

Type

integer

Default

30

Environment Variable

AIRFLOW__CORE__MAX_NUM_RENDERED_TI_FIELDS_PER_TASK

min_serialized_dag_fetch_interval

New in version 1.10.12.

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

Type

string

Default

10

Environment Variable

AIRFLOW__CORE__MIN_SERIALIZED_DAG_FETCH_INTERVAL

min_serialized_dag_update_interval

New in version 1.10.7.

Updating serialized DAG can not be faster than a minimum interval to reduce database write rate.

Type

string

Default

30

Environment Variable

AIRFLOW__CORE__MIN_SERIALIZED_DAG_UPDATE_INTERVAL

parallelism

This defines the maximum number of task instances that can run concurrently in Airflow regardless of scheduler count and worker count. Generally, this value is reflective of the number of task instances with the running state in the metadata database.

Type

string

Default

32

Environment Variable

AIRFLOW__CORE__PARALLELISM

plugins_folder

Path to the folder containing Airflow plugins

Type

string

Default

{AIRFLOW_HOME}/plugins

Environment Variable

AIRFLOW__CORE__PLUGINS_FOLDER

security

What security module to use (for example kerberos)

Type

string

Default

''

Environment Variable

AIRFLOW__CORE__SECURITY

sensitive_var_conn_names

New in version 2.1.0.

A comma-separated list of extra sensitive keywords to look for in variables names or connection’s extra JSON.

Type

string

Default

''

Environment Variable

AIRFLOW__CORE__SENSITIVE_VAR_CONN_NAMES

task_runner

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.

Type

string

Default

StandardTaskRunner

Environment Variable

AIRFLOW__CORE__TASK_RUNNER

unit_test_mode

Turn unit test mode on (overwrites many configuration options with test values at runtime)

Type

string

Default

False

Environment Variable

AIRFLOW__CORE__UNIT_TEST_MODE

xcom_backend

New in version 1.10.12.

Path to custom XCom class that will be used to store and resolve operators results

Type

string

Default

airflow.models.xcom.BaseXCom

Environment Variable

AIRFLOW__CORE__XCOM_BACKEND

Example

path.to.CustomXCom

base_log_folder (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.base_log_folder

colored_console_log (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.colored_console_log

colored_formatter_class (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.colored_formatter_class

colored_log_format (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.colored_log_format

dag_concurrency (Deprecated)

Deprecated since version 2.2.0: The option has been moved to core.max_active_tasks_per_dag

dag_processor_manager_log_location (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.dag_processor_manager_log_location

encrypt_s3_logs (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.encrypt_s3_logs

fab_logging_level (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.fab_logging_level

load_default_connections (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.load_default_connections

log_filename_template (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.log_filename_template

log_format (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.log_format

log_processor_filename_template (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.log_processor_filename_template

logging_config_class (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.logging_config_class

logging_level (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.logging_level

max_db_retries (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.max_db_retries

non_pooled_task_slot_count (Deprecated)

Deprecated since version 1.10.4: The option has been moved to core.default_pool_task_slot_count

remote_base_log_folder (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.remote_base_log_folder

remote_log_conn_id (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.remote_log_conn_id

remote_logging (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.remote_logging

simple_log_format (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.simple_log_format

sql_alchemy_conn (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_conn

sql_alchemy_connect_args (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_connect_args

sql_alchemy_max_overflow (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_max_overflow

sql_alchemy_pool_enabled (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_pool_enabled

sql_alchemy_pool_pre_ping (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_pool_pre_ping

sql_alchemy_pool_recycle (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_pool_recycle

sql_alchemy_pool_size (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_pool_size

sql_alchemy_schema (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_alchemy_schema

sql_engine_collation_for_ids (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_engine_collation_for_ids

sql_engine_encoding (Deprecated)

Deprecated since version 2.3.0: The option has been moved to database.sql_engine_encoding

task_log_prefix_template (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.task_log_prefix_template

task_log_reader (Deprecated)

Deprecated since version 2.0.0: The option has been moved to logging.task_log_reader

worker_precheck (Deprecated)

Deprecated since version 2.0.0: The option has been moved to celery.worker_precheck

[dask]

This section only applies if you are using the DaskExecutor in [core] section above

cluster_address

The IP address and port of the Dask cluster’s scheduler.

Type

string

Default

127.0.0.1:8786

Environment Variable

AIRFLOW__DASK__CLUSTER_ADDRESS

tls_ca

TLS/ SSL settings to access a secured Dask scheduler.

Type

string

Default

''

Environment Variable

AIRFLOW__DASK__TLS_CA

tls_cert

Type

string

Default

''

Environment Variable

AIRFLOW__DASK__TLS_CERT

tls_key

Type

string

Default

''

Environment Variable

AIRFLOW__DASK__TLS_KEY

[database]

load_default_connections

New in version 2.3.0.

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

Type

string

Default

True

Environment Variable

AIRFLOW__DATABASE__LOAD_DEFAULT_CONNECTIONS

max_db_retries

New in version 2.3.0.

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.

Type

integer

Default

3

Environment Variable

AIRFLOW__DATABASE__MAX_DB_RETRIES

sql_alchemy_conn

New in version 2.3.0.

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

Type

string

Default

sqlite:///{AIRFLOW_HOME}/airflow.db

Environment Variables

AIRFLOW__DATABASE__SQL_ALCHEMY_CONN

AIRFLOW__DATABASE__SQL_ALCHEMY_CONN_CMD

AIRFLOW__DATABASE__SQL_ALCHEMY_CONN_SECRET

sql_alchemy_connect_args

New in version 2.3.0.

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/13/core/engines.html#sqlalchemy.create_engine.params.connect_args

Type

string

Default

None

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_CONNECT_ARGS

sql_alchemy_engine_args

New in version 2.3.0.

Extra engine specific keyword args passed to SQLAlchemy’s create_engine, as a JSON-encoded value

Type

string

Default

None

Environment Variables

AIRFLOW__DATABASE__SQL_ALCHEMY_ENGINE_ARGS

AIRFLOW__DATABASE__SQL_ALCHEMY_ENGINE_ARGS_CMD

AIRFLOW__DATABASE__SQL_ALCHEMY_ENGINE_ARGS_SECRET

Example

{"arg1": True}

sql_alchemy_max_overflow

New in version 2.3.0.

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.

Type

string

Default

10

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_MAX_OVERFLOW

sql_alchemy_pool_enabled

New in version 2.3.0.

If SqlAlchemy should pool database connections.

Type

string

Default

True

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_POOL_ENABLED

sql_alchemy_pool_pre_ping

New in version 2.3.0.

Check connection at the start of each connection pool checkout. Typically, this is a simple statement like “SELECT 1”. More information here: https://docs.sqlalchemy.org/en/13/core/pooling.html#disconnect-handling-pessimistic

Type

string

Default

True

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_POOL_PRE_PING

sql_alchemy_pool_recycle

New in version 2.3.0.

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.

Type

string

Default

1800

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_POOL_RECYCLE

sql_alchemy_pool_size

New in version 2.3.0.

The SqlAlchemy pool size is the maximum number of database connections in the pool. 0 indicates no limit.

Type

string

Default

5

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_POOL_SIZE

sql_alchemy_schema

New in version 2.3.0.

The schema to use for the metadata database. SqlAlchemy supports databases with the concept of multiple schemas.

Type

string

Default

''

Environment Variable

AIRFLOW__DATABASE__SQL_ALCHEMY_SCHEMA

sql_engine_collation_for_ids

New in version 2.3.0.

Collation for dag_id, task_id, key 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).

Type

string

Default

None

Environment Variable

AIRFLOW__DATABASE__SQL_ENGINE_COLLATION_FOR_IDS

sql_engine_encoding

New in version 2.3.0.

The encoding for the databases

Type

string

Default

utf-8

Environment Variable

AIRFLOW__DATABASE__SQL_ENGINE_ENCODING

[debug]

fail_fast

New in version 1.10.8.

Used only with DebugExecutor. If set to True DAG will fail with first failed task. Helpful for debugging purposes.

Type

string

Default

False

Environment Variable

AIRFLOW__DEBUG__FAIL_FAST

[elasticsearch]

end_of_log_mark

New in version 1.10.4.

Used to mark the end of a log stream for a task

Type

string

Default

end_of_log

Environment Variable

AIRFLOW__ELASTICSEARCH__END_OF_LOG_MARK

frontend

New in version 1.10.4.

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

Type

string

Default

''

Environment Variable

AIRFLOW__ELASTICSEARCH__FRONTEND

Example

http://localhost:5601/app/kibana#/discover?_a=(columns:!(message),query:(language:kuery,query:'log_id: "{log_id}"'),sort:!(log.offset,asc))

host

New in version 1.10.4.

Elasticsearch host

Type

string

Default

''

Environment Variable

AIRFLOW__ELASTICSEARCH__HOST

host_field

New in version 2.1.1.

The field where host name is stored (normally either host or host.name)

Type

string

Default

host

Environment Variable

AIRFLOW__ELASTICSEARCH__HOST_FIELD

json_fields

New in version 1.10.4.

Log fields to also attach to the json output, if enabled

Type

string

Default

asctime, filename, lineno, levelname, message

Environment Variable

AIRFLOW__ELASTICSEARCH__JSON_FIELDS

json_format

New in version 1.10.4.

Instead of the default log formatter, write the log lines as JSON

Type

string

Default

False

Environment Variable

AIRFLOW__ELASTICSEARCH__JSON_FORMAT

log_id_template

New in version 1.10.4.

Format of the log_id, which is used to query for a given tasks logs

Type

string

Default

{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}

Environment Variable

AIRFLOW__ELASTICSEARCH__LOG_ID_TEMPLATE

offset_field

New in version 2.1.1.

The field where offset is stored (normally either offset or log.offset)

Type

string

Default

offset

Environment Variable

AIRFLOW__ELASTICSEARCH__OFFSET_FIELD

write_stdout

New in version 1.10.4.

Write the task logs to the stdout of the worker, rather than the default files

Type

string

Default

False

Environment Variable

AIRFLOW__ELASTICSEARCH__WRITE_STDOUT

[elasticsearch_configs]

use_ssl

New in version 1.10.5.

Type

string

Default

False

Environment Variable

AIRFLOW__ELASTICSEARCH_CONFIGS__USE_SSL

verify_certs

New in version 1.10.5.

Type

string

Default

True

Environment Variable

AIRFLOW__ELASTICSEARCH_CONFIGS__VERIFY_CERTS

[email]

Configuration email backend and whether to send email alerts on retry or failure

default_email_on_failure

New in version 2.0.0.

Whether email alerts should be sent when a task failed

Type

boolean

Default

True

Environment Variable

AIRFLOW__EMAIL__DEFAULT_EMAIL_ON_FAILURE

default_email_on_retry

New in version 2.0.0.

Whether email alerts should be sent when a task is retried

Type

boolean

Default

True

Environment Variable

AIRFLOW__EMAIL__DEFAULT_EMAIL_ON_RETRY

email_backend

Email backend to use

Type

string

Default

airflow.utils.email.send_email_smtp

Environment Variable

AIRFLOW__EMAIL__EMAIL_BACKEND

email_conn_id

New in version 2.1.0.

Email connection to use

Type

string

Default

smtp_default

Environment Variable

AIRFLOW__EMAIL__EMAIL_CONN_ID

from_email

New in version 2.2.4.

Email address that will be used as sender address. It can either be raw email or the complete address in a format Sender Name <sender@email.com>

Type

string

Default

None

Environment Variable

AIRFLOW__EMAIL__FROM_EMAIL

Example

Airflow <airflow@example.com>

html_content_template

New in version 2.0.1.

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.

Type

string

Default

None

Environment Variable

AIRFLOW__EMAIL__HTML_CONTENT_TEMPLATE

Example

/path/to/my_html_content_template_file

subject_template

New in version 2.0.1.

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.

Type

string

Default

None

Environment Variable

AIRFLOW__EMAIL__SUBJECT_TEMPLATE

Example

/path/to/my_subject_template_file

[github_enterprise]

api_rev

Type

string

Default

v3

Environment Variable

AIRFLOW__GITHUB_ENTERPRISE__API_REV

[hive]

default_hive_mapred_queue

Default mapreduce queue for HiveOperator tasks

Type

string

Default

''

Environment Variable

AIRFLOW__HIVE__DEFAULT_HIVE_MAPRED_QUEUE

mapred_job_name_template

New in version 2.0.0.

Template for mapred_job_name in HiveOperator, supports the following named parameters hostname, dag_id, task_id, execution_date

Type

string

Default

None

Environment Variable

AIRFLOW__HIVE__MAPRED_JOB_NAME_TEMPLATE

[kerberos]

ccache

Type

string

Default

/tmp/airflow_krb5_ccache

Environment Variable

AIRFLOW__KERBEROS__CCACHE

forwardable

New in version 2.2.0.

Allow to disable ticket forwardability.

Type

boolean

Default

True

Environment Variable

AIRFLOW__KERBEROS__FORWARDABLE

include_ip

New in version 2.2.0.

Allow to remove source IP from token, useful when using token behind NATted Docker host.

Type

boolean

Default

True

Environment Variable

AIRFLOW__KERBEROS__INCLUDE_IP

keytab

Type

string

Default

airflow.keytab

Environment Variable

AIRFLOW__KERBEROS__KEYTAB

kinit_path

Type

string

Default

kinit

Environment Variable

AIRFLOW__KERBEROS__KINIT_PATH

principal

gets augmented with fqdn

Type

string

Default

airflow

Environment Variable

AIRFLOW__KERBEROS__PRINCIPAL

reinit_frequency

Type

string

Default

3600

Environment Variable

AIRFLOW__KERBEROS__REINIT_FREQUENCY

[kubernetes]

cluster_context

New in version 1.10.3.

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.

Type

string

Default

None

Environment Variable

AIRFLOW__KUBERNETES__CLUSTER_CONTEXT

config_file

New in version 1.10.3.

Path to the kubernetes configfile to be used when in_cluster is set to False

Type

string

Default

None

Environment Variable

AIRFLOW__KUBERNETES__CONFIG_FILE

delete_option_kwargs

New in version 1.10.12.

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

Type

string

Default

''

Environment Variable

AIRFLOW__KUBERNETES__DELETE_OPTION_KWARGS

Example

{"grace_period_seconds": 10}

delete_worker_pods

If True, all worker pods will be deleted upon termination

Type

string

Default

True

Environment Variable

AIRFLOW__KUBERNETES__DELETE_WORKER_PODS

delete_worker_pods_on_failure

New in version 1.10.11.

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.

Type

string

Default

False

Environment Variable

AIRFLOW__KUBERNETES__DELETE_WORKER_PODS_ON_FAILURE

enable_tcp_keepalive

New in version 2.0.0.

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.

Type

boolean

Default

True

Environment Variable

AIRFLOW__KUBERNETES__ENABLE_TCP_KEEPALIVE

in_cluster

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.

Type

string

Default

True

Environment Variable

AIRFLOW__KUBERNETES__IN_CLUSTER

kube_client_request_args

New in version 1.10.4.

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

Type

string

Default

''

Environment Variable

AIRFLOW__KUBERNETES__KUBE_CLIENT_REQUEST_ARGS

multi_namespace_mode

New in version 1.10.12.

Allows users to launch pods in multiple namespaces. Will require creating a cluster-role for the scheduler

Type

boolean

Default

False

Environment Variable

AIRFLOW__KUBERNETES__MULTI_NAMESPACE_MODE

namespace

The Kubernetes namespace where airflow workers should be created. Defaults to default

Type

string

Default

default

Environment Variable

AIRFLOW__KUBERNETES__NAMESPACE

pod_template_file

New in version 1.10.11.

Path to the YAML pod file that forms the basis for KubernetesExecutor workers.

Type

string

Default

''

Environment Variable

AIRFLOW__KUBERNETES__POD_TEMPLATE_FILE

tcp_keep_cnt

New in version 2.0.0.

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.

Type

integer

Default

6

Environment Variable

AIRFLOW__KUBERNETES__TCP_KEEP_CNT

tcp_keep_idle

New in version 2.0.0.

When the enable_tcp_keepalive option is enabled, TCP probes a connection that has been idle for tcp_keep_idle seconds.

Type

integer

Default

120

Environment Variable

AIRFLOW__KUBERNETES__TCP_KEEP_IDLE

tcp_keep_intvl

New in version 2.0.0.

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.

Type

integer

Default

30

Environment Variable

AIRFLOW__KUBERNETES__TCP_KEEP_INTVL

verify_ssl

New in version 2.1.0.

Set this to false to skip verifying SSL certificate of Kubernetes python client.

Type

boolean

Default

True

Environment Variable

AIRFLOW__KUBERNETES__VERIFY_SSL

worker_container_repository

The repository of the Kubernetes Image for the Worker to Run

Type

string

Default

''

Environment Variable

AIRFLOW__KUBERNETES__WORKER_CONTAINER_REPOSITORY

worker_container_tag

The tag of the Kubernetes Image for the Worker to Run

Type

string

Default

''

Environment Variable

AIRFLOW__KUBERNETES__WORKER_CONTAINER_TAG

worker_pods_creation_batch_size

New in version 1.10.3.

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.

Type

string

Default

1

Environment Variable

AIRFLOW__KUBERNETES__WORKER_PODS_CREATION_BATCH_SIZE

worker_pods_pending_timeout

New in version 2.1.0.

How long in seconds a worker can be in Pending before it is considered a failure

Type

integer

Default

300

Environment Variable

AIRFLOW__KUBERNETES__WORKER_PODS_PENDING_TIMEOUT

worker_pods_pending_timeout_batch_size

New in version 2.1.0.

How many pending pods to check for timeout violations in each check interval. You may want this higher if you have a very large cluster and/or use multi_namespace_mode.

Type

integer

Default

100

Environment Variable

AIRFLOW__KUBERNETES__WORKER_PODS_PENDING_TIMEOUT_BATCH_SIZE

worker_pods_pending_timeout_check_interval

New in version 2.1.0.

How often in seconds to check if Pending workers have exceeded their timeouts

Type

integer

Default

120

Environment Variable

AIRFLOW__KUBERNETES__WORKER_PODS_PENDING_TIMEOUT_CHECK_INTERVAL

worker_pods_queued_check_interval

New in version 2.2.0.

How often in seconds to check for task instances stuck in “queued” status without a pod

Type

integer

Default

60

Environment Variable

AIRFLOW__KUBERNETES__WORKER_PODS_QUEUED_CHECK_INTERVAL

[lineage]

backend

what lineage backend to use

Type

string

Default

''

Environment Variable

AIRFLOW__LINEAGE__BACKEND

[local_kubernetes_executor]

This section only applies if you are using the LocalKubernetesExecutor in [core] section above

kubernetes_queue

New in version 2.3.0.

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

Type

string

Default

kubernetes

Environment Variable

AIRFLOW__LOCAL_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE

[logging]

base_log_folder

New in version 2.0.0.

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.

Type

string

Default

{AIRFLOW_HOME}/logs

Environment Variable

AIRFLOW__LOGGING__BASE_LOG_FOLDER

celery_logging_level

New in version 2.3.0.

Logging level for celery. If not set, it uses the value of logging_level

Supported values: CRITICAL, ERROR, WARNING, INFO, DEBUG.

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__CELERY_LOGGING_LEVEL

colored_console_log

New in version 2.0.0.

Flag to enable/disable Colored logs in Console Colour the logs when the controlling terminal is a TTY.

Type

string

Default

True

Environment Variable

AIRFLOW__LOGGING__COLORED_CONSOLE_LOG

colored_formatter_class

New in version 2.0.0.

Type

string

Default

airflow.utils.log.colored_log.CustomTTYColoredFormatter

Environment Variable

AIRFLOW__LOGGING__COLORED_FORMATTER_CLASS

colored_log_format

New in version 2.0.0.

Log format for when Colored logs is enabled

Type

string

Default

[%%(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

Environment Variable

AIRFLOW__LOGGING__COLORED_LOG_FORMAT

dag_processor_manager_log_location

New in version 2.0.0.

Full path of dag_processor_manager logfile.

Type

string

Default

{AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log

Environment Variable

AIRFLOW__LOGGING__DAG_PROCESSOR_MANAGER_LOG_LOCATION

encrypt_s3_logs

New in version 2.0.0.

Use server-side encryption for logs stored in S3

Type

string

Default

False

Environment Variable

AIRFLOW__LOGGING__ENCRYPT_S3_LOGS

extra_logger_names

New in version 2.0.0.

A comma-separated list of third-party logger names that will be configured to print messages to consoles.

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__EXTRA_LOGGER_NAMES

Example

connexion,sqlalchemy

fab_logging_level

New in version 2.0.0.

Logging level for Flask-appbuilder UI.

Supported values: CRITICAL, ERROR, WARNING, INFO, DEBUG.

Type

string

Default

WARNING

Environment Variable

AIRFLOW__LOGGING__FAB_LOGGING_LEVEL

google_key_path

New in version 2.0.0.

Path to Google Credential JSON file. If omitted, authorization based on the Application Default Credentials will be used.

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__GOOGLE_KEY_PATH

log_filename_template

New in version 2.0.0.

Formatting for how airflow generates file names/paths for each task run.

Type

string

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

Environment Variable

AIRFLOW__LOGGING__LOG_FILENAME_TEMPLATE

log_format

New in version 2.0.0.

Format of Log line

Type

string

Default

[%%(asctime)s] {%%(filename)s:%%(lineno)d} %%(levelname)s - %%(message)s

Environment Variable

AIRFLOW__LOGGING__LOG_FORMAT

log_processor_filename_template

New in version 2.0.0.

Formatting for how airflow generates file names for log

Type

string

Default

{{ filename }}.log

Environment Variable

AIRFLOW__LOGGING__LOG_PROCESSOR_FILENAME_TEMPLATE

logging_config_class

New in version 2.0.0.

Logging class Specify the class that will specify the logging configuration This class has to be on the python classpath

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__LOGGING_CONFIG_CLASS

Example

my.path.default_local_settings.LOGGING_CONFIG

logging_level

New in version 2.0.0.

Logging level.

Supported values: CRITICAL, ERROR, WARNING, INFO, DEBUG.

Type

string

Default

INFO

Environment Variable

AIRFLOW__LOGGING__LOGGING_LEVEL

remote_base_log_folder

New in version 2.0.0.

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://”

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__REMOTE_BASE_LOG_FOLDER

remote_log_conn_id

New in version 2.0.0.

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.

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__REMOTE_LOG_CONN_ID

remote_logging

New in version 2.0.0.

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.

Type

string

Default

False

Environment Variable

AIRFLOW__LOGGING__REMOTE_LOGGING

simple_log_format

New in version 2.0.0.

Type

string

Default

%%(asctime)s %%(levelname)s - %%(message)s

Environment Variable

AIRFLOW__LOGGING__SIMPLE_LOG_FORMAT

task_log_prefix_template

New in version 2.0.0.

Specify prefix pattern like mentioned below with stream handler TaskHandlerWithCustomFormatter

Type

string

Default

''

Environment Variable

AIRFLOW__LOGGING__TASK_LOG_PREFIX_TEMPLATE

Example

{ti.dag_id}-{ti.task_id}-{execution_date}-{try_number}

task_log_reader

New in version 2.0.0.

Name of handler to read task instance logs. Defaults to use task handler.

Type

string

Default

task

Environment Variable

AIRFLOW__LOGGING__TASK_LOG_READER

worker_log_server_port

New in version 2.2.0.

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.

Type

string

Default

8793

Environment Variable

AIRFLOW__LOGGING__WORKER_LOG_SERVER_PORT

[metrics]

StatsD (https://github.com/etsy/statsd) integration settings.

stat_name_handler

New in version 2.0.0.

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:

Type

string

Default

''

Environment Variable

AIRFLOW__METRICS__STAT_NAME_HANDLER

statsd_allow_list

New in version 2.0.0.

If you want to avoid sending all the available metrics to StatsD, 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”)

Type

string

Default

''

Environment Variable

AIRFLOW__METRICS__STATSD_ALLOW_LIST

statsd_custom_client_path

New in version 2.0.0.

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

Type

string

Default

None

Environment Variable

AIRFLOW__METRICS__STATSD_CUSTOM_CLIENT_PATH

statsd_datadog_enabled

New in version 2.0.0.

To enable datadog integration to send airflow metrics.

Type

string

Default

False

Environment Variable

AIRFLOW__METRICS__STATSD_DATADOG_ENABLED

statsd_datadog_tags

New in version 2.0.0.

List of datadog tags attached to all metrics(e.g: key1:value1,key2:value2)

Type

string

Default

''

Environment Variable

AIRFLOW__METRICS__STATSD_DATADOG_TAGS

statsd_host

New in version 2.0.0.

Type

string

Default

localhost

Environment Variable

AIRFLOW__METRICS__STATSD_HOST

statsd_on

New in version 2.0.0.

Enables sending metrics to StatsD.

Type

string

Default

False

Environment Variable

AIRFLOW__METRICS__STATSD_ON

statsd_port

New in version 2.0.0.

Type

string

Default

8125

Environment Variable

AIRFLOW__METRICS__STATSD_PORT

statsd_prefix

New in version 2.0.0.

Type

string

Default

airflow

Environment Variable

AIRFLOW__METRICS__STATSD_PREFIX

[operators]

allow_illegal_arguments

New in version 2.0.0.

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.

Type

string

Default

False

Environment Variable

AIRFLOW__OPERATORS__ALLOW_ILLEGAL_ARGUMENTS

default_cpus

Type

string

Default

1

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_CPUS

default_disk

Type

string

Default

512

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_DISK

default_gpus

Type

string

Default

0

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_GPUS

default_owner

The default owner assigned to each new operator, unless provided explicitly or passed via default_args

Type

string

Default

airflow

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_OWNER

default_queue

New in version 2.1.0.

Default queue that tasks get assigned to and that worker listen on.

Type

string

Default

default

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_QUEUE

default_ram

Type

string

Default

512

Environment Variable

AIRFLOW__OPERATORS__DEFAULT_RAM

[scheduler]

allow_trigger_in_future

New in version 1.10.8.

Allow externally triggered DagRuns for Execution Dates in the future Only has effect if schedule_interval is set to None in DAG

Type

string

Default

False

Environment Variable

AIRFLOW__SCHEDULER__ALLOW_TRIGGER_IN_FUTURE

catchup_by_default

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)

Type

string

Default

True

Environment Variable

AIRFLOW__SCHEDULER__CATCHUP_BY_DEFAULT

child_process_log_directory

Type

string

Default

{AIRFLOW_HOME}/logs/scheduler

Environment Variable

AIRFLOW__SCHEDULER__CHILD_PROCESS_LOG_DIRECTORY

dag_dir_list_interval

How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes.

Type

string

Default

300

Environment Variable

AIRFLOW__SCHEDULER__DAG_DIR_LIST_INTERVAL

deactivate_stale_dags_interval

New in version 2.2.5.

How often (in seconds) to check for stale DAGs (DAGs which are no longer present in the expected files) which should be deactivated.

Type

integer

Default

60

Environment Variable

AIRFLOW__SCHEDULER__DEACTIVATE_STALE_DAGS_INTERVAL

dependency_detector

New in version 2.1.0.

DAG dependency detector class to use

Type

string

Default

airflow.serialization.serialized_objects.DependencyDetector

Environment Variable

AIRFLOW__SCHEDULER__DEPENDENCY_DETECTOR

file_parsing_sort_mode

New in version 2.1.0.

One of modified_time, random_seeded_by_host and alphabetical. The scheduler will list and sort the dag files to decide the parsing order.

  • 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

Type

string

Default

modified_time

Environment Variable

AIRFLOW__SCHEDULER__FILE_PARSING_SORT_MODE

ignore_first_depends_on_past_by_default

New in version 2.3.0.

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.

Type

string

Default

True

Environment Variable

AIRFLOW__SCHEDULER__IGNORE_FIRST_DEPENDS_ON_PAST_BY_DEFAULT

job_heartbeat_sec

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

Type

string

Default

5

Environment Variable

AIRFLOW__SCHEDULER__JOB_HEARTBEAT_SEC

max_callbacks_per_loop

New in version 2.3.0.

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.

Type

integer

Default

20

Environment Variable

AIRFLOW__SCHEDULER__MAX_CALLBACKS_PER_LOOP

max_dagruns_per_loop_to_schedule

New in version 2.0.0.

How many DagRuns should a scheduler examine (and lock) when scheduling and queuing tasks.

Type

string

Default

20

Environment Variable

AIRFLOW__SCHEDULER__MAX_DAGRUNS_PER_LOOP_TO_SCHEDULE

max_dagruns_to_create_per_loop

New in version 2.0.0.

Max number of DAGs to create DagRuns for per scheduler loop.

Type

string

Default

10

Environment Variable

AIRFLOW__SCHEDULER__MAX_DAGRUNS_TO_CREATE_PER_LOOP

max_tis_per_query

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)

Type

string

Default

512

Environment Variable

AIRFLOW__SCHEDULER__MAX_TIS_PER_QUERY

min_file_process_interval

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.

Type

string

Default

30

Environment Variable

AIRFLOW__SCHEDULER__MIN_FILE_PROCESS_INTERVAL

num_runs

New in version 1.10.6.

The number of times to try to schedule each DAG file -1 indicates unlimited number

Type

string

Default

-1

Environment Variable

AIRFLOW__SCHEDULER__NUM_RUNS

orphaned_tasks_check_interval

New in version 2.0.0.

How often (in seconds) should the scheduler check for orphaned tasks and SchedulerJobs

Type

float

Default

300.0

Environment Variable

AIRFLOW__SCHEDULER__ORPHANED_TASKS_CHECK_INTERVAL

parsing_processes

New in version 1.10.14.

The scheduler can run multiple processes in parallel to parse dags. This defines how many processes will run.

Type

string

Default

2

Environment Variable

AIRFLOW__SCHEDULER__PARSING_PROCESSES

pool_metrics_interval

New in version 2.0.0.

How often (in seconds) should pool usage stats be sent to StatsD (if statsd_on is enabled)

Type

float

Default

5.0

Environment Variable

AIRFLOW__SCHEDULER__POOL_METRICS_INTERVAL

schedule_after_task_execution

New in version 2.0.0.

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

Type

boolean

Default

True

Environment Variable

AIRFLOW__SCHEDULER__SCHEDULE_AFTER_TASK_EXECUTION

scheduler_health_check_threshold

New in version 1.10.2.

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

Type

string

Default

30

Environment Variable

AIRFLOW__SCHEDULER__SCHEDULER_HEALTH_CHECK_THRESHOLD

scheduler_heartbeat_sec

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

Type

string

Default

5

Environment Variable

AIRFLOW__SCHEDULER__SCHEDULER_HEARTBEAT_SEC

scheduler_idle_sleep_time

New in version 2.2.0.

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.

Type

string

Default

1

Environment Variable

AIRFLOW__SCHEDULER__SCHEDULER_IDLE_SLEEP_TIME

scheduler_zombie_task_threshold

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.

Type

string

Default

300

Environment Variable

AIRFLOW__SCHEDULER__SCHEDULER_ZOMBIE_TASK_THRESHOLD

standalone_dag_processor

New in version 2.3.0.

Whether the dag processor is running as a standalone process or it is a subprocess of a scheduler job.

Type

boolean

Default

False

Environment Variable

AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR

trigger_timeout_check_interval

New in version 2.2.0.

How often to check for expired trigger requests that have not run yet.

Type

string

Default

15

Environment Variable

AIRFLOW__SCHEDULER__TRIGGER_TIMEOUT_CHECK_INTERVAL

use_job_schedule

New in version 1.10.2.

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.

Type

string

Default

True

Environment Variable

AIRFLOW__SCHEDULER__USE_JOB_SCHEDULE

use_row_level_locking

New in version 2.0.0.

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

Type

boolean

Default

True

Environment Variable

AIRFLOW__SCHEDULER__USE_ROW_LEVEL_LOCKING

zombie_detection_interval

New in version 2.3.0.

How often (in seconds) should the scheduler check for zombie tasks.

Type

float

Default

10.0

Environment Variable

AIRFLOW__SCHEDULER__ZOMBIE_DETECTION_INTERVAL

max_threads (Deprecated)

Deprecated since version 1.10.14: The option has been moved to scheduler.parsing_processes

processor_poll_interval (Deprecated)

Deprecated since version 2.2.0: The option has been moved to scheduler.scheduler_idle_sleep_time

stat_name_handler (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.stat_name_handler

statsd_allow_list (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_allow_list

statsd_custom_client_path (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_custom_client_path

statsd_datadog_enabled (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_datadog_enabled

statsd_datadog_tags (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_datadog_tags

statsd_host (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_host

statsd_on (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_on

statsd_port (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_port

statsd_prefix (Deprecated)

Deprecated since version 2.0.0: The option has been moved to metrics.statsd_prefix

[secrets]

backend

New in version 1.10.10.

Full class name of secrets backend to enable (will precede env vars and metastore in search path)

Type

string

Default

''

Environment Variable

AIRFLOW__SECRETS__BACKEND

Example

airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend

backend_kwargs

New in version 1.10.10.

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"}}

Type

string

Default

''

Environment Variable

AIRFLOW__SECRETS__BACKEND_KWARGS

[sensors]

default_timeout

New in version 2.3.0.

Sensor default timeout, 7 days by default (7 * 24 * 60 * 60).

Type

float

Default

604800

Environment Variable

AIRFLOW__SENSORS__DEFAULT_TIMEOUT

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

before_send

New in version 2.2.0.

Dotted path to a before_send function that the sentry SDK should be configured to use.

Type

string

Default

None

Environment Variable

AIRFLOW__SENTRY__BEFORE_SEND

sentry_dsn

New in version 1.10.6.

Type

string

Default

''

Environment Variable

AIRFLOW__SENTRY__SENTRY_DSN

sentry_on

New in version 2.0.0.

Enable error reporting to Sentry

Type

string

Default

false

Environment Variable

AIRFLOW__SENTRY__SENTRY_ON

[smart_sensor]

sensors_enabled

New in version 2.0.0.

comma separated sensor classes support in smart_sensor.

Type

string

Default

NamedHivePartitionSensor

Environment Variable

AIRFLOW__SMART_SENSOR__SENSORS_ENABLED

shard_code_upper_limit

New in version 2.0.0.

shard_code_upper_limit is the upper limit of shard_code value. The shard_code is generated by hashcode % shard_code_upper_limit.

Type

integer

Default

10000

Environment Variable

AIRFLOW__SMART_SENSOR__SHARD_CODE_UPPER_LIMIT

shards

New in version 2.0.0.

The number of running smart sensor processes for each service.

Type

integer

Default

5

Environment Variable

AIRFLOW__SMART_SENSOR__SHARDS

use_smart_sensor

New in version 2.0.0.

When use_smart_sensor is True, Airflow redirects multiple qualified sensor tasks to smart sensor task.

Type

boolean

Default

False

Environment Variable

AIRFLOW__SMART_SENSOR__USE_SMART_SENSOR

[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

Type

string

Default

localhost

Environment Variable

AIRFLOW__SMTP__SMTP_HOST

smtp_mail_from

Type

string

Default

airflow@example.com

Environment Variable

AIRFLOW__SMTP__SMTP_MAIL_FROM

smtp_password

Type

string

Default

None

Environment Variables

AIRFLOW__SMTP__SMTP_PASSWORD

AIRFLOW__SMTP__SMTP_PASSWORD_CMD

AIRFLOW__SMTP__SMTP_PASSWORD_SECRET

Example

airflow

smtp_port

Type

string

Default

25

Environment Variable

AIRFLOW__SMTP__SMTP_PORT

smtp_retry_limit

New in version 2.0.0.

Type

integer

Default

5

Environment Variable

AIRFLOW__SMTP__SMTP_RETRY_LIMIT

smtp_ssl

Type

string

Default

False

Environment Variable

AIRFLOW__SMTP__SMTP_SSL

smtp_starttls

Type

string

Default

True

Environment Variable

AIRFLOW__SMTP__SMTP_STARTTLS

smtp_timeout

New in version 2.0.0.

Type

integer

Default

30

Environment Variable

AIRFLOW__SMTP__SMTP_TIMEOUT

smtp_user

Type

string

Default

None

Environment Variable

AIRFLOW__SMTP__SMTP_USER

Example

airflow

[triggerer]

default_capacity

New in version 2.2.0.

How many triggers a single Triggerer will run at once, by default.

Type

string

Default

1000

Environment Variable

AIRFLOW__TRIGGERER__DEFAULT_CAPACITY

[webserver]

access_logfile

Log files for the gunicorn webserver. ‘-‘ means log to stderr.

Type

string

Default

-

Environment Variable

AIRFLOW__WEBSERVER__ACCESS_LOGFILE

access_logformat

New in version 2.0.0.

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

Type

string

Default

''

Environment Variable

AIRFLOW__WEBSERVER__ACCESS_LOGFORMAT

analytics_id

New in version 1.10.5.

Unique ID of your account in the analytics tool

Type

string

Default

None

Environment Variable

AIRFLOW__WEBSERVER__ANALYTICS_ID

analytics_tool

Send anonymous user activity to your analytics tool choose from google_analytics, segment, or metarouter

Type

string

Default

None

Environment Variable

AIRFLOW__WEBSERVER__ANALYTICS_TOOL

audit_view_excluded_events

New in version 2.3.0.

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.

Type

string

Default

gantt,landing_times,tries,duration,calendar,graph,grid,tree,tree_data

Environment Variable

AIRFLOW__WEBSERVER__AUDIT_VIEW_EXCLUDED_EVENTS

audit_view_included_events

New in version 2.3.0.

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.

Type

string

Default

None

Environment Variable

AIRFLOW__WEBSERVER__AUDIT_VIEW_INCLUDED_EVENTS

Example

dagrun_cleared,failed

auto_refresh_interval

New in version 2.2.0.

How frequently, in seconds, the DAG data will auto-refresh in graph or grid view when auto-refresh is turned on

Type

integer

Default

3

Environment Variable

AIRFLOW__WEBSERVER__AUTO_REFRESH_INTERVAL

base_url

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

Type

string

Default

http://localhost:8080

Environment Variable

AIRFLOW__WEBSERVER__BASE_URL

dag_default_view

Default DAG view. Valid values are: grid, graph, duration, gantt, landing_times

Type

string

Default

grid

Environment Variable

AIRFLOW__WEBSERVER__DAG_DEFAULT_VIEW

dag_orientation

Default DAG orientation. Valid values are: LR (Left->Right), TB (Top->Bottom), RL (Right->Left), BT (Bottom->Top)

Type

string

Default

LR

Environment Variable

AIRFLOW__WEBSERVER__DAG_ORIENTATION

default_dag_run_display_number

Default dagrun to show in UI

Type

string

Default

25

Environment Variable

AIRFLOW__WEBSERVER__DEFAULT_DAG_RUN_DISPLAY_NUMBER

default_ui_timezone

New in version 1.10.10.

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

Type

string

Default

UTC

Environment Variable

AIRFLOW__WEBSERVER__DEFAULT_UI_TIMEZONE

Example

America/New_York

default_wrap

New in version 1.10.4.

Default setting for wrap toggle on DAG code and TI log views.

Type

boolean

Default

False

Environment Variable

AIRFLOW__WEBSERVER__DEFAULT_WRAP

enable_proxy_fix

New in version 1.10.1.

Enable werkzeug ProxyFix middleware for reverse proxy

Type

boolean

Default

False

Environment Variable

AIRFLOW__WEBSERVER__ENABLE_PROXY_FIX

error_logfile

Log files for the gunicorn webserver. ‘-‘ means log to stderr.

Type

string

Default

-

Environment Variable

AIRFLOW__WEBSERVER__ERROR_LOGFILE

expose_config

Expose the configuration file in the web server

Type

string

Default

False

Environment Variable

AIRFLOW__WEBSERVER__EXPOSE_CONFIG

expose_hostname

New in version 1.10.8.

Expose hostname in the web server

Type

string

Default

True

Environment Variable

AIRFLOW__WEBSERVER__EXPOSE_HOSTNAME

expose_stacktrace

New in version 1.10.8.

Expose stacktrace in the web server

Type

string

Default

True

Environment Variable

AIRFLOW__WEBSERVER__EXPOSE_STACKTRACE

hide_paused_dags_by_default

By default, the webserver shows paused DAGs. Flip this to hide paused DAGs by default

Type

string

Default

False

Environment Variable

AIRFLOW__WEBSERVER__HIDE_PAUSED_DAGS_BY_DEFAULT

instance_name

New in version 2.1.0.

Sets a custom page title for the DAGs overview page and site title for all pages

Type

string

Default

None

Environment Variable

AIRFLOW__WEBSERVER__INSTANCE_NAME

instance_name_has_markup

New in version 2.3.0.

Whether the custom page title for the DAGs overview page contains any Markup language

Type

boolean

Default

False

Environment Variable

AIRFLOW__WEBSERVER__INSTANCE_NAME_HAS_MARKUP

log_animation_speed

New in version 1.10.8.

Animation speed for auto tailing log display.

Type

integer

Default

1000

Environment Variable

AIRFLOW__WEBSERVER__LOG_ANIMATION_SPEED

log_auto_tailing_offset

New in version 1.10.8.

Distance away from page bottom to enable auto tailing.

Type

integer

Default

30

Environment Variable

AIRFLOW__WEBSERVER__LOG_AUTO_TAILING_OFFSET

log_fetch_delay_sec

New in version 1.10.8.

Time interval (in secs) to wait before next log fetching.

Type

integer

Default

2

Environment Variable

AIRFLOW__WEBSERVER__LOG_FETCH_DELAY_SEC

log_fetch_timeout_sec

The amount of time (in secs) webserver will wait for initial handshake while fetching logs from other worker machine

Type

string

Default

5

Environment Variable

AIRFLOW__WEBSERVER__LOG_FETCH_TIMEOUT_SEC

page_size

Consistent page size across all listing views in the UI

Type

string

Default

100

Environment Variable

AIRFLOW__WEBSERVER__PAGE_SIZE

proxy_fix_x_for

New in version 1.10.7.

Number of values to trust for X-Forwarded-For. More info: https://werkzeug.palletsprojects.com/en/0.16.x/middleware/proxy_fix/

Type

integer

Default

1

Environment Variable

AIRFLOW__WEBSERVER__PROXY_FIX_X_FOR

proxy_fix_x_host

New in version 1.10.7.

Number of values to trust for X-Forwarded-Host

Type

integer

Default

1

Environment Variable

AIRFLOW__WEBSERVER__PROXY_FIX_X_HOST

proxy_fix_x_port

New in version 1.10.7.

Number of values to trust for X-Forwarded-Port

Type

integer

Default

1

Environment Variable

AIRFLOW__WEBSERVER__PROXY_FIX_X_PORT

proxy_fix_x_prefix

New in version 1.10.7.

Number of values to trust for X-Forwarded-Prefix

Type

integer

Default

1

Environment Variable

AIRFLOW__WEBSERVER__PROXY_FIX_X_PREFIX

proxy_fix_x_proto

New in version 1.10.7.

Number of values to trust for X-Forwarded-Proto

Type

integer

Default

1

Environment Variable

AIRFLOW__WEBSERVER__PROXY_FIX_X_PROTO

reload_on_plugin_change

New in version 1.10.11.

If set to True, Airflow will track files in plugins_folder directory. When it detects changes, then reload the gunicorn.

Type

boolean

Default

False

Environment Variable

AIRFLOW__WEBSERVER__RELOAD_ON_PLUGIN_CHANGE

secret_key

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.

Type

string

Default

{SECRET_KEY}

Environment Variables

AIRFLOW__WEBSERVER__SECRET_KEY

AIRFLOW__WEBSERVER__SECRET_KEY_CMD

AIRFLOW__WEBSERVER__SECRET_KEY_SECRET

session_backend

New in version 2.2.4.

The type of backend used to store web session data, can be ‘database’ or ‘securecookie’

Type

string

Default

database

Environment Variable

AIRFLOW__WEBSERVER__SESSION_BACKEND

Example

securecookie

session_lifetime_minutes

New in version 1.10.13.

The UI cookie lifetime in minutes. User will be logged out from UI after session_lifetime_minutes of non-activity

Type

integer

Default

43200

Environment Variable

AIRFLOW__WEBSERVER__SESSION_LIFETIME_MINUTES

show_recent_stats_for_completed_runs

New in version 2.0.0.

‘Recent Tasks’ stats will show for old DagRuns if set

Type

boolean

Default

True

Environment Variable

AIRFLOW__WEBSERVER__SHOW_RECENT_STATS_FOR_COMPLETED_RUNS

update_fab_perms

New in version 1.10.7.

Update FAB permissions and sync security manager roles on webserver startup

Type

string

Default

True

Environment Variable

AIRFLOW__WEBSERVER__UPDATE_FAB_PERMS

warn_deployment_exposure

New in version 2.3.0.

Boolean for displaying warning for publicly viewable deployment

Type

boolean

Default

True

Environment Variable

AIRFLOW__WEBSERVER__WARN_DEPLOYMENT_EXPOSURE

web_server_host

The ip specified when starting the web server

Type

string

Default

0.0.0.0

Environment Variable

AIRFLOW__WEBSERVER__WEB_SERVER_HOST

web_server_master_timeout

Number of seconds the webserver waits before killing gunicorn master that doesn’t respond

Type

string

Default

120

Environment Variable

AIRFLOW__WEBSERVER__WEB_SERVER_MASTER_TIMEOUT

web_server_port

The port on which to run the web server

Type

string

Default

8080

Environment Variable

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

Type

string

Default

''

Environment Variable

AIRFLOW__WEBSERVER__WEB_SERVER_SSL_CERT

web_server_ssl_key

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.

Type

string

Default

''

Environment Variable

AIRFLOW__WEBSERVER__WEB_SERVER_SSL_KEY

web_server_worker_timeout

Number of seconds the gunicorn webserver waits before timing out on a worker

Type

string

Default

120

Environment Variable

AIRFLOW__WEBSERVER__WEB_SERVER_WORKER_TIMEOUT

worker_class

The worker class gunicorn should use. Choices include sync (default), eventlet, gevent

Type

string

Default

sync

Environment Variable

AIRFLOW__WEBSERVER__WORKER_CLASS

worker_refresh_batch_size

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.

Type

string

Default

1

Environment Variable

AIRFLOW__WEBSERVER__WORKER_REFRESH_BATCH_SIZE

worker_refresh_interval

Number of seconds to wait before refreshing a batch of workers.

Type

string

Default

6000

Environment Variable

AIRFLOW__WEBSERVER__WORKER_REFRESH_INTERVAL

workers

Number of workers to run the Gunicorn web server

Type

string

Default

4

Environment Variable

AIRFLOW__WEBSERVER__WORKERS

x_frame_enabled

New in version 1.10.8.

Allow the UI to be rendered in a frame

Type

boolean

Default

True

Environment Variable

AIRFLOW__WEBSERVER__X_FRAME_ENABLED

Was this entry helpful?