From b7af1c44329c460f572c21a30250322e799eaec0 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 22 Jul 2023 00:40:19 +0200 Subject: [PATCH 1/2] Move all k8S classes to cncf.kubernetes provider This is the big move of all Kubenetes classes to go to provider. The changes that are implemented in this move: * replaced all imports from airflow.kubernetes to cncf.kubernetes Swith PEP-563 dynamic import rediretion and deprecation messages those messages now support overriding the "replacement" hints to make K8s deprecations more accurate * pre_7_4_0_compatibility package with classes used by past providerrs have been "frozen" and stored in the package with import redirections from airflow.kubernetes(with deprecation warnings) * kubernetes configuration is moved to kubernetes provider * mypy started complaining about conf and set used in configuration. so better solution to handle deprecations and hinting conf returning AirlfowConfigParsing was added. * example_kuberntes_executor uses configuration reading not in top level but in execute method * PodMutationHookException and PodReconciliationError have been moved to cncf.kubernetes provider and they are imported from there with fallback to an airflow.exception ones in case old provider is used in Airflow 2.7.0 * k8s methods in task_instance have been deprecated and reolaced with functions in "cncf.kubernetes` template_rendering module the old way still works but raise deprecaton warnings. * added extras with versions for celery and k8s * raise AirflowOptionalProviderFeatureException in case there is attempt to use CeleryK8sExecutor and cncf.k8s is not installed. * added few "new" core utils to k8s (hashlib_wrapper etc) * both warnings and errors indicate minimum versions for both cncf.k8s and Celery providers. --- .github/CODEOWNERS | 1 - .github/boring-cyborg.yml | 6 - .pre-commit-config.yaml | 16 + STATIC_CODE_CHECKS.rst | 4 + airflow/cli/commands/kubernetes_command.py | 10 +- airflow/config_templates/__init__.py | 2 +- airflow/config_templates/config.yml | 210 ------ airflow/configuration.py | 9 +- airflow/decorators/__init__.pyi | 2 +- .../example_kubernetes_executor.py | 5 +- airflow/exceptions.py | 28 +- airflow/executors/__init__.py | 30 +- airflow/executors/executor_loader.py | 9 +- airflow/kubernetes/__init__.py | 125 ++++ .../__init__.py} | 23 +- .../k8s_model.py | 0 .../kube_client.py | 0 .../pre_7_4_0_compatibility/pod_generator.py | 666 ++++++++++++++++++ .../pod_generator_deprecated.py | 0 .../pre_7_4_0_compatibility/secret.py | 124 ++++ airflow/models/base.py | 2 +- airflow/models/renderedtifields.py | 6 +- airflow/models/taskinstance.py | 80 ++- .../amazon/aws/notifications/chime.py | 2 +- airflow/providers/apache/spark/CHANGELOG.rst | 19 + airflow/providers/apache/spark/__init__.py | 2 +- .../apache/spark/hooks/spark_submit.py | 2 +- airflow/providers/apache/spark/provider.yaml | 6 + airflow/providers/celery/CHANGELOG.rst | 17 +- airflow/providers/celery/__init__.py | 2 +- .../executors/celery_kubernetes_executor.py | 9 +- airflow/providers/celery/provider.yaml | 7 +- .../providers/cncf/kubernetes/CHANGELOG.rst | 17 +- .../cncf/kubernetes/executors}/__init__.py | 0 .../executors/kubernetes_executor.py | 48 +- .../executors/kubernetes_executor_types.py | 0 .../executors/kubernetes_executor_utils.py | 38 +- .../executors/local_kubernetes_executor.py | 2 +- .../cncf/kubernetes/hooks/kubernetes.py | 2 +- .../providers/cncf/kubernetes/k8s_model.py | 59 ++ .../providers/cncf/kubernetes/kube_client.py | 146 ++++ .../cncf}/kubernetes/kube_config.py | 1 - .../__init__.py | 0 .../basic_template.yaml | 0 .../kubernetes/kubernetes_helper_functions.py | 10 +- .../cncf/kubernetes/operators/pod.py | 6 +- .../cncf}/kubernetes/pod_generator.py | 20 +- .../kubernetes/pod_generator_deprecated.py | 309 ++++++++ .../kubernetes/pod_launcher_deprecated.py | 4 +- .../pod_template_file_examples/__init__.py} | 17 - .../dags_in_image_template.yaml | 0 .../dags_in_volume_template.yaml | 0 .../git_sync_template.yaml | 0 .../providers/cncf/kubernetes/provider.yaml | 210 ++++++ .../{ => providers/cncf}/kubernetes/secret.py | 2 +- .../cncf/kubernetes/template_rendering.py | 67 ++ .../kubernetes/utils/k8s_hashlib_wrapper.py | 42 ++ .../cncf/kubernetes/utils/pod_manager.py | 4 +- airflow/serialization/serialized_objects.py | 9 +- .../serialization/serializers/kubernetes.py | 7 +- airflow/timetables/_cron.py | 4 +- airflow/utils/deprecation_tools.py | 68 +- .../volume.py => utils/empty_set.py} | 22 +- airflow/utils/hashlib_wrapper.py | 5 +- airflow/utils/sqlalchemy.py | 8 +- airflow/www/views.py | 6 +- .../commands/kubernetes_commands.py | 2 +- .../src/airflow_breeze/pre_commit_ids.py | 2 + dev/breeze/tests/test_selective_checks.py | 3 +- .../changelog.rst | 2 - .../commits.rst | 16 + .../index.rst | 30 +- .../changelog.rst | 2 - .../commits.rst | 21 + .../apache-airflow-providers-celery/index.rst | 30 +- .../configurations-ref.rst | 18 + .../index.rst | 1 + .../operators.rst | 2 +- docs/apache-airflow/configurations-ref.rst | 1 + .../core-concepts/executor/celery.rst | 7 + .../executor/celery_kubernetes.rst | 8 + .../core-concepts/executor/kubernetes.rst | 14 +- .../executor/local_kubernetes.rst | 9 +- docs/apache-airflow/extra-packages-ref.rst | 90 +-- .../howto/upgrading-from-1-10/index.rst | 4 +- docs/spelling_wordlist.txt | 5 + generated/provider_dependencies.json | 8 +- images/breeze/output-commands-hash.txt | 2 +- images/breeze/output_static-checks.svg | 142 ++-- .../conftest.py | 15 +- .../test_kubernetes_pod_operator.py | 8 +- newsfragments/32767.significant.rst | 7 + .../pre_commit_check_airflow_k8s_not_used.py | 81 +++ ...eck_cncf_k8s_used_for_k8s_executor_only.py | 84 +++ .../run_provider_yaml_files_check.py | 2 + setup.py | 3 - tests/cli/commands/test_kubernetes_command.py | 6 +- tests/cli/commands/test_task_command.py | 8 +- tests/models/test_renderedtifields.py | 48 -- tests/models/test_taskinstance.py | 81 --- .../apache/spark/hooks/test_spark_submit.py | 2 +- .../test_celery_kubernetes_executor.py | 2 +- tests/providers/cncf/kubernetes/__init__.py | 1 + .../cncf}/kubernetes/basic_pod.yaml | 0 .../cncf/kubernetes/executors/__init__.py | 17 - .../__init__.py | 16 + .../basic_template.yaml | 0 .../executors/test_kubernetes_executor.py | 158 +++-- .../test_local_kubernetes_executor.py | 4 +- .../cncf}/kubernetes/kube_config | 0 .../cncf}/kubernetes/models/__init__.py | 0 .../cncf}/kubernetes/models/test_secret.py | 10 +- .../cncf/kubernetes/operators/test_pod.py | 2 +- .../{ => providers/cncf}/kubernetes/pod.yaml | 0 .../cncf}/kubernetes/pod_generator_base.yaml | 0 .../pod_generator_base_with_secrets.yaml | 0 .../cncf}/kubernetes/test_client.py | 18 +- .../test_kubernetes_helper_functions.py | 2 +- .../cncf}/kubernetes/test_pod_generator.py | 33 +- .../kubernetes/test_template_rendering.py | 156 ++++ tests/sensors/test_base.py | 6 +- tests/serialization/test_dag_serialization.py | 4 +- .../cncf/kubernetes/example_kubernetes.py | 2 +- .../kubernetes/example_kubernetes_async.py | 2 +- tests/utils/test_log_handlers.py | 6 +- 125 files changed, 2960 insertions(+), 800 deletions(-) rename airflow/kubernetes/{pod.py => pre_7_4_0_compatibility/__init__.py} (57%) rename airflow/kubernetes/{ => pre_7_4_0_compatibility}/k8s_model.py (100%) rename airflow/kubernetes/{ => pre_7_4_0_compatibility}/kube_client.py (100%) create mode 100644 airflow/kubernetes/pre_7_4_0_compatibility/pod_generator.py rename airflow/kubernetes/{ => pre_7_4_0_compatibility}/pod_generator_deprecated.py (100%) create mode 100644 airflow/kubernetes/pre_7_4_0_compatibility/secret.py rename {tests/kubernetes => airflow/providers/cncf/kubernetes/executors}/__init__.py (100%) rename airflow/{ => providers/cncf/kubernetes}/executors/kubernetes_executor.py (93%) rename airflow/{ => providers/cncf/kubernetes}/executors/kubernetes_executor_types.py (100%) rename airflow/{ => providers/cncf/kubernetes}/executors/kubernetes_executor_utils.py (93%) rename airflow/{ => providers/cncf/kubernetes}/executors/local_kubernetes_executor.py (99%) create mode 100644 airflow/providers/cncf/kubernetes/k8s_model.py create mode 100644 airflow/providers/cncf/kubernetes/kube_client.py rename airflow/{ => providers/cncf}/kubernetes/kube_config.py (99%) rename {tests/executors/kubernetes_executor_template_files => airflow/providers/cncf/kubernetes/kubernetes_executor_templates}/__init__.py (100%) rename airflow/{ => providers/cncf/kubernetes}/kubernetes_executor_templates/basic_template.yaml (100%) rename airflow/{ => providers/cncf}/kubernetes/kubernetes_helper_functions.py (94%) rename airflow/{ => providers/cncf}/kubernetes/pod_generator.py (97%) create mode 100644 airflow/providers/cncf/kubernetes/pod_generator_deprecated.py rename airflow/{ => providers/cncf}/kubernetes/pod_launcher_deprecated.py (98%) rename airflow/{kubernetes/pod_runtime_info_env.py => providers/cncf/kubernetes/pod_template_file_examples/__init__.py} (58%) rename airflow/{ => providers/cncf}/kubernetes/pod_template_file_examples/dags_in_image_template.yaml (100%) rename airflow/{ => providers/cncf}/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml (100%) rename airflow/{ => providers/cncf}/kubernetes/pod_template_file_examples/git_sync_template.yaml (100%) rename airflow/{ => providers/cncf}/kubernetes/secret.py (98%) create mode 100644 airflow/providers/cncf/kubernetes/template_rendering.py create mode 100644 airflow/providers/cncf/kubernetes/utils/k8s_hashlib_wrapper.py rename airflow/{kubernetes/volume.py => utils/empty_set.py} (62%) create mode 100644 docs/apache-airflow-providers-cncf-kubernetes/configurations-ref.rst rename airflow/kubernetes/pod_launcher.py => kubernetes_tests/conftest.py (79%) create mode 100644 newsfragments/32767.significant.rst create mode 100755 scripts/ci/pre_commit/pre_commit_check_airflow_k8s_not_used.py create mode 100755 scripts/ci/pre_commit/pre_commit_check_cncf_k8s_used_for_k8s_executor_only.py rename tests/{ => providers/cncf}/kubernetes/basic_pod.yaml (100%) rename airflow/kubernetes/volume_mount.py => tests/providers/cncf/kubernetes/executors/__init__.py (58%) create mode 100644 tests/providers/cncf/kubernetes/executors/kubernetes_executor_template_files/__init__.py rename tests/{ => providers/cncf/kubernetes}/executors/kubernetes_executor_template_files/basic_template.yaml (100%) rename tests/{ => providers/cncf/kubernetes}/executors/test_kubernetes_executor.py (89%) rename tests/{ => providers/cncf/kubernetes}/executors/test_local_kubernetes_executor.py (97%) rename tests/{ => providers/cncf}/kubernetes/kube_config (100%) rename tests/{ => providers/cncf}/kubernetes/models/__init__.py (100%) rename tests/{ => providers/cncf}/kubernetes/models/test_secret.py (93%) rename tests/{ => providers/cncf}/kubernetes/pod.yaml (100%) rename tests/{ => providers/cncf}/kubernetes/pod_generator_base.yaml (100%) rename tests/{ => providers/cncf}/kubernetes/pod_generator_base_with_secrets.yaml (100%) rename tests/{ => providers/cncf}/kubernetes/test_client.py (87%) rename tests/{ => providers/cncf}/kubernetes/test_kubernetes_helper_functions.py (98%) rename tests/{ => providers/cncf}/kubernetes/test_pod_generator.py (95%) create mode 100644 tests/providers/cncf/kubernetes/test_template_rendering.py diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index dfc0e13a3a7e..aa036aeac58e 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -11,7 +11,6 @@ # Kubernetes /airflow/kubernetes/ @dstandish @jedcunningham -/airflow/kubernetes_executor_templates/ @dstandish @jedcunningham /airflow/executors/celery_kubernetes_executor.py @dstandish @jedcunningham /airflow/executors/kubernetes_executor.py @dstandish @jedcunningham diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index 989bb055df74..e2f7c037ecb8 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -164,9 +164,6 @@ labelPRBasedOnFilePath: - airflow/**/kubernetes_*.py - airflow/example_dags/example_kubernetes_executor.py - airflow/providers/cncf/kubernetes/**/* - - airflow/kubernetes/**/* - - airflow/kubernetes_executor_templates/**/* - - airflow/executors/kubernetes_executor.py - airflow/providers/celery/executors/celery_kubernetes_executor.py - docs/apache-airflow/core-concepts/executor/kubernetes.rst - docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst @@ -174,9 +171,6 @@ labelPRBasedOnFilePath: - kubernetes_tests/**/* - tests/providers/cncf/kubernetes/**/* - tests/system/providers/cncf/kubernetes/**/* - - tests/kubernetes/**/* - - tests/executors/kubernetes_executor_template_files/**/* - - tests/executors/*kubernetes*.py area:API: - airflow/api/**/* diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index ff8085d03033..d289ef97c489 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -333,6 +333,22 @@ repos: pass_filenames: false entry: ./scripts/ci/pre_commit/pre_commit_check_order_setup.py additional_dependencies: ['rich>=12.4.4'] + - id: check-airflow-k8s-not-used + name: Check airflow.kubernetes imports are not used + language: python + files: ^airflow/.*\.py$ + require_serial: true + exclude: ^airflow/kubernetes/ + entry: ./scripts/ci/pre_commit/pre_commit_check_airflow_k8s_not_used.py + additional_dependencies: ['rich>=12.4.4'] + - id: check-cncf-k8s-only-for-executors + name: Check cncf.kubernetes imports used for executors only + language: python + files: ^airflow/.*\.py$ + require_serial: true + exclude: ^airflow/kubernetes/|^airflow/providers/ + entry: ./scripts/ci/pre_commit/pre_commit_check_cncf_k8s_used_for_k8s_executor_only.py + additional_dependencies: ['rich>=12.4.4'] - id: check-extra-packages-references name: Checks setup extra packages description: Checks if all the libraries in setup.py are listed in extra-packages-ref.rst file diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst index 1729c8ba8e61..bdb77e5c83c9 100644 --- a/STATIC_CODE_CHECKS.rst +++ b/STATIC_CODE_CHECKS.rst @@ -146,6 +146,8 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-aiobotocore-optional | Check if aiobotocore is an optional dependency only | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ +| check-airflow-k8s-not-used | Check airflow.kubernetes imports are not used | | ++-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-airflow-provider-compatibility | Check compatibility of Providers with Airflow | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-apache-license-rat | Check if licenses are OK for Apache | | @@ -163,6 +165,8 @@ require Breeze Docker image to be built locally. +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-changelog-has-no-duplicates | Check changelogs for duplicate entries | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ +| check-cncf-k8s-only-for-executors | Check cncf.kubernetes imports used for executors only | | ++-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-core-deprecation-classes | Verify usage of Airflow deprecation classes in core | | +-----------------------------------------------------------+--------------------------------------------------------------+---------+ | check-daysago-import-from-utils | Make sure days_ago is imported from airflow.utils.dates | | diff --git a/airflow/cli/commands/kubernetes_command.py b/airflow/cli/commands/kubernetes_command.py index 1555f7be9258..056465577fcd 100644 --- a/airflow/cli/commands/kubernetes_command.py +++ b/airflow/cli/commands/kubernetes_command.py @@ -25,12 +25,12 @@ from kubernetes.client.api_client import ApiClient from kubernetes.client.rest import ApiException -from airflow.executors.kubernetes_executor import KubeConfig -from airflow.kubernetes import pod_generator -from airflow.kubernetes.kube_client import get_kube_client -from airflow.kubernetes.kubernetes_helper_functions import create_pod_id -from airflow.kubernetes.pod_generator import PodGenerator from airflow.models import DagRun, TaskInstance +from airflow.providers.cncf.kubernetes import pod_generator +from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubeConfig +from airflow.providers.cncf.kubernetes.kube_client import get_kube_client +from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import create_pod_id +from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator from airflow.utils import cli as cli_utils, yaml from airflow.utils.cli import get_dag from airflow.utils.providers_configuration_loader import providers_configuration_loaded diff --git a/airflow/config_templates/__init__.py b/airflow/config_templates/__init__.py index 6dd06760cdfd..4bdf46b9a970 100644 --- a/airflow/config_templates/__init__.py +++ b/airflow/config_templates/__init__.py @@ -25,4 +25,4 @@ }, } -add_deprecated_classes(__deprecated_classes, __name__) +add_deprecated_classes(__deprecated_classes, __name__, {}, "The `celery` provider must be >= 3.3.0 for that.") diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 35f16c57a8f4..b550c25932ed 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1943,21 +1943,6 @@ sentry: type: string example: ~ default: ~ -local_kubernetes_executor: - description: | - This section only applies if you are using the ``LocalKubernetesExecutor`` in - ``[core]`` section above - options: - kubernetes_queue: - description: | - Define when to send a task to ``KubernetesExecutor`` when using ``LocalKubernetesExecutor``. - When the queue of a task is the value of ``kubernetes_queue`` (default ``kubernetes``), - the task is executed via ``KubernetesExecutor``, - otherwise via ``LocalExecutor`` - version_added: 2.3.0 - type: string - example: ~ - default: "kubernetes" scheduler: description: ~ options: @@ -2442,201 +2427,6 @@ elasticsearch_configs: type: string example: ~ default: "True" -kubernetes_executor: - description: ~ - renamed: - previous_name: kubernetes - version: 2.5.0 - options: - api_client_retry_configuration: - description: | - Kwargs to override the default urllib3 Retry used in the kubernetes API client - version_added: 2.6.0 - type: string - example: '{ "total": 3, "backoff_factor": 0.5 }' - default: "" - logs_task_metadata: - description: | - Flag to control the information added to kubernetes executor logs for better traceability - version_added: 2.7.0 - type: boolean - example: ~ - default: "False" - pod_template_file: - description: | - Path to the YAML pod file that forms the basis for KubernetesExecutor workers. - version_added: 1.10.11 - type: string - example: ~ - default: "" - see_also: ":ref:`concepts:pod_template_file`" - worker_container_repository: - description: | - The repository of the Kubernetes Image for the Worker to Run - version_added: ~ - type: string - example: ~ - default: "" - worker_container_tag: - description: | - The tag of the Kubernetes Image for the Worker to Run - version_added: ~ - type: string - example: ~ - default: "" - namespace: - description: | - The Kubernetes namespace where airflow workers should be created. Defaults to ``default`` - version_added: ~ - type: string - example: ~ - default: "default" - delete_worker_pods: - description: | - If True, all worker pods will be deleted upon termination - version_added: ~ - type: string - example: ~ - default: "True" - delete_worker_pods_on_failure: - description: | - If False (and delete_worker_pods is True), - failed worker pods will not be deleted so users can investigate them. - This only prevents removal of worker pods where the worker itself failed, - not when the task it ran failed. - version_added: 1.10.11 - type: string - example: ~ - default: "False" - worker_pods_creation_batch_size: - description: | - Number of Kubernetes Worker Pod creation calls per scheduler loop. - Note that the current default of "1" will only launch a single pod - per-heartbeat. It is HIGHLY recommended that users increase this - number to match the tolerance of their kubernetes cluster for - better performance. - version_added: 1.10.3 - type: string - example: ~ - default: "1" - multi_namespace_mode: - description: | - Allows users to launch pods in multiple namespaces. - Will require creating a cluster-role for the scheduler, - or use multi_namespace_mode_namespace_list configuration. - version_added: 1.10.12 - type: boolean - example: ~ - default: "False" - multi_namespace_mode_namespace_list: - description: | - If multi_namespace_mode is True while scheduler does not have a cluster-role, - give the list of namespaces where the scheduler will schedule jobs - Scheduler needs to have the necessary permissions in these namespaces. - version_added: 2.6.0 - type: string - example: ~ - default: "" - in_cluster: - description: | - Use the service account kubernetes gives to pods to connect to kubernetes cluster. - It's intended for clients that expect to be running inside a pod running on kubernetes. - It will raise an exception if called from a process not running in a kubernetes environment. - version_added: ~ - type: string - example: ~ - default: "True" - cluster_context: - description: | - When running with in_cluster=False change the default cluster_context or config_file - options to Kubernetes client. Leave blank these to use default behaviour like ``kubectl`` has. - version_added: 1.10.3 - type: string - example: ~ - default: ~ - config_file: - description: | - Path to the kubernetes configfile to be used when ``in_cluster`` is set to False - version_added: 1.10.3 - type: string - example: ~ - default: ~ - kube_client_request_args: - description: | - Keyword parameters to pass while calling a kubernetes client core_v1_api methods - from Kubernetes Executor provided as a single line formatted JSON dictionary string. - List of supported params are similar for all core_v1_apis, hence a single config - variable for all apis. See: - https://raw.githubusercontent.com/kubernetes-client/python/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/api/core_v1_api.py - version_added: 1.10.4 - type: string - example: ~ - default: "" - delete_option_kwargs: - description: | - Optional keyword arguments to pass to the ``delete_namespaced_pod`` kubernetes client - ``core_v1_api`` method when using the Kubernetes Executor. - This should be an object and can contain any of the options listed in the ``v1DeleteOptions`` - class defined here: - https://github.com/kubernetes-client/python/blob/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/models/v1_delete_options.py#L19 - version_added: 1.10.12 - type: string - example: '{"grace_period_seconds": 10}' - default: "" - enable_tcp_keepalive: - description: | - Enables TCP keepalive mechanism. This prevents Kubernetes API requests to hang indefinitely - when idle connection is time-outed on services like cloud load balancers or firewalls. - version_added: 2.0.0 - type: boolean - example: ~ - default: "True" - tcp_keep_idle: - description: | - When the `enable_tcp_keepalive` option is enabled, TCP probes a connection that has - been idle for `tcp_keep_idle` seconds. - version_added: 2.0.0 - type: integer - example: ~ - default: "120" - tcp_keep_intvl: - description: | - When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond - to a keepalive probe, TCP retransmits the probe after `tcp_keep_intvl` seconds. - version_added: 2.0.0 - type: integer - example: ~ - default: "30" - tcp_keep_cnt: - description: | - When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond - to a keepalive probe, TCP retransmits the probe `tcp_keep_cnt number` of times before - a connection is considered to be broken. - version_added: 2.0.0 - type: integer - example: ~ - default: "6" - verify_ssl: - description: | - Set this to false to skip verifying SSL certificate of Kubernetes python client. - version_added: 2.1.0 - type: boolean - example: ~ - default: "True" - worker_pods_queued_check_interval: - description: | - How often in seconds to check for task instances stuck in "queued" status without a pod - version_added: 2.2.0 - type: integer - example: ~ - default: "60" - ssl_ca_cert: - description: | - Path to a CA certificate to be used by the Kubernetes client to verify the server's SSL certificate. - version_added: 2.6.0 - type: string - example: ~ - default: "" sensors: description: ~ options: diff --git a/airflow/configuration.py b/airflow/configuration.py index 54ce39225dd8..afbb02183415 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -46,6 +46,7 @@ from airflow.exceptions import AirflowConfigException from airflow.secrets import DEFAULT_SECRETS_SEARCH_PATH, BaseSecretsBackend from airflow.utils import yaml +from airflow.utils.empty_set import _get_empty_set_for_configuration from airflow.utils.module_loading import import_string from airflow.utils.providers_configuration_loader import providers_configuration_loaded from airflow.utils.weight_rule import WeightRule @@ -66,8 +67,6 @@ ENV_VAR_PREFIX = "AIRFLOW__" -EMPTY_SET: Set[tuple[str, str]] = set() # noqa: UP006 - def _parse_sqlite_version(s: str) -> tuple[int, ...]: match = _SQLITE3_VERSION_PATTERN.match(s) @@ -299,7 +298,9 @@ def get_default_pre_2_7_value(self, section: str, key: str, **kwargs) -> Any: @functools.cached_property def sensitive_config_values(self) -> Set[tuple[str, str]]: # noqa: UP006 if self.configuration_description is None: - return EMPTY_SET.copy() # we can't use set() here because set is defined below # ¯\_(ツ)_/¯ + return ( + _get_empty_set_for_configuration() + ) # we can't use set() here because set is defined below # ¯\_(ツ)_/¯ flattened = { (s, k): item for s, s_c in self.configuration_description.items() @@ -2313,6 +2314,6 @@ def initialize_auth_manager() -> BaseAuthManager: FERNET_KEY = "" # Set only if needed when generating a new file WEBSERVER_CONFIG = "" # Set by initialize_config -conf = initialize_config() +conf: AirflowConfigParser = initialize_config() secrets_backend_list = initialize_secrets_backends() conf.validate() diff --git a/airflow/decorators/__init__.pyi b/airflow/decorators/__init__.pyi index 92fa4dda5b1e..7b2ac1c9ce62 100644 --- a/airflow/decorators/__init__.pyi +++ b/airflow/decorators/__init__.pyi @@ -33,8 +33,8 @@ from airflow.decorators.python_virtualenv import virtualenv_task from airflow.decorators.sensor import sensor_task from airflow.decorators.short_circuit import short_circuit_task from airflow.decorators.task_group import task_group -from airflow.kubernetes.secret import Secret from airflow.models.dag import dag +from airflow.providers.cncf.kubernetes.secret import Secret # Please keep this in sync with __init__.py's __all__. __all__ = [ diff --git a/airflow/example_dags/example_kubernetes_executor.py b/airflow/example_dags/example_kubernetes_executor.py index b9e6bdba3544..a3d6570ac898 100644 --- a/airflow/example_dags/example_kubernetes_executor.py +++ b/airflow/example_dags/example_kubernetes_executor.py @@ -32,9 +32,6 @@ log = logging.getLogger(__name__) -worker_container_repository = conf.get("kubernetes_executor", "worker_container_repository") -worker_container_tag = conf.get("kubernetes_executor", "worker_container_tag") - try: from kubernetes.client import models as k8s except ImportError: @@ -163,6 +160,8 @@ def other_namespace_task(): print_stuff() other_ns_task = other_namespace_task() + worker_container_repository = conf.get("kubernetes_executor", "worker_container_repository") + worker_container_tag = conf.get("kubernetes_executor", "worker_container_tag") # You can also change the base image, here we used the worker image for demonstration. # Note that the image must have the same configuration as the diff --git a/airflow/exceptions.py b/airflow/exceptions.py index 8c65a1f66f97..ea162fe8db15 100644 --- a/airflow/exceptions.py +++ b/airflow/exceptions.py @@ -375,12 +375,28 @@ class TaskDeferralError(AirflowException): """Raised when a task failed during deferral for some reason.""" -class PodMutationHookException(AirflowException): - """Raised when exception happens during Pod Mutation Hook execution.""" - - -class PodReconciliationError(AirflowException): - """Raised when an error is encountered while trying to merge pod configs.""" +# The try/except handling is needed after we moved all k8s classes to cncf.kubernetes provider +# These two exceptions are used internally by Kubernetes Executor but also by PodGenerator, so we need +# to leave them here in case older version of cncf.kubernetes provider is used to run KubernetesPodOperator +# and it raises one of those exceptions. The code should be backwards compatible even if you import +# and try/except the exception using direct imports from airflow.exceptions. +# 1) if you have old provider, both provider and pod generator will throw the "airflow.exceptions" exception. +# 2) if you have new provider, both provider and pod generator will throw the +# "airflow.providers.cncf.kubernetes" as it will be imported here from the provider. +try: + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import PodMutationHookException +except ImportError: + + class PodMutationHookException(AirflowException): # type: ignore[no-redef] + """Raised when exception happens during Pod Mutation Hook execution.""" + + +try: + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import PodReconciliationError +except ImportError: + + class PodReconciliationError(AirflowException): # type: ignore[no-redef] + """Raised when an error is encountered while trying to merge pod configs.""" class RemovedInAirflow3Warning(DeprecationWarning): diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py index ceeec523397b..f6e18b20a81a 100644 --- a/airflow/executors/__init__.py +++ b/airflow/executors/__init__.py @@ -31,6 +31,34 @@ "dask_executor": { "DaskExecutor": "airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor", }, + "kubernetes_executor": { + "KubernetesExecutor": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor.KubernetesExecutor", + }, + "kubernetes_executor_types": { + "ALL_NAMESPACES": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor_types.ALL_NAMESPACES", + "POD_EXECUTOR_DONE_KEY": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor_types.POD_EXECUTOR_DONE_KEY", + }, + "kubernetes_executor_utils": { + "AirflowKubernetesScheduler": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor_utils.AirflowKubernetesScheduler", + "KubernetesJobWatcher": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor_utils.KubernetesJobWatcher", + "ResourceVersion": "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor_utils.ResourceVersion", + }, + "local_kubernetes_executor": { + "LocalKubernetesExecutor": "airflow.providers.cncf.kubernetes.executors.LocalKubernetesExecutor", + }, } -add_deprecated_classes(__deprecated_classes, __name__) +add_deprecated_classes( + __deprecated_classes, + __name__, + {}, + "For Celery executors, the `celery` provider should be >= 3.3.0. " + "For Kubernetes executors, the `cncf.kubernetes` provider should be >= 7.4.0 for that. " + "For Dask executors, any version of `daskexecutor` provider is needed.", +) diff --git a/airflow/executors/executor_loader.py b/airflow/executors/executor_loader.py index 3553a7118360..ca21bdf05ac8 100644 --- a/airflow/executors/executor_loader.py +++ b/airflow/executors/executor_loader.py @@ -58,13 +58,15 @@ class ExecutorLoader: _default_executor: BaseExecutor | None = None executors = { LOCAL_EXECUTOR: "airflow.executors.local_executor.LocalExecutor", - LOCAL_KUBERNETES_EXECUTOR: "airflow.executors.local_kubernetes_executor.LocalKubernetesExecutor", + LOCAL_KUBERNETES_EXECUTOR: "airflow.providers.cncf.kubernetes." + "executors.local_kubernetes_executor.LocalKubernetesExecutor", SEQUENTIAL_EXECUTOR: "airflow.executors.sequential_executor.SequentialExecutor", CELERY_EXECUTOR: "airflow.providers.celery.executors.celery_executor.CeleryExecutor", CELERY_KUBERNETES_EXECUTOR: "airflow.providers.celery." "executors.celery_kubernetes_executor.CeleryKubernetesExecutor", DASK_EXECUTOR: "airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor", - KUBERNETES_EXECUTOR: "airflow.executors.kubernetes_executor.KubernetesExecutor", + KUBERNETES_EXECUTOR: "airflow.providers.cncf.kubernetes." + "executors.kubernetes_executor.KubernetesExecutor", DEBUG_EXECUTOR: "airflow.executors.debug_executor.DebugExecutor", } @@ -98,9 +100,6 @@ def load_executor(cls, executor_name: str) -> BaseExecutor: :return: an instance of executor class via executor_name """ - from airflow.providers_manager import ProvidersManager - - ProvidersManager().initialize_providers_configuration() if executor_name == CELERY_KUBERNETES_EXECUTOR: return cls.__load_celery_kubernetes_executor() elif executor_name == LOCAL_KUBERNETES_EXECUTOR: diff --git a/airflow/kubernetes/__init__.py b/airflow/kubernetes/__init__.py index 13a83393a912..cf4e9a9591ae 100644 --- a/airflow/kubernetes/__init__.py +++ b/airflow/kubernetes/__init__.py @@ -14,3 +14,128 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from __future__ import annotations + +from airflow.utils.deprecation_tools import add_deprecated_classes + +__deprecated_classes: dict[str, dict[str, str]] = { + "kubernetes_helper_functions": { + "add_pod_suffix": "airflow.providers.cncf.kubernetes.kubernetes_helper_functions.add_pod_suffix.", + "annotations_for_logging_task_metadata": "airflow.providers.cncf.kubernetes." + "kubernetes_helper_functions." + "annotations_for_logging_task_metadata.", + "annotations_to_key": "airflow.providers.cncf.kubernetes." + "kubernetes_helper_functions.annotations_to_key", + "create_pod_id": "airflow.providers.cncf.kubernetes.kubernetes_helper_functions.create_pod_id", + "get_logs_task_metadata": "airflow.providers.cncf.kubernetes." + "kubernetes_helper_functions.get_logs_task_metadata", + "rand_str": "airflow.providers.cncf.kubernetes.kubernetes_helper_functions.rand_str", + }, + "pod": { + "Port": "airflow.providers.cncf.kubernetes.backcompat.pod.Port", + "Resources": "airflow.providers.cncf.kubernetes.backcompat.pod.Resources", + }, + "pod_launcher": { + "PodLauncher": "airflow.providers.cncf.kubernetes.pod_launcher.PodLauncher", + "PodStatus": "airflow.providers.cncf.kubernetes.pod_launcher.PodStatus", + }, + "pod_launcher_deprecated": { + "PodLauncher": "airflow.providers.cncf.kubernetes.pod_launcher_deprecated.PodLauncher", + "PodStatus": "airflow.providers.cncf.kubernetes.pod_launcher_deprecated.PodStatus", + }, + "pod_runtime_info_env": { + "PodRuntimeInfoEnv": "airflow.providers.cncf.kubernetes.backcompat." + "pod_runtime_info_env.PodRuntimeInfoEnv", + }, + "volume": { + "Volume": "airflow.providers.cncf.kubernetes.backcompat.volume.Volume", + }, + "volume_mount": { + "VolumeMount": "airflow.providers.cncf.kubernetes.backcompat.volume_mount.VolumeMount", + }, + # the below classes are not served from provider but from internal pre_7_4_0_compatibility package + "k8s_model": { + "K8SModel": "airflow.kubernetes.pre_7_4_0_compatibility.k8s_model.K8SModel", + "append_to_pod": "airflow.kubernetes.pre_7_4_0_compatibility.k8s_model.append_to_pod", + }, + "kube_client": { + "_disable_verify_ssl": "airflow.kubernetes.pre_7_4_0_compatibility.kube_client._disable_verify_ssl", + "_enable_tcp_keepalive": "airflow.kubernetes.pre_7_4_0_compatibility.kube_client." + "_enable_tcp_keepalive", + "get_kube_client": "airflow.kubernetes.pre_7_4_0_compatibility.kube_client.get_kube_client", + }, + "pod_generator": { + "datetime_to_label_safe_datestring": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator" + ".datetime_to_label_safe_datestring", + "extend_object_field": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator." + "extend_object_field", + "label_safe_datestring_to_datetime": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator." + "label_safe_datestring_to_datetime", + "make_safe_label_value": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator." + "make_safe_label_value", + "merge_objects": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator.merge_objects", + "PodGenerator": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator.PodGenerator", + }, + "pod_generator_deprecated": { + "make_safe_label_value": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator_deprecated." + "make_safe_label_value", + "PodDefaults": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator_deprecated.PodDefaults", + "PodGenerator": "airflow.kubernetes.pre_7_4_0_compatibility.pod_generator_deprecated.PodGenerator", + }, + "secret": { + "Secret": "airflow.kubernetes.pre_7_4_0_compatibility.secret.Secret", + }, +} + +__override_deprecated_names: dict[str, dict[str, str]] = { + "pod": { + "Port": "kubernetes.client.models.V1ContainerPort", + "Resources": "kubernetes.client.models.V1ResourceRequirements", + }, + "pod_runtime_info_env": { + "PodRuntimeInfoEnv": "kubernetes.client.models.V1EnvVar", + }, + "volume": { + "Volume": "kubernetes.client.models.V1Volume", + }, + "volume_mount": { + "VolumeMount": "kubernetes.client.models.V1VolumeMount", + }, + "k8s_model": { + "K8SModel": "airflow.airflow.providers.cncf.kubernetes.k8s_model.K8SModel", + "append_to_pod": "airflow.airflow.providers.cncf.kubernetes.k8s_model.append_to_pod", + }, + "kube_client": { + "_disable_verify_ssl": "airflow.kubernetes.airflow.providers.cncf.kubernetes." + "kube_client._disable_verify_ssl", + "_enable_tcp_keepalive": "airflow.kubernetes.airflow.providers.cncf.kubernetes.kube_client." + "_enable_tcp_keepalive", + "get_kube_client": "airflow.kubernetes.airflow.providers.cncf.kubernetes.kube_client.get_kube_client", + }, + "pod_generator": { + "datetime_to_label_safe_datestring": "airflow.providers.cncf.kubernetes.pod_generator" + ".datetime_to_label_safe_datestring", + "extend_object_field": "airflow.kubernetes.airflow.providers.cncf.kubernetes.pod_generator." + "extend_object_field", + "label_safe_datestring_to_datetime": "airflow.providers.cncf.kubernetes.pod_generator." + "label_safe_datestring_to_datetime", + "make_safe_label_value": "airflow.providers.cncf.kubernetes.pod_generator.make_safe_label_value", + "merge_objects": "airflow.providers.cncf.kubernetes.pod_generator.merge_objects", + "PodGenerator": "airflow.providers.cncf.kubernetes.pod_generator.PodGenerator", + }, + "pod_generator_deprecated": { + "make_safe_label_value": "airflow.providers.cncf.kubernetes.pod_generator_deprecated." + "make_safe_label_value", + "PodDefaults": "airflow.providers.cncf.kubernetes.pod_generator_deprecated.PodDefaults", + "PodGenerator": "airflow.providers.cncf.kubernetes.pod_generator_deprecated.PodGenerator", + }, + "secret": { + "Secret": "airflow.providers.cncf.kubernetes.secret.Secret", + }, +} +add_deprecated_classes( + __deprecated_classes, + __name__, + __override_deprecated_names, + "The `cncf.kubernetes` provider must be >= 7.4.0 for that.", +) diff --git a/airflow/kubernetes/pod.py b/airflow/kubernetes/pre_7_4_0_compatibility/__init__.py similarity index 57% rename from airflow/kubernetes/pod.py rename to airflow/kubernetes/pre_7_4_0_compatibility/__init__.py index 629cbad17c78..18c84b6d03e5 100644 --- a/airflow/kubernetes/pod.py +++ b/airflow/kubernetes/pre_7_4_0_compatibility/__init__.py @@ -1,4 +1,3 @@ -# # 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 @@ -15,24 +14,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -""" -This module is deprecated. -Please use :mod:`kubernetes.client.models` for `V1ResourceRequirements` and `Port`. -""" from __future__ import annotations +# All the classes in this module should only be kept for backwards-compatibility reasons. +# old cncf.kubernetes providers will use those in their frozen version for pre-7.4.0 release import warnings -from airflow.exceptions import RemovedInAirflow3Warning - -# flake8: noqa - -with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources - warnings.warn( - "This module is deprecated. Please use `kubernetes.client.models` for `V1ResourceRequirements` and `Port`.", - RemovedInAirflow3Warning, + "This module is deprecated. The `cncf.kubernetes` provider before version 7.4.0 uses this module - " + "you should migrate to a newer version of `cncf.kubernetes` to get rid of this warning. If you " + "import the module via `airflow.kubernetes` import, please use `cncf.kubernetes' " + "provider 7.4.0+ and switch all your imports to use `apache.airflow.providers.cncf.kubernetes` " + "to get rid of the warning.", + DeprecationWarning, stacklevel=2, ) diff --git a/airflow/kubernetes/k8s_model.py b/airflow/kubernetes/pre_7_4_0_compatibility/k8s_model.py similarity index 100% rename from airflow/kubernetes/k8s_model.py rename to airflow/kubernetes/pre_7_4_0_compatibility/k8s_model.py diff --git a/airflow/kubernetes/kube_client.py b/airflow/kubernetes/pre_7_4_0_compatibility/kube_client.py similarity index 100% rename from airflow/kubernetes/kube_client.py rename to airflow/kubernetes/pre_7_4_0_compatibility/kube_client.py diff --git a/airflow/kubernetes/pre_7_4_0_compatibility/pod_generator.py b/airflow/kubernetes/pre_7_4_0_compatibility/pod_generator.py new file mode 100644 index 000000000000..aaacc8ce45fb --- /dev/null +++ b/airflow/kubernetes/pre_7_4_0_compatibility/pod_generator.py @@ -0,0 +1,666 @@ +# 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. +""" +Pod generator compatible with cncf-providers released before 2.7.0 of airflow (so pre-7.4.0 of +the cncf.kubernetes provider). + +This module provides an interface between the previous Pod +API and outputs a kubernetes.client.models.V1Pod. +The advantage being that the full Kubernetes API +is supported and no serialization need be written. +""" +from __future__ import annotations + +import copy +import datetime +import logging +import os +import secrets +import string +import warnings +from functools import reduce + +import re2 +from dateutil import parser +from kubernetes.client import models as k8s +from kubernetes.client.api_client import ApiClient + +from airflow.exceptions import ( + AirflowConfigException, + PodMutationHookException, + PodReconciliationError, + RemovedInAirflow3Warning, +) +from airflow.kubernetes.pre_7_4_0_compatibility.pod_generator_deprecated import ( + PodDefaults, + PodGenerator as PodGeneratorDeprecated, +) +from airflow.utils import yaml +from airflow.utils.hashlib_wrapper import md5 +from airflow.version import version as airflow_version + +log = logging.getLogger(__name__) + +MAX_LABEL_LEN = 63 + +alphanum_lower = string.ascii_lowercase + string.digits + + +def rand_str(num): + """Generate random lowercase alphanumeric string of length num. + + :meta private: + """ + return "".join(secrets.choice(alphanum_lower) for _ in range(num)) + + +def add_pod_suffix(pod_name: str, rand_len: int = 8, max_len: int = 80) -> str: + """Add random string to pod name while staying under max length. + + :param pod_name: name of the pod + :param rand_len: length of the random string to append + :max_len: maximum length of the pod name + :meta private: + """ + suffix = "-" + rand_str(rand_len) + return pod_name[: max_len - len(suffix)].strip("-.") + suffix + + +def make_safe_label_value(string: str) -> str: + """ + Normalize a provided label to be of valid length and characters. + + Valid label values must be 63 characters or less and must be empty or begin and + end with an alphanumeric character ([a-z0-9A-Z]) with dashes (-), underscores (_), + dots (.), and alphanumerics between. + + If the label value is greater than 63 chars once made safe, or differs in any + way from the original value sent to this function, then we need to truncate to + 53 chars, and append it with a unique hash. + """ + safe_label = re2.sub(r"^[^a-z0-9A-Z]*|[^a-zA-Z0-9_\-\.]|[^a-z0-9A-Z]*$", "", string) + + if len(safe_label) > MAX_LABEL_LEN or string != safe_label: + safe_hash = md5(string.encode()).hexdigest()[:9] + safe_label = safe_label[: MAX_LABEL_LEN - len(safe_hash) - 1] + "-" + safe_hash + + return safe_label + + +def datetime_to_label_safe_datestring(datetime_obj: datetime.datetime) -> str: + """ + Transform a datetime string to use as a label. + + Kubernetes doesn't like ":" in labels, since ISO datetime format uses ":" but + not "_" let's + replace ":" with "_" + + :param datetime_obj: datetime.datetime object + :return: ISO-like string representing the datetime + """ + return datetime_obj.isoformat().replace(":", "_").replace("+", "_plus_") + + +def label_safe_datestring_to_datetime(string: str) -> datetime.datetime: + """ + Transform a label back to a datetime object. + + Kubernetes doesn't permit ":" in labels. ISO datetime format uses ":" but not + "_", let's + replace ":" with "_" + + :param string: str + :return: datetime.datetime object + """ + return parser.parse(string.replace("_plus_", "+").replace("_", ":")) + + +class PodGenerator: + """ + Contains Kubernetes Airflow Worker configuration logic. + + Represents a kubernetes pod and manages execution of a single pod. + Any configuration that is container specific gets applied to + the first container in the list of containers. + + :param pod: The fully specified pod. Mutually exclusive with `pod_template_file` + :param pod_template_file: Path to YAML file. Mutually exclusive with `pod` + :param extract_xcom: Whether to bring up a container for xcom + """ + + def __init__( + self, + pod: k8s.V1Pod | None = None, + pod_template_file: str | None = None, + extract_xcom: bool = True, + ): + if not pod_template_file and not pod: + raise AirflowConfigException( + "Podgenerator requires either a `pod` or a `pod_template_file` argument" + ) + if pod_template_file and pod: + raise AirflowConfigException("Cannot pass both `pod` and `pod_template_file` arguments") + + if pod_template_file: + self.ud_pod = self.deserialize_model_file(pod_template_file) + else: + self.ud_pod = pod + + # Attach sidecar + self.extract_xcom = extract_xcom + + def gen_pod(self) -> k8s.V1Pod: + """Generates pod.""" + warnings.warn("This function is deprecated. ", RemovedInAirflow3Warning) + result = self.ud_pod + + result.metadata.name = add_pod_suffix(pod_name=result.metadata.name) + + if self.extract_xcom: + result = self.add_xcom_sidecar(result) + + return result + + @staticmethod + def add_xcom_sidecar(pod: k8s.V1Pod) -> k8s.V1Pod: + """Adds sidecar.""" + warnings.warn( + "This function is deprecated. " + "Please use airflow.providers.cncf.kubernetes.utils.xcom_sidecar.add_xcom_sidecar instead" + ) + pod_cp = copy.deepcopy(pod) + pod_cp.spec.volumes = pod.spec.volumes or [] + pod_cp.spec.volumes.insert(0, PodDefaults.VOLUME) + pod_cp.spec.containers[0].volume_mounts = pod_cp.spec.containers[0].volume_mounts or [] + pod_cp.spec.containers[0].volume_mounts.insert(0, PodDefaults.VOLUME_MOUNT) + pod_cp.spec.containers.append(PodDefaults.SIDECAR_CONTAINER) + + return pod_cp + + @staticmethod + def from_obj(obj) -> dict | k8s.V1Pod | None: + """Converts to pod from obj.""" + if obj is None: + return None + + k8s_legacy_object = obj.get("KubernetesExecutor", None) + k8s_object = obj.get("pod_override", None) + + if k8s_legacy_object and k8s_object: + raise AirflowConfigException( + "Can not have both a legacy and new" + "executor_config object. Please delete the KubernetesExecutor" + "dict and only use the pod_override kubernetes.client.models.V1Pod" + "object." + ) + if not k8s_object and not k8s_legacy_object: + return None + + if isinstance(k8s_object, k8s.V1Pod): + return k8s_object + elif isinstance(k8s_legacy_object, dict): + warnings.warn( + "Using a dictionary for the executor_config is deprecated and will soon be removed." + 'please use a `kubernetes.client.models.V1Pod` class with a "pod_override" key' + " instead. ", + category=RemovedInAirflow3Warning, + ) + return PodGenerator.from_legacy_obj(obj) + else: + raise TypeError( + "Cannot convert a non-kubernetes.client.models.V1Pod object into a KubernetesExecutorConfig" + ) + + @staticmethod + def from_legacy_obj(obj) -> k8s.V1Pod | None: + """Converts to pod from obj.""" + if obj is None: + return None + + # We do not want to extract constant here from ExecutorLoader because it is just + # A name in dictionary rather than executor selection mechanism and it causes cyclic import + namespaced = obj.get("KubernetesExecutor", {}) + + if not namespaced: + return None + + resources = namespaced.get("resources") + + if resources is None: + requests = { + "cpu": namespaced.pop("request_cpu", None), + "memory": namespaced.pop("request_memory", None), + "ephemeral-storage": namespaced.get("ephemeral-storage"), # We pop this one in limits + } + limits = { + "cpu": namespaced.pop("limit_cpu", None), + "memory": namespaced.pop("limit_memory", None), + "ephemeral-storage": namespaced.pop("ephemeral-storage", None), + } + all_resources = list(requests.values()) + list(limits.values()) + if all(r is None for r in all_resources): + resources = None + else: + # remove None's so they don't become 0's + requests = {k: v for k, v in requests.items() if v is not None} + limits = {k: v for k, v in limits.items() if v is not None} + resources = k8s.V1ResourceRequirements(requests=requests, limits=limits) + namespaced["resources"] = resources + return PodGeneratorDeprecated(**namespaced).gen_pod() + + @staticmethod + def reconcile_pods(base_pod: k8s.V1Pod, client_pod: k8s.V1Pod | None) -> k8s.V1Pod: + """ + Merge Kubernetes Pod objects. + + :param base_pod: has the base attributes which are overwritten if they exist + in the client pod and remain if they do not exist in the client_pod + :param client_pod: the pod that the client wants to create. + :return: the merged pods + + This can't be done recursively as certain fields are overwritten and some are concatenated. + """ + if client_pod is None: + return base_pod + + client_pod_cp = copy.deepcopy(client_pod) + client_pod_cp.spec = PodGenerator.reconcile_specs(base_pod.spec, client_pod_cp.spec) + client_pod_cp.metadata = PodGenerator.reconcile_metadata(base_pod.metadata, client_pod_cp.metadata) + client_pod_cp = merge_objects(base_pod, client_pod_cp) + + return client_pod_cp + + @staticmethod + def reconcile_metadata(base_meta, client_meta): + """ + Merge Kubernetes Metadata objects. + + :param base_meta: has the base attributes which are overwritten if they exist + in the client_meta and remain if they do not exist in the client_meta + :param client_meta: the spec that the client wants to create. + :return: the merged specs + """ + if base_meta and not client_meta: + return base_meta + if not base_meta and client_meta: + return client_meta + elif client_meta and base_meta: + client_meta.labels = merge_objects(base_meta.labels, client_meta.labels) + client_meta.annotations = merge_objects(base_meta.annotations, client_meta.annotations) + extend_object_field(base_meta, client_meta, "managed_fields") + extend_object_field(base_meta, client_meta, "finalizers") + extend_object_field(base_meta, client_meta, "owner_references") + return merge_objects(base_meta, client_meta) + + return None + + @staticmethod + def reconcile_specs( + base_spec: k8s.V1PodSpec | None, client_spec: k8s.V1PodSpec | None + ) -> k8s.V1PodSpec | None: + """ + Merge Kubernetes PodSpec objects. + + :param base_spec: has the base attributes which are overwritten if they exist + in the client_spec and remain if they do not exist in the client_spec + :param client_spec: the spec that the client wants to create. + :return: the merged specs + """ + if base_spec and not client_spec: + return base_spec + if not base_spec and client_spec: + return client_spec + elif client_spec and base_spec: + client_spec.containers = PodGenerator.reconcile_containers( + base_spec.containers, client_spec.containers + ) + merged_spec = extend_object_field(base_spec, client_spec, "init_containers") + merged_spec = extend_object_field(base_spec, merged_spec, "volumes") + return merge_objects(base_spec, merged_spec) + + return None + + @staticmethod + def reconcile_containers( + base_containers: list[k8s.V1Container], client_containers: list[k8s.V1Container] + ) -> list[k8s.V1Container]: + """ + Merge Kubernetes Container objects. + + :param base_containers: has the base attributes which are overwritten if they exist + in the client_containers and remain if they do not exist in the client_containers + :param client_containers: the containers that the client wants to create. + :return: the merged containers + + The runs recursively over the list of containers. + """ + if not base_containers: + return client_containers + if not client_containers: + return base_containers + + client_container = client_containers[0] + base_container = base_containers[0] + client_container = extend_object_field(base_container, client_container, "volume_mounts") + client_container = extend_object_field(base_container, client_container, "env") + client_container = extend_object_field(base_container, client_container, "env_from") + client_container = extend_object_field(base_container, client_container, "ports") + client_container = extend_object_field(base_container, client_container, "volume_devices") + client_container = merge_objects(base_container, client_container) + + return [client_container] + PodGenerator.reconcile_containers( + base_containers[1:], client_containers[1:] + ) + + @classmethod + def construct_pod( + cls, + dag_id: str, + task_id: str, + pod_id: str, + try_number: int, + kube_image: str, + date: datetime.datetime | None, + args: list[str], + pod_override_object: k8s.V1Pod | None, + base_worker_pod: k8s.V1Pod, + namespace: str, + scheduler_job_id: str, + run_id: str | None = None, + map_index: int = -1, + *, + with_mutation_hook: bool = False, + ) -> k8s.V1Pod: + """ + Create a Pod. + + Construct a pod by gathering and consolidating the configuration from 3 places: + - airflow.cfg + - executor_config + - dynamic arguments + """ + if len(pod_id) > 253: + warnings.warn( + "pod_id supplied is longer than 253 characters; truncating and adding unique suffix." + ) + pod_id = add_pod_suffix(pod_name=pod_id, max_len=253) + try: + image = pod_override_object.spec.containers[0].image # type: ignore + if not image: + image = kube_image + except Exception: + image = kube_image + + annotations = { + "dag_id": dag_id, + "task_id": task_id, + "try_number": str(try_number), + } + if map_index >= 0: + annotations["map_index"] = str(map_index) + if date: + annotations["execution_date"] = date.isoformat() + if run_id: + annotations["run_id"] = run_id + + dynamic_pod = k8s.V1Pod( + metadata=k8s.V1ObjectMeta( + namespace=namespace, + annotations=annotations, + name=pod_id, + labels=cls.build_labels_for_k8s_executor_pod( + dag_id=dag_id, + task_id=task_id, + try_number=try_number, + airflow_worker=scheduler_job_id, + map_index=map_index, + execution_date=date, + run_id=run_id, + ), + ), + spec=k8s.V1PodSpec( + containers=[ + k8s.V1Container( + name="base", + args=args, + image=image, + env=[k8s.V1EnvVar(name="AIRFLOW_IS_K8S_EXECUTOR_POD", value="True")], + ) + ] + ), + ) + + # Reconcile the pods starting with the first chronologically, + # Pod from the pod_template_File -> Pod from executor_config arg -> Pod from the K8s executor + pod_list = [base_worker_pod, pod_override_object, dynamic_pod] + + try: + pod = reduce(PodGenerator.reconcile_pods, pod_list) + except Exception as e: + raise PodReconciliationError from e + + if with_mutation_hook: + from airflow.settings import pod_mutation_hook + + try: + pod_mutation_hook(pod) + except Exception as e: + raise PodMutationHookException from e + + return pod + + @classmethod + def build_selector_for_k8s_executor_pod( + cls, + *, + dag_id, + task_id, + try_number, + map_index=None, + execution_date=None, + run_id=None, + airflow_worker=None, + ): + """ + Generate selector for kubernetes executor pod. + + :meta private: + """ + labels = cls.build_labels_for_k8s_executor_pod( + dag_id=dag_id, + task_id=task_id, + try_number=try_number, + map_index=map_index, + execution_date=execution_date, + run_id=run_id, + airflow_worker=airflow_worker, + ) + label_strings = [f"{label_id}={label}" for label_id, label in sorted(labels.items())] + selector = ",".join(label_strings) + if not airflow_worker: # this filters out KPO pods even when we don't know the scheduler job id + selector += ",airflow-worker" + return selector + + @classmethod + def build_labels_for_k8s_executor_pod( + cls, + *, + dag_id, + task_id, + try_number, + airflow_worker=None, + map_index=None, + execution_date=None, + run_id=None, + ): + """ + Generate labels for kubernetes executor pod. + + :meta private: + """ + labels = { + "dag_id": make_safe_label_value(dag_id), + "task_id": make_safe_label_value(task_id), + "try_number": str(try_number), + "kubernetes_executor": "True", + "airflow_version": airflow_version.replace("+", "-"), + } + if airflow_worker is not None: + labels["airflow-worker"] = make_safe_label_value(str(airflow_worker)) + if map_index is not None and map_index >= 0: + labels["map_index"] = str(map_index) + if execution_date: + labels["execution_date"] = datetime_to_label_safe_datestring(execution_date) + if run_id: + labels["run_id"] = make_safe_label_value(run_id) + return labels + + @staticmethod + def serialize_pod(pod: k8s.V1Pod) -> dict: + """ + Convert a k8s.V1Pod into a json serializable dictionary. + + :param pod: k8s.V1Pod object + :return: Serialized version of the pod returned as dict + """ + api_client = ApiClient() + return api_client.sanitize_for_serialization(pod) + + @staticmethod + def deserialize_model_file(path: str) -> k8s.V1Pod: + """ + Generate a Pod from a file. + + :param path: Path to the file + :return: a kubernetes.client.models.V1Pod + """ + if os.path.exists(path): + with open(path) as stream: + pod = yaml.safe_load(stream) + else: + pod = None + log.warning("Model file %s does not exist", path) + + return PodGenerator.deserialize_model_dict(pod) + + @staticmethod + def deserialize_model_dict(pod_dict: dict | None) -> k8s.V1Pod: + """ + Deserializes a Python dictionary to k8s.V1Pod. + + Unfortunately we need access to the private method + ``_ApiClient__deserialize_model`` from the kubernetes client. + This issue is tracked here; https://github.com/kubernetes-client/python/issues/977. + + :param pod_dict: Serialized dict of k8s.V1Pod object + :return: De-serialized k8s.V1Pod + """ + api_client = ApiClient() + return api_client._ApiClient__deserialize_model(pod_dict, k8s.V1Pod) + + @staticmethod + def make_unique_pod_id(pod_id: str) -> str | None: + r""" + Generate a unique Pod name. + + Kubernetes pod names must consist of one or more lowercase + rfc1035/rfc1123 labels separated by '.' with a maximum length of 253 + characters. + + Name must pass the following regex for validation + ``^[a-z0-9]([-a-z0-9]*[a-z0-9])?(\\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*$`` + + For more details, see: + https://github.com/kubernetes/kubernetes/blob/release-1.1/docs/design/identifiers.md + + :param pod_id: requested pod name + :return: ``str`` valid Pod name of appropriate length + """ + warnings.warn( + "This function is deprecated. Use `add_pod_suffix` in `kubernetes_helper_functions`.", + RemovedInAirflow3Warning, + ) + + if not pod_id: + return None + + max_pod_id_len = 100 # arbitrarily chosen + suffix = rand_str(8) # 8 seems good enough + base_pod_id_len = max_pod_id_len - len(suffix) - 1 # -1 for separator + trimmed_pod_id = pod_id[:base_pod_id_len].rstrip("-.") + return f"{trimmed_pod_id}-{suffix}" + + +def merge_objects(base_obj, client_obj): + """ + Merge objects. + + :param base_obj: has the base attributes which are overwritten if they exist + in the client_obj and remain if they do not exist in the client_obj + :param client_obj: the object that the client wants to create. + :return: the merged objects + """ + if not base_obj: + return client_obj + if not client_obj: + return base_obj + + client_obj_cp = copy.deepcopy(client_obj) + + if isinstance(base_obj, dict) and isinstance(client_obj_cp, dict): + base_obj_cp = copy.deepcopy(base_obj) + base_obj_cp.update(client_obj_cp) + return base_obj_cp + + for base_key in base_obj.to_dict().keys(): + base_val = getattr(base_obj, base_key, None) + if not getattr(client_obj, base_key, None) and base_val: + if not isinstance(client_obj_cp, dict): + setattr(client_obj_cp, base_key, base_val) + else: + client_obj_cp[base_key] = base_val + return client_obj_cp + + +def extend_object_field(base_obj, client_obj, field_name): + """ + Add field values to existing objects. + + :param base_obj: an object which has a property `field_name` that is a list + :param client_obj: an object which has a property `field_name` that is a list. + A copy of this object is returned with `field_name` modified + :param field_name: the name of the list field + :return: the client_obj with the property `field_name` being the two properties appended + """ + client_obj_cp = copy.deepcopy(client_obj) + base_obj_field = getattr(base_obj, field_name, None) + client_obj_field = getattr(client_obj, field_name, None) + + if (not isinstance(base_obj_field, list) and base_obj_field is not None) or ( + not isinstance(client_obj_field, list) and client_obj_field is not None + ): + raise ValueError("The chosen field must be a list.") + + if not base_obj_field: + return client_obj_cp + if not client_obj_field: + setattr(client_obj_cp, field_name, base_obj_field) + return client_obj_cp + + appended_fields = base_obj_field + client_obj_field + setattr(client_obj_cp, field_name, appended_fields) + return client_obj_cp diff --git a/airflow/kubernetes/pod_generator_deprecated.py b/airflow/kubernetes/pre_7_4_0_compatibility/pod_generator_deprecated.py similarity index 100% rename from airflow/kubernetes/pod_generator_deprecated.py rename to airflow/kubernetes/pre_7_4_0_compatibility/pod_generator_deprecated.py diff --git a/airflow/kubernetes/pre_7_4_0_compatibility/secret.py b/airflow/kubernetes/pre_7_4_0_compatibility/secret.py new file mode 100644 index 000000000000..14295f5c7a89 --- /dev/null +++ b/airflow/kubernetes/pre_7_4_0_compatibility/secret.py @@ -0,0 +1,124 @@ +# 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. +"""Classes for interacting with Kubernetes API.""" +from __future__ import annotations + +import copy +import uuid + +from kubernetes.client import models as k8s + +from airflow.exceptions import AirflowConfigException +from airflow.kubernetes.pre_7_4_0_compatibility.k8s_model import K8SModel + + +class Secret(K8SModel): + """Defines Kubernetes Secret Volume.""" + + def __init__(self, deploy_type, deploy_target, secret, key=None, items=None): + """ + Initialize a Kubernetes Secret Object. + + Used to track requested secrets from the user. + + :param deploy_type: The type of secret deploy in Kubernetes, either `env` or + `volume` + :param deploy_target: (Optional) The environment variable when + `deploy_type` `env` or file path when `deploy_type` `volume` where + expose secret. If `key` is not provided deploy target should be None. + :param secret: Name of the secrets object in Kubernetes + :param key: (Optional) Key of the secret within the Kubernetes Secret + if not provided in `deploy_type` `env` it will mount all secrets in object + :param items: (Optional) items that can be added to a volume secret for specifying projects of + secret keys to paths + https://kubernetes.io/docs/concepts/configuration/secret/#projection-of-secret-keys-to-specific-paths + """ + if deploy_type not in ("env", "volume"): + raise AirflowConfigException("deploy_type must be env or volume") + + self.deploy_type = deploy_type + self.deploy_target = deploy_target + self.items = items or [] + + if deploy_target is not None and deploy_type == "env": + # if deploying to env, capitalize the deploy target + self.deploy_target = deploy_target.upper() + + if key is not None and deploy_target is None: + raise AirflowConfigException("If `key` is set, `deploy_target` should not be None") + + self.secret = secret + self.key = key + + def to_env_secret(self) -> k8s.V1EnvVar: + """Stores es environment secret.""" + return k8s.V1EnvVar( + name=self.deploy_target, + value_from=k8s.V1EnvVarSource( + secret_key_ref=k8s.V1SecretKeySelector(name=self.secret, key=self.key) + ), + ) + + def to_env_from_secret(self) -> k8s.V1EnvFromSource: + """Reads from environment to secret.""" + return k8s.V1EnvFromSource(secret_ref=k8s.V1SecretEnvSource(name=self.secret)) + + def to_volume_secret(self) -> tuple[k8s.V1Volume, k8s.V1VolumeMount]: + """Converts to volume secret.""" + vol_id = f"secretvol{uuid.uuid4()}" + volume = k8s.V1Volume(name=vol_id, secret=k8s.V1SecretVolumeSource(secret_name=self.secret)) + if self.items: + volume.secret.items = self.items + return (volume, k8s.V1VolumeMount(mount_path=self.deploy_target, name=vol_id, read_only=True)) + + def attach_to_pod(self, pod: k8s.V1Pod) -> k8s.V1Pod: + """Attaches to pod.""" + cp_pod = copy.deepcopy(pod) + + if self.deploy_type == "volume": + volume, volume_mount = self.to_volume_secret() + if cp_pod.spec.volumes is None: + cp_pod.spec.volumes = [] + cp_pod.spec.volumes.append(volume) + if cp_pod.spec.containers[0].volume_mounts is None: + cp_pod.spec.containers[0].volume_mounts = [] + cp_pod.spec.containers[0].volume_mounts.append(volume_mount) + + if self.deploy_type == "env" and self.key is not None: + env = self.to_env_secret() + if cp_pod.spec.containers[0].env is None: + cp_pod.spec.containers[0].env = [] + cp_pod.spec.containers[0].env.append(env) + + if self.deploy_type == "env" and self.key is None: + env_from = self.to_env_from_secret() + if cp_pod.spec.containers[0].env_from is None: + cp_pod.spec.containers[0].env_from = [] + cp_pod.spec.containers[0].env_from.append(env_from) + + return cp_pod + + def __eq__(self, other): + return ( + self.deploy_type == other.deploy_type + and self.deploy_target == other.deploy_target + and self.secret == other.secret + and self.key == other.key + ) + + def __repr__(self): + return f"Secret({self.deploy_type}, {self.deploy_target}, {self.secret}, {self.key})" diff --git a/airflow/models/base.py b/airflow/models/base.py index 9965de7ec707..5f6b7e9893dc 100644 --- a/airflow/models/base.py +++ b/airflow/models/base.py @@ -74,7 +74,7 @@ def get_id_collation_args(): return {} -COLLATION_ARGS = get_id_collation_args() +COLLATION_ARGS: dict[str, Any] = get_id_collation_args() def StringID(*, length=ID_LEN, **kwargs) -> String: diff --git a/airflow/models/renderedtifields.py b/airflow/models/renderedtifields.py index 4b2f80e47d1b..269af8276abe 100644 --- a/airflow/models/renderedtifields.py +++ b/airflow/models/renderedtifields.py @@ -101,7 +101,11 @@ def __init__(self, ti: TaskInstance, render_templates=True): ti.render_templates() self.task = ti.task if os.environ.get("AIRFLOW_IS_K8S_EXECUTOR_POD", None): - self.k8s_pod_yaml = ti.render_k8s_pod_yaml() + # we can safely import it here from provider. In Airflow 2.7.0+ you need to have new version + # of kubernetes provider installed to reach this place + from airflow.providers.cncf.kubernetes.template_rendering import render_k8s_pod_yaml + + self.k8s_pod_yaml = render_k8s_pod_yaml(ti) self.rendered_fields = { field: serialize_template_field(getattr(self.task, field)) for field in self.task.template_fields } diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 520d07f092d8..6eafbab4a42c 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -2251,19 +2251,6 @@ def get_rendered_template_fields(self, session: Session = NEW_SESSION) -> None: "rendering of template_fields." ) from e - @provide_session - def get_rendered_k8s_spec(self, session: Session = NEW_SESSION): - """Fetch rendered template fields from DB.""" - from airflow.models.renderedtifields import RenderedTaskInstanceFields - - rendered_k8s_spec = RenderedTaskInstanceFields.get_k8s_pod_yaml(self, session=session) - if not rendered_k8s_spec: - try: - rendered_k8s_spec = self.render_k8s_pod_yaml() - except (TemplateAssertionError, UndefinedError) as e: - raise AirflowException(f"Unable to render a k8s spec for this taskinstance: {e}") from e - return rendered_k8s_spec - def overwrite_params_with_dag_run_conf(self, params, dag_run): """Overwrite Task Params with DagRun.conf.""" if dag_run and dag_run.conf: @@ -2290,32 +2277,51 @@ def render_templates(self, context: Context | None = None) -> Operator: return original_task def render_k8s_pod_yaml(self) -> dict | None: - """Render k8s pod yaml.""" - from kubernetes.client.api_client import ApiClient - - from airflow.kubernetes.kube_config import KubeConfig - from airflow.kubernetes.kubernetes_helper_functions import create_pod_id # Circular import - from airflow.kubernetes.pod_generator import PodGenerator + """Render the k8s pod yaml.""" + try: + from airflow.providers.cncf.kubernetes.template_rendering import ( + render_k8s_pod_yaml as render_k8s_pod_yaml_from_provider, + ) + except ImportError: + raise RuntimeError( + "You need to have the `cncf.kubernetes` provider installed to use this feature. " + "Also rather than calling it directly you should import " + "render_k8s_pod_yaml from airflow.providers.cncf.kubernetes.template_rendering " + "and call it with TaskInstance as the first argument." + ) + warnings.warn( + "You should not call `task_instance.render_k8s_pod_yaml` directly. This method will be removed" + "in Airflow 3. Rather than calling it directly you should import " + "`render_k8s_pod_yaml` from `airflow.providers.cncf.kubernetes.template_rendering` " + "and call it with `TaskInstance` as the first argument.", + DeprecationWarning, + stacklevel=2, + ) + return render_k8s_pod_yaml_from_provider(self) - kube_config = KubeConfig() - pod = PodGenerator.construct_pod( - dag_id=self.dag_id, - run_id=self.run_id, - task_id=self.task_id, - map_index=self.map_index, - date=None, - pod_id=create_pod_id(self.dag_id, self.task_id), - try_number=self.try_number, - kube_image=kube_config.kube_image, - args=self.command_as_list(), - pod_override_object=PodGenerator.from_obj(self.executor_config), - scheduler_job_id="0", - namespace=kube_config.executor_namespace, - base_worker_pod=PodGenerator.deserialize_model_file(kube_config.pod_template_file), - with_mutation_hook=True, + @provide_session + def get_rendered_k8s_spec(self, session: Session = NEW_SESSION): + """Render the k8s pod yaml.""" + try: + from airflow.providers.cncf.kubernetes.template_rendering import ( + get_rendered_k8s_spec as get_rendered_k8s_spec_from_provider, + ) + except ImportError: + raise RuntimeError( + "You need to have the `cncf.kubernetes` provider installed to use this feature. " + "Also rather than calling it directly you should import " + "`get_rendered_k8s_spec` from `airflow.providers.cncf.kubernetes.template_rendering` " + "and call it with `TaskInstance` as the first argument." + ) + warnings.warn( + "You should not call `task_instance.render_k8s_pod_yaml` directly. This method will be removed" + "in Airflow 3. Rather than calling it directly you should import " + "`get_rendered_k8s_spec` from `airflow.providers.cncf.kubernetes.template_rendering` " + "and call it with `TaskInstance` as the first argument.", + DeprecationWarning, + stacklevel=2, ) - sanitized_pod = ApiClient().sanitize_for_serialization(pod) - return sanitized_pod + return get_rendered_k8s_spec_from_provider(self, session=session) def get_email_subject_content( self, exception: BaseException, task: BaseOperator | None = None diff --git a/airflow/providers/amazon/aws/notifications/chime.py b/airflow/providers/amazon/aws/notifications/chime.py index c505b3e227ea..d62b4b21d6fa 100644 --- a/airflow/providers/amazon/aws/notifications/chime.py +++ b/airflow/providers/amazon/aws/notifications/chime.py @@ -53,7 +53,7 @@ def hook(self): """To reduce overhead cache the hook for the notifier.""" return ChimeWebhookHook(chime_conn_id=self.chime_conn_id) - def notify(self, context: Context) -> None: + def notify(self, context: Context) -> None: # type: ignore[override] """Send a message to a Chime Chat Room.""" self.hook.send_message(message=self.message) diff --git a/airflow/providers/apache/spark/CHANGELOG.rst b/airflow/providers/apache/spark/CHANGELOG.rst index 8129b380a320..c855333665e9 100644 --- a/airflow/providers/apache/spark/CHANGELOG.rst +++ b/airflow/providers/apache/spark/CHANGELOG.rst @@ -29,6 +29,25 @@ Changelog --------- +4.1.2 +..... + +.. note:: + + The provider now expects ``apache-airflow-providers-cncf-kubernetes`` in version 7.4.0+ installed + in order to run Spark on Kubernetes jobs. You can install the provider with ``cncf.kubernetes`` extra with + ``pip install apache-airflow-providers-spark[cncf.kubernetes]`` to get the right version of the + ``cncf.kubernetes`` provider installed. + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` + * ``Remove spurious headers for provider changelogs (#32373)`` + * ``Prepare docs for July 2023 wave of Providers (#32298)`` + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` + 4.1.1 ..... diff --git a/airflow/providers/apache/spark/__init__.py b/airflow/providers/apache/spark/__init__.py index 888104a68bf7..298c8a54f929 100644 --- a/airflow/providers/apache/spark/__init__.py +++ b/airflow/providers/apache/spark/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "4.1.1" +__version__ = "4.1.2" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/apache/spark/hooks/spark_submit.py b/airflow/providers/apache/spark/hooks/spark_submit.py index 5687df8ac851..22fd6d14363b 100644 --- a/airflow/providers/apache/spark/hooks/spark_submit.py +++ b/airflow/providers/apache/spark/hooks/spark_submit.py @@ -31,7 +31,7 @@ from airflow.utils.log.logging_mixin import LoggingMixin with contextlib.suppress(ImportError, NameError): - from airflow.kubernetes import kube_client + from airflow.providers.cncf.kubernetes import kube_client ALLOWED_SPARK_BINARIES = ["spark-submit", "spark2-submit", "spark3-submit"] diff --git a/airflow/providers/apache/spark/provider.yaml b/airflow/providers/apache/spark/provider.yaml index 134e7b709e07..30878d49139e 100644 --- a/airflow/providers/apache/spark/provider.yaml +++ b/airflow/providers/apache/spark/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 4.1.2 - 4.1.1 - 4.1.0 - 4.0.1 @@ -76,3 +77,8 @@ connection-types: connection-type: spark_sql - hook-class-name: airflow.providers.apache.spark.hooks.spark_submit.SparkSubmitHook connection-type: spark + +additional-extras: + - name: cncf.kubernetes + dependencies: + - apache-airflow-providers-cncf-kubernetes>=7.4.0 diff --git a/airflow/providers/celery/CHANGELOG.rst b/airflow/providers/celery/CHANGELOG.rst index 0d0038cc366a..807f7db9eabc 100644 --- a/airflow/providers/celery/CHANGELOG.rst +++ b/airflow/providers/celery/CHANGELOG.rst @@ -33,7 +33,22 @@ Changelog .. note:: This provider release is the first release that has Celery Executor and Celery Kubernetes Executor moved from the core ``apache-airflow`` package to a Celery - provider package. + provider package. It also expects ``apache-airflow-providers-cncf-kubernetes`` in version 7.4.0+ installed + in order to use ``CeleryKubernetesExecutor``. You can install the provider with ``cncf.kubernetes`` extra + with ``pip install apache-airflow-providers-celery[cncf.kubernetes]`` to get the right version of the + ``cncf.kubernetes`` provider installed. + +.. Review and move the new changes to one of the sections above: + * ``Introduce decorator to load providers configuration (#32765)`` + * ``Allow configuration to be contributed by providers (#32604)`` + * ``Move default_celery.py to inside the provider (#32628)`` + * ``Add Executors discovery and documentation (#32532)`` + * ``Move CeleryExecutor to the celery provider (#32526)`` + * ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` + * ``Remove spurious headers for provider changelogs (#32373)`` + * ``Prepare docs for July 2023 wave of Providers (#32298)`` + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` 3.2.1 ..... diff --git a/airflow/providers/celery/__init__.py b/airflow/providers/celery/__init__.py index 80c2021e933a..1ff2700b3b04 100644 --- a/airflow/providers/celery/__init__.py +++ b/airflow/providers/celery/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "3.2.1" +__version__ = "3.3.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/celery/executors/celery_kubernetes_executor.py b/airflow/providers/celery/executors/celery_kubernetes_executor.py index d79b3cd1db67..c5bbaac08139 100644 --- a/airflow/providers/celery/executors/celery_kubernetes_executor.py +++ b/airflow/providers/celery/executors/celery_kubernetes_executor.py @@ -23,8 +23,15 @@ from airflow.callbacks.base_callback_sink import BaseCallbackSink from airflow.callbacks.callback_requests import CallbackRequest from airflow.configuration import conf -from airflow.executors.kubernetes_executor import KubernetesExecutor from airflow.providers.celery.executors.celery_executor import CeleryExecutor + +try: + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubernetesExecutor +except ImportError as e: + from airflow.exceptions import AirflowOptionalProviderFeatureException + + raise AirflowOptionalProviderFeatureException(e) + from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.providers_configuration_loader import providers_configuration_loaded diff --git a/airflow/providers/celery/provider.yaml b/airflow/providers/celery/provider.yaml index 26787d4694ff..0eccce68904f 100644 --- a/airflow/providers/celery/provider.yaml +++ b/airflow/providers/celery/provider.yaml @@ -308,8 +308,13 @@ config: 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 + version_added: ~ type: string sensitive: true example: '{"password": "password_for_redis_server"}' default: ~ + +additional-extras: + - name: cncf.kubernetes + dependencies: + - apache-airflow-providers-cncf-kubernetes>=7.4.0 diff --git a/airflow/providers/cncf/kubernetes/CHANGELOG.rst b/airflow/providers/cncf/kubernetes/CHANGELOG.rst index a356a6c6dcfe..92ab38bdd742 100644 --- a/airflow/providers/cncf/kubernetes/CHANGELOG.rst +++ b/airflow/providers/cncf/kubernetes/CHANGELOG.rst @@ -27,6 +27,15 @@ Changelog --------- +7.4.0 +..... + +.. note:: + This provider release is the first release that has Kubernetes Executor and + Local Kubernetes Executor moved from the core ``apache-airflow`` package to the ``cncf.kubernetes`` + provider package. + + 7.3.0 ..... @@ -310,10 +319,10 @@ Param ``node_selectors`` has been removed in ``KubernetesPodOperator``; use ``no The following backcompat modules for KubernetesPodOperator are removed and you must now use the corresponding objects from the kubernetes library: -* ``airflow.providers.cncf.kubernetes.backcompat.pod`` -* ``airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env`` -* ``airflow.providers.cncf.kubernetes.backcompat.volume`` -* ``airflow.providers.cncf.kubernetes.backcompat.volume_mount`` +* ``airflow.kubernetes.backcompat.pod`` +* ``airflow.kubernetes.backcompat.pod_runtime_info_env`` +* ``airflow.kubernetes.backcompat.volume`` +* ``airflow.kubernetes.backcompat.volume_mount`` * ``Remove deprecated backcompat objects for KPO (#27518)`` * ``Remove support for node_selectors param in KPO (#27515)`` diff --git a/tests/kubernetes/__init__.py b/airflow/providers/cncf/kubernetes/executors/__init__.py similarity index 100% rename from tests/kubernetes/__init__.py rename to airflow/providers/cncf/kubernetes/executors/__init__.py diff --git a/airflow/executors/kubernetes_executor.py b/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py similarity index 93% rename from airflow/executors/kubernetes_executor.py rename to airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py index d525417a2715..e57ca205fe3b 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py @@ -35,12 +35,12 @@ from sqlalchemy.orm import Session +from airflow import AirflowException from airflow.configuration import conf -from airflow.exceptions import PodMutationHookException, PodReconciliationError from airflow.executors.base_executor import BaseExecutor -from airflow.executors.kubernetes_executor_types import POD_EXECUTOR_DONE_KEY -from airflow.kubernetes.kube_config import KubeConfig -from airflow.kubernetes.kubernetes_helper_functions import annotations_to_key +from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import POD_EXECUTOR_DONE_KEY +from airflow.providers.cncf.kubernetes.kube_config import KubeConfig +from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import annotations_to_key from airflow.utils.event_scheduler import EventScheduler from airflow.utils.log.logging_mixin import remove_escape_codes from airflow.utils.session import NEW_SESSION, provide_session @@ -51,13 +51,23 @@ from kubernetes.client import models as k8s from airflow.executors.base_executor import CommandType - from airflow.executors.kubernetes_executor_types import ( + from airflow.models.taskinstance import TaskInstance + from airflow.models.taskinstancekey import TaskInstanceKey + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import ( KubernetesJobType, KubernetesResultsType, ) - from airflow.executors.kubernetes_executor_utils import AirflowKubernetesScheduler - from airflow.models.taskinstance import TaskInstance - from airflow.models.taskinstancekey import TaskInstanceKey + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_utils import ( + AirflowKubernetesScheduler, + ) + + +class PodMutationHookException(AirflowException): + """Raised when exception happens during Pod Mutation Hook execution.""" + + +class PodReconciliationError(AirflowException): + """Raised when an error is encountered while trying to merge pod configs.""" class KubernetesExecutor(BaseExecutor): @@ -98,11 +108,11 @@ def _list_pods(self, query_kwargs): def _make_safe_label_value(self, input_value: str | datetime) -> str: """ Normalize a provided label to be of valid length and characters. - See airflow.kubernetes.pod_generator.make_safe_label_value for more details. + See airflow.providers.cncf.kubernetes.pod_generator.make_safe_label_value for more details. """ - # airflow.kubernetes is an expensive import, locally import it here to + # airflow.providers.cncf.kubernetes is an expensive import, locally import it here to # speed up load times of the kubernetes_executor module. - from airflow.kubernetes import pod_generator + from airflow.providers.cncf.kubernetes import pod_generator if isinstance(input_value, datetime): return pod_generator.datetime_to_label_safe_datestring(input_value) @@ -188,8 +198,10 @@ def start(self) -> None: self.log.info("Start Kubernetes executor") self.scheduler_job_id = str(self.job_id) self.log.debug("Start with scheduler_job_id: %s", self.scheduler_job_id) - from airflow.executors.kubernetes_executor_utils import AirflowKubernetesScheduler - from airflow.kubernetes.kube_client import get_kube_client + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_utils import ( + AirflowKubernetesScheduler, + ) + from airflow.providers.cncf.kubernetes.kube_client import get_kube_client self.kube_client = get_kube_client() self.kube_scheduler = AirflowKubernetesScheduler( @@ -224,7 +236,7 @@ def execute_async( else: self.log.info("Add task %s with command %s", key, command) - from airflow.kubernetes.pod_generator import PodGenerator + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator try: kube_executor_config = PodGenerator.from_obj(executor_config) @@ -282,7 +294,7 @@ def sync(self) -> None: except Empty: break - from airflow.executors.kubernetes_executor_utils import ResourceVersion + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_utils import ResourceVersion resource_instance = ResourceVersion() for ns in resource_instance.resource_version.keys(): @@ -386,8 +398,8 @@ def get_task_log(self, ti: TaskInstance, try_number: int) -> tuple[list[str], li log = [] try: - from airflow.kubernetes.kube_client import get_kube_client - from airflow.kubernetes.pod_generator import PodGenerator + from airflow.providers.cncf.kubernetes.kube_client import get_kube_client + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator client = get_kube_client() @@ -463,7 +475,7 @@ def cleanup_stuck_queued_tasks(self, tis: list[TaskInstance]) -> list[str]: :param tis: List of Task Instances to clean up :return: List of readable task instances for a warning message """ - from airflow.kubernetes.pod_generator import PodGenerator + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator if TYPE_CHECKING: assert self.kube_client diff --git a/airflow/executors/kubernetes_executor_types.py b/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_types.py similarity index 100% rename from airflow/executors/kubernetes_executor_types.py rename to airflow/providers/cncf/kubernetes/executors/kubernetes_executor_types.py diff --git a/airflow/executors/kubernetes_executor_utils.py b/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py similarity index 93% rename from airflow/executors/kubernetes_executor_utils.py rename to airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py index b1ee49cb695e..c1ee9d1ebe31 100644 --- a/airflow/executors/kubernetes_executor_utils.py +++ b/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py @@ -20,7 +20,7 @@ import multiprocessing import time from queue import Empty, Queue -from typing import TYPE_CHECKING, Any +from typing import TYPE_CHECKING, Any, Generic, TypeVar from kubernetes import client, watch from kubernetes.client import Configuration, models as k8s @@ -28,26 +28,50 @@ from urllib3.exceptions import ReadTimeoutError from airflow.exceptions import AirflowException -from airflow.kubernetes.kube_client import get_kube_client -from airflow.kubernetes.kubernetes_helper_functions import ( +from airflow.providers.cncf.kubernetes.kube_client import get_kube_client +from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import ( annotations_for_logging_task_metadata, annotations_to_key, create_pod_id, ) -from airflow.kubernetes.pod_generator import PodGenerator +from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator from airflow.utils.log.logging_mixin import LoggingMixin -from airflow.utils.singleton import Singleton from airflow.utils.state import State +try: + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import ( + ALL_NAMESPACES, + POD_EXECUTOR_DONE_KEY, + ) +except ImportError: + # avoid failing import when Airflow pre 2.7 is installed + from airflow.kubernetes.kubernetes_executor_types import ( # type: ignore[no-redef] + ALL_NAMESPACES, + POD_EXECUTOR_DONE_KEY, + ) + if TYPE_CHECKING: - from airflow.executors.kubernetes_executor_types import ( + from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import ( KubernetesJobType, KubernetesResultsType, KubernetesWatchType, ) +# Singleton here is duplicated version of airflow.utils.singleton.Singleton until +# min-airflow version is 2.7.0 for the provider. then it can be imported from airflow.utils.singleton. + +T = TypeVar("T") + + +class Singleton(type, Generic[T]): + """Metaclass that allows to implement singleton pattern.""" + + _instances: dict[Singleton[T], T] = {} -from airflow.executors.kubernetes_executor_types import ALL_NAMESPACES, POD_EXECUTOR_DONE_KEY + def __call__(cls: Singleton[T], *args, **kwargs) -> T: + if cls not in cls._instances: + cls._instances[cls] = super().__call__(*args, **kwargs) + return cls._instances[cls] class ResourceVersion(metaclass=Singleton): diff --git a/airflow/executors/local_kubernetes_executor.py b/airflow/providers/cncf/kubernetes/executors/local_kubernetes_executor.py similarity index 99% rename from airflow/executors/local_kubernetes_executor.py rename to airflow/providers/cncf/kubernetes/executors/local_kubernetes_executor.py index cc95232c9151..ee8225209680 100644 --- a/airflow/executors/local_kubernetes_executor.py +++ b/airflow/providers/cncf/kubernetes/executors/local_kubernetes_executor.py @@ -22,8 +22,8 @@ from airflow.callbacks.base_callback_sink import BaseCallbackSink from airflow.callbacks.callback_requests import CallbackRequest from airflow.configuration import conf -from airflow.executors.kubernetes_executor import KubernetesExecutor from airflow.executors.local_executor import LocalExecutor +from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubernetesExecutor from airflow.utils.log.logging_mixin import LoggingMixin if TYPE_CHECKING: diff --git a/airflow/providers/cncf/kubernetes/hooks/kubernetes.py b/airflow/providers/cncf/kubernetes/hooks/kubernetes.py index f237461155df..56852fb1a200 100644 --- a/airflow/providers/cncf/kubernetes/hooks/kubernetes.py +++ b/airflow/providers/cncf/kubernetes/hooks/kubernetes.py @@ -30,8 +30,8 @@ from airflow.exceptions import AirflowException, AirflowNotFoundException from airflow.hooks.base import BaseHook -from airflow.kubernetes.kube_client import _disable_verify_ssl, _enable_tcp_keepalive from airflow.models import Connection +from airflow.providers.cncf.kubernetes.kube_client import _disable_verify_ssl, _enable_tcp_keepalive from airflow.providers.cncf.kubernetes.utils.pod_manager import PodOperatorHookProtocol from airflow.utils import yaml diff --git a/airflow/providers/cncf/kubernetes/k8s_model.py b/airflow/providers/cncf/kubernetes/k8s_model.py new file mode 100644 index 000000000000..8280a3265f09 --- /dev/null +++ b/airflow/providers/cncf/kubernetes/k8s_model.py @@ -0,0 +1,59 @@ +# 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. +"""Classes for interacting with Kubernetes API.""" +from __future__ import annotations + +from abc import ABC, abstractmethod +from functools import reduce + +from kubernetes.client import models as k8s + + +class K8SModel(ABC): + """ + Airflow Kubernetes models are here for backwards compatibility reasons only. + + Ideally clients should use the kubernetes API + and the process of + + client input -> Airflow k8s models -> k8s models + + can be avoided. All of these models implement the + `attach_to_pod` method so that they integrate with the kubernetes client. + """ + + @abstractmethod + def attach_to_pod(self, pod: k8s.V1Pod) -> k8s.V1Pod: + """ + Attaches to pod. + + :param pod: A pod to attach this Kubernetes object to + :return: The pod with the object attached + """ + + +def append_to_pod(pod: k8s.V1Pod, k8s_objects: list[K8SModel] | None): + """ + Attach additional specs to an existing pod object. + + :param pod: A pod to attach a list of Kubernetes objects to + :param k8s_objects: a potential None list of K8SModels + :return: pod with the objects attached if they exist + """ + if not k8s_objects: + return pod + return reduce(lambda p, o: o.attach_to_pod(p), k8s_objects, pod) diff --git a/airflow/providers/cncf/kubernetes/kube_client.py b/airflow/providers/cncf/kubernetes/kube_client.py new file mode 100644 index 000000000000..b9dec69402ef --- /dev/null +++ b/airflow/providers/cncf/kubernetes/kube_client.py @@ -0,0 +1,146 @@ +# 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. +"""Client for kubernetes communication.""" +from __future__ import annotations + +import logging + +import urllib3.util + +from airflow.configuration import conf + +log = logging.getLogger(__name__) + +try: + from kubernetes import client, config + from kubernetes.client import Configuration + from kubernetes.client.rest import ApiException + + has_kubernetes = True + + def _get_default_configuration() -> Configuration: + if hasattr(Configuration, "get_default_copy"): + return Configuration.get_default_copy() + return Configuration() + + def _disable_verify_ssl() -> None: + configuration = _get_default_configuration() + configuration.verify_ssl = False + Configuration.set_default(configuration) + +except ImportError as e: + # We need an exception class to be able to use it in ``except`` elsewhere + # in the code base + ApiException = BaseException + has_kubernetes = False + _import_err = e + + +def _enable_tcp_keepalive() -> None: + """ + This function enables TCP keepalive mechanism. + + This prevents urllib3 connection to hang indefinitely when idle connection + is time-outed on services like cloud load balancers or firewalls. + + See https://github.com/apache/airflow/pull/11406 for detailed explanation. + + Please ping @michalmisiewicz or @dimberman in the PR if you want to modify this function. + """ + import socket + + from urllib3.connection import HTTPConnection, HTTPSConnection + + tcp_keep_idle = conf.getint("kubernetes_executor", "tcp_keep_idle") + tcp_keep_intvl = conf.getint("kubernetes_executor", "tcp_keep_intvl") + tcp_keep_cnt = conf.getint("kubernetes_executor", "tcp_keep_cnt") + + socket_options = [(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1)] + + if hasattr(socket, "TCP_KEEPIDLE"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, tcp_keep_idle)) + else: + log.debug("Unable to set TCP_KEEPIDLE on this platform") + + if hasattr(socket, "TCP_KEEPINTVL"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, tcp_keep_intvl)) + else: + log.debug("Unable to set TCP_KEEPINTVL on this platform") + + if hasattr(socket, "TCP_KEEPCNT"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPCNT, tcp_keep_cnt)) + else: + log.debug("Unable to set TCP_KEEPCNT on this platform") + + HTTPSConnection.default_socket_options = HTTPSConnection.default_socket_options + socket_options + HTTPConnection.default_socket_options = HTTPConnection.default_socket_options + socket_options + + +def get_kube_client( + in_cluster: bool | None = None, + cluster_context: str | None = None, + config_file: str | None = None, +) -> client.CoreV1Api: + """ + Retrieves Kubernetes client. + + :param in_cluster: whether we are in cluster + :param cluster_context: context of the cluster + :param config_file: configuration file + :return: kubernetes client + """ + if in_cluster is None: + in_cluster = conf.getboolean("kubernetes_executor", "in_cluster") + if not has_kubernetes: + raise _import_err + + if conf.getboolean("kubernetes_executor", "enable_tcp_keepalive"): + _enable_tcp_keepalive() + + configuration = _get_default_configuration() + api_client_retry_configuration = conf.getjson( + "kubernetes_executor", "api_client_retry_configuration", fallback={} + ) + + if not conf.getboolean("kubernetes_executor", "verify_ssl"): + _disable_verify_ssl() + + if isinstance(api_client_retry_configuration, dict): + configuration.retries = urllib3.util.Retry(**api_client_retry_configuration) + else: + raise ValueError("api_client_retry_configuration should be a dictionary") + + if in_cluster: + config.load_incluster_config(client_configuration=configuration) + else: + if cluster_context is None: + cluster_context = conf.get("kubernetes_executor", "cluster_context", fallback=None) + if config_file is None: + config_file = conf.get("kubernetes_executor", "config_file", fallback=None) + config.load_kube_config( + config_file=config_file, context=cluster_context, client_configuration=configuration + ) + + if not conf.getboolean("kubernetes_executor", "verify_ssl"): + configuration.verify_ssl = False + + ssl_ca_cert = conf.get("kubernetes_executor", "ssl_ca_cert") + if ssl_ca_cert: + configuration.ssl_ca_cert = ssl_ca_cert + + api_client = client.ApiClient(configuration=configuration) + return client.CoreV1Api(api_client) diff --git a/airflow/kubernetes/kube_config.py b/airflow/providers/cncf/kubernetes/kube_config.py similarity index 99% rename from airflow/kubernetes/kube_config.py rename to airflow/providers/cncf/kubernetes/kube_config.py index 20bddf82f38a..8db861a7f5d5 100644 --- a/airflow/kubernetes/kube_config.py +++ b/airflow/providers/cncf/kubernetes/kube_config.py @@ -43,7 +43,6 @@ def __init__(self): self.worker_pods_creation_batch_size = conf.getint( self.kubernetes_section, "worker_pods_creation_batch_size" ) - self.worker_container_repository = conf.get(self.kubernetes_section, "worker_container_repository") self.worker_container_tag = conf.get(self.kubernetes_section, "worker_container_tag") if self.worker_container_repository and self.worker_container_tag: diff --git a/tests/executors/kubernetes_executor_template_files/__init__.py b/airflow/providers/cncf/kubernetes/kubernetes_executor_templates/__init__.py similarity index 100% rename from tests/executors/kubernetes_executor_template_files/__init__.py rename to airflow/providers/cncf/kubernetes/kubernetes_executor_templates/__init__.py diff --git a/airflow/kubernetes_executor_templates/basic_template.yaml b/airflow/providers/cncf/kubernetes/kubernetes_executor_templates/basic_template.yaml similarity index 100% rename from airflow/kubernetes_executor_templates/basic_template.yaml rename to airflow/providers/cncf/kubernetes/kubernetes_executor_templates/basic_template.yaml diff --git a/airflow/kubernetes/kubernetes_helper_functions.py b/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py similarity index 94% rename from airflow/kubernetes/kubernetes_helper_functions.py rename to airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py index 27762421ca51..b54519fe2733 100644 --- a/airflow/kubernetes/kubernetes_helper_functions.py +++ b/airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py @@ -43,8 +43,14 @@ def rand_str(num): return "".join(secrets.choice(alphanum_lower) for _ in range(num)) -def add_pod_suffix(*, pod_name, rand_len=8, max_len=80): - """Add random string to pod name while staying under max len.""" +def add_pod_suffix(*, pod_name: str, rand_len: int = 8, max_len: int = 80) -> str: + """Add random string to pod name while staying under max length. + + :param pod_name: name of the pod + :param rand_len: length of the random string to append + :param max_len: maximum length of the pod name + :meta private: + """ suffix = "-" + rand_str(rand_len) return pod_name[: max_len - len(suffix)].strip("-.") + suffix diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index 49940144b5dd..6e6acbec4e38 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -35,10 +35,8 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException -from airflow.kubernetes import pod_generator -from airflow.kubernetes.pod_generator import PodGenerator -from airflow.kubernetes.secret import Secret from airflow.models import BaseOperator +from airflow.providers.cncf.kubernetes import pod_generator from airflow.providers.cncf.kubernetes.backcompat.backwards_compat_converters import ( convert_affinity, convert_configmap, @@ -51,6 +49,8 @@ convert_volume_mount, ) from airflow.providers.cncf.kubernetes.hooks.kubernetes import KubernetesHook +from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator +from airflow.providers.cncf.kubernetes.secret import Secret from airflow.providers.cncf.kubernetes.triggers.pod import KubernetesPodTrigger from airflow.providers.cncf.kubernetes.utils import xcom_sidecar # type: ignore[attr-defined] from airflow.providers.cncf.kubernetes.utils.pod_manager import ( diff --git a/airflow/kubernetes/pod_generator.py b/airflow/providers/cncf/kubernetes/pod_generator.py similarity index 97% rename from airflow/kubernetes/pod_generator.py rename to airflow/providers/cncf/kubernetes/pod_generator.py index 844a1abfd8a8..c0f2e7e89f94 100644 --- a/airflow/kubernetes/pod_generator.py +++ b/airflow/providers/cncf/kubernetes/pod_generator.py @@ -38,14 +38,21 @@ from airflow.exceptions import ( AirflowConfigException, + RemovedInAirflow3Warning, +) +from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import ( PodMutationHookException, PodReconciliationError, - RemovedInAirflow3Warning, ) -from airflow.kubernetes.kubernetes_helper_functions import add_pod_suffix, rand_str -from airflow.kubernetes.pod_generator_deprecated import PodDefaults, PodGenerator as PodGeneratorDeprecated +from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import add_pod_suffix, rand_str +from airflow.providers.cncf.kubernetes.pod_generator_deprecated import ( + PodDefaults, + PodGenerator as PodGeneratorDeprecated, +) + +# replace it with airflow.utils.hashlib_wrapper.md5 when min airflow version for k8s provider is 2.6.0 +from airflow.providers.cncf.kubernetes.utils.k8s_hashlib_wrapper import md5 from airflow.utils import yaml -from airflow.utils.hashlib_wrapper import md5 from airflow.version import version as airflow_version log = logging.getLogger(__name__) @@ -626,7 +633,10 @@ def extend_object_field(base_obj, client_obj, field_name): if (not isinstance(base_obj_field, list) and base_obj_field is not None) or ( not isinstance(client_obj_field, list) and client_obj_field is not None ): - raise ValueError("The chosen field must be a list.") + raise ValueError( + f"The chosen field must be a list. Got {type(base_obj_field)} base_object_field " + f"and {type(client_obj_field)} client_object_field." + ) if not base_obj_field: return client_obj_cp diff --git a/airflow/providers/cncf/kubernetes/pod_generator_deprecated.py b/airflow/providers/cncf/kubernetes/pod_generator_deprecated.py new file mode 100644 index 000000000000..8d64e96d6c40 --- /dev/null +++ b/airflow/providers/cncf/kubernetes/pod_generator_deprecated.py @@ -0,0 +1,309 @@ +# 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. +""" +Backwards compatibility for Pod generation. + +This module provides an interface between the previous Pod +API and outputs a kubernetes.client.models.V1Pod. +The advantage being that the full Kubernetes API +is supported and no serialization need be written. +""" +from __future__ import annotations + +import copy +import uuid + +import re2 +from kubernetes.client import models as k8s + +# replace it with airflow.utils.hashlib_wrapper.md5 when min airflow version for k8s provider is 2.6.0 +from airflow.providers.cncf.kubernetes.utils.k8s_hashlib_wrapper import md5 + +MAX_POD_ID_LEN = 253 + +MAX_LABEL_LEN = 63 + + +class PodDefaults: + """Static defaults for Pods.""" + + XCOM_MOUNT_PATH = "/airflow/xcom" + SIDECAR_CONTAINER_NAME = "airflow-xcom-sidecar" + XCOM_CMD = 'trap "exit 0" INT; while true; do sleep 30; done;' + VOLUME_MOUNT = k8s.V1VolumeMount(name="xcom", mount_path=XCOM_MOUNT_PATH) + VOLUME = k8s.V1Volume(name="xcom", empty_dir=k8s.V1EmptyDirVolumeSource()) + SIDECAR_CONTAINER = k8s.V1Container( + name=SIDECAR_CONTAINER_NAME, + command=["sh", "-c", XCOM_CMD], + image="alpine", + volume_mounts=[VOLUME_MOUNT], + resources=k8s.V1ResourceRequirements( + requests={ + "cpu": "1m", + } + ), + ) + + +def make_safe_label_value(string): + """ + Normalize a provided label to be of valid length and characters. + + Valid label values must be 63 characters or less and must be empty or begin and + end with an alphanumeric character ([a-z0-9A-Z]) with dashes (-), underscores (_), + dots (.), and alphanumerics between. + + If the label value is greater than 63 chars once made safe, or differs in any + way from the original value sent to this function, then we need to truncate to + 53 chars, and append it with a unique hash. + """ + safe_label = re2.sub(r"^[^a-z0-9A-Z]*|[^a-zA-Z0-9_\-\.]|[^a-z0-9A-Z]*$", "", string) + + if len(safe_label) > MAX_LABEL_LEN or string != safe_label: + safe_hash = md5(string.encode()).hexdigest()[:9] + safe_label = safe_label[: MAX_LABEL_LEN - len(safe_hash) - 1] + "-" + safe_hash + + return safe_label + + +class PodGenerator: + """ + Contains Kubernetes Airflow Worker configuration logic. + + Represents a kubernetes pod and manages execution of a single pod. + Any configuration that is container specific gets applied to + the first container in the list of containers. + + :param image: The docker image + :param name: name in the metadata section (not the container name) + :param namespace: pod namespace + :param volume_mounts: list of kubernetes volumes mounts + :param envs: A dict containing the environment variables + :param cmds: The command to be run on the first container + :param args: The arguments to be run on the pod + :param labels: labels for the pod metadata + :param node_selectors: node selectors for the pod + :param ports: list of ports. Applies to the first container. + :param volumes: Volumes to be attached to the first container + :param image_pull_policy: Specify a policy to cache or always pull an image + :param restart_policy: The restart policy of the pod + :param image_pull_secrets: Any image pull secrets to be given to the pod. + If more than one secret is required, provide a comma separated list: + secret_a,secret_b + :param init_containers: A list of init containers + :param service_account_name: Identity for processes that run in a Pod + :param resources: Resource requirements for the first containers + :param annotations: annotations for the pod + :param affinity: A dict containing a group of affinity scheduling rules + :param hostnetwork: If True enable host networking on the pod + :param tolerations: A list of kubernetes tolerations + :param security_context: A dict containing the security context for the pod + :param configmaps: Any configmap refs to read ``configmaps`` for environments from. + If more than one configmap is required, provide a comma separated list + configmap_a,configmap_b + :param dnspolicy: Specify a dnspolicy for the pod + :param schedulername: Specify a schedulername for the pod + :param pod: The fully specified pod. Mutually exclusive with `path_or_string` + :param extract_xcom: Whether to bring up a container for xcom + :param priority_class_name: priority class name for the launched Pod + """ + + def __init__( + self, + image: str | None = None, + name: str | None = None, + namespace: str | None = None, + volume_mounts: list[k8s.V1VolumeMount | dict] | None = None, + envs: dict[str, str] | None = None, + cmds: list[str] | None = None, + args: list[str] | None = None, + labels: dict[str, str] | None = None, + node_selectors: dict[str, str] | None = None, + ports: list[k8s.V1ContainerPort | dict] | None = None, + volumes: list[k8s.V1Volume | dict] | None = None, + image_pull_policy: str | None = None, + restart_policy: str | None = None, + image_pull_secrets: str | None = None, + init_containers: list[k8s.V1Container] | None = None, + service_account_name: str | None = None, + resources: k8s.V1ResourceRequirements | dict | None = None, + annotations: dict[str, str] | None = None, + affinity: dict | None = None, + hostnetwork: bool = False, + tolerations: list | None = None, + security_context: k8s.V1PodSecurityContext | dict | None = None, + configmaps: list[str] | None = None, + dnspolicy: str | None = None, + schedulername: str | None = None, + extract_xcom: bool = False, + priority_class_name: str | None = None, + ): + self.pod = k8s.V1Pod() + self.pod.api_version = "v1" + self.pod.kind = "Pod" + + # Pod Metadata + self.metadata = k8s.V1ObjectMeta() + self.metadata.labels = labels + self.metadata.name = name + self.metadata.namespace = namespace + self.metadata.annotations = annotations + + # Pod Container + self.container = k8s.V1Container(name="base") + self.container.image = image + self.container.env = [] + + if envs: + if isinstance(envs, dict): + for key, val in envs.items(): + self.container.env.append(k8s.V1EnvVar(name=key, value=val)) + elif isinstance(envs, list): + self.container.env.extend(envs) + + configmaps = configmaps or [] + self.container.env_from = [] + for configmap in configmaps: + self.container.env_from.append( + k8s.V1EnvFromSource(config_map_ref=k8s.V1ConfigMapEnvSource(name=configmap)) + ) + + self.container.command = cmds or [] + self.container.args = args or [] + if image_pull_policy: + self.container.image_pull_policy = image_pull_policy + self.container.ports = ports or [] + self.container.resources = resources + self.container.volume_mounts = volume_mounts or [] + + # Pod Spec + self.spec = k8s.V1PodSpec(containers=[]) + self.spec.security_context = security_context + self.spec.tolerations = tolerations + if dnspolicy: + self.spec.dns_policy = dnspolicy + self.spec.scheduler_name = schedulername + self.spec.host_network = hostnetwork + self.spec.affinity = affinity + self.spec.service_account_name = service_account_name + self.spec.init_containers = init_containers + self.spec.volumes = volumes or [] + self.spec.node_selector = node_selectors + if restart_policy: + self.spec.restart_policy = restart_policy + self.spec.priority_class_name = priority_class_name + + self.spec.image_pull_secrets = [] + + if image_pull_secrets: + for image_pull_secret in image_pull_secrets.split(","): + self.spec.image_pull_secrets.append(k8s.V1LocalObjectReference(name=image_pull_secret)) + + # Attach sidecar + self.extract_xcom = extract_xcom + + def gen_pod(self) -> k8s.V1Pod: + """Generates pod.""" + result = None + + if result is None: + result = self.pod + result.spec = self.spec + result.metadata = self.metadata + result.spec.containers = [self.container] + + result.metadata.name = self.make_unique_pod_id(result.metadata.name) + + if self.extract_xcom: + result = self.add_sidecar(result) + + return result + + @staticmethod + def add_sidecar(pod: k8s.V1Pod) -> k8s.V1Pod: + """Adds sidecar.""" + pod_cp = copy.deepcopy(pod) + pod_cp.spec.volumes = pod.spec.volumes or [] + pod_cp.spec.volumes.insert(0, PodDefaults.VOLUME) + pod_cp.spec.containers[0].volume_mounts = pod_cp.spec.containers[0].volume_mounts or [] + pod_cp.spec.containers[0].volume_mounts.insert(0, PodDefaults.VOLUME_MOUNT) + pod_cp.spec.containers.append(PodDefaults.SIDECAR_CONTAINER) + + return pod_cp + + @staticmethod + def from_obj(obj) -> k8s.V1Pod | None: + """Converts to pod from obj.""" + if obj is None: + return None + + if isinstance(obj, PodGenerator): + return obj.gen_pod() + + if not isinstance(obj, dict): + raise TypeError( + "Cannot convert a non-dictionary or non-PodGenerator " + "object into a KubernetesExecutorConfig" + ) + + # We do not want to extract constant here from ExecutorLoader because it is just + # A name in dictionary rather than executor selection mechanism and it causes cyclic import + namespaced = obj.get("KubernetesExecutor", {}) + + if not namespaced: + return None + + resources = namespaced.get("resources") + + if resources is None: + requests = { + "cpu": namespaced.get("request_cpu"), + "memory": namespaced.get("request_memory"), + "ephemeral-storage": namespaced.get("ephemeral-storage"), + } + limits = { + "cpu": namespaced.get("limit_cpu"), + "memory": namespaced.get("limit_memory"), + "ephemeral-storage": namespaced.get("ephemeral-storage"), + } + all_resources = list(requests.values()) + list(limits.values()) + if all(r is None for r in all_resources): + resources = None + else: + resources = k8s.V1ResourceRequirements(requests=requests, limits=limits) + namespaced["resources"] = resources + return PodGenerator(**namespaced).gen_pod() + + @staticmethod + def make_unique_pod_id(dag_id): + r""" + Generate a unique Pod name. + + Kubernetes pod names must be <= 253 chars and must pass the following regex for + validation + ``^[a-z0-9]([-a-z0-9]*[a-z0-9])?(\\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*$`` + + :param dag_id: a dag_id with only alphanumeric characters + :return: ``str`` valid Pod name of appropriate length + """ + if not dag_id: + return None + + safe_uuid = uuid.uuid4().hex + safe_pod_id = dag_id[: MAX_POD_ID_LEN - len(safe_uuid) - 1] + "-" + safe_uuid + + return safe_pod_id diff --git a/airflow/kubernetes/pod_launcher_deprecated.py b/airflow/providers/cncf/kubernetes/pod_launcher_deprecated.py similarity index 98% rename from airflow/kubernetes/pod_launcher_deprecated.py rename to airflow/providers/cncf/kubernetes/pod_launcher_deprecated.py index fa5c884a5176..9fc964cdb6e0 100644 --- a/airflow/kubernetes/pod_launcher_deprecated.py +++ b/airflow/providers/cncf/kubernetes/pod_launcher_deprecated.py @@ -32,8 +32,8 @@ from requests.exceptions import HTTPError from airflow.exceptions import AirflowException, RemovedInAirflow3Warning -from airflow.kubernetes.kube_client import get_kube_client -from airflow.kubernetes.pod_generator import PodDefaults +from airflow.providers.cncf.kubernetes.kube_client import get_kube_client +from airflow.providers.cncf.kubernetes.pod_generator import PodDefaults from airflow.settings import pod_mutation_hook from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import State diff --git a/airflow/kubernetes/pod_runtime_info_env.py b/airflow/providers/cncf/kubernetes/pod_template_file_examples/__init__.py similarity index 58% rename from airflow/kubernetes/pod_runtime_info_env.py rename to airflow/providers/cncf/kubernetes/pod_template_file_examples/__init__.py index 32e178263b12..13a83393a912 100644 --- a/airflow/kubernetes/pod_runtime_info_env.py +++ b/airflow/providers/cncf/kubernetes/pod_template_file_examples/__init__.py @@ -1,4 +1,3 @@ -# # 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 @@ -15,19 +14,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""This module is deprecated. Please use :mod:`kubernetes.client.models.V1EnvVar`.""" -from __future__ import annotations - -import warnings - -from airflow.exceptions import RemovedInAirflow3Warning - -with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv # noqa - -warnings.warn( - "This module is deprecated. Please use `kubernetes.client.models.V1EnvVar`.", - RemovedInAirflow3Warning, - stacklevel=2, -) diff --git a/airflow/kubernetes/pod_template_file_examples/dags_in_image_template.yaml b/airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_image_template.yaml similarity index 100% rename from airflow/kubernetes/pod_template_file_examples/dags_in_image_template.yaml rename to airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_image_template.yaml diff --git a/airflow/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml b/airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml similarity index 100% rename from airflow/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml rename to airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml diff --git a/airflow/kubernetes/pod_template_file_examples/git_sync_template.yaml b/airflow/providers/cncf/kubernetes/pod_template_file_examples/git_sync_template.yaml similarity index 100% rename from airflow/kubernetes/pod_template_file_examples/git_sync_template.yaml rename to airflow/providers/cncf/kubernetes/pod_template_file_examples/git_sync_template.yaml diff --git a/airflow/providers/cncf/kubernetes/provider.yaml b/airflow/providers/cncf/kubernetes/provider.yaml index 86ab5f70160b..d9b2966bba30 100644 --- a/airflow/providers/cncf/kubernetes/provider.yaml +++ b/airflow/providers/cncf/kubernetes/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 7.4.0 - 7.3.0 - 7.2.0 - 7.1.0 @@ -124,3 +125,212 @@ connection-types: task-decorators: - class-name: airflow.providers.cncf.kubernetes.decorators.kubernetes.kubernetes_task name: kubernetes + +config: + local_kubernetes_executor: + description: | + This section only applies if you are using the ``LocalKubernetesExecutor`` in + ``[core]`` section above + options: + kubernetes_queue: + description: | + Define when to send a task to ``KubernetesExecutor`` when using ``LocalKubernetesExecutor``. + When the queue of a task is the value of ``kubernetes_queue`` (default ``kubernetes``), + the task is executed via ``KubernetesExecutor``, + otherwise via ``LocalExecutor`` + version_added: ~ + type: string + example: ~ + default: "kubernetes" + kubernetes_executor: + description: ~ + options: + api_client_retry_configuration: + description: | + Kwargs to override the default urllib3 Retry used in the kubernetes API client + version_added: ~ + type: string + example: '{ "total": 3, "backoff_factor": 0.5 }' + default: "" + logs_task_metadata: + description: | + Flag to control the information added to kubernetes executor logs for better traceability + version_added: ~ + type: boolean + example: ~ + default: "False" + pod_template_file: + description: | + Path to the YAML pod file that forms the basis for KubernetesExecutor workers. + version_added: ~ + type: string + example: ~ + default: "" + see_also: ":ref:`concepts:pod_template_file`" + worker_container_repository: + description: | + The repository of the Kubernetes Image for the Worker to Run + version_added: ~ + type: string + example: ~ + default: "" + worker_container_tag: + description: | + The tag of the Kubernetes Image for the Worker to Run + version_added: ~ + type: string + example: ~ + default: "" + namespace: + description: | + The Kubernetes namespace where airflow workers should be created. Defaults to ``default`` + version_added: ~ + type: string + example: ~ + default: "default" + delete_worker_pods: + description: | + If True, all worker pods will be deleted upon termination + version_added: ~ + type: string + example: ~ + default: "True" + delete_worker_pods_on_failure: + description: | + If False (and delete_worker_pods is True), + failed worker pods will not be deleted so users can investigate them. + This only prevents removal of worker pods where the worker itself failed, + not when the task it ran failed. + version_added: ~ + type: string + example: ~ + default: "False" + worker_pods_creation_batch_size: + description: | + Number of Kubernetes Worker Pod creation calls per scheduler loop. + Note that the current default of "1" will only launch a single pod + per-heartbeat. It is HIGHLY recommended that users increase this + number to match the tolerance of their kubernetes cluster for + better performance. + version_added: ~ + type: string + example: ~ + default: "1" + multi_namespace_mode: + description: | + Allows users to launch pods in multiple namespaces. + Will require creating a cluster-role for the scheduler, + or use multi_namespace_mode_namespace_list configuration. + version_added: ~ + type: boolean + example: ~ + default: "False" + multi_namespace_mode_namespace_list: + description: | + If multi_namespace_mode is True while scheduler does not have a cluster-role, + give the list of namespaces where the scheduler will schedule jobs + Scheduler needs to have the necessary permissions in these namespaces. + version_added: ~ + type: string + example: ~ + default: "" + in_cluster: + description: | + Use the service account kubernetes gives to pods to connect to kubernetes cluster. + It's intended for clients that expect to be running inside a pod running on kubernetes. + It will raise an exception if called from a process not running in a kubernetes environment. + version_added: ~ + type: string + example: ~ + default: "True" + cluster_context: + description: | + When running with in_cluster=False change the default cluster_context or config_file + options to Kubernetes client. Leave blank these to use default behaviour like ``kubectl`` has. + version_added: ~ + type: string + example: ~ + default: ~ + config_file: + description: | + Path to the kubernetes configfile to be used when ``in_cluster`` is set to False + version_added: ~ + type: string + example: ~ + default: ~ + kube_client_request_args: + description: | + Keyword parameters to pass while calling a kubernetes client core_v1_api methods + from Kubernetes Executor provided as a single line formatted JSON dictionary string. + List of supported params are similar for all core_v1_apis, hence a single config + variable for all apis. See: + https://raw.githubusercontent.com/kubernetes-client/python/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/api/core_v1_api.py + version_added: ~ + type: string + example: ~ + default: "" + delete_option_kwargs: + description: | + Optional keyword arguments to pass to the ``delete_namespaced_pod`` kubernetes client + ``core_v1_api`` method when using the Kubernetes Executor. + This should be an object and can contain any of the options listed in the ``v1DeleteOptions`` + class defined here: + https://github.com/kubernetes-client/python/blob/41f11a09995efcd0142e25946adc7591431bfb2f/kubernetes/client/models/v1_delete_options.py#L19 + version_added: ~ + type: string + example: '{"grace_period_seconds": 10}' + default: "" + enable_tcp_keepalive: + description: | + Enables TCP keepalive mechanism. This prevents Kubernetes API requests to hang indefinitely + when idle connection is time-outed on services like cloud load balancers or firewalls. + version_added: ~ + type: boolean + example: ~ + default: "True" + tcp_keep_idle: + description: | + When the `enable_tcp_keepalive` option is enabled, TCP probes a connection that has + been idle for `tcp_keep_idle` seconds. + version_added: ~ + type: integer + example: ~ + default: "120" + tcp_keep_intvl: + description: | + When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond + to a keepalive probe, TCP retransmits the probe after `tcp_keep_intvl` seconds. + version_added: ~ + type: integer + example: ~ + default: "30" + tcp_keep_cnt: + description: | + When the `enable_tcp_keepalive` option is enabled, if Kubernetes API does not respond + to a keepalive probe, TCP retransmits the probe `tcp_keep_cnt number` of times before + a connection is considered to be broken. + version_added: ~ + type: integer + example: ~ + default: "6" + verify_ssl: + description: | + Set this to false to skip verifying SSL certificate of Kubernetes python client. + version_added: ~ + type: boolean + example: ~ + default: "True" + worker_pods_queued_check_interval: + description: | + How often in seconds to check for task instances stuck in "queued" status without a pod + version_added: ~ + type: integer + example: ~ + default: "60" + ssl_ca_cert: + description: | + Path to a CA certificate to be used by the Kubernetes client to verify the server's SSL certificate. + version_added: ~ + type: string + example: ~ + default: "" diff --git a/airflow/kubernetes/secret.py b/airflow/providers/cncf/kubernetes/secret.py similarity index 98% rename from airflow/kubernetes/secret.py rename to airflow/providers/cncf/kubernetes/secret.py index 32ce92e2de01..d4fba36f16b5 100644 --- a/airflow/kubernetes/secret.py +++ b/airflow/providers/cncf/kubernetes/secret.py @@ -23,7 +23,7 @@ from kubernetes.client import models as k8s from airflow.exceptions import AirflowConfigException -from airflow.kubernetes.k8s_model import K8SModel +from airflow.providers.cncf.kubernetes.k8s_model import K8SModel class Secret(K8SModel): diff --git a/airflow/providers/cncf/kubernetes/template_rendering.py b/airflow/providers/cncf/kubernetes/template_rendering.py new file mode 100644 index 000000000000..5461284e4625 --- /dev/null +++ b/airflow/providers/cncf/kubernetes/template_rendering.py @@ -0,0 +1,67 @@ +# 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. + +from __future__ import annotations + +from jinja2 import TemplateAssertionError, UndefinedError +from kubernetes.client.api_client import ApiClient + +from airflow import AirflowException +from airflow.models.taskinstance import TaskInstance +from airflow.providers.cncf.kubernetes.kube_config import KubeConfig +from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import ( + create_pod_id, +) +from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator +from airflow.utils.session import NEW_SESSION, provide_session + + +def render_k8s_pod_yaml(task_instance: TaskInstance) -> dict | None: + """Render k8s pod yaml.""" + kube_config = KubeConfig() + pod = PodGenerator.construct_pod( + dag_id=task_instance.dag_id, + run_id=task_instance.run_id, + task_id=task_instance.task_id, + map_index=task_instance.map_index, + date=None, + pod_id=create_pod_id(task_instance.dag_id, task_instance.task_id), + try_number=task_instance.try_number, + kube_image=kube_config.kube_image, + args=task_instance.command_as_list(), + pod_override_object=PodGenerator.from_obj(task_instance.executor_config), + scheduler_job_id="0", + namespace=kube_config.executor_namespace, + base_worker_pod=PodGenerator.deserialize_model_file(kube_config.pod_template_file), + with_mutation_hook=True, + ) + sanitized_pod = ApiClient().sanitize_for_serialization(pod) + return sanitized_pod + + +@provide_session +def get_rendered_k8s_spec(task_instance: TaskInstance, session=NEW_SESSION) -> dict | None: + """Fetch rendered template fields from DB.""" + from airflow.models.renderedtifields import RenderedTaskInstanceFields + + rendered_k8s_spec = RenderedTaskInstanceFields.get_k8s_pod_yaml(task_instance, session=session) + if not rendered_k8s_spec: + try: + rendered_k8s_spec = render_k8s_pod_yaml(task_instance) + except (TemplateAssertionError, UndefinedError) as e: + raise AirflowException(f"Unable to render a k8s spec for this taskinstance: {e}") from e + return rendered_k8s_spec diff --git a/airflow/providers/cncf/kubernetes/utils/k8s_hashlib_wrapper.py b/airflow/providers/cncf/kubernetes/utils/k8s_hashlib_wrapper.py new file mode 100644 index 000000000000..72a1f8dac6b5 --- /dev/null +++ b/airflow/providers/cncf/kubernetes/utils/k8s_hashlib_wrapper.py @@ -0,0 +1,42 @@ +# 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. + +# this is a backcompat hashlib wrapper for kubernetes provider. It should be removed (with corresponding +# imports in the provider when min airflow version is 2.6.0 for the provider + +from __future__ import annotations + +import hashlib +from typing import TYPE_CHECKING + +if TYPE_CHECKING: + from _typeshed import ReadableBuffer + +from airflow import PY39 + + +def md5(__string: ReadableBuffer = b"") -> hashlib._Hash: + """ + Safely allows calling the ``hashlib.md5`` function when ``usedforsecurity`` is disabled in + the configuration. + + :param __string: The data to hash. Default to empty str byte. + :return: The hashed value. + """ + if PY39: + return hashlib.md5(__string, usedforsecurity=False) # type: ignore + return hashlib.md5(__string) diff --git a/airflow/providers/cncf/kubernetes/utils/pod_manager.py b/airflow/providers/cncf/kubernetes/utils/pod_manager.py index acd40d4b1755..c8ac74382d6d 100644 --- a/airflow/providers/cncf/kubernetes/utils/pod_manager.py +++ b/airflow/providers/cncf/kubernetes/utils/pod_manager.py @@ -43,7 +43,7 @@ from urllib3.response import HTTPResponse from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning -from airflow.kubernetes.pod_generator import PodDefaults +from airflow.providers.cncf.kubernetes.pod_generator import PodDefaults from airflow.typing_compat import Literal, Protocol from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.timezone import utcnow @@ -89,7 +89,7 @@ class PodOperatorHookProtocol(Protocol): @property def core_v1_client(self) -> client.CoreV1Api: - """Get authenticated CoreV1Api object.""" + """Get authenticated client object.""" @property def is_in_cluster(self) -> bool: diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index 60e2d4b02d7d..5efa3b3da574 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -73,7 +73,7 @@ try: from kubernetes.client import models as k8s - from airflow.kubernetes.pod_generator import PodGenerator + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator except ImportError: pass @@ -1482,7 +1482,12 @@ def _has_kubernetes() -> bool: try: from kubernetes.client import models as k8s - from airflow.kubernetes.pod_generator import PodGenerator + try: + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator + except ImportError: + from airflow.kubernetes.pre_7_4_0_compatibility.pod_generator import ( # type: ignore[assignment] + PodGenerator, + ) globals()["k8s"] = k8s globals()["PodGenerator"] = PodGenerator diff --git a/airflow/serialization/serializers/kubernetes.py b/airflow/serialization/serializers/kubernetes.py index d38836affc4a..f095400ee926 100644 --- a/airflow/serialization/serializers/kubernetes.py +++ b/airflow/serialization/serializers/kubernetes.py @@ -44,7 +44,12 @@ def serialize(o: object) -> tuple[U, str, int, bool]: return "", "", 0, False if isinstance(o, (k8s.V1Pod, k8s.V1ResourceRequirements)): - from airflow.kubernetes.pod_generator import PodGenerator + try: + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator + except ImportError: + from airflow.kubernetes.pre_7_4_0_compatibility.pod_generator import ( # type: ignore[assignment] + PodGenerator, + ) # We're running this in an except block, so we don't want it to fail # under any circumstances, e.g. accessing a non-existing attribute. diff --git a/airflow/timetables/_cron.py b/airflow/timetables/_cron.py index 49f5771966ad..89cae4bdcbed 100644 --- a/airflow/timetables/_cron.py +++ b/airflow/timetables/_cron.py @@ -65,10 +65,10 @@ def __init__(self, cron: str, timezone: str | Timezone) -> None: # as Croniter has inconsistent evaluation with other libraries if len(croniter(self._expression).expanded) > 5: raise FormatException() - interval_description = descriptor.get_description() + interval_description: str = descriptor.get_description() except (CroniterBadCronError, FormatException, MissingFieldException): interval_description = "" - self.description = interval_description + self.description: str = interval_description def __eq__(self, other: Any) -> bool: """Both expression and timezone should match. diff --git a/airflow/utils/deprecation_tools.py b/airflow/utils/deprecation_tools.py index a189fe5479ee..6be6650f093e 100644 --- a/airflow/utils/deprecation_tools.py +++ b/airflow/utils/deprecation_tools.py @@ -23,25 +23,75 @@ from types import ModuleType -def getattr_with_deprecation(imports: dict[str, str], module: str, name: str): +def getattr_with_deprecation( + imports: dict[str, str], + module: str, + override_deprecated_classes: dict[str, str], + extra_message: str, + name: str, +): + """ + Retrieve the imported attribute from the redirected module and raises a deprecation warning. + + :param imports: dict of imports and their redirection for the module + :param module: name of the module in the package to get the attribute from + :param override_deprecated_classes: override target classes with deprecated ones. If target class is + found in the dictionary, it will be displayed in the warning message. + :param extra_message: extra message to display in the warning or import error message + :param name: attribute name + :return: + """ target_class_full_name = imports.get(name) if not target_class_full_name: raise AttributeError(f"The module `{module!r}` has no attribute `{name!r}`") - warnings.warn( - f"The `{module}.{name}` class is deprecated. Please use `{target_class_full_name!r}`.", - DeprecationWarning, - stacklevel=2, - ) + warning_class_name = target_class_full_name + if override_deprecated_classes and name in override_deprecated_classes: + warning_class_name = override_deprecated_classes[name] + message = f"The `{module}.{name}` class is deprecated. Please use `{warning_class_name!r}`." + if extra_message: + message += f" {extra_message}." + warnings.warn(message, DeprecationWarning, stacklevel=2) new_module, new_class_name = target_class_full_name.rsplit(".", 1) - return getattr(importlib.import_module(new_module), new_class_name) + try: + return getattr(importlib.import_module(new_module), new_class_name) + except ImportError as e: + error_message = ( + f"Could not import `{new_module}.{new_class_name}` while trying to import `{module}.{name}`." + ) + if extra_message: + error_message += f" {extra_message}." + raise ImportError(error_message) from e -def add_deprecated_classes(module_imports: dict[str, dict[str, str]], package: str): +def add_deprecated_classes( + module_imports: dict[str, dict[str, str]], + package: str, + override_deprecated_classes: dict[str, dict[str, str]] | None = None, + extra_message: str | None = None, +): + """ + Add deprecated class PEP-563 imports and warnings modules to the package. + + :param module_imports: imports to use + :param package: package name + :param override_deprecated_classes: override target classes with deprecated ones. If module + + target class is found in the dictionary, it will be displayed in the warning message. + :param extra_message: extra message to display in the warning or import error message + """ for module_name, imports in module_imports.items(): full_module_name = f"{package}.{module_name}" module_type = ModuleType(full_module_name) + if override_deprecated_classes and module_name in override_deprecated_classes: + override_deprecated_classes_for_module = override_deprecated_classes[module_name] + else: + override_deprecated_classes_for_module = {} + # Mypy is not able to derive the right function signature https://github.com/python/mypy/issues/2427 module_type.__getattr__ = functools.partial( # type: ignore[assignment] - getattr_with_deprecation, imports, full_module_name + getattr_with_deprecation, + imports, + full_module_name, + override_deprecated_classes_for_module, + extra_message if extra_message else "", ) sys.modules.setdefault(full_module_name, module_type) diff --git a/airflow/kubernetes/volume.py b/airflow/utils/empty_set.py similarity index 62% rename from airflow/kubernetes/volume.py rename to airflow/utils/empty_set.py index ecb39e457fd4..0871c1ec5a12 100644 --- a/airflow/kubernetes/volume.py +++ b/airflow/utils/empty_set.py @@ -1,4 +1,3 @@ -# # 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 @@ -15,19 +14,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""This module is deprecated. Please use :mod:`kubernetes.client.models.V1Volume`.""" + from __future__ import annotations -import warnings -from airflow.exceptions import RemovedInAirflow3Warning +def _get_empty_set_for_configuration() -> set[tuple[str, str]]: + """ + Retrieves an empty_set_for_configuration. -with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - from airflow.providers.cncf.kubernetes.backcompat.volume import Volume # noqa: autoflake + This method is only needed because configuration module has a deprecated method called set, and it + confuses mypy. This method will be removed when we remove the deprecated method. -warnings.warn( - "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.", - RemovedInAirflow3Warning, - stacklevel=2, -) + :meta private: + :return: empty set + """ + return set() diff --git a/airflow/utils/hashlib_wrapper.py b/airflow/utils/hashlib_wrapper.py index 09850c565cb0..65a18566b775 100644 --- a/airflow/utils/hashlib_wrapper.py +++ b/airflow/utils/hashlib_wrapper.py @@ -28,9 +28,10 @@ def md5(__string: ReadableBuffer = b"") -> hashlib._Hash: """ - Safely allows calling the hashlib.md5 function when "usedforsecurity" is disabled in the configuration. + Safely allows calling the ``hashlib.md5`` function when ``usedforsecurity`` is disabled in + the configuration. - :param string: The data to hash. Default to empty str byte. + :param __string: The data to hash. Default to empty str byte. :return: The hashed value. """ if PY39: diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index 499aa31c3497..5690799ce468 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -234,8 +234,12 @@ def ensure_pod_is_valid_after_unpickling(pod: V1Pod) -> V1Pod | None: if not isinstance(pod, V1Pod): return None try: - from airflow.kubernetes.pod_generator import PodGenerator - + try: + from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator + except ImportError: + from airflow.kubernetes.pre_7_4_0_compatibility.pod_generator import ( # type: ignore[assignment] + PodGenerator, + ) # now we actually reserialize / deserialize the pod pod_dict = sanitize_for_serialization(pod) return PodGenerator.deserialize_model_dict(pod_dict) diff --git a/airflow/www/views.py b/airflow/www/views.py index 54f46575be35..ae8811c05cc3 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1541,6 +1541,10 @@ def rendered_k8s(self, *, session: Session = NEW_SESSION): """Get rendered k8s yaml.""" if not settings.IS_K8S_OR_K8SCELERY_EXECUTOR: abort(404) + # This part is only used for k8s executor so providers.cncf.kubernetes must be installed + # with the get_rendered_k8s_spec method + from airflow.providers.cncf.kubernetes.template_rendering import get_rendered_k8s_spec + dag_id = request.args.get("dag_id") task_id = request.args.get("task_id") if task_id is None: @@ -1560,7 +1564,7 @@ def rendered_k8s(self, *, session: Session = NEW_SESSION): pod_spec = None try: - pod_spec = ti.get_rendered_k8s_spec(session=session) + pod_spec = get_rendered_k8s_spec(ti, session=session) except AirflowException as e: if not e.__cause__: flash(f"Error rendering Kubernetes POD Spec: {e}", "error") diff --git a/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py b/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py index fae38c0a53b5..b96829539aa3 100644 --- a/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/kubernetes_commands.py @@ -577,7 +577,7 @@ def _rebuild_k8s_image( COPY airflow/example_dags/ /opt/airflow/dags/ -COPY airflow/kubernetes_executor_templates/ /opt/airflow/pod_templates/ +COPY airflow/providers/cncf/kubernetes/kubernetes_executor_templates/ /opt/airflow/pod_templates/ ENV GUNICORN_CMD_ARGS='--preload' AIRFLOW__WEBSERVER__WORKER_REFRESH_INTERVAL=0 """ diff --git a/dev/breeze/src/airflow_breeze/pre_commit_ids.py b/dev/breeze/src/airflow_breeze/pre_commit_ids.py index 0ef8ef65b908..019058eeff82 100644 --- a/dev/breeze/src/airflow_breeze/pre_commit_ids.py +++ b/dev/breeze/src/airflow_breeze/pre_commit_ids.py @@ -27,6 +27,7 @@ "black", "blacken-docs", "check-aiobotocore-optional", + "check-airflow-k8s-not-used", "check-airflow-provider-compatibility", "check-apache-license-rat", "check-base-operator-partial-arguments", @@ -35,6 +36,7 @@ "check-breeze-top-dependencies-limited", "check-builtin-literals", "check-changelog-has-no-duplicates", + "check-cncf-k8s-only-for-executors", "check-core-deprecation-classes", "check-daysago-import-from-utils", "check-decorated-operator-implements-custom-name", diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 7437642e4e9b..799a8a5ae5cc 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -1049,7 +1049,8 @@ def test_upgrade_to_newer_dependencies(files: tuple[str, ...], expected_outputs: ("airflow/providers/celery/file.py",), { "docs-filter-list-as-string": "--package-filter apache-airflow " - "--package-filter apache-airflow-providers-celery", + "--package-filter apache-airflow-providers-celery " + "--package-filter apache-airflow-providers-cncf-kubernetes" }, id="Celery python files changed", ), diff --git a/docs/apache-airflow-providers-apache-spark/changelog.rst b/docs/apache-airflow-providers-apache-spark/changelog.rst index 1a8c7b1c3ea1..36a0eb511894 100644 --- a/docs/apache-airflow-providers-apache-spark/changelog.rst +++ b/docs/apache-airflow-providers-apache-spark/changelog.rst @@ -16,6 +16,4 @@ specific language governing permissions and limitations under the License. - - .. include:: ../../airflow/providers/apache/spark/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-spark/commits.rst b/docs/apache-airflow-providers-apache-spark/commits.rst index ba6baeae5a04..6b2f87851096 100644 --- a/docs/apache-airflow-providers-apache-spark/commits.rst +++ b/docs/apache-airflow-providers-apache-spark/commits.rst @@ -28,6 +28,21 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-07-06 ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` +`3878fe6fab `_ 2023-07-05 ``Remove spurious headers for provider changelogs (#32373)`` +`cb4927a018 `_ 2023-07-05 ``Prepare docs for July 2023 wave of Providers (#32298)`` +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== =================================================================== + 4.1.1 ..... @@ -36,6 +51,7 @@ Latest change: 2023-06-20 ================================================================================================= =========== ================================================================= Commit Committed Subject ================================================================================================= =========== ================================================================= +`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` `8b146152d6 `_ 2023-06-20 ``Add note about dropping Python 3.7 for providers (#32015)`` `6becb70316 `_ 2023-06-16 ``SparkSubmitOperator: rename spark_conn_id to conn_id (#31952)`` `13890788ae `_ 2023-06-07 ``Apache provider docstring improvements (#31730)`` diff --git a/docs/apache-airflow-providers-apache-spark/index.rst b/docs/apache-airflow-providers-apache-spark/index.rst index 1bd8a1def93c..300a89e540ee 100644 --- a/docs/apache-airflow-providers-apache-spark/index.rst +++ b/docs/apache-airflow-providers-apache-spark/index.rst @@ -76,7 +76,7 @@ Package apache-airflow-providers-apache-spark `Apache Spark `__ -Release: 4.1.1 +Release: 4.1.2 Provider package ---------------- @@ -102,3 +102,31 @@ PIP package Version required ``apache-airflow`` ``>=2.4.0`` ``pyspark`` ================== ================== + +Cross provider package dependencies +----------------------------------- + +Those are dependencies that might be needed in order to use all the features of the package. +You need to install the specified provider packages in order to use them. + +You can install such cross-provider dependencies when installing from PyPI. For example: + +.. code-block:: bash + + pip install apache-airflow-providers-apache-spark[cncf.kubernetes] + + +====================================================================================================================== =================== +Dependent package Extra +====================================================================================================================== =================== +`apache-airflow-providers-cncf-kubernetes `_ ``cncf.kubernetes`` +====================================================================================================================== =================== + +Downloading official packages +----------------------------- + +You can download officially released packages and verify their checksums and signatures from the +`Official Apache Download site `_ + +* `The apache-airflow-providers-apache-spark 4.1.2 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-apache-spark 4.1.2 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-celery/changelog.rst b/docs/apache-airflow-providers-celery/changelog.rst index 301ee64f5426..70dcac56ec81 100644 --- a/docs/apache-airflow-providers-celery/changelog.rst +++ b/docs/apache-airflow-providers-celery/changelog.rst @@ -16,6 +16,4 @@ specific language governing permissions and limitations under the License. - - .. include:: ../../airflow/providers/celery/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-celery/commits.rst b/docs/apache-airflow-providers-celery/commits.rst index 760e1b64aed7..66647f17f951 100644 --- a/docs/apache-airflow-providers-celery/commits.rst +++ b/docs/apache-airflow-providers-celery/commits.rst @@ -28,6 +28,26 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-07-22 ``Introduce decorator to load providers configuration (#32765)`` +`73b90c48b1 `_ 2023-07-21 ``Allow configuration to be contributed by providers (#32604)`` +`ea0deaa993 `_ 2023-07-16 ``Move default_celery.py to inside the provider (#32628)`` +`624211f33f `_ 2023-07-12 ``Add Executors discovery and documentation (#32532)`` +`40d54eac1a `_ 2023-07-12 ``Move CeleryExecutor to the celery provider (#32526)`` +`225e3041d2 `_ 2023-07-06 ``Prepare docs for July 2023 wave of Providers (RC2) (#32381)`` +`3878fe6fab `_ 2023-07-05 ``Remove spurious headers for provider changelogs (#32373)`` +`cb4927a018 `_ 2023-07-05 ``Prepare docs for July 2023 wave of Providers (#32298)`` +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== =================================================================== + 3.2.1 ..... @@ -36,6 +56,7 @@ Latest change: 2023-06-20 ================================================================================================= =========== ============================================================= Commit Committed Subject ================================================================================================= =========== ============================================================= +`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` `8b146152d6 `_ 2023-06-20 ``Add note about dropping Python 3.7 for providers (#32015)`` ================================================================================================= =========== ============================================================= diff --git a/docs/apache-airflow-providers-celery/index.rst b/docs/apache-airflow-providers-celery/index.rst index 464f12cd92e4..88a2a0ddd306 100644 --- a/docs/apache-airflow-providers-celery/index.rst +++ b/docs/apache-airflow-providers-celery/index.rst @@ -56,7 +56,7 @@ Package apache-airflow-providers-celery `Celery `__ -Release: 3.2.1 +Release: 3.3.0 Provider package ---------------- @@ -83,3 +83,31 @@ PIP package Version required ``celery`` ``>=5.2.3,<6`` ``flower`` ``>=1.0.0`` ================== ================== + +Cross provider package dependencies +----------------------------------- + +Those are dependencies that might be needed in order to use all the features of the package. +You need to install the specified provider packages in order to use them. + +You can install such cross-provider dependencies when installing from PyPI. For example: + +.. code-block:: bash + + pip install apache-airflow-providers-celery[cncf.kubernetes] + + +====================================================================================================================== =================== +Dependent package Extra +====================================================================================================================== =================== +`apache-airflow-providers-cncf-kubernetes `_ ``cncf.kubernetes`` +====================================================================================================================== =================== + +Downloading official packages +----------------------------- + +You can download officially released packages and verify their checksums and signatures from the +`Official Apache Download site `_ + +* `The apache-airflow-providers-celery 3.3.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-celery 3.3.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-cncf-kubernetes/configurations-ref.rst b/docs/apache-airflow-providers-cncf-kubernetes/configurations-ref.rst new file mode 100644 index 000000000000..5885c9d91b6e --- /dev/null +++ b/docs/apache-airflow-providers-cncf-kubernetes/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-cncf-kubernetes/index.rst b/docs/apache-airflow-providers-cncf-kubernetes/index.rst index 95ce846ab13e..9a19797d8f80 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/index.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/index.rst @@ -42,6 +42,7 @@ :maxdepth: 1 :caption: References + Configuration Python API <_api/airflow/providers/cncf/kubernetes/index> .. toctree:: diff --git a/docs/apache-airflow-providers-cncf-kubernetes/operators.rst b/docs/apache-airflow-providers-cncf-kubernetes/operators.rst index 2c020a53e087..7b7a77b4f773 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/operators.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/operators.rst @@ -97,7 +97,7 @@ like this: With this API object, you can have access to all Kubernetes API objects in the form of python classes. Using this method will ensure correctness and type safety. While we have removed almost all Kubernetes convenience classes, we have kept the -:class:`~airflow.kubernetes.secret.Secret` class to simplify the process of generating secret volumes/env variables. +:class:`~airflow.providers.cncf.kubernetes.secret.Secret` class to simplify the process of generating secret volumes/env variables. .. exampleinclude:: /../../tests/system/providers/cncf/kubernetes/example_kubernetes.py :language: python diff --git a/docs/apache-airflow/configurations-ref.rst b/docs/apache-airflow/configurations-ref.rst index 7a32f1790dea..c4882a8b903c 100644 --- a/docs/apache-airflow/configurations-ref.rst +++ b/docs/apache-airflow/configurations-ref.rst @@ -37,6 +37,7 @@ in the provider's documentation. The pre-installed providers that you may want t * :doc:`Configuration Reference for Celery Provider ` * :doc:`Configuration Reference for Apache Hive Provider ` +* :doc:`Configuration Reference for CNCF Kubernetes Provider ` .. note:: For more information see :doc:`/howto/set-config`. diff --git a/docs/apache-airflow/core-concepts/executor/celery.rst b/docs/apache-airflow/core-concepts/executor/celery.rst index f69da1570fc3..1113bda53068 100644 --- a/docs/apache-airflow/core-concepts/executor/celery.rst +++ b/docs/apache-airflow/core-concepts/executor/celery.rst @@ -21,6 +21,13 @@ Celery Executor =============== +.. note:: + + As of Airflow 2.7.0, you need to install the ``celery`` provider package to use this executor. + This can be done by installing ``apache-airflow-providers-celery>=3.3.0`` or by installing Airflow + with the ``celery`` extra: ``pip install 'apache-airflow[celery]'``. + + ``CeleryExecutor`` is one of the ways you can scale out the number of workers. For this to work, you need to setup a Celery backend (**RabbitMQ**, **Redis**, **Redis Sentinel** ...) and change your ``airflow.cfg`` to point the executor parameter to diff --git a/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst b/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst index 1eaf9bda5843..6416f5d2fa58 100644 --- a/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/celery_kubernetes.rst @@ -21,6 +21,14 @@ CeleryKubernetes Executor ========================= +.. note:: + + As of Airflow 2.7.0, you need to install both the ``celery`` and ``cncf.kubernetes`` provider package to use + this executor. This can be done by installing ``apache-airflow-providers-celery>=3.3.0`` and + ``apache-airflow-providers-cncf-kubernetes>=7.4.0`` or by installing Airflow + with the ``celery`` and ``cncf.kubernetes`` extras: ``pip install 'apache-airflow[celery,cncf.kubernetes]'``. + + The :class:`~airflow.providers.celery.executors.celery_kubernetes_executor.CeleryKubernetesExecutor` allows users to run simultaneously a ``CeleryExecutor`` and a ``KubernetesExecutor``. An executor is chosen to run a task based on the task's queue. diff --git a/docs/apache-airflow/core-concepts/executor/kubernetes.rst b/docs/apache-airflow/core-concepts/executor/kubernetes.rst index c5361e341737..e19264c96e5e 100644 --- a/docs/apache-airflow/core-concepts/executor/kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/kubernetes.rst @@ -21,6 +21,14 @@ Kubernetes Executor =================== +.. note:: + + As of Airflow 2.7.0, you need to install the ``cncf.kubernetes`` provider package to use + this executor. This can done by installing ``apache-airflow-providers-cncf-kubernetes>=7.4.0`` + or by installing Airflow with the ``cncf.kubernetes`` extras: + ``pip install 'apache-airflow[cncf.kubernetes]'``. + + The Kubernetes executor runs each task instance in its own pod on a Kubernetes cluster. KubernetesExecutor runs as a process in the Airflow Scheduler. The scheduler itself does @@ -100,21 +108,21 @@ With these requirements in mind, here are some examples of basic ``pod_template_ Storing DAGs in the image: -.. literalinclude:: /../../airflow/kubernetes/pod_template_file_examples/dags_in_image_template.yaml +.. literalinclude:: /../../airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_image_template.yaml :language: yaml :start-after: [START template_with_dags_in_image] :end-before: [END template_with_dags_in_image] Storing DAGs in a ``persistentVolume``: -.. literalinclude:: /../../airflow/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml +.. literalinclude:: /../../airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml :language: yaml :start-after: [START template_with_dags_in_volume] :end-before: [END template_with_dags_in_volume] Pulling DAGs from ``git``: -.. literalinclude:: /../../airflow/kubernetes/pod_template_file_examples/git_sync_template.yaml +.. literalinclude:: /../../airflow/providers/cncf/kubernetes/pod_template_file_examples/git_sync_template.yaml :language: yaml :start-after: [START git_sync_template] :end-before: [END git_sync_template] diff --git a/docs/apache-airflow/core-concepts/executor/local_kubernetes.rst b/docs/apache-airflow/core-concepts/executor/local_kubernetes.rst index ecb0dc536596..d50c7fc8780b 100644 --- a/docs/apache-airflow/core-concepts/executor/local_kubernetes.rst +++ b/docs/apache-airflow/core-concepts/executor/local_kubernetes.rst @@ -21,7 +21,14 @@ LocalKubernetes Executor ========================= -The :class:`~airflow.executors.local_kubernetes_executor.LocalKubernetesExecutor` allows users +.. note:: + + As of Airflow 2.7.0, you need to install the ``cncf.kubernetes`` provider package to use + this executor. This can be done by installing ``apache-airflow-providers-cncf-kubernetes>=7.4.0`` + or by installing Airflow with the ``cncf.kubernetes`` extras: + ``pip install 'apache-airflow[cncf.kubernetes]'``. + +The :class:`~airflow.providers.cncf.kubernetes.executors.local_kubernetes_executor.LocalKubernetesExecutor` allows users to simultaneously run a ``LocalExecutor`` and a ``KubernetesExecutor``. An executor is chosen to run a task based on the task's queue. diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst index ef6c4bf1b212..a17bfa073636 100644 --- a/docs/apache-airflow/extra-packages-ref.rst +++ b/docs/apache-airflow/extra-packages-ref.rst @@ -37,47 +37,43 @@ These are core airflow extras that extend capabilities of core Airflow. They usu packages (with the exception of ``celery`` and ``cncf.kubernetes`` extras), they just install necessary python dependencies for the provided package. -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| extra | install command | enables | Preinstalled | -+=====================+=====================================================+============================================================================+==============+ -| aiobotocore | ``pip install 'apache-airflow[aiobotocore]'`` | Support for asynchronous (deferrable) operators for Amazon integration | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| async | ``pip install 'apache-airflow[async]'`` | Async worker classes for Gunicorn | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| celery | ``pip install 'apache-airflow[celery]'`` | CeleryExecutor (also installs the celery provider package!) | * | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| cgroups | ``pip install 'apache-airflow[cgroups]'`` | Needed To use CgroupTaskRunner | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| cncf.kubernetes | ``pip install 'apache-airflow[cncf.kubernetes]'`` | Kubernetes Executor (also installs the Kubernetes provider package) | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| daskexecutor | ``pip install 'apache-airflow[daskexecutor]'`` | DaskExecutor ((also installs the Daskexecutor provider package) | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| deprecated_api | ``pip install 'apache-airflow[deprecated_api]'`` | Deprecated, experimental API that is replaced with the new REST API | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| github_enterprise | ``pip install 'apache-airflow[github_enterprise]'`` | GitHub Enterprise auth backend | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| google_auth | ``pip install 'apache-airflow[google_auth]'`` | Google auth backend | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| kerberos | ``pip install 'apache-airflow[kerberos]'`` | Kerberos integration for Kerberized services (Hadoop, Presto, Trino) | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| ldap | ``pip install 'apache-airflow[ldap]'`` | LDAP authentication for users | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| leveldb | ``pip install 'apache-airflow[leveldb]'`` | Required for use leveldb extra in google provider | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| otel | ``pip install 'apache-airflow[otel]'`` | Required for OpenTelemetry metrics | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| pandas | ``pip install 'apache-airflow[pandas]'`` | Install Pandas library compatible with Airflow | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| password | ``pip install 'apache-airflow[password]'`` | Password authentication for users | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| rabbitmq | ``pip install 'apache-airflow[rabbitmq]'`` | RabbitMQ support as a Celery backend | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| sentry | ``pip install 'apache-airflow[sentry]'`` | Sentry service for application logging and monitoring | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| statsd | ``pip install 'apache-airflow[statsd]'`` | Needed by StatsD metrics | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ -| virtualenv | ``pip install 'apache-airflow[virtualenv]'`` | Running python tasks in local virtualenv | | -+---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+--------------+ ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| extra | install command | enables | ++=====================+=====================================================+============================================================================+ +| aiobotocore | ``pip install 'apache-airflow[aiobotocore]'`` | Support for asynchronous (deferrable) operators for Amazon integration | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| async | ``pip install 'apache-airflow[async]'`` | Async worker classes for Gunicorn | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| cgroups | ``pip install 'apache-airflow[cgroups]'`` | Needed To use CgroupTaskRunner | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| daskexecutor | ``pip install 'apache-airflow[daskexecutor]'`` | DaskExecutor ((also installs the Daskexecutor provider package) | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| deprecated_api | ``pip install 'apache-airflow[deprecated_api]'`` | Deprecated, experimental API that is replaced with the new REST API | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| github_enterprise | ``pip install 'apache-airflow[github_enterprise]'`` | GitHub Enterprise auth backend | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| google_auth | ``pip install 'apache-airflow[google_auth]'`` | Google auth backend | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| kerberos | ``pip install 'apache-airflow[kerberos]'`` | Kerberos integration for Kerberized services (Hadoop, Presto, Trino) | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| ldap | ``pip install 'apache-airflow[ldap]'`` | LDAP authentication for users | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| leveldb | ``pip install 'apache-airflow[leveldb]'`` | Required for use leveldb extra in google provider | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| otel | ``pip install 'apache-airflow[otel]'`` | Required for OpenTelemetry metrics | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| pandas | ``pip install 'apache-airflow[pandas]'`` | Install Pandas library compatible with Airflow | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| password | ``pip install 'apache-airflow[password]'`` | Password authentication for users | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| rabbitmq | ``pip install 'apache-airflow[rabbitmq]'`` | RabbitMQ support as a Celery backend | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| sentry | ``pip install 'apache-airflow[sentry]'`` | Sentry service for application logging and monitoring | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| statsd | ``pip install 'apache-airflow[statsd]'`` | Needed by StatsD metrics | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ +| virtualenv | ``pip install 'apache-airflow[virtualenv]'`` | Running python tasks in local virtualenv | ++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+ Providers extras @@ -188,6 +184,8 @@ These are extras that add dependencies needed for integration with external serv +---------------------+-----------------------------------------------------+-----------------------------------------------------+ | facebook | ``pip install 'apache-airflow[facebook]'`` | Facebook Social | +---------------------+-----------------------------------------------------+-----------------------------------------------------+ +| github | ``pip install 'apache-airflow[github]'`` | GitHub operators and hook | ++---------------------+-----------------------------------------------------+-----------------------------------------------------+ | google | ``pip install 'apache-airflow[google]'`` | Google Cloud | +---------------------+-----------------------------------------------------+-----------------------------------------------------+ | hashicorp | ``pip install 'apache-airflow[hashicorp]'`` | Hashicorp Services (Vault) | @@ -232,6 +230,10 @@ These are extras that add dependencies needed for integration with other softwar +=====================+=====================================================+===========================================+ | arangodb | ``pip install 'apache-airflow[arangodb]'`` | ArangoDB operators, sensors and hook | +---------------------+-----------------------------------------------------+-------------------------------------------+ +| celery | ``pip install 'apache-airflow[celery]'`` | CeleryExecutor | ++---------------------+-----------------------------------------------------+-------------------------------------------+ +| cncf.kubernetes | ``pip install 'apache-airflow[cncf.kubernetes]'`` | Kubernetes Executor | ++---------------------+-----------------------------------------------------+-------------------------------------------+ | docker | ``pip install 'apache-airflow[docker]'`` | Docker hooks and operators | +---------------------+-----------------------------------------------------+-------------------------------------------+ | elasticsearch | ``pip install 'apache-airflow[elasticsearch]'`` | Elasticsearch hooks and Log Handler | @@ -349,10 +351,10 @@ Those are the extras that are needed to generated documentation for Airflow. Thi +---------------------+-----------------------------------------------------+----------------------------------------------------------------------+ -Deprecated extras ------------------ +Deprecated 1.10 extras +---------------------- -These are the extras that have been used before and deprecated in 2.0 and will be removed in Airflow 3.0.0. They were +These are the extras that have been deprecated in 2.0 and will be removed in Airflow 3.0.0. They were all replaced by new extras, which have naming consistent with the names of provider packages. The ``crypto`` extra is not needed any more, because all crypto dependencies are part of airflow package, diff --git a/docs/apache-airflow/howto/upgrading-from-1-10/index.rst b/docs/apache-airflow/howto/upgrading-from-1-10/index.rst index 782043f23406..f1956d89b95f 100644 --- a/docs/apache-airflow/howto/upgrading-from-1-10/index.rst +++ b/docs/apache-airflow/howto/upgrading-from-1-10/index.rst @@ -171,7 +171,7 @@ by using the ``| default`` Jinja filter as shown below. Much like the ``KubernetesExecutor``, the ``KubernetesPodOperator`` will no longer take Airflow custom classes and will instead expect either a pod_template yaml file, or ``kubernetes.client.models`` objects. -The one notable exception is that we will continue to support the ``airflow.kubernetes.secret.Secret`` class. +The one notable exception is that we will continue to support the ``airflow.providers.cncf.kubernetes.secret.Secret`` class. Whereas previously a user would import each individual class to build the pod as so: @@ -218,7 +218,7 @@ Now the user can use the ``kubernetes.client.models`` class as a single point of .. code-block:: python from kubernetes.client import models as k8s - from airflow.kubernetes.secret import Secret + from airflow.providers.cncf.kubernetes.secret import Secret configmaps = ["test-configmap-1", "test-configmap-2"] diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 6bfca3ef37d5..fb721ada25a5 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -35,6 +35,7 @@ alibaba allAuthenticatedUsers allowinsert allUsers +alphanumerics Alphasort amazonaws amqp @@ -296,6 +297,7 @@ ContainerPort contentUrl contextmgr contrib +CoreV coroutine coverals cp @@ -678,6 +680,7 @@ Harenslak Hashable Hashicorp hashicorp +hashlib hasn HCatalog hcatalog @@ -1105,7 +1108,9 @@ pluggable pluggy plyvel png +PodManager podName +PodSpec podSpec podspec poller diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index f4eeb8a27341..646b296444f3 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -197,7 +197,9 @@ "apache-airflow>=2.4.0", "pyspark" ], - "cross-providers-deps": [], + "cross-providers-deps": [ + "cncf.kubernetes" + ], "excluded-python-versions": [] }, "apache.sqoop": { @@ -245,7 +247,9 @@ "celery>=5.2.3,<6", "flower>=1.0.0" ], - "cross-providers-deps": [], + "cross-providers-deps": [ + "cncf.kubernetes" + ], "excluded-python-versions": [] }, "cloudant": { diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt index 540a5938a00e..c0ccd98d5a1a 100644 --- a/images/breeze/output-commands-hash.txt +++ b/images/breeze/output-commands-hash.txt @@ -62,7 +62,7 @@ setup:version:be116d90a21c2afe01087f7609774e1e setup:cdf58a9c27af2877dc5a658ed0a1d99c shell:87e7bdcebe1180395adfec86b2a065f5 start-airflow:f7216a8126ecf14b033e2ea677e1a105 -static-checks:f9ec0d7edaba84180403d95469d94ea0 +static-checks:a6b76083afc49c0c80270facd5236b42 testing:docker-compose-tests:0c810047fc66a0cfe91119e2d08b3507 testing:helm-tests:8e491da2e01ebd815322c37562059d77 testing:integration-tests:486e4d91449ecdb7630ef2a470d705a3 diff --git a/images/breeze/output_static-checks.svg b/images/breeze/output_static-checks.svg index 6220e1db62e0..083d5f668118 100644 --- a/images/breeze/output_static-checks.svg +++ b/images/breeze/output_static-checks.svg @@ -1,4 +1,4 @@ - +