blob: ea2bb03fa348a0b3bfcca8b883e75a9052b3481c [file] [log] [blame]
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
---
core:
description: ~
options:
dags_folder:
description: |
The folder where your airflow pipelines live, most likely a
subfolder in a code repository. This path must be absolute.
version_added: ~
type: string
example: ~
default: "{AIRFLOW_HOME}/dags"
hostname_callable:
description: |
Hostname by providing a path to a callable, which will resolve the hostname.
The format is "package.function".
For example, default value "airflow.utils.net.getfqdn" means that result from patched
version of socket.getfqdn() - see https://github.com/python/cpython/issues/49254.
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``
version_added: ~
type: string
example: ~
default: "airflow.utils.net.getfqdn"
might_contain_dag_callable:
description: |
A callable to check if a python file has airflow dags defined or not
with argument as: `(file_path: str, zip_file: zipfile.ZipFile | None = None)`
return True if it has dags otherwise False
If this is not provided, Airflow uses its own heuristic rules.
version_added: 2.6.0
type: string
example: ~
default: "airflow.utils.file.might_contain_dag_via_default_heuristic"
default_timezone:
description: |
Default timezone in case supplied date times are naive
can be utc (default), system, or any IANA timezone string (e.g. Europe/Amsterdam)
version_added: ~
type: string
example: ~
default: "utc"
executor:
description: |
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.
version_added: ~
type: string
example: ~
default: "SequentialExecutor"
parallelism:
description: |
This defines the maximum number of task instances that can run concurrently per scheduler in
Airflow, regardless of the worker count. Generally this value, multiplied by the number of
schedulers in your cluster, is the maximum number of task instances with the running
state in the metadata database.
version_added: ~
type: string
example: ~
default: "32"
max_active_tasks_per_dag:
description: |
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.
version_added: 2.2.0
type: string
example: ~
default: "16"
dags_are_paused_at_creation:
description: |
Are DAGs paused by default at creation
version_added: ~
type: string
example: ~
default: "True"
max_active_runs_per_dag:
description: |
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``.
version_added: ~
type: string
example: ~
default: "16"
mp_start_method:
description: |
The name of the method used in order to start Python processes via the multiprocessing module.
This corresponds directly with the options available in the Python docs:
https://docs.python.org/3/library/multiprocessing.html#multiprocessing.set_start_method.
Must be one of the values returned by:
https://docs.python.org/3/library/multiprocessing.html#multiprocessing.get_all_start_methods.
version_added: "2.0.0"
type: string
default: ~
example: "fork"
load_examples:
description: |
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
version_added: ~
type: string
example: ~
default: "True"
plugins_folder:
description: |
Path to the folder containing Airflow plugins
version_added: ~
type: string
example: ~
default: "{AIRFLOW_HOME}/plugins"
execute_tasks_new_python_interpreter:
description: |
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)
default: "False"
example: ~
version_added: 2.0.0
see_also: ":ref:`plugins:loading`"
type: boolean
fernet_key:
description: |
Secret key to save connection passwords in the db
version_added: ~
type: string
sensitive: true
example: ~
default: "{FERNET_KEY}"
donot_pickle:
description: |
Whether to disable pickling dags
version_added: ~
type: string
example: ~
default: "True"
dagbag_import_timeout:
description: |
How long before timing out a python file import
version_added: ~
type: float
example: ~
default: "30.0"
dagbag_import_error_tracebacks:
description: |
Should a traceback be shown in the UI for dagbag import errors,
instead of just the exception message
version_added: 2.0.0
type: boolean
example: ~
default: "True"
dagbag_import_error_traceback_depth:
description: |
If tracebacks are shown, how many entries from the traceback should be shown
version_added: 2.0.0
type: integer
example: ~
default: "2"
dag_file_processor_timeout:
description: |
How long before timing out a DagFileProcessor, which processes a dag file
version_added: 1.10.6
type: string
example: ~
default: "50"
task_runner:
description: |
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.
version_added: ~
type: string
example: ~
default: "StandardTaskRunner"
default_impersonation:
description: |
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
version_added: ~
type: string
example: ~
default: ""
security:
description: |
What security module to use (for example kerberos)
version_added: ~
type: string
example: ~
default: ""
unit_test_mode:
description: |
Turn unit test mode on (overwrites many configuration options with test
values at runtime)
version_added: ~
type: string
example: ~
default: "False"
enable_xcom_pickling:
description: |
Whether to enable pickling for xcom (note that this is insecure and allows for
RCE exploits).
version_added: ~
type: string
example: ~
default: "False"
see_also: "https://docs.python.org/3/library/pickle.html#comparison-with-json"
allowed_deserialization_classes:
description: |
What classes can be imported during deserialization. This is a multi line value.
The individual items will be parsed as regexp. Python built-in classes (like dict)
are always allowed. Bare "." will be replaced so you can set airflow.* .
version_added: 2.5.0
type: string
default: 'airflow\..*'
example: ~
killed_task_cleanup_time:
description: |
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
version_added: ~
type: string
example: ~
default: "60"
dag_run_conf_overrides_params:
description: |
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.
version_added: ~
type: string
example: ~
default: "True"
dag_discovery_safe_mode:
description: |
If enabled, Airflow will only scan files containing both ``DAG`` and ``airflow`` (case-insensitive).
version_added: 1.10.3
type: string
example: ~
default: "True"
dag_ignore_file_syntax:
description: |
The pattern syntax used in the ".airflowignore" files in the DAG directories. Valid values are
``regexp`` or ``glob``.
version_added: 2.3.0
type: string
example: ~
default: "regexp"
default_task_retries:
description: |
The number of retries each task is going to have by default. Can be overridden at dag or task level.
version_added: 1.10.6
type: string
example: ~
default: "0"
default_task_retry_delay:
description: |
The number of seconds each task is going to wait by default between retries. Can be overridden at
dag or task level.
version_added: 2.4.0
type: integer
example: ~
default: "300"
max_task_retry_delay:
description: |
The maximum delay (in seconds) each task is going to wait by default between retries.
This is a global setting and cannot be overridden at task or DAG level.
version_added: 2.6.0
type: integer
default: "86400"
example: ~
default_task_weight_rule:
description: |
The weighting method used for the effective total priority weight of the task
version_added: 2.2.0
type: string
example: ~
default: "downstream"
default_task_execution_timeout:
description: |
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.
version_added: 2.3.0
type: integer
example: ~
default: ""
min_serialized_dag_update_interval:
description: |
Updating serialized DAG can not be faster than a minimum interval to reduce database write rate.
version_added: 1.10.7
type: string
example: ~
default: "30"
compress_serialized_dags:
description: |
If True, serialized DAGs are compressed before writing to DB.
Note: this will disable the DAG dependencies view
version_added: 2.3.0
type: string
example: ~
default: "False"
min_serialized_dag_fetch_interval:
description: |
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
version_added: 1.10.12
type: string
example: ~
default: "10"
max_num_rendered_ti_fields_per_task:
description: |
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.
version_added: 1.10.10
type: integer
example: ~
default: "30"
check_slas:
description: |
On each dagrun check against defined SLAs
version_added: 1.10.8
type: string
example: ~
default: "True"
xcom_backend:
description: |
Path to custom XCom class that will be used to store and resolve operators results
version_added: 1.10.12
type: string
example: "path.to.CustomXCom"
default: "airflow.models.xcom.BaseXCom"
lazy_load_plugins:
description: |
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.
version_added: 2.0.0
type: boolean
example: ~
default: "True"
lazy_discover_providers:
description: |
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.
version_added: 2.0.0
type: boolean
example: ~
default: "True"
hide_sensitive_var_conn_fields:
description: |
Hide sensitive Variables or Connection extra json keys from UI and task logs when set to True
(Connection passwords are always hidden in logs)
version_added: 2.1.0
type: boolean
example: ~
default: "True"
sensitive_var_conn_names:
description: |
A comma-separated list of extra sensitive keywords to look for in variables names or connection's
extra JSON.
version_added: 2.1.0
type: string
example: ~
default: ""
default_pool_task_slot_count:
description: |
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
version_added: 2.2.0
type: string
example: ~
default: "128"
max_map_length:
description: |
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.
version_added: 2.3.0
type: integer
example: ~
default: "1024"
daemon_umask:
description: |
The default umask to use for process when run in daemon mode (scheduler, worker, etc.)
This controls the file-creation mode mask which determines the initial value of file permission bits
for newly created files.
This value is treated as an octal-integer.
version_added: 2.3.4
type: string
default: "0o077"
example: ~
dataset_manager_class:
description: Class to use as dataset manager.
version_added: 2.4.0
type: string
default: ~
example: 'airflow.datasets.manager.DatasetManager'
dataset_manager_kwargs:
description: Kwargs to supply to dataset manager.
version_added: 2.4.0
type: string
sensitive: true
default: ~
example: '{"some_param": "some_value"}'
database_access_isolation:
description: (experimental) Whether components should use Airflow Internal API for DB connectivity.
version_added: 2.7.0
type: boolean
example: ~
default: "False"
internal_api_url:
description: |
(experimental) Airflow Internal API url. Only used if [core] database_access_isolation is True.
version_added: 2.7.0
type: string
default: ~
example: 'http://localhost:8080'
database:
description: ~
options:
alembic_ini_file_path:
description: |
Path to the ``alembic.ini`` file. You can either provide the file path relative
to the Airflow home directory or the absolute path if it is located elsewhere.
version_added: 2.7.0
type: string
example: ~
default: "alembic.ini"
sql_alchemy_conn:
description: |
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
version_added: 2.3.0
type: string
sensitive: true
example: ~
default: "sqlite:///{AIRFLOW_HOME}/airflow.db"
sql_alchemy_engine_args:
description: |
Extra engine specific keyword args passed to SQLAlchemy's create_engine, as a JSON-encoded value
version_added: 2.3.0
type: string
sensitive: true
example: '{"arg1": True}'
default: ~
sql_engine_encoding:
description: |
The encoding for the databases
version_added: 2.3.0
type: string
example: ~
default: "utf-8"
sql_engine_collation_for_ids:
description: |
Collation for ``dag_id``, ``task_id``, ``key``, ``external_executor_id`` columns
in case they have different encoding.
By default this collation is the same as the database collation, however for ``mysql`` and ``mariadb``
the default is ``utf8mb3_bin`` so that the index sizes of our index keys will not exceed
the maximum size of allowed index when collation is set to ``utf8mb4`` variant
(see https://github.com/apache/airflow/pull/17603#issuecomment-901121618).
version_added: 2.3.0
type: string
example: ~
default: ~
sql_alchemy_pool_enabled:
description: |
If SqlAlchemy should pool database connections.
version_added: 2.3.0
type: string
example: ~
default: "True"
sql_alchemy_pool_size:
description: |
The SqlAlchemy pool size is the maximum number of database connections
in the pool. 0 indicates no limit.
version_added: 2.3.0
type: string
example: ~
default: "5"
sql_alchemy_max_overflow:
description: |
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``.
version_added: 2.3.0
type: string
example: ~
default: "10"
sql_alchemy_pool_recycle:
description: |
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.
version_added: 2.3.0
type: string
example: ~
default: "1800"
sql_alchemy_pool_pre_ping:
description: |
Check connection at the start of each connection pool checkout.
Typically, this is a simple statement like "SELECT 1".
More information here:
https://docs.sqlalchemy.org/en/14/core/pooling.html#disconnect-handling-pessimistic
version_added: 2.3.0
type: string
example: ~
default: "True"
sql_alchemy_schema:
description: |
The schema to use for the metadata database.
SqlAlchemy supports databases with the concept of multiple schemas.
version_added: 2.3.0
type: string
example: ~
default: ""
sql_alchemy_connect_args:
description: |
Import path for connect args in SqlAlchemy. Defaults to an empty dict.
This is useful when you want to configure db engine args that SqlAlchemy won't parse
in connection string.
See https://docs.sqlalchemy.org/en/14/core/engines.html#sqlalchemy.create_engine.params.connect_args
version_added: 2.3.0
type: string
example: '{"timeout": 30}'
default: ~
load_default_connections:
description: |
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
version_added: 2.3.0
type: string
example: ~
default: "True"
max_db_retries:
description: |
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``.
version_added: 2.3.0
type: integer
example: ~
default: "3"
check_migrations:
description: |
Whether to run alembic migrations during Airflow start up. Sometimes this operation can be expensive,
and the users can assert the correct version through other means (e.g. through a Helm chart).
Accepts "True" or "False".
version_added: 2.6.0
type: string
example: ~
default: "True"
logging:
description: ~
options:
base_log_folder:
description: |
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.
version_added: 2.0.0
type: string
example: ~
default: "{AIRFLOW_HOME}/logs"
remote_logging:
description: |
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.
version_added: 2.0.0
type: string
example: ~
default: "False"
remote_log_conn_id:
description: |
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.
version_added: 2.0.0
type: string
example: ~
default: ""
delete_local_logs:
description: |
Whether the local log files for GCS, S3, WASB and OSS remote logging should be deleted after
they are uploaded to the remote location.
version_added: 2.6.0
type: string
example: ~
default: "False"
google_key_path:
description: |
Path to Google Credential JSON file. If omitted, authorization based on `the Application Default
Credentials
<https://cloud.google.com/docs/authentication/production#finding_credentials_automatically>`__ will
be used.
version_added: 2.0.0
type: string
example: ~
default: ""
remote_base_log_folder:
description: |
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://"
version_added: 2.0.0
type: string
example: ~
default: ""
remote_task_handler_kwargs:
description: |
The remote_task_handler_kwargs param is loaded into a dictionary and passed to __init__ of remote
task handler and it overrides the values provided by Airflow config. For example if you set
`delete_local_logs=False` and you provide ``{{"delete_local_copy": true}}``, then the local
log files will be deleted after they are uploaded to remote location.
version_added: 2.6.0
type: string
sensitive: true
example: '{"delete_local_copy": true}'
default: ""
encrypt_s3_logs:
description: |
Use server-side encryption for logs stored in S3
version_added: 2.0.0
type: string
example: ~
default: "False"
logging_level:
description: |
Logging level.
Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``.
version_added: 2.0.0
type: string
example: ~
default: "INFO"
celery_logging_level:
description: |
Logging level for celery. If not set, it uses the value of logging_level
Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``.
version_added: 2.3.0
type: string
example: ~
default: ""
fab_logging_level:
description: |
Logging level for Flask-appbuilder UI.
Supported values: ``CRITICAL``, ``ERROR``, ``WARNING``, ``INFO``, ``DEBUG``.
version_added: 2.0.0
type: string
example: ~
default: "WARNING"
logging_config_class:
description: |
Logging class
Specify the class that will specify the logging configuration
This class has to be on the python classpath
version_added: 2.0.0
type: string
example: "my.path.default_local_settings.LOGGING_CONFIG"
default: ""
colored_console_log:
description: |
Flag to enable/disable Colored logs in Console
Colour the logs when the controlling terminal is a TTY.
version_added: 2.0.0
type: string
example: ~
default: "True"
colored_log_format:
description: |
Log format for when Colored logs is enabled
version_added: 2.0.0
type: string
example: ~
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
colored_formatter_class:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "airflow.utils.log.colored_log.CustomTTYColoredFormatter"
log_format:
description: |
Format of Log line
version_added: 2.0.0
type: string
example: ~
default: "[%%(asctime)s] {{%%(filename)s:%%(lineno)d}} %%(levelname)s - %%(message)s"
simple_log_format:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "%%(asctime)s %%(levelname)s - %%(message)s"
dag_processor_log_target:
description: Where to send dag parser logs. If "file",
logs are sent to log files defined by child_process_log_directory.
version_added: 2.4.0
type: string
example: ~
default: "file"
dag_processor_log_format:
description: |
Format of Dag Processor Log line
version_added: 2.4.0
type: string
example: ~
default: "[%%(asctime)s] [SOURCE:DAG_PROCESSOR]
{{%%(filename)s:%%(lineno)d}} %%(levelname)s - %%(message)s"
log_formatter_class:
description: ~
version_added: 2.3.4
type: string
example: ~
default: "airflow.utils.log.timezone_aware.TimezoneAware"
secret_mask_adapter:
description: |
An import path to a function to add adaptations of each secret added with
`airflow.utils.log.secrets_masker.mask_secret` to be masked in log messages. The given function
is expected to require a single parameter: the secret to be adapted. It may return a
single adaptation of the secret or an iterable of adaptations to each be masked as secrets.
The original secret will be masked as well as any adaptations returned.
version_added: 2.6.0
type: string
default: ""
example: "urllib.parse.quote"
task_log_prefix_template:
description: |
Specify prefix pattern like mentioned below with stream handler TaskHandlerWithCustomFormatter
version_added: 2.0.0
type: string
example: "{{ti.dag_id}}-{{ti.task_id}}-{{execution_date}}-{{try_number}}"
default: ""
log_filename_template:
description: |
Formatting for how airflow generates file names/paths for each task run.
version_added: 2.0.0
type: string
example: ~
default: "dag_id={{{{ ti.dag_id }}}}/run_id={{{{ ti.run_id }}}}/task_id={{{{ ti.task_id }}}}/\
{{%% if ti.map_index >= 0 %%}}map_index={{{{ ti.map_index }}}}/{{%% endif %%}}\
attempt={{{{ try_number }}}}.log"
log_processor_filename_template:
description: |
Formatting for how airflow generates file names for log
version_added: 2.0.0
type: string
example: ~
default: "{{{{ filename }}}}.log"
dag_processor_manager_log_location:
description: |
Full path of dag_processor_manager logfile.
version_added: 2.0.0
type: string
example: ~
default: "{AIRFLOW_HOME}/logs/dag_processor_manager/dag_processor_manager.log"
task_log_reader:
description: |
Name of handler to read task instance logs.
Defaults to use ``task`` handler.
version_added: 2.0.0
type: string
example: ~
default: "task"
extra_logger_names:
description: |
A comma\-separated list of third-party logger names that will be configured to print messages to
consoles\.
version_added: 2.0.0
type: string
example: "connexion,sqlalchemy"
default: ""
worker_log_server_port:
description: |
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.
version_added: 2.2.0
type: string
example: ~
default: "8793"
trigger_log_server_port:
description: |
Port to serve logs from for triggerer. See worker_log_server_port description
for more info.
version_added: 2.6.0
type: string
example: ~
default: "8794"
interleave_timestamp_parser:
description: |
We must parse timestamps to interleave logs between trigger and task. To do so,
we need to parse timestamps in log files. In case your log format is non-standard,
you may provide import path to callable which takes a string log line and returns
the timestamp (datetime.datetime compatible).
version_added: 2.6.0
type: string
example: path.to.my_func
default: ~
file_task_handler_new_folder_permissions:
description: |
Permissions in the form or of octal string as understood by chmod. The permissions are important
when you use impersonation, when logs are written by a different user than airflow. The most secure
way of configuring it in this case is to add both users to the same group and make it the default
group of both users. Group-writeable logs are default in airflow, but you might decide that you are
OK with having the logs other-writeable, in which case you should set it to `0o777`. You might
decide to add more security if you do not use impersonation and change it to `0o755` to make it
only owner-writeable. You can also make it just readable only for owner by changing it to `0o700` if
all the access (read/write) for your logs happens from the same user.
version_added: 2.6.0
type: string
example: "0o775"
default: "0o775"
file_task_handler_new_file_permissions:
description: |
Permissions in the form or of octal string as understood by chmod. The permissions are important
when you use impersonation, when logs are written by a different user than airflow. The most secure
way of configuring it in this case is to add both users to the same group and make it the default
group of both users. Group-writeable logs are default in airflow, but you might decide that you are
OK with having the logs other-writeable, in which case you should set it to `0o666`. You might
decide to add more security if you do not use impersonation and change it to `0o644` to make it
only owner-writeable. You can also make it just readable only for owner by changing it to `0o600` if
all the access (read/write) for your logs happens from the same user.
version_added: 2.6.0
type: string
example: "0o664"
default: "0o664"
celery_stdout_stderr_separation:
description: |
By default Celery sends all logs into stderr.
If enabled any previous logging handlers will get *removed*.
With this option AirFlow will create new handlers
and send low level logs like INFO and WARNING to stdout,
while sending higher severity logs to stderr.
version_added: 2.5.4
type: boolean
example: ~
default: "False"
metrics:
description: |
StatsD (https://github.com/etsy/statsd) integration settings.
options:
metrics_allow_list:
description: |
If you want to avoid emitting all the available metrics, you can configure an
allow list of prefixes (comma separated) to send only the metrics that start
with the elements of the list (e.g: "scheduler,executor,dagrun")
version_added: 2.6.0
type: string
example: ~
default: ""
metrics_block_list:
description: |
If you want to avoid emitting all the available metrics, you can configure a
block list of prefixes (comma separated) to filter out metrics that start with
the elements of the list (e.g: "scheduler,executor,dagrun").
If metrics_allow_list and metrics_block_list are both configured, metrics_block_list is ignored.
version_added: 2.6.0
type: string
example: ~
default: ""
statsd_on:
description: |
Enables sending metrics to StatsD.
version_added: 2.0.0
type: string
example: ~
default: "False"
statsd_host:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "localhost"
statsd_port:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "8125"
statsd_prefix:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "airflow"
stat_name_handler:
description: |
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:
version_added: 2.0.0
type: string
example: ~
default: ""
statsd_datadog_enabled:
description: |
To enable datadog integration to send airflow metrics.
version_added: 2.0.0
type: string
example: ~
default: "False"
statsd_datadog_tags:
description: |
List of datadog tags attached to all metrics(e.g: key1:value1,key2:value2)
version_added: 2.0.0
type: string
example: ~
default: ""
statsd_datadog_metrics_tags:
description: |
Set to False to disable metadata tags for some of the emitted metrics
version_added: 2.6.0
type: boolean
example: ~
default: "True"
statsd_custom_client_path:
description: |
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
version_added: 2.0.0
type: string
example: ~
default: ~
statsd_disabled_tags:
description: |
If you want to avoid sending all the available metrics tags to StatsD,
you can configure a block list of prefixes (comma separated) to filter out metric tags
that start with the elements of the list (e.g: "job_id,run_id")
version_added: 2.6.0
type: string
example: job_id,run_id,dag_id,task_id
default: job_id,run_id
statsd_influxdb_enabled:
description: |
To enable sending Airflow metrics with StatsD-Influxdb tagging convention.
version_added: 2.6.0
type: boolean
example: ~
default: "False"
otel_on:
description: |
Enables sending metrics to OpenTelemetry.
version_added: 2.5.1
type: string
example: ~
default: "False"
otel_host:
description: ~
version_added: 2.5.1
type: string
example: ~
default: "localhost"
otel_port:
description: ~
version_added: 2.5.1
type: string
example: ~
default: "8889"
otel_prefix:
description: ~
version_added: 2.0.0
type: string
example: ~
default: "airflow"
otel_interval_milliseconds:
description: ~
version_added: 2.0.0
type: integer
example: ~
default: "60000"
otel_debugging_on:
description: |
If True, all metrics are also emitted to the console. Defaults to False.
version_added: 2.7.0
type: string
example: ~
default: "False"
secrets:
description: ~
options:
backend:
description: |
Full class name of secrets backend to enable (will precede env vars and metastore in search path)
version_added: 1.10.10
type: string
example: "airflow.providers.amazon.aws.secrets.systems_manager.SystemsManagerParameterStoreBackend"
default: ""
backend_kwargs:
description: |
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"}}``
version_added: 1.10.10
type: string
sensitive: true
example: ~
default: ""
cli:
description: ~
options:
api_client:
description: |
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
version_added: ~
type: string
example: ~
default: "airflow.api.client.local_client"
endpoint_url:
description: |
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/...``
version_added: ~
type: string
example: ~
default: "http://localhost:8080"
debug:
description: ~
options:
fail_fast:
description: |
Used only with ``DebugExecutor``. If set to ``True`` DAG will fail with first
failed task. Helpful for debugging purposes.
version_added: 1.10.8
type: string
example: ~
default: "False"
api:
description: ~
options:
enable_experimental_api:
description: |
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 <https://airflow.readthedocs.io/en/latest/rest-api-ref.html>`__ is
deprecated since version 2.0. Please consider using
`the Stable REST API <https://airflow.readthedocs.io/en/latest/stable-rest-api-ref.html>`__.
For more information on migration, see
`RELEASE_NOTES.rst <https://github.com/apache/airflow/blob/main/RELEASE_NOTES.rst>`_
version_added: 2.0.0
type: boolean
example: ~
default: "False"
auth_backends:
description: |
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)
version_added: 2.3.0
type: string
example: ~
default: "airflow.api.auth.backend.session"
maximum_page_limit:
description: |
Used to set the maximum page limit for API requests. If limit passed as param
is greater than maximum page limit, it will be ignored and maximum page limit value
will be set as the limit
version_added: 2.0.0
type: integer
example: ~
default: "100"
fallback_page_limit:
description: |
Used to set the default page limit when limit param is zero or not provided in API
requests. Otherwise if positive integer is passed in the API requests as limit, the
smallest number of user given limit or maximum page limit is taken as limit.
type: integer
example: ~
version_added: 2.0.0
default: "100"
google_oauth2_audience:
description: 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
version_added: 2.0.0
example: project-id-random-value.apps.googleusercontent.com
default: ""
google_key_path:
description: |
Path to Google Cloud Service Account key file (JSON). If omitted, authorization based on
`the Application Default Credentials
<https://cloud.google.com/docs/authentication/production#finding_credentials_automatically>`__ will
be used.
type: string
version_added: 2.0.0
example: /files/service-account-json
default: ""
access_control_allow_headers:
description: |
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
version_added: 2.1.0
example: ~
default: ""
access_control_allow_methods:
description: |
Specifies the method or methods allowed when accessing the resource.
type: string
version_added: 2.1.0
example: ~
default: ""
access_control_allow_origins:
description: |
Indicates whether the response can be shared with requesting code from the given origins.
Separate URLs with space.
type: string
version_added: 2.2.0
example: ~
default: ""
lineage:
description: ~
options:
backend:
description: |
what lineage backend to use
version_added: ~
type: string
example: ~
default: ""
openlineage:
description: |
This section applies settings for OpenLineage integration.
For backwards compatibility with `openlineage-python` one can still use
`openlineage.yml` file or `OPENLINEAGE_` environment variables. However, below
configuration takes precedence over those.
More in documentation - https://openlineage.io/docs/client/python#configuration.
options:
disabled:
description: |
Set this to true if you don't want OpenLineage to emit events.
type: boolean
example: ~
default: "False"
version_added: ~
namespace:
description: |
OpenLineage namespace
version_added: ~
type: string
example: "food_delivery"
default: ~
extractors:
description: |
Comma-separated paths to custom OpenLineage extractors.
type: string
example: full.path.to.ExtractorClass;full.path.to.AnotherExtractorClass
default: ''
version_added: ~
config_path:
description: |
Path to YAML config. This provides backwards compatibility to pass config as
`openlineage.yml` file.
version_added: ~
type: string
example: ~
default: ''
transport:
description: |
OpenLineage Client transport configuration. It should contain type
and additional options per each type.
Currently supported types are:
* HTTP
* Kafka
* Console
type: string
example: '{"type": "http", "url": "http://localhost:5000"}'
default: ''
version_added: ~
disable_source_code:
description: |
If disabled, OpenLineage events do not contain source code of particular
operators, like PythonOperator.
default: ~
example: ~
type: boolean
version_added: ~
atlas:
description: ~
options:
sasl_enabled:
description: ~
version_added: ~
type: string
example: ~
default: "False"
host:
description: ~
version_added: ~
type: string
example: ~
default: ""
port:
description: ~
version_added: ~
type: string
example: ~
default: "21000"
username:
description: ~
version_added: ~
type: string
example: ~
default: ""
password:
description: ~
version_added: ~
type: string
sensitive: true
example: ~
default: ""
operators:
description: ~
options:
default_owner:
description: |
The default owner assigned to each new operator, unless
provided explicitly or passed via ``default_args``
version_added: ~
type: string
example: ~
default: "airflow"
default_cpus:
description: ~
version_added: ~
type: string
example: ~
default: "1"
default_ram:
description: ~
version_added: ~
type: string
example: ~
default: "512"
default_disk:
description: ~
version_added: ~
type: string
example: ~
default: "512"
default_gpus:
description: ~
version_added: ~
type: string
example: ~
default: "0"
default_queue:
description: |
Default queue that tasks get assigned to and that worker listen on.
version_added: 2.1.0
type: string
example: ~
default: "default"
allow_illegal_arguments:
description: |
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.
version_added: 2.0.0
type: string
example: ~
default: "False"
hive:
description: ~
options:
default_hive_mapred_queue:
description: |
Default mapreduce queue for HiveOperator tasks
version_added: ~
type: string
example: ~
default: ""
mapred_job_name_template:
description: |
Template for mapred_job_name in HiveOperator, supports the following named parameters
hostname, dag_id, task_id, execution_date
version_added: 2.0.0
type: string
example: ~
default: ~
webserver:
description: ~
options:
config_file:
description: |
Path of webserver config file used for configuring the webserver parameters
version_added: 2.7.0
type: string
example: ~
default: "{AIRFLOW_HOME}/webserver_config.py"
base_url:
description: |
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
version_added: ~
type: string
example: ~
default: "http://localhost:8080"
default_ui_timezone:
description: |
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
version_added: 1.10.10
type: string
example: "America/New_York"
# Default is left as UTC for now so the date's don't "suddenly" change on upgrade
default: "UTC"
web_server_host:
description: |
The ip specified when starting the web server
version_added: ~
type: string
example: ~
default: "0.0.0.0"
web_server_port:
description: |
The port on which to run the web server
version_added: ~
type: string
example: ~
default: "8080"
web_server_ssl_cert:
description: |
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.
version_added: ~
type: string
example: ~
default: ""
web_server_ssl_key:
description: |
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.
version_added: ~
type: string
example: ~
default: ""
session_backend:
description: |
The type of backend used to store web session data, can be 'database' or 'securecookie'
version_added: 2.2.4
type: string
example: securecookie
default: database
web_server_master_timeout:
description: |
Number of seconds the webserver waits before killing gunicorn master that doesn't respond
version_added: ~
type: string
example: ~
default: "120"
web_server_worker_timeout:
description: |
Number of seconds the gunicorn webserver waits before timing out on a worker
version_added: ~
type: string
example: ~
default: "120"
worker_refresh_batch_size:
description: |
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.
version_added: ~
type: string
example: ~
default: "1"
worker_refresh_interval:
description: |
Number of seconds to wait before refreshing a batch of workers.
version_added: ~
type: string
example: ~
default: "6000"
reload_on_plugin_change:
description: |
If set to True, Airflow will track files in plugins_folder directory. When it detects changes,
then reload the gunicorn.
version_added: 1.10.11
type: boolean
example: ~
default: "False"
secret_key:
description: |
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.
version_added: ~
type: string
sensitive: true
example: ~
default: "{SECRET_KEY}"
workers:
description: |
Number of workers to run the Gunicorn web server
version_added: ~
type: string
example: ~
default: "4"
worker_class:
description: |
The worker class gunicorn should use. Choices include
sync (default), eventlet, gevent. Note when using gevent you might also want to set the
"_AIRFLOW_PATCH_GEVENT" environment variable to "1" to make sure gevent patching is done as
early as possible.
version_added: ~
type: string
example: ~
default: "sync"
access_logfile:
description: |
Log files for the gunicorn webserver. '-' means log to stderr.
version_added: ~
type: string
example: ~
default: "-"
error_logfile:
description: |
Log files for the gunicorn webserver. '-' means log to stderr.
version_added: ~
type: string
example: ~
default: "-"
access_logformat:
description: |
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
version_added: 2.0.0
type: string
example: ~
default: ""
expose_config:
description: |
Expose the configuration file in the web server. Set to "non-sensitive-only" to show all values
except those that have security implications. "True" shows all values. "False" hides the
configuration completely.
version_added: ~
type: string
example: ~
default: "False"
expose_hostname:
description: |
Expose hostname in the web server
version_added: 1.10.8
type: string
example: ~
default: "False"
expose_stacktrace:
description: |
Expose stacktrace in the web server
version_added: 1.10.8
type: string
example: ~
default: "False"
dag_default_view:
description: |
Default DAG view. Valid values are: ``grid``, ``graph``, ``duration``, ``gantt``, ``landing_times``
version_added: ~
type: string
example: ~
default: "grid"
dag_orientation:
description: |
Default DAG orientation. Valid values are:
``LR`` (Left->Right), ``TB`` (Top->Bottom), ``RL`` (Right->Left), ``BT`` (Bottom->Top)
version_added: ~
type: string
example: ~
default: "LR"
log_fetch_timeout_sec:
description: |
The amount of time (in secs) webserver will wait for initial handshake
while fetching logs from other worker machine
version_added: ~
type: string
example: ~
default: "5"
log_fetch_delay_sec:
description: |
Time interval (in secs) to wait before next log fetching.
version_added: 1.10.8
type: integer
example: ~
default: "2"
log_auto_tailing_offset:
description: |
Distance away from page bottom to enable auto tailing.
version_added: 1.10.8
type: integer
example: ~
default: "30"
log_animation_speed:
description: |
Animation speed for auto tailing log display.
version_added: 1.10.8
type: integer
example: ~
default: "1000"
hide_paused_dags_by_default:
description: |
By default, the webserver shows paused DAGs. Flip this to hide paused
DAGs by default
version_added: ~
type: string
example: ~
default: "False"
page_size:
description: |
Consistent page size across all listing views in the UI
version_added: ~
type: string
example: ~
default: "100"
navbar_color:
description: |
Define the color of navigation bar
version_added: ~
type: string
example: ~
default: "#fff"
default_dag_run_display_number:
description: |
Default dagrun to show in UI
version_added: ~
type: string
example: ~
default: "25"
enable_proxy_fix:
description: |
Enable werkzeug ``ProxyFix`` middleware for reverse proxy
version_added: 1.10.1
type: boolean
example: ~
default: "False"
proxy_fix_x_for:
description: |
Number of values to trust for ``X-Forwarded-For``.
More info: https://werkzeug.palletsprojects.com/en/0.16.x/middleware/proxy_fix/
version_added: 1.10.7
type: integer
example: ~
default: "1"
proxy_fix_x_proto:
description: |
Number of values to trust for ``X-Forwarded-Proto``
version_added: 1.10.7
type: integer
example: ~
default: "1"
proxy_fix_x_host:
description: |
Number of values to trust for ``X-Forwarded-Host``
version_added: 1.10.7
type: integer
example: ~
default: "1"
proxy_fix_x_port:
description: |
Number of values to trust for ``X-Forwarded-Port``
version_added: 1.10.7
type: integer
example: ~
default: "1"
proxy_fix_x_prefix:
description: |
Number of values to trust for ``X-Forwarded-Prefix``
version_added: 1.10.7
type: integer
example: ~
default: "1"
cookie_secure:
description: |
Set secure flag on session cookie
version_added: 1.10.3
type: string
example: ~
default: "False"
cookie_samesite:
description: |
Set samesite policy on session cookie
version_added: 1.10.3
type: string
example: ~
default: "Lax"
default_wrap:
description: |
Default setting for wrap toggle on DAG code and TI log views.
version_added: 1.10.4
type: boolean
example: ~
default: "False"
x_frame_enabled:
description: |
Allow the UI to be rendered in a frame
version_added: 1.10.8
type: boolean
example: ~
default: "True"
analytics_tool:
description: |
Send anonymous user activity to your analytics tool
choose from google_analytics, segment, or metarouter
version_added: ~
type: string
example: ~
default: ~
analytics_id:
description: |
Unique ID of your account in the analytics tool
version_added: 1.10.5
type: string
example: ~
default: ~
show_recent_stats_for_completed_runs:
description: |
'Recent Tasks' stats will show for old DagRuns if set
version_added: 2.0.0
type: boolean
example: ~
default: "True"
update_fab_perms:
description: |
Update FAB permissions and sync security manager roles
on webserver startup
version_added: 1.10.7
type: string
example: ~
default: "True"
session_lifetime_minutes:
description: |
The UI cookie lifetime in minutes. User will be logged out from UI after
``session_lifetime_minutes`` of non-activity
version_added: 1.10.13
type: integer
example: ~
default: "43200"
instance_name:
description: |
Sets a custom page title for the DAGs overview page and site title for all pages
version_added: 2.1.0
type: string
example: ~
default:
instance_name_has_markup:
description: |
Whether the custom page title for the DAGs overview page contains any Markup language
version_added: 2.3.0
type: boolean
example: ~
default: "False"
auto_refresh_interval:
description: |
How frequently, in seconds, the DAG data will auto-refresh in graph or grid view
when auto-refresh is turned on
version_added: 2.2.0
type: integer
example: ~
default: "3"
warn_deployment_exposure:
description: |
Boolean for displaying warning for publicly viewable deployment
version_added: 2.3.0
type: boolean
example: ~
default: "True"
audit_view_excluded_events:
description: |
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.
version_added: 2.3.0
type: string
example: ~
default: "gantt,landing_times,tries,duration,calendar,graph,grid,tree,tree_data"
audit_view_included_events:
description: |
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.
version_added: 2.3.0
type: string
example: "dagrun_cleared,failed"
default: ~
enable_swagger_ui:
description: |
Boolean for running SwaggerUI in the webserver.
version_added: 2.6.0
type: boolean
example: ~
default: "True"
run_internal_api:
description: |
Boolean for running Internal API in the webserver.
version_added: 2.7.0
type: boolean
example: ~
default: "False"
auth_rate_limited:
description: |
Boolean for enabling rate limiting on authentication endpoints.
version_added: 2.6.0
type: boolean
example: ~
default: "True"
auth_rate_limit:
description: |
Rate limit for authentication endpoints.
version_added: 2.6.0
type: string
example: ~
default: "5 per 40 second"
caching_hash_method:
description: |
The caching algorithm used by the webserver. Must be a valid hashlib function name.
version_added: 2.6.0
type: string
example: "sha256"
default: "md5"
email:
description: |
Configuration email backend and whether to
send email alerts on retry or failure
options:
email_backend:
description: Email backend to use
version_added: ~
type: string
example: ~
default: "airflow.utils.email.send_email_smtp"
email_conn_id:
description: Email connection to use
version_added: 2.1.0
type: string
example: ~
default: "smtp_default"
default_email_on_retry:
description: |
Whether email alerts should be sent when a task is retried
version_added: 2.0.0
type: boolean
example: ~
default: "True"
default_email_on_failure:
description: |
Whether email alerts should be sent when a task failed
version_added: 2.0.0
type: boolean
example: ~
default: "True"
subject_template:
description: |
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.
version_added: 2.0.1
type: string
example: "/path/to/my_subject_template_file"
default: ~
see_also: ":doc:`Email Configuration </howto/email-config>`"
html_content_template:
description: |
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.
version_added: 2.0.1
type: string
example: "/path/to/my_html_content_template_file"
default: ~
see_also: ":doc:`Email Configuration </howto/email-config>`"
from_email:
description: |
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>``
version_added: 2.2.4
type: string
example: "Airflow <airflow@example.com>"
default: ~
smtp:
description: |
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
options:
smtp_host:
description: ~
version_added: ~
type: string
example: ~
default: "localhost"
smtp_starttls:
description: ~
version_added: ~
type: string
example: ~
default: "True"
smtp_ssl:
description: ~
version_added: ~
type: string
example: ~
default: "False"
smtp_user:
description: ~
version_added: ~
type: string
example: "airflow"
default: ~
smtp_password:
description: ~
version_added: ~
type: string
sensitive: true
example: "airflow"
default: ~
smtp_port:
description: ~
version_added: ~
type: string
example: ~
default: "25"
smtp_mail_from:
description: ~
version_added: ~
type: string
example: ~
default: "airflow@example.com"
smtp_timeout:
description: ~
version_added: 2.0.0
type: integer
example: ~
default: "30"
smtp_retry_limit:
description: ~
version_added: 2.0.0
type: integer
example: ~
default: "5"
sentry:
description: |
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``.
options:
sentry_on:
description: Enable error reporting to Sentry
version_added: 2.0.0
type: string
example: ~
default: "false"
sentry_dsn:
description: ~
version_added: 1.10.6
type: string
sensitive: true
example: ~
default: ""
before_send:
description: Dotted path to a before_send function that the sentry SDK should be configured to use.
version_added: 2.2.0
type: string
example: ~
default: ~
local_kubernetes_executor:
description: |
This section only applies if you are using the ``LocalKubernetesExecutor`` in
``[core]`` section above
options:
kubernetes_queue:
description: |
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``
version_added: 2.3.0
type: string
example: ~
default: "kubernetes"
celery_kubernetes_executor:
description: |
This section only applies if you are using the ``CeleryKubernetesExecutor`` in
``[core]`` section above
options:
kubernetes_queue:
description: |
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``
version_added: 2.0.0
type: string
example: ~
default: "kubernetes"
celery:
description: |
This section only applies if you are using the CeleryExecutor in
``[core]`` section above
options:
celery_app_name:
description: |
The app name that will be used by celery
version_added: ~
type: string
example: ~
default: "airflow.executors.celery_executor"
worker_concurrency:
description: |
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
version_added: ~
type: string
example: ~
default: "16"
worker_autoscale:
description: |
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
version_added: ~
type: string
example: 16,12
default: ~
worker_prefetch_multiplier:
description: |
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
version_added: 2.0.0
type: integer
example: ~
default: "1"
worker_enable_remote_control:
description: |
Specify if remote control of the workers is enabled.
In some cases when the broker does not support remote control, Celery creates lots of
``.*reply-celery-pidbox`` queues. You can prevent this by setting this to false.
However, with this disabled Flower won't work.
https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/index.html#broker-overview
version_added: 2.3.0
type: boolean
example: ~
default: "true"
broker_url:
description: |
The Celery broker URL. Celery supports RabbitMQ, Redis and experimentally
a sqlalchemy database. Refer to the Celery documentation for more information.
version_added: ~
type: string
sensitive: true
example: ~
default: "redis://redis:6379/0"
result_backend:
description: |
The Celery result_backend. When a job finishes, it needs to update the
metadata of the job. Therefore it will post a message on a message bus,
or insert it into a database (depending of the backend)
This status is used by the scheduler to update the state of the task
The use of a database is highly recommended
When not specified, sql_alchemy_conn with a db+ scheme prefix will be used
http://docs.celeryproject.org/en/latest/userguide/configuration.html#task-result-backend-settings
version_added: ~
type: string
sensitive: true
example: "db+postgresql://postgres:airflow@postgres/airflow"
default: ~
result_backend_sqlalchemy_engine_options:
description: |
Optional configuration dictionary to pass to the Celery result backend SQLAlchemy engine.
version_added: 2.6.0
type: string
example: '{"pool_recycle": 1800}'
default: ""
flower_host:
description: |
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
version_added: ~
type: string
example: ~
default: "0.0.0.0"
flower_url_prefix:
description: |
The root URL for Flower
version_added: ~
type: string
example: "/flower"
default: ""
flower_port:
description: |
This defines the port that Celery Flower runs on
version_added: ~
type: string
example: ~
default: "5555"
flower_basic_auth:
description: |
Securing Flower with Basic Authentication
Accepts user:password pairs separated by a comma
version_added: 1.10.2
type: string
sensitive: true
example: "user1:password1,user2:password2"
default: ""
sync_parallelism:
description: |
How many processes CeleryExecutor uses to sync task state.
0 means to use max(1, number of cores - 1) processes.
version_added: 1.10.3
type: string
example: ~
default: "0"
celery_config_options:
description: |
Import path for celery configuration options
version_added: ~
type: string
example: ~
default: "airflow.config_templates.default_celery.DEFAULT_CELERY_CONFIG"
ssl_active:
description: ~
version_added: ~
type: string
example: ~
default: "False"
ssl_key:
description: |
Path to the client key.
version_added: ~
type: string
example: ~
default: ""
ssl_cert:
description: |
Path to the client certificate.
version_added: ~
type: string
example: ~
default: ""
ssl_cacert:
description: |
Path to the CA certificate.
version_added: ~
type: string
example: ~
default: ""
pool:
description: |
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
version_added: 1.10.4
type: string
example: ~
default: "prefork"
operation_timeout:
description: |
The number of seconds to wait before timing out ``send_task_to_executor`` or
``fetch_celery_task_state`` operations.
version_added: 1.10.8
type: float
example: ~
default: "1.0"
task_track_started:
description: |
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.
version_added: 2.0.0
type: boolean
example: ~
default: "True"
task_publish_max_retries:
description: |
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.
version_added: 2.0.0
type: integer
example: ~
default: "3"
worker_precheck:
description: |
Worker initialisation check to validate Metadata Database connection
version_added: 2.0.0
type: string
example: ~
default: "False"
celery_broker_transport_options:
description: |
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
options:
visibility_timeout:
description: |
The visibility timeout defines the number of seconds to wait for the worker
to acknowledge the task before the message is redelivered to another worker.
Make sure to increase the visibility timeout to match the time of the longest
ETA you're planning to use.
visibility_timeout is only supported for Redis and SQS celery brokers.
See:
https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/redis.html#visibility-timeout
version_added: ~
type: string
example: "21600"
default: ~
sentinel_kwargs:
description: |
The sentinel_kwargs parameter allows passing additional options to the Sentinel client.
In a typical scenario where Redis Sentinel is used as the broker and Redis servers are
password-protected, the password needs to be passed through this parameter. Although its
type is string, it is required to pass a string that conforms to the dictionary format.
See:
https://docs.celeryq.dev/en/stable/getting-started/backends-and-brokers/redis.html#configuration
version_added: 2.7.0
type: string
sensitive: true
example: '{"password": "password_for_redis_server"}'
default: ~
dask:
description: |
This section only applies if you are using the DaskExecutor in
[core] section above
options:
cluster_address:
description: |
The IP address and port of the Dask cluster's scheduler.
version_added: ~
type: string
example: ~
default: "127.0.0.1:8786"
tls_ca:
description: |
Path to a CA certificate file encoded in PEM format to access a secured Dask scheduler.
version_added: ~
type: string
example: ~
default: ""
tls_cert:
description: |
Path to a certificate file for the client, encoded in PEM format.
version_added: ~
type: string
example: ~
default: ""
tls_key:
description: |
Path to a key file for the client, encoded in PEM format.
version_added: ~
type: string
example: ~
default: ""
scheduler:
description: ~
options:
job_heartbeat_sec:
description: |
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).
version_added: ~
type: string
example: ~
default: "5"
scheduler_heartbeat_sec:
description: |
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).
version_added: ~
type: string
example: ~
default: "5"
num_runs:
description: |
The number of times to try to schedule each DAG file
-1 indicates unlimited number
version_added: 1.10.6
type: string
example: ~
default: "-1"
scheduler_idle_sleep_time:
description: |
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.
version_added: 2.2.0
type: string
example: ~
default: "1"
min_file_process_interval:
description: |
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.
version_added: ~
type: string
example: ~
default: "30"
parsing_cleanup_interval:
description: |
How often (in seconds) to check for stale DAGs (DAGs which are no longer present in
the expected files) which should be deactivated, as well as datasets that are no longer
referenced and should be marked as orphaned.
version_added: 2.5.0
type: integer
example: ~
default: "60"
stale_dag_threshold:
description: |
How long (in seconds) to wait after we have re-parsed a DAG file before deactivating stale
DAGs (DAGs which are no longer present in the expected files). The reason why we need
this threshold is to account for the time between when the file is parsed and when the
DAG is loaded. The absolute maximum that this could take is `dag_file_processor_timeout`,
but when you have a long timeout configured, it results in a significant delay in the
deactivation of stale dags.
version_added: 2.6.0
type: integer
example: ~
default: "50"
dag_dir_list_interval:
description: |
How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes.
version_added: ~
type: string
example: ~
default: "300"
print_stats_interval:
description: |
How often should stats be printed to the logs. Setting to 0 will disable printing stats
version_added: ~
type: string
example: ~
default: "30"
pool_metrics_interval:
description: |
How often (in seconds) should pool usage stats be sent to StatsD (if statsd_on is enabled)
version_added: 2.0.0
type: float
example: ~
default: "5.0"
scheduler_health_check_threshold:
description: |
If the last scheduler heartbeat happened more than scheduler_health_check_threshold
ago (in seconds), scheduler is considered unhealthy.
This is used by the health check in the "/health" endpoint and in `airflow jobs check` CLI
for SchedulerJob.
version_added: 1.10.2
type: string
example: ~
default: "30"
enable_health_check:
description: |
When you start a scheduler, airflow starts a tiny web server
subprocess to serve a health check if this is set to True
version_added: 2.4.0
type: boolean
example: ~
default: "False"
scheduler_health_check_server_port:
description: |
When you start a scheduler, airflow starts a tiny web server
subprocess to serve a health check on this port
version_added: 2.4.0
type: string
example: ~
default: "8974"
orphaned_tasks_check_interval:
description: |
How often (in seconds) should the scheduler check for orphaned tasks and SchedulerJobs
version_added: 2.0.0
type: float
example: ~
default: "300.0"
child_process_log_directory:
description: ~
version_added: ~
type: string
example: ~
default: "{AIRFLOW_HOME}/logs/scheduler"
scheduler_zombie_task_threshold:
description: |
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.
version_added: ~
type: string
example: ~
default: "300"
zombie_detection_interval:
description: |
How often (in seconds) should the scheduler check for zombie tasks.
version_added: 2.3.0
type: float
example: ~
default: "10.0"
catchup_by_default:
description: |
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)
version_added: ~
type: string
example: ~
default: "True"
ignore_first_depends_on_past_by_default:
description: |
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.
version_added: 2.3.0
type: string
example: ~
default: "True"
max_tis_per_query:
description: |
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)
version_added: ~
type: string
example: ~
default: "512"
use_row_level_locking:
description: |
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
version_added: 2.0.0
type: boolean
example: ~
default: "True"
max_dagruns_to_create_per_loop:
description: |
Max number of DAGs to create DagRuns for per scheduler loop.
example: ~
version_added: 2.0.0
type: string
default: "10"
see_also: ":ref:`scheduler:ha:tunables`"
max_dagruns_per_loop_to_schedule:
description: |
How many DagRuns should a scheduler examine (and lock) when scheduling
and queuing tasks.
example: ~
version_added: 2.0.0
type: string
default: "20"
see_also: ":ref:`scheduler:ha:tunables`"
schedule_after_task_execution:
description: |
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
example: ~
version_added: 2.0.0
type: boolean
default: "True"
parsing_pre_import_modules:
description: |
The scheduler reads dag files to extract the airflow modules that are going to be used,
and imports them ahead of time to avoid having to re-do it for each parsing process.
This flag can be set to False to disable this behavior in case an airflow module needs to be freshly
imported each time (at the cost of increased DAG parsing time).
version_added: 2.6.0
type: boolean
example: ~
default: "True"
parsing_processes:
description: |
The scheduler can run multiple processes in parallel to parse dags.
This defines how many processes will run.
version_added: 1.10.14
type: string
example: ~
default: "2"
file_parsing_sort_mode:
description: |
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
version_added: 2.1.0
type: string
example: ~
default: "modified_time"
standalone_dag_processor:
description: |
Whether the dag processor is running as a standalone process or it is a subprocess of a scheduler
job.
version_added: 2.3.0
type: boolean
example: ~
default: "False"
max_callbacks_per_loop:
description: |
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.
version_added: 2.3.0
type: integer
example: ~
default: "20"
dag_stale_not_seen_duration:
description: |
Only applicable if `[scheduler]standalone_dag_processor` is true.
Time in seconds after which dags, which were not updated by Dag Processor are deactivated.
version_added: 2.4.0
type: integer
example: ~
default: "600"
use_job_schedule:
description: |
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.
version_added: 1.10.2
type: string
example: ~
default: "True"
allow_trigger_in_future:
description: |
Allow externally triggered DagRuns for Execution Dates in the future
Only has effect if schedule_interval is set to None in DAG
version_added: 1.10.8
type: string
example: ~
default: "False"
trigger_timeout_check_interval:
description: |
How often to check for expired trigger requests that have not run yet.
version_added: 2.2.0
type: string
example: ~
default: "15"
task_queued_timeout:
description: |
Amount of time a task can be in the queued state before being retried or set to failed.
version_added: 2.6.0
type: float
example: ~
default: "600.0"
task_queued_timeout_check_interval:
description: |
How often to check for tasks that have been in the queued state for
longer than `[scheduler] task_queued_timeout`.
version_added: 2.6.0
type: float
example: ~
default: "120.0"
triggerer:
description: ~
options:
default_capacity:
description: |
How many triggers a single Triggerer will run at once, by default.
version_added: 2.2.0
type: string
example: ~
default: "1000"
kerberos:
description: ~
options:
ccache:
description: ~
version_added: ~
type: string
example: ~
default: "/tmp/airflow_krb5_ccache"
principal:
description: |
gets augmented with fqdn
version_added: ~
type: string
example: ~
default: "airflow"
reinit_frequency:
description: ~
version_added: ~
type: string
example: ~
default: "3600"
kinit_path:
description: ~
version_added: ~
type: string
example: ~
default: "kinit"
keytab:
description: ~
version_added: ~
type: string
example: ~
default: "airflow.keytab"
forwardable:
description: |
Allow to disable ticket forwardability.
version_added: 2.2.0
type: boolean
example: ~
default: "True"
include_ip:
description: |
Allow to remove source IP from token, useful when using token behind NATted Docker host.
version_added: 2.2.0
type: boolean
example: ~
default: "True"
elasticsearch:
description: ~
options:
host:
description: |
Elasticsearch host
version_added: 1.10.4
type: string
example: ~
default: ""
log_id_template:
description: |
Format of the log_id, which is used to query for a given tasks logs
version_added: 1.10.4
type: string
example: ~
default: "{{dag_id}}-{{task_id}}-{{run_id}}-{{map_index}}-{{try_number}}"
end_of_log_mark:
description: |
Used to mark the end of a log stream for a task
version_added: 1.10.4
type: string
example: ~
default: "end_of_log"
frontend:
description: |
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
version_added: 1.10.4
type: string
example: "http://localhost:5601/app/kibana#/discover\
?_a=(columns:!(message),query:(language:kuery,query:'log_id: \"{log_id}\"'),sort:!(log.offset,asc))"
default: ""
write_stdout:
description: |
Write the task logs to the stdout of the worker, rather than the default files
version_added: 1.10.4
type: string
example: ~
default: "False"
json_format:
description: |
Instead of the default log formatter, write the log lines as JSON
version_added: 1.10.4
type: string
example: ~
default: "False"
json_fields:
description: |
Log fields to also attach to the json output, if enabled
version_added: 1.10.4
type: string
example: ~
default: "asctime, filename, lineno, levelname, message"
host_field:
description: |
The field where host name is stored (normally either `host` or `host.name`)
version_added: 2.1.1
type: string
example: ~
default: "host"
offset_field:
description: |
The field where offset is stored (normally either `offset` or `log.offset`)
version_added: 2.1.1
type: string
example: ~
default: "offset"
index_patterns:
description: |
Comma separated list of index patterns to use when searching for logs (default: `_all`).
version_added: 2.6.0
type: string
example: something-*
default: "_all"
elasticsearch_configs:
description: ~
options:
use_ssl:
description: ~
version_added: 1.10.5
type: string
example: ~
default: "False"
verify_certs:
description: ~
version_added: 1.10.5
type: string
example: ~
default: "True"
kubernetes_executor:
description: ~
renamed:
previous_name: kubernetes
version: 2.5.0
options:
api_client_retry_configuration:
description: |
Kwargs to override the default urllib3 Retry used in the kubernetes API client
version_added: 2.6.0
type: string
example: '{ "total": 3, "backoff_factor": 0.5 }'
default: ""
logs_task_metadata:
description: |
Flag to control the information added to kubernetes executor logs for better traceability
version_added: 2.7.0
type: boolean
example: ~
default: "False"
pod_template_file:
description: |
Path to the YAML pod file that forms the basis for KubernetesExecutor workers.
version_added: 1.10.11
type: string
example: ~
default: ""
see_also: ":ref:`concepts:pod_template_file`"
worker_container_repository:
description: |
The repository of the Kubernetes Image for the Worker to Run
version_added: ~
type: string
example: ~
default: ""
worker_container_tag:
description: |
The tag of the Kubernetes Image for the Worker to Run
version_added: ~
type: string
example: ~
default: ""
namespace:
description: |
The Kubernetes namespace where airflow workers should be created. Defaults to ``default``
version_added: ~
type: string
example: ~
default: "default"
delete_worker_pods:
description: |
If True, all worker pods will be deleted upon termination
version_added: ~
type: string
example: ~
default: "True"
delete_worker_pods_on_failure:
description: |
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.
version_added: 1.10.11
type: string
example: ~
default: "False"
worker_pods_creation_batch_size:
description: |
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.
version_added: 1.10.3
type: string
example: ~
default: "1"
multi_namespace_mode:
description: |
Allows users to launch pods in multiple namespaces.
Will require creating a cluster-role for the scheduler,
or use multi_namespace_mode_namespace_list configuration.
version_added: 1.10.12
type: boolean
example: ~
default: "False"
multi_namespace_mode_namespace_list:
description: |
If multi_namespace_mode is True while scheduler does not have a cluster-role,
give the list of namespaces where the scheduler will schedule jobs
Scheduler needs to have the necessary permissions in these namespaces.
version_added: 2.6.0
type: string
example: ~
default: ""
in_cluster:
description: |
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.
version_added: ~
type: string
example: ~
default: "True"
cluster_context:
description: |
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.
version_added: 1.10.3
type: string
example: ~
default: ~
config_file:
description: |
Path to the kubernetes configfile to be used when ``in_cluster`` is set to False
version_added: 1.10.3
type: string
example: ~
default: ~
kube_client_request_args:
description: |
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
version_added: 1.10.4
type: string
example: ~
default: ""
delete_option_kwargs:
description: |
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
version_added: 1.10.12
type: string
example: '{"grace_period_seconds": 10}'
default: ""
enable_tcp_keepalive:
description: |
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.
version_added: 2.0.0
type: boolean
example: ~
default: "True"
tcp_keep_idle:
description: |
When the `enable_tcp_keepalive` option is enabled, TCP probes a connection that has
been idle for `tcp_keep_idle` seconds.
version_added: 2.0.0
type: integer
example: ~
default: "120"
tcp_keep_intvl:
description: |
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.
version_added: 2.0.0
type: integer
example: ~
default: "30"
tcp_keep_cnt:
description: |
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.
version_added: 2.0.0
type: integer
example: ~
default: "6"
verify_ssl:
description: |
Set this to false to skip verifying SSL certificate of Kubernetes python client.
version_added: 2.1.0
type: boolean
example: ~
default: "True"
worker_pods_queued_check_interval:
description: |
How often in seconds to check for task instances stuck in "queued" status without a pod
version_added: 2.2.0
type: integer
example: ~
default: "60"
ssl_ca_cert:
description: |
Path to a CA certificate to be used by the Kubernetes client to verify the server's SSL certificate.
version_added: 2.6.0
type: string
example: ~
default: ""
sensors:
description: ~
options:
default_timeout:
description: |
Sensor default timeout, 7 days by default (7 * 24 * 60 * 60).
version_added: 2.3.0
type: float
example: ~
default: "604800"