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.
Note
For more information on setting the configuration, see Setting Configuration Options
Sections:
[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_backend¶
How to authenticate users of the API. See https://airflow.apache.org/docs/apache-airflow/stable/security.html for possible values. (“airflow.api.auth.backend.default” allows all requests for historic reasons)
- Type
string
- Default
airflow.api.auth.backend.deny_all
- Environment Variable
AIRFLOW__API__AUTH_BACKEND
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 UPDATING.md
- 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
[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
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 are cleared by CeleryExecutor. This is helpful to clear stalled 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_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
None
- Environment Variable
AIRFLOW__CELERY__WORKER_PREFETCH_MULTIPLIER
- Example
1
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
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_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_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)
See also
- 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_default_connections¶
New in version 1.10.10.
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__CORE__LOAD_DEFAULT_CONNECTIONS
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_db_retries¶
New in version 2.0.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__CORE__MAX_DB_RETRIES
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
sql_alchemy_conn¶
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__CORE__SQL_ALCHEMY_CONN
AIRFLOW__CORE__SQL_ALCHEMY_CONN_CMD
AIRFLOW__CORE__SQL_ALCHEMY_CONN_SECRET
sql_alchemy_connect_args¶
New in version 1.10.11.
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__CORE__SQL_ALCHEMY_CONNECT_ARGS
sql_alchemy_max_overflow¶
New in version 1.10.4.
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__CORE__SQL_ALCHEMY_MAX_OVERFLOW
sql_alchemy_pool_enabled¶
If SqlAlchemy should pool database connections.
- Type
string
- Default
True
- Environment Variable
AIRFLOW__CORE__SQL_ALCHEMY_POOL_ENABLED
sql_alchemy_pool_pre_ping¶
New in version 1.10.6.
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__CORE__SQL_ALCHEMY_POOL_PRE_PING
sql_alchemy_pool_recycle¶
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__CORE__SQL_ALCHEMY_POOL_RECYCLE
sql_alchemy_pool_size¶
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__CORE__SQL_ALCHEMY_POOL_SIZE
sql_alchemy_schema¶
New in version 1.10.3.
The schema to use for the metadata database. SqlAlchemy supports databases with the concept of multiple schemas.
- Type
string
- Default
''
- Environment Variable
AIRFLOW__CORE__SQL_ALCHEMY_SCHEMA
sql_engine_collation_for_ids¶
New in version 2.0.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__CORE__SQL_ENGINE_COLLATION_FOR_IDS
sql_engine_encoding¶
New in version 1.10.1.
The encoding for the databases
- Type
string
- Default
utf-8
- Environment Variable
AIRFLOW__CORE__SQL_ENGINE_ENCODING
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
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
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
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
[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}-{execution_date}-{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.
See also
- 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.
See also
- 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¶
Allow to disable ticket forwardability.
- Type
boolean
- Default
True
- Environment Variable
AIRFLOW__KERBEROS__FORWARDABLE
include_ip¶
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.
See also
- 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
[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
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
{{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ 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.
- 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
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
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_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.
See also
- 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.
See also
- 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
print_stats_interval¶
How often should stats be printed to the logs. Setting to 0 will disable printing stats
- Type
string
- Default
30
- Environment Variable
AIRFLOW__SCHEDULER__PRINT_STATS_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
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
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
[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
auto_refresh_interval¶
New in version 2.2.0.
How frequently, in seconds, the DAG data will auto-refresh in graph or tree 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
cookie_samesite¶
New in version 1.10.3.
Set samesite policy on session cookie
- Type
string
- Default
Lax
- Environment Variable
AIRFLOW__WEBSERVER__COOKIE_SAMESITE
cookie_secure¶
New in version 1.10.3.
Set secure flag on session cookie
- Type
string
- Default
False
- Environment Variable
AIRFLOW__WEBSERVER__COOKIE_SECURE
dag_default_view¶
Default DAG view. Valid values are: tree
, graph
, duration
, gantt
, landing_times
- Type
string
- Default
tree
- 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
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”.
- 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
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