Skip to content

Commit

Permalink
Add deprecated config options to docs (#13883)
Browse files Browse the repository at this point in the history
closes: #12772
  • Loading branch information
vemikhaylov authored Jan 27, 2021
1 parent 810c15e commit 65e49fc
Show file tree
Hide file tree
Showing 4 changed files with 64 additions and 37 deletions.
68 changes: 37 additions & 31 deletions airflow/configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -132,39 +132,43 @@ class AirflowConfigParser(ConfigParser): # pylint: disable=too-many-ancestors
('webserver', 'secret_key'),
}

# A mapping of (new option -> old option). where option is a tuple of section name and key.
# A mapping of (new section, new option) -> (old section, old option, since_version).
# When reading new option, the old option will be checked to see if it exists. If it does a
# DeprecationWarning will be issued and the old option will be used instead
deprecated_options = {
('celery', 'worker_precheck'): ('core', 'worker_precheck'),
('logging', 'base_log_folder'): ('core', 'base_log_folder'),
('logging', 'remote_logging'): ('core', 'remote_logging'),
('logging', 'remote_log_conn_id'): ('core', 'remote_log_conn_id'),
('logging', 'remote_base_log_folder'): ('core', 'remote_base_log_folder'),
('logging', 'encrypt_s3_logs'): ('core', 'encrypt_s3_logs'),
('logging', 'logging_level'): ('core', 'logging_level'),
('logging', 'fab_logging_level'): ('core', 'fab_logging_level'),
('logging', 'logging_config_class'): ('core', 'logging_config_class'),
('logging', 'colored_console_log'): ('core', 'colored_console_log'),
('logging', 'colored_log_format'): ('core', 'colored_log_format'),
('logging', 'colored_formatter_class'): ('core', 'colored_formatter_class'),
('logging', 'log_format'): ('core', 'log_format'),
('logging', 'simple_log_format'): ('core', 'simple_log_format'),
('logging', 'task_log_prefix_template'): ('core', 'task_log_prefix_template'),
('logging', 'log_filename_template'): ('core', 'log_filename_template'),
('logging', 'log_processor_filename_template'): ('core', 'log_processor_filename_template'),
('logging', 'dag_processor_manager_log_location'): ('core', 'dag_processor_manager_log_location'),
('logging', 'task_log_reader'): ('core', 'task_log_reader'),
('metrics', 'statsd_on'): ('scheduler', 'statsd_on'),
('metrics', 'statsd_host'): ('scheduler', 'statsd_host'),
('metrics', 'statsd_port'): ('scheduler', 'statsd_port'),
('metrics', 'statsd_prefix'): ('scheduler', 'statsd_prefix'),
('metrics', 'statsd_allow_list'): ('scheduler', 'statsd_allow_list'),
('metrics', 'stat_name_handler'): ('scheduler', 'stat_name_handler'),
('metrics', 'statsd_datadog_enabled'): ('scheduler', 'statsd_datadog_enabled'),
('metrics', 'statsd_datadog_tags'): ('scheduler', 'statsd_datadog_tags'),
('metrics', 'statsd_custom_client_path'): ('scheduler', 'statsd_custom_client_path'),
('scheduler', 'parsing_processes'): ('scheduler', 'max_threads'),
('celery', 'worker_precheck'): ('core', 'worker_precheck', '2.0.0'),
('logging', 'base_log_folder'): ('core', 'base_log_folder', '2.0.0'),
('logging', 'remote_logging'): ('core', 'remote_logging', '2.0.0'),
('logging', 'remote_log_conn_id'): ('core', 'remote_log_conn_id', '2.0.0'),
('logging', 'remote_base_log_folder'): ('core', 'remote_base_log_folder', '2.0.0'),
('logging', 'encrypt_s3_logs'): ('core', 'encrypt_s3_logs', '2.0.0'),
('logging', 'logging_level'): ('core', 'logging_level', '2.0.0'),
('logging', 'fab_logging_level'): ('core', 'fab_logging_level', '2.0.0'),
('logging', 'logging_config_class'): ('core', 'logging_config_class', '2.0.0'),
('logging', 'colored_console_log'): ('core', 'colored_console_log', '2.0.0'),
('logging', 'colored_log_format'): ('core', 'colored_log_format', '2.0.0'),
('logging', 'colored_formatter_class'): ('core', 'colored_formatter_class', '2.0.0'),
('logging', 'log_format'): ('core', 'log_format', '2.0.0'),
('logging', 'simple_log_format'): ('core', 'simple_log_format', '2.0.0'),
('logging', 'task_log_prefix_template'): ('core', 'task_log_prefix_template', '2.0.0'),
('logging', 'log_filename_template'): ('core', 'log_filename_template', '2.0.0'),
('logging', 'log_processor_filename_template'): ('core', 'log_processor_filename_template', '2.0.0'),
('logging', 'dag_processor_manager_log_location'): (
'core',
'dag_processor_manager_log_location',
'2.0.0',
),
('logging', 'task_log_reader'): ('core', 'task_log_reader', '2.0.0'),
('metrics', 'statsd_on'): ('scheduler', 'statsd_on', '2.0.0'),
('metrics', 'statsd_host'): ('scheduler', 'statsd_host', '2.0.0'),
('metrics', 'statsd_port'): ('scheduler', 'statsd_port', '2.0.0'),
('metrics', 'statsd_prefix'): ('scheduler', 'statsd_prefix', '2.0.0'),
('metrics', 'statsd_allow_list'): ('scheduler', 'statsd_allow_list', '2.0.0'),
('metrics', 'stat_name_handler'): ('scheduler', 'stat_name_handler', '2.0.0'),
('metrics', 'statsd_datadog_enabled'): ('scheduler', 'statsd_datadog_enabled', '2.0.0'),
('metrics', 'statsd_datadog_tags'): ('scheduler', 'statsd_datadog_tags', '2.0.0'),
('metrics', 'statsd_custom_client_path'): ('scheduler', 'statsd_custom_client_path', '2.0.0'),
('scheduler', 'parsing_processes'): ('scheduler', 'max_threads', '1.10.14'),
}

# A mapping of old default values that we want to change and warn the user
Expand Down Expand Up @@ -321,7 +325,9 @@ def get(self, section, key, **kwargs):
section = str(section).lower()
key = str(key).lower()

deprecated_section, deprecated_key = self.deprecated_options.get((section, key), (None, None))
deprecated_section, deprecated_key, _ = self.deprecated_options.get(
(section, key), (None, None, None)
)

option = self._get_environment_variables(deprecated_key, deprecated_section, key, section)
if option is not None:
Expand Down
14 changes: 14 additions & 0 deletions docs/apache-airflow/configurations-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -79,4 +79,18 @@ can set in ``airflow.cfg`` file or using environment variables.
{% 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 }} <config:{{ new_section_name }}__{{ new_option_name }}>`
{% endfor %}
{% endif %}

{% endfor %}
13 changes: 10 additions & 3 deletions docs/conf.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,13 @@
import glob
import os
import sys
from typing import Any, Dict, List, Optional
from collections import defaultdict
from typing import Any, Dict, List, Optional, Tuple

import yaml

import airflow
from airflow.configuration import default_config_yaml
from airflow.configuration import AirflowConfigParser, default_config_yaml
from docs.exts.docs_build.third_party_inventories import ( # pylint: disable=no-name-in-module,wrong-import-order
THIRD_PARTY_INDEXES,
)
Expand Down Expand Up @@ -308,8 +309,14 @@ def _get_rst_filepath_from_path(filepath: str):

# Jinja context
if PACKAGE_NAME == 'apache-airflow':
deprecated_options: Dict[str, Dict[str, Tuple[str, str, str]]] = defaultdict(dict)
for (section, key), (
(deprecated_section, deprecated_key, since_version)
) in AirflowConfigParser.deprecated_options.items():
deprecated_options[deprecated_section][deprecated_key] = section, key, since_version

jinja_contexts = {
'config_ctx': {"configs": default_config_yaml()},
'config_ctx': {"configs": default_config_yaml(), "deprecated_options": deprecated_options},
'quick_start_ctx': {
'doc_root_url': (f'https://airflow.apache.org/docs/apache-airflow/{PACKAGE_VERSION}/')
if FOR_PRODUCTION
Expand Down
6 changes: 3 additions & 3 deletions tests/core/test_configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,7 @@ def test_deprecated_options(self):
# Guarantee we have a deprecated setting, so we test the deprecation
# lookup even if we remove this explicit fallback
conf.deprecated_options = {
('celery', 'worker_concurrency'): ('celery', 'celeryd_concurrency'),
('celery', 'worker_concurrency'): ('celery', 'celeryd_concurrency', '2.0.0'),
}

# Remove it so we are sure we use the right setting
Expand All @@ -447,7 +447,7 @@ def test_deprecated_options_with_new_section(self):
# Guarantee we have a deprecated setting, so we test the deprecation
# lookup even if we remove this explicit fallback
conf.deprecated_options = {
('logging', 'logging_level'): ('core', 'logging_level'),
('logging', 'logging_level'): ('core', 'logging_level', '2.0.0'),
}

# Remove it so we are sure we use the right setting
Expand All @@ -471,7 +471,7 @@ def test_deprecated_options_with_new_section(self):
def test_deprecated_options_cmd(self):
# Guarantee we have a deprecated setting, so we test the deprecation
# lookup even if we remove this explicit fallback
conf.deprecated_options[('celery', "result_backend")] = ('celery', 'celery_result_backend')
conf.deprecated_options[('celery', "result_backend")] = 'celery', 'celery_result_backend', '2.0.0'
conf.sensitive_config_values.add(('celery', 'celery_result_backend'))

conf.remove_option('celery', 'result_backend')
Expand Down

0 comments on commit 65e49fc

Please sign in to comment.