diff --git a/TESTING.rst b/TESTING.rst index 09b7e9d31491..95bdc5cdeb6d 100644 --- a/TESTING.rst +++ b/TESTING.rst @@ -75,7 +75,7 @@ as default for all tests you should add the value to this file. You can also of course override the values in individual test by patching environment variables following the usual ``AIRFLOW__SECTION__KEY`` pattern or ``conf_vars`` context manager. -.. note:: +.. note:: Previous way of setting the test configuration The test configuration for Airflow before July 2023 was automatically generated in a file named ``AIRFLOW_HOME/unittest.cfg``. The template for it was stored in "config_templates" next to the yaml file. @@ -87,6 +87,7 @@ the usual ``AIRFLOW__SECTION__KEY`` pattern or ``conf_vars`` context manager. The unittest.cfg file generated in {AIRFLOW_HOME} will no longer be used and can be removed. + Airflow test types ------------------ diff --git a/airflow/__main__.py b/airflow/__main__.py index 5eca0b84ed6d..893c6bde099d 100644 --- a/airflow/__main__.py +++ b/airflow/__main__.py @@ -45,6 +45,13 @@ def main(): parser = cli_parser.get_parser() argcomplete.autocomplete(parser) args = parser.parse_args() + + # Here we ensure that the default configuration is written if needed before running any command + # that might need it. This used to be done during configuration initialization but having it + # in main ensures that it is not done during tests and other ways airflow imports are used + from airflow.configuration import write_default_airflow_configuration_if_needed + + write_default_airflow_configuration_if_needed() args.func(args) diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index b661f9361ab8..a57a8dae1dec 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -330,6 +330,11 @@ def string_lower_type(val): help="Comment out all configuration options. Useful as starting point for new installation", action="store_true", ) +ARG_EXCLUDE_PROVIDERS = Arg( + ("-p", "--exclude-providers"), + help="Exclude provider configuration (they are included by default)", + action="store_true", +) ARG_DEFAULTS = Arg( ("-a", "--defaults"), help="Show only defaults - do not include local configuration, sources," @@ -818,6 +823,16 @@ def string_lower_type(val): action="store_true", ) +# IMPORTANT NOTE! ONLY FOR CELERY ARGUMENTS +# +# Celery configs below have explicit fallback values because celery provider defaults are not yet loaded +# via provider at the time we parse the command line, so in case it is not set, we need to have manual +# fallback. After ProvidersManager.initialize_providers_configuration() is called, the fallbacks are +# not needed anymore and everywhere where you access configuration in provider-specific code and when +# you are sure that providers configuration has been initialized, you can use conf.get() without fallbacks. +# +# DO NOT REMOVE THE FALLBACKS in args parsing even if you are tempted to. +# TODO: possibly move the commands to providers but that could be big performance hit on the CLI # worker ARG_QUEUES = Arg( ("-q", "--queues"), @@ -828,7 +843,7 @@ def string_lower_type(val): ("-c", "--concurrency"), type=int, help="The number of worker processes", - default=conf.get("celery", "worker_concurrency"), + default=conf.getint("celery", "worker_concurrency", fallback=16), ) ARG_CELERY_HOSTNAME = Arg( ("-H", "--celery-hostname"), @@ -855,22 +870,24 @@ def string_lower_type(val): ARG_BROKER_API = Arg(("-a", "--broker-api"), help="Broker API") ARG_FLOWER_HOSTNAME = Arg( ("-H", "--hostname"), - default=conf.get("celery", "FLOWER_HOST"), + default=conf.get("celery", "FLOWER_HOST", fallback="0.0.0.0"), help="Set the hostname on which to run the server", ) ARG_FLOWER_PORT = Arg( ("-p", "--port"), - default=conf.get("celery", "FLOWER_PORT"), + default=conf.getint("celery", "FLOWER_PORT", fallback=5555), type=int, help="The port on which to run the server", ) ARG_FLOWER_CONF = Arg(("-c", "--flower-conf"), help="Configuration file for flower") ARG_FLOWER_URL_PREFIX = Arg( - ("-u", "--url-prefix"), default=conf.get("celery", "FLOWER_URL_PREFIX"), help="URL prefix for Flower" + ("-u", "--url-prefix"), + default=conf.get("celery", "FLOWER_URL_PREFIX", fallback=""), + help="URL prefix for Flower", ) ARG_FLOWER_BASIC_AUTH = Arg( ("-A", "--basic-auth"), - default=conf.get("celery", "FLOWER_BASIC_AUTH"), + default=conf.get("celery", "FLOWER_BASIC_AUTH", fallback=""), help=( "Securing Flower with Basic Authentication. " "Accepts user:password pairs separated by a comma. " @@ -1848,6 +1865,12 @@ class GroupCommand(NamedTuple): func=lazy_load_command("airflow.cli.commands.provider_command.executors_list"), args=(ARG_OUTPUT, ARG_VERBOSE), ), + ActionCommand( + name="configs", + help="Get information about provider configuration", + func=lazy_load_command("airflow.cli.commands.provider_command.config_list"), + args=(ARG_OUTPUT, ARG_VERBOSE), + ), ActionCommand( name="status", help="Get information about provider initialization status", @@ -2038,6 +2061,7 @@ class GroupCommand(NamedTuple): ARG_INCLUDE_SOURCES, ARG_INCLUDE_ENV_VARS, ARG_COMMENT_OUT_EVERYTHING, + ARG_EXCLUDE_PROVIDERS, ARG_DEFAULTS, ARG_VERBOSE, ), diff --git a/airflow/cli/commands/config_command.py b/airflow/cli/commands/config_command.py index 8d664417237d..86130dab102c 100644 --- a/airflow/cli/commands/config_command.py +++ b/airflow/cli/commands/config_command.py @@ -37,6 +37,7 @@ def show_config(args): include_descriptions=args.include_descriptions or args.defaults, include_sources=args.include_sources and not args.defaults, include_env_vars=args.include_env_vars or args.defaults, + include_providers=not args.exclude_providers, comment_out_everything=args.comment_out_everything or args.defaults, only_defaults=args.defaults, ) @@ -48,6 +49,14 @@ def show_config(args): def get_value(args): """Get one value from configuration.""" + # while this will make get_value quite a bit slower we must initialize configuration + # for providers because we do not know what sections and options will be available after + # providers are initialized. Theoretically Providers might add new sections and options + # but also override defaults for existing options, so without loading all providers we + # cannot be sure what is the final value of the option. + from airflow.providers_manager import ProvidersManager + + ProvidersManager().initialize_providers_configuration() if not conf.has_option(args.section, args.option): raise SystemExit(f"The option [{args.section}/{args.option}] is not found in config.") diff --git a/airflow/cli/commands/provider_command.py b/airflow/cli/commands/provider_command.py index 0256a74089c0..adcb1d18fa14 100644 --- a/airflow/cli/commands/provider_command.py +++ b/airflow/cli/commands/provider_command.py @@ -183,6 +183,18 @@ def executors_list(args): ) +@suppress_logs_and_warning +def config_list(args): + """Lists all configurations at the command line.""" + AirflowConsole().print_as( + data=list(ProvidersManager().provider_configs), + output=args.output, + mapper=lambda x: { + "provider_config": x, + }, + ) + + @suppress_logs_and_warning def status(args): """Informs if providers manager has been initialized. diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 9432b03d1f69..4cef07dee1de 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2010,257 +2010,6 @@ local_kubernetes_executor: 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.providers.celery.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.providers.celery.executors.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 diff --git a/airflow/configuration.py b/airflow/configuration.py index 10f4cf5a704a..86d3ca52528e 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -33,9 +33,9 @@ from collections import OrderedDict from configparser import ConfigParser, NoOptionError, NoSectionError from contextlib import contextmanager +from copy import deepcopy from json.decoder import JSONDecodeError -from threading import Lock -from typing import IO, Any, Dict, Iterable, Pattern, Set, Tuple, Union +from typing import IO, Any, Dict, Generator, Iterable, Pattern, Set, Tuple, Union from urllib.parse import urlsplit import re2 @@ -151,15 +151,31 @@ def _default_config_file_path(file_name: str) -> str: return os.path.join(templates_dir, file_name) -def retrieve_configuration_description() -> dict[str, dict[str, Any]]: +def retrieve_configuration_description( + include_airflow: bool = True, + include_providers: bool = True, + selected_provider: str | None = None, +) -> dict[str, dict[str, Any]]: """ Read Airflow configuration description from YAML file. + :param include_airflow: Include Airflow configs + :param include_providers: Include provider configs + :param selected_provider: If specified, include selected provider only + :param config_file_name: name of the file in "config_templates" directory to read default config from :return: Python dictionary containing configs & their info """ base_configuration_description: dict[str, dict[str, Any]] = {} - with open(_default_config_file_path("config.yml")) as config_file: - base_configuration_description.update(yaml.safe_load(config_file)) + if include_airflow: + with open(_default_config_file_path("config.yml")) as config_file: + base_configuration_description.update(yaml.safe_load(config_file)) + if include_providers: + from airflow.providers_manager import ProvidersManager + + for provider, config in ProvidersManager().provider_configs: + if selected_provider and provider != selected_provider: + continue + base_configuration_description.update(config) return base_configuration_description @@ -183,7 +199,7 @@ def __init__( **kwargs, ): super().__init__(*args, **kwargs) - self.configuration_description = retrieve_configuration_description() + self.configuration_description = retrieve_configuration_description(include_providers=False) self.upgraded_values = {} # For those who would like to use a different data structure to keep defaults: # We have to keep the default values in a ConfigParser rather than in any other @@ -196,7 +212,7 @@ def __init__( self._update_logging_deprecated_template_to_one_from_defaults() self.is_validated = False self._suppress_future_warnings = False - self._interpolation_lock = Lock() + self._providers_configuration_loaded = False def _update_logging_deprecated_template_to_one_from_defaults(self): default = self.get_default_value("logging", "log_filename_template") @@ -452,6 +468,7 @@ def get_sections_including_defaults(self) -> list[str]: :return: list of section names """ my_own_sections = self.sections() + all_sections_from_defaults = list(self.configuration_description.keys()) for section in my_own_sections: if section not in all_sections_from_defaults: @@ -485,6 +502,129 @@ def optionxform(self, optionstr: str) -> str: """ return optionstr + @contextmanager + def make_sure_configuration_loaded(self, with_providers: bool) -> Generator[None, None, None]: + """ + Make sure configuration is loaded with or without providers, regardless if the provider configuration + has been loaded before or not. Restores configuration to the state before entering the context. + + :param with_providers: whether providers should be loaded + """ + reload_providers_when_leaving = False + if with_providers and not self._providers_configuration_loaded: + # make sure providers are initialized + from airflow.providers_manager import ProvidersManager + + # run internal method to initialize providers configuration in ordered to not trigger the + # initialize_providers_configuration cache (because we will be unloading it now + ProvidersManager()._initialize_providers_configuration() + elif not with_providers and self._providers_configuration_loaded: + reload_providers_when_leaving = True + self.restore_core_default_configuration() + yield + if reload_providers_when_leaving: + self.load_providers_configuration() + + @staticmethod + def _write_section_header( + file: IO[str], + include_descriptions: bool, + section_config_description: dict[str, str], + section_to_write: str, + ) -> None: + """Writes header for configuration section.""" + file.write(f"[{section_to_write}]\n") + section_description = section_config_description.get("description") + if section_description and include_descriptions: + for line in section_description.splitlines(): + file.write(f"# {line}\n") + file.write("\n") + + def _write_option_header( + self, + file: IO[str], + option: str, + extra_spacing: bool, + include_descriptions: bool, + include_env_vars: bool, + include_examples: bool, + include_sources: bool, + section_config_description: dict[str, dict[str, Any]], + section_to_write: str, + sources_dict: ConfigSourcesType, + ) -> tuple[bool, bool]: + """Writes header for configuration option. + + Returns tuple of (should_continue, needs_separation) where needs_separation should be + set if the option needs additional separation to visually separate it from the next option. + """ + from airflow import __version__ as airflow_version + + option_config_description = ( + section_config_description.get("options", {}).get(option, {}) + if section_config_description + else {} + ) + version_added = option_config_description.get("version_added") + if version_added is not None and parse_version(version_added) > parse_version( + parse_version(airflow_version).base_version + ): + # skip if option is going to be added in the future version + return False, False + description = option_config_description.get("description") + needs_separation = False + if description and include_descriptions: + for line in description.splitlines(): + file.write(f"# {line}\n") + needs_separation = True + example = option_config_description.get("example") + if example is not None and include_examples: + if extra_spacing: + file.write("#\n") + file.write(f"# Example: {option} = {example}\n") + needs_separation = True + if include_sources and sources_dict: + sources_section = sources_dict.get(section_to_write) + value_with_source = sources_section.get(option) if sources_section else None + if value_with_source is None: + file.write("#\n# Source: not defined\n") + else: + file.write(f"#\n# Source: {value_with_source[1]}\n") + needs_separation = True + if include_env_vars: + file.write(f"#\n# Variable: AIRFLOW__{section_to_write.upper()}__{option.upper()}\n") + if extra_spacing: + file.write("#\n") + needs_separation = True + return True, needs_separation + + def _write_value( + self, + file: IO[str], + option: str, + comment_out_everything: bool, + needs_separation: bool, + only_defaults: bool, + section_to_write: str, + ): + if self._default_values is None: + default_value = None + else: + default_value = self.get_default_value(section_to_write, option, raw=True) + if only_defaults: + value = default_value + else: + value = self.get(section_to_write, option, fallback=default_value, raw=True) + if value is None: + file.write(f"# {option} = \n") + else: + if comment_out_everything: + file.write(f"# {option} = {value}\n") + else: + file.write(f"{option} = {value}\n") + if needs_separation: + file.write("\n") + def write( # type: ignore[override] self, file: IO[str], @@ -493,6 +633,7 @@ def write( # type: ignore[override] include_descriptions: bool = True, include_sources: bool = True, include_env_vars: bool = True, + include_providers: bool = True, comment_out_everything: bool = False, hide_sensitive_values: bool = False, extra_spacing: bool = True, @@ -508,12 +649,12 @@ def write( # type: ignore[override] :param include_descriptions: Include descriptions in the output :param include_sources: Include the source of each config option :param include_env_vars: Include environment variables corresponding to each config option + :param include_providers: Include providers configuration + :param comment_out_everything: Comment out all values :param hide_sensitive_values: Include sensitive values in the output :param extra_spacing: Add extra spacing before examples and after variables :param only_defaults: Only include default values when writing the config, not the actual values """ - from airflow import __version__ as airflow_version - sources_dict = {} if include_sources: sources_dict = self.as_dict(display_source=True) @@ -521,75 +662,49 @@ def write( # type: ignore[override] raise RuntimeError("Cannot write default config, no default config set") if self.configuration_description is None: raise RuntimeError("Cannot write default config, no default configuration description set") - for section_to_write in self.get_sections_including_defaults(): - section_config_description = self.configuration_description.get(section_to_write, {}) - if section_to_write != section and section is not None: - continue - if self._default_values.has_section(section_to_write) or self.has_section(section_to_write): - needs_separation = False - file.write(f"[{section_to_write}]\n") - section_description = section_config_description.get("description") - if section_description and include_descriptions: - for line in section_description.splitlines(): - file.write(f"# {line}\n") - file.write("\n") - for key in self.get_options_including_defaults(section_to_write): - option_config_description = ( - section_config_description.get("options", {}).get(key, {}) - if section_config_description - else {} + with self.make_sure_configuration_loaded(with_providers=include_providers): + for section_to_write in self.get_sections_including_defaults(): + section_config_description = self.configuration_description.get(section_to_write, {}) + if section_to_write != section and section is not None: + continue + if self._default_values.has_section(section_to_write) or self.has_section(section_to_write): + self._write_section_header( + file, include_descriptions, section_config_description, section_to_write ) - version_added = option_config_description.get("version_added") - if version_added is not None and parse_version(version_added) > parse_version( - parse_version(airflow_version).base_version - ): - # skip if option is going to be added in the future version - continue - description = option_config_description.get("description") - needs_separation = False - if description and include_descriptions: - for line in description.splitlines(): - file.write(f"# {line}\n") - needs_separation = True - example = option_config_description.get("example") - if example is not None and include_examples: - if extra_spacing: - file.write("#\n") - file.write(f"# Example: {key} = {example}\n") - needs_separation = True - if include_sources and sources_dict: - sources_section = sources_dict.get(section_to_write) - value_with_source = sources_section.get(key) if sources_section else None - if value_with_source is None: - file.write("#\n# Source: not defined\n") - else: - file.write(f"#\n# Source: {value_with_source[1]}\n") - needs_separation = True - if include_env_vars: - file.write(f"#\n# Variable: AIRFLOW__{section_to_write.upper()}__{key.upper()}\n") - if extra_spacing: - file.write("#\n") - needs_separation = True - if self._default_values is None: - default_value = None - else: - default_value = self.get_default_value(section_to_write, key, raw=True) - if only_defaults: - value = default_value - else: - value = self.get(section_to_write, key, fallback=default_value, raw=True) - if value is None: - file.write(f"# {key} = \n") - else: - if comment_out_everything: - file.write(f"# {key} = {value}\n") - else: - file.write(f"{key} = {value}\n") - if needs_separation: + for option in self.get_options_including_defaults(section_to_write): + should_continue, needs_separation = self._write_option_header( + file=file, + option=option, + extra_spacing=extra_spacing, + include_descriptions=include_descriptions, + include_env_vars=include_env_vars, + include_examples=include_examples, + include_sources=include_sources, + section_config_description=section_config_description, + section_to_write=section_to_write, + sources_dict=sources_dict, + ) + self._write_value( + file=file, + option=option, + comment_out_everything=comment_out_everything, + needs_separation=needs_separation, + only_defaults=only_defaults, + section_to_write=section_to_write, + ) + if include_descriptions and not needs_separation: + # extra separation between sections in case last option did not need it file.write("\n") - if include_descriptions and not needs_separation: - # extra separation between sections in case last option did not need it - file.write("\n") + + def restore_core_default_configuration(self) -> None: + """Restore default configuration for core Airflow. + + It does not restore configuration for providers. If you want to restore configuration for + providers, you need to call ``load_providers_configuration`` method. + """ + self.configuration_description = retrieve_configuration_description(include_providers=False) + self._default_values = create_default_config_parser(self.configuration_description) + self._providers_configuration_loaded = False def validate(self): self._validate_sqlite3_version() @@ -1617,7 +1732,13 @@ def _replace_section_config_with_display_sources( ): continue if display_source: - sect[k] = (val, source_name) + updated_source_name = source_name + if source_name == "default": + source_description_section = configuration_description.get(section, {}) + source_description_key = source_description_section.get("options", {}).get(k, {}) + if source_description_key is not None: + updated_source_name = source_description_key.get("source", source_name) + sect[k] = (val, updated_source_name) else: sect[k] = val @@ -1664,6 +1785,61 @@ def remove_all_read_configurations(self): for section in self.sections(): self.remove_section(section) + @property + def providers_configuration_loaded(self) -> bool: + """Checks if providers have been loaded.""" + return self._providers_configuration_loaded + + def load_providers_configuration(self): + """ + Loads configuration for providers. + + This should be done after initial configuration have been performed. Initializing and discovering + providers is an expensive operation and cannot be performed when we load configuration for the first + time when airflow starts, because we initialize configuration very early, during importing of the + `airflow` package and the module is not yet ready to be used when it happens and until configuration + and settings are loaded. Therefore, in order to reload provider configuration we need to additionally + load provider - specific configuration. + """ + log.debug("Loading providers configuration") + from airflow.providers_manager import ProvidersManager + + self.restore_core_default_configuration() + for provider, config in ProvidersManager().already_initialized_provider_configs: + for provider_section, provider_section_content in config.items(): + provider_options = provider_section_content["options"] + section_in_current_config = self.configuration_description.get(provider_section) + if not section_in_current_config: + self.configuration_description[provider_section] = deepcopy(provider_section_content) + section_in_current_config = self.configuration_description.get(provider_section) + section_in_current_config["source"] = f"default-{provider}" + for option in provider_options: + section_in_current_config["options"][option]["source"] = f"default-{provider}" + else: + section_source = section_in_current_config.get("source", "Airflow's core package").split( + "default-" + )[-1] + raise AirflowConfigException( + f"The provider {provider} is attempting to contribute " + f"configuration section {provider_section} that " + f"has already been added before. The source of it: {section_source}." + "This is forbidden. A provider can only add new sections. It" + "cannot contribute options to existing sections or override other " + "provider's configuration.", + UserWarning, + ) + self._default_values = create_default_config_parser(self.configuration_description) + # sensitive_config_values needs to be refreshed here. This is a cached_property, so we can delete + # the cached values, and it will be refreshed on next access. This has been an implementation + # detail in Python 3.8 but as of Python 3.9 it is documented behaviour. + # See https://docs.python.org/3/library/functools.html#functools.cached_property + try: + del self.sensitive_config_values + except AttributeError: + # no problem if cache is not set yet + pass + self._providers_configuration_loaded = True + @staticmethod def _warn_deprecate( section: str, key: str, deprecated_section: str, deprecated_name: str, extra_stacklevel: int @@ -1752,6 +1928,29 @@ def create_default_config_parser(configuration_description: dict[str, dict[str, return parser +def write_default_airflow_configuration_if_needed(): + if not os.path.isfile(AIRFLOW_CONFIG): + log.debug("Creating new Airflow config file in: %s", AIRFLOW_CONFIG) + pathlib.Path(AIRFLOW_HOME).mkdir(parents=True, exist_ok=True) + if conf.get("core", "fernet_key", fallback=None) is None: + # We know that FERNET_KEY is not set, so we can generate it, set as global key + # and also write it to the config file so that same key will be used next time + global FERNET_KEY + FERNET_KEY = _generate_fernet_key() + conf.remove_option("core", "fernet_key") + conf.set("core", "fernet_key", FERNET_KEY) + with open(AIRFLOW_CONFIG, "w") as file: + conf.write( + file, + include_sources=False, + include_env_vars=True, + include_providers=True, + extra_spacing=True, + only_defaults=True, + ) + make_group_other_inaccessible(AIRFLOW_CONFIG) + + def load_standard_airflow_configuration(airflow_config_parser: AirflowConfigParser): """ Loads standard airflow configuration. @@ -1763,23 +1962,7 @@ def load_standard_airflow_configuration(airflow_config_parser: AirflowConfigPars :param airflow_config_parser: parser to which the configuration will be loaded """ - global FERNET_KEY, AIRFLOW_HOME - if not os.path.isfile(AIRFLOW_CONFIG): - from cryptography.fernet import Fernet - - log.info("Creating new Airflow config file in: %s", AIRFLOW_CONFIG) - pathlib.Path(AIRFLOW_HOME).mkdir(parents=True, exist_ok=True) - FERNET_KEY = Fernet.generate_key().decode() - if airflow_config_parser._default_values and airflow_config_parser.configuration_description: - with open(AIRFLOW_CONFIG, "w") as file: - airflow_config_parser.write( - file, - include_sources=False, - include_env_vars=True, - extra_spacing=True, - only_defaults=True, - ) - make_group_other_inaccessible(AIRFLOW_CONFIG) + global AIRFLOW_HOME log.info("Reading the config from %s", AIRFLOW_CONFIG) airflow_config_parser.read(AIRFLOW_CONFIG) if airflow_config_parser.has_option("core", "AIRFLOW_HOME"): diff --git a/docs/apache-airflow-providers/howto/create-update-providers.rst b/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst similarity index 71% rename from docs/apache-airflow-providers/howto/create-update-providers.rst rename to airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst index 723e70bec3e7..be249b53bb04 100644 --- a/docs/apache-airflow-providers/howto/create-update-providers.rst +++ b/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst @@ -263,102 +263,3 @@ main Airflow documentation that involves some steps with the providers is also w breeze build-docs --package-filter apache-airflow-providers- breeze build-docs --package-filter apache-airflow - -Optional provider features --------------------------- - - .. versionadded:: 2.3.0 - - This feature is available in Airflow 2.3+. - -Some providers might provide optional features, which are only available when some packages or libraries -are installed. Such features will typically result in ``ImportErrors``; however, those import errors -should be silently ignored rather than pollute the logs of Airflow with false warnings. False warnings -are a very bad pattern, as they tend to turn into blind spots, so avoiding false warnings is encouraged. -However, until Airflow 2.3, Airflow had no mechanism to selectively ignore "known" ImportErrors. So -Airflow 2.1 and 2.2 silently ignored all ImportErrors coming from providers with actually lead to -ignoring even important import errors - without giving the clue to Airflow users that there is something -missing in provider dependencies. - -Using Providers with dynamic task mapping ------------------------------------------ - -Airflow 2.3 added `Dynamic Task Mapping `_ -and it added the possibility of assigning a unique key to each task. Which means that when such dynamically -mapped task wants to retrieve a value from XCom (for example in case an extra link should calculated) -it should always check if the ti_key value passed is not None an only then retrieve the XCom value using -XCom.get_value. This allows to keep backwards compatibility with earlier versions of Airflow. - -Typical code to access XCom Value in providers that want to keep backwards compatibility should look similar to -this (note the ``if ti_key is not None:`` condition). - - .. code-block:: python - - def get_link( - self, - operator: BaseOperator, - dttm: datetime | None = None, - ti_key: "TaskInstanceKey" | None = None, - ): - if ti_key is not None: - job_ids = XCom.get_value(key="job_id", ti_key=ti_key) - else: - assert dttm is not None - job_ids = XCom.get_one( - key="job_id", - dag_id=operator.dag.dag_id, - task_id=operator.task_id, - execution_date=dttm, - ) - if not job_ids: - return None - if len(job_ids) < self.index: - return None - job_id = job_ids[self.index] - return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id) - - -Having sensors return XCOM values ---------------------------------- -In Airflow 2.3, sensor operators will be able to return XCOM values. This is achieved by returning an instance of the ``PokeReturnValue`` object at the end of the ``poke()`` method: - - .. code-block:: python - - from airflow.sensors.base import PokeReturnValue - - - class SensorWithXcomValue(BaseSensorOperator): - def poke(self, context: Context) -> Union[bool, PokeReturnValue]: - # ... - is_done = ... # set to true if the sensor should stop poking. - xcom_value = ... # return value of the sensor operator to be pushed to XCOM. - return PokeReturnValue(is_done, xcom_value) - - -To implement a sensor operator that pushes a XCOM value and supports both version 2.3 and pre-2.3, you need to explicitly push the XCOM value if the version is pre-2.3. - - .. code-block:: python - - try: - from airflow.sensors.base import PokeReturnValue - except ImportError: - PokeReturnValue = None - - - class SensorWithXcomValue(BaseSensorOperator): - def poke(self, context: Context) -> bool: - # ... - is_done = ... # set to true if the sensor should stop poking. - xcom_value = ... # return value of the sensor operator to be pushed to XCOM. - if PokeReturnValue is not None: - return PokeReturnValue(is_done, xcom_value) - else: - if is_done: - context["ti"].xcom_push(key="xcom_key", value=xcom_value) - return is_done - - -How-to Update a community provider ----------------------------------- - -See `Provider packages versioning `_ diff --git a/airflow/providers/celery/executors/celery_kubernetes_executor.py b/airflow/providers/celery/executors/celery_kubernetes_executor.py index 0522b12f41aa..72c037b684ea 100644 --- a/airflow/providers/celery/executors/celery_kubernetes_executor.py +++ b/airflow/providers/celery/executors/celery_kubernetes_executor.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +from functools import cached_property from typing import TYPE_CHECKING, Sequence from airflow.callbacks.base_callback_sink import BaseCallbackSink @@ -55,14 +56,21 @@ class CeleryKubernetesExecutor(LoggingMixin): callback_sink: BaseCallbackSink | None = None - KUBERNETES_QUEUE = conf.get("celery_kubernetes_executor", "kubernetes_queue") + @cached_property + def kubernetes_queue(self) -> str: + # lazily retrieve the value of kubernetes_queue from the configuration + # because it might need providers + from airflow.providers_manager import ProvidersManager + + ProvidersManager().initialize_providers_configuration() + return conf.get("celery_kubernetes_executor", "kubernetes_queue") def __init__(self, celery_executor: CeleryExecutor, kubernetes_executor: KubernetesExecutor): super().__init__() self._job_id: int | None = None self.celery_executor = celery_executor self.kubernetes_executor = kubernetes_executor - self.kubernetes_executor.kubernetes_queue = self.KUBERNETES_QUEUE + self.kubernetes_executor.kubernetes_queue = self.kubernetes_queue @property def queued_tasks(self) -> dict[TaskInstanceKey, QueuedTaskInstanceType]: @@ -194,16 +202,16 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task :return: any TaskInstances that were unable to be adopted """ - celery_tis = [ti for ti in tis if ti.queue != self.KUBERNETES_QUEUE] - kubernetes_tis = [ti for ti in tis if ti.queue == self.KUBERNETES_QUEUE] + celery_tis = [ti for ti in tis if ti.queue != self.kubernetes_queue] + kubernetes_tis = [ti for ti in tis if ti.queue == self.kubernetes_queue] return [ *self.celery_executor.try_adopt_task_instances(celery_tis), *self.kubernetes_executor.try_adopt_task_instances(kubernetes_tis), ] def cleanup_stuck_queued_tasks(self, tis: list[TaskInstance]) -> list[str]: - celery_tis = [ti for ti in tis if ti.queue != self.KUBERNETES_QUEUE] - kubernetes_tis = [ti for ti in tis if ti.queue == self.KUBERNETES_QUEUE] + celery_tis = [ti for ti in tis if ti.queue != self.kubernetes_queue] + kubernetes_tis = [ti for ti in tis if ti.queue == self.kubernetes_queue] return [ *self.celery_executor.cleanup_stuck_queued_tasks(celery_tis), *self.kubernetes_executor.cleanup_stuck_queued_tasks(kubernetes_tis), @@ -226,7 +234,7 @@ def _router(self, simple_task_instance: SimpleTaskInstance) -> CeleryExecutor | :param simple_task_instance: SimpleTaskInstance :return: celery_executor or kubernetes_executor """ - if simple_task_instance.queue == self.KUBERNETES_QUEUE: + if simple_task_instance.queue == self.kubernetes_queue: return self.kubernetes_executor return self.celery_executor diff --git a/airflow/providers/celery/executors/default_celery.py b/airflow/providers/celery/executors/default_celery.py index 8d546952f038..f5c23cbb81a3 100644 --- a/airflow/providers/celery/executors/default_celery.py +++ b/airflow/providers/celery/executors/default_celery.py @@ -25,6 +25,7 @@ from airflow.configuration import conf from airflow.exceptions import AirflowConfigException, AirflowException +from airflow.providers_manager import ProvidersManager def _broker_supports_visibility_timeout(url): @@ -33,6 +34,7 @@ def _broker_supports_visibility_timeout(url): log = logging.getLogger(__name__) +ProvidersManager().initialize_providers_configuration() broker_url = conf.get("celery", "BROKER_URL") broker_transport_options = conf.getsection("celery_broker_transport_options") or {} diff --git a/airflow/providers/celery/provider.yaml b/airflow/providers/celery/provider.yaml index 32682e51e5b1..26787d4694ff 100644 --- a/airflow/providers/celery/provider.yaml +++ b/airflow/providers/celery/provider.yaml @@ -60,3 +60,256 @@ sensors: executors: - airflow.providers.celery.executors.celery_executor.CeleryExecutor - airflow.providers.celery.executors.celery_kubernetes_executor.CeleryKubernetesExecutor + +config: + 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: ~ + 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.providers.celery.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: ~ + 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: ~ + 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: ~ + 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: ~ + 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: ~ + type: string + example: ~ + default: "0" + celery_config_options: + description: | + Import path for celery configuration options + version_added: ~ + type: string + example: ~ + default: "airflow.providers.celery.executors.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: ~ + 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: ~ + 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: ~ + 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: ~ + type: integer + example: ~ + default: "3" + worker_precheck: + description: | + Worker initialisation check to validate Metadata Database connection + version_added: ~ + 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: 3.3.0 + type: string + sensitive: true + example: '{"password": "password_for_redis_server"}' + default: ~ diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py index 108712d484ad..bda6df3441e8 100644 --- a/airflow/providers_manager.py +++ b/airflow/providers_manager.py @@ -136,7 +136,7 @@ def _read_schema_from_resources_or_local_file(filename: str) -> dict: try: with resource_files("airflow").joinpath(filename).open("rb") as f: schema = json.load(f) - except FileNotFoundError: + except (TypeError, FileNotFoundError): import pathlib with (pathlib.Path(__file__).parent / filename).open("rb") as f: @@ -490,12 +490,25 @@ def initialize_providers_executors(self): @provider_info_cache("config") def initialize_providers_configuration(self): """Lazy initialization of providers configuration information.""" + self._initialize_providers_configuration() + + def _initialize_providers_configuration(self): + """ + Internal method to initialize providers configuration information. + + Should be used if we do not want to trigger caching for ``initialize_providers_configuration`` method. + In some cases we might want to make sure that the configuration is initialized, but we do not want + to cache the initialization method - for example when we just want to write configuration with + providers, but it is used in the context where no providers are loaded yet we will eventually + restore the original configuration and we want the subsequent ``initialize_providers_configuration`` + method to be run in order to load the configuration for providers again. + """ self.initialize_providers_list() self._discover_config() # Now update conf with the new provider configuration from providers from airflow.configuration import conf - conf.load_provider_configuration() + conf.load_providers_configuration() @provider_info_cache("auth_backends") def initialize_providers_auth_backends(self): @@ -551,9 +564,10 @@ def _discover_all_airflow_builtin_providers_from_local_sources(self) -> None: except ImportError: log.info("You have no providers installed.") return - try: - seen = set() - for path in airflow.providers.__path__: # type: ignore[attr-defined] + + seen = set() + for path in airflow.providers.__path__: # type: ignore[attr-defined] + try: # The same path can appear in the __path__ twice, under non-normalized paths (ie. # /path/to/repo/airflow/providers and /path/to/repo/./airflow/providers) path = os.path.realpath(path) @@ -561,8 +575,8 @@ def _discover_all_airflow_builtin_providers_from_local_sources(self) -> None: continue seen.add(path) self._add_provider_info_from_local_source_files_on_path(path) - except Exception as e: - log.warning("Error when loading 'provider.yaml' files from airflow sources: %s", e) + except Exception as e: + log.warning(f"Error when loading 'provider.yaml' files from {path} airflow sources: {e}") def _add_provider_info_from_local_source_files_on_path(self, path) -> None: """ @@ -573,9 +587,14 @@ def _add_provider_info_from_local_source_files_on_path(self, path) -> None: root_path = path for folder, subdirs, files in os.walk(path, topdown=True): for filename in fnmatch.filter(files, "provider.yaml"): - package_name = "apache-airflow-providers" + folder[len(root_path) :].replace(os.sep, "-") - self._add_provider_info_from_local_source_file(os.path.join(folder, filename), package_name) - subdirs[:] = [] + try: + package_name = "apache-airflow-providers" + folder[len(root_path) :].replace(os.sep, "-") + self._add_provider_info_from_local_source_file( + os.path.join(folder, filename), package_name + ) + subdirs[:] = [] + except Exception as e: + log.warning("Error when loading 'provider.yaml' file from %s %e", folder, e) def _add_provider_info_from_local_source_file(self, path, package_name) -> None: """ diff --git a/dev/README_RELEASE_AIRFLOW.md b/dev/README_RELEASE_AIRFLOW.md index 06d98041fd14..0147f418a0ad 100644 --- a/dev/README_RELEASE_AIRFLOW.md +++ b/dev/README_RELEASE_AIRFLOW.md @@ -949,7 +949,7 @@ EOF This includes: - Modify `./scripts/ci/pre_commit/pre_commit_supported_versions.py` and let pre-commit do the job. -- For major/minor release, update version in `airflow/__init__.py`, `docs/docker-stack/` and `airflow/api_connexion/openapi/v1.yaml` to the next likely minor version release. +- For major/minor release, update version in `airflow/__main__.py`, `docs/docker-stack/` and `airflow/api_connexion/openapi/v1.yaml` to the next likely minor version release. - Update the `REVISION_HEADS_MAP` at airflow/utils/db.py to include the revision head of the release even if there are no migrations. - Sync `RELEASE_NOTES.rst` (including deleting relevant `newsfragments`) and `README.md` changes. - Updating `airflow_bug_report.yml` issue template in `.github/ISSUE_TEMPLATE/` with the new version. diff --git a/docs/apache-airflow-providers-airbyte/installing-providers-from-sources.rst b/docs/apache-airflow-providers-airbyte/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-airbyte/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-airbyte/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-alibaba/installing-providers-from-sources.rst b/docs/apache-airflow-providers-alibaba/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-alibaba/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-alibaba/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-amazon/installing-providers-from-sources.rst b/docs/apache-airflow-providers-amazon/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-amazon/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-amazon/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-beam/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-beam/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-beam/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-beam/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-cassandra/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-cassandra/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-cassandra/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-cassandra/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-drill/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-drill/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-drill/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-drill/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-druid/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-druid/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-druid/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-druid/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-flink/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-flink/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-flink/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-flink/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-hdfs/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-hdfs/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-hdfs/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-hdfs/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-hive/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-hive/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-hive/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-hive/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-impala/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-impala/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-impala/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-impala/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-kafka/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-kafka/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-kafka/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-kafka/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-kylin/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-kylin/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-kylin/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-kylin/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-livy/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-livy/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-livy/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-livy/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-pig/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-pig/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-pig/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-pig/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-pinot/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-pinot/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-pinot/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-pinot/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-spark/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-spark/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-spark/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-spark/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apache-sqoop/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-apprise/installing-providers-from-sources.rst b/docs/apache-airflow-providers-apprise/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-apprise/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-apprise/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-arangodb/installing-providers-from-sources.rst b/docs/apache-airflow-providers-arangodb/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-arangodb/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-arangodb/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-asana/installing-providers-from-sources.rst b/docs/apache-airflow-providers-asana/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-asana/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-asana/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-atlassian-jira/installing-providers-from-sources.rst b/docs/apache-airflow-providers-atlassian-jira/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-atlassian-jira/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-atlassian-jira/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-celery/configurations-ref.rst b/docs/apache-airflow-providers-celery/configurations-ref.rst new file mode 100644 index 000000000000..5885c9d91b6e --- /dev/null +++ b/docs/apache-airflow-providers-celery/configurations-ref.rst @@ -0,0 +1,18 @@ + .. 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. + +.. include:: ../exts/includes/providers-configurations-ref.rst diff --git a/docs/apache-airflow-providers-celery/index.rst b/docs/apache-airflow-providers-celery/index.rst index fbf57bead943..464f12cd92e4 100644 --- a/docs/apache-airflow-providers-celery/index.rst +++ b/docs/apache-airflow-providers-celery/index.rst @@ -34,6 +34,7 @@ :maxdepth: 1 :caption: References + Configuration Python API <_api/airflow/providers/celery/index> PyPI Repository Installing from sources diff --git a/docs/apache-airflow-providers-celery/installing-providers-from-sources.rst b/docs/apache-airflow-providers-celery/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-celery/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-celery/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-cloudant/installing-providers-from-sources.rst b/docs/apache-airflow-providers-cloudant/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-cloudant/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-cloudant/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-cncf-kubernetes/installing-providers-from-sources.rst b/docs/apache-airflow-providers-cncf-kubernetes/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst b/docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-common-sql/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-databricks/installing-providers-from-sources.rst b/docs/apache-airflow-providers-databricks/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-databricks/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-databricks/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-datadog/installing-providers-from-sources.rst b/docs/apache-airflow-providers-datadog/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-datadog/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-datadog/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-dbt-cloud/installing-providers-from-sources.rst b/docs/apache-airflow-providers-dbt-cloud/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-dbt-cloud/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-dbt-cloud/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-dingding/installing-providers-from-sources.rst b/docs/apache-airflow-providers-dingding/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-dingding/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-dingding/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-discord/installing-providers-from-sources.rst b/docs/apache-airflow-providers-discord/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-discord/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-discord/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-docker/installing-providers-from-sources.rst b/docs/apache-airflow-providers-docker/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-docker/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-docker/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-elasticsearch/installing-providers-from-sources.rst b/docs/apache-airflow-providers-elasticsearch/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-elasticsearch/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-elasticsearch/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-exasol/installing-providers-from-sources.rst b/docs/apache-airflow-providers-exasol/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-exasol/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-exasol/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-facebook/installing-providers-from-sources.rst b/docs/apache-airflow-providers-facebook/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-facebook/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-facebook/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-ftp/installing-providers-from-sources.rst b/docs/apache-airflow-providers-ftp/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-ftp/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-ftp/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-github/installing-providers-from-sources.rst b/docs/apache-airflow-providers-github/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-github/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-github/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-google/configurations-ref.rst b/docs/apache-airflow-providers-google/configurations-ref.rst deleted file mode 100644 index 9f243dc763ac..000000000000 --- a/docs/apache-airflow-providers-google/configurations-ref.rst +++ /dev/null @@ -1,71 +0,0 @@ - .. Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - .. http://www.apache.org/licenses/LICENSE-2.0 - - .. Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. - - -Configuration Reference -======================= - -This page contains the list of all the provider configurations that you -can set in ``airflow.cfg`` file or using environment variables. - -.. note:: - For more information on setting the configuration, see :doc:`apache-airflow:howto/set-config` - -.. contents:: Sections: - :local: - :depth: 1 - -.. jinja:: config_ctx - - {% for section in configs %} - - [{{ section["name"] }}] - {{ "=" * (section["name"]|length + 2) }} - - {% if section["description"] %} - {{ section["description"] }} - {% endif %} - - {% for option in section["options"] %} - - .. _config:{{ section["name"] }}__{{ option["name"] }}: - - {{ option["name"] }} - {{ "-" * option["name"]|length }} - - {% if option["version_added"] %} - .. versionadded:: {{ option["version_added"] }} - {% endif %} - - {% if option["description"] %} - {{ option["description"] }} - {% endif %} - - {% if option.get("see_also") %} - .. seealso:: {{ option["see_also"] }} - {% endif %} - - :Type: {{ option["type"] }} - :Default: ``{{ "''" if option["default"] == "" else option["default"] }}`` - :Environment Variable: ``AIRFLOW__{{ section["name"] | upper }}__{{ option["name"] | upper }}`` - {% if option["example"] %} - :Example: - ``{{ option["example"] }}`` - {% endif %} - - {% endfor %} - {% endfor %} diff --git a/docs/apache-airflow-providers-google/index.rst b/docs/apache-airflow-providers-google/index.rst index 0c2fca3e2cdc..40969f38dd91 100644 --- a/docs/apache-airflow-providers-google/index.rst +++ b/docs/apache-airflow-providers-google/index.rst @@ -46,7 +46,6 @@ :caption: References Python API <_api/airflow/providers/google/index> - Configuration .. toctree:: :hidden: diff --git a/docs/apache-airflow-providers-google/installing-providers-from-sources.rst b/docs/apache-airflow-providers-google/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-google/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-google/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-grpc/installing-providers-from-sources.rst b/docs/apache-airflow-providers-grpc/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-grpc/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-grpc/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-hashicorp/installing-providers-from-sources.rst b/docs/apache-airflow-providers-hashicorp/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-hashicorp/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-hashicorp/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-http/installing-providers-from-sources.rst b/docs/apache-airflow-providers-http/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-http/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-http/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-imap/installing-providers-from-sources.rst b/docs/apache-airflow-providers-imap/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-imap/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-imap/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-influxdb/installing-providers-from-sources.rst b/docs/apache-airflow-providers-influxdb/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-influxdb/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-influxdb/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-jdbc/installing-providers-from-sources.rst b/docs/apache-airflow-providers-jdbc/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-jdbc/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-jdbc/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-jenkins/installing-providers-from-sources.rst b/docs/apache-airflow-providers-jenkins/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-jenkins/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-jenkins/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-microsoft-azure/installing-providers-from-sources.rst b/docs/apache-airflow-providers-microsoft-azure/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-microsoft-azure/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-microsoft-azure/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-microsoft-mssql/installing-providers-from-sources.rst b/docs/apache-airflow-providers-microsoft-mssql/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-microsoft-mssql/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-microsoft-mssql/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-microsoft-psrp/installing-providers-from-sources.rst b/docs/apache-airflow-providers-microsoft-psrp/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-microsoft-psrp/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-microsoft-psrp/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-microsoft-winrm/installing-providers-from-sources.rst b/docs/apache-airflow-providers-microsoft-winrm/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-microsoft-winrm/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-microsoft-winrm/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-mongo/installing-providers-from-sources.rst b/docs/apache-airflow-providers-mongo/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-mongo/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-mongo/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-mysql/installing-providers-from-sources.rst b/docs/apache-airflow-providers-mysql/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-mysql/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-mysql/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-neo4j/installing-providers-from-sources.rst b/docs/apache-airflow-providers-neo4j/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-neo4j/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-neo4j/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-odbc/installing-providers-from-sources.rst b/docs/apache-airflow-providers-odbc/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-odbc/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-odbc/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-openfaas/installing-providers-from-sources.rst b/docs/apache-airflow-providers-openfaas/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-openfaas/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-openfaas/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-openlineage/installing-providers-from-sources.rst b/docs/apache-airflow-providers-openlineage/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-openlineage/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-openlineage/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-opsgenie/installing-providers-from-sources.rst b/docs/apache-airflow-providers-opsgenie/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-opsgenie/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-opsgenie/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-oracle/installing-providers-from-sources.rst b/docs/apache-airflow-providers-oracle/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-oracle/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-oracle/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-pagerduty/installing-providers-from-sources.rst b/docs/apache-airflow-providers-pagerduty/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-pagerduty/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-pagerduty/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-papermill/installing-providers-from-sources.rst b/docs/apache-airflow-providers-papermill/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-papermill/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-papermill/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst b/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-plexus/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-postgres/installing-providers-from-sources.rst b/docs/apache-airflow-providers-postgres/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-postgres/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-postgres/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-presto/installing-providers-from-sources.rst b/docs/apache-airflow-providers-presto/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-presto/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-presto/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-qubole/installing-providers-from-sources.rst b/docs/apache-airflow-providers-qubole/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-qubole/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-qubole/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-redis/installing-providers-from-sources.rst b/docs/apache-airflow-providers-redis/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-redis/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-redis/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-salesforce/installing-providers-from-sources.rst b/docs/apache-airflow-providers-salesforce/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-salesforce/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-salesforce/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-samba/installing-providers-from-sources.rst b/docs/apache-airflow-providers-samba/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-samba/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-samba/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-segment/installing-providers-from-sources.rst b/docs/apache-airflow-providers-segment/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-segment/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-segment/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-sendgrid/installing-providers-from-sources.rst b/docs/apache-airflow-providers-sendgrid/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-sendgrid/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-sendgrid/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-sftp/installing-providers-from-sources.rst b/docs/apache-airflow-providers-sftp/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-sftp/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-sftp/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-singularity/installing-providers-from-sources.rst b/docs/apache-airflow-providers-singularity/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-singularity/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-singularity/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-slack/installing-providers-from-sources.rst b/docs/apache-airflow-providers-slack/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-slack/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-slack/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-smtp/installing-providers-from-sources.rst b/docs/apache-airflow-providers-smtp/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-smtp/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-smtp/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-snowflake/installing-providers-from-sources.rst b/docs/apache-airflow-providers-snowflake/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-snowflake/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-snowflake/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-sqlite/installing-providers-from-sources.rst b/docs/apache-airflow-providers-sqlite/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-sqlite/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-sqlite/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-ssh/installing-providers-from-sources.rst b/docs/apache-airflow-providers-ssh/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-ssh/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-ssh/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-tableau/installing-providers-from-sources.rst b/docs/apache-airflow-providers-tableau/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-tableau/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-tableau/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst b/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-tabular/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-telegram/installing-providers-from-sources.rst b/docs/apache-airflow-providers-telegram/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-telegram/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-telegram/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-trino/installing-providers-from-sources.rst b/docs/apache-airflow-providers-trino/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-trino/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-trino/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-vertica/installing-providers-from-sources.rst b/docs/apache-airflow-providers-vertica/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-vertica/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-vertica/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst b/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-yandex/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers-zendesk/installing-providers-from-sources.rst b/docs/apache-airflow-providers-zendesk/installing-providers-from-sources.rst index 1c90205d15b3..b4e730f4ff21 100644 --- a/docs/apache-airflow-providers-zendesk/installing-providers-from-sources.rst +++ b/docs/apache-airflow-providers-zendesk/installing-providers-from-sources.rst @@ -15,4 +15,4 @@ specific language governing permissions and limitations under the License. -.. include:: ../installing-providers-from-sources.rst +.. include:: ../exts/includes/installing-providers-from-sources.rst diff --git a/docs/apache-airflow-providers/core-extensions/configurations.rst b/docs/apache-airflow-providers/core-extensions/configurations.rst new file mode 100644 index 000000000000..c90f28a730c8 --- /dev/null +++ b/docs/apache-airflow-providers/core-extensions/configurations.rst @@ -0,0 +1,31 @@ + .. 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. + +Configurations +-------------- + +This is a summary of all Apache Airflow Community provider custom configurations. + + +You can take a look at Configuration available in the core Airflow and how to set the +configuration options in :doc:`apache-airflow:configurations-ref`. + +Those provided by the community-managed providers: + +.. airflow-configurations:: + :tags: None + :header-separator: " diff --git a/docs/apache-airflow-providers/howto/create-custom-providers.rst b/docs/apache-airflow-providers/howto/create-custom-providers.rst new file mode 100644 index 000000000000..6fe27674f226 --- /dev/null +++ b/docs/apache-airflow-providers/howto/create-custom-providers.rst @@ -0,0 +1,268 @@ + .. 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. + +How to create your own provider +------------------------------- + +Custom provider packages +'''''''''''''''''''''''' + +You can develop and release your own providers. Your custom operators, hooks, sensors, transfer operators +can be packaged together in a standard airflow package and installed using the same mechanisms. +Moreover they can also use the same mechanisms to extend the Airflow Core with auth backends, +custom connections, logging, secret backends and extra operator links as described in the previous chapter. + +As mentioned in the `Providers `_ +documentation, custom providers can extend Airflow core - they can add extra links to operators as well +as custom connections. You can use build your own providers and install them as packages if you would like +to use the mechanism for your own, custom providers. + +How to create a provider +'''''''''''''''''''''''' + +Adding a provider to Airflow is just a matter of building a Python package and adding the right meta-data to +the package. We are using standard mechanism of python to define +`entry points `_ . Your package +needs to define appropriate entry-point ``apache_airflow_provider`` which has to point to a callable +implemented by your package and return a dictionary containing the list of discoverable capabilities +of your package. The dictionary has to follow the +`json-schema specification `_. + +Most of the schema provides extension point for the documentation (which you might want to also use for +your own purpose) but the important fields from the extensibility point of view are those: + +Displaying package information in CLI/API: + +* ``package-name`` - Name of the package for the provider. + +* ``name`` - Human-friendly name of the provider. + +* ``description`` - Additional description of the provider. + +* ``version`` - List of versions of the package (in reverse-chronological order). The first version in the + list is the current package version. It is taken from the version of package installed, not from the + provider_info information. + +Exposing customized functionality to the Airflow's core: + +* ``extra-links`` - this field should contain the list of all the operator class names that are adding extra links + capability. See :doc:`apache-airflow:howto/define-extra-link` for description of how to add extra link + capability to the operators of yours. + +* ``connection-types`` - this field should contain the list of all the connection types together with hook + class names implementing those custom connection types (providing custom extra fields and + custom field behaviour). This field is available as of Airflow 2.2.0 and it replaces deprecated + ``hook-class-names``. See :doc:`apache-airflow:howto/connection` for more details. + +* ``secret-backends`` - this field should contain the list of all the secret backends class names that the + provider provides. See :doc:`apache-airflow:security/secrets/secrets-backend/index` for description of how + to add. + +* ``task-decorators`` - this field should contain the list of dictionaries of name/path where the decorators + are available. See :doc:`apache-airflow:howto/create-custom-decorator` for description of how to add + custom decorators. + +* ``logging`` - this field should contain the list of all the logging handler class names that the + provider provides. See :doc:`apache-airflow:administration-and-deployment/logging-monitoring/logging-tasks` + for description of the logging handlers. + +* ``auth-backends`` - this field should contain the authentication backend module names for API/UI. + See :doc:`apache-airflow:security/api` for description of the auth backends. + +* ``notifications`` - this field should contain the notification classes. + See :doc:`apache-airflow:howto/notifications` for description of the notifications. + +* ``executors`` - this field should contain the executor class class names. + See :doc:`apache-airflow:core-concepts/executor/index` for description of the executors. + +* ``config`` - this field should contain dictionary that should conform to the + ``airflow/config_templates/config.yml.schema.json`` with configuration contributed by the providers + See :doc:`apache-airflow:howto/set-config` for details about setting configuration. + +.. note:: Deprecated values + + * ``hook-class-names`` (deprecated) - this field should contain the list of all hook class names that provide + custom connection types with custom extra fields and field behaviour. The ``hook-class-names`` array + is deprecated as of Airflow 2.2.0 (for optimization reasons) and will be removed in Airflow 3. If your + providers are targeting Airflow 2.2.0+ you do not have to include the ``hook-class-names`` array, if + you want to also target earlier versions of Airflow 2, you should include both ``hook-class-names`` and + ``connection-types`` arrays. See :doc:`apache-airflow:howto/connection` for more details. + + +When your providers are installed you can query the installed providers and their capabilities with the +``airflow providers`` command. This way you can verify if your providers are properly recognized and whether +they define the extensions properly. See :doc:`apache-airflow:cli-and-env-variables-ref` for details of available CLI +sub-commands. + +When you write your own provider, consider following the +`Naming conventions for provider packages `_ + +Special considerations +'''''''''''''''''''''' + +Optional provider features +-------------------------- + + .. versionadded:: 2.3.0 + + This feature is available in Airflow 2.3+. + +Some providers might provide optional features, which are only available when some packages or libraries +are installed. Such features will typically result in ``ImportErrors``; however, those import errors +should be silently ignored rather than pollute the logs of Airflow with false warnings. False warnings +are a very bad pattern, as they tend to turn into blind spots, so avoiding false warnings is encouraged. +However, until Airflow 2.3, Airflow had no mechanism to selectively ignore "known" ImportErrors. So +Airflow 2.1 and 2.2 silently ignored all ImportErrors coming from providers with actually lead to +ignoring even important import errors - without giving the clue to Airflow users that there is something +missing in provider dependencies. + +Using Providers with dynamic task mapping +----------------------------------------- + +Airflow 2.3 added `Dynamic Task Mapping `_ +and it added the possibility of assigning a unique key to each task. Which means that when such dynamically +mapped task wants to retrieve a value from XCom (for example in case an extra link should calculated) +it should always check if the ti_key value passed is not None an only then retrieve the XCom value using +XCom.get_value. This allows to keep backwards compatibility with earlier versions of Airflow. + +Typical code to access XCom Value in providers that want to keep backwards compatibility should look similar to +this (note the ``if ti_key is not None:`` condition). + + .. code-block:: python + + def get_link( + self, + operator: BaseOperator, + dttm: datetime | None = None, + ti_key: "TaskInstanceKey" | None = None, + ): + if ti_key is not None: + job_ids = XCom.get_value(key="job_id", ti_key=ti_key) + else: + assert dttm is not None + job_ids = XCom.get_one( + key="job_id", + dag_id=operator.dag.dag_id, + task_id=operator.task_id, + execution_date=dttm, + ) + if not job_ids: + return None + if len(job_ids) < self.index: + return None + job_id = job_ids[self.index] + return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id) + + + + +FAQ for custom providers +'''''''''''''''''''''''' + +**When I write my own provider, do I need to do anything special to make it available to others?** + +You do not need to do anything special besides creating the ``apache_airflow_provider`` entry point +returning properly formatted meta-data - dictionary with ``extra-links`` and ``connection-types`` fields +(and deprecated ``hook-class-names`` field if you are also targeting versions of Airflow before 2.2.0). + +Anyone who runs airflow in an environment that has your Python package installed will be able to use the +package as a provider package. + + +**Should I name my provider specifically or should it be created in ``airflow.providers`` package?** + +We have quite a number (>80) of providers managed by the community and we are going to maintain them +together with Apache Airflow. All those providers have well-defined structured and follow the +naming conventions we defined and they are all in ``airflow.providers`` package. If your intention is +to contribute your provider, then you should follow those conventions and make a PR to Apache Airflow +to contribute to it. But you are free to use any package name as long as there are no conflicts with other +names, so preferably choose package that is in your "domain". + +**What do I need to do to turn a package into a provider?** + +You need to do the following to turn an existing Python package into a provider (see below for examples): + +* Add the ``apache_airflow_provider`` entry point in the ``setup.cfg`` - this tells airflow where to get + the required provider metadata +* Create the function that you refer to in the first step as part of your package: this functions returns a + dictionary that contains all meta-data about your provider package +* If you want Airflow to link to documentation of your Provider in the providers page, make sure + to add "project-url/documentation" `metadata `_ to your package. + This will also add link to your documentation in PyPI. +* note that the dictionary should be compliant with ``airflow/provider_info.schema.json`` JSON-schema + specification. The community-managed providers have more fields there that are used to build + documentation, but the requirement for runtime information only contains several fields which are defined + in the schema: + +.. exampleinclude:: /../../airflow/provider_info.schema.json + :language: json + +Example ``setup.cfg``: + +.. code-block:: cfg + + [options.entry_points] + # the function get_provider_info is defined in myproviderpackage.somemodule + apache_airflow_provider= + provider_info=myproviderpackage.somemodule:get_provider_info + +Example ``myproviderpackage/somemodule.py``: + +.. code-block:: Python + + def get_provider_info(): + return { + "package-name": "my-package-name", + "name": "name", + "description": "a description", + "hook-class-names": [ + "myproviderpackage.hooks.source.SourceHook", + ], + } + + + +**Is there a convention for a connection id and type?** + +Very good question. Glad that you asked. We usually follow the convention ``_default`` for connection +id and just ```` for connection type. Few examples: + +* ``google_cloud_default`` id and ``google_cloud_platform`` type +* ``aws_default`` id and ``aws`` type + +You should follow this convention. It is important, to use unique names for connection type, +so it should be unique for your provider. If two providers try to add connection with the same type +only one of them will succeed. + +**Can I contribute my own provider to Apache Airflow?** + +The answer depends on the provider. We have a policy for that in the +`PROVIDERS.rst `_ developer documentation. + +**Can I advertise my own provider to Apache Airflow users and share it with others as package in PyPI?** + +Absolutely! We have an `Ecosystem `_ area on our website where +we share non-community managed extensions and work for Airflow. Feel free to make a PR to the page and +add we will evaluate and merge it when we see that such provider can be useful for the community of +Airflow users. + +**Can I charge for the use of my provider?** + +This is something that is outside of our control and domain. As an Apache project, we are +commercial-friendly and there are many businesses built around Apache Airflow and many other +Apache projects. As a community, we provide all the software for free and this will never +change. What 3rd-party developers are doing is not under control of Apache Airflow community. diff --git a/docs/apache-airflow-providers/index.rst b/docs/apache-airflow-providers/index.rst index 0c960a8eeef4..8b8390e5e6a8 100644 --- a/docs/apache-airflow-providers/index.rst +++ b/docs/apache-airflow-providers/index.rst @@ -32,10 +32,15 @@ for ``Apache Airflow`` are designed in the way that you can write your own provi develop your own providers - the providers you build have exactly the same capability as the providers written by the community, so you can release and share those providers with others. -The full list of community managed providers is available at + +If you want to learn how to build your own custom provider, you can find all the information about it at +:doc:`/howto/create-custom-providers`. + + +The full list of all the community managed providers is available at `Providers Index `_. -You can also see index of all community provider's operators and hooks in +You can also see index of all the community provider's operators and hooks in :doc:`/operators-and-hooks-ref/index` Extending Airflow core functionality @@ -48,9 +53,17 @@ describe all the custom capabilities. Airflow automatically discovers which providers add those additional capabilities and, once you install provider package and re-start Airflow, those become automatically available to Airflow Users. -The summary of the core functionalities that can be extended are available in +The summary of all the core functionalities that can be extended are available in :doc:`/core-extensions/index`. +Configuration +''''''''''''' + +Providers can have their own configuration options which allow you to configure how they work: + +You can see all community-managed providers with their own configuration in +:doc:`/core-extensions/configurations` + Auth backends ''''''''''''' @@ -66,7 +79,7 @@ Custom connections The providers can add custom connection types, extending connection form and handling custom form field behaviour for the connections defined by the provider. -You can see all custom connections available via community-managed providers in +You can see all the custom connections available via community-managed providers in :doc:`/core-extensions/connections`. Extra links @@ -97,7 +110,7 @@ Secret backends Airflow has the capability of reading connections, variables and configuration from Secret Backends rather than from its own Database. -You can see all secret backends available via community-managed providers in +You can see all the secret backends available via community-managed providers in :doc:`/core-extensions/secrets-backends`. Notifications @@ -132,6 +145,8 @@ Providers have the same capacity - no matter if they are provided by the communi third-party providers. This chapter explains how community managed providers are versioned and released and how you can create your own providers. +.. _providers:community-maintained-providers: + Community maintained providers '''''''''''''''''''''''''''''' @@ -167,224 +182,18 @@ provider packages are automatically documented in the release notes of every pro Airflow 1.10. The last release of backport providers was done on March 17, 2021 and the backport providers will no longer be released, since Airflow 1.10 has reached End-Of-Life as of June 17, 2021. -If you want to contribute to ``Apache Airflow``, you can see how to build and extend community -managed providers in :doc:`howto/create-update-providers`. - -.. _providers:community-maintained-providers: - -Custom provider packages -'''''''''''''''''''''''' - -You can develop and release your own providers. Your custom operators, hooks, sensors, transfer operators -can be packaged together in a standard airflow package and installed using the same mechanisms. -Moreover they can also use the same mechanisms to extend the Airflow Core with auth backends, -custom connections, logging, secret backends and extra operator links as described in the previous chapter. - -How to create your own provider -------------------------------- - -As mentioned in the `Providers `_ -documentation, custom providers can extend Airflow core - they can add extra links to operators as well -as custom connections. You can use build your own providers and install them as packages if you would like -to use the mechanism for your own, custom providers. - -Adding a provider to Airflow is just a matter of building a Python package and adding the right meta-data to -the package. We are using standard mechanism of python to define -`entry points `_ . Your package -needs to define appropriate entry-point ``apache_airflow_provider`` which has to point to a callable -implemented by your package and return a dictionary containing the list of discoverable capabilities -of your package. The dictionary has to follow the -`json-schema specification `_. - -Most of the schema provides extension point for the documentation (which you might want to also use for -your own purpose) but the important fields from the extensibility point of view are those: - -Displaying package information in CLI/API: - -* ``package-name`` - Name of the package for the provider. - -* ``name`` - Human-friendly name of the provider. - -* ``description`` - Additional description of the provider. - -* ``version`` - List of versions of the package (in reverse-chronological order). The first version in the - list is the current package version. It is taken from the version of package installed, not from the - provider_info information. - -Exposing customized functionality to the Airflow's core: - -* ``extra-links`` - this field should contain the list of all operator class names that are adding extra links - capability. See :doc:`apache-airflow:howto/define-extra-link` for description of how to add extra link - capability to the operators of yours. - -* ``connection-types`` - this field should contain the list of all connection types together with hook - class names implementing those custom connection types (providing custom extra fields and - custom field behaviour). This field is available as of Airflow 2.2.0 and it replaces deprecated - ``hook-class-names``. See :doc:`apache-airflow:howto/connection` for more details - -* ``hook-class-names`` (deprecated) - this field should contain the list of all hook class names that provide - custom connection types with custom extra fields and field behaviour. The ``hook-class-names`` array - is deprecated as of Airflow 2.2.0 (for optimization reasons) and will be removed in Airflow 3. If your - providers are targeting Airflow 2.2.0+ you do not have to include the ``hook-class-names`` array, if - you want to also target earlier versions of Airflow 2, you should include both ``hook-class-names`` and - ``connection-types`` arrays. See :doc:`apache-airflow:howto/connection` for more details. - - -When your providers are installed you can query the installed providers and their capabilities with the -``airflow providers`` command. This way you can verify if your providers are properly recognized and whether -they define the extensions properly. See :doc:`apache-airflow:cli-and-env-variables-ref` for details of available CLI -sub-commands. - -When you write your own provider, consider following the -`Naming conventions for provider packages `_ - - -FAQ for Airflow and Providers ------------------------------ - -Upgrading Airflow 2.0 and Providers -''''''''''''''''''''''''''''''''''' - -**When upgrading to a new Airflow version such as 2.0, but possibly 2.0.1 and beyond, is the best practice -to also upgrade provider packages at the same time?** - -It depends on your use case. If you have automated or semi-automated verification of your installation, -that you can run a new version of Airflow including all provider packages, then definitely go for it. -If you rely more on manual testing, it is advised that you upgrade in stages. Depending on your choice -you can either upgrade all used provider packages first, and then upgrade Airflow Core or the other way -round. The first approach - when you first upgrade all providers is probably safer, as you can do it -incrementally, step-by-step replacing provider by provider in your environment. - -Creating your own providers -''''''''''''''''''''''''''' - -**When I write my own provider, do I need to do anything special to make it available to others?** - -You do not need to do anything special besides creating the ``apache_airflow_provider`` entry point -returning properly formatted meta-data - dictionary with ``extra-links`` and ``connection-types`` fields -(and deprecated ``hook-class-names`` field if you are also targeting versions of Airflow before 2.2.0). - -Anyone who runs airflow in an environment that has your Python package installed will be able to use the -package as a provider package. - -**What do I need to do to turn a package into a provider?** - -You need to do the following to turn an existing Python package into a provider (see below for examples): - -* Add the ``apache_airflow_provider`` entry point in the ``setup.cfg`` - this tells airflow where to get - the required provider metadata -* Create the function that you refer to in the first step as part of your package: this functions returns a - dictionary that contains all meta-data about your provider package -* If you want Airflow to link to documentation of your Provider in the providers page, make sure - to add "project-url/documentation" `metadata `_ to your package. - This will also add link to your documentation in PyPI. -* note that the dictionary should be compliant with ``airflow/provider_info.schema.json`` JSON-schema - specification. The community-managed providers have more fields there that are used to build - documentation, but the requirement for runtime information only contains several fields which are defined - in the schema: - -.. exampleinclude:: /../../airflow/provider_info.schema.json - :language: json - -Example ``setup.cfg``: - -.. code-block:: cfg - - [options.entry_points] - # the function get_provider_info is defined in myproviderpackage.somemodule - apache_airflow_provider= - provider_info=myproviderpackage.somemodule:get_provider_info - -Example ``myproviderpackage/somemodule.py``: - -.. code-block:: Python - - def get_provider_info(): - return { - "package-name": "my-package-name", - "name": "name", - "description": "a description", - "hook-class-names": [ - "myproviderpackage.hooks.source.SourceHook", - ], - } - -**How do provider packages work under the hood?** - -When running Airflow with your provider package, there will be (at least) three components to your airflow installation: - -* The installation itself (for example, a ``venv`` where you installed airflow with ``pip install apache-airflow``) - together with the related files (e.g. ``dags`` folder) -* The ``apache-airflow`` package -* Your own ``myproviderpackage`` package that is independent of ``apache-airflow`` or your airflow installation, which - can be a local Python package (that you install via ``pip install -e /path/to/my-package``), a normal pip package - (``pip install myproviderpackage``), or any other type of Python package - -In the ``myproviderpackage`` package you need to add the entry point and provide the appropriate metadata as described above. -If you have done that, airflow does the following at runtime: - -* Loop through ALL packages installed in your environment / ``venv`` -* For each package, if the package's ``setup.cfg`` has a section ``[options.entry_points]``, and if that section has a value - for ``apache_airflow_provider``, then get the value for ``provider_info``, e.g. ``myproviderpackage.somemodule:get_provider_info`` -* That value works like an import statement: ``myproviderpackage.somemodule:get_provider_info`` translates to something like - ``from myproviderpackage.somemodule import get_provider_info``, and the ``get_provider_info`` that is being imported should be a - callable, i.e. a function -* This function should return a dictionary with metadata -* If you have custom connection types as part of your package, that metadata will including a field called ``hook-class-names`` - which should be a list of strings of your custom hooks - those strings should also be in an import-like format, e.g. - ``myproviderpackage.hooks.source.SourceHook`` means that there is a class ``SourceHook`` in ``myproviderpackage/hooks/source.py`` - - airflow then imports these hooks and looks for the functions ``get_ui_field_behaviour`` and ``get_connection_form_widgets`` - (both optional) as well as the attributes ``conn_type`` and ``hook_name`` to create the custom connection type in the airflow UI - -**Should I name my provider specifically or should it be created in ``airflow.providers`` package?** - -We have quite a number (>80) of providers managed by the community and we are going to maintain them -together with Apache Airflow. All those providers have well-defined structured and follow the -naming conventions we defined and they are all in ``airflow.providers`` package. If your intention is -to contribute your provider, then you should follow those conventions and make a PR to Apache Airflow -to contribute to it. But you are free to use any package name as long as there are no conflicts with other -names, so preferably choose package that is in your "domain". - -**Is there a convention for a connection id and type?** - -Very good question. Glad that you asked. We usually follow the convention ``_default`` for connection -id and just ```` for connection type. Few examples: - -* ``google_cloud_default`` id and ``google_cloud_platform`` type -* ``aws_default`` id and ``aws`` type - -You should follow this convention. It is important, to use unique names for connection type, -so it should be unique for your provider. If two providers try to add connection with the same type -only one of them will succeed. - -**Can I contribute my own provider to Apache Airflow?** - -The answer depends on the provider. We have a policy for that in the -`PROVIDERS.rst `_ developer documentation. - -**Can I advertise my own provider to Apache Airflow users and share it with others as package in PyPI?** - -Absolutely! We have an `Ecosystem `_ area on our website where -we share non-community managed extensions and work for Airflow. Feel free to make a PR to the page and -add we will evaluate and merge it when we see that such provider can be useful for the community of -Airflow users. - -**Can I charge for the use of my provider?** - -This is something that is outside of our control and domain. As an Apache project, we are -commercial-friendly and there are many businesses built around Apache Airflow and many other -Apache projects. As a community, we provide all the software for free and this will never -change. What 3rd-party developers are doing is not under control of Apache Airflow community. +If you want to contribute to ``Apache Airflow``, you can see how to build and extend community +managed providers in ``https://github.com/apache/airflow/blob/main/airflow/providers/CREATING_COMMUNITY_PROVIDERS.rst``. .. toctree:: :hidden: :maxdepth: 2 Providers + Installing from PyPI + Installing from sources + Create custom providers Packages Operators and hooks Core Extensions - Update community providers - Installing from sources - Installing from PyPI diff --git a/docs/apache-airflow-providers/redirects.txt b/docs/apache-airflow-providers/redirects.txt new file mode 100644 index 000000000000..a6f42e867073 --- /dev/null +++ b/docs/apache-airflow-providers/redirects.txt @@ -0,0 +1,19 @@ +# 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. + +# Administration and deployment security -> security +howto/create-update-providers.rst index.rst diff --git a/docs/apache-airflow/configurations-ref.rst b/docs/apache-airflow/configurations-ref.rst index 09f7fd427047..eb34b10c3592 100644 --- a/docs/apache-airflow/configurations-ref.rst +++ b/docs/apache-airflow/configurations-ref.rst @@ -17,7 +17,7 @@ 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. @@ -32,79 +32,12 @@ generated using the secret key has a short expiry time though - make sure that t that you run airflow components on is synchronized (for example using ntpd) otherwise you might get "forbidden" errors when the logs are accessed. -.. note:: - For more information on setting the configuration, see :doc:`howto/set-config` - -.. contents:: Sections: - :local: - :depth: 1 - -.. jinja:: config_ctx - - {% for section_name, section in configs.items() %} - - .. _config:{{ section_name }}: - - [{{ section_name }}] - {{ "=" * (section_name|length + 2) }} - - {% if 'renamed' in section %} - *Renamed in version {{ section['renamed']['version'] }}, previous name was {{ section['renamed']['previous_name'] }}* - {% endif %} - - {% if section["description"] %} - {{ section["description"] }} - {% endif %} - - {% for option_name, option in section["options"].items() %} - - .. _config:{{ section_name }}__{{ option_name }}: - - {{ option_name }} - {{ "-" * option_name|length }} +Some of the providers have their own configuration options, you will find details of their configuration +in the provider's documentation. The pre-installed providers that you may want to configure are: - {% if option["version_added"] %} - .. versionadded:: {{ option["version_added"] }} - {% endif %} +* :doc:`Configuration Reference for Celery Provider ` - {% if option["description"] %} - {{ option["description"] }} - {% endif %} - - {% if option.get("see_also") %} - .. seealso:: {{ option["see_also"] }} - {% endif %} - - :Type: {{ option["type"] }} - :Default: ``{{ "''" if option["default"] == "" else option["default"] }}`` - {% if option.get("sensitive") %} - :Environment Variables: - ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}`` - - ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}_CMD`` - - ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}_SECRET`` - {% else %} - :Environment Variable: ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}`` - {% endif %} - {% if option["example"] %} - :Example: - ``{{ option["example"] }}`` - {% endif %} - - {% endfor %} - - {% if section_name in deprecated_options %} - - {% for deprecated_option_name, (new_section_name, new_option_name, since_version) in deprecated_options[section_name].items() %} - .. _config:{{ section_name }}__{{ deprecated_option_name }}: - - {{ deprecated_option_name }} (Deprecated) - {{ "-" * (deprecated_option_name + " (Deprecated)")|length }} - - .. deprecated:: {{ since_version }} - The option has been moved to :ref:`{{ new_section_name }}.{{ new_option_name }} ` - {% endfor %} - {% endif %} +.. note:: + For more information see :doc:`/howto/set-config`. - {% endfor %} +.. include:: ../exts/includes/sections-and-options.rst diff --git a/docs/apache-airflow/core-concepts/executor/celery.rst b/docs/apache-airflow/core-concepts/executor/celery.rst index ca137fa8e3c3..f69da1570fc3 100644 --- a/docs/apache-airflow/core-concepts/executor/celery.rst +++ b/docs/apache-airflow/core-concepts/executor/celery.rst @@ -29,6 +29,8 @@ change your ``airflow.cfg`` to point the executor parameter to For more information about setting up a Celery broker, refer to the exhaustive `Celery documentation on the topic `_. +The configuration parameters of the Celery Executor can be found in :doc:`apache-airflow-providers-celery:configurations-ref`. + Here are a few imperative requirements for your workers: - ``airflow`` needs to be installed, and the CLI needs to be in the path diff --git a/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst b/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst index 720fe7e743dd..1eaf9bda5843 100644 --- a/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst @@ -28,6 +28,8 @@ An executor is chosen to run a task based on the task's queue. ``CeleryKubernetesExecutor`` inherits the scalability of the ``CeleryExecutor`` to handle the high load at the peak time and runtime isolation of the ``KubernetesExecutor``. +The configuration parameters of the Celery Executor can be found in :doc:`apache-airflow-providers-celery:configurations-ref`. + When to use CeleryKubernetesExecutor #################################### diff --git a/docs/apache-airflow/howto/create-custom-decorator.rst b/docs/apache-airflow/howto/create-custom-decorator.rst index dc7efd21c44c..81312b2335f7 100644 --- a/docs/apache-airflow/howto/create-custom-decorator.rst +++ b/docs/apache-airflow/howto/create-custom-decorator.rst @@ -63,7 +63,8 @@ tasks. The steps to create and register ``@task.foo`` are: 3. Register your new decorator in get_provider_info of your provider - Finally, add a key-value ``task-decorators`` to the dict returned from the provider entrypoint. This should be + Finally, add a key-value ``task-decorators`` to the dict returned from the provider entrypoint as described + in :doc:`apache-airflow-providers:howto/create-custom-providers`. This should be a list with each item containing ``name`` and ``class-name`` keys. When Airflow starts, the ``ProviderManager`` class will automatically import this value and ``task.foo`` will work as a new decorator! diff --git a/docs/apache-airflow/howto/setup-and-teardown.rst b/docs/apache-airflow/howto/setup-and-teardown.rst index c897031a6c6c..08eb4157eff3 100644 --- a/docs/apache-airflow/howto/setup-and-teardown.rst +++ b/docs/apache-airflow/howto/setup-and-teardown.rst @@ -106,7 +106,7 @@ In this case, everything downstream of create_cluster is assumed to require it. Now, Airflow can infer that other_task does not require create_cluster, so if we clear other_task, create_cluster will not also be cleared. -In that example, we (in our pretend docs land) actually wanted to delete the cluster. But supposing we didn't, and we just wanted to say "other_task does not require create_cluster", then we could use an EmptyOperator to limit the setup's scope: +In that example, we (in our pretend docs land) actually wanted to delete the cluster. But supposing we did not, and we just wanted to say "other_task does not require create_cluster", then we could use an EmptyOperator to limit the setup's scope: .. code-block:: python diff --git a/docs/apache-airflow/tutorial/taskflow.rst b/docs/apache-airflow/tutorial/taskflow.rst index f39eb78fac9a..e007f263f919 100644 --- a/docs/apache-airflow/tutorial/taskflow.rst +++ b/docs/apache-airflow/tutorial/taskflow.rst @@ -360,11 +360,51 @@ Notes on using the operator: Using the TaskFlow API with Sensor operators -------------------------------------------- + You can apply the ``@task.sensor`` decorator to convert a regular Python function to an instance of the BaseSensorOperator class. The Python function implements the poke logic and returns an instance of -the ``PokeReturnValue`` class as the ``poke()`` method in the BaseSensorOperator does. The ``PokeReturnValue`` is -a new feature in Airflow 2.3 that allows a sensor operator to push an XCom value as described in -section "Having sensors return XCOM values" of :doc:`apache-airflow-providers:howto/create-update-providers`. +the ``PokeReturnValue`` class as the ``poke()`` method in the BaseSensorOperator does. +In Airflow 2.3, sensor operators will be able to return XCOM values. This is achieved by returning +an instance of the ``PokeReturnValue`` object at the end of the ``poke()`` method: + + .. code-block:: python + + from airflow.sensors.base import PokeReturnValue + + + class SensorWithXcomValue(BaseSensorOperator): + def poke(self, context: Context) -> Union[bool, PokeReturnValue]: + # ... + is_done = ... # set to true if the sensor should stop poking. + xcom_value = ... # return value of the sensor operator to be pushed to XCOM. + return PokeReturnValue(is_done, xcom_value) + + +To implement a sensor operator that pushes a XCOM value and supports both version 2.3 and +pre-2.3, you need to explicitly push the XCOM value if the version is pre-2.3. + + .. code-block:: python + + try: + from airflow.sensors.base import PokeReturnValue + except ImportError: + PokeReturnValue = None + + + class SensorWithXcomValue(BaseSensorOperator): + def poke(self, context: Context) -> bool: + # ... + is_done = ... # set to true if the sensor should stop poking. + xcom_value = ... # return value of the sensor operator to be pushed to XCOM. + if PokeReturnValue is not None: + return PokeReturnValue(is_done, xcom_value) + else: + if is_done: + context["ti"].xcom_push(key="xcom_key", value=xcom_value) + return is_done + + + Alternatively in cases where the sensor doesn't need to push XCOM values: both ``poke()`` and the wrapped function can return a boolean-like value where ``True`` designates the sensor's operation as complete and diff --git a/docs/conf.py b/docs/conf.py index a8939833fd6b..0618a4401d33 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -407,8 +407,10 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): ).groups(0)[0] ) -# Jinja context -if PACKAGE_NAME == "apache-airflow": + +def get_configs_and_deprecations( + package_name: str, +) -> tuple[dict[str, dict[str, tuple[str, str, str]]], dict[str, dict[str, tuple[str, str, str]]]]: deprecated_options: dict[str, dict[str, tuple[str, str, str]]] = defaultdict(dict) for (section, key), ( (deprecated_section, deprecated_key, since_version) @@ -419,7 +421,12 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): for deprecated_section, deprecated_key, since_version in deprecated: deprecated_options[deprecated_section][deprecated_key] = section, key, since_version - configs = retrieve_configuration_description() + if package_name == "apache-airflow": + configs = retrieve_configuration_description(include_providers=False) + else: + configs = retrieve_configuration_description( + include_airflow=False, include_providers=True, selected_provider=package_name + ) # We want the default/example we show in the docs to reflect the value _after_ # the config has been templated, not before @@ -440,7 +447,12 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): configs = {k: v for k, v in sorted(configs.items())} for section in deprecated_options: deprecated_options[section] = {k: v for k, v in sorted(deprecated_options[section].items())} + return configs, deprecated_options + +# Jinja context +if PACKAGE_NAME == "apache-airflow": + configs, deprecated_options = get_configs_and_deprecations(PACKAGE_NAME) jinja_contexts = { "config_ctx": {"configs": configs, "deprecated_options": deprecated_options}, "quick_start_ctx": { @@ -453,18 +465,13 @@ def _get_rst_filepath_from_path(filepath: pathlib.Path): }, } elif PACKAGE_NAME.startswith("apache-airflow-providers-"): - - def _load_config(): - file_path = PACKAGE_DIR / "config_templates" / "config.yml" - if not file_path.exists(): - return {} - - with file_path.open() as f: - return yaml.safe_load(f) - - config = _load_config() + configs, deprecated_options = get_configs_and_deprecations(PACKAGE_NAME) jinja_contexts = { - "config_ctx": {"configs": config}, + "config_ctx": { + "configs": configs, + "deprecated_options": deprecated_options, + "package_name": PACKAGE_NAME, + }, "official_download_page": { "base_url": "https://downloads.apache.org/airflow/providers", "closer_lua_url": "https://www.apache.org/dyn/closer.lua/airflow/providers", diff --git a/docs/installing-providers-from-sources.rst b/docs/exts/includes/installing-providers-from-sources.rst similarity index 100% rename from docs/installing-providers-from-sources.rst rename to docs/exts/includes/installing-providers-from-sources.rst diff --git a/docs/exts/includes/providers-configurations-ref.rst b/docs/exts/includes/providers-configurations-ref.rst new file mode 100644 index 000000000000..d696e126bbc5 --- /dev/null +++ b/docs/exts/includes/providers-configurations-ref.rst @@ -0,0 +1,29 @@ + .. 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. + +Configuration Reference +....................... + +.. jinja:: config_ctx + + This page contains the list of all available Airflow configurations for the + ``{{ package_name }}`` provider that can be set in the ``airflow.cfg`` file or using environment variables. + + .. note:: + For more information see :doc:`apache-airflow:howto/set-config`. + +.. include:: ../exts/includes/sections-and-options.rst diff --git a/docs/exts/includes/sections-and-options.rst b/docs/exts/includes/sections-and-options.rst new file mode 100644 index 000000000000..497223fcee1e --- /dev/null +++ b/docs/exts/includes/sections-and-options.rst @@ -0,0 +1,90 @@ + .. 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. + +.. contents:: Sections: + :local: + :depth: 1 + +.. jinja:: config_ctx + + {% for section_name, section in configs.items() %} + + .. _config:{{ section_name }}: + + [{{ section_name }}] + {{ "=" * (section_name|length + 2) }} + + {% if 'renamed' in section %} + *Renamed in version {{ section['renamed']['version'] }}, previous name was {{ section['renamed']['previous_name'] }}* + {% endif %} + + {% if section["description"] %} + {{ section["description"] }} + {% endif %} + + {% for option_name, option in section["options"].items() %} + + .. _config:{{ section_name }}__{{ option_name }}: + + {{ option_name }} + {{ "-" * option_name|length }} + + {% if option["version_added"] %} + .. versionadded:: {{ option["version_added"] }} + {% endif %} + + {% if option["description"] %} + {{ option["description"] }} + {% endif %} + + {% if option.get("see_also") %} + .. seealso:: {{ option["see_also"] }} + {% endif %} + + :Type: {{ option["type"] }} + :Default: ``{{ "''" if option["default"] == "" else option["default"] }}`` + {% if option.get("sensitive") %} + :Environment Variables: + ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}`` + + ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}_CMD`` + + ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}_SECRET`` + {% else %} + :Environment Variable: ``AIRFLOW__{{ section_name | upper }}__{{ option_name | upper }}`` + {% endif %} + {% if option["example"] %} + :Example: + ``{{ option["example"] }}`` + {% endif %} + + {% endfor %} + + {% if section_name in deprecated_options %} + + {% for deprecated_option_name, (new_section_name, new_option_name, since_version) in deprecated_options[section_name].items() %} + .. _config:{{ section_name }}__{{ deprecated_option_name }}: + + {{ deprecated_option_name }} (Deprecated) + {{ "-" * (deprecated_option_name + " (Deprecated)")|length }} + + .. deprecated:: {{ since_version }} + The option has been moved to :ref:`{{ new_section_name }}.{{ new_option_name }} ` + {% endfor %} + {% endif %} + + {% endfor %} diff --git a/docs/exts/operators_and_hooks_ref.py b/docs/exts/operators_and_hooks_ref.py index eeaf00726c8a..06c37cf0b427 100644 --- a/docs/exts/operators_and_hooks_ref.py +++ b/docs/exts/operators_and_hooks_ref.py @@ -48,14 +48,15 @@ """ DEFAULT_HEADER_SEPARATOR = "=" -CURRENT_DIR = os.path.dirname(__file__) +CURRENT_DIR = Path(os.path.dirname(__file__)) +TEMPLATE_DIR = CURRENT_DIR / "templates" ROOT_DIR = os.path.abspath(os.path.join(CURRENT_DIR, os.pardir, os.pardir)) DOCS_DIR = os.path.join(ROOT_DIR, "docs") @lru_cache(maxsize=None) def _get_jinja_env(): - loader = jinja2.FileSystemLoader(CURRENT_DIR, followlinks=True) + loader = jinja2.FileSystemLoader(TEMPLATE_DIR, followlinks=True) env = jinja2.Environment(loader=loader, undefined=jinja2.StrictUndefined) return env @@ -213,7 +214,7 @@ def _render_deferrable_operator_content(*, header_separator: str): provider_yaml_content = yaml.safe_load(Path(provider_yaml_path).read_text()) provider_info["name"] = provider_yaml_content["package-name"] providers.append(provider_info) - return _render_template("deferrable_operatos_list.rst.jinja2", providers=providers) + return _render_template("deferrable_operators_list.rst.jinja2", providers=providers) class BaseJinjaReferenceDirective(Directive): @@ -308,6 +309,20 @@ def render_content( ) +class AuthConfigurations(BaseJinjaReferenceDirective): + """Generate list of configurations""" + + def render_content( + self, *, tags: set[str] | None, header_separator: str = DEFAULT_HEADER_SEPARATOR + ) -> str: + tabular_data = [ + provider["package-name"] for provider in load_package_data() if provider.get("config") is not None + ] + return _render_template( + "configuration.rst.jinja2", items=tabular_data, header_separator=header_separator + ) + + class SecretsBackendDirective(BaseJinjaReferenceDirective): """Generate list of secret backend handlers""" @@ -384,6 +399,7 @@ def setup(app): app.add_directive("transfers-ref", TransfersReferenceDirective) app.add_directive("airflow-logging", LoggingDirective) app.add_directive("airflow-auth-backends", AuthBackendDirective) + app.add_directive("airflow-configurations", AuthConfigurations) app.add_directive("airflow-secrets-backends", SecretsBackendDirective) app.add_directive("airflow-connections", ConnectionsDirective) app.add_directive("airflow-extra-links", ExtraLinksDirective) diff --git a/docs/exts/auth_backend.rst.jinja2 b/docs/exts/templates/auth_backend.rst.jinja2 similarity index 100% rename from docs/exts/auth_backend.rst.jinja2 rename to docs/exts/templates/auth_backend.rst.jinja2 diff --git a/docs/exts/templates/configuration.rst.jinja2 b/docs/exts/templates/configuration.rst.jinja2 new file mode 100644 index 000000000000..be43b4706ca7 --- /dev/null +++ b/docs/exts/templates/configuration.rst.jinja2 @@ -0,0 +1,21 @@ +{# + 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. +#} +{%for provider_package_name in items %} +* :doc:`Configuration for {{ provider_package_name }} <{{ provider_package_name }}:configurations-ref>` +{% endfor %} diff --git a/docs/exts/connections.rst.jinja2 b/docs/exts/templates/connections.rst.jinja2 similarity index 100% rename from docs/exts/connections.rst.jinja2 rename to docs/exts/templates/connections.rst.jinja2 diff --git a/docs/exts/deferrable_operatos_list.rst.jinja2 b/docs/exts/templates/deferrable_operators_list.rst.jinja2 similarity index 98% rename from docs/exts/deferrable_operatos_list.rst.jinja2 rename to docs/exts/templates/deferrable_operators_list.rst.jinja2 index e6eeb66385cf..58f29d295a7c 100644 --- a/docs/exts/deferrable_operatos_list.rst.jinja2 +++ b/docs/exts/templates/deferrable_operators_list.rst.jinja2 @@ -19,7 +19,7 @@ {%- for provider in providers %} **{{ provider["name"] }}** -.. list-table:: +.. list-table::F :header-rows: 1 * - Modules diff --git a/docs/exts/executors.rst.jinja2 b/docs/exts/templates/executors.rst.jinja2 similarity index 100% rename from docs/exts/executors.rst.jinja2 rename to docs/exts/templates/executors.rst.jinja2 diff --git a/docs/exts/extra_links.rst.jinja2 b/docs/exts/templates/extra_links.rst.jinja2 similarity index 100% rename from docs/exts/extra_links.rst.jinja2 rename to docs/exts/templates/extra_links.rst.jinja2 diff --git a/docs/exts/logging.rst.jinja2 b/docs/exts/templates/logging.rst.jinja2 similarity index 100% rename from docs/exts/logging.rst.jinja2 rename to docs/exts/templates/logging.rst.jinja2 diff --git a/docs/exts/notifications.rst.jinja2 b/docs/exts/templates/notifications.rst.jinja2 similarity index 100% rename from docs/exts/notifications.rst.jinja2 rename to docs/exts/templates/notifications.rst.jinja2 diff --git a/docs/exts/operators_and_hooks_ref-transfers.rst.jinja2 b/docs/exts/templates/operators_and_hooks_ref-transfers.rst.jinja2 similarity index 100% rename from docs/exts/operators_and_hooks_ref-transfers.rst.jinja2 rename to docs/exts/templates/operators_and_hooks_ref-transfers.rst.jinja2 diff --git a/docs/exts/operators_and_hooks_ref.rst.jinja2 b/docs/exts/templates/operators_and_hooks_ref.rst.jinja2 similarity index 100% rename from docs/exts/operators_and_hooks_ref.rst.jinja2 rename to docs/exts/templates/operators_and_hooks_ref.rst.jinja2 diff --git a/docs/exts/secret_backend.rst.jinja2 b/docs/exts/templates/secret_backend.rst.jinja2 similarity index 100% rename from docs/exts/secret_backend.rst.jinja2 rename to docs/exts/templates/secret_backend.rst.jinja2 diff --git a/tests/cli/commands/test_celery_command.py b/tests/cli/commands/test_celery_command.py index 0e8dee24570b..f3e3d391180e 100644 --- a/tests/cli/commands/test_celery_command.py +++ b/tests/cli/commands/test_celery_command.py @@ -39,7 +39,7 @@ def test_error(self, mock_validate_session): by mocking validate_session method """ mock_validate_session.return_value = False - with pytest.raises(SystemExit) as ctx: + with pytest.raises(SystemExit) as ctx, conf_vars({("core", "executor"): "CeleryExecutor"}): celery_command.worker(Namespace(queues=1, concurrency=1)) assert str(ctx.value) == "Worker exiting, database connection precheck failed." diff --git a/tests/cli/commands/test_config_command.py b/tests/cli/commands/test_config_command.py index f5b427f0cfca..59d04de860b6 100644 --- a/tests/cli/commands/test_config_command.py +++ b/tests/cli/commands/test_config_command.py @@ -43,6 +43,7 @@ def test_cli_show_config_should_write_data(self, mock_conf, mock_stringio): include_descriptions=False, include_sources=False, include_env_vars=False, + include_providers=True, comment_out_everything=False, only_defaults=False, ) @@ -60,6 +61,7 @@ def test_cli_show_config_should_write_data_specific_section(self, mock_conf, moc include_descriptions=False, include_sources=False, include_env_vars=False, + include_providers=True, comment_out_everything=False, only_defaults=False, ) @@ -178,6 +180,13 @@ def test_cli_show_changed_defaults_when_overridden_in_env(self): lines = output.split("\n") assert any(line.startswith("task_runner = test-env-runner") for line in lines if line) + def test_cli_has_providers(self): + with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: + config_command.show_config(self.parser.parse_args(["config", "list", "--color", "off"])) + output = temp_stdout.getvalue() + lines = output.split("\n") + assert any(line.startswith("celery_config_options") for line in lines if line) + def test_cli_comment_out_everything(self): with contextlib.redirect_stdout(io.StringIO()) as temp_stdout: config_command.show_config( diff --git a/tests/cli/test_cli_parser.py b/tests/cli/test_cli_parser.py index 7cac8f7ad1d1..e770e19416eb 100644 --- a/tests/cli/test_cli_parser.py +++ b/tests/cli/test_cli_parser.py @@ -1,5 +1,4 @@ #!/usr/bin/env python -# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -16,20 +15,25 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + from __future__ import annotations import argparse import contextlib import io +import os import re import subprocess +import sys import timeit from collections import Counter +from pathlib import Path from unittest.mock import patch import pytest from airflow.cli import cli_config, cli_parser +from airflow.configuration import AIRFLOW_HOME from tests.test_utils.config import conf_vars # Can not be `--snake_case` or contain uppercase letter @@ -284,9 +288,25 @@ def test_invalid_choice_raises_for_export_format_in_db_export_archived_command( "(choose from 'csv'), see help above.\n" ) + +# We need to run it from sources with PYTHONPATH, not command line tool, +# because we need to make sure that we have providers configured from source provider.yaml files + +CONFIG_FILE = Path(AIRFLOW_HOME) / "airflow.cfg" + + +class TestCliSubprocess: + """ + We need to run it from sources using "__main__" and setting the PYTHONPATH, not command line tool, + because we need to make sure that we have providers loaded from source provider.yaml files rather + than from provider packages which might not be installed in the test environment. + """ + def test_cli_run_time(self): setup_code = "import subprocess" - timing_code = 'subprocess.run(["airflow", "--help"])' + command = [sys.executable, "-m", "airflow", "--help"] + env = {"PYTHONPATH": os.pathsep.join(sys.path)} + timing_code = f"subprocess.run({command},env={env})" # Limit the number of samples otherwise the test will take a very long time num_samples = 3 threshold = 3.5 @@ -301,4 +321,59 @@ def test_cli_parsing_does_not_initialize_providers_manager(self): separate subprocess, to make sure we do not have providers manager initialized in the main process from other tests. """ - subprocess.check_call(["airflow", "providers", "status"]) + CONFIG_FILE.parent.mkdir(parents=True, exist_ok=True) + CONFIG_FILE.touch(exist_ok=True) + result = subprocess.run( + [sys.executable, "-m", "airflow", "providers", "status"], + env={"PYTHONPATH": os.pathsep.join(sys.path)}, + check=False, + text=True, + ) + assert result.returncode == 0 + + def test_airflow_config_contains_providers(self): + """Test that airflow config has providers included by default. + + This test is run as a separate subprocess, to make sure we do not have providers manager + initialized in the main process from other tests. + """ + CONFIG_FILE.unlink(missing_ok=True) + result = subprocess.run( + [sys.executable, "-m", "airflow", "version"], + env={"PYTHONPATH": os.pathsep.join(sys.path)}, + check=False, + text=True, + ) + assert result.returncode == 0 + assert CONFIG_FILE.exists() + assert "celery_config_options" in CONFIG_FILE.read_text() + + def test_airflow_config_output_contains_providers_by_default(self): + """Test that airflow config has providers excluded in config list when asked for it.""" + CONFIG_FILE.parent.mkdir(parents=True, exist_ok=True) + CONFIG_FILE.touch(exist_ok=True) + + result = subprocess.run( + [sys.executable, "-m", "airflow", "config", "list"], + env={"PYTHONPATH": os.pathsep.join(sys.path)}, + check=False, + text=True, + capture_output=True, + ) + assert result.returncode == 0 + assert "celery_config_options" in result.stdout + + def test_airflow_config_output_does_not_contain_providers_when_excluded(self): + """Test that airflow config has providers excluded in config list when asked for it.""" + CONFIG_FILE.parent.mkdir(parents=True, exist_ok=True) + CONFIG_FILE.unlink(missing_ok=True) + CONFIG_FILE.touch(exist_ok=True) + result = subprocess.run( + [sys.executable, "-m", "airflow", "config", "list", "--exclude-providers"], + env={"PYTHONPATH": os.pathsep.join(sys.path)}, + check=False, + text=True, + capture_output=True, + ) + assert result.returncode == 0 + assert "celery_config_options" not in result.stdout diff --git a/tests/conftest.py b/tests/conftest.py index b2ef0b1e6e90..49733a7da4e2 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -935,3 +935,10 @@ def refuse_to_run_test_from_wrongly_named_files(request): f"contains {request.function} that looks like a test case. Please rename the file to " f"follow the test_* pattern if you want to run the tests in it." ) + + +@pytest.fixture(autouse=True) +def initialize_providers_manager(): + from airflow.providers_manager import ProvidersManager + + ProvidersManager().initialize_providers_configuration() diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py index ecb01f5beeec..6eb50d919f10 100644 --- a/tests/core/test_configuration.py +++ b/tests/core/test_configuration.py @@ -27,6 +27,7 @@ import warnings from collections import OrderedDict from unittest import mock +from unittest.mock import patch import pytest from pytest import param @@ -627,7 +628,7 @@ def test_command_from_env(self): # sensitive_config_values and therefore should return 'OK' from the environment variable's # echo command, and must not return 'NOT OK' from the configuration assert test_cmdenv_conf.get("testcmdenv", "itsacommand") == "OK" - # AIRFLOW__TESTCMDENV__xNOTACOMMAND_CMD maps to no entry in sensitive_config_values and therefore + # AIRFLOW__TESTCMDENV__NOTACOMMAND_CMD maps to no entry in sensitive_config_values and therefore # the option should return 'OK' from the configuration, and must not return 'NOT OK' from # the environment variable's echo command assert test_cmdenv_conf.get("testcmdenv", "notacommand") == "OK" @@ -1599,9 +1600,6 @@ def test_sensitive_values(): sensitive_values = { ("database", "sql_alchemy_conn"), ("core", "fernet_key"), - ("celery", "broker_url"), - ("celery", "flower_basic_auth"), - ("celery", "result_backend"), ("atlas", "password"), ("smtp", "smtp_password"), ("webserver", "secret_key"), @@ -1609,6 +1607,10 @@ def test_sensitive_values(): ("sentry", "sentry_dsn"), ("database", "sql_alchemy_engine_args"), ("core", "sql_alchemy_conn"), + ("celery_broker_transport_options", "sentinel_kwargs"), + ("celery", "broker_url"), + ("celery", "flower_basic_auth"), + ("celery", "result_backend"), } all_keys = {(s, k) for s, v in conf.configuration_description.items() for k in v.get("options")} suspected_sensitive = {(s, k) for (s, k) in all_keys if k.endswith(("password", "kwargs"))} @@ -1618,3 +1620,51 @@ def test_sensitive_values(): suspected_sensitive -= exclude_list sensitive_values.update(suspected_sensitive) assert sensitive_values == conf.sensitive_config_values + + +def test_restore_and_reload_provider_configuration(): + from airflow.settings import conf + + assert conf.providers_configuration_loaded is True + assert conf.get("celery", "celery_app_name") == "airflow.providers.celery.executors.celery_executor" + conf.restore_core_default_configuration() + assert conf.providers_configuration_loaded is False + with pytest.raises(AirflowConfigException, match="not found"): + conf.get("celery", "celery_app_name") + conf.load_providers_configuration() + assert conf.providers_configuration_loaded is True + assert conf.get("celery", "celery_app_name") == "airflow.providers.celery.executors.celery_executor" + + +def test_error_when_contributing_to_existing_section(): + from airflow.settings import conf + + try: + assert conf.providers_configuration_loaded is True + assert conf.get("celery", "celery_app_name") == "airflow.providers.celery.executors.celery_executor" + conf.restore_core_default_configuration() + assert conf.providers_configuration_loaded is False + conf.configuration_description["celery"] = { + "description": "Celery Executor configuration", + "options": { + "celery_app_name": { + "default": "test", + } + }, + } + conf._default_values.add_section("celery") + conf._default_values.set("celery", "celery_app_name", "test") + assert conf.get("celery", "celery_app_name") == "test" + # patching restoring_core_default_configuration to avoid reloading the defaults + with patch.object(conf, "restore_core_default_configuration"): + with pytest.raises( + AirflowConfigException, + match="The provider apache-airflow-providers-celery is attempting to contribute " + "configuration section celery that has already been added before. " + "The source of it: Airflow's core package", + ): + conf.load_providers_configuration() + assert conf.get("celery", "celery_app_name") == "test" + finally: + conf.restore_core_default_configuration() + conf.load_providers_configuration() diff --git a/tests/providers/celery/executors/test_celery_kubernetes_executor.py b/tests/providers/celery/executors/test_celery_kubernetes_executor.py index 6302efea21da..1ef45ab22d9b 100644 --- a/tests/providers/celery/executors/test_celery_kubernetes_executor.py +++ b/tests/providers/celery/executors/test_celery_kubernetes_executor.py @@ -27,7 +27,7 @@ from airflow.providers.celery.executors.celery_executor import CeleryExecutor from airflow.providers.celery.executors.celery_kubernetes_executor import CeleryKubernetesExecutor -KUBERNETES_QUEUE = CeleryKubernetesExecutor.KUBERNETES_QUEUE +KUBERNETES_QUEUE = "kubernetes" class TestCeleryKubernetesExecutor: diff --git a/tests/providers/hashicorp/hooks/test_vault.py b/tests/providers/hashicorp/hooks/test_vault.py index 4bd3e90e5614..29bace064241 100644 --- a/tests/providers/hashicorp/hooks/test_vault.py +++ b/tests/providers/hashicorp/hooks/test_vault.py @@ -16,13 +16,17 @@ # under the License. from __future__ import annotations +import re from unittest import mock from unittest.mock import PropertyMock, mock_open, patch import pytest from hvac.exceptions import VaultError +from airflow.configuration import AirflowConfigParser +from airflow.exceptions import AirflowConfigException from airflow.providers.hashicorp.hooks.vault import VaultHook +from tests.test_utils.config import conf_vars class TestVaultHook: @@ -859,7 +863,6 @@ def test_radius_dejson_wrong_port(self, mock_hvac, mock_get_connection): kwargs = { "vault_conn_id": "vault_conn_id", } - with pytest.raises(VaultError, match="Radius port was wrong: wrong"): VaultHook(**kwargs) @@ -1252,3 +1255,87 @@ def test_create_or_update_secret_v1(self, mock_hvac, mock_get_connection, method mock_client.secrets.kv.v1.create_or_update_secret.assert_called_once_with( mount_point="secret", secret_path="path", secret={"key": "value"}, method=expected_method ) + + +class TestConfigurationFromSecrets: + @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") + @conf_vars( + { + ("secrets", "backend"): "airflow.providers.hashicorp.secrets.vault.VaultBackend", + ("secrets", "backend_kwargs"): '{"url": "http://127.0.0.1:8200", "token": "token"}', + } + ) + def test_config_from_secret_backend(self, mock_hvac): + """Get Config Value from a Secret Backend""" + mock_client = mock.MagicMock() + mock_hvac.Client.return_value = mock_client + mock_client.secrets.kv.v2.read_secret_version.return_value = { + "request_id": "2d48a2ad-6bcb-e5b6-429d-da35fdf31f56", + "lease_id": "", + "renewable": False, + "lease_duration": 0, + "data": { + "data": {"value": "sqlite:////Users/airflow/airflow/airflow.db"}, + "metadata": { + "created_time": "2020-03-28T02:10:54.301784Z", + "deletion_time": "", + "destroyed": False, + "version": 1, + }, + }, + "wrap_info": None, + "warnings": None, + "auth": None, + } + + test_config = """[test] + sql_alchemy_conn_secret = sql_alchemy_conn + """ + test_config_default = """[test] + sql_alchemy_conn = airflow + """ + + test_conf = AirflowConfigParser(default_config=test_config_default) + test_conf.read_string(test_config) + test_conf.sensitive_config_values = test_conf.sensitive_config_values | { + ("test", "sql_alchemy_conn"), + } + + assert "sqlite:////Users/airflow/airflow/airflow.db" == test_conf.get("test", "sql_alchemy_conn") + + @mock.patch("airflow.providers.hashicorp._internal_client.vault_client.hvac") + @conf_vars( + { + ("secrets", "backend"): "airflow.providers.hashicorp.secrets.vault.VaultBackend", + ("secrets", "backend_kwargs"): '{"url": "http://127.0.0.1:8200", "token": "token"}', + } + ) + def test_config_raise_exception_from_secret_backend_connection_error(self, mock_hvac): + """Get Config Value from a Secret Backend""" + + mock_client = mock.MagicMock() + # mock_client.side_effect = AirflowConfigException + mock_hvac.Client.return_value = mock_client + mock_client.secrets.kv.v2.read_secret_version.return_value = Exception + + test_config = """[test] +sql_alchemy_conn_secret = sql_alchemy_conn +""" + test_config_default = """[test] +sql_alchemy_conn = airflow +""" + test_conf = AirflowConfigParser(default_config=test_config_default) + test_conf.read_string(test_config) + test_conf.sensitive_config_values = test_conf.sensitive_config_values | { + ("test", "sql_alchemy_conn"), + } + + with pytest.raises( + AirflowConfigException, + match=re.escape( + "Cannot retrieve config from alternative secrets backend. " + "Make sure it is configured properly and that the Backend " + "is accessible." + ), + ): + test_conf.get("test", "sql_alchemy_conn")