From 64a3787c2414cc5f2e07ff10477c824efc2e9a5d Mon Sep 17 00:00:00 2001 From: darkag Date: Fri, 30 Jun 2023 09:55:23 +0200 Subject: [PATCH 001/533] Add various Vertica connection parameters (#32089) --------- Co-authored-by: ivascot --- airflow/providers/vertica/hooks/vertica.py | 51 ++++++++++++ .../connections/vertica.rst | 83 +++++++++++++++++++ .../index.rst | 7 ++ tests/providers/vertica/hooks/test_vertica.py | 76 +++++++++++++++++ 4 files changed, 217 insertions(+) create mode 100644 docs/apache-airflow-providers-vertica/connections/vertica.rst diff --git a/airflow/providers/vertica/hooks/vertica.py b/airflow/providers/vertica/hooks/vertica.py index 92a74ea36901..06b2e3cf179b 100644 --- a/airflow/providers/vertica/hooks/vertica.py +++ b/airflow/providers/vertica/hooks/vertica.py @@ -46,5 +46,56 @@ def get_conn(self) -> connect: else: conn_config["port"] = int(conn.port) + bool_options = [ + "connection_load_balance", + "binary_transfer", + "disable_copy_local", + "request_complex_types", + "use_prepared_statements", + ] + std_options = [ + "session_label", + "backup_server_node", + "kerberos_host_name", + "kerberos_service_name", + "unicode_error", + "workload", + "ssl", + ] + conn_extra = conn.extra_dejson + + for bo in bool_options: + if bo in conn_extra: + conn_config[bo] = str(conn_extra[bo]).lower() in ["true", "on"] + + for so in std_options: + if so in conn_extra: + conn_config[so] = conn_extra[so] + + if "connection_timeout" in conn_extra: + conn_config["connection_timeout"] = float(conn_extra["connection_timeout"]) + + if "log_level" in conn_extra: + import logging + + log_lvl = conn_extra["log_level"] + conn_config["log_path"] = None + if isinstance(log_lvl, str): + log_lvl = log_lvl.lower() + if log_lvl == "critical": + conn_config["log_level"] = logging.CRITICAL + elif log_lvl == "error": + conn_config["log_level"] = logging.ERROR + elif log_lvl == "warning": + conn_config["log_level"] = logging.WARNING + elif log_lvl == "info": + conn_config["log_level"] = logging.INFO + elif log_lvl == "debug": + conn_config["log_level"] = logging.DEBUG + elif log_lvl == "notset": + conn_config["log_level"] = logging.NOTSET + else: + conn_config["log_level"] = int(conn_extra["log_level"]) + conn = connect(**conn_config) return conn diff --git a/docs/apache-airflow-providers-vertica/connections/vertica.rst b/docs/apache-airflow-providers-vertica/connections/vertica.rst new file mode 100644 index 000000000000..86f583a54812 --- /dev/null +++ b/docs/apache-airflow-providers-vertica/connections/vertica.rst @@ -0,0 +1,83 @@ + .. 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. + + + +.. _howto/connection:vertica: + +Vertica Connection +================== +The Vertica connection type provides connection to a Vertica database. + +Configuring the Connection +-------------------------- +Host (required) + The host to connect to. + +Schema (optional) + Specify the schema name to be used in the database. + +Login (required) + Specify the user name to connect. + +Password (required) + Specify the password to connect. + +Extra (optional) + Specify the extra parameters (as json dictionary) that can be used in Vertica + connection. + + The following extras are supported: + + * ``backup_server_node``: See `Connection Failover `_. + * ``binary_transfer``: See `Data Transfer Format `_. + * ``connection_load_balance``: See `Connection Load Balancing `_. + * ``connection_timeout``: The number of seconds (can be a nonnegative floating point number) the client + waits for a socket operation (Establishing a TCP connection or read/write operation). + * ``disable_copy_local``: See `COPY FROM LOCAL `_. + * ``kerberos_host_name``: See `Kerberos Authentication `_. + * ``kerberos_service_name``: See `Kerberos Authentication `_. + * ``log_level``: Enable vertica client logging. Traces will be visible in tasks log. See `Logging `_. + * ``request_complex_types:``: See `SQL Data conversion to Python objects `_. + * ``session_label``: Sets a label for the connection on the server. + * ``ssl``: Support only True or False. See `TLS/SSL `_. + * ``unicode_error``: See `UTF-8 encoding issues `_. + * ``use_prepared_statements``: See `Passing parameters to SQL queries `_. + * ``workload``: Sets the workload name associated with this session. + + See `vertica-python docs `_ for details. + + + Example "extras" field: + + .. code-block:: json + + { + "connection_load_balance": true, + "log_level": "error", + "ssl": true + } + + or + + .. code-block:: json + + { + "session_label": "airflow-session", + "connection_timeout": 30, + "backup_server_node": ["bck_server_1", "bck_server_2"] + } diff --git a/docs/apache-airflow-providers-vertica/index.rst b/docs/apache-airflow-providers-vertica/index.rst index db09f1924c4d..ae7c2457b811 100644 --- a/docs/apache-airflow-providers-vertica/index.rst +++ b/docs/apache-airflow-providers-vertica/index.rst @@ -29,6 +29,13 @@ Changelog Security +.. toctree:: + :hidden: + :maxdepth: 1 + :caption: Guides + + Connection types + .. toctree:: :hidden: :maxdepth: 1 diff --git a/tests/providers/vertica/hooks/test_vertica.py b/tests/providers/vertica/hooks/test_vertica.py index e78c2a0c5c81..146c3bcd1136 100644 --- a/tests/providers/vertica/hooks/test_vertica.py +++ b/tests/providers/vertica/hooks/test_vertica.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import json from unittest import mock from unittest.mock import patch @@ -47,6 +48,81 @@ def test_get_conn(self, mock_connect): host="host", port=5433, database="vertica", user="login", password="password" ) + @patch("airflow.providers.vertica.hooks.vertica.connect") + def test_get_conn_extra_parameters_no_cast(self, mock_connect): + """Test if parameters are correctly passed to connection""" + extra_dict = self.connection.extra_dejson + bool_options = [ + "connection_load_balance", + "binary_transfer", + "disable_copy_local", + "use_prepared_statements", + ] + for bo in bool_options: + extra_dict.update({bo: True}) + extra_dict.update({"request_complex_types": False}) + + std_options = [ + "session_label", + "kerberos_host_name", + "kerberos_service_name", + "unicode_error", + "workload", + "ssl", + ] + for so in std_options: + extra_dict.update({so: so}) + bck_server_node = ["1.2.3.4", "4.3.2.1"] + conn_timeout = 30 + log_lvl = 40 + extra_dict.update({"backup_server_node": bck_server_node}) + extra_dict.update({"connection_timeout": conn_timeout}) + extra_dict.update({"log_level": log_lvl}) + self.connection.extra = json.dumps(extra_dict) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + for bo in bool_options: + assert kwargs[bo] is True + assert kwargs["request_complex_types"] is False + for so in std_options: + assert kwargs[so] == so + assert bck_server_node[0] in kwargs["backup_server_node"] + assert bck_server_node[1] in kwargs["backup_server_node"] + assert kwargs["connection_timeout"] == conn_timeout + assert kwargs["log_level"] == log_lvl + assert kwargs["log_path"] is None + + @patch("airflow.providers.vertica.hooks.vertica.connect") + def test_get_conn_extra_parameters_cast(self, mock_connect): + """Test if parameters that can be passed either as string or int/bool + like log_level are correctly converted when passed as string + (while test_get_conn_extra_parameters_no_cast tests them passed as int/bool)""" + import logging + + extra_dict = self.connection.extra_dejson + bool_options = [ + "connection_load_balance", + "binary_transfer", + "disable_copy_local", + "use_prepared_statements", + ] + for bo in bool_options: + extra_dict.update({bo: "True"}) + extra_dict.update({"request_complex_types": "False"}) + extra_dict.update({"log_level": "Error"}) + self.connection.extra = json.dumps(extra_dict) + self.db_hook.get_conn() + assert mock_connect.call_count == 1 + args, kwargs = mock_connect.call_args + assert args == () + for bo in bool_options: + assert kwargs[bo] is True + assert kwargs["request_complex_types"] is False + assert kwargs["log_level"] == logging.ERROR + assert kwargs["log_path"] is None + class TestVerticaHook: def setup_method(self): From dd937e51fe1ae3cd36a6993bd42e425960644e1d Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Fri, 30 Jun 2023 10:01:13 +0200 Subject: [PATCH 002/533] Add `on_finish_action` to `KubernetesPodOperator` (#30718) * Add a new arg for KPO to only delete the pod when it doesn't fail * deprecate is_delete_operator_pod and add on_finish_action * Add deprecated properties and fix unit tests * add missing attribute * Apply suggestions from code review Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> * update GKEStartPodOperator to be consistent with KPO * update EksPodOperator to be consistent with KPO * update unit tests and the method used to check the kpo compatibility * Fix a bug and add a new unit test for each provider * warn with AirflowProviderDeprecationWarning instead of DeprecationWarning * Bump KPO min version in GCP provider and add a new one to AWS provider * Add the new param to the GKE trigger * Apply suggestions from code review Co-authored-by: Jarek Potiuk --------- Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> Co-authored-by: Jarek Potiuk --- airflow/providers/amazon/aws/operators/eks.py | 34 ++++++-- airflow/providers/amazon/provider.yaml | 3 + .../cncf/kubernetes/operators/pod.py | 43 ++++++++-- .../providers/cncf/kubernetes/triggers/pod.py | 34 ++++++-- .../cncf/kubernetes/utils/pod_manager.py | 9 ++ .../cloud/operators/kubernetes_engine.py | 38 ++++++--- .../cloud/triggers/kubernetes_engine.py | 34 ++++++-- airflow/providers/google/provider.yaml | 2 +- .../amazon/aws/operators/test_eks.py | 58 ++++++++++++- .../cncf/kubernetes/operators/test_pod.py | 84 +++++++++++++++---- .../cncf/kubernetes/triggers/test_pod.py | 74 ++++++++++++++-- .../cloud/operators/test_kubernetes_engine.py | 61 ++++++++++++++ .../cloud/triggers/test_kubernetes_engine.py | 4 +- .../amazon/aws/example_eks_templated.py | 2 +- .../example_eks_with_fargate_in_one_step.py | 2 +- .../aws/example_eks_with_fargate_profile.py | 2 +- .../example_eks_with_nodegroup_in_one_step.py | 2 +- .../cncf/kubernetes/example_kubernetes.py | 6 +- .../kubernetes/example_kubernetes_async.py | 6 +- .../example_kubernetes_engine.py | 4 +- .../example_kubernetes_engine_async.py | 4 +- 21 files changed, 430 insertions(+), 76 deletions(-) diff --git a/airflow/providers/amazon/aws/operators/eks.py b/airflow/providers/amazon/aws/operators/eks.py index 9c27e88350c4..bea422398726 100644 --- a/airflow/providers/amazon/aws/operators/eks.py +++ b/airflow/providers/amazon/aws/operators/eks.py @@ -34,6 +34,7 @@ EksNodegroupTrigger, ) from airflow.providers.amazon.aws.utils.waiter_with_logging import wait +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction try: from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator @@ -854,10 +855,15 @@ class EksPodOperator(KubernetesPodOperator): running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). + :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. + If "delete_pod", the pod will be deleted regardless it's state; if "delete_succeeded_pod", + only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. + Current default is `keep_pod`, but this will be changed in the next major release of this provider. :param is_delete_operator_pod: What to do when the pod reaches its final state, or the execution is interrupted. If True, delete the - pod; if False, leave the pod. Current default is False, but this will be + pod; if False, leave the pod. Current default is False, but this will be changed in the next major release of this provider. + Deprecated - use `on_finish_action` instead. """ @@ -885,19 +891,32 @@ def __init__( pod_username: str | None = None, aws_conn_id: str = DEFAULT_CONN_ID, region: str | None = None, + on_finish_action: str | None = None, is_delete_operator_pod: bool | None = None, **kwargs, ) -> None: - if is_delete_operator_pod is None: + if is_delete_operator_pod is not None: warnings.warn( - f"You have not set parameter `is_delete_operator_pod` in class {self.__class__.__name__}. " - "Currently the default for this parameter is `False` but in a future release the default " - "will be changed to `True`. To ensure pods are not deleted in the future you will need to " - "set `is_delete_operator_pod=False` explicitly.", + "`is_delete_operator_pod` parameter is deprecated, please use `on_finish_action`", AirflowProviderDeprecationWarning, stacklevel=2, ) - is_delete_operator_pod = False + kwargs["on_finish_action"] = ( + OnFinishAction.DELETE_POD if is_delete_operator_pod else OnFinishAction.KEEP_POD + ) + else: + if on_finish_action is not None: + kwargs["on_finish_action"] = OnFinishAction(on_finish_action) + else: + warnings.warn( + f"You have not set parameter `on_finish_action` in class {self.__class__.__name__}. " + "Currently the default for this parameter is `keep_pod` but in a future release" + " the default will be changed to `delete_pod`. To ensure pods are not deleted in" + " the future you will need to set `on_finish_action=keep_pod` explicitly.", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) + kwargs["on_finish_action"] = OnFinishAction.KEEP_POD self.cluster_name = cluster_name self.in_cluster = in_cluster @@ -909,7 +928,6 @@ def __init__( in_cluster=self.in_cluster, namespace=self.namespace, name=self.pod_name, - is_delete_operator_pod=is_delete_operator_pod, **kwargs, ) # There is no need to manage the kube_config file, as it will be generated automatically. diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index 0c74094e8193..a7dac9fad481 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -667,3 +667,6 @@ additional-extras: - name: aiobotocore dependencies: - aiobotocore[boto3]>=2.2.0 + - name: cncf.kubernetes + dependencies: + - apache-airflow-providers-cncf-kubernetes>=7.2.0 diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index 87bfc41301b8..696611c6c2e9 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -23,6 +23,7 @@ import re import secrets import string +import warnings from collections.abc import Container from contextlib import AbstractContextManager from functools import cached_property @@ -32,7 +33,7 @@ from slugify import slugify from urllib3.exceptions import HTTPError -from airflow.exceptions import AirflowException, AirflowSkipException +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 @@ -52,6 +53,7 @@ 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 ( + OnFinishAction, PodLaunchFailedException, PodManager, PodOperatorHookProtocol, @@ -188,9 +190,6 @@ class KubernetesPodOperator(BaseOperator): If more than one secret is required, provide a comma separated list: secret_a,secret_b :param service_account_name: Name of the service account - :param is_delete_operator_pod: What to do when the pod reaches its final - state, or the execution is interrupted. If True (default), delete the - pod; if False, leave the pod. :param hostnetwork: If True enable host networking on the pod. :param tolerations: A list of kubernetes tolerations. :param security_context: security options the pod should run with (PodSecurityContext). @@ -226,6 +225,13 @@ class KubernetesPodOperator(BaseOperator): :param deferrable: Run operator in the deferrable mode. :param poll_interval: Polling period in seconds to check for the status. Used only in deferrable mode. :param log_pod_spec_on_failure: Log the pod's specification if a failure occurs + :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. + If "delete_pod", the pod will be deleted regardless it's state; if "delete_succeeded_pod", + only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. + :param is_delete_operator_pod: What to do when the pod reaches its final + state, or the execution is interrupted. If True (default), delete the + pod; if False, leave the pod. + Deprecated - use `on_finish_action` instead. """ # This field can be overloaded at the instance level via base_container_name @@ -279,7 +285,6 @@ def __init__( node_selector: dict | None = None, image_pull_secrets: list[k8s.V1LocalObjectReference] | None = None, service_account_name: str | None = None, - is_delete_operator_pod: bool = True, hostnetwork: bool = False, tolerations: list[k8s.V1Toleration] | None = None, security_context: dict | None = None, @@ -303,6 +308,8 @@ def __init__( deferrable: bool = False, poll_interval: float = 2, log_pod_spec_on_failure: bool = True, + on_finish_action: str = "delete_pod", + is_delete_operator_pod: None | bool = None, **kwargs, ) -> None: # TODO: remove in provider 6.0.0 release. This is a mitigate step to advise users to switch to the @@ -350,7 +357,6 @@ def __init__( self.config_file = config_file self.image_pull_secrets = convert_image_pull_secrets(image_pull_secrets) if image_pull_secrets else [] self.service_account_name = service_account_name - self.is_delete_operator_pod = is_delete_operator_pod self.hostnetwork = hostnetwork self.tolerations = ( [convert_toleration(toleration) for toleration in tolerations] if tolerations else [] @@ -384,6 +390,20 @@ def __init__( self.poll_interval = poll_interval self.remote_pod: k8s.V1Pod | None = None self.log_pod_spec_on_failure = log_pod_spec_on_failure + if is_delete_operator_pod is not None: + warnings.warn( + "`is_delete_operator_pod` parameter is deprecated, please use `on_finish_action`", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) + self.on_finish_action = ( + OnFinishAction.DELETE_POD if is_delete_operator_pod else OnFinishAction.KEEP_POD + ) + self.is_delete_operator_pod = is_delete_operator_pod + else: + self.on_finish_action = OnFinishAction(on_finish_action) + self.is_delete_operator_pod = self.on_finish_action == OnFinishAction.DELETE_POD + self._config_dict: dict | None = None # TODO: remove it when removing convert_config_file_to_dict @cached_property @@ -595,10 +615,10 @@ def invoke_defer_method(self): config_file=self.config_file, in_cluster=self.in_cluster, poll_interval=self.poll_interval, - should_delete_pod=self.is_delete_operator_pod, get_logs=self.get_logs, startup_timeout=self.startup_timeout_seconds, base_container_name=self.base_container_name, + on_finish_action=self.on_finish_action.value, ), method_name="execute_complete", ) @@ -669,7 +689,8 @@ def post_complete_action(self, *, pod, remote_pod, **kwargs): def cleanup(self, pod: k8s.V1Pod, remote_pod: k8s.V1Pod): pod_phase = remote_pod.status.phase if hasattr(remote_pod, "status") else None - if pod_phase != PodPhase.SUCCEEDED or not self.is_delete_operator_pod: + # if the pod fails or success, but we don't want to delete it + if pod_phase != PodPhase.SUCCEEDED or self.on_finish_action == OnFinishAction.KEEP_POD: self.patch_already_checked(remote_pod, reraise=False) if pod_phase != PodPhase.SUCCEEDED: @@ -722,7 +743,11 @@ def _read_pod_events(self, pod, *, reraise=True): def process_pod_deletion(self, pod: k8s.V1Pod, *, reraise=True): with _optionally_suppress(reraise=reraise): if pod is not None: - if self.is_delete_operator_pod: + should_delete_pod = (self.on_finish_action == OnFinishAction.DELETE_POD) or ( + self.on_finish_action == OnFinishAction.DELETE_SUCCEEDED_POD + and pod.status.phase == PodPhase.SUCCEEDED + ) + if should_delete_pod: self.log.info("Deleting pod: %s", pod.metadata.name) self.pod_manager.delete_pod(pod) else: diff --git a/airflow/providers/cncf/kubernetes/triggers/pod.py b/airflow/providers/cncf/kubernetes/triggers/pod.py index 6cfb6c523a31..6fdf763eceda 100644 --- a/airflow/providers/cncf/kubernetes/triggers/pod.py +++ b/airflow/providers/cncf/kubernetes/triggers/pod.py @@ -17,6 +17,7 @@ from __future__ import annotations import asyncio +import warnings from asyncio import CancelledError from datetime import datetime from enum import Enum @@ -25,8 +26,9 @@ import pytz from kubernetes_asyncio.client.models import V1Pod +from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.providers.cncf.kubernetes.hooks.kubernetes import AsyncKubernetesHook -from airflow.providers.cncf.kubernetes.utils.pod_manager import PodPhase +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction, PodPhase from airflow.triggers.base import BaseTrigger, TriggerEvent @@ -57,11 +59,15 @@ class KubernetesPodTrigger(BaseTrigger): :param poll_interval: Polling period in seconds to check for the status. :param trigger_start_time: time in Datetime format when the trigger was started :param in_cluster: run kubernetes client with in_cluster configuration. + :param get_logs: get the stdout of the container as logs of the tasks. + :param startup_timeout: timeout in seconds to start up the pod. + :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. + If "delete_pod", the pod will be deleted regardless it's state; if "delete_succeeded_pod", + only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. :param should_delete_pod: What to do when the pod reaches its final state, or the execution is interrupted. If True (default), delete the pod; if False, leave the pod. - :param get_logs: get the stdout of the container as logs of the tasks. - :param startup_timeout: timeout in seconds to start up the pod. + Deprecated - use `on_finish_action` instead. """ def __init__( @@ -75,9 +81,10 @@ def __init__( cluster_context: str | None = None, config_file: str | None = None, in_cluster: bool | None = None, - should_delete_pod: bool = True, get_logs: bool = True, startup_timeout: int = 120, + on_finish_action: str = "delete_pod", + should_delete_pod: bool | None = None, ): super().__init__() self.pod_name = pod_name @@ -89,10 +96,22 @@ def __init__( self.cluster_context = cluster_context self.config_file = config_file self.in_cluster = in_cluster - self.should_delete_pod = should_delete_pod self.get_logs = get_logs self.startup_timeout = startup_timeout + if should_delete_pod is not None: + warnings.warn( + "`should_delete_pod` parameter is deprecated, please use `on_finish_action`", + AirflowProviderDeprecationWarning, + ) + self.on_finish_action = ( + OnFinishAction.DELETE_POD if should_delete_pod else OnFinishAction.KEEP_POD + ) + self.should_delete_pod = should_delete_pod + else: + self.on_finish_action = OnFinishAction(on_finish_action) + self.should_delete_pod = self.on_finish_action == OnFinishAction.DELETE_POD + self._hook: AsyncKubernetesHook | None = None self._since_time = None @@ -109,10 +128,11 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "cluster_context": self.cluster_context, "config_file": self.config_file, "in_cluster": self.in_cluster, - "should_delete_pod": self.should_delete_pod, "get_logs": self.get_logs, "startup_timeout": self.startup_timeout, "trigger_start_time": self.trigger_start_time, + "should_delete_pod": self.should_delete_pod, + "on_finish_action": self.on_finish_action.value, }, ) @@ -191,7 +211,7 @@ async def run(self) -> AsyncIterator[TriggerEvent]: # type: ignore[override] name=self.pod_name, namespace=self.pod_namespace, ) - if self.should_delete_pod: + if self.on_finish_action == OnFinishAction.DELETE_POD: self.log.info("Deleting pod...") await self._get_async_hook().delete_pod( name=self.pod_name, diff --git a/airflow/providers/cncf/kubernetes/utils/pod_manager.py b/airflow/providers/cncf/kubernetes/utils/pod_manager.py index 3bacb95f4ff7..71b5e171271b 100644 --- a/airflow/providers/cncf/kubernetes/utils/pod_manager.py +++ b/airflow/providers/cncf/kubernetes/utils/pod_manager.py @@ -17,6 +17,7 @@ """Launches PODs.""" from __future__ import annotations +import enum import json import logging import math @@ -585,3 +586,11 @@ def _exec_pod_command(self, resp, command: str) -> str | None: if res: return res return res + + +class OnFinishAction(enum.Enum): + """Action to take when the pod finishes.""" + + KEEP_POD = "keep_pod" + DELETE_POD = "delete_pod" + DELETE_SUCCEEDED_POD = "delete_succeeded_pod" diff --git a/airflow/providers/google/cloud/operators/kubernetes_engine.py b/airflow/providers/google/cloud/operators/kubernetes_engine.py index 55971f52b786..bf14828d8702 100644 --- a/airflow/providers/google/cloud/operators/kubernetes_engine.py +++ b/airflow/providers/google/cloud/operators/kubernetes_engine.py @@ -27,6 +27,7 @@ from kubernetes.client.models import V1Pod from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction try: from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator @@ -45,7 +46,6 @@ if TYPE_CHECKING: from airflow.utils.context import Context - KUBE_CONFIG_ENV_VAR = "KUBECONFIG" @@ -427,11 +427,16 @@ class GKEStartPodOperator(KubernetesPodOperator): Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). :param regional: The location param is region name. + :param deferrable: Run operator in the deferrable mode. + :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. + If "delete_pod", the pod will be deleted regardless it's state; if "delete_succeeded_pod", + only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. + Current default is `keep_pod`, but this will be changed in the next major release of this provider. :param is_delete_operator_pod: What to do when the pod reaches its final state, or the execution is interrupted. If True, delete the pod; if False, leave the pod. Current default is False, but this will be changed in the next major release of this provider. - :param deferrable: Run operator in the deferrable mode. + Deprecated - use `on_finish_action` instead. """ template_fields: Sequence[str] = tuple( @@ -449,19 +454,32 @@ def __init__( gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, regional: bool | None = None, + on_finish_action: str | None = None, is_delete_operator_pod: bool | None = None, **kwargs, ) -> None: - if is_delete_operator_pod is None: + if is_delete_operator_pod is not None: warnings.warn( - f"You have not set parameter `is_delete_operator_pod` in class {self.__class__.__name__}. " - "Currently the default for this parameter is `False` but in a future release the default " - "will be changed to `True`. To ensure pods are not deleted in the future you will need to " - "set `is_delete_operator_pod=False` explicitly.", + "`is_delete_operator_pod` parameter is deprecated, please use `on_finish_action`", AirflowProviderDeprecationWarning, stacklevel=2, ) - is_delete_operator_pod = False + kwargs["on_finish_action"] = ( + OnFinishAction.DELETE_POD if is_delete_operator_pod else OnFinishAction.KEEP_POD + ) + else: + if on_finish_action is not None: + kwargs["on_finish_action"] = OnFinishAction(on_finish_action) + else: + warnings.warn( + f"You have not set parameter `on_finish_action` in class {self.__class__.__name__}. " + "Currently the default for this parameter is `keep_pod` but in a future release" + " the default will be changed to `delete_pod`. To ensure pods are not deleted in" + " the future you will need to set `on_finish_action=keep_pod` explicitly.", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) + kwargs["on_finish_action"] = OnFinishAction.KEEP_POD if regional is not None: warnings.warn( @@ -472,7 +490,7 @@ def __init__( stacklevel=2, ) - super().__init__(is_delete_operator_pod=is_delete_operator_pod, **kwargs) + super().__init__(**kwargs) self.project_id = project_id self.location = location self.cluster_name = cluster_name @@ -560,8 +578,8 @@ def invoke_defer_method(self): cluster_context=self.cluster_context, poll_interval=self.poll_interval, in_cluster=self.in_cluster, - should_delete_pod=self.is_delete_operator_pod, base_container_name=self.base_container_name, + on_finish_action=self.on_finish_action, ), method_name="execute_complete", kwargs={"cluster_url": self._cluster_url, "ssl_ca_cert": self._ssl_ca_cert}, diff --git a/airflow/providers/google/cloud/triggers/kubernetes_engine.py b/airflow/providers/google/cloud/triggers/kubernetes_engine.py index 1ec0e420f93a..ba0df0fc153c 100644 --- a/airflow/providers/google/cloud/triggers/kubernetes_engine.py +++ b/airflow/providers/google/cloud/triggers/kubernetes_engine.py @@ -18,11 +18,15 @@ from __future__ import annotations import asyncio +import warnings from datetime import datetime from typing import Any, AsyncIterator, Sequence from google.cloud.container_v1.types import Operation +from airflow.exceptions import AirflowProviderDeprecationWarning +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction + try: from airflow.providers.cncf.kubernetes.triggers.pod import KubernetesPodTrigger except ImportError: @@ -44,15 +48,19 @@ class GKEStartPodTrigger(KubernetesPodTrigger): :param poll_interval: Polling period in seconds to check for the status. :param trigger_start_time: time in Datetime format when the trigger was started :param in_cluster: run kubernetes client with in_cluster configuration. - :param should_delete_pod: What to do when the pod reaches its final - state, or the execution is interrupted. If True (default), delete the - pod; if False, leave the pod. :param get_logs: get the stdout of the container as logs of the tasks. :param startup_timeout: timeout in seconds to start up the pod. :param base_container_name: The name of the base container in the pod. This container's logs will appear as part of this task's logs if get_logs is True. Defaults to None. If None, will consult the class variable BASE_CONTAINER_NAME (which defaults to "base") for the base container name to use. + :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. + If "delete_pod", the pod will be deleted regardless it's state; if "delete_succeeded_pod", + only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. + :param should_delete_pod: What to do when the pod reaches its final + state, or the execution is interrupted. If True (default), delete the + pod; if False, leave the pod. + Deprecated - use `on_finish_action` instead. """ def __init__( @@ -66,9 +74,10 @@ def __init__( cluster_context: str | None = None, poll_interval: float = 2, in_cluster: bool | None = None, - should_delete_pod: bool = True, get_logs: bool = True, startup_timeout: int = 120, + on_finish_action: str = "delete_pod", + should_delete_pod: bool | None = None, *args, **kwargs, ): @@ -87,10 +96,22 @@ def __init__( self.poll_interval = poll_interval self.cluster_context = cluster_context self.in_cluster = in_cluster - self.should_delete_pod = should_delete_pod self.get_logs = get_logs self.startup_timeout = startup_timeout + if should_delete_pod is not None: + warnings.warn( + "`should_delete_pod` parameter is deprecated, please use `on_finish_action`", + AirflowProviderDeprecationWarning, + ) + self.on_finish_action = ( + OnFinishAction.DELETE_POD if should_delete_pod else OnFinishAction.KEEP_POD + ) + self.should_delete_pod = should_delete_pod + else: + self.on_finish_action = OnFinishAction(on_finish_action) + self.should_delete_pod = self.on_finish_action == OnFinishAction.DELETE_POD + self._cluster_url = cluster_url self._ssl_ca_cert = ssl_ca_cert @@ -105,11 +126,12 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "poll_interval": self.poll_interval, "cluster_context": self.cluster_context, "in_cluster": self.in_cluster, - "should_delete_pod": self.should_delete_pod, "get_logs": self.get_logs, "startup_timeout": self.startup_timeout, "trigger_start_time": self.trigger_start_time, "base_container_name": self.base_container_name, + "should_delete_pod": self.should_delete_pod, + "on_finish_action": self.on_finish_action.value, }, ) diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml index 3fc2c90b9b52..f22ff420821d 100644 --- a/airflow/providers/google/provider.yaml +++ b/airflow/providers/google/provider.yaml @@ -1130,7 +1130,7 @@ additional-extras: - apache-beam[gcp] - name: cncf.kubernetes dependencies: - - apache-airflow-providers-cncf-kubernetes>=6.2.0 + - apache-airflow-providers-cncf-kubernetes>=7.2.0 - name: leveldb dependencies: - plyvel diff --git a/tests/providers/amazon/aws/operators/test_eks.py b/tests/providers/amazon/aws/operators/test_eks.py index 5534f635d8f3..9ea8ab72d72f 100644 --- a/tests/providers/amazon/aws/operators/test_eks.py +++ b/tests/providers/amazon/aws/operators/test_eks.py @@ -39,6 +39,7 @@ EksDeleteFargateProfileTrigger, EksNodegroupTrigger, ) +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction from airflow.typing_compat import TypedDict from tests.providers.amazon.aws.utils.eks_test_constants import ( NODEROLE_ARN, @@ -641,7 +642,7 @@ def test_existing_nodegroup( labels={"demo": "hello_world"}, get_logs=True, # Delete the pod when it reaches its final state, or the execution is interrupted. - is_delete_operator_pod=True, + on_finish_action="delete_pod", ) op_return_value = op.execute(ti_context) mock_k8s_pod_operator_execute.assert_called_once_with(ti_context) @@ -651,3 +652,58 @@ def test_existing_nodegroup( ) assert mock_k8s_pod_operator_execute.return_value == op_return_value assert mock_generate_config_file.return_value.__enter__.return_value == op.config_file + + @pytest.mark.parametrize( + "compatible_kpo, kwargs, expected_attributes", + [ + ( + True, + {"on_finish_action": "delete_succeeded_pod"}, + {"on_finish_action": OnFinishAction.DELETE_SUCCEEDED_POD}, + ), + ( + # test that priority for deprecated param + True, + {"on_finish_action": "keep_pod", "is_delete_operator_pod": True}, + {"on_finish_action": OnFinishAction.DELETE_POD, "is_delete_operator_pod": True}, + ), + ( + # test default + True, + {}, + {"on_finish_action": OnFinishAction.KEEP_POD, "is_delete_operator_pod": False}, + ), + ( + False, + {"is_delete_operator_pod": True}, + {"is_delete_operator_pod": True}, + ), + ( + False, + {"is_delete_operator_pod": False}, + {"is_delete_operator_pod": False}, + ), + ( + # test default + False, + {}, + {"is_delete_operator_pod": False}, + ), + ], + ) + def test_on_finish_action_handler(self, compatible_kpo, kwargs, expected_attributes): + kpo_init_args_mock = mock.MagicMock(**{"parameters": ["on_finish_action"] if compatible_kpo else []}) + + with mock.patch("inspect.signature", return_value=kpo_init_args_mock): + op = EksPodOperator( + task_id="run_pod", + pod_name="run_pod", + cluster_name=CLUSTER_NAME, + image="amazon/aws-cli:latest", + cmds=["sh", "-c", "ls"], + labels={"demo": "hello_world"}, + get_logs=True, + **kwargs, + ) + for expected_attr in expected_attributes: + assert op.__getattribute__(expected_attr) == expected_attributes[expected_attr] diff --git a/tests/providers/cncf/kubernetes/operators/test_pod.py b/tests/providers/cncf/kubernetes/operators/test_pod.py index 40feb3078928..1ce25190ba4c 100644 --- a/tests/providers/cncf/kubernetes/operators/test_pod.py +++ b/tests/providers/cncf/kubernetes/operators/test_pod.py @@ -589,9 +589,22 @@ def test_image_pull_policy_correctly_set(self): pod = k.build_pod_request_obj(create_context(k)) assert pod.spec.containers[0].image_pull_policy == "Always" + @pytest.mark.parametrize( + "task_kwargs, should_be_deleted", + [ + ({}, True), # default values + ({"is_delete_operator_pod": True}, True), # check b/c of is_delete_operator_pod + ({"is_delete_operator_pod": False}, False), # check b/c of is_delete_operator_pod + ({"on_finish_action": "delete_pod"}, True), + ({"on_finish_action": "delete_succeeded_pod"}, False), + ({"on_finish_action": "keep_pod"}, False), + ], + ) @patch(f"{POD_MANAGER_CLASS}.delete_pod") @patch(f"{KPO_MODULE}.KubernetesPodOperator.find_pod") - def test_pod_delete_after_await_container_error(self, find_pod_mock, delete_pod_mock): + def test_pod_delete_after_await_container_error( + self, find_pod_mock, delete_pod_mock, task_kwargs, should_be_deleted + ): """ When KPO fails unexpectedly during await_container, we should still try to delete the pod, and the pod we try to delete should be the one returned from find_pod earlier. @@ -600,13 +613,16 @@ def test_pod_delete_after_await_container_error(self, find_pod_mock, delete_pod_ cont_status.name = "base" cont_status.state.terminated.message = "my-failure" find_pod_mock.return_value.status.container_statuses = [cont_status] - k = KubernetesPodOperator(task_id="task") + k = KubernetesPodOperator(task_id="task", **task_kwargs) self.await_pod_mock.side_effect = AirflowException("fake failure") with pytest.raises(AirflowException, match="my-failure"): context = create_context(k) context["ti"].xcom_push = MagicMock() k.execute(context=context) - delete_pod_mock.assert_called_with(find_pod_mock.return_value) + if should_be_deleted: + delete_pod_mock.assert_called_with(find_pod_mock.return_value) + else: + delete_pod_mock.assert_not_called() @pytest.mark.parametrize("should_fail", [True, False]) @patch(f"{POD_MANAGER_CLASS}.delete_pod") @@ -618,7 +634,7 @@ def test_pod_delete_not_called_when_creation_fails(self, await_pod_mock, delete_ """ k = KubernetesPodOperator( task_id="task", - is_delete_operator_pod=True, + on_finish_action="delete_pod", ) if should_fail: @@ -1019,7 +1035,7 @@ def test_mark_checked_unexpected_exception(self, mock_patch_already_checked, moc """If we aren't deleting pods and have an exception, mark it so we don't reattach to it""" k = KubernetesPodOperator( task_id="task", - is_delete_operator_pod=False, + on_finish_action="keep_pod", ) self.await_pod_mock.side_effect = AirflowException("oops") context = create_context(k) @@ -1045,16 +1061,50 @@ def test_wait_for_xcom_sidecar_iff_push_xcom(self, mock_await, mock_extract_xcom else: mock_await.assert_not_called() - @pytest.mark.parametrize("should_fail", [True, False]) + @pytest.mark.parametrize( + "task_kwargs, should_fail, should_be_deleted", + [ + ({}, False, True), + ({}, True, True), + ( + {"is_delete_operator_pod": True, "on_finish_action": "keep_pod"}, + False, + True, + ), # check backcompat of is_delete_operator_pod + ( + {"is_delete_operator_pod": True, "on_finish_action": "keep_pod"}, + True, + True, + ), # check b/c of is_delete_operator_pod + ( + {"is_delete_operator_pod": False, "on_finish_action": "delete_pod"}, + False, + False, + ), # check b/c of is_delete_operator_pod + ( + {"is_delete_operator_pod": False, "on_finish_action": "delete_pod"}, + True, + False, + ), # check b/c of is_delete_operator_pod + ({"on_finish_action": "keep_pod"}, False, False), + ({"on_finish_action": "keep_pod"}, True, False), + ({"on_finish_action": "delete_pod"}, False, True), + ({"on_finish_action": "delete_pod"}, True, True), + ({"on_finish_action": "delete_succeeded_pod"}, False, True), + ({"on_finish_action": "delete_succeeded_pod"}, True, False), + ], + ) @patch(f"{POD_MANAGER_CLASS}.delete_pod") @patch(f"{KPO_MODULE}.KubernetesPodOperator.patch_already_checked") - def test_mark_checked_if_not_deleted(self, mock_patch_already_checked, mock_delete_pod, should_fail): + def test_mark_checked_if_not_deleted( + self, mock_patch_already_checked, mock_delete_pod, task_kwargs, should_fail, should_be_deleted + ): """If we aren't deleting pods mark "checked" if the task completes (successful or otherwise)""" dag = DAG("hello2", start_date=pendulum.now()) k = KubernetesPodOperator( task_id="task", - is_delete_operator_pod=False, dag=dag, + **task_kwargs, ) remote_pod_mock = MagicMock() remote_pod_mock.status.phase = "Failed" if should_fail else "Succeeded" @@ -1065,8 +1115,14 @@ def test_mark_checked_if_not_deleted(self, mock_patch_already_checked, mock_dele k.execute(context=context) else: k.execute(context=context) - mock_patch_already_checked.assert_called_once() - mock_delete_pod.assert_not_called() + if should_fail or not should_be_deleted: + mock_patch_already_checked.assert_called_once() + else: + mock_patch_already_checked.assert_not_called() + if should_be_deleted: + mock_delete_pod.assert_called_once() + else: + mock_delete_pod.assert_not_called() @patch(HOOK_CLASS, new=MagicMock) def test_patch_already_checked(self): @@ -1141,7 +1197,7 @@ def test_task_skip_when_pod_exit_with_certain_code( ): """Tests that an AirflowSkipException is raised when the container exits with the skip_on_exit_code""" k = KubernetesPodOperator( - task_id="task", is_delete_operator_pod=True, **(extra_kwargs if extra_kwargs else {}) + task_id="task", on_finish_action="delete_pod", **(extra_kwargs if extra_kwargs else {}) ) base_container = MagicMock() @@ -1282,7 +1338,7 @@ def test_async_create_pod_should_execute_successfully(self, mocked_pod, mocked_p arguments=TEST_ARGS, labels=TEST_LABELS, name=TEST_NAME, - is_delete_operator_pod=False, + on_finish_action="keep_pod", in_cluster=True, get_logs=True, deferrable=True, @@ -1306,7 +1362,7 @@ def test_async_create_pod_should_throw_exception(self, mocked_hook, mocked_clean arguments=TEST_ARGS, labels=TEST_LABELS, name=TEST_NAME, - is_delete_operator_pod=False, + on_finish_action="keep_pod", in_cluster=True, get_logs=True, deferrable=True, @@ -1353,7 +1409,7 @@ def test_async_create_pod_with_skip_on_exit_code_should_skip( arguments=TEST_ARGS, labels=TEST_LABELS, name=TEST_NAME, - is_delete_operator_pod=False, + on_finish_action="keep_pod", in_cluster=True, get_logs=True, deferrable=True, diff --git a/tests/providers/cncf/kubernetes/triggers/test_pod.py b/tests/providers/cncf/kubernetes/triggers/test_pod.py index 6d5d18d028c1..4ed731b42578 100644 --- a/tests/providers/cncf/kubernetes/triggers/test_pod.py +++ b/tests/providers/cncf/kubernetes/triggers/test_pod.py @@ -39,12 +39,12 @@ CLUSTER_CONTEXT = "test-context" CONFIG_FILE = "/path/to/config/file" IN_CLUSTER = False -SHOULD_DELETE_POD = True GET_LOGS = True STARTUP_TIMEOUT_SECS = 120 TRIGGER_START_TIME = datetime.now(tz=pytz.UTC) FAILED_RESULT_MSG = "Test message that appears when trigger have failed event." BASE_CONTAINER_NAME = "base" +ON_FINISH_ACTION = "delete_pod" @pytest.fixture @@ -58,10 +58,10 @@ def trigger(): cluster_context=CLUSTER_CONTEXT, config_file=CONFIG_FILE, in_cluster=IN_CLUSTER, - should_delete_pod=SHOULD_DELETE_POD, get_logs=GET_LOGS, startup_timeout=STARTUP_TIMEOUT_SECS, trigger_start_time=TRIGGER_START_TIME, + on_finish_action=ON_FINISH_ACTION, ) @@ -85,10 +85,11 @@ def test_serialize(self, trigger): "cluster_context": CLUSTER_CONTEXT, "config_file": CONFIG_FILE, "in_cluster": IN_CLUSTER, - "should_delete_pod": SHOULD_DELETE_POD, "get_logs": GET_LOGS, "startup_timeout": STARTUP_TIMEOUT_SECS, "trigger_start_time": TRIGGER_START_TIME, + "on_finish_action": ON_FINISH_ACTION, + "should_delete_pod": ON_FINISH_ACTION == "delete_pod", } @pytest.mark.asyncio @@ -237,8 +238,10 @@ async def test_logging_in_trigger_when_fail_should_execute_successfully( @pytest.mark.asyncio @mock.patch(f"{TRIGGER_PATH}._get_async_hook") - async def test_logging_in_trigger_when_cancelled_should_execute_successfully( - self, mock_hook, trigger, caplog + async def test_logging_in_trigger_when_cancelled_should_execute_successfully_and_delete_pod( + self, + mock_hook, + caplog, ): """ Test that KubernetesPodTrigger fires the correct event in case if the task was cancelled. @@ -248,6 +251,21 @@ async def test_logging_in_trigger_when_cancelled_should_execute_successfully( mock_hook.return_value.read_logs.return_value = self._mock_pod_result(mock.MagicMock()) mock_hook.return_value.delete_pod.return_value = self._mock_pod_result(mock.MagicMock()) + trigger = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_file=CONFIG_FILE, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action="delete_pod", + ) + generator = trigger.run() actual = await generator.asend(None) assert ( @@ -264,6 +282,52 @@ async def test_logging_in_trigger_when_cancelled_should_execute_successfully( assert "Outputting container logs..." in caplog.text assert "Deleting pod..." in caplog.text + @pytest.mark.asyncio + @mock.patch(f"{TRIGGER_PATH}._get_async_hook") + async def test_logging_in_trigger_when_cancelled_should_execute_successfully_without_delete_pod( + self, + mock_hook, + caplog, + ): + """ + Test that KubernetesPodTrigger fires the correct event if the task was cancelled. + """ + + mock_hook.return_value.get_pod.side_effect = CancelledError() + mock_hook.return_value.read_logs.return_value = self._mock_pod_result(mock.MagicMock()) + mock_hook.return_value.delete_pod.return_value = self._mock_pod_result(mock.MagicMock()) + + trigger = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_file=CONFIG_FILE, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action="delete_succeeded_pod", + ) + + generator = trigger.run() + actual = await generator.asend(None) + assert ( + TriggerEvent( + { + "name": POD_NAME, + "namespace": NAMESPACE, + "status": "cancelled", + "message": "Pod execution was cancelled", + } + ) + == actual + ) + assert "Outputting container logs..." in caplog.text + assert "Deleting pod..." not in caplog.text + @pytest.mark.parametrize( "container_state, expected_state", [ diff --git a/tests/providers/google/cloud/operators/test_kubernetes_engine.py b/tests/providers/google/cloud/operators/test_kubernetes_engine.py index 5d6acb196ad2..07f0c34c7a14 100644 --- a/tests/providers/google/cloud/operators/test_kubernetes_engine.py +++ b/tests/providers/google/cloud/operators/test_kubernetes_engine.py @@ -26,6 +26,7 @@ from airflow.exceptions import AirflowException, TaskDeferred from airflow.models import Connection from airflow.providers.cncf.kubernetes.operators.pod import KubernetesPodOperator +from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction from airflow.providers.google.cloud.operators.kubernetes_engine import ( GKECreateClusterOperator, GKEDeleteClusterOperator, @@ -319,6 +320,66 @@ def test_cluster_info(self, get_cluster_mock, use_internal_ip): assert cluster_url == CLUSTER_PRIVATE_URL if use_internal_ip else CLUSTER_URL assert ssl_ca_cert == SSL_CA_CERT + @pytest.mark.parametrize( + "compatible_kpo, kwargs, expected_attributes", + [ + ( + True, + {"on_finish_action": "delete_succeeded_pod"}, + {"on_finish_action": OnFinishAction.DELETE_SUCCEEDED_POD}, + ), + ( + # test that priority for deprecated param + True, + {"on_finish_action": "keep_pod", "is_delete_operator_pod": True}, + {"on_finish_action": OnFinishAction.DELETE_POD, "is_delete_operator_pod": True}, + ), + ( + # test default + True, + {}, + {"on_finish_action": OnFinishAction.KEEP_POD, "is_delete_operator_pod": False}, + ), + ( + False, + {"is_delete_operator_pod": True}, + {"is_delete_operator_pod": True}, + ), + ( + False, + {"is_delete_operator_pod": False}, + {"is_delete_operator_pod": False}, + ), + ( + # test default + False, + {}, + {"is_delete_operator_pod": False}, + ), + ], + ) + def test_on_finish_action_handler( + self, + compatible_kpo, + kwargs, + expected_attributes, + ): + kpo_init_args_mock = mock.MagicMock(**{"parameters": ["on_finish_action"] if compatible_kpo else []}) + + with mock.patch("inspect.signature", return_value=kpo_init_args_mock): + op = GKEStartPodOperator( + project_id=TEST_GCP_PROJECT_ID, + location=PROJECT_LOCATION, + cluster_name=CLUSTER_NAME, + task_id=PROJECT_TASK_ID, + name=TASK_NAME, + namespace=NAMESPACE, + image=IMAGE, + **kwargs, + ) + for expected_attr in expected_attributes: + assert op.__getattribute__(expected_attr) == expected_attributes[expected_attr] + class TestGKEPodOperatorAsync: def setup_method(self): diff --git a/tests/providers/google/cloud/triggers/test_kubernetes_engine.py b/tests/providers/google/cloud/triggers/test_kubernetes_engine.py index e957767e3eea..154908a6c4b9 100644 --- a/tests/providers/google/cloud/triggers/test_kubernetes_engine.py +++ b/tests/providers/google/cloud/triggers/test_kubernetes_engine.py @@ -48,6 +48,7 @@ SSL_CA_CERT = "TEST_SSL_CA_CERT_CONTENT" FAILED_RESULT_MSG = "Test message that appears when trigger have failed event." BASE_CONTAINER_NAME = "base" +ON_FINISH_ACTION = "delete_pod" OPERATION_NAME = "test-operation-name" PROJECT_ID = "test-project-id" @@ -93,13 +94,14 @@ def test_serialize_should_execute_successfully(self, trigger): "poll_interval": POLL_INTERVAL, "cluster_context": CLUSTER_CONTEXT, "in_cluster": IN_CLUSTER, - "should_delete_pod": SHOULD_DELETE_POD, "get_logs": GET_LOGS, "startup_timeout": STARTUP_TIMEOUT_SECS, "trigger_start_time": TRIGGER_START_TIME, "cluster_url": CLUSTER_URL, "ssl_ca_cert": SSL_CA_CERT, "base_container_name": BASE_CONTAINER_NAME, + "on_finish_action": ON_FINISH_ACTION, + "should_delete_pod": SHOULD_DELETE_POD, } @pytest.mark.asyncio diff --git a/tests/system/providers/amazon/aws/example_eks_templated.py b/tests/system/providers/amazon/aws/example_eks_templated.py index d09eabf95982..e51783b53e76 100644 --- a/tests/system/providers/amazon/aws/example_eks_templated.py +++ b/tests/system/providers/amazon/aws/example_eks_templated.py @@ -107,7 +107,7 @@ labels={"demo": "hello_world"}, get_logs=True, # Delete the pod when it reaches its final state, or the execution is interrupted. - is_delete_operator_pod=True, + on_finish_action="delete_pod", ) delete_nodegroup = EksDeleteNodegroupOperator( diff --git a/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py b/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py index d14e7a6885a4..ae67a26588bd 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py +++ b/tests/system/providers/amazon/aws/example_eks_with_fargate_in_one_step.py @@ -101,7 +101,7 @@ get_logs=True, startup_timeout_seconds=600, # Keep the pod alive, so we can describe it in case of trouble. It's deleted with the cluster anyway. - is_delete_operator_pod=False, + on_finish_action="keep_pod", ) describe_pod = get_describe_pod_operator( diff --git a/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py b/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py index 0c19fda2d518..9e19c7594ffb 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py +++ b/tests/system/providers/amazon/aws/example_eks_with_fargate_profile.py @@ -119,7 +119,7 @@ labels={"demo": "hello_world"}, get_logs=True, # Keep the pod alive, so we can describe it in case of trouble. It's deleted with the cluster anyway. - is_delete_operator_pod=False, + on_finish_action="keep_pod", startup_timeout_seconds=200, ) diff --git a/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py b/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py index 76aa01bf5c96..9bfce2cc1696 100644 --- a/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py +++ b/tests/system/providers/amazon/aws/example_eks_with_nodegroup_in_one_step.py @@ -111,7 +111,7 @@ def delete_launch_template(template_name: str): labels={"demo": "hello_world"}, get_logs=True, # Keep the pod alive, so we can describe it in case of trouble. It's deleted with the cluster anyway. - is_delete_operator_pod=False, + on_finish_action="keep_pod", ) describe_pod = get_describe_pod_operator( diff --git a/tests/system/providers/cncf/kubernetes/example_kubernetes.py b/tests/system/providers/cncf/kubernetes/example_kubernetes.py index 2f1f791d7882..74078541bdcf 100644 --- a/tests/system/providers/cncf/kubernetes/example_kubernetes.py +++ b/tests/system/providers/cncf/kubernetes/example_kubernetes.py @@ -122,7 +122,7 @@ name="airflow-test-pod", task_id="task", affinity=affinity, - is_delete_operator_pod=True, + on_finish_action="delete_pod", hostnetwork=False, tolerations=tolerations, init_containers=[init_container], @@ -138,7 +138,7 @@ arguments=["echo", "10", "echo pwd"], labels={"foo": "bar"}, name="airflow-private-image-pod", - is_delete_operator_pod=True, + on_finish_action="delete_pod", in_cluster=True, task_id="task-two", get_logs=True, @@ -152,7 +152,7 @@ cmds=["sh", "-c", "mkdir -p /airflow/xcom/;echo '[1,2,3,4]' > /airflow/xcom/return.json"], name="write-xcom", do_xcom_push=True, - is_delete_operator_pod=True, + on_finish_action="delete_pod", in_cluster=True, task_id="write-xcom", get_logs=True, diff --git a/tests/system/providers/cncf/kubernetes/example_kubernetes_async.py b/tests/system/providers/cncf/kubernetes/example_kubernetes_async.py index f44e8637ffc4..36e143c096b9 100644 --- a/tests/system/providers/cncf/kubernetes/example_kubernetes_async.py +++ b/tests/system/providers/cncf/kubernetes/example_kubernetes_async.py @@ -123,7 +123,7 @@ env_from=configmaps, name="airflow-test-pod", affinity=affinity, - is_delete_operator_pod=True, + on_finish_action="delete_pod", hostnetwork=False, tolerations=tolerations, init_containers=[init_container], @@ -141,7 +141,7 @@ arguments=["echo", "10", "echo pwd"], labels={"foo": "bar"}, name="airflow-private-image-pod", - is_delete_operator_pod=True, + on_finish_action="delete_pod", in_cluster=True, get_logs=True, deferrable=True, @@ -156,7 +156,7 @@ cmds=["sh", "-c", "mkdir -p /airflow/xcom/;echo '[1,2,3,4]' > /airflow/xcom/return.json"], name="write-xcom", do_xcom_push=True, - is_delete_operator_pod=True, + on_finish_action="delete_pod", in_cluster=True, get_logs=True, deferrable=True, diff --git a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py index d2a6161ed8da..bf155499f294 100644 --- a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py +++ b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine.py @@ -67,7 +67,7 @@ image="perl", name="test-pod", in_cluster=False, - is_delete_operator_pod=True, + on_finish_action="delete_pod", ) # [START howto_operator_gke_start_pod_xcom] @@ -82,7 +82,7 @@ cmds=["sh", "-c", "mkdir -p /airflow/xcom/;echo '[1,2,3,4]' > /airflow/xcom/return.json"], name="test-pod-xcom", in_cluster=False, - is_delete_operator_pod=True, + on_finish_action="delete_pod", ) # [END howto_operator_gke_start_pod_xcom] diff --git a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py index 13c310d88021..73430a39385a 100644 --- a/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py +++ b/tests/system/providers/google/cloud/kubernetes_engine/example_kubernetes_engine_async.py @@ -66,7 +66,7 @@ image="perl", name="test-pod-async", in_cluster=False, - is_delete_operator_pod=True, + on_finish_action="delete_pod", get_logs=True, deferrable=True, ) @@ -82,7 +82,7 @@ cmds=["sh", "-c", "mkdir -p /airflow/xcom/;echo '[1,2,3,4]' > /airflow/xcom/return.json"], name="test-pod-xcom-async", in_cluster=False, - is_delete_operator_pod=True, + on_finish_action="delete_pod", do_xcom_push=True, deferrable=True, get_logs=True, From 7e06c80702a16db466e6321f7a46cd94c19058c8 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 30 Jun 2023 10:49:40 +0200 Subject: [PATCH 003/533] Add licence preamble also in hidden files (#32276) Licence preamble adding has beeen skipped for markdown files in hidden/.github folders. Technically the licence is not needed there as we do not distribute those files in sources, but it does not hurt to have it for consistency --- .github/PULL_REQUEST_TEMPLATE.md | 19 +++++++++++++++++++ .pre-commit-config.yaml | 2 +- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 021313b68eb7..011b4caa5e71 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -1,3 +1,22 @@ + + " From d6e254db689db070f2f181006e7d6bc593482300 Mon Sep 17 00:00:00 2001 From: Shahar Epstein <60007259+shahar1@users.noreply.github.com> Date: Fri, 30 Jun 2023 11:52:14 +0300 Subject: [PATCH 004/533] Deprecate `delimiter` param and source object's wildcards in GCS, introduce `match_glob` param. (#31261) * Deprecate `delimiter` param and source object's wildcards in GCS, introduce `match_glob` param. --------- Co-authored-by: eladkal <45845474+eladkal@users.noreply.github.com> --- .../amazon/aws/transfers/gcs_to_s3.py | 18 +- airflow/providers/google/cloud/hooks/gcs.py | 126 +++++++++-- .../providers/google/cloud/operators/gcs.py | 48 +++-- .../google/cloud/transfers/gcs_to_gcs.py | 53 ++++- .../google/cloud/transfers/gcs_to_sftp.py | 5 +- .../google/suite/transfers/gcs_to_gdrive.py | 4 + .../operators/transfer/gcs_to_gcs.rst | 15 +- .../amazon/aws/transfers/test_gcs_to_s3.py | 195 +++++++++++------- .../providers/google/cloud/hooks/test_gcs.py | 60 +++++- .../google/cloud/operators/test_gcs.py | 16 +- .../google/cloud/transfers/test_gcs_to_gcs.py | 26 ++- .../cloud/transfers/test_gcs_to_sftp.py | 2 + .../suite/transfers/test_gcs_to_gdrive.py | 6 + .../google/cloud/gcs/example_gcs_to_gcs.py | 12 ++ 14 files changed, 448 insertions(+), 138 deletions(-) diff --git a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py index 4004eea8c2c2..2213de2b60c1 100644 --- a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py @@ -19,8 +19,10 @@ from __future__ import annotations import os +import warnings from typing import TYPE_CHECKING, Sequence +from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.google.cloud.hooks.gcs import GCSHook @@ -40,7 +42,7 @@ class GCSToS3Operator(BaseOperator): :param bucket: The Google Cloud Storage bucket to find the objects. (templated) :param prefix: Prefix string which filters objects whose name begin with this prefix. (templated) - :param delimiter: The delimiter by which you want to filter the objects. (templated) + :param delimiter: (Deprecated) The delimiter by which you want to filter the objects. (templated) For e.g to lists the CSV files from in a directory in GCS you would use delimiter='.csv'. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. @@ -76,6 +78,8 @@ class GCSToS3Operator(BaseOperator): object to be uploaded in S3 :param keep_directory_structure: (Optional) When set to False the path of the file on the bucket is recreated within path passed in dest_s3_key. + :param match_glob: (Optional) filters objects based on the glob pattern given by the string + (e.g, ``'**/*/.json'``) """ template_fields: Sequence[str] = ( @@ -102,12 +106,19 @@ def __init__( dest_s3_extra_args: dict | None = None, s3_acl_policy: str | None = None, keep_directory_structure: bool = True, + match_glob: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.bucket = bucket self.prefix = prefix + if delimiter: + warnings.warn( + "Usage of 'delimiter' is deprecated, please use 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) self.delimiter = delimiter self.gcp_conn_id = gcp_conn_id self.dest_aws_conn_id = dest_aws_conn_id @@ -118,6 +129,7 @@ def __init__( self.dest_s3_extra_args = dest_s3_extra_args or {} self.s3_acl_policy = s3_acl_policy self.keep_directory_structure = keep_directory_structure + self.match_glob = match_glob def execute(self, context: Context) -> list[str]: # list all files in an Google Cloud Storage bucket @@ -133,7 +145,9 @@ def execute(self, context: Context) -> list[str]: self.prefix, ) - files = hook.list(bucket_name=self.bucket, prefix=self.prefix, delimiter=self.delimiter) + files = hook.list( + bucket_name=self.bucket, prefix=self.prefix, delimiter=self.delimiter, match_glob=self.match_glob + ) s3_hook = S3Hook( aws_conn_id=self.dest_aws_conn_id, verify=self.dest_verify, extra_args=self.dest_s3_extra_args diff --git a/airflow/providers/google/cloud/hooks/gcs.py b/airflow/providers/google/cloud/hooks/gcs.py index b17af1d9b917..42a27e80f898 100644 --- a/airflow/providers/google/cloud/hooks/gcs.py +++ b/airflow/providers/google/cloud/hooks/gcs.py @@ -24,6 +24,7 @@ import os import shutil import time +import warnings from contextlib import contextmanager from datetime import datetime from functools import partial @@ -44,7 +45,7 @@ from google.cloud.storage.retry import DEFAULT_RETRY from requests import Session -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.utils.helpers import normalize_directory_path from airflow.providers.google.common.consts import CLIENT_INFO from airflow.providers.google.common.hooks.base_google import GoogleBaseAsyncHook, GoogleBaseHook @@ -709,6 +710,7 @@ def list( max_results: int | None = None, prefix: str | List[str] | None = None, delimiter: str | None = None, + match_glob: str | None = None, ): """ List all objects from the bucket with the given a single prefix or multiple prefixes. @@ -717,9 +719,19 @@ def list( :param versions: if true, list all versions of the objects :param max_results: max count of items to return in a single page of responses :param prefix: string or list of strings which filter objects whose name begin with it/them - :param delimiter: filters objects based on the delimiter (for e.g '.csv') + :param delimiter: (Deprecated) filters objects based on the delimiter (for e.g '.csv') + :param match_glob: (Optional) filters objects based on the glob pattern given by the string + (e.g, ``'**/*/.json'``). :return: a stream of object names matching the filtering criteria """ + if delimiter and delimiter != "/": + warnings.warn( + "Usage of 'delimiter' param is deprecated, please use 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) + if match_glob and delimiter and delimiter != "/": + raise AirflowException("'match_glob' param cannot be used with 'delimiter' that differs than '/'") objects = [] if isinstance(prefix, list): for prefix_item in prefix: @@ -730,6 +742,7 @@ def list( max_results=max_results, prefix=prefix_item, delimiter=delimiter, + match_glob=match_glob, ) ) else: @@ -740,6 +753,7 @@ def list( max_results=max_results, prefix=prefix, delimiter=delimiter, + match_glob=match_glob, ) ) return objects @@ -751,6 +765,7 @@ def _list( max_results: int | None = None, prefix: str | None = None, delimiter: str | None = None, + match_glob: str | None = None, ) -> List: """ List all objects from the bucket with the give string prefix in name. @@ -759,7 +774,9 @@ def _list( :param versions: if true, list all versions of the objects :param max_results: max count of items to return in a single page of responses :param prefix: string which filters objects whose name begin with it - :param delimiter: filters objects based on the delimiter (for e.g '.csv') + :param delimiter: (Deprecated) filters objects based on the delimiter (for e.g '.csv') + :param match_glob: (Optional) filters objects based on the glob pattern given by the string + (e.g, ``'**/*/.json'``). :return: a stream of object names matching the filtering criteria """ client = self.get_conn() @@ -768,13 +785,25 @@ def _list( ids = [] page_token = None while True: - blobs = bucket.list_blobs( - max_results=max_results, - page_token=page_token, - prefix=prefix, - delimiter=delimiter, - versions=versions, - ) + if match_glob: + blobs = self._list_blobs_with_match_glob( + bucket=bucket, + client=client, + match_glob=match_glob, + max_results=max_results, + page_token=page_token, + path=bucket.path + "/o", + prefix=prefix, + versions=versions, + ) + else: + blobs = bucket.list_blobs( + max_results=max_results, + page_token=page_token, + prefix=prefix, + delimiter=delimiter, + versions=versions, + ) blob_names = [] for blob in blobs: @@ -792,6 +821,52 @@ def _list( break return ids + @staticmethod + def _list_blobs_with_match_glob( + bucket, + client, + path: str, + max_results: int | None = None, + page_token: str | None = None, + match_glob: str | None = None, + prefix: str | None = None, + versions: bool | None = None, + ) -> Any: + """ + List blobs when match_glob param is given. + This method is a patched version of google.cloud.storage Client.list_blobs(). + It is used as a temporary workaround to support "match_glob" param, + as it isn't officially supported by GCS Python client. + (follow `issue #1035`__). + """ + from google.api_core import page_iterator + from google.cloud.storage.bucket import _blobs_page_start, _item_to_blob + + extra_params: Any = {} + if prefix is not None: + extra_params["prefix"] = prefix + if match_glob is not None: + extra_params["matchGlob"] = match_glob + if versions is not None: + extra_params["versions"] = versions + api_request = functools.partial( + client._connection.api_request, timeout=DEFAULT_TIMEOUT, retry=DEFAULT_RETRY + ) + + blobs: Any = page_iterator.HTTPIterator( + client=client, + api_request=api_request, + path=path, + item_to_value=_item_to_blob, + page_token=page_token, + max_results=max_results, + extra_params=extra_params, + page_start=_blobs_page_start, + ) + blobs.prefixes = set() + blobs.bucket = bucket + return blobs + def list_by_timespan( self, bucket_name: str, @@ -801,6 +876,7 @@ def list_by_timespan( max_results: int | None = None, prefix: str | None = None, delimiter: str | None = None, + match_glob: str | None = None, ) -> List[str]: """ List all objects from the bucket with the give string prefix in name that were @@ -813,7 +889,9 @@ def list_by_timespan( :param max_results: max count of items to return in a single page of responses :param prefix: prefix string which filters objects whose name begin with this prefix - :param delimiter: filters objects based on the delimiter (for e.g '.csv') + :param delimiter: (Deprecated) filters objects based on the delimiter (for e.g '.csv') + :param match_glob: (Optional) filters objects based on the glob pattern given by the string + (e.g, ``'**/*/.json'``). :return: a stream of object names matching the filtering criteria """ client = self.get_conn() @@ -823,13 +901,25 @@ def list_by_timespan( page_token = None while True: - blobs = bucket.list_blobs( - max_results=max_results, - page_token=page_token, - prefix=prefix, - delimiter=delimiter, - versions=versions, - ) + if match_glob: + blobs = self._list_blobs_with_match_glob( + bucket=bucket, + client=client, + match_glob=match_glob, + max_results=max_results, + page_token=page_token, + path=bucket.path + "/o", + prefix=prefix, + versions=versions, + ) + else: + blobs = bucket.list_blobs( + max_results=max_results, + page_token=page_token, + prefix=prefix, + delimiter=delimiter, + versions=versions, + ) blob_names = [] for blob in blobs: diff --git a/airflow/providers/google/cloud/operators/gcs.py b/airflow/providers/google/cloud/operators/gcs.py index e2ac68c90d65..6e50cbad2d16 100644 --- a/airflow/providers/google/cloud/operators/gcs.py +++ b/airflow/providers/google/cloud/operators/gcs.py @@ -21,6 +21,7 @@ import datetime import subprocess import sys +import warnings from pathlib import Path from tempfile import NamedTemporaryFile, TemporaryDirectory from typing import TYPE_CHECKING, Sequence @@ -33,7 +34,7 @@ from google.api_core.exceptions import Conflict from google.cloud.exceptions import GoogleCloudError -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator from airflow.providers.google.common.links.storage import FileDetailsLink, StorageLink @@ -157,16 +158,17 @@ def execute(self, context: Context) -> None: class GCSListObjectsOperator(GoogleCloudBaseOperator): """ - List all objects from the bucket with the given string prefix and delimiter in name. + List all objects from the bucket filtered by given string prefix and delimiter in name, + or match_glob. This operator returns a python list with the name of objects which can be used by XCom in the downstream task. :param bucket: The Google Cloud Storage bucket to find the objects. (templated) - :param prefix: String or list of strings, which filter objects whose name begin with + :param prefix: String or list of strings, which filter objects whose name begins with it/them. (templated) - :param delimiter: The delimiter by which you want to filter the objects. (templated) - For example, to lists the CSV files from in a directory in GCS you would use + :param delimiter: (Deprecated) The delimiter by which you want to filter the objects. (templated) + For example, to list the CSV files from in a directory in GCS you would use delimiter='.csv'. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param impersonation_chain: Optional service account to impersonate using short-term @@ -177,6 +179,8 @@ class GCSListObjectsOperator(GoogleCloudBaseOperator): If set as a sequence, the identities from the list must grant Service Account Token Creator IAM role to the directly preceding identity, with first account from the list granting this role to the originating account (templated). + :param match_glob: (Optional) filters objects based on the glob pattern given by the string + (e.g, ``'**/*/.json'``) **Example**: The following Operator would list all the Avro files from ``sales/sales-2017`` @@ -186,7 +190,7 @@ class GCSListObjectsOperator(GoogleCloudBaseOperator): task_id='GCS_Files', bucket='data', prefix='sales/sales-2017/', - delimiter='.avro', + match_glob='**/*/.avro', gcp_conn_id=google_cloud_conn_id ) """ @@ -210,14 +214,22 @@ def __init__( delimiter: str | None = None, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, + match_glob: str | None = None, **kwargs, ) -> None: super().__init__(**kwargs) self.bucket = bucket self.prefix = prefix + if delimiter: + warnings.warn( + "Usage of 'delimiter' is deprecated, please use 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) self.delimiter = delimiter self.gcp_conn_id = gcp_conn_id self.impersonation_chain = impersonation_chain + self.match_glob = match_glob def execute(self, context: Context) -> list: hook = GCSHook( @@ -225,12 +237,20 @@ def execute(self, context: Context) -> list: impersonation_chain=self.impersonation_chain, ) - self.log.info( - "Getting list of the files. Bucket: %s; Delimiter: %s; Prefix(es): %s", - self.bucket, - self.delimiter, - self.prefix, - ) + if self.match_glob: + self.log.info( + "Getting list of the files. Bucket: %s; MatchGlob: %s; Prefix(es): %s", + self.bucket, + self.match_glob, + self.prefix, + ) + else: + self.log.info( + "Getting list of the files. Bucket: %s; Delimiter: %s; Prefix(es): %s", + self.bucket, + self.delimiter, + self.prefix, + ) StorageLink.persist( context=context, @@ -238,7 +258,9 @@ def execute(self, context: Context) -> list: uri=self.bucket, project_id=hook.project_id, ) - return hook.list(bucket_name=self.bucket, prefix=self.prefix, delimiter=self.delimiter) + return hook.list( + bucket_name=self.bucket, prefix=self.prefix, delimiter=self.delimiter, match_glob=self.match_glob + ) class GCSDeleteObjectsOperator(GoogleCloudBaseOperator): diff --git a/airflow/providers/google/cloud/transfers/gcs_to_gcs.py b/airflow/providers/google/cloud/transfers/gcs_to_gcs.py index c8f811c6e7d8..2b39df1c6a79 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_gcs.py @@ -18,9 +18,10 @@ """This module contains a Google Cloud Storage operator.""" from __future__ import annotations +import warnings from typing import TYPE_CHECKING, Sequence -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.google.cloud.hooks.gcs import GCSHook @@ -66,8 +67,8 @@ class GCSToGCSOperator(BaseOperator): of copied to the new location. This is the equivalent of a mv command as opposed to a cp command. :param replace: Whether you want to replace existing destination files or not. - :param delimiter: This is used to restrict the result to only the 'files' in a given 'folder'. - If source_objects = ['foo/bah/'] and delimiter = '.avro', then only the 'files' in the + :param delimiter: (Deprecated) This is used to restrict the result to only the 'files' in a given + 'folder'. If source_objects = ['foo/bah/'] and delimiter = '.avro', then only the 'files' in the folder 'foo/bah/' with '.avro' delimiter will be copied to the destination object. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param last_modified_time: When specified, the objects will be copied or moved, @@ -90,6 +91,8 @@ class GCSToGCSOperator(BaseOperator): doesn't exist. It doesn't have any effect when the source objects are folders or patterns. :param exact_match: When specified, only exact match of the source object (filename) will be copied. + :param match_glob: (Optional) filters objects based on the glob pattern given by the string ( + e.g, ``'**/*/.json'``) :Example: @@ -116,7 +119,7 @@ class GCSToGCSOperator(BaseOperator): source_objects=['sales/sales-2017'], destination_bucket='data_backup', destination_object='copied_sales/2017/', - delimiter='.avro' + match_glob='**/*.avro' gcp_conn_id=google_cloud_conn_id ) @@ -190,15 +193,34 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, source_object_required=False, exact_match=False, + match_glob: str | None = None, **kwargs, ): super().__init__(**kwargs) self.source_bucket = source_bucket + if source_object and WILDCARD in source_object: + warnings.warn( + "Usage of wildcard (*) in 'source_object' is deprecated, utilize 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) self.source_object = source_object + if source_objects and any([WILDCARD in obj for obj in source_objects]): + warnings.warn( + "Usage of wildcard (*) in 'source_objects' is deprecated, utilize 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) self.source_objects = source_objects self.destination_bucket = destination_bucket self.destination_object = destination_object + if delimiter: + warnings.warn( + "Usage of 'delimiter' is deprecated, please use 'match_glob' instead", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) self.delimiter = delimiter self.move_object = move_object self.replace = replace @@ -209,6 +231,7 @@ def __init__( self.impersonation_chain = impersonation_chain self.source_object_required = source_object_required self.exact_match = exact_match + self.match_glob = match_glob def execute(self, context: Context): @@ -251,6 +274,7 @@ def execute(self, context: Context): for prefix in self.source_objects: # Check if prefix contains wildcard if WILDCARD in prefix: + self._copy_source_with_wildcard(hook=hook, prefix=prefix) # Now search with prefix using provided delimiter if any else: @@ -261,15 +285,19 @@ def _ignore_existing_files(self, hook, prefix, **kwargs): # and only keep those files which are present in # Source GCS bucket and not in Destination GCS bucket delimiter = kwargs.get("delimiter") + match_glob = kwargs.get("match_glob") objects = kwargs.get("objects") if self.destination_object is None: - existing_objects = hook.list(self.destination_bucket, prefix=prefix, delimiter=delimiter) + existing_objects = hook.list( + self.destination_bucket, prefix=prefix, delimiter=delimiter, match_glob=match_glob + ) else: self.log.info("Replaced destination_object with source_object prefix.") destination_objects = hook.list( self.destination_bucket, prefix=self.destination_object, delimiter=delimiter, + match_glob=match_glob, ) existing_objects = [ dest_object.replace(self.destination_object, prefix, 1) for dest_object in destination_objects @@ -338,11 +366,15 @@ def _copy_source_without_wildcard(self, hook, prefix): gcp_conn_id=google_cloud_conn_id ) """ - objects = hook.list(self.source_bucket, prefix=prefix, delimiter=self.delimiter) + objects = hook.list( + self.source_bucket, prefix=prefix, delimiter=self.delimiter, match_glob=self.match_glob + ) if not self.replace: # If we are not replacing, ignore files already existing in source buckets - objects = self._ignore_existing_files(hook, prefix, objects=objects, delimiter=self.delimiter) + objects = self._ignore_existing_files( + hook, prefix, objects=objects, delimiter=self.delimiter, match_glob=self.match_glob + ) # If objects is empty, and we have prefix, let's check if prefix is a blob # and copy directly @@ -397,11 +429,18 @@ def _copy_source_with_wildcard(self, hook, prefix): self.log.info("Delimiter ignored because wildcard is in prefix") prefix_, delimiter = prefix.split(WILDCARD, 1) objects = hook.list(self.source_bucket, prefix=prefix_, delimiter=delimiter) + # TODO: After deprecating delimiter and wildcards in source objects, + # remove previous line and uncomment the following: + # match_glob = f"**/*{delimiter}" if delimiter else None + # objects = hook.list(self.source_bucket, prefix=prefix_, match_glob=match_glob) if not self.replace: # If we are not replacing, list all files in the Destination GCS bucket # and only keep those files which are present in # Source GCS bucket and not in Destination GCS bucket objects = self._ignore_existing_files(hook, prefix_, delimiter=delimiter, objects=objects) + # TODO: After deprecating delimiter and wildcards in source objects, + # remove previous line and uncomment the following: + # objects = self._ignore_existing_files(hook, prefix_, match_glob=match_glob, objects=objects) for source_object in objects: if self.destination_object is None: diff --git a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py index f4942311da9d..150c801861ee 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_sftp.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_sftp.py @@ -145,8 +145,11 @@ def execute(self, context: Context): prefix, delimiter = self.source_object.split(WILDCARD, 1) prefix_dirname = os.path.dirname(prefix) - objects = gcs_hook.list(self.source_bucket, prefix=prefix, delimiter=delimiter) + # TODO: After deprecating delimiter and wildcards in source objects, + # remove the previous line and uncomment the following: + # match_glob = f"**/*{delimiter}" if delimiter else None + # objects = gcs_hook.list(self.source_bucket, prefix=prefix, match_glob=match_glob) for source_object in objects: destination_path = self._resolve_destination_path(source_object, prefix=prefix_dirname) diff --git a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py index 7f8568688c7e..c1e796258f53 100644 --- a/airflow/providers/google/suite/transfers/gcs_to_gdrive.py +++ b/airflow/providers/google/suite/transfers/gcs_to_gdrive.py @@ -132,6 +132,10 @@ def execute(self, context: Context): prefix, delimiter = self.source_object.split(WILDCARD, 1) objects = self.gcs_hook.list(self.source_bucket, prefix=prefix, delimiter=delimiter) + # TODO: After deprecating delimiter and wildcards in source objects, + # remove the previous line and uncomment the following: + # match_glob = f"**/*{delimiter}" if delimiter else None + # objects = self.gcs_hook.list(self.source_bucket, prefix=prefix, match_glob=match_glob) for source_object in objects: if self.destination_object is None: diff --git a/docs/apache-airflow-providers-google/operators/transfer/gcs_to_gcs.rst b/docs/apache-airflow-providers-google/operators/transfer/gcs_to_gcs.rst index b3fa2ac19115..ef805355c4c9 100644 --- a/docs/apache-airflow-providers-google/operators/transfer/gcs_to_gcs.rst +++ b/docs/apache-airflow-providers-google/operators/transfer/gcs_to_gcs.rst @@ -83,6 +83,10 @@ When you use this operator, you can specify whether objects should be deleted fr they are transferred to the sink. Source objects can be specified using a single wildcard, as well as based on the file modification date. +Filtering objects according to their path could be done by using the `match_glob field `__. +You should avoid using the ``delimiter`` field nor a wildcard in the path of the source object(s), as both practices are deprecated. +Additionally, filtering could be achieved based on the file's creation date (``is_older_than``) or modification date (``last_modified_time`` and ``maximum_modified_time``). + The way this operator works by default can be compared to the ``cp`` command. When the file move option is active, this operator functions like the ``mv`` command. @@ -124,6 +128,15 @@ folder in ``BUCKET_1_DST``, with file names unchanged. For source_objects with no wildcard, all files in source_objects would be listed, using provided delimiter if any. Then copy files from source_objects to destination_object and rename each source file. +As previously stated, the ``delimiter`` field, as well as utilizing a wildcard (``*``) in the source object(s), +are both deprecated. Thus, it is not recommended to use them - but to utilize ``match_glob`` instead, as follows: + +.. exampleinclude:: /../../tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py + :language: python + :dedent: 4 + :start-after: [START howto_operator_gcs_to_gcs_match_glob] + :end-before: [END howto_operator_gcs_to_gcs_match_glob] + The following example would copy all the files in ``subdir/`` folder (i.e subdir/a.csv, subdir/b.csv, subdir/c.csv) from the ``BUCKET_1_SRC`` GCS bucket to the ``backup/`` folder in ``BUCKET_1_DST`` bucket. (i.e backup/a.csv, backup/b.csv, backup/c.csv) @@ -133,7 +146,7 @@ the ``BUCKET_1_SRC`` GCS bucket to the ``backup/`` folder in ``BUCKET_1_DST`` bu :start-after: [START howto_operator_gcs_to_gcs_without_wildcard] :end-before: [END howto_operator_gcs_to_gcs_without_wildcard] -The delimiter filed may be specified to select any source files starting with ``source_object`` and ending with the +The delimiter field may be specified to select any source files starting with ``source_object`` and ending with the value supplied to ``delimiter``. This example uses the ``delimiter`` value to implement the same functionality as the prior example. diff --git a/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py b/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py index dce115551b6e..a7a0b2e4305a 100644 --- a/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py +++ b/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py @@ -20,6 +20,7 @@ from tempfile import NamedTemporaryFile from unittest import mock +import pytest from moto import mock_s3 from airflow.providers.amazon.aws.hooks.s3 import S3Hook @@ -47,9 +48,9 @@ def _create_test_bucket(): @mock_s3 class TestGCSToS3Operator: - # Test1: incremental behaviour (just some files missing) + # Test0: match_glob @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") - def test_execute_incremental(self, mock_hook): + def test_execute__match_glob(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES with NamedTemporaryFile() as f: gcs_provide_file = mock_hook.return_value.provide_file @@ -59,14 +60,40 @@ def test_execute_incremental(self, mock_hook): task_id=TASK_ID, bucket=GCS_BUCKET, prefix=PREFIX, - delimiter=DELIMITER, dest_aws_conn_id="aws_default", dest_s3_key=S3_BUCKET, replace=False, + match_glob=f"**/*{DELIMITER}", ) hook, bucket = _create_test_bucket() bucket.put_object(Key=MOCK_FILES[0], Body=b"testing") + operator.execute(None) + mock_hook.return_value.list.assert_called_once_with( + bucket_name=GCS_BUCKET, delimiter=None, match_glob=f"**/*{DELIMITER}", prefix=PREFIX + ) + + # Test1: incremental behaviour (just some files missing) + @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") + def test_execute_incremental(self, mock_hook): + mock_hook.return_value.list.return_value = MOCK_FILES + with NamedTemporaryFile() as f: + gcs_provide_file = mock_hook.return_value.provide_file + gcs_provide_file.return_value.__enter__.return_value.name = f.name + + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=False, + ) + hook, bucket = _create_test_bucket() + bucket.put_object(Key=MOCK_FILES[0], Body=b"testing") + # we expect all except first file in MOCK_FILES to be uploaded # and all the MOCK_FILES to be present at the S3 bucket uploaded_files = operator.execute(None) @@ -81,15 +108,16 @@ def test_execute_without_replace(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=False, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=False, + ) hook, bucket = _create_test_bucket() for mock_file in MOCK_FILES: bucket.put_object(Key=mock_file, Body=b"testing") @@ -108,15 +136,16 @@ def test_execute(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=False, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=False, + ) hook, _ = _create_test_bucket() # we expect all MOCK_FILES to be uploaded @@ -133,15 +162,16 @@ def test_execute_with_replace(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=True, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=True, + ) hook, bucket = _create_test_bucket() for mock_file in MOCK_FILES: bucket.put_object(Key=mock_file, Body=b"testing") @@ -160,15 +190,16 @@ def test_execute_incremental_with_replace(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=True, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=True, + ) hook, bucket = _create_test_bucket() for mock_file in MOCK_FILES[:2]: bucket.put_object(Key=mock_file, Body=b"testing") @@ -187,15 +218,16 @@ def test_execute_should_handle_with_default_dest_s3_extra_args(self, s3_mock_hoo s3_mock_hook.return_value = mock.Mock() s3_mock_hook.parse_s3_url.return_value = mock.Mock() - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=True, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=True, + ) operator.execute(None) s3_mock_hook.assert_called_once_with(aws_conn_id="aws_default", extra_args={}, verify=None) @@ -209,18 +241,19 @@ def test_execute_should_pass_dest_s3_extra_args_to_s3_hook(self, s3_mock_hook, m s3_mock_hook.return_value = mock.Mock() s3_mock_hook.parse_s3_url.return_value = mock.Mock() - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=True, - dest_s3_extra_args={ - "ContentLanguage": "value", - }, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=True, + dest_s3_extra_args={ + "ContentLanguage": "value", + }, + ) operator.execute(None) s3_mock_hook.assert_called_once_with( aws_conn_id="aws_default", extra_args={"ContentLanguage": "value"}, verify=None @@ -235,16 +268,17 @@ def test_execute_with_s3_acl_policy(self, mock_load_file, mock_gcs_hook): gcs_provide_file = mock_gcs_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=False, - s3_acl_policy=S3_ACL_POLICY, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=False, + s3_acl_policy=S3_ACL_POLICY, + ) _create_test_bucket() operator.execute(None) @@ -259,16 +293,17 @@ def test_execute_without_keep_director_structure(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - operator = GCSToS3Operator( - task_id=TASK_ID, - bucket=GCS_BUCKET, - prefix=PREFIX, - delimiter=DELIMITER, - dest_aws_conn_id="aws_default", - dest_s3_key=S3_BUCKET, - replace=False, - keep_directory_structure=False, - ) + with pytest.deprecated_call(): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=S3_BUCKET, + replace=False, + keep_directory_structure=False, + ) hook, _ = _create_test_bucket() # we expect all except first file in MOCK_FILES to be uploaded diff --git a/tests/providers/google/cloud/hooks/test_gcs.py b/tests/providers/google/cloud/hooks/test_gcs.py index 926ad36431b2..9e7ff971d2ba 100644 --- a/tests/providers/google/cloud/hooks/test_gcs.py +++ b/tests/providers/google/cloud/hooks/test_gcs.py @@ -817,14 +817,66 @@ def test_provide_file_upload(self, mock_upload, mock_temp_file): ), ) @mock.patch(GCS_STRING.format("GCSHook.get_conn")) - def test_list(self, mock_service, prefix, result): + def test_list__delimiter(self, mock_service, prefix, result): mock_service.return_value.bucket.return_value.list_blobs.return_value.next_page_token = None + with pytest.deprecated_call(): + self.gcs_hook.list( + bucket_name="test_bucket", + prefix=prefix, + delimiter=",", + ) + assert mock_service.return_value.bucket.return_value.list_blobs.call_args_list == result + + @mock.patch(GCS_STRING.format("GCSHook.get_conn")) + @mock.patch("airflow.providers.google.cloud.hooks.gcs.functools") + @mock.patch("google.cloud.storage.bucket._item_to_blob") + @mock.patch("google.cloud.storage.bucket._blobs_page_start") + @mock.patch("google.api_core.page_iterator.HTTPIterator") + def test_list__match_glob( + self, http_iterator, _blobs_page_start, _item_to_blob, mocked_functools, mock_service + ): + http_iterator.return_value.next_page_token = None self.gcs_hook.list( bucket_name="test_bucket", - prefix=prefix, - delimiter=",", + prefix="prefix", + match_glob="**/*.json", + ) + http_iterator.assert_has_calls( + [ + mock.call( + api_request=mocked_functools.partial.return_value, + client=mock_service.return_value, + extra_params={"prefix": "prefix", "matchGlob": "**/*.json"}, + item_to_value=_item_to_blob, + max_results=None, + page_start=_blobs_page_start, + page_token=None, + path=mock_service.return_value.bucket.return_value.path.__add__.return_value, + ) + ] + ) + + @mock.patch(GCS_STRING.format("GCSHook.get_conn")) + def test_list__error_match_glob_and_invalid_delimiter(self, _): + with pytest.raises(AirflowException): + self.gcs_hook.list( + bucket_name="test_bucket", + prefix="prefix", + delimiter=",", + match_glob="**/*.json", + ) + + @pytest.mark.parametrize("delimiter", [None, "", "/"]) + @mock.patch("google.api_core.page_iterator.HTTPIterator") + @mock.patch(GCS_STRING.format("GCSHook.get_conn")) + def test_list__error_match_glob_and_valid_delimiter(self, mock_service, http_iterator, delimiter): + http_iterator.return_value.next_page_token = None + self.gcs_hook.list( + bucket_name="test_bucket", + prefix="prefix", + delimiter="/", + match_glob="**/*.json", ) - assert mock_service.return_value.bucket.return_value.list_blobs.call_args_list == result @mock.patch(GCS_STRING.format("GCSHook.get_conn")) def test_list_by_timespans(self, mock_service): diff --git a/tests/providers/google/cloud/operators/test_gcs.py b/tests/providers/google/cloud/operators/test_gcs.py index bf9a4f5d7ea5..b048aa0c8e7c 100644 --- a/tests/providers/google/cloud/operators/test_gcs.py +++ b/tests/providers/google/cloud/operators/test_gcs.py @@ -159,14 +159,26 @@ def test_delete_prefix_as_empty_string(self, mock_hook): class TestGoogleCloudStorageListOperator: @mock.patch("airflow.providers.google.cloud.operators.gcs.GCSHook") - def test_execute(self, mock_hook): + def test_execute__delimiter(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES operator = GCSListObjectsOperator( task_id=TASK_ID, bucket=TEST_BUCKET, prefix=PREFIX, delimiter=DELIMITER ) files = operator.execute(context=mock.MagicMock()) mock_hook.return_value.list.assert_called_once_with( - bucket_name=TEST_BUCKET, prefix=PREFIX, delimiter=DELIMITER + bucket_name=TEST_BUCKET, prefix=PREFIX, delimiter=DELIMITER, match_glob=None + ) + assert sorted(files) == sorted(MOCK_FILES) + + @mock.patch("airflow.providers.google.cloud.operators.gcs.GCSHook") + def test_execute__match_glob(self, mock_hook): + mock_hook.return_value.list.return_value = MOCK_FILES + operator = GCSListObjectsOperator( + task_id=TASK_ID, bucket=TEST_BUCKET, prefix=PREFIX, match_glob=f"**/*{DELIMITER}", delimiter=None + ) + files = operator.execute(context=mock.MagicMock()) + mock_hook.return_value.list.assert_called_once_with( + bucket_name=TEST_BUCKET, prefix=PREFIX, match_glob=f"**/*{DELIMITER}", delimiter=None ) assert sorted(files) == sorted(MOCK_FILES) diff --git a/tests/providers/google/cloud/transfers/test_gcs_to_gcs.py b/tests/providers/google/cloud/transfers/test_gcs_to_gcs.py index d8fa94c0c690..1cf7be116679 100644 --- a/tests/providers/google/cloud/transfers/test_gcs_to_gcs.py +++ b/tests/providers/google/cloud/transfers/test_gcs_to_gcs.py @@ -54,6 +54,8 @@ MOD_TIME_2 = datetime(2019, 1, 1) +# TODO: After deprecating delimiter and wildcards in source objects, +# implement reverted changes from the first commit of PR #31261 class TestGoogleCloudStorageToCloudStorageOperator: """ Tests the three use-cases for the wildcard operator. These are @@ -100,7 +102,7 @@ def test_execute_wildcard_with_replace_flag_false(self, mock_hook): operator.execute(None) mock_calls = [ mock.call(TEST_BUCKET, prefix="test_object", delimiter=""), - mock.call(DESTINATION_BUCKET, prefix="test_object", delimiter=""), + mock.call(DESTINATION_BUCKET, prefix="test_object", delimiter="", match_glob=None), ] mock_hook.return_value.list.assert_has_calls(mock_calls) @@ -117,8 +119,8 @@ def test_execute_no_wildcard_with_replace_flag_false(self, mock_hook): operator.execute(None) mock_calls = [ - mock.call(TEST_BUCKET, prefix=SOURCE_OBJECT_NO_WILDCARD, delimiter=None), - mock.call(DESTINATION_BUCKET, prefix=SOURCE_OBJECT_NO_WILDCARD, delimiter=None), + mock.call(TEST_BUCKET, prefix=SOURCE_OBJECT_NO_WILDCARD, delimiter=None, match_glob=None), + mock.call(DESTINATION_BUCKET, prefix=SOURCE_OBJECT_NO_WILDCARD, delimiter=None, match_glob=None), ] mock_hook.return_value.list.assert_has_calls(mock_calls) @@ -140,7 +142,7 @@ def test_copy_file_with_exact_match(self, mock_hook): operator.execute(None) mock_calls = [ - mock.call(TEST_BUCKET, prefix="test_object.txt", delimiter=None), + mock.call(TEST_BUCKET, prefix="test_object.txt", delimiter=None, match_glob=None), ] mock_hook.return_value.list.assert_has_calls(mock_calls) @@ -450,7 +452,9 @@ def test_executes_with_empty_source_objects(self, mock_hook): ) operator.execute(None) - mock_hook.return_value.list.assert_called_once_with(TEST_BUCKET, prefix="", delimiter=None) + mock_hook.return_value.list.assert_called_once_with( + TEST_BUCKET, prefix="", delimiter=None, match_glob=None + ) @mock.patch("airflow.providers.google.cloud.transfers.gcs_to_gcs.GCSHook") def test_raises_exception_with_two_empty_list_inside_source_objects(self, mock_hook): @@ -469,7 +473,7 @@ def test_executes_with_single_item_in_source_objects(self, mock_hook): ) operator.execute(None) mock_hook.return_value.list.assert_called_once_with( - TEST_BUCKET, prefix=SOURCE_OBJECTS_SINGLE_FILE[0], delimiter=None + TEST_BUCKET, prefix=SOURCE_OBJECTS_SINGLE_FILE[0], delimiter=None, match_glob=None ) @mock.patch("airflow.providers.google.cloud.transfers.gcs_to_gcs.GCSHook") @@ -480,8 +484,8 @@ def test_executes_with_multiple_items_in_source_objects(self, mock_hook): operator.execute(None) mock_hook.return_value.list.assert_has_calls( [ - mock.call(TEST_BUCKET, prefix="test_object/file1.txt", delimiter=None), - mock.call(TEST_BUCKET, prefix="test_object/file2.txt", delimiter=None), + mock.call(TEST_BUCKET, prefix="test_object/file1.txt", delimiter=None, match_glob=None), + mock.call(TEST_BUCKET, prefix="test_object/file2.txt", delimiter=None, match_glob=None), ], any_order=True, ) @@ -495,7 +499,9 @@ def test_executes_with_a_delimiter(self, mock_hook): delimiter=DELIMITER, ) operator.execute(None) - mock_hook.return_value.list.assert_called_once_with(TEST_BUCKET, prefix="", delimiter=DELIMITER) + mock_hook.return_value.list.assert_called_once_with( + TEST_BUCKET, prefix="", delimiter=DELIMITER, match_glob=None + ) # COPY @mock.patch("airflow.providers.google.cloud.transfers.gcs_to_gcs.GCSHook") @@ -593,7 +599,7 @@ def test_execute_wildcard_with_replace_flag_false_with_destination_object(self, operator.execute(None) mock_calls = [ mock.call(TEST_BUCKET, prefix="test_object", delimiter=""), - mock.call(DESTINATION_BUCKET, prefix="foo/bar", delimiter=""), + mock.call(DESTINATION_BUCKET, prefix="foo/bar", delimiter="", match_glob=None), ] mock_hook.return_value.list.assert_has_calls(mock_calls) diff --git a/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py b/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py index 9c6884766bea..f2ede74a8a73 100644 --- a/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py +++ b/tests/providers/google/cloud/transfers/test_gcs_to_sftp.py @@ -34,6 +34,8 @@ DESTINATION_SFTP = "destination_path" +# TODO: After deprecating delimiter and wildcards in source objects, +# implement reverted changes from the first commit of PR #31261 class TestGoogleCloudStorageToSFTPOperator: @pytest.mark.parametrize( "source_object, target_object, keep_directory_structure", diff --git a/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py b/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py index 5730a4b66ed5..525f20398ab3 100644 --- a/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py +++ b/tests/providers/google/suite/transfers/test_gcs_to_gdrive.py @@ -95,6 +95,9 @@ def test_should_copy_files(self, mock_named_temporary_file, mock_gdrive, mock_gc impersonation_chain=IMPERSONATION_CHAIN, ), mock.call().list("data", delimiter=".avro", prefix="sales/sales-2017/"), + # TODO: After deprecating delimiter and wildcards in source objects, + # remove previous line and uncomment the following: + # mock.call().list("data", match_glob="**/*.avro", prefix="sales/sales-2017/"), mock.call().download(bucket_name="data", filename="TMP1", object_name="sales/A.avro"), mock.call().download(bucket_name="data", filename="TMP2", object_name="sales/B.avro"), mock.call().download(bucket_name="data", filename="TMP3", object_name="sales/C.avro"), @@ -137,6 +140,9 @@ def test_should_move_files(self, mock_named_temporary_file, mock_gdrive, mock_gc impersonation_chain=IMPERSONATION_CHAIN, ), mock.call().list("data", delimiter=".avro", prefix="sales/sales-2017/"), + # TODO: After deprecating delimiter and wildcards in source objects, + # remove previous line and uncomment the following: + # mock.call().list("data", match_glob="**/*.avro", prefix="sales/sales-2017/"), mock.call().download(bucket_name="data", filename="TMP1", object_name="sales/A.avro"), mock.call().delete("data", "sales/A.avro"), mock.call().download(bucket_name="data", filename="TMP2", object_name="sales/B.avro"), diff --git a/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py index 7931295d23e7..e13f76ebc469 100644 --- a/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py +++ b/tests/system/providers/google/cloud/gcs/example_gcs_to_gcs.py @@ -173,6 +173,17 @@ ) # [END howto_operator_gcs_to_gcs_delimiter] + # [START howto_operator_gcs_to_gcs_match_glob] + copy_files_with_match_glob = GCSToGCSOperator( + task_id="copy_files_with_match_glob", + source_bucket=BUCKET_NAME_SRC, + source_object="data/", + destination_bucket=BUCKET_NAME_DST, + destination_object="backup/", + match_glob="**/*.txt", + ) + # [END howto_operator_gcs_to_gcs_match_glob] + # [START howto_operator_gcs_to_gcs_list] copy_files_with_list = GCSToGCSOperator( task_id="copy_files_with_list", @@ -226,6 +237,7 @@ copy_files_with_wildcard, copy_files_without_wildcard, copy_files_with_delimiter, + copy_files_with_match_glob, copy_files_with_list, move_single_file, move_files_with_list, From b6ca28e55026acb98fb50aa098c7f621a165cd62 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 30 Jun 2023 11:14:38 +0200 Subject: [PATCH 005/533] Update git-sync description in Helm Chart documentation (#32181) There are quite a few recurring themes when it comes to using git-sync for DAG synchronisation and this documentation is an attempt to capture results of a number of discussions and conversations. It adds some notes that might make it possible to make more informed decisions by our users and Deployment managers who want to make decisions on how they should synchronize their DAGs. The changes include: * notes on potential side-effects one has to be aware when using both git-sync and persistence together (there are some unobvious operations performed by git-sync that might affect performances of persistence solutions) * notes on how you can use multiple git repositories with git-sync using submodule approach - including link to a real-life use case from Airflow summit where it has been used in production for 100s of repositories. --- docs/helm-chart/manage-dags-files.rst | 80 ++++++++++++++++++++++++++- docs/spelling_wordlist.txt | 2 +- 2 files changed, 79 insertions(+), 3 deletions(-) diff --git a/docs/helm-chart/manage-dags-files.rst b/docs/helm-chart/manage-dags-files.rst index 05b283410ff7..9d74c2c852a9 100644 --- a/docs/helm-chart/manage-dags-files.rst +++ b/docs/helm-chart/manage-dags-files.rst @@ -98,8 +98,11 @@ If you are deploying an image from a private repository, you need to create a se --set images.airflow.pullPolicy=Always \ --set registry.secretName=gitlab-registry-credentials +Using Git-sync +-------------- + Mounting DAGs using Git-Sync sidecar with Persistence enabled -------------------------------------------------------------- +............................................................. This option will use a Persistent Volume Claim with an access mode of ``ReadWriteMany``. The scheduler pod will sync DAGs from a git repository onto the PVC every configured number of @@ -117,8 +120,9 @@ for details. # by setting the dags.persistence.* and dags.gitSync.* values # Please refer to values.yaml for details + Mounting DAGs using Git-Sync sidecar without Persistence --------------------------------------------------------- +........................................................ This option will use an always running Git-Sync sidecar on every scheduler, webserver (if ``airflowVersion < 2.0.0``) and worker pods. @@ -137,6 +141,78 @@ seconds. If you are using the ``KubernetesExecutor``, Git-sync will run as an in When using ``apache-airflow >= 2.0.0``, :ref:`DAG Serialization ` is enabled by default, hence Webserver does not need access to DAG files, so ``git-sync`` sidecar is not run on Webserver. +Notes for combining git-sync and persistence +............................................ + +While using both git-sync and persistence for DAGs is possible, it is generally not recommended unless the +deployment manager carefully considered the trade-offs it brings. There are cases when git-sync without +persistence has other trade-offs (for example delays in synchronization of DAGS vs. rate-limiting of Git +servers) that can often be mitigated (for example by sending signals to git-sync containers via web-hooks +when new commits are pushed to the repository) but there might be cases where you still might want to choose +git-sync and Persistence together, but as a Deployment Manager you should be aware of some consequences it has. + +git-sync solution is primarily designed to be used for local, POSIX-compliant volumes to checkout Git +repositories into. Part of the process of synchronization of commits from git-sync involves checking out +new version of files in a freshly created folder and swapping symbolic links to the new folder, after the +checkout is complete. This is done to ensure that the whole DAGs folder is consistent at all times. The way +git-sync works with symbolic-link swaps, makes sure that Parsing the DAGs always work on a consistent +(single-commit-based) set of files in the whole DAG folder. + +This approach, however might have undesirable side effects when the folder that git-sync works on is not +a local volume, but is a persistent volume (so effectively a networked, distributed volume). Depending on +the technology behind the persistent volumes might handle git-sync approach differently and with non-obvious +consequences. There are a lot of persistence solutions available for various K8S installations and each of +them has different characteristics, so you need to carefully test and monitor your filesystem to make sure +those undesired side effects do not affect you. Those effects might change over time or depend on parameters +like how often the files are being scanned by the Dag File Processor, the number and complexity of your +DAGs, how remote and how distributed your persistent volumes are, how many IOPS you allocate for some of +the filesystem (usually highly paid feature of such filesystems is how many IOPS you can get) and many other +factors. + +The way git-sync works with symbolic links swapping generally causes a linear growth of the throughput and +potential delays in synchronization. The networking traffic from checkouts comes in bursts and the bursts +are linearly proportional to the number and size of files you have in the repository, makes it vulnerable +to pretty sudden and unexpected demand increase. Most of the persistence solution work "good enough" for +smaller/shorter burst of traffic, but when they outgrow certain thresholds, you need to upgrade the +networking to a much more capable and expensive options. This is difficult to control and impossible to +mitigate, so you might be suddenly faced with situation to pay a lot more for IOPS/persistence option to +keep your DAGs sufficiently synchronized to avoid inconsistencies and delays in synchronization. + +The side-effects that you might observe: + +* burst of networking/communication at the moment when new commit is checked out (because of the quick + succession of deleting old files, creating new files, symbolic link swapping. +* temporary lack of consistency between files in DAG folders while DAGS are being synced (because of delays + in distributing changes to individual files for various nodes in the cluster) +* visible drops of performance of the persistence solution when your DAG number grows, drops that might + amplify the side effects described above. +* some of persistence solutions might lack filesystem functionality that git-sync needs to perform the sync + (for example changing permissions or creating symbolic links). While those can often be mitigated it is + only recommended to use git-sync with fully POSIX-filesystem compliant persistence filesystems. + +General recommendation to use git-sync with local volumes only, and if you want to also use persistence, you +need to make sure that the persistence solution you use is POSIX-compliant and you monitor the side-effects +it might have. + +Synchronizing multiple Git repositories with git-sync +..................................................... + +Airflow git-sync integration in the Helm Chart, does not allow to configure multiple repositories to be +synchronized at the same time. The DAG folder must come from single git repository. However it is possible +to use `submodules `_ to create an "umbrella" repository +that you can use to bring a number of git repositories checked out together (with ``--submodules recursive`` +option). There are success stories of Airflow users using such approach with 100s of repositories put +together as submodules via such "umbrella" repo approach. When you choose this solution, however, +you need to work out the way how to ling the submodules, when to updated the umbrella repo when "submodule" +repository change and work out versioning approach and automate it. This might be as simple as always +using latest versions of all the submodule repositories, or as complex as managing versioning of shared +libraries, DAGs and code across multiple teams and doing that following your release process. + +An example of such complex approach can found in this +`Manage DAGs at scale `_ presentation from the Airflow +Summit. + + Mounting DAGs from an externally populated PVC ---------------------------------------------- diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index f51dc1215fc0..d3725c6e4db4 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -1426,7 +1426,7 @@ Subdirectory subdirectory subfolder subfolders -Submodules +submodule submodules subnet subnets From ead2530d3500dd27df54383a0802b6c94828c359 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 30 Jun 2023 11:15:17 +0200 Subject: [PATCH 006/533] Separate out advanced logging configuration (#32131) The "advanced logging configuration" applies not only to task logs but also to component logs and you can use it not only to configure custom way how task logs are created but also custom way how "regular" component logs are created. This has been a source of confusion for those who wanted to configure (for example) elasticsearch or opensearch for the whole airflow deployment, because the "advanced configuration" and how to modify standard configuration chapter was a small section in "task logging". This change extracts "advanced logging configuration" to separate page right under the "logging and monitoring" and directs the user from the "task" logging section to this page. It also adds a bit more explanation on how standard Python logging framework is leveraged here and links to Python logging documentation for those who never used it before, to understand more about Loggers, Handlers and Formatters. Co-authored-by: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Co-authored-by: Tzu-ping Chung --- .../logging/stackdriver.rst | 3 +- .../core-extensions/logging.rst | 2 +- .../core-extensions/secrets-backends.rst | 2 +- .../advanced-logging-configuration.rst | 90 +++++++++++++++++++ .../logging-monitoring/index.rst | 1 + .../logging-architecture.rst | 10 ++- .../logging-monitoring/logging-tasks.rst | 53 ++--------- docs/spelling_wordlist.txt | 1 + 8 files changed, 113 insertions(+), 49 deletions(-) create mode 100644 docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst diff --git a/docs/apache-airflow-providers-google/logging/stackdriver.rst b/docs/apache-airflow-providers-google/logging/stackdriver.rst index fa3f3391af6c..12be70cf65e4 100644 --- a/docs/apache-airflow-providers-google/logging/stackdriver.rst +++ b/docs/apache-airflow-providers-google/logging/stackdriver.rst @@ -66,7 +66,8 @@ be used. Make sure that with those credentials, you can read and write the logs. the logs. For security reasons, limiting the access of the log reader to only allow log reading and writing is an important security measure. -By using the ``logging_config_class`` option you can get :ref:`advanced features ` of +By using the ``logging_config_class`` option you can get +:doc:`advanced features ` of this handler. Details are available in the handler's documentation - :class:`~airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler`. diff --git a/docs/apache-airflow-providers/core-extensions/logging.rst b/docs/apache-airflow-providers/core-extensions/logging.rst index 822cf54d2b2f..0fa10d98c412 100644 --- a/docs/apache-airflow-providers/core-extensions/logging.rst +++ b/docs/apache-airflow-providers/core-extensions/logging.rst @@ -20,7 +20,7 @@ Writing logs This is a summary of all Apache Airflow Community provided implementations of writing task logs exposed via community-managed providers. You can also see logging options available in the core Airflow in -:doc:`apache-airflow:administration-and-deployment/logging-monitoring/logging-tasks` and here you can see those +:doc:`/administration-and-deployment/logging-monitoring/logging-tasks` and here you can see those provided by the community-managed providers: .. airflow-logging:: diff --git a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst index 22fae4b8a6c4..c1cc5a0c0508 100644 --- a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst @@ -28,7 +28,7 @@ via providers that implement secrets backends for services Airflow integrates wi You can also take a look at Secret backends available in the core Airflow in -:doc:`apache-airflow:administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones +:doc:`/administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones provided by the community-managed providers: .. airflow-secrets-backends:: diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst new file mode 100644 index 000000000000..739b5380e3c6 --- /dev/null +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst @@ -0,0 +1,90 @@ + .. Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + .. http://www.apache.org/licenses/LICENSE-2.0 + + .. Unless required by applicable law or agreed to in writing, + software distributed under the License is distributed on an + "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + KIND, either express or implied. See the License for the + specific language governing permissions and limitations + under the License. + + + +Advanced logging configuration +------------------------------ + +Not all configuration options are available from the ``airflow.cfg`` file. The config file describes +how to configure logging for tasks, because the logs generated by tasks are not only logged in separate +files by default but has to be also accessible via the webserver. + +By default standard airflow component logs are written to the ``$AIRFLOW_HOME/logs`` directory, but you +can also customize it and configure it as you want by overriding Python logger configuration that can +be configured by providing custom logging configuration object. Some configuration options require +that the logging config class be overwritten. You can do it by copying the default +configuration of Airflow and modifying it to suit your needs. The default configuration can be seen in the +`airflow_local_settings.py template `_ +and you can see the loggers and handlers used there. Except the custom loggers and handlers configurable there +via the ``airflow.cfg``, the logging methods in Airflow follow the usual Python logging convention, +that Python objects log to loggers that follow naming convention of ``.``. + +You can read more about standard python logging classes (Loggers, Handlers, Formatters) in the +`Python logging documentation `_. + +Configuring your logging classes can be done via the ``logging_config_class`` option in ``airflow.cfg`` file. +This configuration should specify the import path to a configuration compatible with +:func:`logging.config.dictConfig`. If your file is a standard import location, then you should set a +:envvar:`PYTHONPATH` environment variable. + +Follow the steps below to enable custom logging config class: + +#. Start by setting environment variable to known directory e.g. ``~/airflow/`` + + .. code-block:: bash + + export PYTHONPATH=~/airflow/ + +#. Create a directory to store the config file e.g. ``~/airflow/config`` +#. Create file called ``~/airflow/config/log_config.py`` with following the contents: + + .. code-block:: python + + from copy import deepcopy + from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG + + LOGGING_CONFIG = deepcopy(DEFAULT_LOGGING_CONFIG) + +#. At the end of the file, add code to modify the default dictionary configuration. +#. Update ``$AIRFLOW_HOME/airflow.cfg`` to contain: + + .. code-block:: ini + + [logging] + logging_config_class = log_config.LOGGING_CONFIG + +You can also use the ``logging_config_class`` together with remote logging if you plan to just extend/update +the configuration with remote logging enabled. Then the deep-copied dictionary will contain the remote logging +configuration generated for you and your modification will apply after remote logging configuration has +been added: + + .. code-block:: ini + + [logging] + remote_logging = True + logging_config_class = log_config.LOGGING_CONFIG + + +#. Restart the application. + +See :doc:`../modules_management` for details on how Python and Airflow manage modules. + + +.. note:: + + You can override the way both standard logs of the components and "task" logs are handled. diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst index 58edec4be0a9..850bfa77a933 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/index.rst @@ -30,6 +30,7 @@ In addition to the standard logging and metrics capabilities, Airflow supports t logging-architecture logging-tasks + advanced-logging-configuration metrics callbacks diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst index 07cd3f42cc2e..ae000ddfe0ba 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-architecture.rst @@ -26,9 +26,15 @@ Airflow supports a variety of logging and monitoring mechanisms as shown below. By default, Airflow supports logging into the local file system. These include logs from the Web server, the Scheduler, and the Workers running tasks. This is suitable for development environments and for quick debugging. -For cloud deployments, Airflow also has handlers contributed by the Community for logging to cloud storage such as AWS, Google Cloud, and Azure. +For cloud deployments, Airflow also has task handlers contributed by the Community for +logging to cloud storage such as AWS, Google Cloud, and Azure. -The logging settings and options can be specified in the Airflow Configuration file, which as usual needs to be available to all the Airflow process: Web server, Scheduler, and Workers. +The logging settings and options can be specified in the Airflow Configuration file, +which as usual needs to be available to all the Airflow process: Web server, Scheduler, and Workers. + +You can customize the logging settings for each of the Airflow components by specifying the logging settings +in the Airflow Configuration file, or for advanced configuration by using +:doc:`advanced features `. For production deployments, we recommend using FluentD to capture logs and send it to destinations such as ElasticSearch or Splunk. diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 0b7ef9a472e2..2b1d93176521 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -117,53 +117,11 @@ the example below. The output of ``airflow info`` above is truncated to only display the section that pertains to the logging configuration. You can also run ``airflow config list`` to check that the logging configuration options have valid values. -.. _write-logs-advanced: - Advanced configuration ---------------------- -Not all configuration options are available from the ``airflow.cfg`` file. Some configuration options require -that the logging config class be overwritten. This can be done via the ``logging_config_class`` option -in ``airflow.cfg`` file. This option should specify the import path to a configuration compatible with -:func:`logging.config.dictConfig`. If your file is a standard import location, then you should set a :envvar:`PYTHONPATH` environment variable. - -Follow the steps below to enable custom logging config class: - -#. Start by setting environment variable to known directory e.g. ``~/airflow/`` - - .. code-block:: bash - - export PYTHONPATH=~/airflow/ - -#. Create a directory to store the config file e.g. ``~/airflow/config`` -#. Create file called ``~/airflow/config/log_config.py`` with following the contents: - - .. code-block:: python - - from copy import deepcopy - from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG - - LOGGING_CONFIG = deepcopy(DEFAULT_LOGGING_CONFIG) - -#. At the end of the file, add code to modify the default dictionary configuration. -#. Update ``$AIRFLOW_HOME/airflow.cfg`` to contain: - - .. code-block:: ini - - [logging] - remote_logging = True - logging_config_class = log_config.LOGGING_CONFIG - -#. Restart the application. - -See :doc:`../modules_management` for details on how Python and Airflow manage modules. - -External Links --------------- - -When using remote logging, you can configure Airflow to show a link to an external UI within the Airflow Web UI. Clicking the link redirects you to the external UI. - -Some external systems require specific configuration in Airflow for redirection to work but others do not. +You can configure :doc:`advanced features ` +- including adding your own custom task log handlers (but also log handlers for all airflow components). .. _serving-worker-trigger-logs: @@ -197,3 +155,10 @@ To accomplish this we have a few attributes that may be set on the handler, eith - ``trigger_should_queue``: Controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. - ``trigger_send_end_marker``: Controls whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. - ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. But if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler. + +External Links +-------------- + +When using remote logging, you can configure Airflow to show a link to an external UI within the Airflow Web UI. Clicking the link redirects you to the external UI. + +Some external systems require specific configuration in Airflow for redirection to work but others do not. diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index d3725c6e4db4..af0a3ff5dcbf 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -599,6 +599,7 @@ fn fo followsa formatter +formatters Formaturas forwardability forwardable From 1b599c9fbfb6151a41a588edaa786745f50eec38 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Fri, 30 Jun 2023 11:26:46 +0200 Subject: [PATCH 007/533] Break AwaitMessageTrigger execution when finding a message with the desired format (#31803) Signed-off-by: Hussein Awala Co-authored-by: eladkal <45845474+eladkal@users.noreply.github.com> --- airflow/providers/apache/kafka/triggers/await_message.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/providers/apache/kafka/triggers/await_message.py b/airflow/providers/apache/kafka/triggers/await_message.py index 5445579e125f..f2c1bb81ca6b 100644 --- a/airflow/providers/apache/kafka/triggers/await_message.py +++ b/airflow/providers/apache/kafka/triggers/await_message.py @@ -113,6 +113,7 @@ async def run(self): if rv: await async_commit(asynchronous=False) yield TriggerEvent(rv) + break else: await async_commit(asynchronous=False) await asyncio.sleep(self.poll_interval) From ef49e595f1ab729cdb72198a3ee47c7dcd182da2 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 30 Jun 2023 13:28:16 +0200 Subject: [PATCH 008/533] Update references to restructured documentation from airflow core (#32282) The change #32131 restructured some of the code for documentation and it broke references to apache-airlfow from "providers" doc package - it has not been visible in selective PR (not sure the reason) but this PR fixes it. --- docs/apache-airflow-providers-google/logging/stackdriver.rst | 2 +- docs/apache-airflow-providers/core-extensions/logging.rst | 2 +- .../core-extensions/secrets-backends.rst | 2 +- .../logging-monitoring/advanced-logging-configuration.rst | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/apache-airflow-providers-google/logging/stackdriver.rst b/docs/apache-airflow-providers-google/logging/stackdriver.rst index 12be70cf65e4..60943bda70ef 100644 --- a/docs/apache-airflow-providers-google/logging/stackdriver.rst +++ b/docs/apache-airflow-providers-google/logging/stackdriver.rst @@ -67,7 +67,7 @@ be used. Make sure that with those credentials, you can read and write the logs. an important security measure. By using the ``logging_config_class`` option you can get -:doc:`advanced features ` of +:ref:`advanced features ` of this handler. Details are available in the handler's documentation - :class:`~airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler`. diff --git a/docs/apache-airflow-providers/core-extensions/logging.rst b/docs/apache-airflow-providers/core-extensions/logging.rst index 0fa10d98c412..822cf54d2b2f 100644 --- a/docs/apache-airflow-providers/core-extensions/logging.rst +++ b/docs/apache-airflow-providers/core-extensions/logging.rst @@ -20,7 +20,7 @@ Writing logs This is a summary of all Apache Airflow Community provided implementations of writing task logs exposed via community-managed providers. You can also see logging options available in the core Airflow in -:doc:`/administration-and-deployment/logging-monitoring/logging-tasks` and here you can see those +:doc:`apache-airflow:administration-and-deployment/logging-monitoring/logging-tasks` and here you can see those provided by the community-managed providers: .. airflow-logging:: diff --git a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst index c1cc5a0c0508..22fae4b8a6c4 100644 --- a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst @@ -28,7 +28,7 @@ via providers that implement secrets backends for services Airflow integrates wi You can also take a look at Secret backends available in the core Airflow in -:doc:`/administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones +:doc:`apache-airflow:administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones provided by the community-managed providers: .. airflow-secrets-backends:: diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst index 739b5380e3c6..21ae801b64ba 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/advanced-logging-configuration.rst @@ -16,6 +16,7 @@ under the License. +.. _write-logs-advanced: Advanced logging configuration ------------------------------ From ba05a1e8706abffe057fa83d891b44125d5913e6 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Fri, 30 Jun 2023 14:41:09 +0200 Subject: [PATCH 009/533] Add apache-airflow-providers to filter in selective checks if needed (#32284) When provider docs in "docs/apache-airflow-providers" changed, and selective docs build was used, the "apache-airflow-providers" package was not added to the filter - thus errors in those docs were not checked. This caused #32131 to fail the main build (fixed in the #32282). Added logic to handle this case. --- .../src/airflow_breeze/utils/selective_checks.py | 2 ++ dev/breeze/tests/test_selective_checks.py | 14 ++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py index c6e414472d06..7c7d622dab61 100644 --- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py +++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py @@ -727,6 +727,8 @@ def docs_filter_list_as_string(self) -> str | None: [file.startswith("airflow/") or file.startswith("docs/apache-airflow/") for file in self._files] ): packages.append("apache-airflow") + if any([file.startswith("docs/apache-airflow-providers/") for file in self._files]): + packages.append("apache-airflow-providers") if any([file.startswith("chart/") or file.startswith("docs/helm-chart") for file in self._files]): packages.append("helm-chart") if any([file.startswith("docs/docker-stack/") for file in self._files]): diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 0dc12501b1c3..61e35ce20f59 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -1024,6 +1024,20 @@ def test_upgrade_to_newer_dependencies(files: tuple[str, ...], expected_outputs: }, id="Airbyte provider and airflow core docs changed", ), + pytest.param( + ( + "docs/apache-airflow-providers-airbyte/docs.rst", + "docs/apache-airflow/docs.rst", + "docs/apache-airflow-providers/docs.rst", + ), + { + "docs-filter-list-as-string": "--package-filter apache-airflow " + "--package-filter apache-airflow-providers " + "--package-filter apache-airflow-providers-airbyte " + "--package-filter apache-airflow-providers-http", + }, + id="Airbyte provider and airflow core and common provider docs changed", + ), pytest.param( ("docs/apache-airflow/docs.rst",), { From 4501f8b352aee9c2cd29126a64cab62fa19fc49d Mon Sep 17 00:00:00 2001 From: Matthias Vongerichten Date: Fri, 30 Jun 2023 15:51:55 +0200 Subject: [PATCH 010/533] Update gcp.rst (#32281) got an "TypeError: __init__() got an unexpected keyword argument 'key_path'" error message when not wrapped in the "extra" object. --- docs/apache-airflow-providers-google/connections/gcp.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow-providers-google/connections/gcp.rst b/docs/apache-airflow-providers-google/connections/gcp.rst index b2f7243fe71e..c8374d455604 100644 --- a/docs/apache-airflow-providers-google/connections/gcp.rst +++ b/docs/apache-airflow-providers-google/connections/gcp.rst @@ -154,7 +154,7 @@ Number of Retries .. code-block:: bash - export AIRFLOW_CONN_GOOGLE_CLOUD_DEFAULT='{"conn_type": "google-cloud-platform", "key_path": "/keys/key.json", "scope": "https://www.googleapis.com/auth/cloud-platform", "project": "airflow", "num_retries": 5}' + export AIRFLOW_CONN_GOOGLE_CLOUD_DEFAULT='{"conn_type": "google_cloud_platform", "extra": {"key_path": "/keys/key.json", "scope": "https://www.googleapis.com/auth/cloud-platform", "project": "airflow", "num_retries": 5}}' .. _howto/connection:gcp:impersonation: From d117728cd6f337266bebcf4916325d5de815fe03 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Fri, 30 Jun 2023 22:19:06 +0200 Subject: [PATCH 011/533] Fix triggerers alive check and add a new conf for triggerer heartbeat rate (#32123) Signed-off-by: Hussein Awala --- airflow/cli/commands/triggerer_command.py | 3 +- airflow/config_templates/config.yml | 7 +++ airflow/config_templates/default_airflow.cfg | 3 + airflow/jobs/triggerer_job_runner.py | 2 +- airflow/models/trigger.py | 13 ++-- tests/models/test_trigger.py | 64 ++++++++++++++++++-- 6 files changed, 81 insertions(+), 11 deletions(-) diff --git a/airflow/cli/commands/triggerer_command.py b/airflow/cli/commands/triggerer_command.py index 2da482187726..aa06d641c760 100644 --- a/airflow/cli/commands/triggerer_command.py +++ b/airflow/cli/commands/triggerer_command.py @@ -55,7 +55,8 @@ def triggerer(args): """Starts Airflow Triggerer.""" settings.MASK_SECRETS_IN_LOGS = True print(settings.HEADER) - triggerer_job_runner = TriggererJobRunner(job=Job(), capacity=args.capacity) + triggerer_heartrate = conf.getfloat("triggerer", "JOB_HEARTBEAT_SEC") + triggerer_job_runner = TriggererJobRunner(job=Job(heartrate=triggerer_heartrate), capacity=args.capacity) if args.daemon: pid, stdout, stderr, log_file = setup_locations( diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index b6b160326523..a252716ed1d0 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2565,6 +2565,13 @@ triggerer: type: string example: ~ default: "1000" + job_heartbeat_sec: + description: | + How often to heartbeat the Triggerer job to ensure it hasn't been killed. + version_added: 2.6.3 + type: float + example: ~ + default: "5" kerberos: description: ~ options: diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 1fbf58f7bccd..4eaab9ae95db 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -1329,6 +1329,9 @@ task_queued_timeout_check_interval = 120.0 # How many triggers a single Triggerer will run at once, by default. default_capacity = 1000 +# How often to heartbeat the Triggerer job to ensure it hasn't been killed. +job_heartbeat_sec = 5 + [kerberos] ccache = /tmp/airflow_krb5_ccache diff --git a/airflow/jobs/triggerer_job_runner.py b/airflow/jobs/triggerer_job_runner.py index 32435cc1e1da..633e90944cd7 100644 --- a/airflow/jobs/triggerer_job_runner.py +++ b/airflow/jobs/triggerer_job_runner.py @@ -374,7 +374,7 @@ def load_triggers(self): adds them to our runner, and then removes ones from it we no longer need. """ - Trigger.assign_unassigned(self.job.id, self.capacity) + Trigger.assign_unassigned(self.job.id, self.capacity, self.job.heartrate) ids = Trigger.ids_for_triggerer(self.job.id) self.trigger_runner.update_triggers(set(ids)) diff --git a/airflow/models/trigger.py b/airflow/models/trigger.py index 603ff95157ca..c0d749eb5905 100644 --- a/airflow/models/trigger.py +++ b/airflow/models/trigger.py @@ -200,10 +200,11 @@ def ids_for_triggerer(cls, triggerer_id, session: Session = NEW_SESSION) -> list @classmethod @internal_api_call @provide_session - def assign_unassigned(cls, triggerer_id, capacity, session: Session = NEW_SESSION) -> None: + def assign_unassigned(cls, triggerer_id, capacity, heartrate, session: Session = NEW_SESSION) -> None: """ - Takes a triggerer_id and the capacity for that triggerer and assigns unassigned - triggers until that capacity is reached, or there are no more unassigned triggers. + Takes a triggerer_id, the capacity for that triggerer and the Triggerer job heartrate, + and assigns unassigned triggers until that capacity is reached, or there are no more + unassigned triggers. """ from airflow.jobs.job import Job # To avoid circular import @@ -212,12 +213,14 @@ def assign_unassigned(cls, triggerer_id, capacity, session: Session = NEW_SESSIO if capacity <= 0: return - + # we multiply heartrate by a grace_multiplier to give the triggerer + # a chance to heartbeat before we consider it dead + health_check_threshold = heartrate * 2.1 alive_triggerer_ids = [ row[0] for row in session.query(Job.id).filter( Job.end_date.is_(None), - Job.latest_heartbeat > timezone.utcnow() - datetime.timedelta(seconds=30), + Job.latest_heartbeat > timezone.utcnow() - datetime.timedelta(seconds=health_check_threshold), Job.job_type == "TriggererJob", ) ] diff --git a/tests/models/test_trigger.py b/tests/models/test_trigger.py index 4abd7e9e42b8..0c5e5eeadcb8 100644 --- a/tests/models/test_trigger.py +++ b/tests/models/test_trigger.py @@ -141,16 +141,17 @@ def test_assign_unassigned(session, create_task_instance): """ Tests that unassigned triggers of all appropriate states are assigned. """ - finished_triggerer = Job(heartrate=10, state=State.SUCCESS) + triggerer_heartrate = 10 + finished_triggerer = Job(heartrate=triggerer_heartrate, state=State.SUCCESS) TriggererJobRunner(finished_triggerer) finished_triggerer.end_date = timezone.utcnow() - datetime.timedelta(hours=1) session.add(finished_triggerer) assert not finished_triggerer.is_alive() - healthy_triggerer = Job(heartrate=10, state=State.RUNNING) + healthy_triggerer = Job(heartrate=triggerer_heartrate, state=State.RUNNING) TriggererJobRunner(healthy_triggerer) session.add(healthy_triggerer) assert healthy_triggerer.is_alive() - new_triggerer = Job(heartrate=10, state=State.RUNNING) + new_triggerer = Job(heartrate=triggerer_heartrate, state=State.RUNNING) TriggererJobRunner(new_triggerer) session.add(new_triggerer) assert new_triggerer.is_alive() @@ -169,7 +170,7 @@ def test_assign_unassigned(session, create_task_instance): session.add(trigger_unassigned_to_triggerer) session.commit() assert session.query(Trigger).count() == 3 - Trigger.assign_unassigned(new_triggerer.id, 100, session=session) + Trigger.assign_unassigned(new_triggerer.id, 100, session=session, heartrate=triggerer_heartrate) session.expire_all() # Check that trigger on killed triggerer and unassigned trigger are assigned to new triggerer assert ( @@ -187,6 +188,61 @@ def test_assign_unassigned(session, create_task_instance): ) +@pytest.mark.parametrize("check_triggerer_heartrate", [10, 60, 300]) +def test_assign_unassigned_missing_heartbeat(session, create_task_instance, check_triggerer_heartrate): + """ + Tests that the triggers assigned to a dead triggers are considered as unassigned + and they are assigned to an alive triggerer. + """ + import time_machine + + block_triggerer_heartrate = 9999 + with time_machine.travel(datetime.datetime.utcnow(), tick=False) as t: + first_triggerer = Job(heartrate=block_triggerer_heartrate, state=State.RUNNING) + TriggererJobRunner(first_triggerer) + session.add(first_triggerer) + assert first_triggerer.is_alive() + second_triggerer = Job(heartrate=block_triggerer_heartrate, state=State.RUNNING) + TriggererJobRunner(second_triggerer) + session.add(second_triggerer) + assert second_triggerer.is_alive() + session.commit() + trigger_on_first_triggerer = Trigger(classpath="airflow.triggers.testing.SuccessTrigger", kwargs={}) + trigger_on_first_triggerer.id = 1 + trigger_on_first_triggerer.triggerer_id = first_triggerer.id + trigger_on_second_triggerer = Trigger(classpath="airflow.triggers.testing.SuccessTrigger", kwargs={}) + trigger_on_second_triggerer.id = 2 + trigger_on_second_triggerer.triggerer_id = second_triggerer.id + session.add(trigger_on_first_triggerer) + session.add(trigger_on_second_triggerer) + session.commit() + assert session.query(Trigger).count() == 2 + triggers_ids = [ + (first_triggerer.id, second_triggerer.id), + (first_triggerer.id, second_triggerer.id), + (first_triggerer.id, second_triggerer.id), + # Check that after more than 2.1 heartrates, the first triggerer is considered dead + # and the first trigger is assigned to the second triggerer + (second_triggerer.id, second_triggerer.id), + ] + for i in range(4): + Trigger.assign_unassigned( + second_triggerer.id, 100, session=session, heartrate=check_triggerer_heartrate + ) + session.expire_all() + # Check that trigger on killed triggerer and unassigned trigger are assigned to new triggerer + assert ( + session.query(Trigger).filter(Trigger.id == trigger_on_first_triggerer.id).one().triggerer_id + == triggers_ids[i][0] + ) + assert ( + session.query(Trigger).filter(Trigger.id == trigger_on_second_triggerer.id).one().triggerer_id + == triggers_ids[i][1] + ) + t.shift(datetime.timedelta(seconds=check_triggerer_heartrate)) + second_triggerer.latest_heartbeat += datetime.timedelta(seconds=check_triggerer_heartrate) + + def test_get_sorted_triggers(session, create_task_instance): """ Tests that triggers are sorted by the creation_date. From df4c8837d022e66921bc0cf33f3249b235de6fdd Mon Sep 17 00:00:00 2001 From: Ashwin Agate <87091342+aagateuip@users.noreply.github.com> Date: Sat, 1 Jul 2023 02:43:48 -0400 Subject: [PATCH 012/533] Fix KubernetesPodOperator validate xcom json and add retries (#32113) * Fix KubernetesPodOperator validate xcom json and add retries --- .../cncf/kubernetes/utils/pod_manager.py | 42 +++++++++++++++- .../cncf/kubernetes/utils/test_pod_manager.py | 48 +++++++++++++++++++ 2 files changed, 89 insertions(+), 1 deletion(-) diff --git a/airflow/providers/cncf/kubernetes/utils/pod_manager.py b/airflow/providers/cncf/kubernetes/utils/pod_manager.py index 71b5e171271b..2251f1d438c0 100644 --- a/airflow/providers/cncf/kubernetes/utils/pod_manager.py +++ b/airflow/providers/cncf/kubernetes/utils/pod_manager.py @@ -545,6 +545,19 @@ def await_xcom_sidecar_container_start(self, pod: V1Pod) -> None: def extract_xcom(self, pod: V1Pod) -> str: """Retrieves XCom value and kills xcom sidecar container.""" + try: + result = self.extract_xcom_json(pod) + return result + finally: + self.extract_xcom_kill(pod) + + @tenacity.retry( + stop=tenacity.stop_after_attempt(5), + wait=tenacity.wait_exponential(multiplier=1, min=4, max=10), + reraise=True, + ) + def extract_xcom_json(self, pod: V1Pod) -> str: + """Retrieves XCom value and also checks if xcom json is valid.""" with closing( kubernetes_stream( self._client.connect_get_namespaced_pod_exec, @@ -563,11 +576,38 @@ def extract_xcom(self, pod: V1Pod) -> str: resp, f"if [ -s {PodDefaults.XCOM_MOUNT_PATH}/return.json ]; then cat {PodDefaults.XCOM_MOUNT_PATH}/return.json; else echo __airflow_xcom_result_empty__; fi", # noqa ) - self._exec_pod_command(resp, "kill -s SIGINT 1") + if result and result.rstrip() != "__airflow_xcom_result_empty__": + # Note: result string is parsed to check if its valid json. + # This function still returns a string which is converted into json in the calling method. + json.loads(result) + if result is None: raise AirflowException(f"Failed to extract xcom from pod: {pod.metadata.name}") return result + @tenacity.retry( + stop=tenacity.stop_after_attempt(5), + wait=tenacity.wait_exponential(multiplier=1, min=4, max=10), + reraise=True, + ) + def extract_xcom_kill(self, pod: V1Pod): + """Kills xcom sidecar container.""" + with closing( + kubernetes_stream( + self._client.connect_get_namespaced_pod_exec, + pod.metadata.name, + pod.metadata.namespace, + container=PodDefaults.SIDECAR_CONTAINER_NAME, + command=["/bin/sh"], + stdin=True, + stdout=True, + stderr=True, + tty=False, + _preload_content=False, + ) + ) as resp: + self._exec_pod_command(resp, "kill -s SIGINT 1") + def _exec_pod_command(self, resp, command: str) -> str | None: res = None if resp.is_open(): diff --git a/tests/providers/cncf/kubernetes/utils/test_pod_manager.py b/tests/providers/cncf/kubernetes/utils/test_pod_manager.py index a55be38a5e22..8f28d33dfdea 100644 --- a/tests/providers/cncf/kubernetes/utils/test_pod_manager.py +++ b/tests/providers/cncf/kubernetes/utils/test_pod_manager.py @@ -18,6 +18,7 @@ import logging from datetime import datetime +from json.decoder import JSONDecodeError from unittest import mock from unittest.mock import MagicMock @@ -370,6 +371,53 @@ def test_container_is_terminated_with_waiting_state(self, container_state, expec pod_info.status.container_statuses = [container_status] assert container_is_terminated(pod_info, "base") == expected_is_terminated + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.kubernetes_stream") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager._exec_pod_command") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager.extract_xcom_kill") + def test_extract_xcom_success(self, mock_exec_xcom_kill, mock_exec_pod_command, mock_kubernetes_stream): + """test when valid json is retrieved from xcom sidecar container.""" + xcom_json = """{"a": "true"}""" + mock_pod = MagicMock() + mock_exec_pod_command.return_value = xcom_json + ret = self.pod_manager.extract_xcom(pod=mock_pod) + assert ret == xcom_json + assert mock_exec_xcom_kill.call_count == 1 + + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.kubernetes_stream") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager._exec_pod_command") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager.extract_xcom_kill") + def test_extract_xcom_failure(self, mock_exec_xcom_kill, mock_exec_pod_command, mock_kubernetes_stream): + """test when invalid json is retrieved from xcom sidecar container.""" + with pytest.raises(JSONDecodeError): + xcom_json = """{"a": "tru""" + mock_pod = MagicMock() + mock_exec_pod_command.return_value = xcom_json + self.pod_manager.extract_xcom(pod=mock_pod) + assert mock_exec_xcom_kill.call_count == 1 + + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.kubernetes_stream") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager._exec_pod_command") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager.extract_xcom_kill") + def test_extract_xcom_empty(self, mock_exec_xcom_kill, mock_exec_pod_command, mock_kubernetes_stream): + """test when __airflow_xcom_result_empty__ is retrieved from xcom sidecar container.""" + mock_pod = MagicMock() + xcom_result = "__airflow_xcom_result_empty__" + mock_exec_pod_command.return_value = xcom_result + ret = self.pod_manager.extract_xcom(pod=mock_pod) + assert ret == xcom_result + assert mock_exec_xcom_kill.call_count == 1 + + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.kubernetes_stream") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager._exec_pod_command") + @mock.patch("airflow.providers.cncf.kubernetes.utils.pod_manager.PodManager.extract_xcom_kill") + def test_extract_xcom_none(self, mock_exec_xcom_kill, mock_exec_pod_command, mock_kubernetes_stream): + """test when None is retrieved from xcom sidecar container.""" + with pytest.raises(AirflowException): + mock_pod = MagicMock() + mock_exec_pod_command.return_value = None + self.pod_manager.extract_xcom(pod=mock_pod) + assert mock_exec_xcom_kill.call_count == 1 + def params_for_test_container_is_running(): """The `container_is_running` method is designed to handle an assortment of bad objects From 4d84e304b86c97d0437fddbc6b6757b5201eefcc Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 1 Jul 2023 23:41:59 +0200 Subject: [PATCH 013/533] Fix Pydantic 2 pickiness about model definition (#32307) The new Pydantic 2 has been released on 30th of June and it is a bit more picky about model definition, thus causing Airflow to fail because there were wrong definitions of models (previously corretly validated). Fixes: #32301 --- airflow/serialization/pydantic/dataset.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/serialization/pydantic/dataset.py b/airflow/serialization/pydantic/dataset.py index 5ee0d18128be..659e5a1899ca 100644 --- a/airflow/serialization/pydantic/dataset.py +++ b/airflow/serialization/pydantic/dataset.py @@ -38,10 +38,10 @@ class TaskOutletDatasetReferencePydantic(BaseModelPydantic): """Serializable version of the TaskOutletDatasetReference ORM SqlAlchemyModel used by internal API.""" dataset_id: int - dag_id = str - task_id = str - created_at = datetime - updated_at = datetime + dag_id: str + task_id: str + created_at: datetime + updated_at: datetime class Config: """Make sure it deals automatically with SQLAlchemy ORM classes.""" From f6db66e16374e504665972feba0831d4148c6d50 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 1 Jul 2023 23:56:03 +0200 Subject: [PATCH 014/533] Add information for users who ask for requirements (#32262) * Add information for users who ask for requirements This change is based on a number of discussions with the users asking what are the minimum requirements for Airflow to run. While we cannot give precise answer, we should also make the users aware that simple answers are not possible, and that when they are deciding to install airflow and manage it on their own, they also take the responsibility to monitor and adjust the resources they need based on the monitoring they have to run. * Apply suggestions from code review Co-authored-by: Pankaj Koti * Update docs/apache-airflow/installation/index.rst --------- Co-authored-by: Pankaj Koti --- .../scheduler.rst | 1 + docs/apache-airflow/installation/index.rst | 72 ++++++++++++++++++- 2 files changed, 72 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/scheduler.rst b/docs/apache-airflow/administration-and-deployment/scheduler.rst index dfc97e6120d7..1a2a41b136ec 100644 --- a/docs/apache-airflow/administration-and-deployment/scheduler.rst +++ b/docs/apache-airflow/administration-and-deployment/scheduler.rst @@ -154,6 +154,7 @@ The following databases are fully supported and provide an "optimal" experience: Microsoft SQLServer has not been tested with HA. +.. _fine-tuning-scheduler: Fine-tuning your Scheduler performance -------------------------------------- diff --git a/docs/apache-airflow/installation/index.rst b/docs/apache-airflow/installation/index.rst index 1ddbf5d66b6e..8f37ca208d78 100644 --- a/docs/apache-airflow/installation/index.rst +++ b/docs/apache-airflow/installation/index.rst @@ -77,6 +77,9 @@ More details: :doc:`installing-from-sources` * You should develop and handle the deployment for all components of Airflow. * You are responsible for setting up database, creating and managing database schema with ``airflow db`` commands, automated startup and recovery, maintenance, cleanup and upgrades of Airflow and the Airflow Providers. +* You need to setup monitoring of your system allowing you to observe resources and react to problems. +* You are expected to configure and manage appropriate resources for the installation (memory, CPU, etc) based + on the monitoring of your installation and feedback loop. See the notes about requirements. **What Apache Airflow Community provides for that method** @@ -123,6 +126,9 @@ More details: :doc:`/installation/installing-from-pypi` * You should develop and handle the deployment for all components of Airflow. * You are responsible for setting up database, creating and managing database schema with ``airflow db`` commands, automated startup and recovery, maintenance, cleanup and upgrades of Airflow and Airflow Providers. +* You need to setup monitoring of your system allowing you to observe resources and react to problems. +* You are expected to configure and manage appropriate resources for the installation (memory, CPU, etc) based + on the monitoring of your installation and feedback loop. **What Apache Airflow Community provides for that method** @@ -181,6 +187,9 @@ and official constraint files- same that are used for installing Airflow from Py deployments of containers. You can use your own custom mechanism, custom Kubernetes deployments, custom Docker Compose, custom Helm charts etc., and you should choose it based on your experience and expectations. +* You need to setup monitoring of your system allowing you to observe resources and react to problems. +* You are expected to configure and manage appropriate resources for the installation (memory, CPU, etc) based + on the monitoring of your installation and feedback loop. **What Apache Airflow Community provides for that method** @@ -238,6 +247,9 @@ More details: :doc:`helm-chart:index` those changes when released by upgrading the base image. However, you are responsible in creating a pipeline of building your own custom images with your own added dependencies and Providers and need to repeat the customization step and building your own image when new version of Airflow image is released. +* You need to setup monitoring of your system allowing you to observe resources and react to problems. +* You are expected to configure and manage appropriate resources for the installation (memory, CPU, etc) based + on the monitoring of your installation and feedback loop. **What Apache Airflow Community provides for that method** @@ -256,7 +268,6 @@ More details: :doc:`helm-chart:index` * If you can provide description of a reproducible problem with Airflow software, you can open issue at `GitHub issues `__ - Using Managed Airflow Services '''''''''''''''''''''''''''''' @@ -316,3 +327,62 @@ Follow the `Ecosystem `__ page to find a **Where to ask for help** * Depends on what the 3rd-party provides. Look at the documentation of the 3rd-party deployment you use. + + +Notes about minimum requirements +'''''''''''''''''''''''''''''''' + +There are often questions about minimum requirements for Airflow for production systems, but it is +not possible to give a simple answer to that question. + +The requirements that Airflow might need depend on many factors, including (but not limited to): + * The deployment your Airflow is installed with (see above ways of installing Airflow) + * The requirements of the deployment environment (for example Kubernetes, Docker, Helm, etc.) that + are completely independent from Airflow (for example DNS resources, sharing the nodes/resources) + with more (or less) pods and containers that are needed that might depend on particular choice of + the technology/cloud/integration of monitoring etc. + * Technical details of database, hardware, network, etc. that your deployment is running on + * The complexity of the code you add to your DAGS, configuration, plugins, settings etc. (note, that + Airflow runs the code that DAG author and Deployment Manager provide) + * The number and choice of providers you install and use (Airflow has more than 80 providers) that can + be installed by choice of the Deployment Manager and using them might require more resources. + * The choice of parameters that you use when tuning Airflow. Airflow has many configuration parameters + that can fine-tuned to your needs + * The number of DagRuns and tasks instances you run with parallel instances of each in consideration + * How complex are the tasks you run + +The above "DAG" characteristics will change over time and even will change depending on the time of the day +or week, so you have to be prepared to continuously monitor the system and adjust the parameters to make +it works smoothly. + +While we can provide some specific minimum requirements for some development "quick start" - such as +in case of our :ref:`running-airflow-in-docker` quick-start guide, it is not possible to provide any minimum +requirements for production systems. + +The best way to think of resource allocation for Airflow instance is to think of it in terms of process +control theory - where there are two types of systems: + +1. Fully predictable, with few knobs and variables, where you can reliably set the values for the + knobs and have an easy way to determine the behaviour of the system + +2. Complex systems with multiple variables, that are hard to predict and where you need to monitor + the system and adjust the knobs continuously to make sure the system is running smoothly. + +Airflow (and generally any modern system running usually on cloud services, with multiple layers responsible +for resources as well multiple parameters to control their behaviour) is a complex system and they fall +much more in the second category. If you decide to run Airflow in production on your own, you should be +prepared for the monitor/observe/adjust feedback loop to make sure the system is running smoothly. + +Having a good monitoring system that will allow you to monitor the system and adjust the parameters +is a must to put that in practice. + +There are few guidelines that you can use for optimizing your resource usage as well. The +:ref:`fine-tuning-scheduler` is a good starting point to fine-tune your scheduler, you can also follow +the :ref:`best_practice` guide to make sure you are using Airflow in the most efficient way. + +Also, one of the important things that Managed Services for Airflow provide is that they make a lot +of opinionated choices and fine-tune the system for you, so you don't have to worry about it too much. +With such managed services, there are usually far less number of knobs to turn and choices to make and one +of the things you pay for is that the Managed Service provider manages the system for you and provides +paid support and allows you to scale the system as needed and allocate the right resources - following the +choices made there when it comes to the kinds of deployment you might have. From 4efbcdc8fc48e929f18fbe75c61a5125072f71fc Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 1 Jul 2023 23:56:33 +0200 Subject: [PATCH 015/533] Update security model of Airflow (#32098) * Update security model of Airflow This change updates the security model of Airflow to better explain what are the capabilities of various kinds of users in Airflow deployments and give both users and security researchers a way to understand what security measures they can take and whether they can qualify potential security issues in Airflow properly - taking into account that various users of Airflow have various capabilities and behaviours considered by some of the users as security vulnerabilities, are standard capabilities of the users. It also splits the security information of ours in two separate pages: * .github/SECURITY.md where we explain how to report the issues to Apache Airflow security team by the researchers * documentation security/index.html which is available via Airflow Website where we explain what our security model is and the different kinds of users we have. Both serve slightly different purpose and both contain cross-reference links to each other in order to be able to redirect people who read about the security model to find out how they can report the issues but also to guide security researchers who want to assess whether their findings are real vulnerabilities, or rather normal behaviours following the Airflow Security model. Security has been also moved to be a top level topic, so that it is easier to find and navigate to. Old links have been redirected to the new locations. Also chapters were added explaining Airflow vs. Providers security releases, what is the relation between Airflow and Providers security issues and how users should treat security announcements in providers. * Update .github/SECURITY.md Co-authored-by: Pankaj Koti * Apply Niko's suggestions from code review Co-authored-by: Niko Oliveira * fixup! Apply Niko's suggestions from code review --------- Co-authored-by: Pankaj Koti Co-authored-by: Niko Oliveira --- .github/{SECURITY.rst => SECURITY.md} | 115 ++++++------ .github/boring-cyborg.yml | 2 +- .pre-commit-config.yaml | 2 +- .../core-extensions/auth-backends.rst | 2 +- .../core-extensions/secrets-backends.rst | 2 +- .../administration-and-deployment/index.rst | 1 - .../security/index.rst | 28 --- docs/apache-airflow/howto/connection.rst | 6 +- docs/apache-airflow/howto/variable.rst | 2 +- docs/apache-airflow/index.rst | 1 + .../installation/installing-from-pypi.rst | 1 + docs/apache-airflow/integration.rst | 8 +- .../public-airflow-interface.rst | 2 +- docs/apache-airflow/redirects.txt | 35 ++-- .../security/access-control.rst | 0 .../security/api.rst | 2 +- .../security/audit_logs.rst | 0 .../security/flower.rst | 0 docs/apache-airflow/security/index.rst | 164 ++++++++++++++++++ .../security/kerberos.rst | 0 .../security/secrets/fernet.rst | 0 .../security/secrets/index.rst | 0 .../secrets/mask-sensitive-values.rst | 0 .../secrets/secrets-backend/index.rst | 0 .../local-filesystem-secrets-backend.rst | 0 .../security/webserver.rst | 2 +- .../security/workload.rst | 0 27 files changed, 254 insertions(+), 121 deletions(-) rename .github/{SECURITY.rst => SECURITY.md} (53%) delete mode 100644 docs/apache-airflow/administration-and-deployment/security/index.rst rename docs/apache-airflow/{administration-and-deployment => }/security/access-control.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/api.rst (98%) rename docs/apache-airflow/{administration-and-deployment => }/security/audit_logs.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/flower.rst (100%) create mode 100644 docs/apache-airflow/security/index.rst rename docs/apache-airflow/{administration-and-deployment => }/security/kerberos.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/secrets/fernet.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/secrets/index.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/secrets/mask-sensitive-values.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/secrets/secrets-backend/index.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst (100%) rename docs/apache-airflow/{administration-and-deployment => }/security/webserver.rst (99%) rename docs/apache-airflow/{administration-and-deployment => }/security/workload.rst (100%) diff --git a/.github/SECURITY.rst b/.github/SECURITY.md similarity index 53% rename from .github/SECURITY.rst rename to .github/SECURITY.md index e7f5a40366db..950ab7f44521 100644 --- a/.github/SECURITY.rst +++ b/.github/SECURITY.md @@ -1,37 +1,28 @@ - .. 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. - -Security Model --------------- - -In the Airflow security model, the system administrators are fully trusted. -They are the only ones who can upload new DAGs, which gives them the ability -to execute any code on the server. - -Authenticated web interface and API users with Admin/Op permissions are trusted, -but to a lesser extent: they can configure the DAGs which gives them some control, -but not arbitrary code execution. - -Authenticated Web interface and API users with 'regular' permissions are trusted -to the point where they can impact resource consumption and pause/unpause configured DAGs, -but not otherwise influence their functionality. - -Reporting Vulnerabilities -------------------------- + + +This document contains information on how to report security vulnerabilities in Apache Airflow and +how the security issues reported to Apache Airflow security team are handled. If you would like +to learn about the security model of Airflow head to +[Airflow Security](https://airflow.apache.org/docs/apache-airflow/stable/security/) + +## Reporting Vulnerabilities **⚠️ Please do not file GitHub issues for security vulnerabilities as they are public! ⚠️** @@ -39,9 +30,9 @@ The Apache Software Foundation takes security issues very seriously. Apache Airflow specifically offers security features and is responsive to issues around its features. If you have any concern around Airflow Security or believe you have uncovered a vulnerability, we suggest that you get in touch via the -e-mail address security@airflow.apache.org. In the message, try to provide a -description of the issue and ideally a way of reproducing it. The security team -will get back to you after assessing the description. +e-mail address [security@airflow.apache.org](mailto:security@airflow.apache.org). +In the message, try to provide a description of the issue and ideally a way of +reproducing it. The security team will get back to you after assessing the report. Note that this security address should be used only for undisclosed vulnerabilities. Dealing with fixed issues or general questions on how to use @@ -49,13 +40,30 @@ the security features should be handled regularly via the user and the dev lists. Please report any security problems to the project security address before disclosing it publicly. -The `ASF Security team's page `_ describes -how vulnerability reports are handled, and includes PGP keys if you wish to use -that. +Before reporting vulnerabilities, please make sure to read and understand the +[security model](https://airflow.apache.org/docs/apache-airflow/stable/security/) of Airflow, because +some of the potential security vulnerabilities that are valid for projects that are publicly accessible +from the Internet, are not valid for Airflow. Airflow is not designed to be used by untrusted users, and some +trusted users are trusted enough to do a variety of operations that could be considered as vulnerabilities +in other products/circumstances. Therefore, some potential security vulnerabilities do not +apply to Airflow, or have a different severity than some generic scoring systems (for example `CVSS`) +calculation suggests. +The [ASF Security team's page](https://www.apache.org/security/) describes +how vulnerability reports are handled in general by all ASF projects, and includes PGP keys if +you wish to use them when you report the issues. -Handling security issues in Airflow ------------------------------------ +## Security vulnerabilities in Airflow and Airflow community managed providers + +Airflow core package is released separately from provider packages. While Airflow comes with ``constraints`` +which describe which version of providers have been tested when the version of Airflow was released, the +users of Airflow are advised to install providers independently from Airflow core when they want to apply +security fixes found and released in providers. Therefore, the issues found and fixed in providers do +not apply to the Airflow core package. There are also Airflow providers released by 3rd-parties, but the +Airflow community is not responsible for releasing and announcing security vulnerabilities in them, this +is handled entirely by the 3rd-parties that release their own providers. + +## Handling security issues in Airflow The security issues in Airflow are handled by the Airflow Security Team. The team consists of selected PMC members that are interested in looking at, discussing about and fixing the @@ -80,7 +88,7 @@ There are certain expectations from the members of the security team: experts that are available through Airflow stakeholders. The intent about involving 3rd parties has to be discussed and agreed up at security@airflow.apache.org. -* They have to have an `ICLA `_ signed with +* They have to have an [ICLA](https://www.apache.org/licenses/contributor-agreements.html) signed with Apache Software Foundation. * The security team members might inform 3rd parties about fixes, for example in order to assess if the fix @@ -92,7 +100,7 @@ There are certain expectations from the members of the security team: with the intent of minimizing the time between the fix being available and the fix being released. In this case the PR might be sent to review and comment to the PMC members on private list, in order to request an expedited voting on the release. The voting for such release might be done on the - ``private@airflow.apache.org`` mailing list and should be made public at the ``dev@apache.airflow.org`` + `private@airflow.apache.org` mailing list and should be made public at the `dev@apache.airflow.org` mailing list as soon as the release is ready to be announced. * The security team members working on the fix might be mentioned as remediation developers in the CVE @@ -104,22 +112,5 @@ There are certain expectations from the members of the security team: release process. This is facilitated by the security tool provided by the Apache Software Foundation. * Severity of the issue is determined based on the criteria described in the - `Severity Rating blog post `_ by the Apache Software + [Severity Rating blog post](https://security.apache.org/blog/severityrating/) by the Apache Software Foundation Security team - -Releasing Airflow with security patches ---------------------------------------- - -Apache Airflow uses strict `SemVer `_ versioning policy, which means that we strive for -any release of a given ``MAJOR`` Version (version "2" currently) to be backwards compatible. When we -release ``MINOR`` version, the development continues in the ``main`` branch where we prepare the next -``MINOR`` version, but we release ``PATCHLEVEL`` releases with selected bugfixes (including security -bugfixes) cherry-picked to the latest released ``MINOR`` line of Apache Airflow. At the moment, when we -release a new ``MINOR`` version, we stop releasing ``PATCHLEVEL`` releases for the previous ``MINOR`` version. - -For example, when we released ``2.6.0`` version on April 30, 2023, until we release ``2.7.0`` version, -all the security patches will be cherry-picked and released in ``2.6.*`` versions only. There will be no -``2.5.*`` versions released after ``2.6.0`` has been released. - -This means that in order to apply security fixes with Apache Airflow software released by us, you -MUST upgrade to the latest ``MINOR`` version of Airflow. diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index b55d145ad6a5..e364b37984a5 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -270,7 +270,7 @@ labelPRBasedOnFilePath: - airflow/providers/**/secrets/* - tests/secrets/**/* - tests/providers/**/secrets/* - - docs/apache-airflow/administration-and-deployment/security/secrets/**/* + - docs/apache-airflow/security/secrets/**/* area:Triggerer: - airflow/cli/commands/triggerer_command.py diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index f8371431aa53..55129ceaa117 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -495,7 +495,7 @@ repos: ^airflow/providers/microsoft/winrm/hooks/winrm.py$| ^airflow/www/fab_security/manager.py$| ^docs/.*commits.rst$| - ^docs/apache-airflow/administration-and-deployment/security/webserver.rst$| + ^docs/apache-airflow/security/webserver.rst$| ^docs/apache-airflow-providers-apache-cassandra/connections/cassandra.rst$| ^airflow/providers/microsoft/winrm/operators/winrm.py$| ^airflow/providers/opsgenie/hooks/opsgenie.py$| diff --git a/docs/apache-airflow-providers/core-extensions/auth-backends.rst b/docs/apache-airflow-providers/core-extensions/auth-backends.rst index edb38bb882f4..325b0c2819a2 100644 --- a/docs/apache-airflow-providers/core-extensions/auth-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/auth-backends.rst @@ -26,7 +26,7 @@ capabilities. You can read more about those in `FAB security docs `_. You can also -take a look at Auth backends available in the core Airflow in :doc:`apache-airflow:administration-and-deployment/security/webserver` +take a look at Auth backends available in the core Airflow in :doc:`apache-airflow:security/webserver` or see those provided by the community-managed providers: .. airflow-auth-backends:: diff --git a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst index 22fae4b8a6c4..26ee3ce882a0 100644 --- a/docs/apache-airflow-providers/core-extensions/secrets-backends.rst +++ b/docs/apache-airflow-providers/core-extensions/secrets-backends.rst @@ -28,7 +28,7 @@ via providers that implement secrets backends for services Airflow integrates wi You can also take a look at Secret backends available in the core Airflow in -:doc:`apache-airflow:administration-and-deployment/security/secrets/secrets-backend/index` and here you can see the ones +:doc:`apache-airflow:security/secrets/secrets-backend/index` and here you can see the ones provided by the community-managed providers: .. airflow-secrets-backends:: diff --git a/docs/apache-airflow/administration-and-deployment/index.rst b/docs/apache-airflow/administration-and-deployment/index.rst index 1acb9d8e3a90..5ba79a5439ba 100644 --- a/docs/apache-airflow/administration-and-deployment/index.rst +++ b/docs/apache-airflow/administration-and-deployment/index.rst @@ -24,7 +24,6 @@ This section contains information about deploying DAGs into production and the a :maxdepth: 2 production-deployment - security/index logging-monitoring/index kubernetes lineage diff --git a/docs/apache-airflow/administration-and-deployment/security/index.rst b/docs/apache-airflow/administration-and-deployment/security/index.rst deleted file mode 100644 index 65d02f7f536b..000000000000 --- a/docs/apache-airflow/administration-and-deployment/security/index.rst +++ /dev/null @@ -1,28 +0,0 @@ - .. Licensed to the Apache Software Foundation (ASF) under one - or more contributor license agreements. See the NOTICE file - distributed with this work for additional information - regarding copyright ownership. The ASF licenses this file - to you under the Apache License, Version 2.0 (the - "License"); you may not use this file except in compliance - with the License. You may obtain a copy of the License at - - .. http://www.apache.org/licenses/LICENSE-2.0 - - .. Unless required by applicable law or agreed to in writing, - software distributed under the License is distributed on an - "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied. See the License for the - specific language governing permissions and limitations - under the License. - -Security -======== - -.. toctree:: - :maxdepth: 1 - :glob: - - * - secrets/index - -.. include:: /../../.github/SECURITY.rst diff --git a/docs/apache-airflow/howto/connection.rst b/docs/apache-airflow/howto/connection.rst index b43094a88bb3..0db100fd2974 100644 --- a/docs/apache-airflow/howto/connection.rst +++ b/docs/apache-airflow/howto/connection.rst @@ -27,7 +27,7 @@ Airflow's :class:`~airflow.models.connection.Connection` object is used for stor Connections may be defined in the following ways: - in :ref:`environment variables ` - - in an external :doc:`/administration-and-deployment/security/secrets/secrets-backend/index` + - in an external :doc:`/security/secrets/secrets-backend/index` - in the :ref:`Airflow metadata database ` (using the :ref:`CLI ` or :ref:`web UI `) @@ -86,7 +86,7 @@ See :ref:`Connection URI format ` for more details on how Storing connections in a Secrets Backend ---------------------------------------- -You can store Airflow connections in external secrets backends like HashiCorp Vault, AWS SSM Parameter Store, and other such services. For more details see :doc:`/administration-and-deployment/security/secrets/secrets-backend/index`. +You can store Airflow connections in external secrets backends like HashiCorp Vault, AWS SSM Parameter Store, and other such services. For more details see :doc:`/security/secrets/secrets-backend/index`. .. _connections-in-database: @@ -94,7 +94,7 @@ Storing connections in the database ----------------------------------- .. seealso:: - Connections can alternatively be stored in :ref:`environment variables ` or an :doc:`external secrets backend ` such as HashiCorp Vault, AWS SSM Parameter Store, etc. + Connections can alternatively be stored in :ref:`environment variables ` or an :doc:`external secrets backend ` such as HashiCorp Vault, AWS SSM Parameter Store, etc. When storing connections in the database, you may manage them using either the web UI or the Airflow CLI. diff --git a/docs/apache-airflow/howto/variable.rst b/docs/apache-airflow/howto/variable.rst index a9e0d82023f9..b4b395dd63c2 100644 --- a/docs/apache-airflow/howto/variable.rst +++ b/docs/apache-airflow/howto/variable.rst @@ -73,4 +73,4 @@ It guarantees that without the encryption password, content cannot be manipulate without the key. For information on configuring Fernet, look at :ref:`security/fernet`. In addition to retrieving variables from environment variables or the metastore database, you can enable -a secrets backend to retrieve variables. For more details see :doc:`/administration-and-deployment/security/secrets/secrets-backend/index`. +a secrets backend to retrieve variables. For more details see :doc:`/security/secrets/secrets-backend/index`. diff --git a/docs/apache-airflow/index.rst b/docs/apache-airflow/index.rst index 548ced7b076a..feaf2d388b50 100644 --- a/docs/apache-airflow/index.rst +++ b/docs/apache-airflow/index.rst @@ -133,6 +133,7 @@ so coding will always be required. Overview start installation/index + security/index tutorial/index howto/index ui diff --git a/docs/apache-airflow/installation/installing-from-pypi.rst b/docs/apache-airflow/installation/installing-from-pypi.rst index 975edb5d3f16..ed345f0cc1f3 100644 --- a/docs/apache-airflow/installation/installing-from-pypi.rst +++ b/docs/apache-airflow/installation/installing-from-pypi.rst @@ -272,6 +272,7 @@ released and tested together when the version of Airflow you are installing was CONSTRAINT_URL="https://raw.githubusercontent.com/apache/airflow/constraints-${AIRFLOW_VERSION}/constraints-${PYTHON_VERSION}.txt" pip install "apache-airflow[postgres,google]==${AIRFLOW_VERSION}" --constraint "${CONSTRAINT_URL}" +.. _installing-from-pypi-managing-providers-separately-from-airflow-core: Managing providers separately from Airflow core =============================================== diff --git a/docs/apache-airflow/integration.rst b/docs/apache-airflow/integration.rst index 3f477911048a..2e9c450f1fa8 100644 --- a/docs/apache-airflow/integration.rst +++ b/docs/apache-airflow/integration.rst @@ -20,18 +20,18 @@ Integration Airflow has a mechanism that allows you to expand its functionality and integrate with other systems. -* :doc:`API Authentication backends ` +* :doc:`API Authentication backends ` * :doc:`Email backends ` * :doc:`Executor ` -* :doc:`Kerberos ` +* :doc:`Kerberos ` * :doc:`Logging ` * :doc:`Metrics (statsd) ` * :doc:`Operators and hooks ` * :doc:`Plugins ` * :doc:`Listeners ` -* :doc:`Secrets backends ` +* :doc:`Secrets backends ` * :doc:`Tracking systems ` -* :doc:`Web UI Authentication backends ` +* :doc:`Web UI Authentication backends ` * :doc:`Serialization ` It also has integration with :doc:`Sentry ` service for error tracking. Other applications can also integrate using diff --git a/docs/apache-airflow/public-airflow-interface.rst b/docs/apache-airflow/public-airflow-interface.rst index 74d86572446d..5f0f51163873 100644 --- a/docs/apache-airflow/public-airflow-interface.rst +++ b/docs/apache-airflow/public-airflow-interface.rst @@ -339,7 +339,7 @@ All Secrets Backend implementations are public. You can extend their functionali _api/airflow/secrets/index -You can read more about Secret Backends in :doc:`administration-and-deployment/security/secrets/secrets-backend/index`. +You can read more about Secret Backends in :doc:`security/secrets/secrets-backend/index`. You can also find all the available Secrets Backends implemented in community providers in :doc:`apache-airflow-providers:core-extensions/secrets-backends`. diff --git a/docs/apache-airflow/redirects.txt b/docs/apache-airflow/redirects.txt index b99a5b253cbc..9d8a5527fbbd 100644 --- a/docs/apache-airflow/redirects.txt +++ b/docs/apache-airflow/redirects.txt @@ -15,9 +15,26 @@ # specific language governing permissions and limitations # under the License. +# Administration and deployment security -> security + +administration-and-deployment/security/index.rst security/index.rst +administration-and-deployment/security/kerberos.rst security/kerberos.rst +administration-and-deployment/security/access-control.rst security/access-control.rst +administration-and-deployment/security/access-control/index.rst security/access-control.rst +administration-and-deployment/security/api.rst security/api.rst +administration-and-deployment/security/audit_logs.rst security/audit_logs.rst +administration-and-deployment/security/flower.rst security/flower.rst +administration-and-deployment/security/webserver.rst security/webserver.rst +administration-and-deployment/security/workload.rst security/workload.rst +administration-and-deployment/security/secrets/secrets-backends/index.rst security/secrets/secrets-backends/index.rst +administration-and-deployment/security/secrets/secrets-backends/local-filesystem-secrets-backend.rst security/secrets/secrets-backends/local-filesystem-secrets-backend.rst +administration-and-deployment/security/secrets/fernet.rst security/secrets/fernet.rst +administration-and-deployment/security/secrets/index.rst security/secrets/index.rst +administration-and-deployment/security/secrets/mask-sensitive-values.rst security/secrets/mask-sensitive-values.rst + # Security -howto/use-alternative-secrets-backend.rst administration-and-deployment/security/secrets/secrets-backend/index.rst -security.rst administration-and-deployment/security/index.rst +howto/use-alternative-secrets-backend.rst security/secrets/secrets-backend/index.rst +security.rst security/index.rst # Operators guides howto/operator/external.rst howto/operator/external_task_sensor.rst @@ -27,7 +44,7 @@ howto/customize-dag-ui-page-instance-name.rst howto/customize-ui.rst#customizing howto/customize-state-colors-ui.rst howto/customize-ui.rst#customizing-state-colours # Web UI -howto/add-new-role.rst administration-and-deployment/security/access-control.rst +howto/add-new-role.rst security/access-control.rst # Set up a database howto/initialize-database.rst howto/set-up-database.rst @@ -68,9 +85,6 @@ tutorial_taskflow_api.rst tutorial/taskflow.rst ## Docs Structure Refactor # indexes -security/index.rst administration-and-deployment/security/index.rst -security/secrets/index.rst administration-and-deployment/security/secrets/index.rst -security/secrets/secrets-backend/index.rst administration-and-deployment/security/secrets/secrets-backend/index.rst logging-monitoring/index.rst administration-and-deployment/logging-monitoring/index.rst concepts/index.rst core-concepts/index.rst executor/index.rst core-concepts/executor/index.rst @@ -88,15 +102,6 @@ usage-cli.rst howto/usage-cli.rst lineage.rst administration-and-deployment/lineage.rst dag-run.rst core-concepts/dag-run.rst concepts/taskflow.rst core-concepts/taskflow.rst -security/kerberos.rst administration-and-deployment/security/kerberos.rst -security/workload.rst administration-and-deployment/security/workload.rst -security/flower.rst administration-and-deployment/security/flower.rst -security/webserver.rst administration-and-deployment/security/webserver.rst -security/api.rst administration-and-deployment/security/api.rst -security/access-control.rst administration-and-deployment/security/access-control.rst -security/secrets/fernet.rst administration-and-deployment/security/secrets/fernet.rst -security/secrets/mask-sensitive-values.rst administration-and-deployment/security/secrets/mask-sensitive-values.rst -security/secrets/secrets-backend/local-filesystem-secrets-backend.rst administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst logging-monitoring/tracking-user-activity.rst administration-and-deployment/logging-monitoring/tracking-user-activity.rst logging-monitoring/errors.rst administration-and-deployment/logging-monitoring/errors.rst logging-monitoring/logging-architecture.rst administration-and-deployment/logging-monitoring/logging-architecture.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/access-control.rst b/docs/apache-airflow/security/access-control.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/access-control.rst rename to docs/apache-airflow/security/access-control.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/api.rst b/docs/apache-airflow/security/api.rst similarity index 98% rename from docs/apache-airflow/administration-and-deployment/security/api.rst rename to docs/apache-airflow/security/api.rst index 6551ef928ff5..61319e21c62b 100644 --- a/docs/apache-airflow/administration-and-deployment/security/api.rst +++ b/docs/apache-airflow/security/api.rst @@ -63,7 +63,7 @@ If you wish to have the experimental API work, and aware of the risks of enablin You can only disable authentication for experimental API, not the stable REST API. -See :doc:`../modules_management` for details on how Python and Airflow manage modules. +See :doc:`../administration-and-deployment/modules_management` for details on how Python and Airflow manage modules. Kerberos authentication ''''''''''''''''''''''' diff --git a/docs/apache-airflow/administration-and-deployment/security/audit_logs.rst b/docs/apache-airflow/security/audit_logs.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/audit_logs.rst rename to docs/apache-airflow/security/audit_logs.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/flower.rst b/docs/apache-airflow/security/flower.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/flower.rst rename to docs/apache-airflow/security/flower.rst diff --git a/docs/apache-airflow/security/index.rst b/docs/apache-airflow/security/index.rst new file mode 100644 index 000000000000..3573348f1006 --- /dev/null +++ b/docs/apache-airflow/security/index.rst @@ -0,0 +1,164 @@ + .. 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. + +Security +======== + +.. toctree:: + :maxdepth: 1 + :glob: + + * + secrets/index + + +This document describes Airflow's security model from the perspective of +the Airflow user. It is intended to help users understand the security +model and make informed decisions about how to deploy and manage Airflow. +If you would like to know how to report security vulnerabilities and how +security reports are handled by the security team of Airflow, head to +`Airflow's Security Policy `_. + +Airflow security model - user types +----------------------------------- + +The Airflow security model involves different types of users with +varying access and capabilities: + +1. **Deployment Managers**: They have the highest level of access and + control. They install and configure Airflow, make decisions about + technologies and permissions. They can potentially delete the entire + installation and have access to all credentials. Deployment Managers + can also decide to keep audits, backups and copies of information + outside of Airflow, which are not covered by Airflow's security + model. + +2. **DAG Authors**: They can upload, modify, and delete DAG files. The + code in DAG files is executed on workers. Therefore, DAG authors can create + and change code executed on workers and potentially access the credentials + that DAG code uses to access external systems. DAG Authors have full access + to the metadata database and internal audit logs. + +3. **Authenticated UI users**: They have access to the UI and API. Admin + users can manage permissions and execute code on workers. Connection + configuration users can configure connections and execute code on + workers. Operations users have access to DAG execution status. Trust + is crucial to prevent abuse and Denial of Service attacks. + +4. **Non-authenticated UI users**: Airflow doesn't support + unauthenticated users by default. If allowed, vulnerabilities must be + addressed by the Deployment Manager. + +Capabilities of authenticated UI users +-------------------------------------- + +The capabilities of **Authenticated UI users** can vary depending on +what roles have been configured by the Deployment Manager or Admin users as well as what permissions those roles have. Permissions on roles can be scoped as tightly as a single DAG, for example, or as broad as Admin. Below are three general categories to help conceptualize some of the capabilities authenticated users may have: + +1. **Admin users**: They manage and grant permissions to other users, + with full access to all UI capabilities. They can potentially execute + code on workers by configuring connections and need to be trusted not + to abuse these privileges. They have access to sensitive credentials + and can modify them. By default, they don't have access to + system-level configuration. They should be trusted not to misuse + sensitive information accessible through connection configuration. + They also have the ability to create a Webserver Denial of Service + situation and should be trusted not to misuse this capability. + +2. **Connection configuration users**: They configure connections and + potentially execute code on workers during DAG execution. Trust is + required to prevent misuse of these privileges. They have full access + to sensitive credentials stored in connections and can modify them. + Access to sensitive information through connection configuration + should be trusted not to be abused. They also have the ability to + create a Webserver Denial of Service situation and should be trusted + not to misuse this capability. + +3. **Operations users**: They have access to DAG execution status via + the UI. Currently, Airflow lacks full protection for accessing groups + of DAGs' history and execution. They can perform actions such as + clearing, re-running, triggering DAGs, and changing parameters. + Depending on access restrictions, they may also have access to + editing variables and viewing Airflow configuration. They should not + have access to sensitive system-level information or connections, and + they should not be able to access sensitive task information unless + deliberately exposed in logs by DAG authors. They should be trusted + not to abuse their privileges, as they can potentially overload the + server and cause Denial of Service situations. + +Responsibilities of Deployment Managers +--------------------------------------- + +Deployment Managers determine access levels and understand the potential +damage users can cause. Some Deployment Managers may further limit +access through fine-grained privileges for the **Authenticated UI +users**. However, these limitations are outside the basic Airflow's +security model and are at the discretion of Deployment Managers. +Examples of fine-grained access control include (but are not limited +to): + +- Limiting login permissions: Restricting the accounts that users can + log in with, allowing only specific accounts or roles belonging to + access the Airflow system. + +- Access restrictions to views or DAGs: Controlling user access to + certain views or specific DAGs, ensuring that users can only view or + interact with authorized components. + +- Implementing static code analysis and code review: Introducing + processes such as static code analysis and code review as part of the + DAG submission pipeline. This helps enforce code quality standards, + security checks, and adherence to best practices before DAGs are + deployed. + +These examples showcase ways in which administrators can refine and +limit user privileges within Airflow, providing tighter control and +ensuring that users have access only to the necessary components and +functionalities based on their roles and responsibilities, however the +fine-grained access control does not provide full isolation and +separation of access allowing isolation of different user groups in a +multi-tenant fashion yet. In the future versions of Airflow some of the +fine-grained access features might become part of the Airflow security +model. The Airflow community is working on a multi-tenant model that might +address some of the fine-grained access components. + + +Releasing Airflow with security patches +--------------------------------------- + +Apache Airflow uses a strict [SemVer](https://semver.org) versioning policy, which means that we strive for +any release of a given ``MAJOR`` Version (version "2" currently) to be backwards compatible. When we +release a ``MINOR`` version, the development continues in the ``main`` branch where we prepare the next +``MINOR`` version, but we release ``PATCHLEVEL`` releases with selected bugfixes (including security +bugfixes) cherry-picked to the latest released ``MINOR`` line of Apache Airflow. At the moment, when we +release a new ``MINOR`` version, we stop releasing ``PATCHLEVEL`` releases for the previous ``MINOR`` version. + +For example, once we released ``2.6.0`` version on April 30, 2023 all the security patches will be cherry-picked and released in ``2.6.*`` versions until we release ``2.7.0`` version. There will be no +``2.5.*`` versions released after ``2.6.0`` has been released. + +This means that in order to apply security fixes with Apache Airflow software released by us, you +MUST upgrade to the latest ``MINOR`` version of Airflow. + +Releasing Airflow providers with security patches +------------------------------------------------- + +Similarly to Airflow, providers use strict [SemVer](https://semver.org) versioning policy, and the same +policies apply for providers as for Airflow itself. This means that you need to upgrade to the latest +``MINOR`` version of the provider to get the latest security fixes. +Airflow providers are released independently from Airflow itself and the information about vulnerabilities +is published separately. You can upgrade providers independently from Airflow itself, following the +instructions found in :ref:`installing-from-pypi-managing-providers-separately-from-airflow-core`. diff --git a/docs/apache-airflow/administration-and-deployment/security/kerberos.rst b/docs/apache-airflow/security/kerberos.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/kerberos.rst rename to docs/apache-airflow/security/kerberos.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/secrets/fernet.rst b/docs/apache-airflow/security/secrets/fernet.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/secrets/fernet.rst rename to docs/apache-airflow/security/secrets/fernet.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/secrets/index.rst b/docs/apache-airflow/security/secrets/index.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/secrets/index.rst rename to docs/apache-airflow/security/secrets/index.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/secrets/mask-sensitive-values.rst b/docs/apache-airflow/security/secrets/mask-sensitive-values.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/secrets/mask-sensitive-values.rst rename to docs/apache-airflow/security/secrets/mask-sensitive-values.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/index.rst b/docs/apache-airflow/security/secrets/secrets-backend/index.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/index.rst rename to docs/apache-airflow/security/secrets/secrets-backend/index.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst b/docs/apache-airflow/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst rename to docs/apache-airflow/security/secrets/secrets-backend/local-filesystem-secrets-backend.rst diff --git a/docs/apache-airflow/administration-and-deployment/security/webserver.rst b/docs/apache-airflow/security/webserver.rst similarity index 99% rename from docs/apache-airflow/administration-and-deployment/security/webserver.rst rename to docs/apache-airflow/security/webserver.rst index d5e551f8b15f..71ad85fb3a9b 100644 --- a/docs/apache-airflow/administration-and-deployment/security/webserver.rst +++ b/docs/apache-airflow/security/webserver.rst @@ -74,7 +74,7 @@ user will have by default: AUTH_ROLE_PUBLIC = 'Admin' -Be sure to checkout :doc:`/administration-and-deployment/security/api` for securing the API. +Be sure to checkout :doc:`/security/api` for securing the API. .. note:: diff --git a/docs/apache-airflow/administration-and-deployment/security/workload.rst b/docs/apache-airflow/security/workload.rst similarity index 100% rename from docs/apache-airflow/administration-and-deployment/security/workload.rst rename to docs/apache-airflow/security/workload.rst From 2811ba710991d71466afa21c500ee55aeda5d8a1 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 2 Jul 2023 08:40:57 +0200 Subject: [PATCH 016/533] Limit Pydantic to < 2.0.0 until we solve 2.0.0 incompatibilities (#32312) The Pydantic 2.0.0 release on 30th of June broke a number of CI tests (including some K8S tests). We limit it temporarily until the incompatibilities are fixed. Related: #32311 --- setup.cfg | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/setup.cfg b/setup.cfg index d5d91d3d8ff6..91d3b86331de 100644 --- a/setup.cfg +++ b/setup.cfg @@ -126,7 +126,9 @@ install_requires = pendulum>=2.0 pluggy>=1.0 psutil>=4.2.0 - pydantic>=1.10.0 + # Limit Pydantic to <2.0.0 because it breaks Kubernetes tests and building providers + # This limit should be removed after https://github.com/apache/airflow/issues/32311 is fixed + pydantic>=1.10.0,<2.0.0 pygments>=2.0.1 pyjwt>=2.0.0 python-daemon>=3.0.0 From 05bd90f563649f2e9c8f0c85cf5838315a665a02 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Sun, 2 Jul 2023 12:31:13 +0100 Subject: [PATCH 017/533] Sanitize `DagRun.run_id` and allow flexibility (#32293) This commit sanitizes the DagRun.run_id parameter by introducing a configurable option. Users now have the ability to select a specific run_id pattern for their runs, ensuring stricter control over the values used. This update does not impact the default run_id generation performed by the scheduler for scheduled DAG runs or for Dag runs triggered without modifying the run_id parameter in the run configuration page. The configuration flexibility empowers users to align the run_id pattern with their specific requirements. --- airflow/config_templates/config.yml | 9 ++++ airflow/config_templates/default_airflow.cfg | 5 ++ airflow/models/dag.py | 55 +++++++++++--------- airflow/models/dagrun.py | 16 +++++- airflow/www/views.py | 32 ++++++++---- tests/models/test_dagrun.py | 32 ++++++++++++ tests/www/views/test_views_trigger_dag.py | 30 +++++++++++ 7 files changed, 143 insertions(+), 36 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index a252716ed1d0..f48a569b545f 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2555,6 +2555,15 @@ scheduler: type: float example: ~ default: "120.0" + allowed_run_id_pattern: + description: | + The run_id pattern used to verify the validity of user input to the run_id parameter when + triggering a DAG. This pattern cannot change the pattern used by scheduler to generate run_id + for scheduled DAG runs or DAG runs triggered without changing the run_id parameter. + version_added: 2.6.3 + type: string + example: ~ + default: "^[A-Za-z0-9_.~:+-]+$" triggerer: description: ~ options: diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 4eaab9ae95db..c28155ac2c40 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -1325,6 +1325,11 @@ task_queued_timeout = 600.0 # longer than `[scheduler] task_queued_timeout`. task_queued_timeout_check_interval = 120.0 +# The run_id pattern used to verify the validity of user input to the run_id parameter when +# triggering a DAG. This pattern cannot change the pattern used by scheduler to generate run_id +# for scheduled DAG runs or DAG runs triggered without changing the run_id parameter. +allowed_run_id_pattern = ^[A-Za-z0-9_.~:+-]+$ + [triggerer] # How many triggers a single Triggerer will run at once, by default. default_capacity = 1000 diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 31070a7b7c53..11468b59ea70 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -80,7 +80,7 @@ import airflow.templates from airflow import settings, utils from airflow.api_internal.internal_api_call import internal_api_call -from airflow.configuration import conf, secrets_backend_list +from airflow.configuration import conf as airflow_conf, secrets_backend_list from airflow.exceptions import ( AirflowDagInconsistent, AirflowException, @@ -96,7 +96,7 @@ from airflow.models.baseoperator import BaseOperator from airflow.models.dagcode import DagCode from airflow.models.dagpickle import DagPickle -from airflow.models.dagrun import DagRun +from airflow.models.dagrun import RUN_ID_REGEX, DagRun from airflow.models.operator import Operator from airflow.models.param import DagParam, ParamsDict from airflow.models.taskinstance import Context, TaskInstance, TaskInstanceKey, clear_task_instances @@ -422,13 +422,13 @@ def __init__( user_defined_filters: dict | None = None, default_args: dict | None = None, concurrency: int | None = None, - max_active_tasks: int = conf.getint("core", "max_active_tasks_per_dag"), - max_active_runs: int = conf.getint("core", "max_active_runs_per_dag"), + max_active_tasks: int = airflow_conf.getint("core", "max_active_tasks_per_dag"), + max_active_runs: int = airflow_conf.getint("core", "max_active_runs_per_dag"), dagrun_timeout: timedelta | None = None, sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, - default_view: str = conf.get_mandatory_value("webserver", "dag_default_view").lower(), - orientation: str = conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: bool = conf.getboolean("scheduler", "catchup_by_default"), + default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), + orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), + catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, @@ -2588,7 +2588,7 @@ def run( mark_success=False, local=False, executor=None, - donot_pickle=conf.getboolean("core", "donot_pickle"), + donot_pickle=airflow_conf.getboolean("core", "donot_pickle"), ignore_task_deps=False, ignore_first_depends_on_past=True, pool=None, @@ -2826,13 +2826,14 @@ def create_dagrun( "Creating DagRun needs either `run_id` or both `run_type` and `execution_date`" ) - if run_id and "/" in run_id: - warnings.warn( - "Using forward slash ('/') in a DAG run ID is deprecated. Note that this character " - "also makes the run impossible to retrieve via Airflow's REST API.", - RemovedInAirflow3Warning, - stacklevel=3, - ) + regex = airflow_conf.get("scheduler", "allowed_run_id_pattern") + + if run_id and not re.match(RUN_ID_REGEX, run_id): + if not regex.strip() or not re.match(regex.strip(), run_id): + raise AirflowException( + f"The provided run ID '{run_id}' is invalid. It does not match either " + f"the configured pattern: '{regex}' or the built-in pattern: '{RUN_ID_REGEX}'" + ) # create a copy of params before validating copied_params = copy.deepcopy(self.params) @@ -3125,7 +3126,7 @@ def sync_to_db(self, processor_subdir: str | None = None, session=NEW_SESSION): def get_default_view(self): """This is only there for backward compatible jinja2 templates.""" if self.default_view is None: - return conf.get("webserver", "dag_default_view").lower() + return airflow_conf.get("webserver", "dag_default_view").lower() else: return self.default_view @@ -3342,7 +3343,7 @@ class DagModel(Base): root_dag_id = Column(StringID()) # A DAG can be paused from the UI / DB # Set this default value of is_paused based on a configuration value! - is_paused_at_creation = conf.getboolean("core", "dags_are_paused_at_creation") + is_paused_at_creation = airflow_conf.getboolean("core", "dags_are_paused_at_creation") is_paused = Column(Boolean, default=is_paused_at_creation) # Whether the DAG is a subdag is_subdag = Column(Boolean, default=False) @@ -3416,7 +3417,9 @@ class DagModel(Base): "TaskOutletDatasetReference", cascade="all, delete, delete-orphan", ) - NUM_DAGS_PER_DAGRUN_QUERY = conf.getint("scheduler", "max_dagruns_to_create_per_loop", fallback=10) + NUM_DAGS_PER_DAGRUN_QUERY = airflow_conf.getint( + "scheduler", "max_dagruns_to_create_per_loop", fallback=10 + ) def __init__(self, concurrency=None, **kwargs): super().__init__(**kwargs) @@ -3429,10 +3432,10 @@ def __init__(self, concurrency=None, **kwargs): ) self.max_active_tasks = concurrency else: - self.max_active_tasks = conf.getint("core", "max_active_tasks_per_dag") + self.max_active_tasks = airflow_conf.getint("core", "max_active_tasks_per_dag") if self.max_active_runs is None: - self.max_active_runs = conf.getint("core", "max_active_runs_per_dag") + self.max_active_runs = airflow_conf.getint("core", "max_active_runs_per_dag") if self.has_task_concurrency_limits is None: # Be safe -- this will be updated later once the DAG is parsed @@ -3510,7 +3513,7 @@ def get_default_view(self) -> str: have a value. """ # This is for backwards-compatibility with old dags that don't have None as default_view - return self.default_view or conf.get_mandatory_value("webserver", "dag_default_view").lower() + return self.default_view or airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower() @property def safe_dag_id(self): @@ -3699,13 +3702,13 @@ def dag( user_defined_filters: dict | None = None, default_args: dict | None = None, concurrency: int | None = None, - max_active_tasks: int = conf.getint("core", "max_active_tasks_per_dag"), - max_active_runs: int = conf.getint("core", "max_active_runs_per_dag"), + max_active_tasks: int = airflow_conf.getint("core", "max_active_tasks_per_dag"), + max_active_runs: int = airflow_conf.getint("core", "max_active_runs_per_dag"), dagrun_timeout: timedelta | None = None, sla_miss_callback: None | SLAMissCallback | list[SLAMissCallback] = None, - default_view: str = conf.get_mandatory_value("webserver", "dag_default_view").lower(), - orientation: str = conf.get_mandatory_value("webserver", "dag_orientation"), - catchup: bool = conf.getboolean("scheduler", "catchup_by_default"), + default_view: str = airflow_conf.get_mandatory_value("webserver", "dag_default_view").lower(), + orientation: str = airflow_conf.get_mandatory_value("webserver", "dag_orientation"), + catchup: bool = airflow_conf.getboolean("scheduler", "catchup_by_default"), on_success_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, on_failure_callback: None | DagStateChangeCallback | list[DagStateChangeCallback] = None, doc_md: str | None = None, diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index dab89c12cd9d..8f3b3a33018a 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -24,6 +24,7 @@ from datetime import datetime from typing import TYPE_CHECKING, Any, Callable, Iterable, Iterator, NamedTuple, Sequence, TypeVar, overload +import re2 as re from sqlalchemy import ( Boolean, Column, @@ -44,7 +45,7 @@ ) from sqlalchemy.exc import IntegrityError from sqlalchemy.ext.associationproxy import association_proxy -from sqlalchemy.orm import Query, Session, declared_attr, joinedload, relationship, synonym +from sqlalchemy.orm import Query, Session, declared_attr, joinedload, relationship, synonym, validates from sqlalchemy.sql.expression import false, select, true from airflow import settings @@ -76,6 +77,8 @@ CreatedTasks = TypeVar("CreatedTasks", Iterator["dict[str, Any]"], Iterator[TI]) TaskCreator = Callable[[Operator, Iterable[int]], CreatedTasks] +RUN_ID_REGEX = r"^(?:manual|scheduled|dataset_triggered)__(?:\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\+00:00)$" + class TISchedulingDecision(NamedTuple): """Type of return for DagRun.task_instance_scheduling_decisions.""" @@ -240,6 +243,17 @@ def __repr__(self): external_trigger=self.external_trigger, ) + @validates("run_id") + def validate_run_id(self, key: str, run_id: str) -> str | None: + if not run_id: + return None + regex = airflow_conf.get("scheduler", "allowed_run_id_pattern") + if not re.match(regex, run_id) and not re.match(RUN_ID_REGEX, run_id): + raise ValueError( + f"The run_id provided '{run_id}' does not match the pattern '{regex}' or '{RUN_ID_REGEX}'" + ) + return run_id + @property def stats_tags(self) -> dict[str, str]: return prune_dict({"dag_id": self.dag_id, "run_type": self.run_type}) diff --git a/airflow/www/views.py b/airflow/www/views.py index 607927650bb1..026f8b533499 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -99,7 +99,7 @@ from airflow.models.abstractoperator import AbstractOperator from airflow.models.dag import DAG, get_dataset_triggered_next_run_info from airflow.models.dagcode import DagCode -from airflow.models.dagrun import DagRun, DagRunType +from airflow.models.dagrun import RUN_ID_REGEX, DagRun, DagRunType from airflow.models.dataset import DagScheduleDatasetReference, DatasetDagRunQueue, DatasetEvent, DatasetModel from airflow.models.mappedoperator import MappedOperator from airflow.models.operator import Operator @@ -1975,7 +1975,7 @@ def delete(self): @provide_session def trigger(self, dag_id: str, session: Session = NEW_SESSION): """Triggers DAG Run.""" - run_id = request.values.get("run_id", "") + run_id = request.values.get("run_id", "").replace(" ", "+") origin = get_safe_url(request.values.get("origin")) unpause = request.values.get("unpause") request_conf = request.values.get("conf") @@ -2096,13 +2096,27 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): flash(message, "error") return redirect(origin) - # Flash a warning when slash is used, but still allow it to continue on. - if run_id and "/" in run_id: - flash( - "Using forward slash ('/') in a DAG run ID is deprecated. Note that this character " - "also makes the run impossible to retrieve via Airflow's REST API.", - "warning", - ) + regex = conf.get("scheduler", "allowed_run_id_pattern") + if run_id and not re.match(RUN_ID_REGEX, run_id): + if not regex.strip() or not re.match(regex.strip(), run_id): + flash( + f"The provided run ID '{run_id}' is invalid. It does not match either " + f"the configured pattern: '{regex}' or the built-in pattern: '{RUN_ID_REGEX}'", + "error", + ) + + form = DateTimeForm(data={"execution_date": execution_date}) + return self.render_template( + "airflow/trigger.html", + form_fields=form_fields, + dag=dag, + dag_id=dag_id, + origin=origin, + conf=request_conf, + form=form, + is_dag_run_conf_overrides_params=is_dag_run_conf_overrides_params, + recent_confs=recent_confs, + ) run_conf = {} if request_conf: diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 024b9df3a1f3..5d7db547d31c 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -30,6 +30,7 @@ from airflow import settings from airflow.callbacks.callback_requests import DagCallbackRequest from airflow.decorators import setup, task, task_group, teardown +from airflow.exceptions import AirflowException from airflow.models import ( DAG, DagBag, @@ -54,6 +55,7 @@ from airflow.utils.types import DagRunType from tests.models import DEFAULT_DATE as _DEFAULT_DATE from tests.test_utils import db +from tests.test_utils.config import conf_vars from tests.test_utils.mock_operators import MockOperator DEFAULT_DATE = pendulum.instance(_DEFAULT_DATE) @@ -2541,3 +2543,33 @@ def make_task(task_id, dag): tis = dr.task_instance_scheduling_decisions(session).tis tis_for_state = {x.task_id for x in dr._tis_for_dagrun_state(dag=dag, tis=tis)} assert tis_for_state == expected + + +@pytest.mark.parametrize( + "pattern, run_id, result", + [ + ["^[A-Z]", "ABC", True], + ["^[A-Z]", "abc", False], + ["^[0-9]", "123", True], + # The below params tests that user configuration does not affect internally generated + # run_ids + ["", "scheduled__2023-01-01T00:00:00+00:00", True], + ["", "manual__2023-01-01T00:00:00+00:00", True], + ["", "dataset_triggered__2023-01-01T00:00:00+00:00", True], + ["", "scheduled_2023-01-01T00", False], + ["", "manual_2023-01-01T00", False], + ["", "dataset_triggered_2023-01-01T00", False], + ["^[0-9]", "scheduled__2023-01-01T00:00:00+00:00", True], + ["^[0-9]", "manual__2023-01-01T00:00:00+00:00", True], + ["^[a-z]", "dataset_triggered__2023-01-01T00:00:00+00:00", True], + ], +) +def test_dag_run_id_config(session, dag_maker, pattern, run_id, result): + with conf_vars({("scheduler", "allowed_run_id_pattern"): pattern}): + with dag_maker(): + ... + if result: + dag_maker.create_dagrun(run_id=run_id) + else: + with pytest.raises(AirflowException): + dag_maker.create_dagrun(run_id=run_id) diff --git a/tests/www/views/test_views_trigger_dag.py b/tests/www/views/test_views_trigger_dag.py index 7cf140ed7ca6..12f090bc4f4e 100644 --- a/tests/www/views/test_views_trigger_dag.py +++ b/tests/www/views/test_views_trigger_dag.py @@ -30,6 +30,7 @@ from airflow.utils.session import create_session from airflow.utils.types import DagRunType from tests.test_utils.api_connexion_utils import create_test_client +from tests.test_utils.config import conf_vars from tests.test_utils.www import check_content_in_response @@ -287,3 +288,32 @@ def test_trigger_dag_params_array_value_none_render(admin_client, dag_maker, ses f'', resp, ) + + +@pytest.mark.parametrize( + "pattern, run_id, result", + [ + ["^[A-Z]", "ABC", True], + ["^[A-Z]", "abc", False], + ["^[0-9]", "123", True], + # The below params tests that user configuration does not affect internally generated + # run_ids. We use manual__ as a prefix for manually triggered DAGs due to a restriction + # in manually triggered DAGs that the run_id must not start with scheduled__. + ["", "manual__2023-01-01T00:00:00+00:00", True], + ["", "scheduled_2023-01-01T00", False], + ["", "manual_2023-01-01T00", False], + ["", "dataset_triggered_2023-01-01T00", False], + ["^[0-9]", "manual__2023-01-01T00:00:00+00:00", True], + ["^[a-z]", "manual__2023-01-01T00:00:00+00:00", True], + ], +) +def test_dag_run_id_pattern(session, admin_client, pattern, run_id, result): + with conf_vars({("scheduler", "allowed_run_id_pattern"): pattern}): + test_dag_id = "example_bash_operator" + admin_client.post(f"dags/{test_dag_id}/trigger?&run_id={run_id}") + run = session.query(DagRun).filter(DagRun.dag_id == test_dag_id).first() + if result: + assert run is not None + assert run.run_type == DagRunType.MANUAL + else: + assert run is None From 00bf2ab2191fd7ebce34583007a4c06b7561b360 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Sun, 2 Jul 2023 11:47:29 -0400 Subject: [PATCH 018/533] Use reactflow for datasets graph (#31775) --- airflow/www/package.json | 2 - .../static/js/api/useDatasetDependencies.ts | 5 +- airflow/www/static/js/api/useGraphData.ts | 7 +- .../graph => components/Graph}/Edge.tsx | 20 +- .../www/static/js/dag/details/graph/index.tsx | 2 +- .../www/static/js/dag/details/graph/utils.ts | 3 +- .../www/static/js/datasets/Graph/DagNode.tsx | 5 +- airflow/www/static/js/datasets/Graph/Edge.tsx | 63 ------ .../www/static/js/datasets/Graph/Legend.tsx | 70 ++---- airflow/www/static/js/datasets/Graph/Node.tsx | 115 +++++----- .../www/static/js/datasets/Graph/index.tsx | 213 ++++++++++-------- airflow/www/static/js/datasets/index.tsx | 31 ++- airflow/www/static/js/types/index.ts | 12 + airflow/www/static/js/utils/graph.ts | 9 +- airflow/www/yarn.lock | 48 +--- 15 files changed, 252 insertions(+), 353 deletions(-) rename airflow/www/static/js/{dag/details/graph => components/Graph}/Edge.tsx (78%) delete mode 100644 airflow/www/static/js/datasets/Graph/Edge.tsx diff --git a/airflow/www/package.json b/airflow/www/package.json index 5eac3744663a..cf1727b45abc 100644 --- a/airflow/www/package.json +++ b/airflow/www/package.json @@ -92,9 +92,7 @@ "@emotion/react": "^11.9.3", "@emotion/styled": "^11", "@visx/group": "^2.10.0", - "@visx/marker": "^2.12.2", "@visx/shape": "^2.12.2", - "@visx/zoom": "^2.10.0", "axios": "^0.26.0", "bootstrap-3-typeahead": "^4.0.2", "camelcase-keys": "^7.0.0", diff --git a/airflow/www/static/js/api/useDatasetDependencies.ts b/airflow/www/static/js/api/useDatasetDependencies.ts index 3760cbfded13..2680ee9a782d 100644 --- a/airflow/www/static/js/api/useDatasetDependencies.ts +++ b/airflow/www/static/js/api/useDatasetDependencies.ts @@ -22,11 +22,10 @@ import { useQuery } from "react-query"; import ELK, { ElkShape, ElkExtendedEdge } from "elkjs"; import { getMetaValue } from "src/utils"; -import type { DepEdge, DepNode } from "src/types"; -import type { NodeType } from "src/datasets/Graph/Node"; - import { getTextWidth } from "src/utils/graph"; +import type { NodeType, DepEdge, DepNode } from "src/types"; + interface DatasetDependencies { edges: DepEdge[]; nodes: DepNode[]; diff --git a/airflow/www/static/js/api/useGraphData.ts b/airflow/www/static/js/api/useGraphData.ts index 9c0f089e9c75..e7a94156ef26 100644 --- a/airflow/www/static/js/api/useGraphData.ts +++ b/airflow/www/static/js/api/useGraphData.ts @@ -21,12 +21,12 @@ import { useQuery } from "react-query"; import axios, { AxiosResponse } from "axios"; import { getMetaValue } from "src/utils"; -import type { DepNode } from "src/types"; import useFilters, { FILTER_DOWNSTREAM_PARAM, FILTER_UPSTREAM_PARAM, ROOT_PARAM, } from "src/dag/useFilters"; +import type { WebserverEdge, DepNode } from "src/types"; const DAG_ID_PARAM = "dag_id"; @@ -38,11 +38,6 @@ interface GraphData { nodes: DepNode; arrange: string; } -export interface WebserverEdge { - label?: string; - sourceId: string; - targetId: string; -} const useGraphData = () => { const { diff --git a/airflow/www/static/js/dag/details/graph/Edge.tsx b/airflow/www/static/js/components/Graph/Edge.tsx similarity index 78% rename from airflow/www/static/js/dag/details/graph/Edge.tsx rename to airflow/www/static/js/components/Graph/Edge.tsx index 561048f79821..0205bf49cd3f 100644 --- a/airflow/www/static/js/dag/details/graph/Edge.tsx +++ b/airflow/www/static/js/components/Graph/Edge.tsx @@ -18,11 +18,11 @@ */ import React from "react"; -import { Text } from "@chakra-ui/react"; +import { Text, useTheme } from "@chakra-ui/react"; import type { ElkEdgeSection, ElkLabel, ElkPoint, LayoutOptions } from "elkjs"; -import Edge from "src/datasets/Graph/Edge"; import { Group } from "@visx/group"; +import { LinePath } from "@visx/shape"; interface EdgeProps { data?: { @@ -40,6 +40,7 @@ interface EdgeProps { } const CustomEdge = ({ data }: EdgeProps) => { + const { colors } = useTheme(); if (!data) return null; const { rest } = data; return ( @@ -54,11 +55,16 @@ const CustomEdge = ({ data }: EdgeProps) => { ); })} - + {(rest.sections || []).map((s) => ( + d.x || 0} + y={(d) => d.y || 0} + data={[s.startPoint, ...(s.bendPoints || []), s.endPoint]} + /> + ))} ); }; diff --git a/airflow/www/static/js/dag/details/graph/index.tsx b/airflow/www/static/js/dag/details/graph/index.tsx index 31578869cf2f..4dae9477bdc5 100644 --- a/airflow/www/static/js/dag/details/graph/index.tsx +++ b/airflow/www/static/js/dag/details/graph/index.tsx @@ -38,8 +38,8 @@ import { useGraphLayout } from "src/utils/graph"; import Tooltip from "src/components/Tooltip"; import { useContainerRef } from "src/context/containerRef"; import useFilters from "src/dag/useFilters"; +import Edge from "src/components/Graph/Edge"; -import Edge from "./Edge"; import Node, { CustomNodeProps } from "./Node"; import { buildEdges, nodeStrokeColor, nodeColor, flattenNodes } from "./utils"; diff --git a/airflow/www/static/js/dag/details/graph/utils.ts b/airflow/www/static/js/dag/details/graph/utils.ts index 703fb38b9ca9..998451e05ba1 100644 --- a/airflow/www/static/js/dag/details/graph/utils.ts +++ b/airflow/www/static/js/dag/details/graph/utils.ts @@ -23,8 +23,7 @@ import type { ElkExtendedEdge } from "elkjs"; import type { SelectionProps } from "src/dag/useSelection"; import { getTask } from "src/utils"; -import type { Task, TaskInstance } from "src/types"; -import type { NodeType } from "src/datasets/Graph/Node"; +import type { Task, TaskInstance, NodeType } from "src/types"; import type { CustomNodeProps } from "./Node"; diff --git a/airflow/www/static/js/datasets/Graph/DagNode.tsx b/airflow/www/static/js/datasets/Graph/DagNode.tsx index cb4900a5c31c..aa12c575221f 100644 --- a/airflow/www/static/js/datasets/Graph/DagNode.tsx +++ b/airflow/www/static/js/datasets/Graph/DagNode.tsx @@ -33,6 +33,7 @@ import { useTheme, } from "@chakra-ui/react"; import { MdOutlineAccountTree } from "react-icons/md"; + import { useContainerRef } from "src/context/containerRef"; import { getMetaValue } from "src/utils"; @@ -41,7 +42,7 @@ const DagNode = ({ isHighlighted, }: { dagId: string; - isHighlighted: boolean; + isHighlighted?: boolean; }) => { const { colors } = useTheme(); const containerRef = useContainerRef(); @@ -63,7 +64,7 @@ const DagNode = ({ alignItems="center" > - {dagId} + {dagId} diff --git a/airflow/www/static/js/datasets/Graph/Edge.tsx b/airflow/www/static/js/datasets/Graph/Edge.tsx deleted file mode 100644 index a5ef5fbfed3d..000000000000 --- a/airflow/www/static/js/datasets/Graph/Edge.tsx +++ /dev/null @@ -1,63 +0,0 @@ -/*! - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -import React from "react"; -import { LinePath } from "@visx/shape"; -import { MarkerArrow } from "@visx/marker"; -import { useTheme } from "@chakra-ui/react"; - -interface Props { - edge: { - id: string; - sources: string[]; - targets: string[]; - sections: Record[]; - }; - isSelected?: boolean; - showMarker?: boolean; -} - -const Edge = ({ edge, isSelected = false, showMarker = true }: Props) => { - const { colors } = useTheme(); - return ( - <> - {showMarker && ( - - )} - {(edge.sections || []).map((s) => ( - d.x || 0} - y={(d) => d.y || 0} - data={[s.startPoint, ...(s.bendPoints || []), s.endPoint]} - markerEnd="url(#marker-arrow)" - /> - ))} - - ); -}; - -export default Edge; diff --git a/airflow/www/static/js/datasets/Graph/Legend.tsx b/airflow/www/static/js/datasets/Graph/Legend.tsx index 250d6852b9b4..3ecbed766473 100644 --- a/airflow/www/static/js/datasets/Graph/Legend.tsx +++ b/airflow/www/static/js/datasets/Graph/Legend.tsx @@ -18,59 +18,29 @@ */ import React from "react"; -import { Flex, Box, IconButton, Text } from "@chakra-ui/react"; -import { - MdOutlineZoomOutMap, - MdFilterCenterFocus, - MdOutlineAccountTree, -} from "react-icons/md"; +import { Flex, Box, Text } from "@chakra-ui/react"; +import { MdOutlineAccountTree } from "react-icons/md"; import { HiDatabase } from "react-icons/hi"; -interface Props { - zoom: any; - center: () => void; -} - -const Legend = ({ zoom, center }: Props) => ( - - - } - /> - } - /> - - - Legend - - - - DAG - - - - Dataset - +const Legend = () => ( + + + + + DAG + + + + Dataset - - + + ); export default Legend; diff --git a/airflow/www/static/js/datasets/Graph/Node.tsx b/airflow/www/static/js/datasets/Graph/Node.tsx index d692861b66b4..ed02653139c4 100644 --- a/airflow/www/static/js/datasets/Graph/Node.tsx +++ b/airflow/www/static/js/datasets/Graph/Node.tsx @@ -18,71 +18,76 @@ */ import React from "react"; -import { Flex, Text, useTheme } from "@chakra-ui/react"; -import { Group } from "@visx/group"; +import { Box, Text, Flex, useTheme } from "@chakra-ui/react"; +import { Handle, NodeProps, Position } from "reactflow"; import { MdPlayArrow, MdSensors } from "react-icons/md"; import { HiDatabase } from "react-icons/hi"; -import type { ElkShape } from "elkjs"; -import type { DepNode } from "src/types"; - import DagNode from "./DagNode"; -export interface NodeType extends ElkShape { - value: DepNode["value"]; - children?: NodeType[]; -} - -interface Props { - node: NodeType; - onSelect: (datasetId: string) => void; - isSelected: boolean; - isHighlighted: boolean; +export interface CustomNodeProps { + label: string; + type?: string; + height?: number; + width?: number; + isSelected?: boolean; + isHighlighted?: boolean; + onSelect: (datasetUri: string) => void; + isOpen?: boolean; + isActive?: boolean; } -const Node = ({ - node: { height, width, x, y, value }, - onSelect, - isSelected, - isHighlighted, -}: Props) => { +const BaseNode = ({ + data: { label, type, isSelected, isHighlighted, onSelect }, +}: NodeProps) => { const { colors } = useTheme(); + return ( - - - {value.class === "dag" && ( - - )} - {value.class !== "dag" && ( - { - e.preventDefault(); - e.stopPropagation(); - if (value.class === "dataset") onSelect(value.label); - }} - cursor="pointer" - fontSize={16} - justifyContent="space-between" - alignItems="center" - > - {value.class === "dataset" && } - {value.class === "sensor" && } - {value.class === "trigger" && } - {value.label} - - )} - - + + {type === "dag" && ( + + )} + {type !== "dag" && ( + { + e.preventDefault(); + e.stopPropagation(); + if (type === "dataset") onSelect(label); + }} + cursor="pointer" + fontSize={16} + justifyContent="space-between" + alignItems="center" + > + {type === "dataset" && } + {type === "sensor" && } + {type === "trigger" && } + {label} + + )} + ); }; +const Node = (props: NodeProps) => ( + <> + + + + +); + export default Node; diff --git a/airflow/www/static/js/datasets/Graph/index.tsx b/airflow/www/static/js/datasets/Graph/index.tsx index f8693659451a..f137be36d107 100644 --- a/airflow/www/static/js/datasets/Graph/index.tsx +++ b/airflow/www/static/js/datasets/Graph/index.tsx @@ -17,28 +17,50 @@ * under the License. */ -import React, { RefObject } from "react"; -import { Box, Spinner } from "@chakra-ui/react"; -import { Zoom } from "@visx/zoom"; -import { Group } from "@visx/group"; +import React, { useEffect } from "react"; +import ReactFlow, { + ReactFlowProvider, + Controls, + Background, + MiniMap, + Node as ReactFlowNode, + useReactFlow, + ControlButton, + Panel, +} from "reactflow"; +import { Box, Tooltip, useTheme } from "@chakra-ui/react"; +import { RiFocus3Line } from "react-icons/ri"; import { useDatasetDependencies } from "src/api"; +import Edge from "src/components/Graph/Edge"; +import { useContainerRef } from "src/context/containerRef"; -import Node from "./Node"; -import Edge from "./Edge"; +import Node, { CustomNodeProps } from "./Node"; import Legend from "./Legend"; interface Props { onSelect: (datasetId: string) => void; selectedUri: string | null; - height: number; - width: number; } -const Graph = ({ onSelect, selectedUri, height, width }: Props) => { - const { data, isLoading } = useDatasetDependencies(); +const nodeTypes = { custom: Node }; +const edgeTypes = { custom: Edge }; + +const Graph = ({ onSelect, selectedUri }: Props) => { + const { data } = useDatasetDependencies(); + const { colors } = useTheme(); + const { setCenter, setViewport } = useReactFlow(); + const containerRef = useContainerRef(); + + useEffect(() => { + setViewport({ x: 0, y: 0, zoom: 1 }); + }, [selectedUri, setViewport]); + + const nodeColor = ({ + data: { isSelected }, + }: ReactFlowNode) => + isSelected ? colors.blue["300"] : colors.gray["300"]; - if (isLoading && !data) return ; if (!data || !data.fullGraph || !data.subGraphs) return null; const graph = selectedUri ? data.subGraphs.find((g) => @@ -46,91 +68,102 @@ const Graph = ({ onSelect, selectedUri, height, width }: Props) => { ) : data.fullGraph; if (!graph) return null; - const { edges, children, width: graphWidth, height: graphHeight } = graph; - const initialTransform = { - scaleX: 1, - scaleY: 1, - translateX: 0, - translateY: 0, - skewX: 0, - skewY: 0, - }; + const edges = graph.edges.map((e) => ({ + id: e.id, + source: e.sources[0], + target: e.targets[0], + type: "custom", + data: { + rest: { + ...e, + isSelected: selectedUri && e.id.includes(selectedUri), + }, + }, + })); - const selectedEdges = selectedUri - ? edges?.filter( - ({ sources, targets }) => - sources[0].includes(selectedUri) || targets[0].includes(selectedUri) - ) - : []; - const highlightedNodes = children.filter((n) => - selectedEdges.some( - ({ sources, targets }) => sources[0] === n.id || targets[0] === n.id - ) - ); + const nodes: ReactFlowNode[] = graph.children.map((c) => ({ + id: c.id, + data: { + label: c.value.label, + type: c.value.class, + width: c.width, + height: c.height, + onSelect, + isSelected: selectedUri === c.value.label, + isHighlighted: edges.some( + (e) => e.data.rest.isSelected && e.id.includes(c.id) + ), + }, + type: "custom", + position: { + x: c.x || 0, + y: c.y || 0, + }, + })); + + const focusNode = () => { + if (selectedUri) { + const node = nodes.find((n) => n.data.label === selectedUri); + if (!node || !node.position) return; + const { x, y } = node.position; + setCenter( + x + (node.data.width || 0) / 2, + y + (node.data.height || 0) / 2, + { + duration: 1000, + } + ); + } + }; return ( - - {(zoom) => ( - - } - style={{ - cursor: zoom.isDragging ? "grabbing" : "grab", - touchAction: "none", - }} + + + + - - - {edges.map((edge) => ( - e.id === edge.id)} - /> - ))} - {children.map((node) => ( - n.id === node.id - )} - /> - ))} - - - - - - zoom.translateTo({ - x: (width - (graphWidth ?? 0)) / 2, - y: (height - (graphHeight ?? 0)) / 2, - }) - } - /> - - - - - )} - + + + + + + + + + + + ); }; -export default Graph; +const GraphWrapper = (props: Props) => ( + + + +); + +export default GraphWrapper; diff --git a/airflow/www/static/js/datasets/index.tsx b/airflow/www/static/js/datasets/index.tsx index bd21ca26f426..166b3f26ec99 100644 --- a/airflow/www/static/js/datasets/index.tsx +++ b/airflow/www/static/js/datasets/index.tsx @@ -23,9 +23,11 @@ import React, { useRef } from "react"; import { createRoot } from "react-dom/client"; import createCache from "@emotion/cache"; import { useSearchParams } from "react-router-dom"; -import { Flex, Box, useDimensions } from "@chakra-ui/react"; +import { Flex, Box } from "@chakra-ui/react"; +import reactFlowStyle from "reactflow/dist/style.css"; import App from "src/App"; +import { useOffsetTop } from "src/utils"; import DatasetsList from "./List"; import DatasetDetails from "./Details"; @@ -45,8 +47,9 @@ const DATASET_URI = "uri"; const Datasets = () => { const [searchParams, setSearchParams] = useSearchParams(); const contentRef = useRef(null); - const graphRef = useRef(null); - const dimensions = useDimensions(graphRef, true); + const offsetTop = useOffsetTop(contentRef); + // 60px for footer height + const height = `calc(100vh - ${offsetTop + 60}px)`; const onBack = () => { searchParams.delete(DATASET_URI); @@ -66,26 +69,15 @@ const Datasets = () => { justifyContent="space-between" ref={contentRef} > - + {datasetUri ? ( ) : ( )} - - + + ); @@ -93,6 +85,11 @@ const Datasets = () => { if (mainElement) { shadowRoot?.appendChild(mainElement); + const styleTag = document.createElement("style"); + const style = reactFlowStyle.toString(); + styleTag.innerHTML = style; + shadowRoot?.appendChild(styleTag); + const reactRoot = createRoot(mainElement); reactRoot.render( diff --git a/airflow/www/static/js/types/index.ts b/airflow/www/static/js/types/index.ts index 45b34c924fa7..08929d33c400 100644 --- a/airflow/www/static/js/types/index.ts +++ b/airflow/www/static/js/types/index.ts @@ -18,6 +18,7 @@ */ import type { CamelCase } from "type-fest"; +import type { ElkShape } from "elkjs"; import type * as API from "./api-generated"; type RunState = "success" | "running" | "queued" | "failed"; @@ -127,6 +128,17 @@ interface DepEdge { target: string; } +export interface NodeType extends ElkShape { + value: DepNode["value"]; + children?: NodeType[]; +} + +export interface WebserverEdge { + label?: string; + sourceId: string; + targetId: string; +} + interface DatasetListItem extends API.Dataset { lastDatasetUpdate: string | null; totalUpdates: number; diff --git a/airflow/www/static/js/utils/graph.ts b/airflow/www/static/js/utils/graph.ts index d2d2923c5ab8..e69a3045d945 100644 --- a/airflow/www/static/js/utils/graph.ts +++ b/airflow/www/static/js/utils/graph.ts @@ -19,8 +19,7 @@ import ELK, { ElkExtendedEdge, ElkShape } from "elkjs"; -import type { DepNode } from "src/types"; -import type { NodeType } from "src/datasets/Graph/Node"; +import type { NodeType, DepNode, WebserverEdge } from "src/types"; import { useQuery } from "react-query"; import useFilters from "src/dag/useFilters"; @@ -32,12 +31,6 @@ interface GenerateProps { arrange: string; } -interface WebserverEdge { - label?: string; - sourceId: string; - targetId: string; -} - interface Graph extends ElkShape { children: NodeType[]; edges: ElkExtendedEdge[]; diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index 6e1fce53343c..1916d9c9001a 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -3575,18 +3575,6 @@ "@typescript-eslint/types" "5.27.1" eslint-visitor-keys "^3.3.0" -"@use-gesture/core@10.2.17": - version "10.2.17" - resolved "https://registry.yarnpkg.com/@use-gesture/core/-/core-10.2.17.tgz#dc78913cd5d105217c3f1d1c16a32ad6426a00ba" - integrity sha512-62hCybe4x6oGZ1/JA9gSYIdghV1FqxCdvYWt9SqCEAAikwT1OmVl2Q/Uu8CP636L57D+DfXtw6PWM+fdhr4oJQ== - -"@use-gesture/react@^10.0.0-beta.22": - version "10.2.17" - resolved "https://registry.yarnpkg.com/@use-gesture/react/-/react-10.2.17.tgz#00bc413da42a358dd3f9173c0631b54522e76614" - integrity sha512-Vfrp1KgdYn/kOEUAYNXtGBCl2dr38s3G6rru1TOPs+cVUjfNyNxvJK56grUyJ336N3rQLK8F9G7+FfrHuc3g/Q== - dependencies: - "@use-gesture/core" "10.2.17" - "@visx/curve@2.1.0": version "2.1.0" resolved "https://registry.yarnpkg.com/@visx/curve/-/curve-2.1.0.tgz#f614bfe3db66df7db7382db7a75ced1506b94602" @@ -3595,14 +3583,6 @@ "@types/d3-shape" "^1.3.1" d3-shape "^1.0.6" -"@visx/event@2.6.0": - version "2.6.0" - resolved "https://registry.yarnpkg.com/@visx/event/-/event-2.6.0.tgz#0718eb1efabd5305cf659a153779c94ba4038996" - integrity sha512-WGp91g82s727g3NAnENF1ppC3ZAlvWg+Y+GG0WFg34NmmOZbvPI/PTOqTqZE3x6B8EUn8NJiMxRjxIMbi+IvRw== - dependencies: - "@types/react" "*" - "@visx/point" "2.6.0" - "@visx/group@2.10.0", "@visx/group@^2.10.0": version "2.10.0" resolved "https://registry.yarnpkg.com/@visx/group/-/group-2.10.0.tgz#95839851832545621eb0d091866a61dafe552ae1" @@ -3612,22 +3592,6 @@ classnames "^2.3.1" prop-types "^15.6.2" -"@visx/marker@^2.12.2": - version "2.12.2" - resolved "https://registry.yarnpkg.com/@visx/marker/-/marker-2.12.2.tgz#b81cea1a5d2b61c065aa97e4baccf9d0f17cab51" - integrity sha512-yvJDMBw9oKQDD2gX5q7O+raR9qk/NYqKFDZ0GtS4ZVH87PfNe0ZyTXt0vWbIaDaix/r58SMpv38GluIOxWE7jg== - dependencies: - "@types/react" "*" - "@visx/group" "2.10.0" - "@visx/shape" "2.12.2" - classnames "^2.3.1" - prop-types "^15.6.2" - -"@visx/point@2.6.0": - version "2.6.0" - resolved "https://registry.yarnpkg.com/@visx/point/-/point-2.6.0.tgz#c4316ca409b5b829c5455f07118d8c14a92cc633" - integrity sha512-amBi7yMz4S2VSchlPdliznN41TuES64506ySI22DeKQ+mc1s1+BudlpnY90sM1EIw4xnqbKmrghTTGfy6SVqvQ== - "@visx/scale@2.2.2": version "2.2.2" resolved "https://registry.yarnpkg.com/@visx/scale/-/scale-2.2.2.tgz#b8eafabdcf92bb45ab196058fe184772ad80fd25" @@ -3640,7 +3604,7 @@ d3-scale "^3.3.0" d3-time "^2.1.1" -"@visx/shape@2.12.2", "@visx/shape@^2.12.2": +"@visx/shape@^2.12.2": version "2.12.2" resolved "https://registry.yarnpkg.com/@visx/shape/-/shape-2.12.2.tgz#81ed88bf823aa84a4f5f32a9c9daf8371a606897" integrity sha512-4gN0fyHWYXiJ+Ck8VAazXX0i8TOnLJvOc5jZBnaJDVxgnSIfCjJn0+Nsy96l9Dy/bCMTh4DBYUBv9k+YICBUOA== @@ -3658,16 +3622,6 @@ lodash "^4.17.21" prop-types "^15.5.10" -"@visx/zoom@^2.10.0": - version "2.10.0" - resolved "https://registry.yarnpkg.com/@visx/zoom/-/zoom-2.10.0.tgz#143248813a35d2057eaf1a6336011d8650955533" - integrity sha512-sId1kuO3NvlzQTOorjeMWXRR3J44zQm8sofwKEt3O9IgaBZ49WzuPUm/owSdVT+YGsXnvxEr2qAdt26GRMzS7Q== - dependencies: - "@types/react" "*" - "@use-gesture/react" "^10.0.0-beta.22" - "@visx/event" "2.6.0" - prop-types "^15.6.2" - "@webassemblyjs/ast@1.11.1": version "1.11.1" resolved "https://registry.yarnpkg.com/@webassemblyjs/ast/-/ast-1.11.1.tgz#2bfd767eae1a6996f432ff7e8d7fc75679c0b6a7" From 1c07ef076108e2eb92a596ff27ea4eb6099893db Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sun, 2 Jul 2023 20:38:08 +0200 Subject: [PATCH 019/533] Fix SemVer link in security documentation (#32320) The SemVer link was an md-style one, we should change it rst-style --- docs/apache-airflow/security/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/security/index.rst b/docs/apache-airflow/security/index.rst index 3573348f1006..4b80e06e8d2d 100644 --- a/docs/apache-airflow/security/index.rst +++ b/docs/apache-airflow/security/index.rst @@ -156,7 +156,7 @@ MUST upgrade to the latest ``MINOR`` version of Airflow. Releasing Airflow providers with security patches ------------------------------------------------- -Similarly to Airflow, providers use strict [SemVer](https://semver.org) versioning policy, and the same +Similarly to Airflow, providers use strict `SemVer `_ versioning policy, and the same policies apply for providers as for Airflow itself. This means that you need to upgrade to the latest ``MINOR`` version of the provider to get the latest security fixes. Airflow providers are released independently from Airflow itself and the information about vulnerabilities From 88da71ed1fdffc558de28d5c3fd78e5ae1ac4e8c Mon Sep 17 00:00:00 2001 From: Sai Pragna Etikyala Date: Sun, 2 Jul 2023 14:26:32 -0700 Subject: [PATCH 020/533] Add Twilio to the list of Apache Airflow platform users (#32314) --------- Co-authored-by: Pankaj Koti --- INTHEWILD.md | 1 + 1 file changed, 1 insertion(+) diff --git a/INTHEWILD.md b/INTHEWILD.md index da58858618d4..d203db35c5ca 100644 --- a/INTHEWILD.md +++ b/INTHEWILD.md @@ -460,6 +460,7 @@ Currently, **officially** using Airflow: 1. [Travix](https://www.travix.com/) 1. [Trocafone](https://www.trocafone.com/) [[@idontdomath](https://github.com/idontdomath) & [@gseva](https://github.com/gseva) & [@ordonezf](https://github.com/ordonezf) & [@PalmaLeandro](https://github.com/PalmaLeandro)] 1. [TruFactor](https://trufactor.io/) [[@gholmes](https://github.com/gholmes) & [@angadsingh](https://github.com/angadsingh/)] +1. [Twilio](https://www.twilio.com/) [[@twilio](https://github.com/twilio)] 1. [Twine Labs](https://www.twinelabs.com/) [[@ivorpeles](https://github.com/ivorpeles)] 1. [Twitter](https://www.twitter.com/) [[@aoen](https://github.com/aoen)] 1. [Ubisoft](https://www.ubisoft.com/) [[@Walkoss](https://github.com/Walkoss)] From 5c72befcfde63ade2870491cfeb708675399d9d6 Mon Sep 17 00:00:00 2001 From: Elad Galili <33863800+eladi99@users.noreply.github.com> Date: Mon, 3 Jul 2023 09:45:24 +0300 Subject: [PATCH 021/533] Fix `LambdaInvokeFunctionOperator` payload parameter type (#32259) * Fixing issue - Fix payload parameter of amazon LambdaCreateFunctionOperator --------- Co-authored-by: Elad Galili --- .../amazon/aws/hooks/lambda_function.py | 5 +++- .../amazon/aws/operators/lambda_function.py | 2 +- .../amazon/aws/hooks/test_lambda_function.py | 11 ++++++--- .../aws/operators/test_lambda_function.py | 23 +++++++++++++------ 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/airflow/providers/amazon/aws/hooks/lambda_function.py b/airflow/providers/amazon/aws/hooks/lambda_function.py index 2d61f0751f7c..58ecac8bcccb 100644 --- a/airflow/providers/amazon/aws/hooks/lambda_function.py +++ b/airflow/providers/amazon/aws/hooks/lambda_function.py @@ -48,7 +48,7 @@ def invoke_lambda( invocation_type: str | None = None, log_type: str | None = None, client_context: str | None = None, - payload: str | None = None, + payload: bytes | str | None = None, qualifier: str | None = None, ): """ @@ -65,6 +65,9 @@ def invoke_lambda( :param payload: The JSON that you want to provide to your Lambda function as input. :param qualifier: AWS Lambda Function Version or Alias Name """ + if isinstance(payload, str): + payload = payload.encode() + invoke_args = { "FunctionName": function_name, "InvocationType": invocation_type, diff --git a/airflow/providers/amazon/aws/operators/lambda_function.py b/airflow/providers/amazon/aws/operators/lambda_function.py index 93907634c12d..28b631320422 100644 --- a/airflow/providers/amazon/aws/operators/lambda_function.py +++ b/airflow/providers/amazon/aws/operators/lambda_function.py @@ -150,7 +150,7 @@ def __init__( qualifier: str | None = None, invocation_type: str | None = None, client_context: str | None = None, - payload: str | None = None, + payload: bytes | str | None = None, aws_conn_id: str = "aws_default", **kwargs, ): diff --git a/tests/providers/amazon/aws/hooks/test_lambda_function.py b/tests/providers/amazon/aws/hooks/test_lambda_function.py index f21c000ea6d9..caaf164be47b 100644 --- a/tests/providers/amazon/aws/hooks/test_lambda_function.py +++ b/tests/providers/amazon/aws/hooks/test_lambda_function.py @@ -26,6 +26,7 @@ FUNCTION_NAME = "test_function" PAYLOAD = '{"hello": "airflow"}' +BYTES_PAYLOAD = b'{"hello": "airflow"}' RUNTIME = "python3.9" ROLE = "role" HANDLER = "handler" @@ -48,13 +49,17 @@ def test_get_conn_returns_a_boto3_connection(self, hook): @mock.patch( "airflow.providers.amazon.aws.hooks.lambda_function.LambdaHook.conn", new_callable=mock.PropertyMock ) - def test_invoke_lambda(self, mock_conn): + @pytest.mark.parametrize( + "payload, invoke_payload", + [(PAYLOAD, BYTES_PAYLOAD), (BYTES_PAYLOAD, BYTES_PAYLOAD)], + ) + def test_invoke_lambda(self, mock_conn, payload, invoke_payload): hook = LambdaHook() - hook.invoke_lambda(function_name=FUNCTION_NAME, payload=PAYLOAD) + hook.invoke_lambda(function_name=FUNCTION_NAME, payload=payload) mock_conn().invoke.assert_called_once_with( FunctionName=FUNCTION_NAME, - Payload=PAYLOAD, + Payload=invoke_payload, ) @pytest.mark.parametrize( diff --git a/tests/providers/amazon/aws/operators/test_lambda_function.py b/tests/providers/amazon/aws/operators/test_lambda_function.py index 6f1d98e8ac8b..f0b4b834eb00 100644 --- a/tests/providers/amazon/aws/operators/test_lambda_function.py +++ b/tests/providers/amazon/aws/operators/test_lambda_function.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import json from unittest import mock from unittest.mock import Mock, patch @@ -30,6 +29,8 @@ ) FUNCTION_NAME = "function_name" +PAYLOAD = '{"hello": "airflow"}' +BYTES_PAYLOAD = b'{"hello": "airflow"}' ROLE_ARN = "role_arn" IMAGE_URI = "image_uri" @@ -70,29 +71,37 @@ def test_create_lambda_with_wait_for_completion(self, mock_hook_conn, mock_hook_ class TestLambdaInvokeFunctionOperator: - def test_init(self): + @pytest.mark.parametrize( + "payload", + [PAYLOAD, BYTES_PAYLOAD], + ) + def test_init(self, payload): lambda_operator = LambdaInvokeFunctionOperator( task_id="test", function_name="test", - payload=json.dumps({"TestInput": "Testdata"}), + payload=payload, log_type="None", aws_conn_id="aws_conn_test", ) assert lambda_operator.task_id == "test" assert lambda_operator.function_name == "test" - assert lambda_operator.payload == json.dumps({"TestInput": "Testdata"}) + assert lambda_operator.payload == payload assert lambda_operator.log_type == "None" assert lambda_operator.aws_conn_id == "aws_conn_test" @patch.object(LambdaInvokeFunctionOperator, "hook", new_callable=mock.PropertyMock) - def test_invoke_lambda(self, hook_mock): + @pytest.mark.parametrize( + "payload", + [PAYLOAD, BYTES_PAYLOAD], + ) + def test_invoke_lambda(self, hook_mock, payload): operator = LambdaInvokeFunctionOperator( task_id="task_test", function_name="a", invocation_type="b", log_type="c", client_context="d", - payload="e", + payload=payload, qualifier="f", ) returned_payload = Mock() @@ -111,7 +120,7 @@ def test_invoke_lambda(self, hook_mock): invocation_type="b", log_type="c", client_context="d", - payload="e", + payload=payload, qualifier="f", ) From b6eeccdfb67409f5035adc3ab8af97c4c03a40dc Mon Sep 17 00:00:00 2001 From: Pierre Jeambrun Date: Mon, 3 Jul 2023 09:42:26 +0200 Subject: [PATCH 022/533] Fix ts/eslint warnings (#32321) --- airflow/www/alias-rest-types.js | 1 + airflow/www/babel.config.js | 2 +- airflow/www/jest-setup.js | 2 ++ airflow/www/package.json | 2 +- airflow/www/static/js/api/useDatasets.ts | 2 +- airflow/www/static/js/api/useExtraLinks.ts | 7 +++++-- airflow/www/static/js/callModal.js | 3 +++ .../static/js/cluster-activity/index.test.tsx | 12 +++++++----- .../www/static/js/components/Table/Cells.tsx | 2 ++ .../static/js/components/Table/Table.test.tsx | 2 +- .../www/static/js/components/Table/index.tsx | 6 ++++-- airflow/www/static/js/components/Tooltip.tsx | 3 ++- airflow/www/static/js/connection_form.js | 4 ++++ .../www/static/js/dag/details/graph/utils.ts | 2 +- .../static/js/dag/grid/dagRuns/index.test.tsx | 1 + airflow/www/static/js/dag/grid/index.test.tsx | 1 + airflow/www/static/js/dag_dependencies.js | 2 ++ airflow/www/static/js/dags.js | 1 + airflow/www/static/js/datasets/List.test.tsx | 12 +++++++++--- airflow/www/static/js/datasets/List.tsx | 2 ++ airflow/www/static/js/datetime_utils.js | 1 + airflow/www/static/js/gantt.js | 13 ++++++++++++- airflow/www/static/js/graph.js | 1 + airflow/www/static/js/main.js | 19 ++++++++++--------- airflow/www/static/js/task.js | 2 ++ airflow/www/static/js/task_instances.js | 2 ++ airflow/www/static/js/ti_log.js | 9 ++++++--- .../static/js/types/react-table-config.d.ts | 4 ++-- .../static/js/utils/URLSearchParamWrapper.ts | 3 ++- airflow/www/static/js/utils/graph.ts | 14 +++++++++++--- 30 files changed, 100 insertions(+), 37 deletions(-) diff --git a/airflow/www/alias-rest-types.js b/airflow/www/alias-rest-types.js index 83ddd89aaee8..c2767acca6e8 100644 --- a/airflow/www/alias-rest-types.js +++ b/airflow/www/alias-rest-types.js @@ -225,6 +225,7 @@ function generate(file) { fs.writeFileSync(file, finalText, (err) => { if (err) { + // eslint-disable-next-line no-console console.error(err); } }); diff --git a/airflow/www/babel.config.js b/airflow/www/babel.config.js index f38c60b91fb1..7f40a9fdf6b3 100644 --- a/airflow/www/babel.config.js +++ b/airflow/www/babel.config.js @@ -17,7 +17,7 @@ * under the License. */ -module.exports = function (api) { +module.exports = (api) => { api.cache(true); const presets = [ diff --git a/airflow/www/jest-setup.js b/airflow/www/jest-setup.js index fcab19bc8f41..b2e403c25ef9 100644 --- a/airflow/www/jest-setup.js +++ b/airflow/www/jest-setup.js @@ -32,8 +32,10 @@ axios.defaults.adapter = require("axios/lib/adapters/http"); axios.interceptors.response.use((res) => res.data || res); setLogger({ + /* eslint-disable no-console */ log: console.log, warn: console.warn, + /* eslint-enable no-console */ // ✅ no more errors on the console error: () => {}, }); diff --git a/airflow/www/package.json b/airflow/www/package.json index cf1727b45abc..c5a9c69e6c1a 100644 --- a/airflow/www/package.json +++ b/airflow/www/package.json @@ -7,7 +7,7 @@ "dev": "NODE_ENV=development webpack --watch --progress --devtool eval-cheap-source-map --mode development", "prod": "NODE_ENV=production node --max_old_space_size=4096 ./node_modules/webpack/bin/webpack.js --mode production --progress", "build": "NODE_ENV=production webpack --progress --mode production", - "lint": "eslint --ignore-path=.eslintignore --ext .js,.jsx,.ts,.tsx . && tsc", + "lint": "eslint --ignore-path=.eslintignore --max-warnings=0 --ext .js,.jsx,.ts,.tsx . && tsc", "lint:fix": "eslint --fix --ignore-path=.eslintignore --ext .js,.jsx,.ts,.tsx . && tsc", "format": "yarn prettier --write .", "generate-api-types": "npx openapi-typescript \"../api_connexion/openapi/v1.yaml\" --output static/js/types/api-generated.ts && node alias-rest-types.js static/js/types/api-generated.ts" diff --git a/airflow/www/static/js/api/useDatasets.ts b/airflow/www/static/js/api/useDatasets.ts index 379d3c3e61b2..62d2583d905a 100644 --- a/airflow/www/static/js/api/useDatasets.ts +++ b/airflow/www/static/js/api/useDatasets.ts @@ -24,7 +24,7 @@ import { getMetaValue } from "src/utils"; import type { DatasetListItem } from "src/types"; import type { unitOfTime } from "moment"; -interface DatasetsData { +export interface DatasetsData { datasets: DatasetListItem[]; totalEntries: number; } diff --git a/airflow/www/static/js/api/useExtraLinks.ts b/airflow/www/static/js/api/useExtraLinks.ts index 302d58aa536c..3d654e8cbc75 100644 --- a/airflow/www/static/js/api/useExtraLinks.ts +++ b/airflow/www/static/js/api/useExtraLinks.ts @@ -46,14 +46,16 @@ export default function useExtraLinks({ async () => { const data = await Promise.all( extraLinks.map(async (link) => { - mapIndex ??= -1; + const definedMapIndex = mapIndex ?? -1; const url = `${extraLinksUrl}?task_id=${encodeURIComponent( taskId )}&dag_id=${encodeURIComponent( dagId )}&execution_date=${encodeURIComponent( executionDate - )}&link_name=${encodeURIComponent(link)}&map_index=${mapIndex}`; + )}&link_name=${encodeURIComponent( + link + )}&map_index=${definedMapIndex}`; try { const datum = await axios.get(url); return { @@ -61,6 +63,7 @@ export default function useExtraLinks({ url: datum.url, }; } catch (e) { + // eslint-disable-next-line no-console console.error(e); return { name: link, diff --git a/airflow/www/static/js/callModal.js b/airflow/www/static/js/callModal.js index a2cac9f26b42..050c7eebffa2 100644 --- a/airflow/www/static/js/callModal.js +++ b/airflow/www/static/js/callModal.js @@ -56,6 +56,7 @@ const showExternalLogRedirect = const buttons = Array.from( document.querySelectorAll('a[id^="btn_"][data-base-url]') ).reduce((obj, elm) => { + // eslint-disable-next-line no-param-reassign obj[elm.id.replace("btn_", "")] = elm; return obj; }, {}); @@ -64,12 +65,14 @@ function updateButtonUrl(elm, params) { let url = elm.dataset.baseUrl; if (params.dag_id && elm.dataset.baseUrl.indexOf(dagId) !== -1) { url = url.replace(dagId, params.dag_id); + // eslint-disable-next-line no-param-reassign delete params.dag_id; } if ( Object.prototype.hasOwnProperty.call(params, "map_index") && params.map_index === undefined ) { + // eslint-disable-next-line no-param-reassign delete params.map_index; } elm.setAttribute("href", `${url}?${$.param(params)}`); diff --git a/airflow/www/static/js/cluster-activity/index.test.tsx b/airflow/www/static/js/cluster-activity/index.test.tsx index f12c4c159abc..13f6d4234434 100644 --- a/airflow/www/static/js/cluster-activity/index.test.tsx +++ b/airflow/www/static/js/cluster-activity/index.test.tsx @@ -29,6 +29,8 @@ import * as useHealthModule from "src/api/useHealth"; import { render } from "@testing-library/react"; import { Wrapper } from "src/utils/testUtils"; +import type { UseQueryResult } from "react-query"; +import type { API, HistoricalMetricsData } from "src/types"; import ClusterActivity from "."; const mockHistoricalMetricsData = { @@ -100,7 +102,7 @@ describe("Test ToggleGroups", () => { ({ data: mockHistoricalMetricsData, isSuccess: true, - } as any) + } as never as UseQueryResult) ); jest.spyOn(useHealthModule, "default").mockImplementation( @@ -108,7 +110,7 @@ describe("Test ToggleGroups", () => { ({ data: mockHealthData, isSuccess: true, - } as any) + } as never as UseQueryResult) ); jest.spyOn(useDagsModule, "default").mockImplementation( @@ -116,7 +118,7 @@ describe("Test ToggleGroups", () => { ({ data: mockDagsData, isSuccess: true, - } as any) + } as never as UseQueryResult) ); jest.spyOn(useDagRunsModule, "default").mockImplementation( @@ -124,7 +126,7 @@ describe("Test ToggleGroups", () => { ({ data: mockDagRunsData, isSuccess: true, - } as any) + } as never as UseQueryResult) ); jest.spyOn(usePoolsModule, "default").mockImplementation( @@ -132,7 +134,7 @@ describe("Test ToggleGroups", () => { ({ data: mockPoolsData, isSuccess: true, - } as any) + } as never as UseQueryResult) ); }); diff --git a/airflow/www/static/js/components/Table/Cells.tsx b/airflow/www/static/js/components/Table/Cells.tsx index 8084eb917c68..9e86a74935f1 100644 --- a/airflow/www/static/js/components/Table/Cells.tsx +++ b/airflow/www/static/js/components/Table/Cells.tsx @@ -41,8 +41,10 @@ import { SimpleStatus } from "src/dag/StatusBox"; interface CellProps { cell: { + // eslint-disable-next-line @typescript-eslint/no-explicit-any value: any; row: { + // eslint-disable-next-line @typescript-eslint/no-explicit-any original: Record; }; }; diff --git a/airflow/www/static/js/components/Table/Table.test.tsx b/airflow/www/static/js/components/Table/Table.test.tsx index a3664d49a237..36d9b22456e2 100644 --- a/airflow/www/static/js/components/Table/Table.test.tsx +++ b/airflow/www/static/js/components/Table/Table.test.tsx @@ -28,7 +28,7 @@ import type { SortingRule } from "react-table"; import { ChakraWrapper } from "src/utils/testUtils"; import { Table } from "."; -const data: Record[] = [ +const data: Record[] = [ { firstName: "Lamont", lastName: "Grimes", country: "United States" }, { firstName: "Alysa", lastName: "Armstrong", country: "Spain" }, { firstName: "Petra", lastName: "Blick", country: "France" }, diff --git a/airflow/www/static/js/components/Table/index.tsx b/airflow/www/static/js/components/Table/index.tsx index a2450707b385..753035da0723 100644 --- a/airflow/www/static/js/components/Table/index.tsx +++ b/airflow/www/static/js/components/Table/index.tsx @@ -89,7 +89,7 @@ interface TableProps { pageSize?: number; isLoading?: boolean; selectRows?: (selectedRows: number[]) => void; - onRowClicked?: (row: Row, e: any) => void; + onRowClicked?: (row: Row, e: unknown) => void; } export const Table = ({ @@ -244,7 +244,9 @@ export const Table = ({ } } onClick={ - onRowClicked ? (e: any) => onRowClicked(row, e) : undefined + onRowClicked + ? (e: unknown) => onRowClicked(row, e) + : undefined } > {row.cells.map((cell) => ( diff --git a/airflow/www/static/js/components/Tooltip.tsx b/airflow/www/static/js/components/Tooltip.tsx index fc09cefc1f88..72a55b0862da 100644 --- a/airflow/www/static/js/components/Tooltip.tsx +++ b/airflow/www/static/js/components/Tooltip.tsx @@ -136,7 +136,7 @@ const Tooltip = forwardRef((props, ref) => { * Ensure tooltip has only one child node */ const child = React.Children.only(children) as React.ReactElement & { - ref?: React.Ref; + ref?: React.Ref; }; const trigger: React.ReactElement = React.cloneElement( child, @@ -168,6 +168,7 @@ const Tooltip = forwardRef((props, ref) => { > { + // eslint-disable-next-line no-param-reassign elem.innerText = elem.dataset.origText; + // eslint-disable-next-line no-param-reassign delete elem.dataset.origText; } ); Array.from(document.querySelectorAll(".form-control")).forEach((elem) => { + // eslint-disable-next-line no-param-reassign elem.placeholder = ""; elem.parentElement.parentElement.classList.remove("hide"); }); @@ -259,6 +262,7 @@ $(document).ready(() => { - All other custom form fields (i.e. fields that are named ``extra__...``) in alphabetical order */ + // eslint-disable-next-line func-names $.each(inArray, function () { if (this.name === "conn_id") { outObj.connection_id = this.value; diff --git a/airflow/www/static/js/dag/details/graph/utils.ts b/airflow/www/static/js/dag/details/graph/utils.ts index 998451e05ba1..459632024692 100644 --- a/airflow/www/static/js/dag/details/graph/utils.ts +++ b/airflow/www/static/js/dag/details/graph/utils.ts @@ -131,7 +131,7 @@ export const nodeColor = ({ export const nodeStrokeColor = ( { data: { isSelected } }: ReactFlowNode, - colors: any + colors: Record ) => (isSelected ? colors.blue[500] : ""); interface BuildEdgesProps { diff --git a/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx b/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx index d77f30af322d..2ac20ea3ccbb 100644 --- a/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx +++ b/airflow/www/static/js/dag/grid/dagRuns/index.test.tsx @@ -48,6 +48,7 @@ const generateRuns = (length: number): DagRun[] => note: "someRandomValue", })); +/* eslint-disable @typescript-eslint/no-explicit-any */ describe("Test DagRuns", () => { test("Durations and manual run arrow render correctly, but without any date ticks", () => { const dagRuns: DagRun[] = [ diff --git a/airflow/www/static/js/dag/grid/index.test.tsx b/airflow/www/static/js/dag/grid/index.test.tsx index 1b3888ffae3f..66e1cafaeb40 100644 --- a/airflow/www/static/js/dag/grid/index.test.tsx +++ b/airflow/www/static/js/dag/grid/index.test.tsx @@ -135,6 +135,7 @@ describe("Test ToggleGroups", () => { const returnValue = { data: mockGridData, isSuccess: true, + // eslint-disable-next-line @typescript-eslint/no-explicit-any } as any; jest diff --git a/airflow/www/static/js/dag_dependencies.js b/airflow/www/static/js/dag_dependencies.js index ddd40c248a59..a3c4a713eac0 100644 --- a/airflow/www/static/js/dag_dependencies.js +++ b/airflow/www/static/js/dag_dependencies.js @@ -97,6 +97,7 @@ function setUpZoomSupport() { } function setUpNodeHighlighting(focusItem = null) { + // eslint-disable-next-line func-names d3.selectAll("g.node").on("mouseover", function (d) { d3.select(this).selectAll("rect").style("stroke", highlightColor); highlightNodes(g.predecessors(d), upstreamColor, highlightStrokeWidth); @@ -114,6 +115,7 @@ function setUpNodeHighlighting(focusItem = null) { }); }); + // eslint-disable-next-line func-names d3.selectAll("g.node").on("mouseout", function (d) { d3.select(this).selectAll("rect,circle").style("stroke", null); highlightNodes(g.predecessors(d), null, initialStrokeWidth); diff --git a/airflow/www/static/js/dags.js b/airflow/www/static/js/dags.js index 1854ae53a292..ea53be18ac02 100644 --- a/airflow/www/static/js/dags.js +++ b/airflow/www/static/js/dags.js @@ -320,6 +320,7 @@ function getDagIds({ activeDagsOnly = false } = {}) { dagIds = dagIds.filter(":checked"); } dagIds = dagIds + // eslint-disable-next-line func-names .map(function () { return $(this).data("dag-id"); }) diff --git a/airflow/www/static/js/datasets/List.test.tsx b/airflow/www/static/js/datasets/List.test.tsx index 34e70f70b027..c1cd0da344db 100644 --- a/airflow/www/static/js/datasets/List.test.tsx +++ b/airflow/www/static/js/datasets/List.test.tsx @@ -25,6 +25,8 @@ import { render } from "@testing-library/react"; import * as useDatasetsModule from "src/api/useDatasets"; import { Wrapper } from "src/utils/testUtils"; +import type { UseQueryResult } from "react-query"; +import type { DatasetListItem } from "src/types"; import DatasetsList from "./List"; const datasets = [ @@ -57,22 +59,26 @@ const datasets = [ }, ]; +type UseDatasetsReturn = UseQueryResult & { + data: useDatasetsModule.DatasetsData; +}; + const returnValue = { data: { datasets, totalEntries: datasets.length, }, isSuccess: true, -} as any; +} as UseDatasetsReturn; const emptyReturnValue = { data: { - datasets: [], + datasets: [] as DatasetListItem[], totalEntries: 0, }, isSuccess: true, isLoading: false, -} as any; +} as UseDatasetsReturn; describe("Test Datasets List", () => { test("Displays a list of datasets", () => { diff --git a/airflow/www/static/js/datasets/List.tsx b/airflow/www/static/js/datasets/List.tsx index e201e1d2d4cc..6bbfd2649790 100644 --- a/airflow/www/static/js/datasets/List.tsx +++ b/airflow/www/static/js/datasets/List.tsx @@ -49,8 +49,10 @@ interface Props { interface CellProps { cell: { + // eslint-disable-next-line @typescript-eslint/no-explicit-any value: any; row: { + // eslint-disable-next-line @typescript-eslint/no-explicit-any original: Record; }; }; diff --git a/airflow/www/static/js/datetime_utils.js b/airflow/www/static/js/datetime_utils.js index b9e4b76eebe5..c23f8d665e83 100644 --- a/airflow/www/static/js/datetime_utils.js +++ b/airflow/www/static/js/datetime_utils.js @@ -104,6 +104,7 @@ export function updateAllDateTimes() { // Since we have set the default timezone for moment, it will automatically // convert it to the new target for us $(".datetime input").each((_, el) => { + // eslint-disable-next-line no-param-reassign el.value = moment(el.value).format(); }); } diff --git a/airflow/www/static/js/gantt.js b/airflow/www/static/js/gantt.js index 048ba741e6d2..2b7cf10c8191 100644 --- a/airflow/www/static/js/gantt.js +++ b/airflow/www/static/js/gantt.js @@ -74,7 +74,7 @@ const replacements = { "%": "%", }; -moment.fn.strftime = function (format) { +moment.fn.strftime = function formatTime(format) { // Break up format string based on strftime tokens const tokens = format.split(/(%-?.)/); const momentFormat = tokens @@ -173,12 +173,15 @@ d3.gantt = () => { tasks.forEach((a) => { if (!(a.start_date instanceof moment)) { + // eslint-disable-next-line no-param-reassign a.start_date = moment(a.start_date); } if (!(a.end_date instanceof moment)) { + // eslint-disable-next-line no-param-reassign a.end_date = moment(a.end_date); } if (a.queued_dttm && !(a.queued_dttm instanceof moment)) { + // eslint-disable-next-line no-param-reassign a.queued_dttm = moment(a.queued_dttm); } }); @@ -349,12 +352,14 @@ d3.gantt = () => { return gantt; }; + // eslint-disable-next-line func-names gantt.margin = function (value) { if (!arguments.length) return margin; margin = value; return gantt; }; + // eslint-disable-next-line func-names gantt.timeDomain = function (value) { if (!arguments.length) return [timeDomainStart, timeDomainEnd]; timeDomainStart = +value[0]; @@ -367,36 +372,42 @@ d3.gantt = () => { * vale The value can be "fit" - the domain fits the data or * "fixed" - fixed domain. */ + // eslint-disable-next-line func-names gantt.timeDomainMode = function (value) { if (!arguments.length) return timeDomainMode; timeDomainMode = value; return gantt; }; + // eslint-disable-next-line func-names gantt.taskTypes = function (value) { if (!arguments.length) return taskTypes; taskTypes = value; return gantt; }; + // eslint-disable-next-line func-names gantt.width = function (value) { if (!arguments.length) return width; width = +value; return gantt; }; + // eslint-disable-next-line func-names gantt.height = function (value) { if (!arguments.length) return height; height = +value; return gantt; }; + // eslint-disable-next-line func-names gantt.tickFormat = function (value) { if (!arguments.length) return tickFormat; tickFormat = value; return gantt; }; + // eslint-disable-next-line func-names gantt.selector = function (value) { if (!arguments.length) return selector; selector = value; diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js index 38b1ae6211fb..b61639d3a86c 100644 --- a/airflow/www/static/js/graph.js +++ b/airflow/www/static/js/graph.js @@ -153,6 +153,7 @@ function collapseGroup(nodeId, node) { getChildrenIds(node).forEach((childId) => mapTaskToNode.set(childId, nodeId)); + // eslint-disable-next-line no-param-reassign node = g.node(nodeId); // Set children edges onto the group edge diff --git a/airflow/www/static/js/main.js b/airflow/www/static/js/main.js index 7350a5cfffa7..95861ea6c13a 100644 --- a/airflow/www/static/js/main.js +++ b/airflow/www/static/js/main.js @@ -69,19 +69,20 @@ export function escapeHtml(text) { window.escapeHtml = escapeHtml; export function convertSecsToHumanReadable(seconds) { + let processedSeconds = seconds; const oriSeconds = seconds; const floatingPart = oriSeconds - Math.floor(oriSeconds); - seconds = Math.floor(seconds); + processedSeconds = Math.floor(processedSeconds); const secondsPerHour = 60 * 60; const secondsPerMinute = 60; - const hours = Math.floor(seconds / secondsPerHour); - seconds -= hours * secondsPerHour; + const hours = Math.floor(processedSeconds / secondsPerHour); + processedSeconds -= hours * secondsPerHour; - const minutes = Math.floor(seconds / secondsPerMinute); - seconds -= minutes * secondsPerMinute; + const minutes = Math.floor(processedSeconds / secondsPerMinute); + processedSeconds -= minutes * secondsPerMinute; let readableFormat = ""; if (hours > 0) { @@ -90,12 +91,12 @@ export function convertSecsToHumanReadable(seconds) { if (minutes > 0) { readableFormat += `${minutes}Min `; } - if (seconds + floatingPart > 0) { + if (processedSeconds + floatingPart > 0) { if (Math.floor(oriSeconds) === oriSeconds) { - readableFormat += `${seconds}Sec`; + readableFormat += `${processedSeconds}Sec`; } else { - seconds += floatingPart; - readableFormat += `${seconds.toFixed(3)}Sec`; + processedSeconds += floatingPart; + readableFormat += `${processedSeconds.toFixed(3)}Sec`; } } return readableFormat; diff --git a/airflow/www/static/js/task.js b/airflow/www/static/js/task.js index df546027290b..d4c03d184894 100644 --- a/airflow/www/static/js/task.js +++ b/airflow/www/static/js/task.js @@ -27,6 +27,7 @@ document.addEventListener("DOMContentLoaded", () => { const value = attr.innerHTML; if (value.length === 32 && moment(value, "YYYY-MM-DD").isValid()) { // 32 is the length of our timestamps + // eslint-disable-next-line no-param-reassign attr.innerHTML = ""; const timeElement = document.createElement("time"); timeElement.setAttribute("datetime", value); @@ -35,6 +36,7 @@ document.addEventListener("DOMContentLoaded", () => { attr.appendChild(timeElement); } else if (validator.isURL(value)) { // very basic url detection + // eslint-disable-next-line no-param-reassign attr.innerHTML = ""; const linkElement = document.createElement("a"); linkElement.setAttribute("href", value); diff --git a/airflow/www/static/js/task_instances.js b/airflow/www/static/js/task_instances.js index 61ff19259975..c539d5c3ea16 100644 --- a/airflow/www/static/js/task_instances.js +++ b/airflow/www/static/js/task_instances.js @@ -119,12 +119,14 @@ export default function tiTooltip(ti, task, { includeTryNumber = false } = {}) { if (ti.state === "running") { const startDate = ti.start_date instanceof moment ? ti.start_date : moment(ti.start_date); + // eslint-disable-next-line no-param-reassign ti.duration = moment().diff(startDate, "second"); } else if (!ti.duration && ti.end_date) { const startDate = ti.start_date instanceof moment ? ti.start_date : moment(ti.start_date); const endDate = ti.end_date instanceof moment ? ti.end_date : moment(ti.end_date); + // eslint-disable-next-line no-param-reassign ti.duration = moment(endDate).diff(startDate, "second"); } diff --git a/airflow/www/static/js/ti_log.js b/airflow/www/static/js/ti_log.js index d2e223b103df..23fd0af76967 100644 --- a/airflow/www/static/js/ti_log.js +++ b/airflow/www/static/js/ti_log.js @@ -37,6 +37,7 @@ function recurse(delay = DELAY) { setTimeout(resolve, delay); }); } +/* eslint-disable no-console */ // Enable auto tailing only when users scroll down to the bottom // of the page. This prevent auto tailing the page if users want @@ -166,15 +167,16 @@ function autoTailingLog(tryNumber, metadata = null, autoTailing = false) { } function setDownloadUrl(tryNumber) { - if (!tryNumber) { + let tryNumberData = tryNumber; + if (!tryNumberData) { // default to the currently selected tab - tryNumber = $("#ti_log_try_number_list .active a").data("try-number"); + tryNumberData = $("#ti_log_try_number_list .active a").data("try-number"); } const query = new URLSearchParams({ dag_id: dagId, task_id: taskId, execution_date: executionDate, - try_number: tryNumber, + try_number: tryNumberData, metadata: "null", format: "file", }); @@ -187,6 +189,7 @@ $(document).ready(() => { autoTailingLog(TOTAL_ATTEMPTS, null, true); setDownloadUrl(); + // eslint-disable-next-line func-names $("#ti_log_try_number_list a").click(function () { const tryNumber = $(this).data("try-number"); diff --git a/airflow/www/static/js/types/react-table-config.d.ts b/airflow/www/static/js/types/react-table-config.d.ts index 0b62ceb16f42..2bc503ed1829 100644 --- a/airflow/www/static/js/types/react-table-config.d.ts +++ b/airflow/www/static/js/types/react-table-config.d.ts @@ -87,7 +87,7 @@ declare module "react-table" { // this matches the spirit of the underlying js library, // but might be cleaner if it's replaced by a more specific type that matches your // feature set, this is a safe default. - Record {} + Record {} export interface Hooks< D extends Record = Record @@ -139,7 +139,7 @@ declare module "react-table" { export interface Cell< D extends Record = Record, // eslint-disable-next-line @typescript-eslint/no-unused-vars - V = any + V = unknown > extends UseGroupByCellProps, UseRowStateCellProps {} diff --git a/airflow/www/static/js/utils/URLSearchParamWrapper.ts b/airflow/www/static/js/utils/URLSearchParamWrapper.ts index befeadcaf5d9..476db61b9067 100644 --- a/airflow/www/static/js/utils/URLSearchParamWrapper.ts +++ b/airflow/www/static/js/utils/URLSearchParamWrapper.ts @@ -18,7 +18,8 @@ */ class URLSearchParamsWrapper extends URLSearchParams { - constructor(init?: { [keys: string]: any }) { + // eslint-disable-next-line @typescript-eslint/no-explicit-any + constructor(init?: Record) { if (init) { const stringValues: { [keys: string]: string } = {}; Object.keys(init).forEach((key) => { diff --git a/airflow/www/static/js/utils/graph.ts b/airflow/www/static/js/utils/graph.ts index e69a3045d945..2dd970bf8c10 100644 --- a/airflow/www/static/js/utils/graph.ts +++ b/airflow/www/static/js/utils/graph.ts @@ -77,13 +77,21 @@ const generateGraph = ({ }: GenerateProps) => { const closedGroupIds: string[] = []; - const formatChildNode = (node: any) => { + const formatChildNode = ( + node: DepNode + ): DepNode & { + label: string; + layoutOptions?: Record; + width?: number; + height?: number; + } => { const { id, value, children } = node; const isOpen = openGroupIds?.includes(value.label); const childCount = - children?.filter((c: any) => !c.id.includes("join_id")).length || 0; - if (isOpen && children.length) { + children?.filter((c: DepNode) => !c.id.includes("join_id")).length || 0; + if (isOpen && children?.length) { return { + ...node, id, value: { ...value, From 1f22db7ed395ab88270faa3c1eee6c7ade53fcc7 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 3 Jul 2023 15:57:28 +0800 Subject: [PATCH 023/533] Fixup config var types under the scheduler section (#32132) --- airflow/config_templates/config.yml | 38 ++++++++++++++--------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index f48a569b545f..88aa055a3901 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -2256,7 +2256,7 @@ scheduler: from the CLI or the UI), this defines the frequency at which they should listen (in seconds). version_added: ~ - type: string + type: float example: ~ default: "5" scheduler_heartbeat_sec: @@ -2265,7 +2265,7 @@ scheduler: scheduler section in the docs for more information). This defines how often the scheduler should run (in seconds). version_added: ~ - type: string + type: integer example: ~ default: "5" num_runs: @@ -2273,7 +2273,7 @@ scheduler: The number of times to try to schedule each DAG file -1 indicates unlimited number version_added: 1.10.6 - type: string + type: integer example: ~ default: "-1" scheduler_idle_sleep_time: @@ -2282,7 +2282,7 @@ scheduler: in the loop. i.e. if it scheduled something then it will start the next loop iteration straight away. version_added: 2.2.0 - type: string + type: float example: ~ default: "1" min_file_process_interval: @@ -2291,7 +2291,7 @@ scheduler: ``min_file_process_interval`` number of seconds. Updates to DAGs are reflected after this interval. Keeping this number low will increase CPU usage. version_added: ~ - type: string + type: integer example: ~ default: "30" parsing_cleanup_interval: @@ -2319,14 +2319,14 @@ scheduler: description: | How often (in seconds) to scan the DAGs directory for new files. Default to 5 minutes. version_added: ~ - type: string + type: integer example: ~ default: "300" print_stats_interval: description: | How often should stats be printed to the logs. Setting to 0 will disable printing stats version_added: ~ - type: string + type: integer example: ~ default: "30" pool_metrics_interval: @@ -2343,7 +2343,7 @@ scheduler: This is used by the health check in the "/health" endpoint and in `airflow jobs check` CLI for SchedulerJob. version_added: 1.10.2 - type: string + type: integer example: ~ default: "30" enable_health_check: @@ -2359,7 +2359,7 @@ scheduler: When you start a scheduler, airflow starts a tiny web server subprocess to serve a health check on this port version_added: 2.4.0 - type: string + type: integer example: ~ default: "8974" orphaned_tasks_check_interval: @@ -2381,7 +2381,7 @@ scheduler: not heartbeat in this many seconds, the scheduler will mark the associated task instance as failed and will re-schedule the task. version_added: ~ - type: string + type: integer example: ~ default: "300" zombie_detection_interval: @@ -2400,7 +2400,7 @@ scheduler: however it can be set on a per DAG basis in the DAG definition (catchup) version_added: ~ - type: string + type: boolean example: ~ default: "True" ignore_first_depends_on_past_by_default: @@ -2411,7 +2411,7 @@ scheduler: in the DB with an execution_date earlier than it., i.e. no manual marking success will be needed for a newly added task to be scheduled. version_added: 2.3.0 - type: string + type: boolean example: ~ default: "True" max_tis_per_query: @@ -2422,7 +2422,7 @@ scheduler: Additionally, you may hit the maximum allowable query length for your db. Set this to 0 for no limit (not advised) version_added: ~ - type: string + type: integer example: ~ default: "512" use_row_level_locking: @@ -2439,7 +2439,7 @@ scheduler: Max number of DAGs to create DagRuns for per scheduler loop. example: ~ version_added: 2.0.0 - type: string + type: integer default: "10" see_also: ":ref:`scheduler:ha:tunables`" max_dagruns_per_loop_to_schedule: @@ -2448,7 +2448,7 @@ scheduler: and queuing tasks. example: ~ version_added: 2.0.0 - type: string + type: integer default: "20" see_also: ":ref:`scheduler:ha:tunables`" schedule_after_task_execution: @@ -2475,7 +2475,7 @@ scheduler: The scheduler can run multiple processes in parallel to parse dags. This defines how many processes will run. version_added: 1.10.14 - type: string + type: integer example: ~ default: "2" file_parsing_sort_mode: @@ -2522,7 +2522,7 @@ scheduler: Turn off scheduler use of cron intervals by setting this to False. DAGs submitted manually in the web UI or with trigger_dag will still run. version_added: 1.10.2 - type: string + type: boolean example: ~ default: "True" allow_trigger_in_future: @@ -2530,14 +2530,14 @@ scheduler: Allow externally triggered DagRuns for Execution Dates in the future Only has effect if schedule_interval is set to None in DAG version_added: 1.10.8 - type: string + type: boolean example: ~ default: "False" trigger_timeout_check_interval: description: | How often to check for expired trigger requests that have not run yet. version_added: 2.2.0 - type: string + type: float example: ~ default: "15" task_queued_timeout: From c03d7b7c337df7eee4cc7722a1c8da56abd7027a Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Mon, 3 Jul 2023 16:09:57 +0530 Subject: [PATCH 024/533] Adds Sensor section in the Azure providers docs (#32299) * added transfers section: * spell * spell * spell * static-checks * fixed static checks * doc added for local_to_wasb * added sensors section * added sensors section * fixed static check --- .../microsoft/azure/example_dag/__init__.py | 16 +++++ .../example_cosmos_document_sensor.py | 53 ++++++++++++++++ .../azure/example_dag/example_wasb_sensors.py | 61 +++++++++++++++++++ .../index.rst | 1 + .../sensors/cosmos_document_sesnor.rst | 35 +++++++++++ .../sensors/index.rst | 27 ++++++++ .../sensors/wasb_sensors.rst | 48 +++++++++++++++ docs/spelling_wordlist.txt | 1 + 8 files changed, 242 insertions(+) create mode 100644 airflow/providers/microsoft/azure/example_dag/__init__.py create mode 100644 airflow/providers/microsoft/azure/example_dag/example_cosmos_document_sensor.py create mode 100644 airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py create mode 100644 docs/apache-airflow-providers-microsoft-azure/sensors/cosmos_document_sesnor.rst create mode 100644 docs/apache-airflow-providers-microsoft-azure/sensors/index.rst create mode 100644 docs/apache-airflow-providers-microsoft-azure/sensors/wasb_sensors.rst diff --git a/airflow/providers/microsoft/azure/example_dag/__init__.py b/airflow/providers/microsoft/azure/example_dag/__init__.py new file mode 100644 index 000000000000..13a83393a912 --- /dev/null +++ b/airflow/providers/microsoft/azure/example_dag/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/airflow/providers/microsoft/azure/example_dag/example_cosmos_document_sensor.py b/airflow/providers/microsoft/azure/example_dag/example_cosmos_document_sensor.py new file mode 100644 index 000000000000..198420147eb0 --- /dev/null +++ b/airflow/providers/microsoft/azure/example_dag/example_cosmos_document_sensor.py @@ -0,0 +1,53 @@ +# +# 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. +""" +Example Airflow DAG that senses document in Azure Cosmos DB. + +This DAG relies on the following OS environment variables + +* DATABASE_NAME - Target CosmosDB database_name. +* COLLECTION_NAME - Target CosmosDB collection_name. +* DOCUMENT_ID - The ID of the target document. +""" +from __future__ import annotations + +import os +from datetime import datetime + +from airflow.models import DAG +from airflow.providers.microsoft.azure.sensors.cosmos import AzureCosmosDocumentSensor + +DATABASE_NAME = os.environ.get("DATABASE_NAME", "example-database-name") +COLLECTION_NAME = os.environ.get("COLLECTION_NAME", "example-collection-name") +DOCUMENT_ID = os.environ.get("DOCUMENT_ID", "example-document-id") + + +with DAG( + "example_cosmos_document_sensor", + start_date=datetime(2022, 8, 8), + catchup=False, + tags=["example"], +) as dag: + # [START cosmos_document_sensor] + azure_wasb_sensor = AzureCosmosDocumentSensor( + database_name=DATABASE_NAME, + collection_name=COLLECTION_NAME, + document_id=DOCUMENT_ID, + task_id="cosmos_document_sensor", + ) + # [END cosmos_document_sensor] diff --git a/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py b/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py new file mode 100644 index 000000000000..349e09c44ad9 --- /dev/null +++ b/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py @@ -0,0 +1,61 @@ +# +# 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. +""" +Example Airflow DAG that senses blob(s) in Azure Blob Storage. + +This DAG relies on the following OS environment variables + +* CONTAINER_NAME - The container under which to look for the blob. +* BLOB_NAME - The name of the blob to match. +* PREFIX - The blob with the specified prefix to match. +""" +from __future__ import annotations + +import os +from datetime import datetime + +from airflow.models import DAG +from airflow.providers.microsoft.azure.sensors.wasb import WasbBlobSensor, WasbPrefixSensor + +CONTAINER_NAME = os.environ.get("CONTAINER_NAME", "example-container-name") +BLOB_NAME = os.environ.get("BLOB_NAME", "example-blob-name") +PREFIX = os.environ.get("PREFIX", "example-prefix") + + +with DAG( + "example_wasb_sensors", + start_date=datetime(2022, 8, 8), + catchup=False, + tags=["example"], +) as dag: + # [START wasb_blob_sensor] + azure_wasb_sensor = WasbBlobSensor( + container_name=CONTAINER_NAME, + blob_name=BLOB_NAME, + task_id="wasb_sense_blob", + ) + # [END wasb_blob_sensor] + + # [START wasb_prefix_sensor] + azure_wasb_prefix_sensor = WasbPrefixSensor( + container_name=CONTAINER_NAME, + blob_name=BLOB_NAME, + prefix=PREFIX, + task_id="wasb_sense_prefix", + ) + # [END wasb_prefix_sensor] diff --git a/docs/apache-airflow-providers-microsoft-azure/index.rst b/docs/apache-airflow-providers-microsoft-azure/index.rst index dfc0bf56e14e..a10c5017314a 100644 --- a/docs/apache-airflow-providers-microsoft-azure/index.rst +++ b/docs/apache-airflow-providers-microsoft-azure/index.rst @@ -39,6 +39,7 @@ Transfers Secrets backends Logging for Tasks + Sensors .. toctree:: :hidden: diff --git a/docs/apache-airflow-providers-microsoft-azure/sensors/cosmos_document_sesnor.rst b/docs/apache-airflow-providers-microsoft-azure/sensors/cosmos_document_sesnor.rst new file mode 100644 index 000000000000..d4b563120452 --- /dev/null +++ b/docs/apache-airflow-providers-microsoft-azure/sensors/cosmos_document_sesnor.rst @@ -0,0 +1,35 @@ + .. 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. + +Azure Cosmos DB +================== +Cosmos Database (DB) is a globally distributed, low latency, multi-model database for managing data at large scales. +It is a cloud-based NoSQL database offered as a PaaS (Platform as a Service) from Microsoft Azure. +It is a highly available, high throughput, reliable database and is often called a serverless database. +Cosmos database contains the Azure Document DB and is available everywhere. + +Azure Cosmos Document Sensor +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Checks for the existence of a document which matches the given query in CosmosDB. +:class:`~airflow.providers.microsoft.azure.sensors.cosmos.AzureCosmosDocumentSensor` + +.. exampleinclude:: /../../airflow/providers/microsoft/azure/example_dag/example_cosmos_document_sensor.py + :language: python + :dedent: 4 + :start-after: [START cosmos_document_sensor] + :end-before: [END cosmos_document_sensor] diff --git a/docs/apache-airflow-providers-microsoft-azure/sensors/index.rst b/docs/apache-airflow-providers-microsoft-azure/sensors/index.rst new file mode 100644 index 000000000000..439d2348972c --- /dev/null +++ b/docs/apache-airflow-providers-microsoft-azure/sensors/index.rst @@ -0,0 +1,27 @@ + .. 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. + + + +Microsoft Azure Sensors +======================= + +.. toctree:: + :maxdepth: 1 + :glob: + + * diff --git a/docs/apache-airflow-providers-microsoft-azure/sensors/wasb_sensors.rst b/docs/apache-airflow-providers-microsoft-azure/sensors/wasb_sensors.rst new file mode 100644 index 000000000000..f901928d9b0f --- /dev/null +++ b/docs/apache-airflow-providers-microsoft-azure/sensors/wasb_sensors.rst @@ -0,0 +1,48 @@ + .. 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. + +Azure Blob Storage +================== +The Blob service stores text and binary data as objects in the cloud. +The Blob service offers the following three resources: the storage account, containers, and blobs. +Within your storage account, containers provide a way to organize sets of blobs. +For more information about the service visit `Azure Blob Storage API documentation `_. +This page shows how to check for blobs in a particular container. + +Wasb Blob Sensor +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Waits for a blob to arrive on Azure Blob Storage. +:class:`~airflow.providers.microsoft.azure.sensors.wasb.WasbBlobSensor` + +.. exampleinclude:: /../../airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py + :language: python + :dedent: 4 + :start-after: [START wasb_blob_sensor] + :end-before: [END wasb_blob_sensor] + +Wasb Prefix Sensor +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Waits for blobs matching a prefix to arrive on Azure Blob Storage. +:class:`~airflow.providers.microsoft.azure.sensors.wasb.WasbPrefixSensor` + +.. exampleinclude:: /../../airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py + :language: python + :dedent: 4 + :start-after: [START wasb_prefix_sensor] + :end-before: [END wasb_prefix_sensor] diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index af0a3ff5dcbf..6389ba181420 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -1056,6 +1056,7 @@ ot otel overridable oversubscription +PaaS Pagerduty pagerduty pageviews From 1065687ec6df2b9b3557e38a67e71f835796427f Mon Sep 17 00:00:00 2001 From: Phani Kumar <94376113+phanikumv@users.noreply.github.com> Date: Mon, 3 Jul 2023 17:49:51 +0530 Subject: [PATCH 025/533] Refactor Sqlalchemy queries to 2.0 style (Part 3) (#32177) --- airflow/utils/db.py | 38 ++-- airflow/www/utils.py | 11 +- airflow/www/views.py | 460 ++++++++++++++++++++++--------------------- 3 files changed, 265 insertions(+), 244 deletions(-) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index a76f0d4f675d..46ddcbfb3453 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -92,7 +92,7 @@ def _format_airflow_moved_table_name(source_table, version, category): @provide_session def merge_conn(conn, session: Session = NEW_SESSION): """Add new Connection.""" - if not session.query(conn.__class__).filter_by(conn_id=conn.conn_id).first(): + if not session.scalar(select(conn.__class__).filter_by(conn_id=conn.conn_id).limit(1)): session.add(conn) session.commit() @@ -959,7 +959,9 @@ def check_conn_id_duplicates(session: Session) -> Iterable[str]: dups = [] try: - dups = session.query(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1).all() + dups = session.execute( + select(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1) + ).all() except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -984,12 +986,11 @@ def check_username_duplicates(session: Session) -> Iterable[str]: for model in [User, RegisterUser]: dups = [] try: - dups = ( - session.query(model.username) # type: ignore[attr-defined] + dups = session.execute( + select(model.username) # type: ignore[attr-defined] .group_by(model.username) # type: ignore[attr-defined] .having(func.count() > 1) - .all() - ) + ).all() except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -1058,13 +1059,13 @@ def check_task_fail_for_duplicates(session): """ minimal_table_obj = table(table_name, *[column(x) for x in uniqueness]) try: - subquery = ( - session.query(minimal_table_obj, func.count().label("dupe_count")) + subquery = session.execute( + select(minimal_table_obj, func.count().label("dupe_count")) .group_by(*[text(x) for x in uniqueness]) .having(func.count() > text("1")) .subquery() ) - dupe_count = session.query(func.sum(subquery.c.dupe_count)).scalar() + dupe_count = session.scalar(select(func.sum(subquery.c.dupe_count))) if not dupe_count: # there are no duplicates; nothing to do. return @@ -1101,7 +1102,7 @@ def check_conn_type_null(session: Session) -> Iterable[str]: n_nulls = [] try: - n_nulls = session.query(Connection.conn_id).filter(Connection.conn_type.is_(None)).all() + n_nulls = session.scalars(select(Connection.conn_id).where(Connection.conn_type.is_(None))).all() except (exc.OperationalError, exc.ProgrammingError, exc.InternalError): # fallback if tables hasn't been created yet session.rollback() @@ -1143,7 +1144,7 @@ def check_run_id_null(session: Session) -> Iterable[str]: dagrun_table.c.run_id.is_(None), dagrun_table.c.execution_date.is_(None), ) - invalid_dagrun_count = session.query(func.count(dagrun_table.c.id)).filter(invalid_dagrun_filter).scalar() + invalid_dagrun_count = session.scalar(select(func.count(dagrun_table.c.id)).where(invalid_dagrun_filter)) if invalid_dagrun_count > 0: dagrun_dangling_table_name = _format_airflow_moved_table_name(dagrun_table.name, "2.2", "dangling") if dagrun_dangling_table_name in inspect(session.get_bind()).get_table_names(): @@ -1240,7 +1241,7 @@ def _move_dangling_data_to_new_table( pk_cols = source_table.primary_key.columns delete = source_table.delete().where( - tuple_(*pk_cols).in_(session.query(*target_table.primary_key.columns).subquery()) + tuple_(*pk_cols).in_(session.select(*target_table.primary_key.columns).subquery()) ) else: delete = source_table.delete().where( @@ -1262,10 +1263,11 @@ def _dangling_against_dag_run(session, source_table, dag_run): source_table.c.dag_id == dag_run.c.dag_id, source_table.c.execution_date == dag_run.c.execution_date, ) + return ( - session.query(*[c.label(c.name) for c in source_table.c]) + select(*[c.label(c.name) for c in source_table.c]) .join(dag_run, source_to_dag_run_join_cond, isouter=True) - .filter(dag_run.c.dag_id.is_(None)) + .where(dag_run.c.dag_id.is_(None)) ) @@ -1304,10 +1306,10 @@ def _dangling_against_task_instance(session, source_table, dag_run, task_instanc ) return ( - session.query(*[c.label(c.name) for c in source_table.c]) + select(*[c.label(c.name) for c in source_table.c]) .join(dag_run, dr_join_cond, isouter=True) .join(task_instance, ti_join_cond, isouter=True) - .filter(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) + .where(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) ) @@ -1331,9 +1333,9 @@ def _move_duplicate_data_to_new_table( """ bind = session.get_bind() dialect_name = bind.dialect.name + query = ( - session.query(source_table) - .with_entities(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) + select(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) .select_from(source_table) .join(subquery, and_(*[getattr(source_table.c, x) == getattr(subquery.c, x) for x in uniqueness])) ) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 25fc1a28f98f..46256ee3359e 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -38,6 +38,7 @@ from pygments.formatters import HtmlFormatter from sqlalchemy import delete, func, types from sqlalchemy.ext.associationproxy import AssociationProxy +from sqlalchemy.sql import Select from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import errors @@ -52,7 +53,6 @@ from airflow.www.widgets import AirflowDateTimePickerWidget if TYPE_CHECKING: - from sqlalchemy.orm.query import Query from sqlalchemy.orm.session import Session from sqlalchemy.sql.operators import ColumnOperators @@ -517,18 +517,21 @@ def _get_run_ordering_expr(name: str) -> ColumnOperators: return expr.desc() -def sorted_dag_runs(query: Query, *, ordering: Sequence[str], limit: int) -> Sequence[DagRun]: +def sorted_dag_runs( + query: Select, *, ordering: Sequence[str], limit: int, session: Session +) -> Sequence[DagRun]: """Produce DAG runs sorted by specified columns. - :param query: An ORM query object against *DagRun*. + :param query: An ORM select object against *DagRun*. :param ordering: Column names to sort the runs. should generally come from a timetable's ``run_ordering``. :param limit: Number of runs to limit to. + :param session: SQLAlchemy ORM session object :return: A list of DagRun objects ordered by the specified columns. The list contains only the *last* objects, but in *ascending* order. """ ordering_exprs = (_get_run_ordering_expr(name) for name in ordering) - runs = query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit).all() + runs = session.scalars(query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit)).all() runs.reverse() return runs diff --git a/airflow/www/views.py b/airflow/www/views.py index 026f8b533499..d791eefc37c7 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -69,7 +69,7 @@ from pendulum.parsing.exceptions import ParserError from pygments import highlight, lexers from pygments.formatters import HtmlFormatter -from sqlalchemy import Date, and_, case, desc, func, inspect, or_, union_all +from sqlalchemy import Date, and_, case, desc, func, inspect, or_, select, union_all from sqlalchemy.exc import IntegrityError from sqlalchemy.orm import Session, joinedload from wtforms import SelectField, validators @@ -95,7 +95,7 @@ from airflow.jobs.job import Job from airflow.jobs.scheduler_job_runner import SchedulerJobRunner from airflow.jobs.triggerer_job_runner import TriggererJobRunner -from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, XCom, errors +from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, Trigger, XCom, errors from airflow.models.abstractoperator import AbstractOperator from airflow.models.dag import DAG, get_dataset_triggered_next_run_info from airflow.models.dagcode import DagCode @@ -231,16 +231,15 @@ def get_date_time_num_runs_dag_runs_form_data(www_request, session, dag): # loaded and the actual requested run would be excluded by the limit(). Once # the user has changed base date to be anything else we want to use that instead. query_date = base_date - if date_time < base_date and date_time + datetime.timedelta(seconds=1) >= base_date: + if date_time < base_date <= date_time + datetime.timedelta(seconds=1): query_date = date_time - drs = ( - session.query(DagRun) - .filter(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) + drs = session.scalars( + select(DagRun) + .where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) .order_by(desc(DagRun.execution_date)) .limit(num_runs) - .all() - ) + ).all() dr_choices = [] dr_state = None for dr in drs: @@ -291,8 +290,8 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): Create a nested dict representation of the DAG's TaskGroup and its children used to construct the Graph and Grid views. """ - query = ( - session.query( + query = session.execute( + select( TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, @@ -303,7 +302,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): func.max(TaskInstance.end_date).label("end_date"), ) .join(TaskInstance.task_instance_note, isouter=True) - .filter( + .where( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), ) @@ -426,11 +425,9 @@ def get_summary(dag_run: DagRun): } def get_mapped_group_summaries(): - mapped_ti_query = ( - session.query( - TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index - ) - .filter( + mapped_ti_query = session.execute( + select(TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index) + .where( TaskInstance.dag_id == dag.dag_id, TaskInstance.task_id.in_(child["id"] for child in children), TaskInstance.run_id.in_(r.run_id for r in dag_runs), @@ -738,21 +735,20 @@ def index(self): with create_session() as session: # read orm_dags from the db - dags_query = session.query(DagModel).filter(~DagModel.is_subdag, DagModel.is_active) + dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) if arg_search_query: escaped_arg_search_query = arg_search_query.replace("_", r"\_") - dags_query = dags_query.filter( + dags_query = dags_query.where( DagModel.dag_id.ilike("%" + escaped_arg_search_query + "%", escape="\\") | DagModel.owners.ilike("%" + escaped_arg_search_query + "%", escape="\\") ) if arg_tags_filter: - dags_query = dags_query.filter(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) + dags_query = dags_query.where(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) - dags_query = dags_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - - filtered_dag_count = dags_query.count() + dags_query = dags_query.where(DagModel.dag_id.in_(filter_dag_ids)) + filtered_dag_count = session.scalar(select(func.count()).select_from(dags_query)) if filtered_dag_count == 0 and len(arg_tags_filter): flash( "No matching DAG tags found.", @@ -762,28 +758,28 @@ def index(self): return redirect(url_for("Airflow.index")) all_dags = dags_query - active_dags = dags_query.filter(~DagModel.is_paused) - paused_dags = dags_query.filter(DagModel.is_paused) + active_dags = dags_query.where(~DagModel.is_paused) + paused_dags = dags_query.where(DagModel.is_paused) # find DAGs which have a RUNNING DagRun - running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).filter( + running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).where( DagRun.state == State.RUNNING ) # find DAGs for which the latest DagRun is FAILED subq_all = ( - session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) .group_by(DagRun.dag_id) .subquery() ) subq_failed = ( - session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) - .filter(DagRun.state == State.FAILED) + select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + .where(DagRun.state == State.FAILED) .group_by(DagRun.dag_id) .subquery() ) subq_join = ( - session.query(subq_all.c.dag_id, subq_all.c.start_date) + select(subq_all.c.dag_id, subq_all.c.start_date) .join( subq_failed, and_( @@ -796,16 +792,18 @@ def index(self): failed_dags = dags_query.join(subq_join, DagModel.dag_id == subq_join.c.dag_id) is_paused_count = dict( - all_dags.with_entities(DagModel.is_paused, func.count(DagModel.dag_id)).group_by( - DagModel.is_paused - ) + session.execute( + select(DagModel.is_paused, func.count(DagModel.dag_id)) + .group_by(DagModel.is_paused) + .select_from(all_dags) + ).all() ) status_count_active = is_paused_count.get(False, 0) status_count_paused = is_paused_count.get(True, 0) - status_count_running = running_dags.count() - status_count_failed = failed_dags.count() + status_count_running = session.scalar(select(func.count()).select_from(running_dags)) + status_count_failed = session.scalar(select(func.count()).select_from(failed_dags)) all_dags_count = status_count_active + status_count_paused if arg_status_filter == "active": @@ -826,7 +824,7 @@ def index(self): if arg_sorting_key == "last_dagrun": dag_run_subquery = ( - session.query( + select( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) @@ -854,7 +852,13 @@ def index(self): else: current_dags = current_dags.order_by(null_case, sort_column) - dags = current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page).all() + dags = ( + session.scalars( + current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page) + ) + .unique() + .all() + ) user_permissions = g.user.perms can_create_dag_run = ( permissions.ACTION_CAN_CREATE, @@ -874,7 +878,7 @@ def index(self): dag.can_trigger = dag.can_edit and can_create_dag_run dag.can_delete = get_airflow_app().appbuilder.sm.can_delete_dag(dag.dag_id, g.user) - dagtags = session.query(func.distinct(DagTag.name)).order_by(DagTag.name).all() + dagtags = session.execute(select(func.distinct(DagTag.name)).order_by(DagTag.name)).all() tags = [ {"name": name, "selected": bool(arg_tags_filter and name in arg_tags_filter)} for name, in dagtags @@ -882,14 +886,15 @@ def index(self): owner_links_dict = DagOwnerAttributes.get_all(session) - import_errors = session.query(errors.ImportError).order_by(errors.ImportError.id) + import_errors = select(errors.ImportError).order_by(errors.ImportError.id) if (permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG) not in user_permissions: # if the user doesn't have access to all DAGs, only display errors from visible DAGs import_errors = import_errors.join( DagModel, DagModel.fileloc == errors.ImportError.filename - ).filter(DagModel.dag_id.in_(filter_dag_ids)) + ).where(DagModel.dag_id.in_(filter_dag_ids)) + import_errors = session.scalars(import_errors) for import_error in import_errors: flash( f"Broken DAG: [{import_error.filename}] {import_error.stacktrace}", @@ -933,10 +938,12 @@ def _iter_parsed_moved_data_table_names(): permissions.RESOURCE_ADMIN_MENU, ) in user_permissions and conf.getboolean("webserver", "warn_deployment_exposure"): robots_file_access_count = ( - session.query(Log) - .filter(Log.event == "robots") - .filter(Log.dttm > (utcnow() - datetime.timedelta(days=7))) - .count() + select(Log) + .where(Log.event == "robots") + .where(Log.dttm > (utcnow() - datetime.timedelta(days=7))) + ) + robots_file_access_count = session.scalar( + select(func.count()).select_from(robots_file_access_count) ) if robots_file_access_count > 0: flash( @@ -1038,9 +1045,11 @@ def next_run_datasets_summary(self, session: Session = NEW_SESSION): dataset_triggered_dag_ids = [ dag.dag_id for dag in ( - session.query(DagModel.dag_id) - .filter(DagModel.dag_id.in_(filter_dag_ids)) - .filter(DagModel.schedule_interval == "Dataset") + session.scalars( + select(DagModel.dag_id) + .where(DagModel.dag_id.in_(filter_dag_ids)) + .where(DagModel.schedule_interval == "Dataset") + ) ) ] @@ -1071,10 +1080,10 @@ def dag_stats(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify({}) - dag_state_stats = ( - session.query(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) + dag_state_stats = session.execute( + select(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) .group_by(DagRun.dag_id, DagRun.state) - .filter(DagRun.dag_id.in_(filter_dag_ids)) + .where(DagRun.dag_id.in_(filter_dag_ids)) ) dag_state_data = {(dag_id, state): count for dag_id, state, count in dag_state_stats} @@ -1112,17 +1121,17 @@ def task_stats(self, session: Session = NEW_SESSION): filter_dag_ids = allowed_dag_ids running_dag_run_query_result = ( - session.query(DagRun.dag_id, DagRun.run_id) + select(DagRun.dag_id, DagRun.run_id) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .filter(DagRun.state == State.RUNNING, DagModel.is_active) + .where(DagRun.state == State.RUNNING, DagModel.is_active) ) - running_dag_run_query_result = running_dag_run_query_result.filter(DagRun.dag_id.in_(filter_dag_ids)) + running_dag_run_query_result = running_dag_run_query_result.where(DagRun.dag_id.in_(filter_dag_ids)) running_dag_run_query_result = running_dag_run_query_result.subquery("running_dag_run") # Select all task_instances from active dag_runs. - running_task_instance_query_result = session.query( + running_task_instance_query_result = select( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(True).label("is_dag_running"), @@ -1136,19 +1145,19 @@ def task_stats(self, session: Session = NEW_SESSION): if conf.getboolean("webserver", "SHOW_RECENT_STATS_FOR_COMPLETED_RUNS", fallback=True): last_dag_run = ( - session.query(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) + select(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .filter(DagRun.state != State.RUNNING, DagModel.is_active) + .where(DagRun.state != State.RUNNING, DagModel.is_active) .group_by(DagRun.dag_id) ) - last_dag_run = last_dag_run.filter(DagRun.dag_id.in_(filter_dag_ids)) + last_dag_run = last_dag_run.where(DagRun.dag_id.in_(filter_dag_ids)) last_dag_run = last_dag_run.subquery("last_dag_run") # Select all task_instances from active dag_runs. # If no dag_run is active, return task instances from most recent dag_run. last_task_instance_query_result = ( - session.query( + select( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(False).label("is_dag_running"), @@ -1169,8 +1178,8 @@ def task_stats(self, session: Session = NEW_SESSION): else: final_task_instance_query_result = running_task_instance_query_result.subquery("final_ti") - qry = ( - session.query( + qry = session.execute( + select( final_task_instance_query_result.c.dag_id, final_task_instance_query_result.c.state, final_task_instance_query_result.c.is_dag_running, @@ -1186,7 +1195,6 @@ def task_stats(self, session: Session = NEW_SESSION): final_task_instance_query_result.c.is_dag_running.desc(), ) ) - data = get_task_stats_from_query(qry) payload: dict[str, list[dict[str, Any]]] = collections.defaultdict(list) for dag_id in filter_dag_ids: @@ -1219,29 +1227,31 @@ def last_dagruns(self, session: Session = NEW_SESSION): return flask.json.jsonify({}) last_runs_subquery = ( - session.query( + select( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) .group_by(DagRun.dag_id) - .filter(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. + .where(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. .subquery("last_runs") ) - query = session.query( - DagRun.dag_id, - DagRun.start_date, - DagRun.end_date, - DagRun.state, - DagRun.execution_date, - DagRun.data_interval_start, - DagRun.data_interval_end, - ).join( - last_runs_subquery, - and_( - last_runs_subquery.c.dag_id == DagRun.dag_id, - last_runs_subquery.c.max_execution_date == DagRun.execution_date, - ), + query = session.execute( + select( + DagRun.dag_id, + DagRun.start_date, + DagRun.end_date, + DagRun.state, + DagRun.execution_date, + DagRun.data_interval_start, + DagRun.data_interval_end, + ).join( + last_runs_subquery, + and_( + last_runs_subquery.c.dag_id == DagRun.dag_id, + last_runs_subquery.c.max_execution_date == DagRun.execution_date, + ), + ) ) resp = { @@ -1340,19 +1350,18 @@ def dag_details(self, dag_id, session: Session = NEW_SESSION): title = "DAG Details" root = request.args.get("root", "") - states = ( - session.query(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) - .filter(TaskInstance.dag_id == dag_id) + states = session.execute( + select(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) + .where(TaskInstance.dag_id == dag_id) .group_by(TaskInstance.state) - .all() - ) + ).all() active_runs = models.DagRun.find(dag_id=dag_id, state=DagRunState.RUNNING, external_trigger=False) - tags = session.query(models.DagTag).filter(models.DagTag.dag_id == dag_id).all() + tags = session.scalars(select(models.DagTag).where(models.DagTag.dag_id == dag_id)).all() # TODO: convert this to a relationship - owner_links = session.query(DagOwnerAttributes).filter_by(dag_id=dag_id).all() + owner_links = session.execute(select(DagOwnerAttributes).filter_by(dag_id=dag_id)).all() attrs_to_avoid = [ "schedule_datasets", @@ -1617,18 +1626,17 @@ def get_logs_with_metadata(self, session: Session = NEW_SESSION): "metadata": {"end_of_log": True}, } - ti = ( - session.query(models.TaskInstance) - .filter( + ti = session.scalar( + select(models.TaskInstance) + .where( TaskInstance.task_id == task_id, TaskInstance.dag_id == dag_id, TaskInstance.execution_date == execution_date, TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) - .options(joinedload("trigger")) - .options(joinedload("trigger.triggerer_job")) - .first() + .options(joinedload(TaskInstance.trigger).joinedload(Trigger.triggerer_job)) + .limit(1) ) if ti is None: @@ -1686,10 +1694,10 @@ def log(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) dag_model = DagModel.get_dagmodel(dag_id) - ti = ( - session.query(models.TaskInstance) + ti = session.scalar( + select(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .first() + .limit(1) ) num_logs = 0 @@ -1730,10 +1738,10 @@ def redirect_to_external_log(self, session: Session = NEW_SESSION): map_index = request.args.get("map_index", -1, type=int) try_number = request.args.get("try_number", 1) - ti = ( - session.query(models.TaskInstance) + ti = session.scalar( + select(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .first() + .limit(1) ) if not ti: @@ -1775,8 +1783,8 @@ def task(self, session: Session = NEW_SESSION): task = copy.copy(dag.get_task(task_id)) task.resolve_template_files() - ti: TaskInstance | None = ( - session.query(TaskInstance) + ti: TaskInstance | None = session.scalar( + select(TaskInstance) .options( # HACK: Eager-load relationships. This is needed because # multiple properties mis-use provide_session() that destroys @@ -1785,7 +1793,6 @@ def task(self, session: Session = NEW_SESSION): joinedload(TaskInstance.trigger, innerjoin=False), ) .filter_by(execution_date=dttm, dag_id=dag_id, task_id=task_id, map_index=map_index) - .one_or_none() ) if ti is None: ti_attrs: list[tuple[str, Any]] | None = None @@ -1902,17 +1909,19 @@ def xcom(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) root = request.args.get("root", "") dag = DagModel.get_dagmodel(dag_id) - ti = session.query(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).first() + ti = session.scalar(select(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).limit(1)) if not ti: flash(f"Task [{dag_id}.{task_id}] doesn't seem to exist at the moment", "error") return redirect(url_for("Airflow.index")) - xcom_query = session.query(XCom.key, XCom.value).filter( - XCom.dag_id == dag_id, - XCom.task_id == task_id, - XCom.execution_date == dttm, - XCom.map_index == map_index, + xcom_query = session.execute( + select(XCom.key, XCom.value).where( + XCom.dag_id == dag_id, + XCom.task_id == task_id, + XCom.execution_date == dttm, + XCom.map_index == map_index, + ) ) attributes = [(k, v) for k, v in xcom_query if not k.startswith("_")] @@ -1982,7 +1991,7 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): request_execution_date = request.values.get("execution_date", default=timezone.utcnow().isoformat()) is_dag_run_conf_overrides_params = conf.getboolean("core", "dag_run_conf_overrides_params") dag = get_airflow_app().dag_bag.get_dag(dag_id) - dag_orm: DagModel = session.query(DagModel).filter(DagModel.dag_id == dag_id).first() + dag_orm: DagModel = session.scalar(select(DagModel).where(DagModel.dag_id == dag_id).limit(1)) # Prepare form fields with param struct details to render a proper form with schema information form_fields = {} @@ -2020,11 +2029,9 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): flash(f"Cannot create dagruns because the dag {dag_id} has import errors", "error") return redirect(origin) - recent_runs = ( - session.query( - DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date) - ) - .filter( + recent_runs = session.execute( + select(DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date)) + .where( DagRun.dag_id == dag_id, DagRun.run_type == DagRunType.MANUAL, DagRun.conf.isnot(None), @@ -2297,15 +2304,17 @@ def clear(self, *, session: Session = NEW_SESSION): # Lock the related dag runs to prevent from possible dead lock. # https://github.com/apache/airflow/pull/26658 - dag_runs_query = session.query(DagRun.id).filter(DagRun.dag_id == dag_id).with_for_update() + dag_runs_query = session.scalars( + select(DagRun.id).where(DagRun.dag_id == dag_id).with_for_update() + ) if start_date is None and end_date is None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date == start_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date == start_date) else: if start_date is not None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date >= start_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date >= start_date) if end_date is not None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date <= end_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date <= end_date) locked_dag_run_ids = dag_runs_query.all() elif task_id: @@ -2394,10 +2403,10 @@ def blocked(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify([]) - dags = ( - session.query(DagRun.dag_id, sqla.func.count(DagRun.id)) - .filter(DagRun.state == DagRunState.RUNNING) - .filter(DagRun.dag_id.in_(filter_dag_ids)) + dags = session.execute( + select(DagRun.dag_id, sqla.func.count(DagRun.id)) + .where(DagRun.state == DagRunState.RUNNING) + .where(DagRun.dag_id.in_(filter_dag_ids)) .group_by(DagRun.dag_id) ) @@ -2478,9 +2487,11 @@ def _mark_dagrun_state_as_queued( # Identify tasks that will be queued up to run when confirmed all_task_ids = [task.task_id for task in dag.tasks] - existing_tis = session.query(TaskInstance.task_id).filter( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.run_id == dag_run_id, + existing_tis = session.execute( + select(TaskInstance.task_id).where( + TaskInstance.dag_id == dag.dag_id, + TaskInstance.run_id == dag_run_id, + ) ) completed_tis_ids = [task_id for task_id, in existing_tis] @@ -2962,19 +2973,18 @@ def _convert_to_date(session, column): if root: dag = dag.partial_subset(task_ids_or_regex=root, include_downstream=False, include_upstream=True) - dag_states = ( - session.query( - (_convert_to_date(session, DagRun.execution_date)).label("date"), + dag_states = session.execute( + select( + _convert_to_date(session, DagRun.execution_date).label("date"), DagRun.state, func.max(DagRun.data_interval_start).label("data_interval_start"), func.max(DagRun.data_interval_end).label("data_interval_end"), func.count("*").label("count"), ) - .filter(DagRun.dag_id == dag.dag_id) + .where(DagRun.dag_id == dag.dag_id) .group_by(_convert_to_date(session, DagRun.execution_date), DagRun.state) .order_by(_convert_to_date(session, DagRun.execution_date).asc()) - .all() - ) + ).all() data_dag_states = [ { @@ -3245,16 +3255,17 @@ def duration(self, dag_id: str, session: Session = NEW_SESSION): else: min_date = timezone.utc_epoch() ti_fails = ( - session.query(TaskFail) + select(TaskFail) .join(TaskFail.dag_run) - .filter( + .where( TaskFail.dag_id == dag.dag_id, DagRun.execution_date >= min_date, DagRun.execution_date <= base_date, ) ) if dag.partial: - ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = session.scalars(ti_fails) fails_totals: dict[tuple[str, str, str], int] = defaultdict(int) for failed_task_instance in ti_fails: dict_key = ( @@ -3595,9 +3606,9 @@ def gantt(self, dag_id: str, session: Session = NEW_SESSION): form = DateTimeWithNumRunsWithDagRunsForm(data=dt_nr_dr_data) form.execution_date.choices = dt_nr_dr_data["dr_choices"] - tis = ( - session.query(TaskInstance) - .filter( + tis = session.scalars( + select(TaskInstance) + .where( TaskInstance.dag_id == dag_id, TaskInstance.run_id == dag_run_id, TaskInstance.start_date.is_not(None), @@ -3606,10 +3617,10 @@ def gantt(self, dag_id: str, session: Session = NEW_SESSION): .order_by(TaskInstance.start_date) ) - ti_fails = session.query(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) + ti_fails = select(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) if dag.partial: - ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) - + ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = session.scalars(ti_fails) tasks = [] for ti in tis: if not dag.has_task(ti.task_id): @@ -3715,12 +3726,13 @@ def extra_links(self, *, session: Session = NEW_SESSION): if link_name is None: return {"url": None, "error": "Link name not passed"}, 400 - ti = ( - session.query(TaskInstance) + ti = session.scalar( + select(TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) .options(joinedload(TaskInstance.dag_run)) - .first() + .limit(1) ) + if not ti: return {"url": None, "error": "Task Instances not found"}, 404 try: @@ -3828,27 +3840,25 @@ def grid_data(self): base_date = dag.get_latest_execution_date() or timezone.utcnow() with create_session() as session: - query = session.query(DagRun).filter( - DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date - ) + query = select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date) - run_type = request.args.get("run_type") - if run_type: - query = query.filter(DagRun.run_type == run_type) + run_type = request.args.get("run_type") + if run_type: + query = query.where(DagRun.run_type == run_type) - run_state = request.args.get("run_state") - if run_state: - query = query.filter(DagRun.state == run_state) + run_state = request.args.get("run_state") + if run_state: + query = query.where(DagRun.state == run_state) - dag_runs = wwwutils.sorted_dag_runs(query, ordering=dag.timetable.run_ordering, limit=num_runs) - encoded_runs = [ - wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs - ] - data = { - "groups": dag_to_grid(dag, dag_runs, session), - "dag_runs": encoded_runs, - "ordering": dag.timetable.run_ordering, - } + dag_runs = wwwutils.sorted_dag_runs( + query, ordering=dag.timetable.run_ordering, limit=num_runs, session=session + ) + encoded_runs = [wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs] + data = { + "groups": dag_to_grid(dag, dag_runs, session), + "dag_runs": encoded_runs, + "ordering": dag.timetable.run_ordering, + } # avoid spaces to reduce payload size return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -3867,37 +3877,34 @@ def historical_metrics_data(self): end_date = _safe_parse_datetime(request.args.get("end_date")) with create_session() as session: # DagRuns - dag_runs_type = ( - session.query(DagRun.run_type, func.count(DagRun.run_id)) - .filter( + dag_runs_type = session.execute( + select(DagRun.run_type, func.count(DagRun.run_id)) + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.run_type) - .all() - ) + ).all() - dag_run_states = ( - session.query(DagRun.state, func.count(DagRun.run_id)) - .filter( + dag_run_states = session.execute( + select(DagRun.state, func.count(DagRun.run_id)) + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.state) - .all() - ) + ).all() # TaskInstances - task_instance_states = ( - session.query(TaskInstance.state, func.count(TaskInstance.run_id)) + task_instance_states = session.execute( + select(TaskInstance.state, func.count(TaskInstance.run_id)) .join(TaskInstance.dag_run) - .filter( + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(TaskInstance.state) - .all() - ) + ).all() data = { "dag_run_types": { @@ -3931,28 +3938,32 @@ def next_run_datasets(self, dag_id): with create_session() as session: data = [ dict(info) - for info in session.query( - DatasetModel.id, - DatasetModel.uri, - func.max(DatasetEvent.timestamp).label("lastUpdate"), - ) - .join(DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id) - .join( - DatasetDagRunQueue, - and_( - DatasetDagRunQueue.dataset_id == DatasetModel.id, - DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, - ), - isouter=True, - ) - .join( - DatasetEvent, - DatasetEvent.dataset_id == DatasetModel.id, - isouter=True, + for info in session.execute( + select( + DatasetModel.id, + DatasetModel.uri, + func.max(DatasetEvent.timestamp).label("lastUpdate"), + ) + .join( + DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id + ) + .join( + DatasetDagRunQueue, + and_( + DatasetDagRunQueue.dataset_id == DatasetModel.id, + DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, + ), + isouter=True, + ) + .join( + DatasetEvent, + DatasetEvent.dataset_id == DatasetModel.id, + isouter=True, + ) + .where(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) + .group_by(DatasetModel.id, DatasetModel.uri) + .order_by(DatasetModel.uri) ) - .filter(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) - .group_by(DatasetModel.id, DatasetModel.uri) - .order_by(DatasetModel.uri) ] return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -4050,12 +4061,12 @@ def datasets_summary(self): if session.bind.dialect.name == "postgresql": order_by = (order_by[0].nulls_first(), *order_by[1:]) - count_query = session.query(func.count(DatasetModel.id)) + count_query = select(func.count(DatasetModel.id)) has_event_filters = bool(updated_before or updated_after) query = ( - session.query( + select( DatasetModel.id, DatasetModel.uri, func.max(DatasetEvent.timestamp).label("last_dataset_update"), @@ -4080,11 +4091,12 @@ def datasets_summary(self): if updated_before: filters.append(DatasetEvent.timestamp <= updated_before) - query = query.filter(*filters).offset(offset).limit(limit) - count_query = count_query.filter(*filters) + query = query.where(*filters).offset(offset).limit(limit) + count_query = count_query.where(*filters) + query = session.execute(query) datasets = [dict(dataset) for dataset in query] - data = {"datasets": datasets, "total_entries": count_query.scalar()} + data = {"datasets": datasets, "total_entries": session.scalar(count_query)} return ( htmlsafe_json_dumps(data, separators=(",", ":"), cls=utils_json.WebEncoder), @@ -4130,20 +4142,20 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): included_events_raw = conf.get("webserver", "audit_view_included_events", fallback=None) excluded_events_raw = conf.get("webserver", "audit_view_excluded_events", fallback=None) - query = session.query(Log).filter(Log.dag_id == dag_id) + query = select(Log).where(Log.dag_id == dag_id) if included_events_raw: included_events = {event.strip() for event in included_events_raw.split(",")} - query = query.filter(Log.event.in_(included_events)) + query = query.where(Log.event.in_(included_events)) elif excluded_events_raw: excluded_events = {event.strip() for event in excluded_events_raw.split(",")} - query = query.filter(Log.event.notin_(excluded_events)) + query = query.where(Log.event.notin_(excluded_events)) current_page = request.args.get("page", default=0, type=int) arg_sorting_key = request.args.get("sorting_key", "dttm") arg_sorting_direction = request.args.get("sorting_direction", default="desc") logs_per_page = PAGE_SIZE - audit_logs_count = query.count() + audit_logs_count = session.scalar(select(func.count()).select_from(query)) num_of_pages = int(math.ceil(audit_logs_count / float(logs_per_page))) start = current_page * logs_per_page @@ -4155,7 +4167,7 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): sort_column = sort_column.desc() query = query.order_by(sort_column) - dag_audit_logs = query.offset(start).limit(logs_per_page).all() + dag_audit_logs = session.scalars(query.offset(start).limit(logs_per_page)).all() return self.render_template( "airflow/dag_audit_log.html", dag=dag, @@ -4276,7 +4288,7 @@ def apply(self, query, func): if get_airflow_app().appbuilder.sm.has_all_dags_access(g.user): return query filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - return query.filter(self.model.dag_id.in_(filter_dag_ids)) + return query.where(self.model.dag_id.in_(filter_dag_ids)) class AirflowModelView(ModelView): @@ -4721,9 +4733,11 @@ def action_mulduplicate(self, connections, session: Session = NEW_SESSION): potential_connection_ids = [f"{base_conn_id}_copy{i}" for i in range(1, 11)] - query = session.query(Connection.conn_id).filter(Connection.conn_id.in_(potential_connection_ids)) + query = session.scalars( + select(Connection.conn_id).where(Connection.conn_id.in_(potential_connection_ids)) + ) - found_conn_id_set = {conn_id for conn_id, in query} + found_conn_id_set = {conn_id for conn_id in query} possible_conn_id_iter = ( connection_id @@ -5393,7 +5407,7 @@ def _set_dag_runs_to_active_state(self, drs: list[DagRun], state: str, session: """This routine only supports Running and Queued state.""" try: count = 0 - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 if state == State.RUNNING: dr.start_date = timezone.utcnow() @@ -5419,7 +5433,7 @@ def action_set_failed(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 altered_tis += set_dag_run_state_to_failed( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5447,7 +5461,7 @@ def action_set_success(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 altered_tis += set_dag_run_state_to_success( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5471,7 +5485,7 @@ def action_clear(self, drs: list[DagRun], session: Session = NEW_SESSION): count = 0 cleared_ti_count = 0 dag_to_tis: dict[DAG, list[TaskInstance]] = {} - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 dag = get_airflow_app().dag_bag.get_dag(dr.dag_id) tis_to_clear = dag_to_tis.setdefault(dag, []) @@ -5887,35 +5901,37 @@ def autocomplete(self, session: Session = NEW_SESSION): return flask.json.jsonify([]) # Provide suggestions of dag_ids and owners - dag_ids_query = session.query( + dag_ids_query = select( sqla.literal("dag").label("type"), DagModel.dag_id.label("name"), - ).filter(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) + ).where(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) owners_query = ( - session.query( + select( sqla.literal("owner").label("type"), DagModel.owners.label("name"), ) .distinct() - .filter(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) + .where(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) ) # Hide DAGs if not showing status: "all" status = flask_session.get(FILTER_STATUS_COOKIE) if status == "active": - dag_ids_query = dag_ids_query.filter(~DagModel.is_paused) - owners_query = owners_query.filter(~DagModel.is_paused) + dag_ids_query = dag_ids_query.where(~DagModel.is_paused) + owners_query = owners_query.where(~DagModel.is_paused) elif status == "paused": - dag_ids_query = dag_ids_query.filter(DagModel.is_paused) - owners_query = owners_query.filter(DagModel.is_paused) + dag_ids_query = dag_ids_query.where(DagModel.is_paused) + owners_query = owners_query.where(DagModel.is_paused) filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - dag_ids_query = dag_ids_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - owners_query = owners_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - - payload = [row._asdict() for row in dag_ids_query.union(owners_query).order_by("name").limit(10)] + dag_ids_query = dag_ids_query.where(DagModel.dag_id.in_(filter_dag_ids)) + owners_query = owners_query.where(DagModel.dag_id.in_(filter_dag_ids)) + payload = [ + row._asdict() + for row in session.execute(dag_ids_query.union(owners_query).order_by("name").limit(10)) + ] return flask.json.jsonify(payload) From 985a6cd2a84daa4511649dfdc9e94b704de5c6d1 Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Mon, 3 Jul 2023 18:54:59 +0530 Subject: [PATCH 026/533] invalid args fix (#32326) --- .../microsoft/azure/example_dag/example_wasb_sensors.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py b/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py index 349e09c44ad9..8d449e52a1d3 100644 --- a/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py +++ b/airflow/providers/microsoft/azure/example_dag/example_wasb_sensors.py @@ -54,7 +54,6 @@ # [START wasb_prefix_sensor] azure_wasb_prefix_sensor = WasbPrefixSensor( container_name=CONTAINER_NAME, - blob_name=BLOB_NAME, prefix=PREFIX, task_id="wasb_sense_prefix", ) From 42eb93b87b8877af5fbb3244cff786c636f1ec4f Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Mon, 3 Jul 2023 19:21:57 +0530 Subject: [PATCH 027/533] Fixing typo in LOCAL_VIRTUALENV.rst (#32327) --- LOCAL_VIRTUALENV.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/LOCAL_VIRTUALENV.rst b/LOCAL_VIRTUALENV.rst index 03b6d25e3927..987a0df4bfa5 100644 --- a/LOCAL_VIRTUALENV.rst +++ b/LOCAL_VIRTUALENV.rst @@ -73,7 +73,7 @@ Extra Packages There are known issues with ``bazel`` that might lead to circular dependencies when using it to install Airflow. Please switch to ``pip`` if you encounter such problems. ``Bazel`` community works on fixing the problem in `this PR `_ so it might be that - n ewer versions of ``bazel`` will handle it. + newer versions of ``bazel`` will handle it. If you wish to install airflow using those tools you should use the constraint files and convert them to appropriate format and workflow that your tool requires. From 2b5d4e319b82267349aa20cb3230f3b2ad35adce Mon Sep 17 00:00:00 2001 From: Clemens Date: Tue, 4 Jul 2023 05:47:27 +0800 Subject: [PATCH 028/533] Render list items in rendered fields view (#32042) --------- Co-authored-by: clemens.valiente --- airflow/www/utils.py | 30 +++++++--- airflow/www/views.py | 6 +- tests/www/views/test_views_rendered.py | 78 +++++++++++++++++++++++++- 3 files changed, 100 insertions(+), 14 deletions(-) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 46256ee3359e..b31f9326d988 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -20,7 +20,7 @@ import json import textwrap import time -from typing import TYPE_CHECKING, Any, Sequence +from typing import TYPE_CHECKING, Any, Callable, Sequence from urllib.parse import urlencode from flask import request, url_for @@ -36,6 +36,7 @@ from pendulum.datetime import DateTime from pygments import highlight, lexers from pygments.formatters import HtmlFormatter +from pygments.lexer import Lexer from sqlalchemy import delete, func, types from sqlalchemy.ext.associationproxy import AssociationProxy from sqlalchemy.sql import Select @@ -549,20 +550,35 @@ def pygment_html_render(s, lexer=lexers.TextLexer): return highlight(s, lexer(), HtmlFormatter(linenos=True)) -def render(obj, lexer): +def render(obj: Any, lexer: Lexer, handler: Callable[[Any], str] | None = None): """Render a given Python object with a given Pygments lexer.""" - out = "" if isinstance(obj, str): - out = Markup(pygment_html_render(obj, lexer)) + return Markup(pygment_html_render(obj, lexer)) + elif isinstance(obj, (tuple, list)): + out = "" for i, text_to_render in enumerate(obj): + if lexer is lexers.PythonLexer: + text_to_render = repr(text_to_render) out += Markup("
List item #{}
").format(i) out += Markup("
" + pygment_html_render(text_to_render, lexer) + "
") + return out + elif isinstance(obj, dict): + out = "" for k, v in obj.items(): + if lexer is lexers.PythonLexer: + v = repr(v) out += Markup('
Dict item "{}"
').format(k) out += Markup("
" + pygment_html_render(v, lexer) + "
") - return out + return out + + elif handler is not None and obj is not None: + return Markup(pygment_html_render(handler(obj), lexer)) + + else: + # Return empty string otherwise + return "" def json_render(obj, lexer): @@ -603,8 +619,8 @@ def get_attr_renderer(): "mysql": lambda x: render(x, lexers.MySqlLexer), "postgresql": lambda x: render(x, lexers.PostgresLexer), "powershell": lambda x: render(x, lexers.PowerShellLexer), - "py": lambda x: render(get_python_source(x), lexers.PythonLexer), - "python_callable": lambda x: render(get_python_source(x), lexers.PythonLexer), + "py": lambda x: render(x, lexers.PythonLexer, get_python_source), + "python_callable": lambda x: render(x, lexers.PythonLexer, get_python_source), "rst": lambda x: render(x, lexers.RstLexer), "sql": lambda x: render(x, lexers.SqlLexer), "tsql": lambda x: render(x, lexers.TransactSqlLexer), diff --git a/airflow/www/views.py b/airflow/www/views.py index d791eefc37c7..90d98a424c4a 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1484,11 +1484,7 @@ def rendered_templates(self, session): content = getattr(task, template_field) renderer = task.template_fields_renderers.get(template_field, template_field) if renderer in renderers: - if isinstance(content, (dict, list)): - json_content = json.dumps(content, sort_keys=True, indent=4) - html_dict[template_field] = renderers[renderer](json_content) - else: - html_dict[template_field] = renderers[renderer](content) + html_dict[template_field] = renderers[renderer](content) else: html_dict[template_field] = Markup("
{}
").format(pformat(content)) diff --git a/tests/www/views/test_views_rendered.py b/tests/www/views/test_views_rendered.py index 6a2cb9717f7e..1557a083d50b 100644 --- a/tests/www/views/test_views_rendered.py +++ b/tests/www/views/test_views_rendered.py @@ -24,7 +24,9 @@ from markupsafe import escape from airflow.models import DAG, RenderedTaskInstanceFields, Variable +from airflow.models.baseoperator import BaseOperator from airflow.operators.bash import BashOperator +from airflow.operators.python import PythonOperator from airflow.serialization.serialized_objects import SerializedDAG from airflow.utils import timezone from airflow.utils.session import create_session @@ -64,6 +66,39 @@ def task2(dag): ) +@pytest.fixture() +def task3(dag): + class TestOperator(BaseOperator): + template_fields = ("sql",) + + def __init__(self, *, sql, **kwargs): + super().__init__(**kwargs) + self.sql = sql + + def execute(self, context): + pass + + return TestOperator( + task_id="task3", + sql=["SELECT 1;", "SELECT 2;"], + dag=dag, + ) + + +@pytest.fixture() +def task4(dag): + def func(*op_args): + pass + + return PythonOperator( + task_id="task4", + python_callable=func, + op_args=["{{ task_instance_key_str }}_args"], + op_kwargs={"0": "{{ task_instance_key_str }}_kwargs"}, + dag=dag, + ) + + @pytest.fixture() def task_secret(dag): return BashOperator( @@ -85,7 +120,7 @@ def init_blank_db(): @pytest.fixture(autouse=True) -def reset_db(dag, task1, task2, task_secret): +def reset_db(dag, task1, task2, task3, task4, task_secret): yield clear_db_dags() clear_db_runs() @@ -93,7 +128,7 @@ def reset_db(dag, task1, task2, task_secret): @pytest.fixture() -def create_dag_run(dag, task1, task2, task_secret): +def create_dag_run(dag, task1, task2, task3, task4, task_secret): def _create_dag_run(*, execution_date, session): dag_run = dag.create_dagrun( state=DagRunState.RUNNING, @@ -108,6 +143,10 @@ def _create_dag_run(*, execution_date, session): ti2.state = TaskInstanceState.SCHEDULED ti3 = dag_run.get_task_instance(task_secret.task_id, session=session) ti3.state = TaskInstanceState.QUEUED + ti4 = dag_run.get_task_instance(task3.task_id, session=session) + ti4.state = TaskInstanceState.SUCCESS + ti5 = dag_run.get_task_instance(task4.task_id, session=session) + ti5.state = TaskInstanceState.SUCCESS session.flush() return dag_run @@ -290,3 +329,38 @@ def test_rendered_task_detail_env_secret(patch_app, admin_client, request, env, if request.node.callspec.id.endswith("-tpld-var"): Variable.delete("plain_var") Variable.delete("secret_var") + + +@pytest.mark.usefixtures("patch_app") +def test_rendered_template_view_for_list_template_field_args(admin_client, create_dag_run, task3): + """ + Test that the Rendered View can show a list of syntax-highlighted SQL statements + """ + assert task3.sql == ["SELECT 1;", "SELECT 2;"] + + with create_session() as session: + create_dag_run(execution_date=DEFAULT_DATE, session=session) + + url = f"rendered-templates?task_id=task3&dag_id=testdag&execution_date={quote_plus(str(DEFAULT_DATE))}" + + resp = admin_client.get(url, follow_redirects=True) + check_content_in_response("List item #0", resp) + check_content_in_response("List item #1", resp) + + +@pytest.mark.usefixtures("patch_app") +def test_rendered_template_view_for_op_args(admin_client, create_dag_run, task4): + """ + Test that the Rendered View can show rendered values in op_args and op_kwargs + """ + assert task4.op_args == ["{{ task_instance_key_str }}_args"] + assert list(task4.op_kwargs.values()) == ["{{ task_instance_key_str }}_kwargs"] + + with create_session() as session: + create_dag_run(execution_date=DEFAULT_DATE, session=session) + + url = f"rendered-templates?task_id=task4&dag_id=testdag&execution_date={quote_plus(str(DEFAULT_DATE))}" + + resp = admin_client.get(url, follow_redirects=True) + check_content_in_response("testdag__task4__20200301_args", resp) + check_content_in_response("testdag__task4__20200301_kwargs", resp) From 17132ef4c65b842dab4ff311fd2b0e6d08969e1b Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Tue, 4 Jul 2023 03:44:45 +0530 Subject: [PATCH 029/533] Added `AzureBlobStorageToS3Operator` transfer operator (#32270) --- .../amazon/aws/transfers/azure_blob_to_s3.py | 167 ++++++++++++++ airflow/providers/amazon/provider.yaml | 4 + .../providers/microsoft/azure/hooks/wasb.py | 27 +++ dev/breeze/tests/test_selective_checks.py | 8 +- .../transfer/azure_blob_to_s3.rst | 52 +++++ generated/provider_dependencies.json | 1 + .../aws/transfers/test_azure_blob_to_s3.py | 218 ++++++++++++++++++ .../microsoft/azure/hooks/test_wasb.py | 25 ++ .../amazon/aws/example_azure_blob_to_s3.py | 83 +++++++ 9 files changed, 581 insertions(+), 4 deletions(-) create mode 100644 airflow/providers/amazon/aws/transfers/azure_blob_to_s3.py create mode 100644 docs/apache-airflow-providers-amazon/transfer/azure_blob_to_s3.rst create mode 100644 tests/providers/amazon/aws/transfers/test_azure_blob_to_s3.py create mode 100644 tests/system/providers/amazon/aws/example_azure_blob_to_s3.py diff --git a/airflow/providers/amazon/aws/transfers/azure_blob_to_s3.py b/airflow/providers/amazon/aws/transfers/azure_blob_to_s3.py new file mode 100644 index 000000000000..ebcd49bdc411 --- /dev/null +++ b/airflow/providers/amazon/aws/transfers/azure_blob_to_s3.py @@ -0,0 +1,167 @@ +# +# 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 + +import os +import tempfile +from typing import TYPE_CHECKING, Sequence + +from airflow.models import BaseOperator +from airflow.providers.amazon.aws.hooks.s3 import S3Hook +from airflow.providers.microsoft.azure.hooks.wasb import WasbHook + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class AzureBlobStorageToS3Operator(BaseOperator): + """ + Operator transfers data from Azure Blob Storage to specified bucket in Amazon S3. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AzureBlobStorageToGCSOperator` + + :param wasb_conn_id: Reference to the wasb connection. + :param container_name: Name of the container + :param prefix: Prefix string which filters objects whose name begin with + this prefix. (templated) + :param delimiter: The delimiter by which you want to filter the objects. (templated) + For e.g to lists the CSV files from in a directory in GCS you would use + delimiter='.csv'. + :param aws_conn_id: Connection id of the S3 connection to use + :param dest_s3_key: The base S3 key to be used to store the files. (templated) + :param dest_verify: Whether or not to verify SSL certificates for S3 connection. + By default SSL certificates are verified. + You can provide the following values: + + - ``False``: do not validate SSL certificates. SSL will still be used + (unless use_ssl is False), but SSL certificates will not be + verified. + - ``path/to/cert/bundle.pem``: A filename of the CA cert bundle to uses. + You can specify this argument if you want to use a different + CA cert bundle than the one used by botocore. + :param dest_s3_extra_args: Extra arguments that may be passed to the download/upload operations. + :param replace: Whether or not to verify the existence of the files in the + destination bucket. + By default is set to False + If set to True, will upload all the files replacing the existing ones in + the destination bucket. + If set to False, will upload only the files that are in the origin but not + in the destination bucket. + :param s3_acl_policy: Optional The string to specify the canned ACL policy for the + object to be uploaded in S3 + :param wasb_extra_kargs: kwargs to pass to WasbHook + :param s3_extra_kargs: kwargs to pass to S3Hook + """ + + template_fields: Sequence[str] = ( + "container_name", + "prefix", + "delimiter", + "dest_s3_key", + ) + + def __init__( + self, + *, + wasb_conn_id: str = "wasb_default", + container_name: str, + prefix: str | None = None, + delimiter: str = "", + aws_conn_id: str = "aws_default", + dest_s3_key: str, + dest_verify: str | bool | None = None, + dest_s3_extra_args: dict | None = None, + replace: bool = False, + s3_acl_policy: str | None = None, + wasb_extra_args: dict = {}, + s3_extra_args: dict = {}, + **kwargs, + ) -> None: + super().__init__(**kwargs) + + self.wasb_conn_id = wasb_conn_id + self.container_name = container_name + self.prefix = prefix + self.delimiter = delimiter + self.aws_conn_id = aws_conn_id + self.dest_s3_key = dest_s3_key + self.dest_verify = dest_verify + self.dest_s3_extra_args = dest_s3_extra_args or {} + self.replace = replace + self.s3_acl_policy = s3_acl_policy + self.wasb_extra_args = wasb_extra_args + self.s3_extra_args = s3_extra_args + + def execute(self, context: Context) -> list[str]: + # list all files in the Azure Blob Storage container + wasb_hook = WasbHook(wasb_conn_id=self.wasb_conn_id, **self.wasb_extra_args) + s3_hook = S3Hook( + aws_conn_id=self.aws_conn_id, + verify=self.dest_verify, + extra_args=self.dest_s3_extra_args, + **self.s3_extra_args, + ) + + self.log.info( + f"Getting list of the files in Container: {self.container_name}; " + f"Prefix: {self.prefix}; Delimiter: {self.delimiter};" + ) + + files = wasb_hook.get_blobs_list_recursive( + container_name=self.container_name, prefix=self.prefix, endswith=self.delimiter + ) + + if not self.replace: + # if we are not replacing -> list all files in the S3 bucket + # and only keep those files which are present in + # Azure Blob Storage and not in S3 + bucket_name, prefix = S3Hook.parse_s3_url(self.dest_s3_key) + # look for the bucket and the prefix to avoid look into + # parent directories/keys + existing_files = s3_hook.list_keys(bucket_name, prefix=prefix) + # in case that no files exists, return an empty array to avoid errors + existing_files = existing_files if existing_files is not None else [] + # remove the prefix for the existing files to allow the match + existing_files = [file.replace(f"{prefix}/", "", 1) for file in existing_files] + files = list(set(files) - set(existing_files)) + + if files: + for file in files: + with tempfile.NamedTemporaryFile() as temp_file: + + dest_key = os.path.join(self.dest_s3_key, file) + self.log.info("Downloading data from blob: %s", file) + wasb_hook.get_file( + file_path=temp_file.name, + container_name=self.container_name, + blob_name=file, + ) + + self.log.info("Uploading data to s3: %s", dest_key) + s3_hook.load_file( + filename=temp_file.name, + key=dest_key, + replace=self.replace, + acl_policy=self.s3_acl_policy, + ) + self.log.info("All done, uploaded %d files to S3", len(files)) + else: + self.log.info("All files are already in sync!") + return files diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index a7dac9fad481..575a2b5cab1b 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -625,6 +625,10 @@ transfers: - source-integration-name: Amazon Web Services target-integration-name: Amazon Web Services python-module: airflow.providers.amazon.aws.transfers.base + - source-integration-name: Microsoft Azure Blob Storage + target-integration-name: Amazon Simple Storage Service (S3) + how-to-guide: /docs/apache-airflow-providers-amazon/transfer/azure_blob_to_s3.rst + python-module: airflow.providers.amazon.aws.transfers.azure_blob_to_s3 extra-links: - airflow.providers.amazon.aws.links.batch.BatchJobDefinitionLink diff --git a/airflow/providers/microsoft/azure/hooks/wasb.py b/airflow/providers/microsoft/azure/hooks/wasb.py index c1d615810b96..ad9c4754c525 100644 --- a/airflow/providers/microsoft/azure/hooks/wasb.py +++ b/airflow/providers/microsoft/azure/hooks/wasb.py @@ -292,6 +292,33 @@ def get_blobs_list( blob_list.append(blob.name) return blob_list + def get_blobs_list_recursive( + self, + container_name: str, + prefix: str | None = None, + include: list[str] | None = None, + endswith: str = "", + **kwargs, + ) -> list: + """ + List blobs in a given container. + + :param container_name: The name of the container + :param prefix: Filters the results to return only blobs whose names + begin with the specified prefix. + :param include: Specifies one or more additional datasets to include in the + response. Options include: ``snapshots``, ``metadata``, ``uncommittedblobs``, + ``copy`, ``deleted``. + :param delimiter: filters objects based on the delimiter (for e.g '.csv') + """ + container = self._get_container_client(container_name) + blob_list = [] + blobs = container.list_blobs(name_starts_with=prefix, include=include, **kwargs) + for blob in blobs: + if blob.name.endswith(endswith): + blob_list.append(blob.name) + return blob_list + def load_file( self, file_path: str, diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 61e35ce20f59..7437642e4e9b 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -311,7 +311,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): ("airflow/providers/amazon/__init__.py",), { "affected-providers-list-as-string": "amazon apache.hive cncf.kubernetes " - "common.sql exasol ftp google http imap " + "common.sql exasol ftp google http imap microsoft.azure " "mongo mysql postgres salesforce ssh", "all-python-versions": "['3.8']", "all-python-versions-list-as-string": "3.8", @@ -325,7 +325,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): "upgrade-to-newer-dependencies": "false", "run-amazon-tests": "true", "parallel-test-types-list-as-string": "Providers[amazon] Always " - "Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp,http,imap," + "Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp,http,imap,microsoft.azure," "mongo,mysql,postgres,salesforce,ssh] Providers[google]", }, id="Providers tests run including amazon tests if amazon provider files changed", @@ -353,7 +353,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): ("airflow/providers/amazon/file.py",), { "affected-providers-list-as-string": "amazon apache.hive cncf.kubernetes " - "common.sql exasol ftp google http imap " + "common.sql exasol ftp google http imap microsoft.azure " "mongo mysql postgres salesforce ssh", "all-python-versions": "['3.8']", "all-python-versions-list-as-string": "3.8", @@ -368,7 +368,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str): "upgrade-to-newer-dependencies": "false", "parallel-test-types-list-as-string": "Providers[amazon] Always " "Providers[apache.hive,cncf.kubernetes,common.sql,exasol,ftp," - "http,imap,mongo,mysql,postgres,salesforce,ssh] Providers[google]", + "http,imap,microsoft.azure,mongo,mysql,postgres,salesforce,ssh] Providers[google]", }, id="Providers tests run including amazon tests if amazon provider files changed", ), diff --git a/docs/apache-airflow-providers-amazon/transfer/azure_blob_to_s3.rst b/docs/apache-airflow-providers-amazon/transfer/azure_blob_to_s3.rst new file mode 100644 index 000000000000..a3b9df5eb1ee --- /dev/null +++ b/docs/apache-airflow-providers-amazon/transfer/azure_blob_to_s3.rst @@ -0,0 +1,52 @@ + .. 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. + +=================================================== +Azure Blob Storage to Amazon S3 transfer operator +=================================================== + +Use the ``AzureBlobStorageToS3Operator`` transfer to copy the data from Azure Blob Storage to Amazon Simple Storage Service (S3). + +Prerequisite Tasks +------------------ + +.. include:: ../_partials/prerequisite_tasks.rst + +Operators +--------- + +.. _howto/operator:AzureBlobStorageToS3Operator: + +Azure Blob Storage to Amazon S3 +================================= + +To copy data from an Azure Blob Storage container to an Amazon S3 bucket you can use +:class:`~airflow.providers.amazon.aws.transfers.azure_blob_to_s3.AzureBlobStorageToS3Operator` + +Example usage: + +.. exampleinclude:: /../../tests/system/providers/amazon/aws/example_azure_blob_to_s3.py + :language: python + :dedent: 4 + :start-after: [START howto_transfer_azure_blob_to_s3] + :end-before: [END howto_transfer_azure_blob_to_s3] + +Reference +--------- + +* `Azure Blob Storage client library `__ +* `AWS boto3 library documentation for Amazon S3 `__ diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index de1f387add83..d80c3d084e01 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -45,6 +45,7 @@ "google", "http", "imap", + "microsoft.azure", "mongo", "salesforce", "ssh" diff --git a/tests/providers/amazon/aws/transfers/test_azure_blob_to_s3.py b/tests/providers/amazon/aws/transfers/test_azure_blob_to_s3.py new file mode 100644 index 000000000000..f34877e18722 --- /dev/null +++ b/tests/providers/amazon/aws/transfers/test_azure_blob_to_s3.py @@ -0,0 +1,218 @@ +# +# 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 io import RawIOBase +from unittest import mock + +from moto import mock_s3 + +from airflow.providers.amazon.aws.hooks.s3 import S3Hook +from airflow.providers.amazon.aws.transfers.azure_blob_to_s3 import AzureBlobStorageToS3Operator + +TASK_ID = "test-gcs-list-operator" +CONTAINER_NAME = "test-container" +DELIMITER = ".csv" +PREFIX = "TEST" +S3_BUCKET = "s3://bucket/" +MOCK_FILES = ["TEST1.csv", "TEST2.csv", "TEST3.csv"] +S3_ACL_POLICY = "private-read" + + +def _create_test_bucket(): + hook = S3Hook(aws_conn_id="airflow_gcs_test") + # We're mocking all actual AWS calls and don't need a connection. + # This avoids an Airflow warning about connection cannot be found. + hook.get_connection = lambda _: None + bucket = hook.get_bucket("bucket") + bucket.create() + return hook, bucket + + +@mock_s3 +class TestAzureBlobToS3Operator: + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + def test_operator_all_file_upload(self, mock_hook): + """ + Destination bucket has no file (of interest) common with origin bucket i.e + Azure - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + S3 - [] + """ + mock_hook.return_value.get_blobs_list_recursive.return_value = MOCK_FILES + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + replace=False, + ) + + hook, _ = _create_test_bucket() + uploaded_files = operator.execute(None) + + assert sorted(MOCK_FILES) == sorted(uploaded_files) + assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) + + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + def test_operator_incremental_file_upload_without_replace(self, mock_hook): + """ + Destination bucket has subset of files common with origin bucket i.e + Azure - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + S3 - ["TEST1.csv"] + """ + mock_hook.return_value.get_blobs_list_recursive.return_value = MOCK_FILES + get_file = mock_hook.return_value.get_file + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + # without replace + replace=False, + ) + + hook, bucket = _create_test_bucket() + # uploading only first file + bucket.put_object(Key=MOCK_FILES[0], Body=b"testing") + + uploaded_files = operator.execute(None) + + assert sorted(MOCK_FILES[1:]) == sorted(uploaded_files) + assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) + assert get_file.call_count == len(MOCK_FILES[1:]) + + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + def test_operator_incremental_file_upload_with_replace(self, mock_hook): + """ + Destination bucket has subset of files common with origin bucket i.e + Azure - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + S3 - ["TEST1.csv"] + """ + mock_hook.return_value.get_blobs_list_recursive.return_value = MOCK_FILES + get_file = mock_hook.return_value.get_file + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + # with replace + replace=True, + ) + + hook, bucket = _create_test_bucket() + # uploading only first file + bucket.put_object(Key=MOCK_FILES[0], Body=b"testing") + + uploaded_files = operator.execute(None) + + assert sorted(MOCK_FILES) == sorted(uploaded_files) + assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) + assert get_file.call_count == len(MOCK_FILES) + + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + def test_operator_no_file_upload_without_replace(self, mock_hook): + """ + Destination bucket has all the files common with origin bucket i.e + Azure - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + S3 - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + """ + mock_hook.return_value.get_blobs_list_recursive.return_value = MOCK_FILES + get_file = mock_hook.return_value.get_file + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + replace=False, + ) + + hook, bucket = _create_test_bucket() + # uploading all the files + for mock_file in MOCK_FILES: + bucket.put_object(Key=mock_file, Body=b"testing") + + uploaded_files = operator.execute(None) + + assert sorted([]) == sorted(uploaded_files) + assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) + assert get_file.call_count == 0 + + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + def test_operator_no_file_upload_with_replace(self, mock_hook): + """ + Destination bucket has all the files common with origin bucket i.e + Azure - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + S3 - ["TEST1.csv", "TEST2.csv", "TEST3.csv"] + """ + mock_hook.return_value.get_blobs_list_recursive.return_value = MOCK_FILES + get_file = mock_hook.return_value.get_file + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + replace=True, + ) + + hook, bucket = _create_test_bucket() + # uploading all the files + for mock_file in MOCK_FILES: + bucket.put_object(Key=mock_file, Body=b"testing") + + uploaded_files = operator.execute(None) + + assert sorted(MOCK_FILES) == sorted(uploaded_files) + assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) + # this ensures that upload happened + assert get_file.call_count == len(MOCK_FILES) + + @mock.patch("tempfile.NamedTemporaryFile") + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.WasbHook") + @mock.patch("airflow.providers.amazon.aws.transfers.azure_blob_to_s3.S3Hook") + def test_operator_should_pass_dest_s3_extra_args_and_s3_acl_policy( + self, s3_hook_mock, wasb_hook_mock, mock_tempfile + ): + wasb_blob_name = "test_file" + s3_acl_policy = "test policy" + s3_extra_args = {"ContentLanguage": "value"} + + wasb_hook_mock.return_value.get_blobs_list_recursive.return_value = [wasb_blob_name] + wasb_hook_mock.return_value.download.return_value = RawIOBase(b"testing") + mock_tempfile.return_value.__enter__.return_value.name = "test_temp_file" + + # with current S3_BUCKET url, parse_s3_url would complain + s3_hook_mock.parse_s3_url.return_value = ("bucket", wasb_blob_name) + mock_load_files = s3_hook_mock.return_value.load_file + + operator = AzureBlobStorageToS3Operator( + task_id=TASK_ID, + container_name=CONTAINER_NAME, + dest_s3_key=S3_BUCKET, + replace=False, + dest_s3_extra_args=s3_extra_args, + s3_acl_policy=s3_acl_policy, + ) + + operator.execute(None) + s3_hook_mock.assert_called_once_with(aws_conn_id="aws_default", extra_args=s3_extra_args, verify=None) + mock_load_files.assert_called_once_with( + filename="test_temp_file", + key=f"{S3_BUCKET}{wasb_blob_name}", + replace=False, + acl_policy=s3_acl_policy, + ) diff --git a/tests/providers/microsoft/azure/hooks/test_wasb.py b/tests/providers/microsoft/azure/hooks/test_wasb.py index 464db0f39f43..672d20d3a65e 100644 --- a/tests/providers/microsoft/azure/hooks/test_wasb.py +++ b/tests/providers/microsoft/azure/hooks/test_wasb.py @@ -23,6 +23,7 @@ import pytest from azure.identity import ClientSecretCredential, DefaultAzureCredential from azure.storage.blob import BlobServiceClient +from azure.storage.blob._models import BlobProperties from airflow.exceptions import AirflowException from airflow.models import Connection @@ -299,6 +300,30 @@ def test_get_blobs_list(self, mock_service): name_starts_with="my", include=None, delimiter="/" ) + @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.BlobServiceClient") + def test_get_blobs_list_recursive(self, mock_service): + hook = WasbHook(wasb_conn_id=self.shared_key_conn_id) + hook.get_blobs_list_recursive( + container_name="mycontainer", prefix="test", include=None, endswith="file_extension" + ) + mock_service.return_value.get_container_client.assert_called_once_with("mycontainer") + mock_service.return_value.get_container_client.return_value.list_blobs.assert_called_once_with( + name_starts_with="test", include=None + ) + + @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.BlobServiceClient") + def test_get_blobs_list_recursive_endswith(self, mock_service): + hook = WasbHook(wasb_conn_id=self.shared_key_conn_id) + mock_service.return_value.get_container_client.return_value.list_blobs.return_value = [ + BlobProperties(name="test/abc.py"), + BlobProperties(name="test/inside_test/abc.py"), + BlobProperties(name="test/abc.csv"), + ] + blob_list_output = hook.get_blobs_list_recursive( + container_name="mycontainer", prefix="test", include=None, endswith=".py" + ) + assert blob_list_output == ["test/abc.py", "test/inside_test/abc.py"] + @pytest.mark.parametrize(argnames="create_container", argvalues=[True, False]) @mock.patch.object(WasbHook, "upload") def test_load_file(self, mock_upload, create_container): diff --git a/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py b/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py new file mode 100644 index 000000000000..79b49966ca54 --- /dev/null +++ b/tests/system/providers/amazon/aws/example_azure_blob_to_s3.py @@ -0,0 +1,83 @@ +# 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 datetime import datetime + +from airflow import DAG +from airflow.models.baseoperator import chain +from airflow.providers.amazon.aws.operators.s3 import S3CreateBucketOperator, S3DeleteBucketOperator +from airflow.providers.amazon.aws.transfers.azure_blob_to_s3 import AzureBlobStorageToS3Operator +from airflow.utils.trigger_rule import TriggerRule +from tests.system.providers.amazon.aws.utils import SystemTestContextBuilder + +sys_test_context_task = SystemTestContextBuilder().build() + +DAG_ID = "example_azure_blob_to_s3" + +with DAG( + dag_id=DAG_ID, + schedule="@once", + start_date=datetime(2021, 1, 1), + tags=["example"], + catchup=False, +) as dag: + test_context = sys_test_context_task() + env_id = test_context["ENV_ID"] + + s3_bucket = f"{env_id}-azure_blob-to-s3-bucket" + s3_key = f"{env_id}-azure_blob-to-s3-key" + s3_key_url = f"s3://{s3_bucket}/{s3_key}" + azure_container_name = f"{env_id}-azure_blob-to-s3-container" + + create_s3_bucket = S3CreateBucketOperator(task_id="create_s3_bucket", bucket_name=s3_bucket) + + # [START howto_transfer_azure_blob_to_s3] + azure_blob_to_s3 = AzureBlobStorageToS3Operator( + task_id="azure_blob_to_s3", + container_name=azure_container_name, + dest_s3_key=s3_key_url, + ) + # [END howto_transfer_azure_blob_to_s3] + + delete_s3_bucket = S3DeleteBucketOperator( + task_id="delete_s3_bucket", + bucket_name=s3_bucket, + force_delete=True, + trigger_rule=TriggerRule.ALL_DONE, + ) + + chain( + # TEST SETUP + test_context, + create_s3_bucket, + # TEST BODY + azure_blob_to_s3, + # TEST TEARDOWN + delete_s3_bucket, + ) + + from tests.system.utils.watcher import watcher + + # This test needs watcher in order to properly mark success/failure + # when "tearDown" task with trigger rule is part of the DAG + list(dag.tasks) >> watcher() + +from tests.system.utils import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) From bc212f80e89e3d460e6320db0983126cea9300e0 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 4 Jul 2023 08:02:21 +0200 Subject: [PATCH 030/533] Exckude not-releaseable providers from extras not only from releasing (#32325) Since we are not releasing some providers yet, we want to exclude them also from being referenced in cross-provider dependencies. --- dev/provider_packages/prepare_provider_packages.py | 14 +++++++++----- generated/provider_dependencies.json | 4 +--- .../pre_commit_update_providers_dependencies.py | 9 ++++++++- setup.py | 4 ++++ 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py index 8bf6e24852dc..7cf37d382bde 100755 --- a/dev/provider_packages/prepare_provider_packages.py +++ b/dev/provider_packages/prepare_provider_packages.py @@ -123,7 +123,11 @@ # those imports need to come after the above sys.path.insert to make sure that Airflow # sources are importable without having to add the airflow sources to the PYTHONPATH before # running the script -from setup import PREINSTALLED_PROVIDERS, ALL_PROVIDERS # type: ignore[attr-defined] # isort:skip # noqa +from setup import ( # type: ignore[attr-defined] # isort:skip # noqa + ALL_PROVIDERS, + NOT_RELEASED_YET_PROVIDERS, + PREINSTALLED_PROVIDERS, +) # Note - we do not test protocols as they are not really part of the official API of # Apache Airflow @@ -1675,10 +1679,10 @@ def verify_changelog_exists(package: str) -> str: def list_providers_packages(): """List all provider packages.""" providers = get_all_providers() - # For now we should exclude open-lineage from being consider for releasing until it is ready to - # be released - if "openlineage" in providers: - providers.remove("openlineage") + # exclude providers we do not release yet + for provider in NOT_RELEASED_YET_PROVIDERS: + if provider in providers: + providers.remove(provider) for provider in providers: console.print(provider) diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index d80c3d084e01..aaa5033b9a25 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -272,9 +272,7 @@ "apache-airflow>=2.4.0", "sqlparse>=0.4.2" ], - "cross-providers-deps": [ - "openlineage" - ], + "cross-providers-deps": [], "excluded-python-versions": [] }, "databricks": { diff --git a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py index 0c489bad63e3..b5a366499190 100755 --- a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py +++ b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py @@ -52,6 +52,7 @@ ALL_PROVIDERS: dict[str, dict[str, Any]] = defaultdict(lambda: defaultdict()) ALL_PROVIDER_FILES: list[Path] = [] + # Allow AST to parse the files. sys.path.append(str(AIRFLOW_SOURCES_ROOT)) @@ -168,9 +169,15 @@ def check_if_different_provider_used(file_path: Path) -> None: imports = get_imports_from_file(file_path) for import_name in imports: imported_provider = get_provider_id_from_import(import_name, file_path) + from setup import NOT_RELEASED_YET_PROVIDERS + if imported_provider is not None and imported_provider not in ALL_PROVIDERS: warnings.append(f"The provider {imported_provider} from {file_path} cannot be found.") - elif imported_provider and file_provider != imported_provider: + elif ( + imported_provider + and file_provider != imported_provider + and imported_provider not in NOT_RELEASED_YET_PROVIDERS + ): ALL_DEPENDENCIES[file_provider]["cross-providers-deps"].append(imported_provider) diff --git a/setup.py b/setup.py index 547e1221d37e..32d5c5444737 100644 --- a/setup.py +++ b/setup.py @@ -60,6 +60,10 @@ DEPS = "deps" CURRENT_PYTHON_VERSION = f"{sys.version_info.major}.{sys.version_info.minor}" +# we do not yet release some providers so we should skip them when releasing and when referring to +# in cross-provider-dependencies. +NOT_RELEASED_YET_PROVIDERS = ["openlineage"] + # # NOTE! IN Airflow 2.4.+ dependencies for providers are maintained in `provider.yaml` files for each From d7c966441ee57da6f612d9c0d41a37a9ff323de7 Mon Sep 17 00:00:00 2001 From: eladkal <45845474+eladkal@users.noreply.github.com> Date: Tue, 4 Jul 2023 09:08:16 +0300 Subject: [PATCH 031/533] Revert "Exckude not-releaseable providers from extras not only from releasing (#32325)" (#32338) This reverts commit bc212f80e89e3d460e6320db0983126cea9300e0. --- dev/provider_packages/prepare_provider_packages.py | 14 +++++--------- generated/provider_dependencies.json | 4 +++- .../pre_commit_update_providers_dependencies.py | 9 +-------- setup.py | 4 ---- 4 files changed, 9 insertions(+), 22 deletions(-) diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py index 7cf37d382bde..8bf6e24852dc 100755 --- a/dev/provider_packages/prepare_provider_packages.py +++ b/dev/provider_packages/prepare_provider_packages.py @@ -123,11 +123,7 @@ # those imports need to come after the above sys.path.insert to make sure that Airflow # sources are importable without having to add the airflow sources to the PYTHONPATH before # running the script -from setup import ( # type: ignore[attr-defined] # isort:skip # noqa - ALL_PROVIDERS, - NOT_RELEASED_YET_PROVIDERS, - PREINSTALLED_PROVIDERS, -) +from setup import PREINSTALLED_PROVIDERS, ALL_PROVIDERS # type: ignore[attr-defined] # isort:skip # noqa # Note - we do not test protocols as they are not really part of the official API of # Apache Airflow @@ -1679,10 +1675,10 @@ def verify_changelog_exists(package: str) -> str: def list_providers_packages(): """List all provider packages.""" providers = get_all_providers() - # exclude providers we do not release yet - for provider in NOT_RELEASED_YET_PROVIDERS: - if provider in providers: - providers.remove(provider) + # For now we should exclude open-lineage from being consider for releasing until it is ready to + # be released + if "openlineage" in providers: + providers.remove("openlineage") for provider in providers: console.print(provider) diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index aaa5033b9a25..d80c3d084e01 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -272,7 +272,9 @@ "apache-airflow>=2.4.0", "sqlparse>=0.4.2" ], - "cross-providers-deps": [], + "cross-providers-deps": [ + "openlineage" + ], "excluded-python-versions": [] }, "databricks": { diff --git a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py index b5a366499190..0c489bad63e3 100755 --- a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py +++ b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py @@ -52,7 +52,6 @@ ALL_PROVIDERS: dict[str, dict[str, Any]] = defaultdict(lambda: defaultdict()) ALL_PROVIDER_FILES: list[Path] = [] - # Allow AST to parse the files. sys.path.append(str(AIRFLOW_SOURCES_ROOT)) @@ -169,15 +168,9 @@ def check_if_different_provider_used(file_path: Path) -> None: imports = get_imports_from_file(file_path) for import_name in imports: imported_provider = get_provider_id_from_import(import_name, file_path) - from setup import NOT_RELEASED_YET_PROVIDERS - if imported_provider is not None and imported_provider not in ALL_PROVIDERS: warnings.append(f"The provider {imported_provider} from {file_path} cannot be found.") - elif ( - imported_provider - and file_provider != imported_provider - and imported_provider not in NOT_RELEASED_YET_PROVIDERS - ): + elif imported_provider and file_provider != imported_provider: ALL_DEPENDENCIES[file_provider]["cross-providers-deps"].append(imported_provider) diff --git a/setup.py b/setup.py index 32d5c5444737..547e1221d37e 100644 --- a/setup.py +++ b/setup.py @@ -60,10 +60,6 @@ DEPS = "deps" CURRENT_PYTHON_VERSION = f"{sys.version_info.major}.{sys.version_info.minor}" -# we do not yet release some providers so we should skip them when releasing and when referring to -# in cross-provider-dependencies. -NOT_RELEASED_YET_PROVIDERS = ["openlineage"] - # # NOTE! IN Airflow 2.4.+ dependencies for providers are maintained in `provider.yaml` files for each From 702209898eeb3061024bdedc5ac7b4d48c13108c Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Tue, 4 Jul 2023 16:22:43 +0530 Subject: [PATCH 032/533] typo fix (#32317) --- docs/apache-airflow-providers-amazon/redirects.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/apache-airflow-providers-amazon/redirects.txt b/docs/apache-airflow-providers-amazon/redirects.txt index bf35801e962e..fad1f914880a 100644 --- a/docs/apache-airflow-providers-amazon/redirects.txt +++ b/docs/apache-airflow-providers-amazon/redirects.txt @@ -28,9 +28,9 @@ operators/redshift_data.rst operators/redshift/redshift_data.rst operators/redshift_cluster.rst operators/redshift/redshift_cluster.rst operators/s3.rst operators/s3/s3.rst operators/glacier.rst operators/s3/glacier.rst -operators/transfer/dynamodb_to_s3.rst.rst transfer/dynamodb_to_s3.rst.rst -operators/transfer/ftp_to_s3.rst.rst transfer/ftp_to_s3.rst.rst -operators/transfer/gcs_to_s3.rst.rst transfer/gcs_to_s3.rst.rst +operators/transfer/dynamodb_to_s3.rst transfer/dynamodb_to_s3.rst +operators/transfer/ftp_to_s3.rst transfer/ftp_to_s3.rst +operators/transfer/gcs_to_s3.rst transfer/gcs_to_s3.rst operators/transfer/glacier_to_gcs.rst transfer/glacier_to_gcs.rst operators/transfer/google_api_to_s3.rst transfer/google_api_to_s3.rst operators/transfer/hive_to_dynamodb.rst transfer/hive_to_dynamodb.rst From 1c1dbd872d5378856b4242961dcdf77c7f0dd712 Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 4 Jul 2023 13:48:59 +0200 Subject: [PATCH 033/533] Add proxy_user template check (#32334) --- airflow/providers/apache/hive/hooks/hive.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow/providers/apache/hive/hooks/hive.py b/airflow/providers/apache/hive/hooks/hive.py index 4ad3b9d12c69..89fcda466ffc 100644 --- a/airflow/providers/apache/hive/hooks/hive.py +++ b/airflow/providers/apache/hive/hooks/hive.py @@ -152,6 +152,8 @@ def _prepare_cli_cmd(self) -> list[Any]: proxy_user = self._get_proxy_user() if ";" in template: raise RuntimeError("The principal should not contain the ';' character") + if ";" in proxy_user: + raise RuntimeError("The proxy_user should not contain the ';' character") jdbc_url += f";principal={template};{proxy_user}" elif self.auth: jdbc_url += ";auth=" + self.auth From 3c3c337edd6a6905c958206dc8f9fe4303c856eb Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Tue, 4 Jul 2023 14:58:29 +0200 Subject: [PATCH 034/533] Revert "Refactor Sqlalchemy queries to 2.0 style (Part 3) (#32177)" (#32343) This reverts commit 1065687ec6df2b9b3557e38a67e71f835796427f. --- airflow/utils/db.py | 38 ++-- airflow/www/utils.py | 11 +- airflow/www/views.py | 460 +++++++++++++++++++++---------------------- 3 files changed, 244 insertions(+), 265 deletions(-) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 46ddcbfb3453..a76f0d4f675d 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -92,7 +92,7 @@ def _format_airflow_moved_table_name(source_table, version, category): @provide_session def merge_conn(conn, session: Session = NEW_SESSION): """Add new Connection.""" - if not session.scalar(select(conn.__class__).filter_by(conn_id=conn.conn_id).limit(1)): + if not session.query(conn.__class__).filter_by(conn_id=conn.conn_id).first(): session.add(conn) session.commit() @@ -959,9 +959,7 @@ def check_conn_id_duplicates(session: Session) -> Iterable[str]: dups = [] try: - dups = session.execute( - select(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1) - ).all() + dups = session.query(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1).all() except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -986,11 +984,12 @@ def check_username_duplicates(session: Session) -> Iterable[str]: for model in [User, RegisterUser]: dups = [] try: - dups = session.execute( - select(model.username) # type: ignore[attr-defined] + dups = ( + session.query(model.username) # type: ignore[attr-defined] .group_by(model.username) # type: ignore[attr-defined] .having(func.count() > 1) - ).all() + .all() + ) except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -1059,13 +1058,13 @@ def check_task_fail_for_duplicates(session): """ minimal_table_obj = table(table_name, *[column(x) for x in uniqueness]) try: - subquery = session.execute( - select(minimal_table_obj, func.count().label("dupe_count")) + subquery = ( + session.query(minimal_table_obj, func.count().label("dupe_count")) .group_by(*[text(x) for x in uniqueness]) .having(func.count() > text("1")) .subquery() ) - dupe_count = session.scalar(select(func.sum(subquery.c.dupe_count))) + dupe_count = session.query(func.sum(subquery.c.dupe_count)).scalar() if not dupe_count: # there are no duplicates; nothing to do. return @@ -1102,7 +1101,7 @@ def check_conn_type_null(session: Session) -> Iterable[str]: n_nulls = [] try: - n_nulls = session.scalars(select(Connection.conn_id).where(Connection.conn_type.is_(None))).all() + n_nulls = session.query(Connection.conn_id).filter(Connection.conn_type.is_(None)).all() except (exc.OperationalError, exc.ProgrammingError, exc.InternalError): # fallback if tables hasn't been created yet session.rollback() @@ -1144,7 +1143,7 @@ def check_run_id_null(session: Session) -> Iterable[str]: dagrun_table.c.run_id.is_(None), dagrun_table.c.execution_date.is_(None), ) - invalid_dagrun_count = session.scalar(select(func.count(dagrun_table.c.id)).where(invalid_dagrun_filter)) + invalid_dagrun_count = session.query(func.count(dagrun_table.c.id)).filter(invalid_dagrun_filter).scalar() if invalid_dagrun_count > 0: dagrun_dangling_table_name = _format_airflow_moved_table_name(dagrun_table.name, "2.2", "dangling") if dagrun_dangling_table_name in inspect(session.get_bind()).get_table_names(): @@ -1241,7 +1240,7 @@ def _move_dangling_data_to_new_table( pk_cols = source_table.primary_key.columns delete = source_table.delete().where( - tuple_(*pk_cols).in_(session.select(*target_table.primary_key.columns).subquery()) + tuple_(*pk_cols).in_(session.query(*target_table.primary_key.columns).subquery()) ) else: delete = source_table.delete().where( @@ -1263,11 +1262,10 @@ def _dangling_against_dag_run(session, source_table, dag_run): source_table.c.dag_id == dag_run.c.dag_id, source_table.c.execution_date == dag_run.c.execution_date, ) - return ( - select(*[c.label(c.name) for c in source_table.c]) + session.query(*[c.label(c.name) for c in source_table.c]) .join(dag_run, source_to_dag_run_join_cond, isouter=True) - .where(dag_run.c.dag_id.is_(None)) + .filter(dag_run.c.dag_id.is_(None)) ) @@ -1306,10 +1304,10 @@ def _dangling_against_task_instance(session, source_table, dag_run, task_instanc ) return ( - select(*[c.label(c.name) for c in source_table.c]) + session.query(*[c.label(c.name) for c in source_table.c]) .join(dag_run, dr_join_cond, isouter=True) .join(task_instance, ti_join_cond, isouter=True) - .where(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) + .filter(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) ) @@ -1333,9 +1331,9 @@ def _move_duplicate_data_to_new_table( """ bind = session.get_bind() dialect_name = bind.dialect.name - query = ( - select(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) + session.query(source_table) + .with_entities(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) .select_from(source_table) .join(subquery, and_(*[getattr(source_table.c, x) == getattr(subquery.c, x) for x in uniqueness])) ) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index b31f9326d988..76914dd9cdcd 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -39,7 +39,6 @@ from pygments.lexer import Lexer from sqlalchemy import delete, func, types from sqlalchemy.ext.associationproxy import AssociationProxy -from sqlalchemy.sql import Select from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import errors @@ -54,6 +53,7 @@ from airflow.www.widgets import AirflowDateTimePickerWidget if TYPE_CHECKING: + from sqlalchemy.orm.query import Query from sqlalchemy.orm.session import Session from sqlalchemy.sql.operators import ColumnOperators @@ -518,21 +518,18 @@ def _get_run_ordering_expr(name: str) -> ColumnOperators: return expr.desc() -def sorted_dag_runs( - query: Select, *, ordering: Sequence[str], limit: int, session: Session -) -> Sequence[DagRun]: +def sorted_dag_runs(query: Query, *, ordering: Sequence[str], limit: int) -> Sequence[DagRun]: """Produce DAG runs sorted by specified columns. - :param query: An ORM select object against *DagRun*. + :param query: An ORM query object against *DagRun*. :param ordering: Column names to sort the runs. should generally come from a timetable's ``run_ordering``. :param limit: Number of runs to limit to. - :param session: SQLAlchemy ORM session object :return: A list of DagRun objects ordered by the specified columns. The list contains only the *last* objects, but in *ascending* order. """ ordering_exprs = (_get_run_ordering_expr(name) for name in ordering) - runs = session.scalars(query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit)).all() + runs = query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit).all() runs.reverse() return runs diff --git a/airflow/www/views.py b/airflow/www/views.py index 90d98a424c4a..3f0965da3889 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -69,7 +69,7 @@ from pendulum.parsing.exceptions import ParserError from pygments import highlight, lexers from pygments.formatters import HtmlFormatter -from sqlalchemy import Date, and_, case, desc, func, inspect, or_, select, union_all +from sqlalchemy import Date, and_, case, desc, func, inspect, or_, union_all from sqlalchemy.exc import IntegrityError from sqlalchemy.orm import Session, joinedload from wtforms import SelectField, validators @@ -95,7 +95,7 @@ from airflow.jobs.job import Job from airflow.jobs.scheduler_job_runner import SchedulerJobRunner from airflow.jobs.triggerer_job_runner import TriggererJobRunner -from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, Trigger, XCom, errors +from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, XCom, errors from airflow.models.abstractoperator import AbstractOperator from airflow.models.dag import DAG, get_dataset_triggered_next_run_info from airflow.models.dagcode import DagCode @@ -231,15 +231,16 @@ def get_date_time_num_runs_dag_runs_form_data(www_request, session, dag): # loaded and the actual requested run would be excluded by the limit(). Once # the user has changed base date to be anything else we want to use that instead. query_date = base_date - if date_time < base_date <= date_time + datetime.timedelta(seconds=1): + if date_time < base_date and date_time + datetime.timedelta(seconds=1) >= base_date: query_date = date_time - drs = session.scalars( - select(DagRun) - .where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) + drs = ( + session.query(DagRun) + .filter(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) .order_by(desc(DagRun.execution_date)) .limit(num_runs) - ).all() + .all() + ) dr_choices = [] dr_state = None for dr in drs: @@ -290,8 +291,8 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): Create a nested dict representation of the DAG's TaskGroup and its children used to construct the Graph and Grid views. """ - query = session.execute( - select( + query = ( + session.query( TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, @@ -302,7 +303,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): func.max(TaskInstance.end_date).label("end_date"), ) .join(TaskInstance.task_instance_note, isouter=True) - .where( + .filter( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), ) @@ -425,9 +426,11 @@ def get_summary(dag_run: DagRun): } def get_mapped_group_summaries(): - mapped_ti_query = session.execute( - select(TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index) - .where( + mapped_ti_query = ( + session.query( + TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index + ) + .filter( TaskInstance.dag_id == dag.dag_id, TaskInstance.task_id.in_(child["id"] for child in children), TaskInstance.run_id.in_(r.run_id for r in dag_runs), @@ -735,20 +738,21 @@ def index(self): with create_session() as session: # read orm_dags from the db - dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) + dags_query = session.query(DagModel).filter(~DagModel.is_subdag, DagModel.is_active) if arg_search_query: escaped_arg_search_query = arg_search_query.replace("_", r"\_") - dags_query = dags_query.where( + dags_query = dags_query.filter( DagModel.dag_id.ilike("%" + escaped_arg_search_query + "%", escape="\\") | DagModel.owners.ilike("%" + escaped_arg_search_query + "%", escape="\\") ) if arg_tags_filter: - dags_query = dags_query.where(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) + dags_query = dags_query.filter(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) - dags_query = dags_query.where(DagModel.dag_id.in_(filter_dag_ids)) - filtered_dag_count = session.scalar(select(func.count()).select_from(dags_query)) + dags_query = dags_query.filter(DagModel.dag_id.in_(filter_dag_ids)) + + filtered_dag_count = dags_query.count() if filtered_dag_count == 0 and len(arg_tags_filter): flash( "No matching DAG tags found.", @@ -758,28 +762,28 @@ def index(self): return redirect(url_for("Airflow.index")) all_dags = dags_query - active_dags = dags_query.where(~DagModel.is_paused) - paused_dags = dags_query.where(DagModel.is_paused) + active_dags = dags_query.filter(~DagModel.is_paused) + paused_dags = dags_query.filter(DagModel.is_paused) # find DAGs which have a RUNNING DagRun - running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).where( + running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).filter( DagRun.state == State.RUNNING ) # find DAGs for which the latest DagRun is FAILED subq_all = ( - select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) .group_by(DagRun.dag_id) .subquery() ) subq_failed = ( - select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) - .where(DagRun.state == State.FAILED) + session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + .filter(DagRun.state == State.FAILED) .group_by(DagRun.dag_id) .subquery() ) subq_join = ( - select(subq_all.c.dag_id, subq_all.c.start_date) + session.query(subq_all.c.dag_id, subq_all.c.start_date) .join( subq_failed, and_( @@ -792,18 +796,16 @@ def index(self): failed_dags = dags_query.join(subq_join, DagModel.dag_id == subq_join.c.dag_id) is_paused_count = dict( - session.execute( - select(DagModel.is_paused, func.count(DagModel.dag_id)) - .group_by(DagModel.is_paused) - .select_from(all_dags) - ).all() + all_dags.with_entities(DagModel.is_paused, func.count(DagModel.dag_id)).group_by( + DagModel.is_paused + ) ) status_count_active = is_paused_count.get(False, 0) status_count_paused = is_paused_count.get(True, 0) - status_count_running = session.scalar(select(func.count()).select_from(running_dags)) - status_count_failed = session.scalar(select(func.count()).select_from(failed_dags)) + status_count_running = running_dags.count() + status_count_failed = failed_dags.count() all_dags_count = status_count_active + status_count_paused if arg_status_filter == "active": @@ -824,7 +826,7 @@ def index(self): if arg_sorting_key == "last_dagrun": dag_run_subquery = ( - select( + session.query( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) @@ -852,13 +854,7 @@ def index(self): else: current_dags = current_dags.order_by(null_case, sort_column) - dags = ( - session.scalars( - current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page) - ) - .unique() - .all() - ) + dags = current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page).all() user_permissions = g.user.perms can_create_dag_run = ( permissions.ACTION_CAN_CREATE, @@ -878,7 +874,7 @@ def index(self): dag.can_trigger = dag.can_edit and can_create_dag_run dag.can_delete = get_airflow_app().appbuilder.sm.can_delete_dag(dag.dag_id, g.user) - dagtags = session.execute(select(func.distinct(DagTag.name)).order_by(DagTag.name)).all() + dagtags = session.query(func.distinct(DagTag.name)).order_by(DagTag.name).all() tags = [ {"name": name, "selected": bool(arg_tags_filter and name in arg_tags_filter)} for name, in dagtags @@ -886,15 +882,14 @@ def index(self): owner_links_dict = DagOwnerAttributes.get_all(session) - import_errors = select(errors.ImportError).order_by(errors.ImportError.id) + import_errors = session.query(errors.ImportError).order_by(errors.ImportError.id) if (permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG) not in user_permissions: # if the user doesn't have access to all DAGs, only display errors from visible DAGs import_errors = import_errors.join( DagModel, DagModel.fileloc == errors.ImportError.filename - ).where(DagModel.dag_id.in_(filter_dag_ids)) + ).filter(DagModel.dag_id.in_(filter_dag_ids)) - import_errors = session.scalars(import_errors) for import_error in import_errors: flash( f"Broken DAG: [{import_error.filename}] {import_error.stacktrace}", @@ -938,12 +933,10 @@ def _iter_parsed_moved_data_table_names(): permissions.RESOURCE_ADMIN_MENU, ) in user_permissions and conf.getboolean("webserver", "warn_deployment_exposure"): robots_file_access_count = ( - select(Log) - .where(Log.event == "robots") - .where(Log.dttm > (utcnow() - datetime.timedelta(days=7))) - ) - robots_file_access_count = session.scalar( - select(func.count()).select_from(robots_file_access_count) + session.query(Log) + .filter(Log.event == "robots") + .filter(Log.dttm > (utcnow() - datetime.timedelta(days=7))) + .count() ) if robots_file_access_count > 0: flash( @@ -1045,11 +1038,9 @@ def next_run_datasets_summary(self, session: Session = NEW_SESSION): dataset_triggered_dag_ids = [ dag.dag_id for dag in ( - session.scalars( - select(DagModel.dag_id) - .where(DagModel.dag_id.in_(filter_dag_ids)) - .where(DagModel.schedule_interval == "Dataset") - ) + session.query(DagModel.dag_id) + .filter(DagModel.dag_id.in_(filter_dag_ids)) + .filter(DagModel.schedule_interval == "Dataset") ) ] @@ -1080,10 +1071,10 @@ def dag_stats(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify({}) - dag_state_stats = session.execute( - select(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) + dag_state_stats = ( + session.query(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) .group_by(DagRun.dag_id, DagRun.state) - .where(DagRun.dag_id.in_(filter_dag_ids)) + .filter(DagRun.dag_id.in_(filter_dag_ids)) ) dag_state_data = {(dag_id, state): count for dag_id, state, count in dag_state_stats} @@ -1121,17 +1112,17 @@ def task_stats(self, session: Session = NEW_SESSION): filter_dag_ids = allowed_dag_ids running_dag_run_query_result = ( - select(DagRun.dag_id, DagRun.run_id) + session.query(DagRun.dag_id, DagRun.run_id) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .where(DagRun.state == State.RUNNING, DagModel.is_active) + .filter(DagRun.state == State.RUNNING, DagModel.is_active) ) - running_dag_run_query_result = running_dag_run_query_result.where(DagRun.dag_id.in_(filter_dag_ids)) + running_dag_run_query_result = running_dag_run_query_result.filter(DagRun.dag_id.in_(filter_dag_ids)) running_dag_run_query_result = running_dag_run_query_result.subquery("running_dag_run") # Select all task_instances from active dag_runs. - running_task_instance_query_result = select( + running_task_instance_query_result = session.query( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(True).label("is_dag_running"), @@ -1145,19 +1136,19 @@ def task_stats(self, session: Session = NEW_SESSION): if conf.getboolean("webserver", "SHOW_RECENT_STATS_FOR_COMPLETED_RUNS", fallback=True): last_dag_run = ( - select(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) + session.query(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .where(DagRun.state != State.RUNNING, DagModel.is_active) + .filter(DagRun.state != State.RUNNING, DagModel.is_active) .group_by(DagRun.dag_id) ) - last_dag_run = last_dag_run.where(DagRun.dag_id.in_(filter_dag_ids)) + last_dag_run = last_dag_run.filter(DagRun.dag_id.in_(filter_dag_ids)) last_dag_run = last_dag_run.subquery("last_dag_run") # Select all task_instances from active dag_runs. # If no dag_run is active, return task instances from most recent dag_run. last_task_instance_query_result = ( - select( + session.query( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(False).label("is_dag_running"), @@ -1178,8 +1169,8 @@ def task_stats(self, session: Session = NEW_SESSION): else: final_task_instance_query_result = running_task_instance_query_result.subquery("final_ti") - qry = session.execute( - select( + qry = ( + session.query( final_task_instance_query_result.c.dag_id, final_task_instance_query_result.c.state, final_task_instance_query_result.c.is_dag_running, @@ -1195,6 +1186,7 @@ def task_stats(self, session: Session = NEW_SESSION): final_task_instance_query_result.c.is_dag_running.desc(), ) ) + data = get_task_stats_from_query(qry) payload: dict[str, list[dict[str, Any]]] = collections.defaultdict(list) for dag_id in filter_dag_ids: @@ -1227,31 +1219,29 @@ def last_dagruns(self, session: Session = NEW_SESSION): return flask.json.jsonify({}) last_runs_subquery = ( - select( + session.query( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) .group_by(DagRun.dag_id) - .where(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. + .filter(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. .subquery("last_runs") ) - query = session.execute( - select( - DagRun.dag_id, - DagRun.start_date, - DagRun.end_date, - DagRun.state, - DagRun.execution_date, - DagRun.data_interval_start, - DagRun.data_interval_end, - ).join( - last_runs_subquery, - and_( - last_runs_subquery.c.dag_id == DagRun.dag_id, - last_runs_subquery.c.max_execution_date == DagRun.execution_date, - ), - ) + query = session.query( + DagRun.dag_id, + DagRun.start_date, + DagRun.end_date, + DagRun.state, + DagRun.execution_date, + DagRun.data_interval_start, + DagRun.data_interval_end, + ).join( + last_runs_subquery, + and_( + last_runs_subquery.c.dag_id == DagRun.dag_id, + last_runs_subquery.c.max_execution_date == DagRun.execution_date, + ), ) resp = { @@ -1350,18 +1340,19 @@ def dag_details(self, dag_id, session: Session = NEW_SESSION): title = "DAG Details" root = request.args.get("root", "") - states = session.execute( - select(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) - .where(TaskInstance.dag_id == dag_id) + states = ( + session.query(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) + .filter(TaskInstance.dag_id == dag_id) .group_by(TaskInstance.state) - ).all() + .all() + ) active_runs = models.DagRun.find(dag_id=dag_id, state=DagRunState.RUNNING, external_trigger=False) - tags = session.scalars(select(models.DagTag).where(models.DagTag.dag_id == dag_id)).all() + tags = session.query(models.DagTag).filter(models.DagTag.dag_id == dag_id).all() # TODO: convert this to a relationship - owner_links = session.execute(select(DagOwnerAttributes).filter_by(dag_id=dag_id)).all() + owner_links = session.query(DagOwnerAttributes).filter_by(dag_id=dag_id).all() attrs_to_avoid = [ "schedule_datasets", @@ -1622,17 +1613,18 @@ def get_logs_with_metadata(self, session: Session = NEW_SESSION): "metadata": {"end_of_log": True}, } - ti = session.scalar( - select(models.TaskInstance) - .where( + ti = ( + session.query(models.TaskInstance) + .filter( TaskInstance.task_id == task_id, TaskInstance.dag_id == dag_id, TaskInstance.execution_date == execution_date, TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) - .options(joinedload(TaskInstance.trigger).joinedload(Trigger.triggerer_job)) - .limit(1) + .options(joinedload("trigger")) + .options(joinedload("trigger.triggerer_job")) + .first() ) if ti is None: @@ -1690,10 +1682,10 @@ def log(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) dag_model = DagModel.get_dagmodel(dag_id) - ti = session.scalar( - select(models.TaskInstance) + ti = ( + session.query(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .limit(1) + .first() ) num_logs = 0 @@ -1734,10 +1726,10 @@ def redirect_to_external_log(self, session: Session = NEW_SESSION): map_index = request.args.get("map_index", -1, type=int) try_number = request.args.get("try_number", 1) - ti = session.scalar( - select(models.TaskInstance) + ti = ( + session.query(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .limit(1) + .first() ) if not ti: @@ -1779,8 +1771,8 @@ def task(self, session: Session = NEW_SESSION): task = copy.copy(dag.get_task(task_id)) task.resolve_template_files() - ti: TaskInstance | None = session.scalar( - select(TaskInstance) + ti: TaskInstance | None = ( + session.query(TaskInstance) .options( # HACK: Eager-load relationships. This is needed because # multiple properties mis-use provide_session() that destroys @@ -1789,6 +1781,7 @@ def task(self, session: Session = NEW_SESSION): joinedload(TaskInstance.trigger, innerjoin=False), ) .filter_by(execution_date=dttm, dag_id=dag_id, task_id=task_id, map_index=map_index) + .one_or_none() ) if ti is None: ti_attrs: list[tuple[str, Any]] | None = None @@ -1905,19 +1898,17 @@ def xcom(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) root = request.args.get("root", "") dag = DagModel.get_dagmodel(dag_id) - ti = session.scalar(select(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).limit(1)) + ti = session.query(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).first() if not ti: flash(f"Task [{dag_id}.{task_id}] doesn't seem to exist at the moment", "error") return redirect(url_for("Airflow.index")) - xcom_query = session.execute( - select(XCom.key, XCom.value).where( - XCom.dag_id == dag_id, - XCom.task_id == task_id, - XCom.execution_date == dttm, - XCom.map_index == map_index, - ) + xcom_query = session.query(XCom.key, XCom.value).filter( + XCom.dag_id == dag_id, + XCom.task_id == task_id, + XCom.execution_date == dttm, + XCom.map_index == map_index, ) attributes = [(k, v) for k, v in xcom_query if not k.startswith("_")] @@ -1987,7 +1978,7 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): request_execution_date = request.values.get("execution_date", default=timezone.utcnow().isoformat()) is_dag_run_conf_overrides_params = conf.getboolean("core", "dag_run_conf_overrides_params") dag = get_airflow_app().dag_bag.get_dag(dag_id) - dag_orm: DagModel = session.scalar(select(DagModel).where(DagModel.dag_id == dag_id).limit(1)) + dag_orm: DagModel = session.query(DagModel).filter(DagModel.dag_id == dag_id).first() # Prepare form fields with param struct details to render a proper form with schema information form_fields = {} @@ -2025,9 +2016,11 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): flash(f"Cannot create dagruns because the dag {dag_id} has import errors", "error") return redirect(origin) - recent_runs = session.execute( - select(DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date)) - .where( + recent_runs = ( + session.query( + DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date) + ) + .filter( DagRun.dag_id == dag_id, DagRun.run_type == DagRunType.MANUAL, DagRun.conf.isnot(None), @@ -2300,17 +2293,15 @@ def clear(self, *, session: Session = NEW_SESSION): # Lock the related dag runs to prevent from possible dead lock. # https://github.com/apache/airflow/pull/26658 - dag_runs_query = session.scalars( - select(DagRun.id).where(DagRun.dag_id == dag_id).with_for_update() - ) + dag_runs_query = session.query(DagRun.id).filter(DagRun.dag_id == dag_id).with_for_update() if start_date is None and end_date is None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date == start_date) + dag_runs_query = dag_runs_query.filter(DagRun.execution_date == start_date) else: if start_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date >= start_date) + dag_runs_query = dag_runs_query.filter(DagRun.execution_date >= start_date) if end_date is not None: - dag_runs_query = dag_runs_query.where(DagRun.execution_date <= end_date) + dag_runs_query = dag_runs_query.filter(DagRun.execution_date <= end_date) locked_dag_run_ids = dag_runs_query.all() elif task_id: @@ -2399,10 +2390,10 @@ def blocked(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify([]) - dags = session.execute( - select(DagRun.dag_id, sqla.func.count(DagRun.id)) - .where(DagRun.state == DagRunState.RUNNING) - .where(DagRun.dag_id.in_(filter_dag_ids)) + dags = ( + session.query(DagRun.dag_id, sqla.func.count(DagRun.id)) + .filter(DagRun.state == DagRunState.RUNNING) + .filter(DagRun.dag_id.in_(filter_dag_ids)) .group_by(DagRun.dag_id) ) @@ -2483,11 +2474,9 @@ def _mark_dagrun_state_as_queued( # Identify tasks that will be queued up to run when confirmed all_task_ids = [task.task_id for task in dag.tasks] - existing_tis = session.execute( - select(TaskInstance.task_id).where( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.run_id == dag_run_id, - ) + existing_tis = session.query(TaskInstance.task_id).filter( + TaskInstance.dag_id == dag.dag_id, + TaskInstance.run_id == dag_run_id, ) completed_tis_ids = [task_id for task_id, in existing_tis] @@ -2969,18 +2958,19 @@ def _convert_to_date(session, column): if root: dag = dag.partial_subset(task_ids_or_regex=root, include_downstream=False, include_upstream=True) - dag_states = session.execute( - select( - _convert_to_date(session, DagRun.execution_date).label("date"), + dag_states = ( + session.query( + (_convert_to_date(session, DagRun.execution_date)).label("date"), DagRun.state, func.max(DagRun.data_interval_start).label("data_interval_start"), func.max(DagRun.data_interval_end).label("data_interval_end"), func.count("*").label("count"), ) - .where(DagRun.dag_id == dag.dag_id) + .filter(DagRun.dag_id == dag.dag_id) .group_by(_convert_to_date(session, DagRun.execution_date), DagRun.state) .order_by(_convert_to_date(session, DagRun.execution_date).asc()) - ).all() + .all() + ) data_dag_states = [ { @@ -3251,17 +3241,16 @@ def duration(self, dag_id: str, session: Session = NEW_SESSION): else: min_date = timezone.utc_epoch() ti_fails = ( - select(TaskFail) + session.query(TaskFail) .join(TaskFail.dag_run) - .where( + .filter( TaskFail.dag_id == dag.dag_id, DagRun.execution_date >= min_date, DagRun.execution_date <= base_date, ) ) if dag.partial: - ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) - ti_fails = session.scalars(ti_fails) + ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) fails_totals: dict[tuple[str, str, str], int] = defaultdict(int) for failed_task_instance in ti_fails: dict_key = ( @@ -3602,9 +3591,9 @@ def gantt(self, dag_id: str, session: Session = NEW_SESSION): form = DateTimeWithNumRunsWithDagRunsForm(data=dt_nr_dr_data) form.execution_date.choices = dt_nr_dr_data["dr_choices"] - tis = session.scalars( - select(TaskInstance) - .where( + tis = ( + session.query(TaskInstance) + .filter( TaskInstance.dag_id == dag_id, TaskInstance.run_id == dag_run_id, TaskInstance.start_date.is_not(None), @@ -3613,10 +3602,10 @@ def gantt(self, dag_id: str, session: Session = NEW_SESSION): .order_by(TaskInstance.start_date) ) - ti_fails = select(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) + ti_fails = session.query(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) if dag.partial: - ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) - ti_fails = session.scalars(ti_fails) + ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + tasks = [] for ti in tis: if not dag.has_task(ti.task_id): @@ -3722,13 +3711,12 @@ def extra_links(self, *, session: Session = NEW_SESSION): if link_name is None: return {"url": None, "error": "Link name not passed"}, 400 - ti = session.scalar( - select(TaskInstance) + ti = ( + session.query(TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) .options(joinedload(TaskInstance.dag_run)) - .limit(1) + .first() ) - if not ti: return {"url": None, "error": "Task Instances not found"}, 404 try: @@ -3836,25 +3824,27 @@ def grid_data(self): base_date = dag.get_latest_execution_date() or timezone.utcnow() with create_session() as session: - query = select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date) + query = session.query(DagRun).filter( + DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date + ) - run_type = request.args.get("run_type") - if run_type: - query = query.where(DagRun.run_type == run_type) + run_type = request.args.get("run_type") + if run_type: + query = query.filter(DagRun.run_type == run_type) - run_state = request.args.get("run_state") - if run_state: - query = query.where(DagRun.state == run_state) + run_state = request.args.get("run_state") + if run_state: + query = query.filter(DagRun.state == run_state) - dag_runs = wwwutils.sorted_dag_runs( - query, ordering=dag.timetable.run_ordering, limit=num_runs, session=session - ) - encoded_runs = [wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs] - data = { - "groups": dag_to_grid(dag, dag_runs, session), - "dag_runs": encoded_runs, - "ordering": dag.timetable.run_ordering, - } + dag_runs = wwwutils.sorted_dag_runs(query, ordering=dag.timetable.run_ordering, limit=num_runs) + encoded_runs = [ + wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs + ] + data = { + "groups": dag_to_grid(dag, dag_runs, session), + "dag_runs": encoded_runs, + "ordering": dag.timetable.run_ordering, + } # avoid spaces to reduce payload size return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -3873,34 +3863,37 @@ def historical_metrics_data(self): end_date = _safe_parse_datetime(request.args.get("end_date")) with create_session() as session: # DagRuns - dag_runs_type = session.execute( - select(DagRun.run_type, func.count(DagRun.run_id)) - .where( + dag_runs_type = ( + session.query(DagRun.run_type, func.count(DagRun.run_id)) + .filter( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.run_type) - ).all() + .all() + ) - dag_run_states = session.execute( - select(DagRun.state, func.count(DagRun.run_id)) - .where( + dag_run_states = ( + session.query(DagRun.state, func.count(DagRun.run_id)) + .filter( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.state) - ).all() + .all() + ) # TaskInstances - task_instance_states = session.execute( - select(TaskInstance.state, func.count(TaskInstance.run_id)) + task_instance_states = ( + session.query(TaskInstance.state, func.count(TaskInstance.run_id)) .join(TaskInstance.dag_run) - .where( + .filter( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(TaskInstance.state) - ).all() + .all() + ) data = { "dag_run_types": { @@ -3934,32 +3927,28 @@ def next_run_datasets(self, dag_id): with create_session() as session: data = [ dict(info) - for info in session.execute( - select( - DatasetModel.id, - DatasetModel.uri, - func.max(DatasetEvent.timestamp).label("lastUpdate"), - ) - .join( - DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id - ) - .join( - DatasetDagRunQueue, - and_( - DatasetDagRunQueue.dataset_id == DatasetModel.id, - DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, - ), - isouter=True, - ) - .join( - DatasetEvent, - DatasetEvent.dataset_id == DatasetModel.id, - isouter=True, - ) - .where(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) - .group_by(DatasetModel.id, DatasetModel.uri) - .order_by(DatasetModel.uri) + for info in session.query( + DatasetModel.id, + DatasetModel.uri, + func.max(DatasetEvent.timestamp).label("lastUpdate"), ) + .join(DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id) + .join( + DatasetDagRunQueue, + and_( + DatasetDagRunQueue.dataset_id == DatasetModel.id, + DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, + ), + isouter=True, + ) + .join( + DatasetEvent, + DatasetEvent.dataset_id == DatasetModel.id, + isouter=True, + ) + .filter(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) + .group_by(DatasetModel.id, DatasetModel.uri) + .order_by(DatasetModel.uri) ] return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -4057,12 +4046,12 @@ def datasets_summary(self): if session.bind.dialect.name == "postgresql": order_by = (order_by[0].nulls_first(), *order_by[1:]) - count_query = select(func.count(DatasetModel.id)) + count_query = session.query(func.count(DatasetModel.id)) has_event_filters = bool(updated_before or updated_after) query = ( - select( + session.query( DatasetModel.id, DatasetModel.uri, func.max(DatasetEvent.timestamp).label("last_dataset_update"), @@ -4087,12 +4076,11 @@ def datasets_summary(self): if updated_before: filters.append(DatasetEvent.timestamp <= updated_before) - query = query.where(*filters).offset(offset).limit(limit) - count_query = count_query.where(*filters) + query = query.filter(*filters).offset(offset).limit(limit) + count_query = count_query.filter(*filters) - query = session.execute(query) datasets = [dict(dataset) for dataset in query] - data = {"datasets": datasets, "total_entries": session.scalar(count_query)} + data = {"datasets": datasets, "total_entries": count_query.scalar()} return ( htmlsafe_json_dumps(data, separators=(",", ":"), cls=utils_json.WebEncoder), @@ -4138,20 +4126,20 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): included_events_raw = conf.get("webserver", "audit_view_included_events", fallback=None) excluded_events_raw = conf.get("webserver", "audit_view_excluded_events", fallback=None) - query = select(Log).where(Log.dag_id == dag_id) + query = session.query(Log).filter(Log.dag_id == dag_id) if included_events_raw: included_events = {event.strip() for event in included_events_raw.split(",")} - query = query.where(Log.event.in_(included_events)) + query = query.filter(Log.event.in_(included_events)) elif excluded_events_raw: excluded_events = {event.strip() for event in excluded_events_raw.split(",")} - query = query.where(Log.event.notin_(excluded_events)) + query = query.filter(Log.event.notin_(excluded_events)) current_page = request.args.get("page", default=0, type=int) arg_sorting_key = request.args.get("sorting_key", "dttm") arg_sorting_direction = request.args.get("sorting_direction", default="desc") logs_per_page = PAGE_SIZE - audit_logs_count = session.scalar(select(func.count()).select_from(query)) + audit_logs_count = query.count() num_of_pages = int(math.ceil(audit_logs_count / float(logs_per_page))) start = current_page * logs_per_page @@ -4163,7 +4151,7 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): sort_column = sort_column.desc() query = query.order_by(sort_column) - dag_audit_logs = session.scalars(query.offset(start).limit(logs_per_page)).all() + dag_audit_logs = query.offset(start).limit(logs_per_page).all() return self.render_template( "airflow/dag_audit_log.html", dag=dag, @@ -4284,7 +4272,7 @@ def apply(self, query, func): if get_airflow_app().appbuilder.sm.has_all_dags_access(g.user): return query filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - return query.where(self.model.dag_id.in_(filter_dag_ids)) + return query.filter(self.model.dag_id.in_(filter_dag_ids)) class AirflowModelView(ModelView): @@ -4729,11 +4717,9 @@ def action_mulduplicate(self, connections, session: Session = NEW_SESSION): potential_connection_ids = [f"{base_conn_id}_copy{i}" for i in range(1, 11)] - query = session.scalars( - select(Connection.conn_id).where(Connection.conn_id.in_(potential_connection_ids)) - ) + query = session.query(Connection.conn_id).filter(Connection.conn_id.in_(potential_connection_ids)) - found_conn_id_set = {conn_id for conn_id in query} + found_conn_id_set = {conn_id for conn_id, in query} possible_conn_id_iter = ( connection_id @@ -5403,7 +5389,7 @@ def _set_dag_runs_to_active_state(self, drs: list[DagRun], state: str, session: """This routine only supports Running and Queued state.""" try: count = 0 - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): + for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): count += 1 if state == State.RUNNING: dr.start_date = timezone.utcnow() @@ -5429,7 +5415,7 @@ def action_set_failed(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): + for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): count += 1 altered_tis += set_dag_run_state_to_failed( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5457,7 +5443,7 @@ def action_set_success(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): + for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): count += 1 altered_tis += set_dag_run_state_to_success( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5481,7 +5467,7 @@ def action_clear(self, drs: list[DagRun], session: Session = NEW_SESSION): count = 0 cleared_ti_count = 0 dag_to_tis: dict[DAG, list[TaskInstance]] = {} - for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): + for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): count += 1 dag = get_airflow_app().dag_bag.get_dag(dr.dag_id) tis_to_clear = dag_to_tis.setdefault(dag, []) @@ -5897,37 +5883,35 @@ def autocomplete(self, session: Session = NEW_SESSION): return flask.json.jsonify([]) # Provide suggestions of dag_ids and owners - dag_ids_query = select( + dag_ids_query = session.query( sqla.literal("dag").label("type"), DagModel.dag_id.label("name"), - ).where(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) + ).filter(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) owners_query = ( - select( + session.query( sqla.literal("owner").label("type"), DagModel.owners.label("name"), ) .distinct() - .where(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) + .filter(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) ) # Hide DAGs if not showing status: "all" status = flask_session.get(FILTER_STATUS_COOKIE) if status == "active": - dag_ids_query = dag_ids_query.where(~DagModel.is_paused) - owners_query = owners_query.where(~DagModel.is_paused) + dag_ids_query = dag_ids_query.filter(~DagModel.is_paused) + owners_query = owners_query.filter(~DagModel.is_paused) elif status == "paused": - dag_ids_query = dag_ids_query.where(DagModel.is_paused) - owners_query = owners_query.where(DagModel.is_paused) + dag_ids_query = dag_ids_query.filter(DagModel.is_paused) + owners_query = owners_query.filter(DagModel.is_paused) filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - dag_ids_query = dag_ids_query.where(DagModel.dag_id.in_(filter_dag_ids)) - owners_query = owners_query.where(DagModel.dag_id.in_(filter_dag_ids)) - payload = [ - row._asdict() - for row in session.execute(dag_ids_query.union(owners_query).order_by("name").limit(10)) - ] + dag_ids_query = dag_ids_query.filter(DagModel.dag_id.in_(filter_dag_ids)) + owners_query = owners_query.filter(DagModel.dag_id.in_(filter_dag_ids)) + + payload = [row._asdict() for row in dag_ids_query.union(owners_query).order_by("name").limit(10)] return flask.json.jsonify(payload) From e4c3ecf8ceaefa17525b495e4bcb5b2f41309603 Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Tue, 4 Jul 2023 19:29:29 +0530 Subject: [PATCH 035/533] Disable default allowing the testing of connections in UI, API and CLI (#32052) * Disable allowing by default testing of connnections in UI Users can enable test connection functionaility in UI with caution by setting the `enable_test_connection` key to `True` in the `[webserver]` section of airflow.cfg or by setting the environment variable `AIRFLOW__WEBSERVER__ENABLE_TEST_CONNECTION` to `True`. --- .../endpoints/connection_endpoint.py | 10 ++++++++- airflow/cli/commands/connection_command.py | 7 +++++++ airflow/config_templates/config.yml | 19 +++++++++++++---- airflow/config_templates/default_airflow.cfg | 11 ++++++++++ airflow/www/extensions/init_jinja_globals.py | 1 + airflow/www/static/js/connection_form.js | 21 +++++++++++++++++++ .../www/templates/airflow/conn_create.html | 1 + airflow/www/templates/airflow/conn_edit.html | 1 + newsfragments/32052.significant.rst | 17 +++++++++++++++ .../endpoints/test_connection_endpoint.py | 16 ++++++++++++++ tests/cli/commands/test_connection_command.py | 14 ++++++++++++- tests/www/views/test_views.py | 1 + 12 files changed, 113 insertions(+), 6 deletions(-) create mode 100644 newsfragments/32052.significant.rst diff --git a/airflow/api_connexion/endpoints/connection_endpoint.py b/airflow/api_connexion/endpoints/connection_endpoint.py index 737ee54d6c73..cfb185d44b4c 100644 --- a/airflow/api_connexion/endpoints/connection_endpoint.py +++ b/airflow/api_connexion/endpoints/connection_endpoint.py @@ -20,7 +20,7 @@ from http import HTTPStatus from connexion import NoContent -from flask import request +from flask import Response, request from marshmallow import ValidationError from sqlalchemy import func, select from sqlalchemy.orm import Session @@ -36,6 +36,7 @@ connection_test_schema, ) from airflow.api_connexion.types import APIResponse, UpdateMask +from airflow.configuration import conf from airflow.models import Connection from airflow.secrets.environment_variables import CONN_ENV_PREFIX from airflow.security import permissions @@ -180,6 +181,13 @@ def test_connection() -> APIResponse: env var, as some hook classes tries to find out the conn from their __init__ method & errors out if not found. It also deletes the conn id env variable after the test. """ + if conf.get("core", "test_connection", fallback="Disabled").lower().strip() != "enabled": + return Response( + "Testing connections is disabled in Airflow configuration. Contact your deployment admin to " + "enable it.", + 403, + ) + body = request.json transient_conn_id = get_random_string() conn_env_var = f"{CONN_ENV_PREFIX}{transient_conn_id.upper()}" diff --git a/airflow/cli/commands/connection_command.py b/airflow/cli/commands/connection_command.py index 6d137bc89685..63888c0ec96e 100644 --- a/airflow/cli/commands/connection_command.py +++ b/airflow/cli/commands/connection_command.py @@ -30,6 +30,7 @@ from airflow.cli.simple_table import AirflowConsole from airflow.compat.functools import cache +from airflow.configuration import conf from airflow.exceptions import AirflowNotFoundException from airflow.hooks.base import BaseHook from airflow.models import Connection @@ -343,6 +344,12 @@ def _import_helper(file_path: str, overwrite: bool) -> None: def connections_test(args) -> None: """Test an Airflow connection.""" console = AirflowConsole() + if conf.get("core", "test_connection", fallback="Disabled").lower().strip() != "enabled": + console.print( + "[bold yellow]\nTesting connections is disabled in Airflow configuration. " + "Contact your deployment admin to enable it.\n" + ) + raise SystemExit(1) print(f"Retrieving connection: {args.conn_id!r}") try: diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 88aa055a3901..1ad8705a7047 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -450,6 +450,21 @@ core: type: string default: ~ example: 'http://localhost:8080' + test_connection: + description: | + The ability to allow testing connections across Airflow UI, API and CLI. + Supported options: Disabled, Enabled, Hidden. Default: Disabled + Disabled - Disables the test connection functionality and disables the Test Connection button in UI. + Enabled - Enables the test connection functionality and shows the Test Connection button in UI. + Hidden - Disables the test connection functionality and hides the Test Connection button in UI. + Before setting this to Enabled, make sure that you review the users who are able to add/edit + connections and ensure they are trusted. Connection testing can be done maliciously leading to + undesired and insecure outcomes. For more information on capabilities of users, see the documentation: + https://airflow.apache.org/docs/apache-airflow/stable/security/index.html#capabilities-of-authenticated-ui-users + version_added: 2.7.0 + type: string + example: ~ + default: "Disabled" database: description: ~ options: @@ -891,8 +906,6 @@ logging: type: boolean example: ~ default: "False" - - metrics: description: | StatsD (https://github.com/etsy/statsd) integration settings. @@ -1801,7 +1814,6 @@ webserver: type: string example: "sha256" default: "md5" - email: description: | Configuration email backend and whether to @@ -1859,7 +1871,6 @@ email: type: string example: "Airflow " default: ~ - smtp: description: | If you want airflow to send emails on retries, failure, and you want to use diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index c28155ac2c40..0cc99a3f4e5f 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -257,6 +257,17 @@ database_access_isolation = False # Example: internal_api_url = http://localhost:8080 # internal_api_url = +# The ability to allow testing connections across Airflow UI, API and CLI. +# Supported options: Disabled, Enabled, Hidden. Default: Disabled +# Disabled - Disables the test connection functionality and disables the Test Connection button in UI. +# Enabled - Enables the test connection functionality and shows the Test Connection button in UI. +# Hidden - Disables the test connection functionality and hides the Test Connection button in UI. +# Before setting this to Enabled, make sure that you review the users who are able to add/edit +# connections and ensure they are trusted. Connection testing can be done maliciously leading to +# undesired and insecure outcomes. For more information on capabilities of users, see the documentation: +# https://airflow.apache.org/docs/apache-airflow/stable/security/index.html#capabilities-of-authenticated-ui-users +test_connection = Disabled + [database] # Path to the ``alembic.ini`` file. You can either provide the file path relative # to the Airflow home directory or the absolute path if it is located elsewhere. diff --git a/airflow/www/extensions/init_jinja_globals.py b/airflow/www/extensions/init_jinja_globals.py index 2d5778af4b4f..0674a8e4a3ea 100644 --- a/airflow/www/extensions/init_jinja_globals.py +++ b/airflow/www/extensions/init_jinja_globals.py @@ -68,6 +68,7 @@ def prepare_jinja_globals(): "git_version": git_version, "k8s_or_k8scelery_executor": IS_K8S_OR_K8SCELERY_EXECUTOR, "rest_api_enabled": False, + "config_test_connection": conf.get("core", "test_connection", fallback="Disabled"), } backends = conf.get("api", "auth_backends") diff --git a/airflow/www/static/js/connection_form.js b/airflow/www/static/js/connection_form.js index 41df55dc07bb..453be58411a9 100644 --- a/airflow/www/static/js/connection_form.js +++ b/airflow/www/static/js/connection_form.js @@ -23,6 +23,9 @@ /* global document, DOMParser, $, CodeMirror */ import { getMetaValue } from "./utils"; +const configTestConnection = getMetaValue("config_test_connection") + .toLowerCase() + .trim(); const restApiEnabled = getMetaValue("rest_api_enabled") === "True"; const connectionTestUrl = getMetaValue("test_url"); @@ -126,6 +129,24 @@ function applyFieldBehaviours(connection) { */ function handleTestConnection(connectionType, testableConnections) { const testButton = document.getElementById("test-connection"); + + if (configTestConnection === "hidden") { + // If test connection is hidden in config, hide button and return. + $(testButton).hide(); + return; + } + if (configTestConnection === "disabled") { + // If test connection is not enabled in config, disable button and display toolip + // alerting the user. + $(testButton) + .prop("disabled", true) + .attr( + "title", + "Testing connections is disabled in Airflow configuration. Contact your deployment admin to enable it." + ); + return; + } + const testConnEnabled = testableConnections.includes(connectionType); if (testConnEnabled) { diff --git a/airflow/www/templates/airflow/conn_create.html b/airflow/www/templates/airflow/conn_create.html index 34c2f32dedb8..ac92b967f7e3 100644 --- a/airflow/www/templates/airflow/conn_create.html +++ b/airflow/www/templates/airflow/conn_create.html @@ -21,6 +21,7 @@ {% block head_css %} {{ super() }} + diff --git a/airflow/www/templates/airflow/conn_edit.html b/airflow/www/templates/airflow/conn_edit.html index e963b8ac0cb5..653b0dd3ce07 100644 --- a/airflow/www/templates/airflow/conn_edit.html +++ b/airflow/www/templates/airflow/conn_edit.html @@ -21,6 +21,7 @@ {% block head_css %} {{ super() }} + diff --git a/newsfragments/32052.significant.rst b/newsfragments/32052.significant.rst new file mode 100644 index 000000000000..d4ec9a934f9d --- /dev/null +++ b/newsfragments/32052.significant.rst @@ -0,0 +1,17 @@ +Disable default allowing the testing of connections in UI, API and CLI + +The test connection functionality is disabled by default across Airflow UI, +API and CLI. The availability of the functionality can be controlled by the +``test_connection`` flag in the ``core`` section of the Airflow +configuration (``airflow.cfg``). It can also be controlled by the +environment variable ``AIRFLOW__CORE__TEST_CONNECTION``. +The following values are accepted for this config param: +1. ``Disabled``: Disables the test connection functionality and +disables(greys out) the Test Connection button in the UI. +This is also the default value set in the Airflow configuration. +2. ``Enabled``: Enables the test connection functionality and +activates the Test Connection button in the UI. +3. ``Hidden``: Disables the test connection functionality and +hides the Test Connection button in UI. +For more information on capabilities of users, see the documentation: +https://airflow.apache.org/docs/apache-airflow/stable/security/index.html#capabilities-of-authenticated-ui-users diff --git a/tests/api_connexion/endpoints/test_connection_endpoint.py b/tests/api_connexion/endpoints/test_connection_endpoint.py index bce7150f8bda..de0b7aad973f 100644 --- a/tests/api_connexion/endpoints/test_connection_endpoint.py +++ b/tests/api_connexion/endpoints/test_connection_endpoint.py @@ -16,6 +16,9 @@ # under the License. from __future__ import annotations +import os +from unittest import mock + import pytest from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP @@ -603,6 +606,7 @@ def test_should_raises_401_unauthenticated(self): class TestConnection(TestConnectionEndpoint): + @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) def test_should_respond_200(self): payload = {"connection_id": "test-connection-id", "conn_type": "sqlite"} response = self.client.post( @@ -614,6 +618,7 @@ def test_should_respond_200(self): "message": "Connection successfully tested", } + @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) def test_post_should_respond_400_for_invalid_payload(self): payload = { "connection_id": "test-connection-id", @@ -635,3 +640,14 @@ def test_should_raises_401_unauthenticated(self): ) assert_401(response) + + def test_should_respond_403_by_default(self): + payload = {"connection_id": "test-connection-id", "conn_type": "sqlite"} + response = self.client.post( + "/api/v1/connections/test", json=payload, environ_overrides={"REMOTE_USER": "test"} + ) + assert response.status_code == 403 + assert response.text == ( + "Testing connections is disabled in Airflow configuration. " + "Contact your deployment admin to enable it." + ) diff --git a/tests/cli/commands/test_connection_command.py b/tests/cli/commands/test_connection_command.py index 42f6fc8cc204..12b3de6170c0 100644 --- a/tests/cli/commands/test_connection_command.py +++ b/tests/cli/commands/test_connection_command.py @@ -18,6 +18,7 @@ import io import json +import os import re import shlex import warnings @@ -938,6 +939,7 @@ class TestCliTestConnections: def setup_class(self): clear_db_connections() + @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) @mock.patch("airflow.providers.http.hooks.http.HttpHook.test_connection") def test_cli_connections_test_success(self, mock_test_conn): """Check that successful connection test result is displayed properly.""" @@ -948,6 +950,7 @@ def test_cli_connections_test_success(self, mock_test_conn): assert "Connection success!" in stdout.getvalue() + @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) @mock.patch("airflow.providers.http.hooks.http.HttpHook.test_connection") def test_cli_connections_test_fail(self, mock_test_conn): """Check that failed connection test result is displayed properly.""" @@ -958,9 +961,18 @@ def test_cli_connections_test_fail(self, mock_test_conn): assert "Connection failed!\nFailed.\n\n" in stdout.getvalue() + @mock.patch.dict(os.environ, {"AIRFLOW__CORE__TEST_CONNECTION": "Enabled"}) def test_cli_connections_test_missing_conn(self): """Check a connection test on a non-existent connection raises a "Connection not found" message.""" with redirect_stdout(io.StringIO()) as stdout, pytest.raises(SystemExit): connection_command.connections_test(self.parser.parse_args(["connections", "test", "missing"])) + assert "Connection not found.\n\n" in stdout.getvalue() - assert "Connection not found.\n\n" in stdout.getvalue() + def test_cli_connections_test_disabled_by_default(self): + """Check that test connection functionality is disabled by default.""" + with redirect_stdout(io.StringIO()) as stdout, pytest.raises(SystemExit): + connection_command.connections_test(self.parser.parse_args(["connections", "test", "missing"])) + assert ( + "Testing connections is disabled in Airflow configuration. Contact your deployment admin to " + "enable it.\n\n" + ) in stdout.getvalue() diff --git a/tests/www/views/test_views.py b/tests/www/views/test_views.py index e266b9b8c1a1..755df3b7aa7f 100644 --- a/tests/www/views/test_views.py +++ b/tests/www/views/test_views.py @@ -84,6 +84,7 @@ def test_redoc_should_render_template(capture_templates, admin_client): assert len(templates) == 1 assert templates[0].name == "airflow/redoc.html" assert templates[0].local_context == { + "config_test_connection": "Disabled", "openapi_spec_url": "/api/v1/openapi.yaml", "rest_api_enabled": True, "get_docs_url": get_docs_url, From d1625188ff20b4b6a3bf8f41792c4d16b79c8bf1 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Tue, 4 Jul 2023 16:25:19 +0200 Subject: [PATCH 036/533] Remove outdated comment from chart values in the workers KEDA conf section (#32300) Signed-off-by: Hussein Awala --- chart/values.schema.json | 2 +- chart/values.yaml | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/chart/values.schema.json b/chart/values.schema.json index 395506701bec..80e25b9b10a6 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -1381,7 +1381,7 @@ "additionalProperties": false, "properties": { "enabled": { - "description": "Allow KEDA autoscaling. `Persistence.enabled` must be set to false to use KEDA.", + "description": "Allow KEDA autoscaling.", "type": "boolean", "default": false }, diff --git a/chart/values.yaml b/chart/values.yaml index b1879f0d4466..3beeb3765cfa 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -510,7 +510,6 @@ workers: annotations: {} # Allow KEDA autoscaling. - # Persistence.enabled must be set to false to use KEDA. keda: enabled: false namespaceLabels: {} From 575bf2f04089b7c99a8ee30637f1d88492ef4742 Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Tue, 4 Jul 2023 20:54:11 +0530 Subject: [PATCH 037/533] Bug fix GCSToS3Operator: avoid `ValueError` when `replace=False` with files already in S3 (#32322) --- .../amazon/aws/transfers/gcs_to_s3.py | 5 ++ .../amazon/aws/transfers/test_gcs_to_s3.py | 69 ++++++++++++++----- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py index 2213de2b60c1..d57de7e11efb 100644 --- a/airflow/providers/amazon/aws/transfers/gcs_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/gcs_to_s3.py @@ -161,6 +161,11 @@ def execute(self, context: Context) -> list[str]: # and only keep those files which are present in # Google Cloud Storage and not in S3 bucket_name, prefix = S3Hook.parse_s3_url(self.dest_s3_key) + # if prefix is empty, do not add "/" at end since it would + # filter all the objects (return empty list) instead of empty + # prefix returning all the objects + if prefix: + prefix = prefix if prefix.endswith("/") else f"{prefix}/" # look for the bucket and the prefix to avoid look into # parent directories/keys existing_files = s3_hook.list_keys(bucket_name, prefix=prefix) diff --git a/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py b/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py index a7a0b2e4305a..5e64f167ba45 100644 --- a/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py +++ b/tests/providers/amazon/aws/transfers/test_gcs_to_s3.py @@ -33,6 +33,7 @@ S3_BUCKET = "s3://bucket/" MOCK_FILES = ["TEST1.csv", "TEST2.csv", "TEST3.csv"] S3_ACL_POLICY = "private-read" +deprecated_call_match = "Usage of 'delimiter' is deprecated, please use 'match_glob' instead" def _create_test_bucket(): @@ -47,8 +48,6 @@ def _create_test_bucket(): @mock_s3 class TestGCSToS3Operator: - - # Test0: match_glob @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute__match_glob(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES @@ -73,7 +72,6 @@ def test_execute__match_glob(self, mock_hook): bucket_name=GCS_BUCKET, delimiter=None, match_glob=f"**/*{DELIMITER}", prefix=PREFIX ) - # Test1: incremental behaviour (just some files missing) @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute_incremental(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES @@ -81,7 +79,7 @@ def test_execute_incremental(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -100,15 +98,17 @@ def test_execute_incremental(self, mock_hook): assert sorted(MOCK_FILES[1:]) == sorted(uploaded_files) assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) - # Test2: All the files are already in origin and destination without replace @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute_without_replace(self, mock_hook): + """ + Tests scenario where all the files are already in origin and destination without replace + """ mock_hook.return_value.list.return_value = MOCK_FILES with NamedTemporaryFile() as f: gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -128,15 +128,53 @@ def test_execute_without_replace(self, mock_hook): assert [] == uploaded_files assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) - # Test3: There are no files in destination bucket + @pytest.mark.parametrize( + argnames="dest_s3_url", + argvalues=[f"{S3_BUCKET}/test/", f"{S3_BUCKET}/test"], + ) + @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") + def test_execute_without_replace_with_folder_structure(self, mock_hook, dest_s3_url): + mock_files_gcs = [f"test{idx}/{mock_file}" for idx, mock_file in enumerate(MOCK_FILES)] + mock_files_s3 = [f"test/test{idx}/{mock_file}" for idx, mock_file in enumerate(MOCK_FILES)] + mock_hook.return_value.list.return_value = mock_files_gcs + + hook, bucket = _create_test_bucket() + for mock_file_s3 in mock_files_s3: + bucket.put_object(Key=mock_file_s3, Body=b"testing") + + with NamedTemporaryFile() as f: + gcs_provide_file = mock_hook.return_value.provide_file + gcs_provide_file.return_value.__enter__.return_value.name = f.name + + with pytest.deprecated_call(match=deprecated_call_match): + operator = GCSToS3Operator( + task_id=TASK_ID, + bucket=GCS_BUCKET, + prefix=PREFIX, + delimiter=DELIMITER, + dest_aws_conn_id="aws_default", + dest_s3_key=dest_s3_url, + replace=False, + ) + + # we expect nothing to be uploaded + # and all the MOCK_FILES to be present at the S3 bucket + uploaded_files = operator.execute(None) + + assert [] == uploaded_files + assert sorted(mock_files_s3) == sorted(hook.list_keys("bucket", prefix="test/")) + @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute(self, mock_hook): + """ + Tests the scenario where there are no files in destination bucket + """ mock_hook.return_value.list.return_value = MOCK_FILES with NamedTemporaryFile() as f: gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -154,7 +192,6 @@ def test_execute(self, mock_hook): assert sorted(MOCK_FILES) == sorted(uploaded_files) assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) - # Test4: Destination and Origin are in sync but replace all files in destination @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute_with_replace(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES @@ -162,7 +199,7 @@ def test_execute_with_replace(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -182,7 +219,6 @@ def test_execute_with_replace(self, mock_hook): assert sorted(MOCK_FILES) == sorted(uploaded_files) assert sorted(MOCK_FILES) == sorted(hook.list_keys("bucket", delimiter="/")) - # Test5: Incremental sync with replace @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") def test_execute_incremental_with_replace(self, mock_hook): mock_hook.return_value.list.return_value = MOCK_FILES @@ -190,7 +226,7 @@ def test_execute_incremental_with_replace(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -218,7 +254,7 @@ def test_execute_should_handle_with_default_dest_s3_extra_args(self, s3_mock_hoo s3_mock_hook.return_value = mock.Mock() s3_mock_hook.parse_s3_url.return_value = mock.Mock() - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -241,7 +277,7 @@ def test_execute_should_pass_dest_s3_extra_args_to_s3_hook(self, s3_mock_hook, m s3_mock_hook.return_value = mock.Mock() s3_mock_hook.parse_s3_url.return_value = mock.Mock() - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -259,7 +295,6 @@ def test_execute_should_pass_dest_s3_extra_args_to_s3_hook(self, s3_mock_hook, m aws_conn_id="aws_default", extra_args={"ContentLanguage": "value"}, verify=None ) - # Test6: s3_acl_policy parameter is set @mock.patch("airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSHook") @mock.patch("airflow.providers.amazon.aws.hooks.s3.S3Hook.load_file") def test_execute_with_s3_acl_policy(self, mock_load_file, mock_gcs_hook): @@ -268,7 +303,7 @@ def test_execute_with_s3_acl_policy(self, mock_load_file, mock_gcs_hook): gcs_provide_file = mock_gcs_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, @@ -293,7 +328,7 @@ def test_execute_without_keep_director_structure(self, mock_hook): gcs_provide_file = mock_hook.return_value.provide_file gcs_provide_file.return_value.__enter__.return_value.name = f.name - with pytest.deprecated_call(): + with pytest.deprecated_call(match=deprecated_call_match): operator = GCSToS3Operator( task_id=TASK_ID, bucket=GCS_BUCKET, From 836bdb29c0c455a5e9f84957947452e50c4d477e Mon Sep 17 00:00:00 2001 From: Joffrey Bienvenu Date: Tue, 4 Jul 2023 17:47:43 +0200 Subject: [PATCH 038/533] feat: Use 'use_ssl' parameter instead of 'ssl' + Use attributes of the patched function (#32305) --- tests/providers/apache/impala/hooks/test_impala.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/providers/apache/impala/hooks/test_impala.py b/tests/providers/apache/impala/hooks/test_impala.py index 829a543e0822..a3e1f119360d 100644 --- a/tests/providers/apache/impala/hooks/test_impala.py +++ b/tests/providers/apache/impala/hooks/test_impala.py @@ -35,17 +35,22 @@ def impala_hook_fixture() -> ImpalaHook: return hook -@patch("airflow.providers.apache.impala.hooks.impala.connect") +@patch("airflow.providers.apache.impala.hooks.impala.connect", autospec=True) def test_get_conn(mock_connect): hook = ImpalaHook() hook.get_connection = MagicMock( return_value=Connection( - login="login", password="password", host="host", port=21050, schema="test", extra={"ssl": True} + login="login", + password="password", + host="host", + port=21050, + schema="test", + extra={"use_ssl": True}, ) ) hook.get_conn() mock_connect.assert_called_once_with( - host="host", port=21050, user="login", password="password", database="test", ssl=True + host="host", port=21050, user="login", password="password", database="test", use_ssl=True ) From 65fad4affc24b33c4499ad0fbcdfff535fbae3bf Mon Sep 17 00:00:00 2001 From: JDarDagran Date: Tue, 4 Jul 2023 17:48:41 +0200 Subject: [PATCH 039/533] Change default schema behaviour in SQLParser. (#32347) Signed-off-by: Jakub Dardzinski --- airflow/providers/openlineage/sqlparser.py | 3 +- airflow/providers/openlineage/utils/sql.py | 9 +++-- tests/providers/openlineage/utils/test_sql.py | 4 ++ .../openlineage/utils/test_sqlparser.py | 39 ++++++++++++------- 4 files changed, 36 insertions(+), 19 deletions(-) diff --git a/airflow/providers/openlineage/sqlparser.py b/airflow/providers/openlineage/sqlparser.py index 657428549eaa..ed3e92e58b5a 100644 --- a/airflow/providers/openlineage/sqlparser.py +++ b/airflow/providers/openlineage/sqlparser.py @@ -103,7 +103,7 @@ def __init__(self, dialect: str | None = None, default_schema: str | None = None def parse(self, sql: list[str] | str) -> SqlMeta | None: """Parse a single or a list of SQL statements.""" - return parse(sql=sql, dialect=self.dialect, default_schema=self.default_schema) + return parse(sql=sql, dialect=self.dialect) def parse_table_schemas( self, @@ -126,6 +126,7 @@ def parse_table_schemas( return get_table_schemas( hook, namespace, + self.default_schema, database or database_info.database, self.create_information_schema_query(tables=inputs, **database_kwargs) if inputs else None, self.create_information_schema_query(tables=outputs, **database_kwargs) if outputs else None, diff --git a/airflow/providers/openlineage/utils/sql.py b/airflow/providers/openlineage/utils/sql.py index 317e46e44255..fe43a25bae67 100644 --- a/airflow/providers/openlineage/utils/sql.py +++ b/airflow/providers/openlineage/utils/sql.py @@ -63,12 +63,12 @@ class TableSchema: database: str | None fields: list[SchemaField] - def to_dataset(self, namespace: str, database: str | None = None) -> Dataset: + def to_dataset(self, namespace: str, database: str | None = None, schema: str | None = None) -> Dataset: # Prefix the table name with database and schema name using # the format: {database_name}.{table_schema}.{table_name}. name = ".".join( part - for part in [self.database if self.database else database, self.schema, self.table] + for part in [self.database or database, self.schema or schema, self.table] if part is not None ) return Dataset( @@ -81,6 +81,7 @@ def to_dataset(self, namespace: str, database: str | None = None) -> Dataset: def get_table_schemas( hook: BaseHook, namespace: str, + schema: str | None, database: str | None, in_query: str | None, out_query: str | None, @@ -97,12 +98,12 @@ def get_table_schemas( with closing(hook.get_conn()) as conn, closing(conn.cursor()) as cursor: if in_query: cursor.execute(in_query) - in_datasets = [x.to_dataset(namespace, database) for x in parse_query_result(cursor)] + in_datasets = [x.to_dataset(namespace, database, schema) for x in parse_query_result(cursor)] else: in_datasets = [] if out_query: cursor.execute(out_query) - out_datasets = [x.to_dataset(namespace, database) for x in parse_query_result(cursor)] + out_datasets = [x.to_dataset(namespace, database, schema) for x in parse_query_result(cursor)] else: out_datasets = [] return in_datasets, out_datasets diff --git a/tests/providers/openlineage/utils/test_sql.py b/tests/providers/openlineage/utils/test_sql.py index be929a1ad643..a82ab36bda26 100644 --- a/tests/providers/openlineage/utils/test_sql.py +++ b/tests/providers/openlineage/utils/test_sql.py @@ -77,6 +77,7 @@ def test_get_table_schemas(): hook=hook, namespace="bigquery", database=DB_NAME, + schema=DB_SCHEMA_NAME, in_query="fake_sql", out_query="another_fake_sql", ) @@ -139,6 +140,7 @@ def test_get_table_schemas_with_mixed_databases(): hook=hook, namespace="bigquery", database=DB_NAME, + schema=DB_SCHEMA_NAME, in_query="fake_sql", out_query="another_fake_sql", ) @@ -179,6 +181,7 @@ def test_get_table_schemas_with_mixed_schemas(): hook=hook, namespace="bigquery", database=DB_NAME, + schema=DB_SCHEMA_NAME, in_query="fake_sql", out_query="another_fake_sql", ) @@ -237,6 +240,7 @@ def test_get_table_schemas_with_other_database(): hook=hook, namespace="bigquery", database=DB_NAME, + schema=DB_SCHEMA_NAME, in_query="fake_sql", out_query="another_fake_sql", ) diff --git a/tests/providers/openlineage/utils/test_sqlparser.py b/tests/providers/openlineage/utils/test_sqlparser.py index 6f11a7ad9438..31611c7c11fe 100644 --- a/tests/providers/openlineage/utils/test_sqlparser.py +++ b/tests/providers/openlineage/utils/test_sqlparser.py @@ -19,6 +19,7 @@ from unittest import mock from unittest.mock import MagicMock +import pytest from openlineage.client.facet import SchemaDatasetFacet, SchemaField, SqlJobFacet from openlineage.client.run import Dataset from openlineage.common.sql import DbTableMeta @@ -155,19 +156,20 @@ def test_parse_table_schemas(self): database_info=db_info, ) + @pytest.mark.parametrize("parser_returns_schema", [True, False]) @mock.patch("airflow.providers.openlineage.sqlparser.SQLParser.parse") - def test_generate_openlineage_metadata_from_sql(self, mock_parse): - parser = SQLParser() + def test_generate_openlineage_metadata_from_sql(self, mock_parse, parser_returns_schema): + parser = SQLParser(default_schema="ANOTHER_SCHEMA") db_info = DatabaseInfo(scheme="myscheme", authority="host:port") hook = MagicMock() - rows = lambda name: [ - (DB_SCHEMA_NAME, name, "ID", 1, "int4"), - (DB_SCHEMA_NAME, name, "AMOUNT_OFF", 2, "int4"), - (DB_SCHEMA_NAME, name, "CUSTOMER_EMAIL", 3, "varchar"), - (DB_SCHEMA_NAME, name, "STARTS_ON", 4, "timestamp"), - (DB_SCHEMA_NAME, name, "ENDS_ON", 5, "timestamp"), + rows = lambda schema, table: [ + (schema, table, "ID", 1, "int4"), + (schema, table, "AMOUNT_OFF", 2, "int4"), + (schema, table, "CUSTOMER_EMAIL", 3, "varchar"), + (schema, table, "STARTS_ON", 4, "timestamp"), + (schema, table, "ENDS_ON", 5, "timestamp"), ] sql = """CREATE TABLE table_out ( @@ -182,13 +184,17 @@ def test_generate_openlineage_metadata_from_sql(self, mock_parse): """ hook.get_conn.return_value.cursor.return_value.fetchall.side_effect = [ - rows("TABLE_IN"), - rows("TABLE_OUT"), + rows(DB_SCHEMA_NAME if parser_returns_schema else None, "TABLE_IN"), + rows(DB_SCHEMA_NAME if parser_returns_schema else None, "TABLE_OUT"), ] mock_sql_meta = MagicMock() - mock_sql_meta.in_tables = [DbTableMeta("PUBLIC.TABLE_IN")] - mock_sql_meta.out_tables = [DbTableMeta("PUBLIC.TABLE_OUT")] + if parser_returns_schema: + mock_sql_meta.in_tables = [DbTableMeta("PUBLIC.TABLE_IN")] + mock_sql_meta.out_tables = [DbTableMeta("PUBLIC.TABLE_OUT")] + else: + mock_sql_meta.in_tables = [DbTableMeta("TABLE_IN")] + mock_sql_meta.out_tables = [DbTableMeta("TABLE_OUT")] mock_sql_meta.errors = [] mock_parse.return_value = mock_sql_meta @@ -201,15 +207,20 @@ def test_generate_openlineage_metadata_from_sql(self, mock_parse): ENDS_ON timestamp )""" + expected_schema = "PUBLIC" if parser_returns_schema else "ANOTHER_SCHEMA" expected = OperatorLineage( inputs=[ Dataset( - namespace="myscheme://host:port", name="PUBLIC.TABLE_IN", facets={"schema": SCHEMA_FACET} + namespace="myscheme://host:port", + name=f"{expected_schema}.TABLE_IN", + facets={"schema": SCHEMA_FACET}, ) ], outputs=[ Dataset( - namespace="myscheme://host:port", name="PUBLIC.TABLE_OUT", facets={"schema": SCHEMA_FACET} + namespace="myscheme://host:port", + name=f"{expected_schema}.TABLE_OUT", + facets={"schema": SCHEMA_FACET}, ) ], job_facets={"sql": SqlJobFacet(query=formatted_sql)}, From 1d6f6131df7e420b9e9dd1535ea7cd1a29e3c548 Mon Sep 17 00:00:00 2001 From: Wei Lee Date: Wed, 5 Jul 2023 03:54:28 +0800 Subject: [PATCH 040/533] refresh connection if an exception is caught in "AzureDataFactory" (#32323) --- .../microsoft/azure/hooks/data_factory.py | 43 +++--- .../microsoft/azure/triggers/data_factory.py | 126 ++++++++++++------ .../azure/hooks/test_azure_data_factory.py | 36 +++-- .../azure/triggers/test_azure_data_factory.py | 6 +- 4 files changed, 131 insertions(+), 80 deletions(-) diff --git a/airflow/providers/microsoft/azure/hooks/data_factory.py b/airflow/providers/microsoft/azure/hooks/data_factory.py index b9e7feea0a13..590f2af92b36 100644 --- a/airflow/providers/microsoft/azure/hooks/data_factory.py +++ b/airflow/providers/microsoft/azure/hooks/data_factory.py @@ -36,6 +36,7 @@ from typing import Any, Callable, TypeVar, Union, cast from asgiref.sync import sync_to_async +from azure.core.exceptions import ServiceRequestError from azure.core.polling import LROPoller from azure.identity import ClientSecretCredential, DefaultAzureCredential from azure.identity.aio import ( @@ -214,6 +215,10 @@ def get_conn(self) -> DataFactoryManagementClient: return self._conn + def refresh_conn(self) -> DataFactoryManagementClient: + self._conn = None + return self.get_conn() + @provide_targeted_factory def get_factory( self, resource_group_name: str | None = None, factory_name: str | None = None, **config: Any @@ -812,6 +817,7 @@ def wait_for_pipeline_run_status( resource_group_name=resource_group_name, ) pipeline_run_status = self.get_pipeline_run_status(**pipeline_run_info) + executed_after_token_refresh = True start_time = time.monotonic() @@ -828,7 +834,14 @@ def wait_for_pipeline_run_status( # Wait to check the status of the pipeline run based on the ``check_interval`` configured. time.sleep(check_interval) - pipeline_run_status = self.get_pipeline_run_status(**pipeline_run_info) + try: + pipeline_run_status = self.get_pipeline_run_status(**pipeline_run_info) + executed_after_token_refresh = True + except ServiceRequestError: + if executed_after_token_refresh: + self.refresh_conn() + continue + raise return pipeline_run_status in expected_statuses @@ -1132,6 +1145,10 @@ async def get_async_conn(self) -> AsyncDataFactoryManagementClient: return self._async_conn + async def refresh_conn(self) -> AsyncDataFactoryManagementClient: + self._conn = None + return await self.get_async_conn() + @provide_targeted_factory_async async def get_pipeline_run( self, @@ -1149,11 +1166,8 @@ async def get_pipeline_run( :param config: Extra parameters for the ADF client. """ client = await self.get_async_conn() - try: - pipeline_run = await client.pipeline_runs.get(resource_group_name, factory_name, run_id) - return pipeline_run - except Exception as e: - raise AirflowException(e) + pipeline_run = await client.pipeline_runs.get(resource_group_name, factory_name, run_id) + return pipeline_run async def get_adf_pipeline_run_status( self, run_id: str, resource_group_name: str | None = None, factory_name: str | None = None @@ -1165,16 +1179,13 @@ async def get_adf_pipeline_run_status( :param resource_group_name: The resource group name. :param factory_name: The factory name. """ - try: - pipeline_run = await self.get_pipeline_run( - run_id=run_id, - factory_name=factory_name, - resource_group_name=resource_group_name, - ) - status: str = pipeline_run.status - return status - except Exception as e: - raise AirflowException(e) + pipeline_run = await self.get_pipeline_run( + run_id=run_id, + factory_name=factory_name, + resource_group_name=resource_group_name, + ) + status: str = pipeline_run.status + return status @provide_targeted_factory_async async def cancel_pipeline_run( diff --git a/airflow/providers/microsoft/azure/triggers/data_factory.py b/airflow/providers/microsoft/azure/triggers/data_factory.py index 40b95559405c..e3dd38ad6646 100644 --- a/airflow/providers/microsoft/azure/triggers/data_factory.py +++ b/airflow/providers/microsoft/azure/triggers/data_factory.py @@ -20,6 +20,8 @@ import time from typing import Any, AsyncIterator +from azure.core.exceptions import ServiceRequestError + from airflow.providers.microsoft.azure.hooks.data_factory import ( AzureDataFactoryAsyncHook, AzureDataFactoryPipelineRunStatus, @@ -68,24 +70,41 @@ def serialize(self) -> tuple[str, dict[str, Any]]: async def run(self) -> AsyncIterator[TriggerEvent]: """Make async connection to Azure Data Factory, polls for the pipeline run status.""" hook = AzureDataFactoryAsyncHook(azure_data_factory_conn_id=self.azure_data_factory_conn_id) + executed_after_token_refresh = False try: while True: - pipeline_status = await hook.get_adf_pipeline_run_status( - run_id=self.run_id, - resource_group_name=self.resource_group_name, - factory_name=self.factory_name, - ) - if pipeline_status == AzureDataFactoryPipelineRunStatus.FAILED: - yield TriggerEvent( - {"status": "error", "message": f"Pipeline run {self.run_id} has Failed."} + try: + pipeline_status = await hook.get_adf_pipeline_run_status( + run_id=self.run_id, + resource_group_name=self.resource_group_name, + factory_name=self.factory_name, ) - elif pipeline_status == AzureDataFactoryPipelineRunStatus.CANCELLED: - msg = f"Pipeline run {self.run_id} has been Cancelled." - yield TriggerEvent({"status": "error", "message": msg}) - elif pipeline_status == AzureDataFactoryPipelineRunStatus.SUCCEEDED: - msg = f"Pipeline run {self.run_id} has been Succeeded." - yield TriggerEvent({"status": "success", "message": msg}) - await asyncio.sleep(self.poke_interval) + executed_after_token_refresh = False + if pipeline_status == AzureDataFactoryPipelineRunStatus.FAILED: + yield TriggerEvent( + {"status": "error", "message": f"Pipeline run {self.run_id} has Failed."} + ) + return + elif pipeline_status == AzureDataFactoryPipelineRunStatus.CANCELLED: + msg = f"Pipeline run {self.run_id} has been Cancelled." + yield TriggerEvent({"status": "error", "message": msg}) + return + elif pipeline_status == AzureDataFactoryPipelineRunStatus.SUCCEEDED: + msg = f"Pipeline run {self.run_id} has been Succeeded." + yield TriggerEvent({"status": "success", "message": msg}) + return + await asyncio.sleep(self.poke_interval) + except ServiceRequestError: + # conn might expire during long running pipeline. + # If expcetion is caught, it tries to refresh connection once. + # If it still doesn't fix the issue, + # than the execute_after_token_refresh would still be False + # and an exception will be raised + if executed_after_token_refresh: + await hook.refresh_conn() + executed_after_token_refresh = False + continue + raise except Exception as e: yield TriggerEvent({"status": "error", "message": str(e)}) @@ -147,33 +166,49 @@ async def run(self) -> AsyncIterator[TriggerEvent]: resource_group_name=self.resource_group_name, factory_name=self.factory_name, ) + executed_after_token_refresh = True if self.wait_for_termination: while self.end_time > time.time(): - pipeline_status = await hook.get_adf_pipeline_run_status( - run_id=self.run_id, - resource_group_name=self.resource_group_name, - factory_name=self.factory_name, - ) - if pipeline_status in AzureDataFactoryPipelineRunStatus.FAILURE_STATES: - yield TriggerEvent( - { - "status": "error", - "message": f"The pipeline run {self.run_id} has {pipeline_status}.", - "run_id": self.run_id, - } + try: + pipeline_status = await hook.get_adf_pipeline_run_status( + run_id=self.run_id, + resource_group_name=self.resource_group_name, + factory_name=self.factory_name, ) - elif pipeline_status == AzureDataFactoryPipelineRunStatus.SUCCEEDED: - yield TriggerEvent( - { - "status": "success", - "message": f"The pipeline run {self.run_id} has {pipeline_status}.", - "run_id": self.run_id, - } + executed_after_token_refresh = True + if pipeline_status in AzureDataFactoryPipelineRunStatus.FAILURE_STATES: + yield TriggerEvent( + { + "status": "error", + "message": f"The pipeline run {self.run_id} has {pipeline_status}.", + "run_id": self.run_id, + } + ) + return + elif pipeline_status == AzureDataFactoryPipelineRunStatus.SUCCEEDED: + yield TriggerEvent( + { + "status": "success", + "message": f"The pipeline run {self.run_id} has {pipeline_status}.", + "run_id": self.run_id, + } + ) + return + self.log.info( + "Sleeping for %s. The pipeline state is %s.", self.check_interval, pipeline_status ) - self.log.info( - "Sleeping for %s. The pipeline state is %s.", self.check_interval, pipeline_status - ) - await asyncio.sleep(self.check_interval) + await asyncio.sleep(self.check_interval) + except ServiceRequestError: + # conn might expire during long running pipeline. + # If expcetion is caught, it tries to refresh connection once. + # If it still doesn't fix the issue, + # than the execute_after_token_refresh would still be False + # and an exception will be raised + if executed_after_token_refresh: + await hook.refresh_conn() + executed_after_token_refresh = False + continue + raise yield TriggerEvent( { @@ -192,10 +227,13 @@ async def run(self) -> AsyncIterator[TriggerEvent]: ) except Exception as e: if self.run_id: - await hook.cancel_pipeline_run( - run_id=self.run_id, - resource_group_name=self.resource_group_name, - factory_name=self.factory_name, - ) - self.log.info("Unexpected error %s caught. Cancel pipeline run %s", str(e), self.run_id) + try: + await hook.cancel_pipeline_run( + run_id=self.run_id, + resource_group_name=self.resource_group_name, + factory_name=self.factory_name, + ) + self.log.info("Unexpected error %s caught. Cancel pipeline run %s", str(e), self.run_id) + except Exception as err: + yield TriggerEvent({"status": "error", "message": str(err), "run_id": self.run_id}) yield TriggerEvent({"status": "error", "message": str(e), "run_id": self.run_id}) diff --git a/tests/providers/microsoft/azure/hooks/test_azure_data_factory.py b/tests/providers/microsoft/azure/hooks/test_azure_data_factory.py index c7b256e9cb28..57f7bc6178dc 100644 --- a/tests/providers/microsoft/azure/hooks/test_azure_data_factory.py +++ b/tests/providers/microsoft/azure/hooks/test_azure_data_factory.py @@ -720,6 +720,14 @@ def test_backcompat_prefix_both_prefers_short(mock_connect): mock_connect.return_value.factories.delete.assert_called_with("non-prefixed", "n/a") +def test_refresh_conn(hook): + """Test refresh_conn method _conn is reset and get_conn is called""" + with patch.object(hook, "get_conn") as mock_get_conn: + hook.refresh_conn() + assert not hook._conn + assert mock_get_conn.called + + class TestAzureDataFactoryAsyncHook: @pytest.mark.asyncio @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_async_conn") @@ -780,16 +788,6 @@ async def test_get_adf_pipeline_run_status_cancelled(self, mock_get_pipeline_run response = await hook.get_adf_pipeline_run_status(RUN_ID, RESOURCE_GROUP_NAME, DATAFACTORY_NAME) assert response == mock_status - @pytest.mark.asyncio - @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_async_conn") - @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_pipeline_run") - async def test_get_adf_pipeline_run_status_exception(self, mock_get_pipeline_run, mock_conn): - """Test get_adf_pipeline_run_status function with exception""" - mock_get_pipeline_run.side_effect = Exception("Test exception") - hook = AzureDataFactoryAsyncHook(AZURE_DATA_FACTORY_CONN_ID) - with pytest.raises(AirflowException): - await hook.get_adf_pipeline_run_status(RUN_ID, RESOURCE_GROUP_NAME, DATAFACTORY_NAME) - @pytest.mark.asyncio @mock.patch("azure.mgmt.datafactory.models._models_py3.PipelineRun") @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_connection") @@ -810,15 +808,6 @@ async def test_get_pipeline_run_exception_without_resource( with pytest.raises(AirflowException): await hook.get_pipeline_run(RUN_ID, None, DATAFACTORY_NAME) - @pytest.mark.asyncio - @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_async_conn") - async def test_get_pipeline_run_exception(self, mock_conn): - """Test get_pipeline_run function with exception""" - mock_conn.return_value.pipeline_runs.get.side_effect = Exception("Test exception") - hook = AzureDataFactoryAsyncHook(AZURE_DATA_FACTORY_CONN_ID) - with pytest.raises(AirflowException): - await hook.get_pipeline_run(RUN_ID, RESOURCE_GROUP_NAME, DATAFACTORY_NAME) - @pytest.mark.asyncio @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_connection") async def test_get_async_conn(self, mock_connection): @@ -958,3 +947,12 @@ def test_get_field_non_prefixed_extras(self): assert get_field(extras, "factory_name", strict=True) == DATAFACTORY_NAME with pytest.raises(KeyError): get_field(extras, "non-existent-field", strict=True) + + @pytest.mark.asyncio + @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_async_conn") + async def test_refresh_conn(self, mock_get_async_conn): + """Test refresh_conn method _conn is reset and get_async_conn is called""" + hook = AzureDataFactoryAsyncHook(AZURE_DATA_FACTORY_CONN_ID) + await hook.refresh_conn() + assert not hook._conn + assert mock_get_async_conn.called diff --git a/tests/providers/microsoft/azure/triggers/test_azure_data_factory.py b/tests/providers/microsoft/azure/triggers/test_azure_data_factory.py index 9df51bae71a5..fd1a20655404 100644 --- a/tests/providers/microsoft/azure/triggers/test_azure_data_factory.py +++ b/tests/providers/microsoft/azure/triggers/test_azure_data_factory.py @@ -163,10 +163,14 @@ async def test_adf_pipeline_run_status_sensors_trigger_cancelled( assert TriggerEvent({"status": "error", "message": mock_message}) == actual @pytest.mark.asyncio + @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.refresh_conn") @mock.patch(f"{MODULE}.hooks.data_factory.AzureDataFactoryAsyncHook.get_adf_pipeline_run_status") - async def test_adf_pipeline_run_status_sensors_trigger_exception(self, mock_data_factory): + async def test_adf_pipeline_run_status_sensors_trigger_exception( + self, mock_data_factory, mock_refresh_token + ): """Test EMR container sensors with raise exception""" mock_data_factory.side_effect = Exception("Test exception") + mock_refresh_token.side_effect = Exception("Test exception") task = [i async for i in self.TRIGGER.run()] assert len(task) == 1 From 9ef9dc46f393a9e093b2d30928c30e72508b9394 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Tue, 4 Jul 2023 22:04:22 +0200 Subject: [PATCH 041/533] Adds support for custom query in workers KEDA trigger (#32308) Signed-off-by: Hussein Awala --- .../workers/worker-kedaautoscaler.yaml | 8 +--- chart/values.schema.json | 5 +++ chart/values.yaml | 9 ++++ tests/charts/airflow_core/test_worker.py | 45 +++++++++++++++++++ 4 files changed, 60 insertions(+), 7 deletions(-) diff --git a/chart/templates/workers/worker-kedaautoscaler.yaml b/chart/templates/workers/worker-kedaautoscaler.yaml index 036e172d8964..44e17e445b59 100644 --- a/chart/templates/workers/worker-kedaautoscaler.yaml +++ b/chart/templates/workers/worker-kedaautoscaler.yaml @@ -51,11 +51,5 @@ spec: metadata: targetQueryValue: "1" connectionFromEnv: AIRFLOW_CONN_AIRFLOW_DB - query: >- - SELECT ceil(COUNT(*)::decimal / {{ .Values.config.celery.worker_concurrency }}) - FROM task_instance - WHERE (state='running' OR state='queued') - {{- if eq .Values.executor "CeleryKubernetesExecutor" }} - AND queue != '{{ .Values.config.celery_kubernetes_executor.kubernetes_queue }}' - {{- end }} + query: {{ tpl .Values.workers.keda.query . | quote }} {{- end }} diff --git a/chart/values.schema.json b/chart/values.schema.json index 80e25b9b10a6..56689532fb9a 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -1433,6 +1433,11 @@ } } } + }, + "query": { + "description": "Query to use for KEDA autoscaling. Must return a single integer.", + "type": "string", + "default": "SELECT ceil(COUNT(*)::decimal / {{ .Values.config.celery.worker_concurrency }}) FROM task_instance WHERE (state='running' OR state='queued') {{- if eq .Values.executor \"CeleryKubernetesExecutor\" }} AND queue != '{{ .Values.config.celery_kubernetes_executor.kubernetes_queue }}' {{- end }}" } } }, diff --git a/chart/values.yaml b/chart/values.yaml index 3beeb3765cfa..3b07b17049dc 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -538,6 +538,15 @@ workers: # value: 100 # periodSeconds: 15 + # Query to use for KEDA autoscaling. Must return a single integer. + query: >- + SELECT ceil(COUNT(*)::decimal / {{ .Values.config.celery.worker_concurrency }}) + FROM task_instance + WHERE (state='running' OR state='queued') + {{- if eq .Values.executor "CeleryKubernetesExecutor" }} + AND queue != '{{ .Values.config.celery_kubernetes_executor.kubernetes_queue }}' + {{- end }} + persistence: # Enable persistent volumes enabled: true diff --git a/tests/charts/airflow_core/test_worker.py b/tests/charts/airflow_core/test_worker.py index a49b78fee433..f4e02024a0f0 100644 --- a/tests/charts/airflow_core/test_worker.py +++ b/tests/charts/airflow_core/test_worker.py @@ -640,6 +640,51 @@ def test_should_remove_replicas_field(self): assert "replicas" not in jmespath.search("spec", docs[0]) + @pytest.mark.parametrize( + "query, executor, expected_query", + [ + # default query with CeleryExecutor + ( + None, + "CeleryExecutor", + "SELECT ceil(COUNT(*)::decimal / 16) FROM task_instance" + " WHERE (state='running' OR state='queued')", + ), + # default query with CeleryKubernetesExecutor + ( + None, + "CeleryKubernetesExecutor", + "SELECT ceil(COUNT(*)::decimal / 16) FROM task_instance" + " WHERE (state='running' OR state='queued') AND queue != 'kubernetes'", + ), + # test custom static query + ( + "SELECT ceil(COUNT(*)::decimal / 16) FROM task_instance", + "CeleryKubernetesExecutor", + "SELECT ceil(COUNT(*)::decimal / 16) FROM task_instance", + ), + # test custom template query + ( + "SELECT ceil(COUNT(*)::decimal / {{ mul .Values.config.celery.worker_concurrency 2 }})" + " FROM task_instance", + "CeleryKubernetesExecutor", + "SELECT ceil(COUNT(*)::decimal / 32) FROM task_instance", + ), + ], + ) + def test_should_use_keda_query(self, query, executor, expected_query): + + docs = render_chart( + values={ + "executor": executor, + "workers": { + "keda": {"enabled": True, **({"query": query} if query else {})}, + }, + }, + show_only=["templates/workers/worker-kedaautoscaler.yaml"], + ) + assert expected_query == jmespath.search("spec.triggers[0].metadata.query", docs[0]) + class TestWorkerNetworkPolicy: def test_should_add_component_specific_labels(self): From c68a319f04d90bd91a116d80a1f0d2b06879571e Mon Sep 17 00:00:00 2001 From: Changhoon Oh <81631424+okayhooni@users.noreply.github.com> Date: Wed, 5 Jul 2023 05:08:02 +0900 Subject: [PATCH 042/533] Introducing AirflowClusterPolicySkipDag exception (#32013) --- airflow/exceptions.py | 9 +++++- airflow/models/dagbag.py | 5 +++- .../cluster-policies.rst | 4 +++ docs/apache-airflow/best-practices.rst | 28 +++++++++++++++++++ tests/cluster_policies/__init__.py | 9 ++++-- 5 files changed, 51 insertions(+), 4 deletions(-) diff --git a/airflow/exceptions.py b/airflow/exceptions.py index 9cdb8c418a1b..8c65a1f66f97 100644 --- a/airflow/exceptions.py +++ b/airflow/exceptions.py @@ -169,8 +169,15 @@ class AirflowClusterPolicyViolation(AirflowException): """Raise when there is a violation of a Cluster Policy in DAG definition.""" +class AirflowClusterPolicySkipDag(AirflowException): + """Raise when skipping dag is needed in Cluster Policy.""" + + class AirflowClusterPolicyError(AirflowException): - """Raise when there is an error except AirflowClusterPolicyViolation in Cluster Policy.""" + """ + Raise when there is an error in Cluster Policy, + except AirflowClusterPolicyViolation and AirflowClusterPolicySkipDag. + """ class AirflowTimetableInvalid(AirflowException): diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index b422d4699e99..1b80cb01582a 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -38,6 +38,7 @@ from airflow.configuration import conf from airflow.exceptions import ( AirflowClusterPolicyError, + AirflowClusterPolicySkipDag, AirflowClusterPolicyViolation, AirflowDagCycleException, AirflowDagDuplicatedIdException, @@ -442,6 +443,8 @@ def _process_modules(self, filepath, mods, file_last_changed_on_disk): try: dag.validate() self.bag_dag(dag=dag, root_dag=dag) + except AirflowClusterPolicySkipDag: + pass except Exception as e: self.log.exception("Failed to bag_dag: %s", dag.fileloc) self.import_errors[dag.fileloc] = f"{type(e).__name__}: {e}" @@ -477,7 +480,7 @@ def _bag_dag(self, *, dag, root_dag, recursive): for task in dag.tasks: settings.task_policy(task) - except AirflowClusterPolicyViolation: + except (AirflowClusterPolicyViolation, AirflowClusterPolicySkipDag): raise except Exception as e: self.log.exception(e) diff --git a/docs/apache-airflow/administration-and-deployment/cluster-policies.rst b/docs/apache-airflow/administration-and-deployment/cluster-policies.rst index ddf05fde69c0..c664d0b5037c 100644 --- a/docs/apache-airflow/administration-and-deployment/cluster-policies.rst +++ b/docs/apache-airflow/administration-and-deployment/cluster-policies.rst @@ -43,6 +43,10 @@ There are three main types of cluster policy: The DAG and Task cluster policies can raise the :class:`~airflow.exceptions.AirflowClusterPolicyViolation` exception to indicate that the dag/task they were passed is not compliant and should not be loaded. +They can also raise the :class:`~airflow.exceptions.AirflowClusterPolicySkipDag` exception +when skipping that DAG is needed intentionally. Unlike :class:`~airflow.exceptions.AirflowClusterPolicyViolation`, +this exception is not displayed on the Airflow web UI (Internally, it's not recorded on ``import_error`` table on meta database.) + Any extra attributes set by a cluster policy take priority over those defined in your DAG file; for example, if you set an ``sla`` on your Task in the DAG file, and then your cluster policy also sets an ``sla``, the cluster policy's value will take precedence. diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index 2e09f9141ded..b7202ef5a097 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -394,6 +394,34 @@ It's important to note, that without ``watcher`` task, the whole DAG Run will ge If we want the ``watcher`` to monitor the state of all tasks, we need to make it dependent on all of them separately. Thanks to this, we can fail the DAG Run if any of the tasks fail. Note that the watcher task has a trigger rule set to ``"one_failed"``. On the other hand, without the ``teardown`` task, the ``watcher`` task will not be needed, because ``failing_task`` will propagate its ``failed`` state to downstream task ``passed_task`` and the whole DAG Run will also get the ``failed`` status. + +Using AirflowClusterPolicySkipDag exception in cluster policies to skip specific DAGs +------------------------------------------------------------------------------------- + +.. versionadded:: 2.7 + +Airflow DAGs can usually be deployed and updated with the specific branch of Git repository via ``git-sync``. +But, when you have to run multiple Airflow clusters for some operational reasons, it's very cumbersome to maintain multiple Git branches. +Especially, you have some difficulties when you need to synchronize two separate branches(like ``prod`` and ``beta``) periodically with proper branching strategy. + +- cherry-pick is too cumbersome to maintain Git repository. +- hard-reset is not recommended way for GitOps + +So, you can consider connecting multiple Airflow clusters with same Git branch (like ``main``), and maintaining those with different environment variables and different connection configurations with same ``connection_id``. +you can also raise :class:`~airflow.exceptions.AirflowClusterPolicySkipDag` exception on the cluster policy, to load specific DAGs to :class:`~airflow.models.dagbag.DagBag` on the specific Airflow deployment only, if needed. + +.. code-block:: python + + def dag_policy(dag: DAG): + """Skipping the DAG with `only_for_beta` tag.""" + + if "only_for_beta" in dag.tags: + raise AirflowClusterPolicySkipDag( + f"DAG {dag.dag_id} is not loaded on the production cluster, due to `only_for_beta` tag." + ) + +The example above, shows the ``dag_policy`` code snippet to skip the DAG depending on the tags it has. + .. _best_practices/reducing_dag_complexity: Reducing DAG complexity diff --git a/tests/cluster_policies/__init__.py b/tests/cluster_policies/__init__.py index 40c3b2140681..5aaf5701acc6 100644 --- a/tests/cluster_policies/__init__.py +++ b/tests/cluster_policies/__init__.py @@ -22,7 +22,7 @@ from typing import Callable from airflow.configuration import conf -from airflow.exceptions import AirflowClusterPolicyViolation +from airflow.exceptions import AirflowClusterPolicySkipDag, AirflowClusterPolicyViolation from airflow.models import DAG, TaskInstance from airflow.models.baseoperator import BaseOperator @@ -73,12 +73,17 @@ def example_task_policy(task: BaseOperator): # [START example_dag_cluster_policy] def dag_policy(dag: DAG): - """Ensure that DAG has at least one tag""" + """Ensure that DAG has at least one tag and skip the DAG with `only_for_beta` tag.""" if not dag.tags: raise AirflowClusterPolicyViolation( f"DAG {dag.dag_id} has no tags. At least one tag required. File path: {dag.fileloc}" ) + if "only_for_beta" in dag.tags: + raise AirflowClusterPolicySkipDag( + f"DAG {dag.dag_id} is not loaded on the production cluster, due to `only_for_beta` tag." + ) + # [END example_dag_cluster_policy] From 5623a21a1fc738ccb97ade4d4197b181bf1128d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rapha=C3=ABl=20Vandon?= Date: Tue, 4 Jul 2023 13:28:15 -0700 Subject: [PATCH 043/533] deprecate arbitrary parameter passing to RDS hook (#32352) * deprecate arbitrary parameter passing to RDS hook * add url to PR --- airflow/providers/amazon/aws/operators/rds.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/rds.py b/airflow/providers/amazon/aws/operators/rds.py index 440d895afa15..9aef6701660e 100644 --- a/airflow/providers/amazon/aws/operators/rds.py +++ b/airflow/providers/amazon/aws/operators/rds.py @@ -18,12 +18,13 @@ from __future__ import annotations import json +import warnings from datetime import timedelta from typing import TYPE_CHECKING, Sequence from mypy_boto3_rds.type_defs import TagTypeDef -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.rds import RdsHook from airflow.providers.amazon.aws.triggers.rds import RdsDbInstanceTrigger @@ -42,6 +43,15 @@ class RdsBaseOperator(BaseOperator): ui_fgcolor = "#ffffff" def __init__(self, *args, aws_conn_id: str = "aws_conn_id", hook_params: dict | None = None, **kwargs): + if hook_params is not None: + warnings.warn( + "The parameter hook_params is deprecated and will be removed. " + "If you were using it, please get in touch either on airflow slack, " + "or by opening a github issue on the project. " + "You can mention https://github.com/apache/airflow/pull/32352", + AirflowProviderDeprecationWarning, + stacklevel=3, # 2 is in the operator's init, 3 is in the user code creating the operator + ) self.hook_params = hook_params or {} self.hook = RdsHook(aws_conn_id=aws_conn_id, **self.hook_params) super().__init__(*args, **kwargs) From 8a6766f0b95cd270b52117115de48fa328bac18c Mon Sep 17 00:00:00 2001 From: Francesco Caronte Date: Tue, 4 Jul 2023 22:43:33 +0200 Subject: [PATCH 044/533] Runtime class name (#31868) --- chart/files/pod-template-file.kubernetes-helm-yaml | 3 +++ chart/templates/workers/worker-deployment.yaml | 3 +++ chart/values.schema.json | 8 ++++++++ chart/values.yaml | 1 + tests/charts/airflow_core/test_worker.py | 10 ++++++++++ 5 files changed, 25 insertions(+) diff --git a/chart/files/pod-template-file.kubernetes-helm-yaml b/chart/files/pod-template-file.kubernetes-helm-yaml index 1f3dbc909d39..70ac65c8e738 100644 --- a/chart/files/pod-template-file.kubernetes-helm-yaml +++ b/chart/files/pod-template-file.kubernetes-helm-yaml @@ -85,6 +85,9 @@ spec: {{- if .Values.workers.priorityClassName }} priorityClassName: {{ .Values.workers.priorityClassName }} {{- end }} + {{- if .Values.workers.runtimeClassName }} + priorityClassName: {{ .Values.workers.runtimeClassName }} + {{- end }} {{- if or .Values.registry.secretName .Values.registry.connection }} imagePullSecrets: - name: {{ template "registry_secret" . }} diff --git a/chart/templates/workers/worker-deployment.yaml b/chart/templates/workers/worker-deployment.yaml index bacba8ff0306..75b2b1b60a4c 100644 --- a/chart/templates/workers/worker-deployment.yaml +++ b/chart/templates/workers/worker-deployment.yaml @@ -99,6 +99,9 @@ spec: {{- toYaml .Values.workers.podAnnotations | nindent 8 }} {{- end }} spec: + {{- if .Values.workers.runtimeClassName }} + runtimeClassName: {{ .Values.workers.runtimeClassName }} + {{- end }} {{- if .Values.workers.priorityClassName }} priorityClassName: {{ .Values.workers.priorityClassName }} {{- end }} diff --git a/chart/values.schema.json b/chart/values.schema.json index 56689532fb9a..3f9b7356796e 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -1625,6 +1625,14 @@ "type": "string" } }, + "runtimeClassName": { + "description": "Specify runtime for worker pods.", + "type": [ + "string", + "null" + ], + "default": null + }, "priorityClassName": { "description": "Specify priority for worker pods.", "type": [ diff --git a/chart/values.yaml b/chart/values.yaml index 3b07b17049dc..70d862b2fb0a 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -617,6 +617,7 @@ workers: # Select certain nodes for airflow worker pods. nodeSelector: {} + runtimeClassName: ~ priorityClassName: ~ affinity: {} # default worker affinity is: diff --git a/tests/charts/airflow_core/test_worker.py b/tests/charts/airflow_core/test_worker.py index f4e02024a0f0..f3127b3bdf46 100644 --- a/tests/charts/airflow_core/test_worker.py +++ b/tests/charts/airflow_core/test_worker.py @@ -378,6 +378,16 @@ def test_should_create_default_affinity(self): docs[0], ) + def test_runtime_class_name_values_are_configurable(self): + docs = render_chart( + values={ + "workers": {"runtimeClassName": "nvidia"}, + }, + show_only=["templates/workers/worker-deployment.yaml"], + ) + + assert jmespath.search("spec.template.spec.runtimeClassName", docs[0]) == "nvidia" + def test_livenessprobe_values_are_configurable(self): docs = render_chart( values={ From 891c2e401928ecafea78f7c6c3b453663ef03dce Mon Sep 17 00:00:00 2001 From: Utkarsh Sharma Date: Wed, 5 Jul 2023 02:19:47 +0530 Subject: [PATCH 045/533] Add Deferrable switch to SnowflakeSqlApiOperator (#31596) --- .../snowflake/hooks/snowflake_sql_api.py | 43 +++++-- .../snowflake/operators/snowflake.py | 46 ++++++- airflow/providers/snowflake/provider.yaml | 5 + .../providers/snowflake/triggers/__init__.py | 16 +++ .../snowflake/triggers/snowflake_trigger.py | 109 ++++++++++++++++ .../operators/snowflake.rst | 2 + .../snowflake/hooks/test_snowflake_sql_api.py | 56 +++++++- .../snowflake/operators/test_snowflake.py | 96 +++++++++++++- .../providers/snowflake/triggers/__init__.py | 16 +++ .../snowflake/triggers/test_snowflake.py | 120 ++++++++++++++++++ 10 files changed, 490 insertions(+), 19 deletions(-) create mode 100644 airflow/providers/snowflake/triggers/__init__.py create mode 100644 airflow/providers/snowflake/triggers/snowflake_trigger.py create mode 100644 tests/providers/snowflake/triggers/__init__.py create mode 100644 tests/providers/snowflake/triggers/test_snowflake.py diff --git a/airflow/providers/snowflake/hooks/snowflake_sql_api.py b/airflow/providers/snowflake/hooks/snowflake_sql_api.py index 0d808291ffa0..eec3c7349e1e 100644 --- a/airflow/providers/snowflake/hooks/snowflake_sql_api.py +++ b/airflow/providers/snowflake/hooks/snowflake_sql_api.py @@ -21,6 +21,7 @@ from pathlib import Path from typing import Any +import aiohttp import requests from cryptography.hazmat.backends import default_backend from cryptography.hazmat.primitives import serialization @@ -59,7 +60,8 @@ class SnowflakeSqlApiHook(SnowflakeHook): :param session_parameters: You can set session-level parameters at the time you connect to Snowflake :param token_life_time: lifetime of the JWT Token in timedelta - :param token_renewal_delta: Renewal time of the JWT Token in timedelta + :param token_renewal_delta: Renewal time of the JWT Token in timedelta + :param deferrable: Run operator in the deferrable mode. """ LIFETIME = timedelta(minutes=59) # The tokens will have a 59 minute lifetime @@ -225,17 +227,7 @@ def check_query_output(self, query_ids: list[str]) -> None: f"Response: {e.response.content}, Status Code: {e.response.status_code}" ) - def get_sql_api_query_status(self, query_id: str) -> dict[str, str | list[str]]: - """ - Based on the query id async HTTP request is made to snowflake SQL API and return response. - - :param query_id: statement handle id for the individual statements. - """ - self.log.info("Retrieving status for query id %s", {query_id}) - header, params, url = self.get_request_url_header_params(query_id) - response = requests.get(url, params=params, headers=header) - status_code = response.status_code - resp = response.json() + def _process_response(self, status_code, resp): self.log.info("Snowflake SQL GET statements status API response: %s", resp) if status_code == 202: return {"status": "running", "message": "Query statements are still running"} @@ -254,3 +246,30 @@ def get_sql_api_query_status(self, query_id: str) -> dict[str, str | list[str]]: } else: return {"status": "error", "message": resp["message"]} + + def get_sql_api_query_status(self, query_id: str) -> dict[str, str | list[str]]: + """ + Based on the query id async HTTP request is made to snowflake SQL API and return response. + + :param query_id: statement handle id for the individual statements. + """ + self.log.info("Retrieving status for query id %s", query_id) + header, params, url = self.get_request_url_header_params(query_id) + response = requests.get(url, params=params, headers=header) + status_code = response.status_code + resp = response.json() + return self._process_response(status_code, resp) + + async def get_sql_api_query_status_async(self, query_id: str) -> dict[str, str | list[str]]: + """ + Based on the query id async HTTP request is made to snowflake SQL API and return response. + + :param query_id: statement handle id for the individual statements. + """ + self.log.info("Retrieving status for query id %s", query_id) + header, params, url = self.get_request_url_header_params(query_id) + async with aiohttp.ClientSession(headers=header) as session: + async with session.get(url, params=params) as response: + status_code = response.status + resp = await response.json() + return self._process_response(status_code, resp) diff --git a/airflow/providers/snowflake/operators/snowflake.py b/airflow/providers/snowflake/operators/snowflake.py index b56b14b0d5a4..db35fa000731 100644 --- a/airflow/providers/snowflake/operators/snowflake.py +++ b/airflow/providers/snowflake/operators/snowflake.py @@ -20,7 +20,7 @@ import time import warnings from datetime import timedelta -from typing import TYPE_CHECKING, Any, Iterable, Mapping, Sequence, SupportsAbs +from typing import TYPE_CHECKING, Any, Iterable, List, Mapping, Sequence, SupportsAbs, cast from airflow import AirflowException from airflow.exceptions import AirflowProviderDeprecationWarning @@ -33,6 +33,7 @@ from airflow.providers.snowflake.hooks.snowflake_sql_api import ( SnowflakeSqlApiHook, ) +from airflow.providers.snowflake.triggers.snowflake_trigger import SnowflakeSqlApiTrigger if TYPE_CHECKING: from airflow.utils.context import Context @@ -430,6 +431,7 @@ class SnowflakeSqlApiOperator(SQLExecuteQueryOperator): :param bindings: (Optional) Values of bind variables in the SQL statement. When executing the statement, Snowflake replaces placeholders (? and :name) in the statement with these specified values. + :param deferrable: Run operator in the deferrable mode. """ # noqa LIFETIME = timedelta(minutes=59) # The tokens will have a 59 minutes lifetime @@ -450,6 +452,7 @@ def __init__( token_life_time: timedelta = LIFETIME, token_renewal_delta: timedelta = RENEWAL_DELTA, bindings: dict[str, Any] | None = None, + deferrable: bool = False, **kwargs: Any, ) -> None: self.snowflake_conn_id = snowflake_conn_id @@ -459,6 +462,7 @@ def __init__( self.token_renewal_delta = token_renewal_delta self.bindings = bindings self.execute_async = False + self.deferrable = deferrable if any([warehouse, database, role, schema, authenticator, session_parameters]): # pragma: no cover hook_params = kwargs.pop("hook_params", {}) # pragma: no cover kwargs["hook_params"] = { @@ -482,6 +486,7 @@ def execute(self, context: Context) -> None: snowflake_conn_id=self.snowflake_conn_id, token_life_time=self.token_life_time, token_renewal_delta=self.token_renewal_delta, + deferrable=self.deferrable, ) self.query_ids = self._hook.execute_query( self.sql, statement_count=self.statement_count, bindings=self.bindings # type: ignore[arg-type] @@ -491,10 +496,23 @@ def execute(self, context: Context) -> None: if self.do_xcom_push: context["ti"].xcom_push(key="query_ids", value=self.query_ids) - statement_status = self.poll_on_queries() - if statement_status["error"]: - raise AirflowException(statement_status["error"]) - self._hook.check_query_output(self.query_ids) + if self.deferrable: + self.defer( + timeout=self.execution_timeout, + trigger=SnowflakeSqlApiTrigger( + poll_interval=self.poll_interval, + query_ids=self.query_ids, + snowflake_conn_id=self.snowflake_conn_id, + token_life_time=self.token_life_time, + token_renewal_delta=self.token_renewal_delta, + ), + method_name="execute_complete", + ) + else: + statement_status = self.poll_on_queries() + if statement_status["error"]: + raise AirflowException(statement_status["error"]) + self._hook.check_query_output(self.query_ids) def poll_on_queries(self): """Poll on requested queries.""" @@ -517,3 +535,21 @@ def poll_on_queries(self): queries_in_progress.remove(query_id) time.sleep(self.poll_interval) return {"success": statement_success_status, "error": statement_error_status} + + def execute_complete(self, context: Context, event: dict[str, str | list[str]] | None = None) -> None: + """ + Callback for when the trigger fires - returns immediately. + Relies on trigger to throw an exception, otherwise it assumes execution was + successful. + """ + if event: + if "status" in event and event["status"] == "error": + msg = f"{event['status']}: {event['message']}" + raise AirflowException(msg) + elif "status" in event and event["status"] == "success": + hook = SnowflakeSqlApiHook(snowflake_conn_id=self.snowflake_conn_id) + query_ids = cast(List[str], event["statement_query_ids"]) + hook.check_query_output(query_ids) + self.log.info("%s completed successfully.", self.task_id) + else: + self.log.info("%s completed successfully.", self.task_id) diff --git a/airflow/providers/snowflake/provider.yaml b/airflow/providers/snowflake/provider.yaml index 1e68fbddca55..2cea953ab428 100644 --- a/airflow/providers/snowflake/provider.yaml +++ b/airflow/providers/snowflake/provider.yaml @@ -100,3 +100,8 @@ transfers: connection-types: - hook-class-name: airflow.providers.snowflake.hooks.snowflake.SnowflakeHook connection-type: snowflake + +triggers: + - integration-name: Snowflake + python-modules: + - airflow.providers.snowflake.triggers.snowflake_trigger diff --git a/airflow/providers/snowflake/triggers/__init__.py b/airflow/providers/snowflake/triggers/__init__.py new file mode 100644 index 000000000000..13a83393a912 --- /dev/null +++ b/airflow/providers/snowflake/triggers/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/airflow/providers/snowflake/triggers/snowflake_trigger.py b/airflow/providers/snowflake/triggers/snowflake_trigger.py new file mode 100644 index 000000000000..4f1e0cffb299 --- /dev/null +++ b/airflow/providers/snowflake/triggers/snowflake_trigger.py @@ -0,0 +1,109 @@ +# 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 + +import asyncio +from datetime import timedelta +from typing import Any, AsyncIterator + +from airflow.providers.snowflake.hooks.snowflake_sql_api import SnowflakeSqlApiHook +from airflow.triggers.base import BaseTrigger, TriggerEvent + + +class SnowflakeSqlApiTrigger(BaseTrigger): + """ + Fetch the status for the query ids passed. + + :param poll_interval: polling period in seconds to check for the status + :param query_ids: List of Query ids to run and poll for the status + :param snowflake_conn_id: Reference to Snowflake connection id + :param token_life_time: lifetime of the JWT Token in timedelta + :param token_renewal_delta: Renewal time of the JWT Token in timedelta + """ + + def __init__( + self, + poll_interval: float, + query_ids: list[str], + snowflake_conn_id: str, + token_life_time: timedelta, + token_renewal_delta: timedelta, + ): + super().__init__() + self.poll_interval = poll_interval + self.query_ids = query_ids + self.snowflake_conn_id = snowflake_conn_id + self.token_life_time = token_life_time + self.token_renewal_delta = token_renewal_delta + + def serialize(self) -> tuple[str, dict[str, Any]]: + """Serializes SnowflakeSqlApiTrigger arguments and classpath.""" + return ( + "airflow.providers.snowflake.triggers.snowflake_trigger.SnowflakeSqlApiTrigger", + { + "poll_interval": self.poll_interval, + "query_ids": self.query_ids, + "snowflake_conn_id": self.snowflake_conn_id, + "token_life_time": self.token_life_time, + "token_renewal_delta": self.token_renewal_delta, + }, + ) + + async def run(self) -> AsyncIterator[TriggerEvent]: + """Wait for the query the snowflake query to complete.""" + SnowflakeSqlApiHook( + self.snowflake_conn_id, + self.token_life_time, + self.token_renewal_delta, + ) + try: + statement_query_ids: list[str] = [] + for query_id in self.query_ids: + while True: + statement_status = await self.get_query_status(query_id) + if statement_status["status"] not in ["running"]: + break + await asyncio.sleep(self.poll_interval) + if statement_status["status"] == "error": + yield TriggerEvent(statement_status) + return + if statement_status["status"] == "success": + statement_query_ids.extend(statement_status["statement_handles"]) + yield TriggerEvent( + { + "status": "success", + "statement_query_ids": statement_query_ids, + } + ) + except Exception as e: + yield TriggerEvent({"status": "error", "message": str(e)}) + + async def get_query_status(self, query_id: str) -> dict[str, Any]: + """ + Async function to check whether the query statement submitted via SQL API is still + running state and returns True if it is still running else + return False. + """ + hook = SnowflakeSqlApiHook( + self.snowflake_conn_id, + self.token_life_time, + self.token_renewal_delta, + ) + return await hook.get_sql_api_query_status_async(query_id) + + def _set_context(self, context): + pass diff --git a/docs/apache-airflow-providers-snowflake/operators/snowflake.rst b/docs/apache-airflow-providers-snowflake/operators/snowflake.rst index 1e80f3af2955..d3ffbec5a4cf 100644 --- a/docs/apache-airflow-providers-snowflake/operators/snowflake.rst +++ b/docs/apache-airflow-providers-snowflake/operators/snowflake.rst @@ -66,6 +66,8 @@ SnowflakeSqlApiOperator Use the :class:`SnowflakeSqlApiHook ` to execute SQL commands in a `Snowflake `__ database. +You can also run this operator in deferrable mode by setting ``deferrable`` param to ``True``. +This will ensure that the task is deferred from the Airflow worker slot and polling for the task status happens on the trigger. Using the Operator ^^^^^^^^^^^^^^^^^^ diff --git a/tests/providers/snowflake/hooks/test_snowflake_sql_api.py b/tests/providers/snowflake/hooks/test_snowflake_sql_api.py index 61e88de864df..fd2da72c928e 100644 --- a/tests/providers/snowflake/hooks/test_snowflake_sql_api.py +++ b/tests/providers/snowflake/hooks/test_snowflake_sql_api.py @@ -21,6 +21,7 @@ from pathlib import Path from typing import Any from unittest import mock +from unittest.mock import AsyncMock import pytest import requests @@ -396,7 +397,6 @@ def test_get_private_key_should_support_private_auth_with_unencrypted_key( ), pytest.raises(TypeError, match="Password was given but private key is not encrypted."): SnowflakeSqlApiHook(snowflake_conn_id="test_conn").get_private_key() - @pytest.mark.asyncio @pytest.mark.parametrize( "status_code,response,expected_response", [ @@ -456,3 +456,57 @@ def json(self): mock_requests.get.return_value = MockResponse(status_code, response) hook = SnowflakeSqlApiHook(snowflake_conn_id="test_conn") assert hook.get_sql_api_query_status("uuid") == expected_response + + @pytest.mark.asyncio + @pytest.mark.parametrize( + "status_code,response,expected_response", + [ + ( + 200, + { + "status": "success", + "message": "Statement executed successfully.", + "statementHandle": "uuid", + }, + { + "status": "success", + "message": "Statement executed successfully.", + "statement_handles": ["uuid"], + }, + ), + ( + 200, + { + "status": "success", + "message": "Statement executed successfully.", + "statementHandles": ["uuid", "uuid1"], + }, + { + "status": "success", + "message": "Statement executed successfully.", + "statement_handles": ["uuid", "uuid1"], + }, + ), + (202, {}, {"status": "running", "message": "Query statements are still running"}), + (422, {"status": "error", "message": "test"}, {"status": "error", "message": "test"}), + (404, {"status": "error", "message": "test"}, {"status": "error", "message": "test"}), + ], + ) + @mock.patch( + "airflow.providers.snowflake.hooks.snowflake_sql_api.SnowflakeSqlApiHook." + "get_request_url_header_params" + ) + @mock.patch("airflow.providers.snowflake.hooks.snowflake_sql_api.aiohttp.ClientSession.get") + async def test_get_sql_api_query_status_async( + self, mock_get, mock_geturl_header_params, status_code, response, expected_response + ): + """Test Async get_sql_api_query_status_async function by mocking the status, + response and expected response""" + req_id = uuid.uuid4() + params = {"requestId": str(req_id), "page": 2, "pageSize": 10} + mock_geturl_header_params.return_value = HEADERS, params, "/test/airflow/" + mock_get.return_value.__aenter__.return_value.status = status_code + mock_get.return_value.__aenter__.return_value.json = AsyncMock(return_value=response) + hook = SnowflakeSqlApiHook(snowflake_conn_id="test_conn") + response = await hook.get_sql_api_query_status_async("uuid") + assert response == expected_response diff --git a/tests/providers/snowflake/operators/test_snowflake.py b/tests/providers/snowflake/operators/test_snowflake.py index 8f32c6e62d8d..41cbfe671746 100644 --- a/tests/providers/snowflake/operators/test_snowflake.py +++ b/tests/providers/snowflake/operators/test_snowflake.py @@ -19,10 +19,13 @@ from unittest import mock +import pendulum import pytest -from airflow.exceptions import AirflowException +from airflow.exceptions import AirflowException, TaskDeferred from airflow.models.dag import DAG +from airflow.models.dagrun import DagRun +from airflow.models.taskinstance import TaskInstance from airflow.providers.snowflake.operators.snowflake import ( SnowflakeCheckOperator, SnowflakeIntervalCheckOperator, @@ -30,7 +33,9 @@ SnowflakeSqlApiOperator, SnowflakeValueCheckOperator, ) +from airflow.providers.snowflake.triggers.snowflake_trigger import SnowflakeSqlApiTrigger from airflow.utils import timezone +from airflow.utils.types import DagRunType DEFAULT_DATE = timezone.datetime(2015, 1, 1) DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat() @@ -88,6 +93,34 @@ def test_get_db_hook( mock_get_db_hook.assert_called_once() +def create_context(task, dag=None): + if dag is None: + dag = DAG(dag_id="dag") + tzinfo = pendulum.timezone("UTC") + execution_date = timezone.datetime(2022, 1, 1, 1, 0, 0, tzinfo=tzinfo) + dag_run = DagRun( + dag_id=dag.dag_id, + execution_date=execution_date, + run_id=DagRun.generate_run_id(DagRunType.MANUAL, execution_date), + ) + + task_instance = TaskInstance(task=task) + task_instance.dag_run = dag_run + task_instance.xcom_push = mock.Mock() + return { + "dag": dag, + "ts": execution_date.isoformat(), + "task": task, + "ti": task_instance, + "task_instance": task_instance, + "run_id": dag_run.run_id, + "dag_run": dag_run, + "execution_date": execution_date, + "data_interval_end": execution_date, + "logical_date": execution_date, + } + + class TestSnowflakeSqlApiOperator: @pytest.fixture def mock_execute_query(self): @@ -142,3 +175,64 @@ def test_snowflake_sql_api_to_fails_when_one_query_fails( mock_get_sql_api_query_status.side_effect = [{"status": "error"}, {"status": "success"}] with pytest.raises(AirflowException): operator.execute(context=None) + + @pytest.mark.parametrize("mock_sql, statement_count", [(SQL_MULTIPLE_STMTS, 4), (SINGLE_STMT, 1)]) + @mock.patch("airflow.providers.snowflake.hooks.snowflake_sql_api.SnowflakeSqlApiHook.execute_query") + def test_snowflake_sql_api_execute_operator_async(self, mock_db_hook, mock_sql, statement_count): + """ + Asserts that a task is deferred and an SnowflakeSqlApiTrigger will be fired + when the SnowflakeSqlApiOperator is executed. + """ + operator = SnowflakeSqlApiOperator( + task_id=TASK_ID, + snowflake_conn_id=CONN_ID, + sql=mock_sql, + statement_count=statement_count, + deferrable=True, + ) + + with pytest.raises(TaskDeferred) as exc: + operator.execute(create_context(operator)) + + assert isinstance( + exc.value.trigger, SnowflakeSqlApiTrigger + ), "Trigger is not a SnowflakeSqlApiTrigger" + + def test_snowflake_sql_api_execute_complete_failure(self): + """Test SnowflakeSqlApiOperator raise AirflowException of error event""" + + operator = SnowflakeSqlApiOperator( + task_id=TASK_ID, + snowflake_conn_id=CONN_ID, + sql=SQL_MULTIPLE_STMTS, + statement_count=4, + deferrable=True, + ) + with pytest.raises(AirflowException): + operator.execute_complete( + context=None, + event={"status": "error", "message": "Test failure message", "type": "FAILED_WITH_ERROR"}, + ) + + @pytest.mark.parametrize( + "mock_event", + [ + None, + ({"status": "success", "statement_query_ids": ["uuid", "uuid"]}), + ], + ) + @mock.patch("airflow.providers.snowflake.hooks.snowflake_sql_api.SnowflakeSqlApiHook.check_query_output") + def test_snowflake_sql_api_execute_complete(self, mock_conn, mock_event): + """Tests execute_complete assert with successful message""" + + operator = SnowflakeSqlApiOperator( + task_id=TASK_ID, + snowflake_conn_id=CONN_ID, + sql=SQL_MULTIPLE_STMTS, + statement_count=4, + deferrable=True, + ) + + with mock.patch.object(operator.log, "info") as mock_log_info: + operator.execute_complete(context=None, event=mock_event) + mock_log_info.assert_called_with("%s completed successfully.", TASK_ID) diff --git a/tests/providers/snowflake/triggers/__init__.py b/tests/providers/snowflake/triggers/__init__.py new file mode 100644 index 000000000000..13a83393a912 --- /dev/null +++ b/tests/providers/snowflake/triggers/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/tests/providers/snowflake/triggers/test_snowflake.py b/tests/providers/snowflake/triggers/test_snowflake.py new file mode 100644 index 000000000000..9fc14591625e --- /dev/null +++ b/tests/providers/snowflake/triggers/test_snowflake.py @@ -0,0 +1,120 @@ +# 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 + +import asyncio +from datetime import timedelta +from unittest import mock + +import pytest + +from airflow.providers.snowflake.triggers.snowflake_trigger import SnowflakeSqlApiTrigger +from airflow.triggers.base import TriggerEvent + +TASK_ID = "snowflake_check" +POLL_INTERVAL = 1.0 +LIFETIME = timedelta(minutes=59) +RENEWAL_DELTA = timedelta(minutes=54) +MODULE = "airflow.providers.snowflake" + + +class TestSnowflakeSqlApiTrigger: + TRIGGER = SnowflakeSqlApiTrigger( + poll_interval=POLL_INTERVAL, + query_ids=["uuid"], + snowflake_conn_id="test_conn", + token_life_time=LIFETIME, + token_renewal_delta=RENEWAL_DELTA, + ) + + def test_snowflake_sql_trigger_serialization(self): + """ + Asserts that the SnowflakeSqlApiTrigger correctly serializes its arguments + and classpath. + """ + classpath, kwargs = self.TRIGGER.serialize() + assert classpath == "airflow.providers.snowflake.triggers.snowflake_trigger.SnowflakeSqlApiTrigger" + assert kwargs == { + "poll_interval": POLL_INTERVAL, + "query_ids": ["uuid"], + "snowflake_conn_id": "test_conn", + "token_life_time": LIFETIME, + "token_renewal_delta": RENEWAL_DELTA, + } + + @pytest.mark.asyncio + @mock.patch(f"{MODULE}.triggers.snowflake_trigger.SnowflakeSqlApiTrigger.get_query_status") + @mock.patch(f"{MODULE}.hooks.snowflake_sql_api.SnowflakeSqlApiHook.get_sql_api_query_status_async") + async def test_snowflake_sql_trigger_running( + self, mock_get_sql_api_query_status_async, mock_get_query_status + ): + """Tests that the SnowflakeSqlApiTrigger in running by mocking get_query_status to true""" + mock_get_query_status.return_value = {"status": "running"} + + task = asyncio.create_task(self.TRIGGER.run().__anext__()) + await asyncio.sleep(0.5) + + # TriggerEvent was not returned + assert task.done() is False + asyncio.get_event_loop().stop() + + @pytest.mark.asyncio + @mock.patch(f"{MODULE}.triggers.snowflake_trigger.SnowflakeSqlApiTrigger.get_query_status") + @mock.patch(f"{MODULE}.hooks.snowflake_sql_api.SnowflakeSqlApiHook.get_sql_api_query_status_async") + async def test_snowflake_sql_trigger_completed( + self, mock_get_sql_api_query_status_async, mock_get_query_status + ): + """ + Test SnowflakeSqlApiTrigger run method with success status and mock the get_sql_api_query_status + result and get_query_status to False. + """ + mock_get_query_status.return_value = {"status": "success", "statement_handles": ["uuid", "uuid1"]} + statement_query_ids = ["uuid", "uuid1"] + mock_get_sql_api_query_status_async.return_value = { + "message": "Statement executed successfully.", + "status": "success", + "statement_handles": statement_query_ids, + } + + generator = self.TRIGGER.run() + actual = await generator.asend(None) + assert TriggerEvent({"status": "success", "statement_query_ids": statement_query_ids}) == actual + + @pytest.mark.asyncio + @mock.patch(f"{MODULE}.hooks.snowflake_sql_api.SnowflakeSqlApiHook.get_sql_api_query_status_async") + async def test_snowflake_sql_trigger_failure_status(self, mock_get_sql_api_query_status_async): + """Test SnowflakeSqlApiTrigger task is executed and triggered with failure status.""" + mock_response = { + "status": "error", + "message": "An error occurred when executing the statement. Check " + "the error code and error message for details", + } + mock_get_sql_api_query_status_async.return_value = mock_response + + generator = self.TRIGGER.run() + actual = await generator.asend(None) + assert TriggerEvent(mock_response) == actual + + @pytest.mark.asyncio + @mock.patch(f"{MODULE}.hooks.snowflake_sql_api.SnowflakeSqlApiHook.get_sql_api_query_status_async") + async def test_snowflake_sql_trigger_exception(self, mock_get_sql_api_query_status_async): + """Tests the SnowflakeSqlApiTrigger does not fire if there is an exception.""" + mock_get_sql_api_query_status_async.side_effect = Exception("Test exception") + + task = [i async for i in self.TRIGGER.run()] + assert len(task) == 1 + assert TriggerEvent({"status": "error", "message": "Test exception"}) in task From 56eae9c9a4b784e7b239335560c31ed30f57e0a1 Mon Sep 17 00:00:00 2001 From: JDarDagran Date: Tue, 4 Jul 2023 22:52:58 +0200 Subject: [PATCH 046/533] Deprecate `runtime_parameters` in favor of options in `hook_params` (#32345) for PostgresOperator. Signed-off-by: Jakub Dardzinski --- airflow/providers/postgres/hooks/postgres.py | 9 +++++- .../providers/postgres/operators/postgres.py | 32 ++++++------------- .../postgres_operator_howto_guide.rst | 9 +++--- .../providers/postgres/hooks/test_postgres.py | 14 ++++++++ .../postgres/operators/test_postgres.py | 1 + .../providers/postgres/example_postgres.py | 2 +- 6 files changed, 39 insertions(+), 28 deletions(-) diff --git a/airflow/providers/postgres/hooks/postgres.py b/airflow/providers/postgres/hooks/postgres.py index 0a7aa0732a43..2e76029028da 100644 --- a/airflow/providers/postgres/hooks/postgres.py +++ b/airflow/providers/postgres/hooks/postgres.py @@ -59,6 +59,9 @@ class PostgresHook(DbApiHook): :param postgres_conn_id: The :ref:`postgres conn id ` reference to a specific postgres database. + :param options: Optional. Specifies command-line options to send to the server + at connection start. For example, setting this to ``-c search_path=myschema`` + sets the session's value of the ``search_path`` to ``myschema``. """ conn_name_attr = "postgres_conn_id" @@ -67,7 +70,7 @@ class PostgresHook(DbApiHook): hook_name = "Postgres" supports_autocommit = True - def __init__(self, *args, **kwargs) -> None: + def __init__(self, *args, options: str | None = None, **kwargs) -> None: if "schema" in kwargs: warnings.warn( 'The "schema" arg has been renamed to "database" as it contained the database name.' @@ -80,6 +83,7 @@ def __init__(self, *args, **kwargs) -> None: self.connection: Connection | None = kwargs.pop("connection", None) self.conn: connection = None self.database: str | None = kwargs.pop("database", None) + self.options = options @property def schema(self): @@ -131,6 +135,9 @@ def get_conn(self) -> connection: if raw_cursor: conn_args["cursor_factory"] = self._get_cursor(raw_cursor) + if self.options: + conn_args["options"] = self.options + for arg_name, arg_val in conn.extra_dejson.items(): if arg_name not in [ "iam", diff --git a/airflow/providers/postgres/operators/postgres.py b/airflow/providers/postgres/operators/postgres.py index e70ac720278f..ef92b51dda75 100644 --- a/airflow/providers/postgres/operators/postgres.py +++ b/airflow/providers/postgres/operators/postgres.py @@ -20,8 +20,6 @@ import warnings from typing import Mapping, Sequence -from psycopg2.sql import SQL, Identifier - from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator @@ -41,6 +39,7 @@ class PostgresOperator(SQLExecuteQueryOperator): :param database: name of database which overwrite defined one in connection :param runtime_parameters: a mapping of runtime params added to the final sql being executed. For example, you could set the schema via `{"search_path": "CUSTOM_SCHEMA"}`. + Deprecated - use `hook_params={'options': '-c '}` instead. """ template_fields: Sequence[str] = ("sql",) @@ -61,26 +60,15 @@ def __init__( kwargs["hook_params"] = {"schema": database, **hook_params} if runtime_parameters: - sql = kwargs.pop("sql") - parameters = kwargs.pop("parameters", {}) - - final_sql = [] - sql_param = {} - for param in runtime_parameters: - set_param_sql = f"SET {{}} TO %({param})s;" - dynamic_sql = SQL(set_param_sql).format(Identifier(f"{param}")) - final_sql.append(dynamic_sql) - for param, val in runtime_parameters.items(): - sql_param.update({f"{param}": f"{val}"}) - if parameters: - sql_param.update(parameters) - if isinstance(sql, str): - final_sql.append(SQL(sql)) - else: - final_sql.extend(list(map(SQL, sql))) - - kwargs["sql"] = final_sql - kwargs["parameters"] = sql_param + warnings.warn( + """`runtime_parameters` is deprecated. + Please use `hook_params={'options': '-c }`.""", + AirflowProviderDeprecationWarning, + stacklevel=2, + ) + hook_params = kwargs.pop("hook_params", {}) + options = " ".join(f"-c {param}={val}" for param, val in runtime_parameters.items()) + kwargs["hook_params"] = {"options": options, **hook_params} super().__init__(conn_id=postgres_conn_id, **kwargs) warnings.warn( diff --git a/docs/apache-airflow-providers-postgres/operators/postgres_operator_howto_guide.rst b/docs/apache-airflow-providers-postgres/operators/postgres_operator_howto_guide.rst index 7fb4150d8aec..962bbf87de77 100644 --- a/docs/apache-airflow-providers-postgres/operators/postgres_operator_howto_guide.rst +++ b/docs/apache-airflow-providers-postgres/operators/postgres_operator_howto_guide.rst @@ -159,8 +159,9 @@ class. Passing Server Configuration Parameters into PostgresOperator ------------------------------------------------------------- -PostgresOperator provides the optional ``runtime_parameters`` attribute which makes it possible to set -the `server configuration parameter values `_ for the SQL request during runtime. +PostgresOperator provides ``hook_params`` attribute that allows you to pass add parameters to PostgresHook. +You can pass ``options`` argument this way so that you specify `command-line options `_ +sent to the server at connection start. .. exampleinclude:: /../../tests/system/providers/postgres/example_postgres.py :language: python @@ -186,5 +187,5 @@ In this how-to guide we explored the Apache Airflow PostgreOperator. Let's quick It is best practice to create subdirectory called ``sql`` in your ``dags`` directory where you can store your sql files. This will make your code more elegant and more maintainable. And finally, we looked at the different ways you can dynamically pass parameters into our PostgresOperator -tasks using ``parameters`` or ``params`` attribute and how you can control the server configuration parameters by passing -the ``runtime_parameters`` attribute. +tasks using ``parameters`` or ``params`` attribute and how you can control the session parameters by passing +options in the ``hook_params`` attribute. diff --git a/tests/providers/postgres/hooks/test_postgres.py b/tests/providers/postgres/hooks/test_postgres.py index 804ee9da9a00..a6fa619634c4 100644 --- a/tests/providers/postgres/hooks/test_postgres.py +++ b/tests/providers/postgres/hooks/test_postgres.py @@ -101,6 +101,20 @@ def test_get_conn_from_connection_with_database(self, mock_connect): user="login-conn", password="password-conn", host="host", dbname="database-override", port=None ) + @mock.patch("airflow.providers.postgres.hooks.postgres.psycopg2.connect") + def test_get_conn_from_connection_with_options(self, mock_connect): + conn = Connection(login="login-conn", password="password-conn", host="host", schema="database") + hook = PostgresHook(connection=conn, options="-c statement_timeout=3000ms") + hook.get_conn() + mock_connect.assert_called_once_with( + user="login-conn", + password="password-conn", + host="host", + dbname="database", + port=None, + options="-c statement_timeout=3000ms", + ) + @mock.patch("airflow.providers.postgres.hooks.postgres.psycopg2.connect") @mock.patch("airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook") @pytest.mark.parametrize("aws_conn_id", [NOTSET, None, "mock_aws_conn"]) diff --git a/tests/providers/postgres/operators/test_postgres.py b/tests/providers/postgres/operators/test_postgres.py index 4b615917ea28..b58abf3497dc 100644 --- a/tests/providers/postgres/operators/test_postgres.py +++ b/tests/providers/postgres/operators/test_postgres.py @@ -112,3 +112,4 @@ def test_runtime_parameter_setting(self): runtime_parameters={"statement_timeout": "3000ms"}, ) op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) + assert op.get_db_hook().get_first("SHOW statement_timeout;")[0] == "3s" diff --git a/tests/system/providers/postgres/example_postgres.py b/tests/system/providers/postgres/example_postgres.py index ceab5271958a..fd0a80783c9f 100644 --- a/tests/system/providers/postgres/example_postgres.py +++ b/tests/system/providers/postgres/example_postgres.py @@ -73,7 +73,7 @@ task_id="get_birth_date", sql="SELECT * FROM pet WHERE birth_date BETWEEN SYMMETRIC %(begin_date)s AND %(end_date)s", parameters={"begin_date": "2020-01-01", "end_date": "2020-12-31"}, - runtime_parameters={"statement_timeout": "3000ms"}, + hook_params={"options": "-c statement_timeout=3000ms"}, ) # [END postgres_operator_howto_guide_get_birth_date] From c2b9fcc108eae4373350fd0559d6c166853bec40 Mon Sep 17 00:00:00 2001 From: "D. Ferruzzi" Date: Tue, 4 Jul 2023 13:56:01 -0700 Subject: [PATCH 047/533] Add ferruzzi as committer (#32353) --- dev/breeze/src/airflow_breeze/global_constants.py | 1 + docs/apache-airflow/project.rst | 1 + 2 files changed, 2 insertions(+) diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 510996a4b591..0a66f7d98ced 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -225,6 +225,7 @@ def get_default_platform_machine() -> str: "ephraimbuddy", "feluelle", "feng-tao", + "ferruzzi", "houqp", "hussein-awala", "jedcunningham", diff --git a/docs/apache-airflow/project.rst b/docs/apache-airflow/project.rst index 75bc4b2e7fc3..3516a6b8c093 100644 --- a/docs/apache-airflow/project.rst +++ b/docs/apache-airflow/project.rst @@ -51,6 +51,7 @@ Committers - Dan Davydov (@aoen) - Daniel Imberman (@dimberman) - Daniel Standish (@dstandish) +- Dennis Ferruzzi (@ferruzzi) - Elad Kalif (@eladkal) - Ephraim Anierobi (@ephraimbuddy) - Felix Uellendall (@feluelle) From bc3b2d16d3563d5b9bccd283db3f9e290d1d823d Mon Sep 17 00:00:00 2001 From: Joffrey Bienvenu Date: Tue, 4 Jul 2023 22:56:24 +0200 Subject: [PATCH 048/533] Add kerberos dependency to Impala Provider (#32304) --- airflow/providers/apache/impala/provider.yaml | 5 ++++ .../apache/impala/hooks/test_impala.py | 25 +++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/airflow/providers/apache/impala/provider.yaml b/airflow/providers/apache/impala/provider.yaml index b08374bc8223..4d02db35c9df 100644 --- a/airflow/providers/apache/impala/provider.yaml +++ b/airflow/providers/apache/impala/provider.yaml @@ -44,3 +44,8 @@ hooks: connection-types: - hook-class-name: airflow.providers.apache.impala.hooks.impala.ImpalaHook connection-type: impala + +additional-extras: + - name: kerberos + dependencies: + - kerberos>=1.3.0 diff --git a/tests/providers/apache/impala/hooks/test_impala.py b/tests/providers/apache/impala/hooks/test_impala.py index a3e1f119360d..97f1822eef37 100644 --- a/tests/providers/apache/impala/hooks/test_impala.py +++ b/tests/providers/apache/impala/hooks/test_impala.py @@ -54,6 +54,31 @@ def test_get_conn(mock_connect): ) +@patch("airflow.providers.apache.impala.hooks.impala.connect", autospec=True) +def test_get_conn_kerberos(mock_connect): + hook = ImpalaHook() + hook.get_connection = MagicMock( + return_value=Connection( + login="login", + password="password", + host="host", + port=21050, + schema="test", + extra={"auth_mechanism": "GSSAPI", "use_ssl": True}, + ) + ) + hook.get_conn() + mock_connect.assert_called_once_with( + host="host", + port=21050, + user="login", + password="password", + database="test", + use_ssl=True, + auth_mechanism="GSSAPI", + ) + + @patch("airflow.providers.common.sql.hooks.sql.DbApiHook.insert_rows") def test_insert_rows(mock_insert_rows, impala_hook_fixture): table = "table" From ab2c861dd8a96f22b0fda692368ce9b103175322 Mon Sep 17 00:00:00 2001 From: Vijayasarathi Balasubramanian Date: Tue, 4 Jul 2023 16:57:49 -0400 Subject: [PATCH 049/533] Clarify Listener API behavior (#32269) * Fixes: #31180 - Plugin for listeners - on_dag_run_running hook ignored Documentation update for Plugin for listeners - on_dag_run_running hook ignored Co-authored-by: Tzu-ping Chung --- docs/apache-airflow/administration-and-deployment/listeners.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/listeners.rst b/docs/apache-airflow/administration-and-deployment/listeners.rst index c34689c53fa4..2847eb514d0f 100644 --- a/docs/apache-airflow/administration-and-deployment/listeners.rst +++ b/docs/apache-airflow/administration-and-deployment/listeners.rst @@ -54,7 +54,7 @@ Airflow defines the specification as `hookspec ` -Listener API is meant to be called across all DAGs and all operators. You can't listen to events generated by specific DAGs. For that behavior, try methods like ``on_success_callback`` and ``pre_execute``. These provide callbacks for particular DAG authors or operator creators. +Listener API is meant to be called across all DAGs and all operators. You can't listen to events generated by specific DAGs. For that behavior, try methods like ``on_success_callback`` and ``pre_execute``. These provide callbacks for particular DAG authors or operator creators. The logs and ``print()`` calls will be handled as part of the listeners. |experimental| From f8593503cbe252c2f4dc5ff48a3f292c9e13baad Mon Sep 17 00:00:00 2001 From: Wei Lee Date: Wed, 5 Jul 2023 13:55:26 +0800 Subject: [PATCH 050/533] Add default_deferrable config (#31712) --- airflow/config_templates/config.yml | 7 +++++ airflow/config_templates/default_airflow.cfg | 3 ++ airflow/operators/trigger_dagrun.py | 6 ++-- .../providers/amazon/aws/operators/athena.py | 3 +- .../providers/amazon/aws/operators/batch.py | 5 ++-- airflow/providers/amazon/aws/operators/ecs.py | 19 ++++-------- airflow/providers/amazon/aws/operators/eks.py | 11 ++++--- airflow/providers/amazon/aws/operators/emr.py | 9 +++--- .../providers/amazon/aws/operators/glue.py | 3 +- .../amazon/aws/operators/glue_crawler.py | 3 +- airflow/providers/amazon/aws/operators/rds.py | 5 ++-- .../amazon/aws/operators/redshift_cluster.py | 12 ++++---- .../amazon/aws/operators/sagemaker.py | 11 +++---- airflow/providers/amazon/aws/sensors/batch.py | 3 +- airflow/providers/amazon/aws/sensors/ec2.py | 3 +- airflow/providers/amazon/aws/sensors/emr.py | 7 +++-- airflow/providers/amazon/aws/sensors/s3.py | 7 +++-- .../providers/apache/livy/operators/livy.py | 4 +-- .../cncf/kubernetes/operators/pod.py | 3 +- .../databricks/operators/databricks.py | 5 ++-- airflow/providers/dbt/cloud/operators/dbt.py | 3 +- airflow/providers/dbt/cloud/sensors/dbt.py | 3 +- .../google/cloud/operators/bigquery.py | 12 ++++---- .../google/cloud/operators/bigquery_dts.py | 3 +- .../google/cloud/operators/cloud_build.py | 3 +- .../google/cloud/operators/cloud_composer.py | 7 +++-- .../google/cloud/operators/cloud_sql.py | 3 +- .../google/cloud/operators/dataflow.py | 6 ++-- .../google/cloud/operators/dataproc.py | 17 ++++++----- .../cloud/operators/kubernetes_engine.py | 6 ++-- .../google/cloud/operators/mlengine.py | 8 ++--- .../google/cloud/sensors/bigquery.py | 5 ++-- airflow/providers/google/cloud/sensors/gcs.py | 12 ++++---- .../providers/google/cloud/sensors/pubsub.py | 4 +-- .../google/cloud/transfers/bigquery_to_gcs.py | 3 +- .../google/cloud/transfers/gcs_to_bigquery.py | 5 ++-- .../microsoft/azure/operators/data_factory.py | 3 +- .../microsoft/azure/sensors/data_factory.py | 3 +- .../providers/microsoft/azure/sensors/wasb.py | 5 ++-- .../authoring-and-scheduling/deferring.rst | 30 +++++++++++++++++++ tests/models/test_baseoperator.py | 1 - 41 files changed, 160 insertions(+), 111 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 1ad8705a7047..d588a7f26e3b 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1305,6 +1305,13 @@ operators: type: string example: ~ default: "airflow" + default_deferrable: + description: | + The default value of attribute "deferrable" in operators and sensors. + version_added: ~ + type: boolean + example: ~ + default: "false" default_cpus: description: ~ version_added: ~ diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 0cc99a3f4e5f..ae6bdec08524 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -703,6 +703,9 @@ password = # The default owner assigned to each new operator, unless # provided explicitly or passed via ``default_args`` default_owner = airflow + +# The default value of attribute "deferrable" in operators and sensors. +default_deferrable = false default_cpus = 1 default_ram = 512 default_disk = 512 diff --git a/airflow/operators/trigger_dagrun.py b/airflow/operators/trigger_dagrun.py index 0165bb470ed9..548ef9189418 100644 --- a/airflow/operators/trigger_dagrun.py +++ b/airflow/operators/trigger_dagrun.py @@ -25,6 +25,7 @@ from sqlalchemy.orm.exc import NoResultFound from airflow.api.common.trigger_dag import trigger_dag +from airflow.configuration import conf from airflow.exceptions import AirflowException, DagNotFound, DagRunAlreadyExists from airflow.models.baseoperator import BaseOperator, BaseOperatorLink from airflow.models.dag import DagModel @@ -113,7 +114,7 @@ def __init__( poke_interval: int = 60, allowed_states: list | None = None, failed_states: list | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -135,7 +136,6 @@ def __init__( self.execution_date = execution_date def execute(self, context: Context): - if isinstance(self.execution_date, datetime.datetime): parsed_execution_date = self.execution_date elif isinstance(self.execution_date, str): @@ -187,7 +187,6 @@ def execute(self, context: Context): ti.xcom_push(key=XCOM_RUN_ID, value=dag_run.run_id) if self.wait_for_completion: - # Kick off the deferral process if self._defer: self.defer( @@ -219,7 +218,6 @@ def execute(self, context: Context): @provide_session def execute_complete(self, context: Context, session: Session, event: tuple[str, dict[str, Any]]): - # This execution date is parsed from the return trigger event provided_execution_date = event[1]["execution_dates"][0] try: diff --git a/airflow/providers/amazon/aws/operators/athena.py b/airflow/providers/amazon/aws/operators/athena.py index 0467fe6d11ae..6dd1432ea480 100644 --- a/airflow/providers/amazon/aws/operators/athena.py +++ b/airflow/providers/amazon/aws/operators/athena.py @@ -21,6 +21,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow import AirflowException +from airflow.configuration import conf from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.athena import AthenaHook from airflow.providers.amazon.aws.triggers.athena import AthenaTrigger @@ -74,7 +75,7 @@ def __init__( sleep_time: int = 30, max_polling_attempts: int | None = None, log_query: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs: Any, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/amazon/aws/operators/batch.py b/airflow/providers/amazon/aws/operators/batch.py index b9b3322c4910..9dd954d05cd8 100644 --- a/airflow/providers/amazon/aws/operators/batch.py +++ b/airflow/providers/amazon/aws/operators/batch.py @@ -29,6 +29,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.batch_client import BatchClientHook @@ -154,7 +155,7 @@ def __init__( region_name: str | None = None, tags: dict | None = None, wait_for_completion: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: int = 30, awslogs_enabled: bool = False, awslogs_fetch_interval: timedelta = timedelta(seconds=30), @@ -437,7 +438,7 @@ def __init__( max_retries: int | None = None, aws_conn_id: str | None = None, region_name: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): if "status_retries" in kwargs: diff --git a/airflow/providers/amazon/aws/operators/ecs.py b/airflow/providers/amazon/aws/operators/ecs.py index 91533cfa6211..e5833bf4c3d5 100644 --- a/airflow/providers/amazon/aws/operators/ecs.py +++ b/airflow/providers/amazon/aws/operators/ecs.py @@ -26,20 +26,14 @@ import boto3 +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator, XCom from airflow.providers.amazon.aws.exceptions import EcsOperatorError, EcsTaskFailToStart from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook -from airflow.providers.amazon.aws.hooks.ecs import ( - EcsClusterStates, - EcsHook, - should_retry_eni, -) +from airflow.providers.amazon.aws.hooks.ecs import EcsClusterStates, EcsHook, should_retry_eni from airflow.providers.amazon.aws.hooks.logs import AwsLogsHook -from airflow.providers.amazon.aws.triggers.ecs import ( - ClusterWaiterTrigger, - TaskDoneTrigger, -) +from airflow.providers.amazon.aws.triggers.ecs import ClusterWaiterTrigger, TaskDoneTrigger from airflow.providers.amazon.aws.utils.task_log_fetcher import AwsTaskLogFetcher from airflow.utils.helpers import prune_dict from airflow.utils.session import provide_session @@ -118,7 +112,7 @@ def __init__( wait_for_completion: bool = True, waiter_delay: int = 15, waiter_max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -201,7 +195,7 @@ def __init__( wait_for_completion: bool = True, waiter_delay: int = 15, waiter_max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -482,7 +476,7 @@ def __init__( wait_for_completion: bool = True, waiter_delay: int = 6, waiter_max_attempts: int = 100, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) @@ -727,7 +721,6 @@ def _check_success_task(self) -> None: raise AirflowException(response) for task in response["tasks"]: - if task.get("stopCode", "") == "TaskFailedToStart": # Reset task arn here otherwise the retry run will not start # a new task but keep polling the old dead one diff --git a/airflow/providers/amazon/aws/operators/eks.py b/airflow/providers/amazon/aws/operators/eks.py index bea422398726..56e9269f88d7 100644 --- a/airflow/providers/amazon/aws/operators/eks.py +++ b/airflow/providers/amazon/aws/operators/eks.py @@ -25,6 +25,7 @@ from botocore.exceptions import ClientError, WaiterError +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.eks import EksHook @@ -83,7 +84,6 @@ def _create_compute( log = logging.getLogger(__name__) eks_hook = EksHook(aws_conn_id=aws_conn_id, region_name=region) if compute == "nodegroup" and nodegroup_name: - # this is to satisfy mypy subnets = subnets or [] create_nodegroup_kwargs = create_nodegroup_kwargs or {} @@ -107,7 +107,6 @@ def _create_compute( status_args=["nodegroup.status"], ) elif compute == "fargate" and fargate_profile_name: - # this is to satisfy mypy create_fargate_profile_kwargs = create_fargate_profile_kwargs or {} fargate_selectors = fargate_selectors or [] @@ -366,7 +365,7 @@ def __init__( region: str | None = None, waiter_delay: int = 30, waiter_max_attempts: int = 80, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: self.nodegroup_subnets = nodegroup_subnets @@ -489,7 +488,7 @@ def __init__( region: str | None = None, waiter_delay: int = 10, waiter_max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: self.cluster_name = cluster_name @@ -690,7 +689,7 @@ def __init__( region: str | None = None, waiter_delay: int = 30, waiter_max_attempts: int = 40, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: self.cluster_name = cluster_name @@ -780,7 +779,7 @@ def __init__( region: str | None = None, waiter_delay: int = 30, waiter_max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/amazon/aws/operators/emr.py b/airflow/providers/amazon/aws/operators/emr.py index f9eacdb79d19..8330a586e442 100644 --- a/airflow/providers/amazon/aws/operators/emr.py +++ b/airflow/providers/amazon/aws/operators/emr.py @@ -24,6 +24,7 @@ from typing import TYPE_CHECKING, Any, Sequence from uuid import uuid4 +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook @@ -96,7 +97,7 @@ def __init__( waiter_delay: int | None = None, waiter_max_attempts: int | None = None, execution_role_arn: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): if not exactly_one(job_flow_id is None, job_flow_name is None): @@ -510,7 +511,7 @@ def __init__( max_tries: int | None = None, tags: dict | None = None, max_polling_attempts: int | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs: Any, ) -> None: super().__init__(**kwargs) @@ -695,7 +696,7 @@ def __init__( waiter_delay: int | None | ArgNotSet = NOTSET, waiter_countdown: int | None = None, waiter_check_interval_seconds: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs: Any, ): if waiter_max_attempts is NOTSET: @@ -900,7 +901,7 @@ def __init__( aws_conn_id: str = "aws_default", waiter_delay: int = 60, waiter_max_attempts: int = 20, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) diff --git a/airflow/providers/amazon/aws/operators/glue.py b/airflow/providers/amazon/aws/operators/glue.py index 060ac358a40c..265d057de51a 100644 --- a/airflow/providers/amazon/aws/operators/glue.py +++ b/airflow/providers/amazon/aws/operators/glue.py @@ -23,6 +23,7 @@ from typing import TYPE_CHECKING, Sequence from airflow import AirflowException +from airflow.configuration import conf from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.glue import GlueJobHook from airflow.providers.amazon.aws.hooks.s3 import S3Hook @@ -98,7 +99,7 @@ def __init__( create_job_kwargs: dict | None = None, run_job_kwargs: dict | None = None, wait_for_completion: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), verbose: bool = False, update_config: bool = False, job_poll_interval: int | float = 6, diff --git a/airflow/providers/amazon/aws/operators/glue_crawler.py b/airflow/providers/amazon/aws/operators/glue_crawler.py index a7efb9f5c0cd..71e2607039c3 100644 --- a/airflow/providers/amazon/aws/operators/glue_crawler.py +++ b/airflow/providers/amazon/aws/operators/glue_crawler.py @@ -21,6 +21,7 @@ from typing import TYPE_CHECKING, Sequence from airflow import AirflowException +from airflow.configuration import conf from airflow.providers.amazon.aws.triggers.glue_crawler import GlueCrawlerCompleteTrigger if TYPE_CHECKING: @@ -61,7 +62,7 @@ def __init__( region_name: str | None = None, poll_interval: int = 5, wait_for_completion: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) diff --git a/airflow/providers/amazon/aws/operators/rds.py b/airflow/providers/amazon/aws/operators/rds.py index 9aef6701660e..c58961db2e8f 100644 --- a/airflow/providers/amazon/aws/operators/rds.py +++ b/airflow/providers/amazon/aws/operators/rds.py @@ -24,6 +24,7 @@ from mypy_boto3_rds.type_defs import TagTypeDef +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.rds import RdsHook @@ -554,7 +555,7 @@ def __init__( rds_kwargs: dict | None = None, aws_conn_id: str = "aws_default", wait_for_completion: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), waiter_delay: int = 30, waiter_max_attempts: int = 60, **kwargs, @@ -645,7 +646,7 @@ def __init__( rds_kwargs: dict | None = None, aws_conn_id: str = "aws_default", wait_for_completion: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), waiter_delay: int = 30, waiter_max_attempts: int = 60, **kwargs, diff --git a/airflow/providers/amazon/aws/operators/redshift_cluster.py b/airflow/providers/amazon/aws/operators/redshift_cluster.py index 905c34ff3ac5..cde4a32226e9 100644 --- a/airflow/providers/amazon/aws/operators/redshift_cluster.py +++ b/airflow/providers/amazon/aws/operators/redshift_cluster.py @@ -20,6 +20,7 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.redshift_cluster import RedshiftHook @@ -148,7 +149,7 @@ def __init__( wait_for_completion: bool = False, max_attempt: int = 5, poll_interval: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) @@ -327,7 +328,7 @@ def __init__( poll_interval: int = 15, max_attempt: int = 20, aws_conn_id: str = "aws_default", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) @@ -470,7 +471,7 @@ def __init__( cluster_identifier: str, aws_conn_id: str = "aws_default", wait_for_completion: bool = False, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: int = 10, max_attempts: int = 10, **kwargs, @@ -560,7 +561,7 @@ def __init__( *, cluster_identifier: str, aws_conn_id: str = "aws_default", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: int = 10, max_attempts: int = 15, **kwargs, @@ -647,7 +648,7 @@ def __init__( wait_for_completion: bool = True, aws_conn_id: str = "aws_default", poll_interval: int = 30, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), max_attempts: int = 30, **kwargs, ): @@ -668,7 +669,6 @@ def __init__( self.max_attempts = max_attempts def execute(self, context: Context): - while self._attempts >= 1: try: self.redshift_hook.delete_cluster( diff --git a/airflow/providers/amazon/aws/operators/sagemaker.py b/airflow/providers/amazon/aws/operators/sagemaker.py index 4dac7df00790..ac1b7a73d2de 100644 --- a/airflow/providers/amazon/aws/operators/sagemaker.py +++ b/airflow/providers/amazon/aws/operators/sagemaker.py @@ -25,6 +25,7 @@ from botocore.exceptions import ClientError +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook @@ -198,7 +199,7 @@ def __init__( max_attempts: int | None = None, max_ingestion_time: int | None = None, action_if_job_exists: str = "timestamp", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(config=config, aws_conn_id=aws_conn_id, **kwargs) @@ -392,7 +393,7 @@ def __init__( check_interval: int = CHECK_INTERVAL_SECOND, max_ingestion_time: int | None = None, operation: str = "create", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(config=config, aws_conn_id=aws_conn_id, **kwargs) @@ -551,7 +552,7 @@ def __init__( max_ingestion_time: int | None = None, check_if_job_exists: bool = True, action_if_job_exists: str = "timestamp", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(config=config, aws_conn_id=aws_conn_id, **kwargs) @@ -700,7 +701,7 @@ def __init__( wait_for_completion: bool = True, check_interval: int = CHECK_INTERVAL_SECOND, max_ingestion_time: int | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(config=config, aws_conn_id=aws_conn_id, **kwargs) @@ -862,7 +863,7 @@ def __init__( max_ingestion_time: int | None = None, check_if_job_exists: bool = True, action_if_job_exists: str = "timestamp", - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(config=config, aws_conn_id=aws_conn_id, **kwargs) diff --git a/airflow/providers/amazon/aws/sensors/batch.py b/airflow/providers/amazon/aws/sensors/batch.py index 2033d1e86bee..32da5b4cf252 100644 --- a/airflow/providers/amazon/aws/sensors/batch.py +++ b/airflow/providers/amazon/aws/sensors/batch.py @@ -22,6 +22,7 @@ from deprecated import deprecated +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.batch_client import BatchClientHook from airflow.providers.amazon.aws.triggers.batch import BatchSensorTrigger @@ -58,7 +59,7 @@ def __init__( job_id: str, aws_conn_id: str = "aws_default", region_name: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poke_interval: float = 5, max_retries: int = 5, **kwargs, diff --git a/airflow/providers/amazon/aws/sensors/ec2.py b/airflow/providers/amazon/aws/sensors/ec2.py index c5d77610319e..2b7b63f7e6c7 100644 --- a/airflow/providers/amazon/aws/sensors/ec2.py +++ b/airflow/providers/amazon/aws/sensors/ec2.py @@ -20,6 +20,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook from airflow.providers.amazon.aws.triggers.ec2 import EC2StateSensorTrigger @@ -55,7 +56,7 @@ def __init__( instance_id: str, aws_conn_id: str = "aws_default", region_name: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): if target_state not in self.valid_states: diff --git a/airflow/providers/amazon/aws/sensors/emr.py b/airflow/providers/amazon/aws/sensors/emr.py index 2f44caab0619..9953dfa78260 100644 --- a/airflow/providers/amazon/aws/sensors/emr.py +++ b/airflow/providers/amazon/aws/sensors/emr.py @@ -23,6 +23,7 @@ from deprecated import deprecated +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook from airflow.providers.amazon.aws.links.emr import EmrClusterLink, EmrLogsLink, get_log_uri @@ -271,7 +272,7 @@ def __init__( max_retries: int | None = None, aws_conn_id: str = "aws_default", poll_interval: int = 10, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs: Any, ) -> None: super().__init__(**kwargs) @@ -425,7 +426,7 @@ def __init__( target_states: Iterable[str] | None = None, failed_states: Iterable[str] | None = None, max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) @@ -549,7 +550,7 @@ def __init__( target_states: Iterable[str] | None = None, failed_states: Iterable[str] | None = None, max_attempts: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) diff --git a/airflow/providers/amazon/aws/sensors/s3.py b/airflow/providers/amazon/aws/sensors/s3.py index 7192585afd58..4d15cdc2125d 100644 --- a/airflow/providers/amazon/aws/sensors/s3.py +++ b/airflow/providers/amazon/aws/sensors/s3.py @@ -26,6 +26,8 @@ from deprecated import deprecated +from airflow.configuration import conf + if TYPE_CHECKING: from airflow.utils.context import Context @@ -87,7 +89,7 @@ def __init__( check_fn: Callable[..., bool] | None = None, aws_conn_id: str = "aws_default", verify: str | bool | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ): super().__init__(**kwargs) @@ -238,10 +240,9 @@ def __init__( min_objects: int = 1, previous_objects: set[str] | None = None, allow_delete: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: - super().__init__(**kwargs) self.bucket_name = bucket_name diff --git a/airflow/providers/apache/livy/operators/livy.py b/airflow/providers/apache/livy/operators/livy.py index fa4f35734371..f5e519315f4b 100644 --- a/airflow/providers/apache/livy/operators/livy.py +++ b/airflow/providers/apache/livy/operators/livy.py @@ -20,6 +20,7 @@ from time import sleep from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.apache.livy.hooks.livy import BatchState, LivyHook @@ -88,10 +89,9 @@ def __init__( extra_options: dict[str, Any] | None = None, extra_headers: dict[str, Any] | None = None, retry_args: dict[str, Any] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs: Any, ) -> None: - super().__init__(**kwargs) self.spark_params = { diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index 696611c6c2e9..e3ac7708e74c 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -33,6 +33,7 @@ from slugify import slugify from urllib3.exceptions import HTTPError +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 @@ -305,7 +306,7 @@ def __init__( configmaps: list[str] | None = None, skip_on_exit_code: int | Container[int] | None = None, base_container_name: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 2, log_pod_spec_on_failure: bool = True, on_finish_action: str = "delete_pod", diff --git a/airflow/providers/databricks/operators/databricks.py b/airflow/providers/databricks/operators/databricks.py index ab93d8f49b7c..fb27f0c01a11 100644 --- a/airflow/providers/databricks/operators/databricks.py +++ b/airflow/providers/databricks/operators/databricks.py @@ -24,6 +24,7 @@ from logging import Logger from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.models import BaseOperator, BaseOperatorLink, XCom from airflow.providers.databricks.hooks.databricks import DatabricksHook, RunState @@ -315,7 +316,7 @@ def __init__( access_control_list: list[dict[str, str]] | None = None, wait_for_termination: bool = True, git_source: dict[str, str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: """Creates a new ``DatabricksSubmitRunOperator``.""" @@ -605,7 +606,7 @@ def __init__( databricks_retry_args: dict[Any, Any] | None = None, do_xcom_push: bool = True, wait_for_termination: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: """Creates a new ``DatabricksRunNowOperator``.""" diff --git a/airflow/providers/dbt/cloud/operators/dbt.py b/airflow/providers/dbt/cloud/operators/dbt.py index f316c47f3db4..c977539afb73 100644 --- a/airflow/providers/dbt/cloud/operators/dbt.py +++ b/airflow/providers/dbt/cloud/operators/dbt.py @@ -22,6 +22,7 @@ from pathlib import Path from typing import TYPE_CHECKING, Any +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models import BaseOperator, BaseOperatorLink, XCom from airflow.providers.dbt.cloud.hooks.dbt import ( @@ -99,7 +100,7 @@ def __init__( timeout: int = 60 * 60 * 24 * 7, check_interval: int = 60, additional_run_config: dict[str, Any] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/dbt/cloud/sensors/dbt.py b/airflow/providers/dbt/cloud/sensors/dbt.py index 5838f6d6247d..3b5ae549a35a 100644 --- a/airflow/providers/dbt/cloud/sensors/dbt.py +++ b/airflow/providers/dbt/cloud/sensors/dbt.py @@ -20,6 +20,7 @@ import warnings from typing import TYPE_CHECKING, Any +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.dbt.cloud.hooks.dbt import DbtCloudHook, DbtCloudJobRunException, DbtCloudJobRunStatus from airflow.providers.dbt.cloud.triggers.dbt import DbtCloudRunJobTrigger @@ -50,7 +51,7 @@ def __init__( dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name, run_id: int, account_id: int | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: if deferrable: diff --git a/airflow/providers/google/cloud/operators/bigquery.py b/airflow/providers/google/cloud/operators/bigquery.py index 98929b6e6dec..970e7813ed9c 100644 --- a/airflow/providers/google/cloud/operators/bigquery.py +++ b/airflow/providers/google/cloud/operators/bigquery.py @@ -29,6 +29,7 @@ from google.cloud.bigquery import DEFAULT_RETRY, CopyJob, ExtractJob, LoadJob, QueryJob from google.cloud.bigquery.table import RowIterator +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException from airflow.models import BaseOperator, BaseOperatorLink from airflow.models.xcom import XCom @@ -200,7 +201,7 @@ def __init__( location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, labels: dict | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 4.0, **kwargs, ) -> None: @@ -320,7 +321,7 @@ def __init__( location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, labels: dict | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 4.0, **kwargs, ) -> None: @@ -460,7 +461,7 @@ def __init__( location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, labels: dict | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 4.0, **kwargs, ) -> None: @@ -854,7 +855,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", location: str | None = None, impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 4.0, as_dict: bool = False, use_legacy_sql: bool = True, @@ -1876,7 +1877,6 @@ def __init__( exists_ok: bool | None = None, **kwargs, ) -> None: - self.dataset_id = dataset_id self.project_id = project_id self.location = location @@ -2623,7 +2623,7 @@ def __init__( cancel_on_kill: bool = True, result_retry: Retry = DEFAULT_RETRY, result_timeout: float | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: float = 4.0, **kwargs, ) -> None: diff --git a/airflow/providers/google/cloud/operators/bigquery_dts.py b/airflow/providers/google/cloud/operators/bigquery_dts.py index d9e013afa68c..e10618bc39bd 100644 --- a/airflow/providers/google/cloud/operators/bigquery_dts.py +++ b/airflow/providers/google/cloud/operators/bigquery_dts.py @@ -32,6 +32,7 @@ ) from airflow import AirflowException +from airflow.configuration import conf from airflow.providers.google.cloud.hooks.bigquery_dts import BiqQueryDataTransferServiceHook, get_object_id from airflow.providers.google.cloud.links.bigquery_dts import BigQueryDataTransferConfigLink from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator @@ -279,7 +280,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id="google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/google/cloud/operators/cloud_build.py b/airflow/providers/google/cloud/operators/cloud_build.py index 4242f561c100..14fed55a3af6 100644 --- a/airflow/providers/google/cloud/operators/cloud_build.py +++ b/airflow/providers/google/cloud/operators/cloud_build.py @@ -28,6 +28,7 @@ from google.api_core.retry import Retry from google.cloud.devtools.cloudbuild_v1.types import Build, BuildTrigger, RepoSource +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.cloud_build import CloudBuildHook from airflow.providers.google.cloud.links.cloud_build import ( @@ -176,7 +177,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, poll_interval: float = 4.0, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), location: str = "global", **kwargs, ) -> None: diff --git a/airflow/providers/google/cloud/operators/cloud_composer.py b/airflow/providers/google/cloud/operators/cloud_composer.py index d04a1606fc84..c9b52d855915 100644 --- a/airflow/providers/google/cloud/operators/cloud_composer.py +++ b/airflow/providers/google/cloud/operators/cloud_composer.py @@ -27,6 +27,7 @@ from google.protobuf.field_mask_pb2 import FieldMask from airflow import AirflowException +from airflow.configuration import conf from airflow.providers.google.cloud.hooks.cloud_composer import CloudComposerHook from airflow.providers.google.cloud.links.base import BaseGoogleLink from airflow.providers.google.cloud.operators.cloud_base import GoogleCloudBaseOperator @@ -135,7 +136,7 @@ def __init__( retry: Retry | _MethodDefault = DEFAULT, timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), pooling_period_seconds: int = 30, **kwargs, ) -> None: @@ -264,7 +265,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), pooling_period_seconds: int = 30, **kwargs, ) -> None: @@ -509,7 +510,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), pooling_period_seconds: int = 30, **kwargs, ) -> None: diff --git a/airflow/providers/google/cloud/operators/cloud_sql.py b/airflow/providers/google/cloud/operators/cloud_sql.py index 5c77cbd86c94..b1144663c706 100644 --- a/airflow/providers/google/cloud/operators/cloud_sql.py +++ b/airflow/providers/google/cloud/operators/cloud_sql.py @@ -22,6 +22,7 @@ from googleapiclient.errors import HttpError +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.models import Connection @@ -955,7 +956,7 @@ def __init__( api_version: str = "v1beta4", validate_body: bool = True, impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poke_interval: int = 10, **kwargs, ) -> None: diff --git a/airflow/providers/google/cloud/operators/dataflow.py b/airflow/providers/google/cloud/operators/dataflow.py index 5ae1115a34ae..a5e958821485 100644 --- a/airflow/providers/google/cloud/operators/dataflow.py +++ b/airflow/providers/google/cloud/operators/dataflow.py @@ -28,6 +28,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow import AirflowException +from airflow.configuration import conf from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.providers.apache.beam.hooks.beam import BeamHook, BeamRunnerType from airflow.providers.google.cloud.hooks.dataflow import ( @@ -419,7 +420,6 @@ def set_current_job_id(job_id): variables=pipeline_options, ) while is_running and self.check_if_running == CheckJobRunning.WaitForRun: - is_running = self.dataflow_hook.is_job_dataflow_running( name=self.job_name, variables=pipeline_options, @@ -611,7 +611,7 @@ def __init__( cancel_timeout: int | None = 10 * 60, wait_until_finished: bool | None = None, append_job_name: bool = True, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -801,7 +801,7 @@ def __init__( cancel_timeout: int | None = 10 * 60, wait_until_finished: bool | None = None, impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), append_job_name: bool = True, *args, **kwargs, diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py index db7d785347a8..d14a495bc0cb 100644 --- a/airflow/providers/google/cloud/operators/dataproc.py +++ b/airflow/providers/google/cloud/operators/dataproc.py @@ -36,6 +36,7 @@ from google.protobuf.duration_pb2 import Duration from google.protobuf.field_mask_pb2 import FieldMask +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.dataproc import DataprocHook, DataProcJobBuilder from airflow.providers.google.cloud.hooks.gcs import GCSHook @@ -484,7 +485,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ) -> None: @@ -849,7 +850,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ): @@ -981,7 +982,7 @@ def __init__( job_error_states: set[str] | None = None, impersonation_chain: str | Sequence[str] | None = None, asynchronous: bool = False, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ) -> None: @@ -1731,7 +1732,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ) -> None: @@ -1859,7 +1860,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ) -> None: @@ -1979,7 +1980,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, asynchronous: bool = False, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, cancel_on_kill: bool = True, wait_timeout: int | None = None, @@ -2139,7 +2140,7 @@ def __init__( metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 10, **kwargs, ): @@ -2270,7 +2271,7 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, result_retry: Retry | _MethodDefault = DEFAULT, asynchronous: bool = False, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), polling_interval_seconds: int = 5, **kwargs, ): diff --git a/airflow/providers/google/cloud/operators/kubernetes_engine.py b/airflow/providers/google/cloud/operators/kubernetes_engine.py index bf14828d8702..086a7d99b7e6 100644 --- a/airflow/providers/google/cloud/operators/kubernetes_engine.py +++ b/airflow/providers/google/cloud/operators/kubernetes_engine.py @@ -26,6 +26,7 @@ from google.cloud.container_v1.types import Cluster from kubernetes.client.models import V1Pod +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.cncf.kubernetes.utils.pod_manager import OnFinishAction @@ -34,6 +35,7 @@ except ImportError: # preserve backward compatibility for older versions of cncf.kubernetes provider from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator + from airflow.providers.google.cloud.hooks.kubernetes_engine import GKEHook, GKEPodHook from airflow.providers.google.cloud.links.kubernetes_engine import ( KubernetesEngineClusterLink, @@ -108,7 +110,7 @@ def __init__( gcp_conn_id: str = "google_cloud_default", api_version: str = "v2", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), poll_interval: int = 10, **kwargs, ) -> None: @@ -255,7 +257,7 @@ def __init__( api_version: str = "v2", impersonation_chain: str | Sequence[str] | None = None, poll_interval: int = 10, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/google/cloud/operators/mlengine.py b/airflow/providers/google/cloud/operators/mlengine.py index b776d20dff42..20d9d1988623 100644 --- a/airflow/providers/google/cloud/operators/mlengine.py +++ b/airflow/providers/google/cloud/operators/mlengine.py @@ -27,6 +27,7 @@ from googleapiclient.errors import HttpError +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.mlengine import MLEngineHook from airflow.providers.google.cloud.links.mlengine import ( @@ -722,7 +723,6 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - super().__init__(**kwargs) self._project_id = project_id self._model_name = model_name @@ -804,7 +804,6 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - super().__init__(**kwargs) self._project_id = project_id self._model_name = model_name @@ -883,7 +882,6 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - super().__init__(**kwargs) self._project_id = project_id self._model_name = model_name @@ -961,7 +959,6 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, **kwargs, ) -> None: - super().__init__(**kwargs) self._project_id = project_id self._model_name = model_name @@ -1098,7 +1095,7 @@ def __init__( labels: dict[str, str] | None = None, impersonation_chain: str | Sequence[str] | None = None, hyperparameters: dict | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), cancel_on_kill: bool = True, **kwargs, ) -> None: @@ -1370,7 +1367,6 @@ def __init__( raise AirflowException("Google Cloud project id is required.") def execute(self, context: Context): - hook = MLEngineHook( gcp_conn_id=self._gcp_conn_id, impersonation_chain=self._impersonation_chain, diff --git a/airflow/providers/google/cloud/sensors/bigquery.py b/airflow/providers/google/cloud/sensors/bigquery.py index db109bf2c1e8..e4e1819ef134 100644 --- a/airflow/providers/google/cloud/sensors/bigquery.py +++ b/airflow/providers/google/cloud/sensors/bigquery.py @@ -22,6 +22,7 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook from airflow.providers.google.cloud.triggers.bigquery import ( @@ -71,7 +72,7 @@ def __init__( table_id: str, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: if deferrable and "poke_interval" not in kwargs: @@ -184,7 +185,7 @@ def __init__( partition_id: str, gcp_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: if deferrable and "poke_interval" not in kwargs: diff --git a/airflow/providers/google/cloud/sensors/gcs.py b/airflow/providers/google/cloud/sensors/gcs.py index 08fd37022dfe..7048789601e4 100644 --- a/airflow/providers/google/cloud/sensors/gcs.py +++ b/airflow/providers/google/cloud/sensors/gcs.py @@ -27,6 +27,7 @@ from google.api_core.retry import Retry from google.cloud.storage.retry import DEFAULT_RETRY +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.google.cloud.hooks.gcs import GCSHook from airflow.providers.google.cloud.triggers.gcs import ( @@ -76,10 +77,9 @@ def __init__( google_cloud_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, retry: Retry = DEFAULT_RETRY, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: - super().__init__(**kwargs) self.bucket = bucket self.object = object @@ -208,10 +208,9 @@ def __init__( ts_func: Callable = ts_function, google_cloud_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: - super().__init__(**kwargs) self.bucket = bucket self.object = object @@ -298,7 +297,7 @@ def __init__( prefix: str, google_cloud_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -412,10 +411,9 @@ def __init__( allow_delete: bool = True, google_cloud_conn_id: str = "google_cloud_default", impersonation_chain: str | Sequence[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: - super().__init__(**kwargs) self.bucket = bucket diff --git a/airflow/providers/google/cloud/sensors/pubsub.py b/airflow/providers/google/cloud/sensors/pubsub.py index 2e03b3669d3d..db9f39b19b0b 100644 --- a/airflow/providers/google/cloud/sensors/pubsub.py +++ b/airflow/providers/google/cloud/sensors/pubsub.py @@ -23,6 +23,7 @@ from google.cloud.pubsub_v1.types import ReceivedMessage +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.providers.google.cloud.hooks.pubsub import PubSubHook from airflow.providers.google.cloud.triggers.pubsub import PubsubPullTrigger @@ -103,10 +104,9 @@ def __init__( messages_callback: Callable[[list[ReceivedMessage], Context], Any] | None = None, impersonation_chain: str | Sequence[str] | None = None, poke_interval: float = 10.0, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: - super().__init__(**kwargs) self.gcp_conn_id = gcp_conn_id self.project_id = project_id diff --git a/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py b/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py index 7ec62db9bfbd..8836e3ee354e 100644 --- a/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py +++ b/airflow/providers/google/cloud/transfers/bigquery_to_gcs.py @@ -25,6 +25,7 @@ from google.cloud.bigquery import DEFAULT_RETRY, UnknownJob from airflow import AirflowException +from airflow.configuration import conf from airflow.models import BaseOperator from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook, BigQueryJob from airflow.providers.google.cloud.links.bigquery import BigQueryTableLink @@ -114,7 +115,7 @@ def __init__( job_id: str | None = None, force_rerun: bool = False, reattach_states: set[str] | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py b/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py index 88b6d0932370..da462eae74df 100644 --- a/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py +++ b/airflow/providers/google/cloud/transfers/gcs_to_bigquery.py @@ -36,6 +36,7 @@ from google.cloud.bigquery.table import EncryptionConfiguration, Table, TableReference from airflow import AirflowException +from airflow.configuration import conf from airflow.models import BaseOperator from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook, BigQueryJob from airflow.providers.google.cloud.hooks.gcs import GCSHook @@ -218,7 +219,7 @@ def __init__( impersonation_chain: str | Sequence[str] | None = None, labels=None, description=None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), result_retry: Retry = DEFAULT_RETRY, result_timeout: float | None = None, cancel_on_kill: bool = True, @@ -228,7 +229,6 @@ def __init__( project_id: str | None = None, **kwargs, ) -> None: - super().__init__(**kwargs) self.hook: BigQueryHook | None = None self.configuration: dict[str, Any] = {} @@ -718,7 +718,6 @@ def _validate_src_fmt_configs( def _cleanse_time_partitioning( self, destination_dataset_table: str | None, time_partitioning_in: dict | None ) -> dict: # if it is a partitioned table ($ is in the table name) add partition load option - if time_partitioning_in is None: time_partitioning_in = {} diff --git a/airflow/providers/microsoft/azure/operators/data_factory.py b/airflow/providers/microsoft/azure/operators/data_factory.py index 8906c02ae12c..a2b2c528bf9b 100644 --- a/airflow/providers/microsoft/azure/operators/data_factory.py +++ b/airflow/providers/microsoft/azure/operators/data_factory.py @@ -20,6 +20,7 @@ import warnings from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.models import BaseOperator, BaseOperatorLink, XCom @@ -140,7 +141,7 @@ def __init__( parameters: dict[str, Any] | None = None, timeout: int = 60 * 60 * 24 * 7, check_interval: int = 60, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/microsoft/azure/sensors/data_factory.py b/airflow/providers/microsoft/azure/sensors/data_factory.py index 70ae1f69dc3f..b4ebedce698a 100644 --- a/airflow/providers/microsoft/azure/sensors/data_factory.py +++ b/airflow/providers/microsoft/azure/sensors/data_factory.py @@ -20,6 +20,7 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.microsoft.azure.hooks.data_factory import ( AzureDataFactoryHook, @@ -60,7 +61,7 @@ def __init__( azure_data_factory_conn_id: str = AzureDataFactoryHook.default_conn_name, resource_group_name: str | None = None, factory_name: str | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/airflow/providers/microsoft/azure/sensors/wasb.py b/airflow/providers/microsoft/azure/sensors/wasb.py index 4e2ec2d502c1..0c227f2ea37b 100644 --- a/airflow/providers/microsoft/azure/sensors/wasb.py +++ b/airflow/providers/microsoft/azure/sensors/wasb.py @@ -21,6 +21,7 @@ from datetime import timedelta from typing import TYPE_CHECKING, Any, Sequence +from airflow.configuration import conf from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.microsoft.azure.hooks.wasb import WasbHook from airflow.providers.microsoft.azure.triggers.wasb import WasbBlobSensorTrigger, WasbPrefixSensorTrigger @@ -53,7 +54,7 @@ def __init__( wasb_conn_id: str = "wasb_default", check_options: dict | None = None, public_read: bool = False, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) @@ -151,7 +152,7 @@ def __init__( prefix: str, wasb_conn_id: str = "wasb_default", check_options: dict | None = None, - deferrable: bool = False, + deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), **kwargs, ) -> None: super().__init__(**kwargs) diff --git a/docs/apache-airflow/authoring-and-scheduling/deferring.rst b/docs/apache-airflow/authoring-and-scheduling/deferring.rst index b1e6c6be983d..81526737e8d1 100644 --- a/docs/apache-airflow/authoring-and-scheduling/deferring.rst +++ b/docs/apache-airflow/authoring-and-scheduling/deferring.rst @@ -56,6 +56,36 @@ Writing a deferrable operator takes a bit more work. There are some main points * You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control. * Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors. * In order for any changes to a Trigger to be reflected, the *triggerer* needs to be restarted whenever the Trigger is modified. +* If you want add an operator or sensor that supports both deferrable and non-deferrable modes. It's suggested to add ``deferable: bool = conf.getboolean("operators", "default_deferrable", fallback=False)`` to the ``__init__`` method of the operator and use it to decide whether to run the operator in deferrable mode. You'll be able to configure the default value of ``deferrable`` of all the operators and sensors that supports switch between deferrable and non-deferrable mode through ``default_deferrable`` in the ``operator`` section. Here's an example of a sensor that supports both modes.:: + + import time + from datetime import timedelta + + from airflow.sensors.base import BaseSensorOperator + from airflow.triggers.temporal import TimeDeltaTrigger + + + class WaitOneHourSensor(BaseSensorOperator): + def __init__( + self, + deferable: bool = conf.getboolean("operators", "default_deferrable", fallback=False), + **kwargs + ): + super().__init__(**kwargs) + self.deferrable = deferable + + def execute(self, context): + if deferrable: + self.defer( + trigger=TimeDeltaTrigger(timedelta(hours=1)), + method_name="execute_complete" + ) + else: + time.sleep(3600) + + def execute_complete(self, context, event=None): + # We have no more work to do here. Mark as complete. + return Triggering Deferral diff --git a/tests/models/test_baseoperator.py b/tests/models/test_baseoperator.py index 2b118176721d..c49e0bb03479 100644 --- a/tests/models/test_baseoperator.py +++ b/tests/models/test_baseoperator.py @@ -919,7 +919,6 @@ def test_render_template_fields_logging( caplog, monkeypatch, task, context, expected_exception, expected_rendering, expected_log, not_expected_log ): """Verify if operator attributes are correctly templated.""" - # Trigger templating and verify results def _do_render(): task.render_template_fields(context=context) From 1240dcc167c4b47331db81deff61fc688df118c2 Mon Sep 17 00:00:00 2001 From: "D. Ferruzzi" Date: Tue, 4 Jul 2023 23:10:26 -0700 Subject: [PATCH 051/533] D205 Support - Providers: GRPC to Oracle (inclusive) (#32357) --- .../hashicorp/_internal_client/vault_client.py | 15 +++++++-------- airflow/providers/hashicorp/secrets/vault.py | 5 +++-- airflow/providers/http/sensors/http.py | 3 +-- airflow/providers/imap/hooks/imap.py | 4 ++-- airflow/providers/jenkins/hooks/jenkins.py | 6 ++++-- .../jenkins/operators/jenkins_job_trigger.py | 4 ++-- airflow/providers/microsoft/azure/hooks/adx.py | 5 +++-- .../microsoft/azure/hooks/base_azure.py | 6 ++++-- .../providers/microsoft/azure/hooks/cosmos.py | 5 +---- .../providers/microsoft/azure/hooks/fileshare.py | 2 ++ airflow/providers/microsoft/azure/hooks/wasb.py | 2 ++ .../microsoft/azure/log/wasb_task_handler.py | 13 ++++++------- .../providers/microsoft/azure/operators/asb.py | 10 ++-------- .../microsoft/azure/operators/data_factory.py | 4 ++-- .../microsoft/azure/secrets/key_vault.py | 1 + .../microsoft/azure/sensors/data_factory.py | 4 ++-- .../providers/microsoft/azure/sensors/wasb.py | 8 ++++---- .../microsoft/azure/transfers/local_to_adls.py | 1 + .../azure/transfers/oracle_to_azure_data_lake.py | 4 +--- .../providers/microsoft/azure/triggers/wasb.py | 8 +++++--- airflow/providers/microsoft/azure/utils.py | 2 ++ airflow/providers/microsoft/mssql/hooks/mssql.py | 16 +++++++++------- airflow/providers/microsoft/psrp/hooks/psrp.py | 5 +++-- airflow/providers/mongo/hooks/mongo.py | 1 + .../providers/mysql/transfers/presto_to_mysql.py | 7 ++++--- .../providers/mysql/transfers/trino_to_mysql.py | 7 ++++--- airflow/providers/neo4j/hooks/neo4j.py | 8 ++------ airflow/providers/openlineage/extractors/bash.py | 6 ++++-- .../providers/openlineage/extractors/python.py | 2 ++ airflow/providers/openlineage/plugins/adapter.py | 5 +---- .../providers/openlineage/plugins/listener.py | 5 +---- airflow/providers/openlineage/plugins/macros.py | 11 ++++++----- .../providers/openlineage/plugins/openlineage.py | 7 +++++-- airflow/providers/openlineage/sqlparser.py | 3 ++- airflow/providers/openlineage/utils/utils.py | 5 ++++- airflow/providers/opsgenie/hooks/opsgenie.py | 1 + airflow/providers/opsgenie/operators/opsgenie.py | 13 +++++++++---- 37 files changed, 115 insertions(+), 99 deletions(-) diff --git a/airflow/providers/hashicorp/_internal_client/vault_client.py b/airflow/providers/hashicorp/_internal_client/vault_client.py index 3ec4384829be..aea8bfb01d76 100644 --- a/airflow/providers/hashicorp/_internal_client/vault_client.py +++ b/airflow/providers/hashicorp/_internal_client/vault_client.py @@ -48,10 +48,12 @@ class _VaultClient(LoggingMixin): """ - Retrieves Authenticated client from Hashicorp Vault. This is purely internal class promoting - authentication code reuse between the Hook and the SecretBackend, it should not be used directly in - Airflow DAGs. Use VaultBackend for backend integration and Hook in case you want to communicate - with VaultHook using standard Airflow Connection definition. + Retrieves Authenticated client from Hashicorp Vault. + + This is purely internal class promoting authentication code reuse between the Hook and the + SecretBackend, it should not be used directly in Airflow DAGs. Use VaultBackend for backend + integration and Hook in case you want to communicate with VaultHook using standard Airflow + Connection definition. :param url: Base URL for the Vault instance being addressed. :param auth_type: Authentication Type for Vault. Default is ``token``. Available values are in @@ -172,12 +174,9 @@ def __init__( @property def client(self): """ - Authentication to Vault can expire. This wrapper function checks that - it is still authenticated to Vault, and invalidates the cache if this - is not the case. + Checks that it is still authenticated to Vault and invalidates the cache if this is not the case. :return: Vault Client - """ if not self._client.is_authenticated(): # Invalidate the cache: diff --git a/airflow/providers/hashicorp/secrets/vault.py b/airflow/providers/hashicorp/secrets/vault.py index 661707d9b853..dd93aeb6c848 100644 --- a/airflow/providers/hashicorp/secrets/vault.py +++ b/airflow/providers/hashicorp/secrets/vault.py @@ -209,8 +209,9 @@ def get_conn_uri(self, conn_id: str) -> str | None: def get_connection(self, conn_id: str) -> Connection | None: """ - Get connection from Vault as secret. Prioritize conn_uri if exists, - if not fall back to normal Connection creation. + Get connection from Vault as secret. + + Prioritize conn_uri if exists, if not fall back to normal Connection creation. :return: A Connection object constructed from Vault data """ diff --git a/airflow/providers/http/sensors/http.py b/airflow/providers/http/sensors/http.py index f691fd0a6922..302ea98ef2a4 100644 --- a/airflow/providers/http/sensors/http.py +++ b/airflow/providers/http/sensors/http.py @@ -29,8 +29,7 @@ class HttpSensor(BaseSensorOperator): """ - Executes a HTTP GET statement and returns False on failure caused by - 404 Not Found or `response_check` returning False. + Execute HTTP GET statement; return False on failure 404 Not Found or `response_check` returning False. HTTP Error codes other than 404 (like 403) or Connection Refused Error would raise an exception and fail the sensor itself directly (no more poking). diff --git a/airflow/providers/imap/hooks/imap.py b/airflow/providers/imap/hooks/imap.py index 7368eedfdb07..4a00e6965a64 100644 --- a/airflow/providers/imap/hooks/imap.py +++ b/airflow/providers/imap/hooks/imap.py @@ -16,8 +16,8 @@ # specific language governing permissions and limitations # under the License. """ -This module provides everything to be able to search in mails for a specific attachment -and also to download it. +This module provides everything to search mail for a specific attachment and download it. + It uses the imaplib library that is already integrated in python 3. """ from __future__ import annotations diff --git a/airflow/providers/jenkins/hooks/jenkins.py b/airflow/providers/jenkins/hooks/jenkins.py index e9024b6a32e4..3a91fd151bcd 100644 --- a/airflow/providers/jenkins/hooks/jenkins.py +++ b/airflow/providers/jenkins/hooks/jenkins.py @@ -27,8 +27,10 @@ def _ensure_prefixes(conn_type): """ - Remove when provider min airflow version >= 2.5.0 since this is handled by - provider manager from that version. + Deprecated. + + Remove when provider min airflow version >= 2.5.0 since + this is handled by provider manager from that version. """ def dec(func): diff --git a/airflow/providers/jenkins/operators/jenkins_job_trigger.py b/airflow/providers/jenkins/operators/jenkins_job_trigger.py index 5b00007748bb..006c199f5957 100644 --- a/airflow/providers/jenkins/operators/jenkins_job_trigger.py +++ b/airflow/providers/jenkins/operators/jenkins_job_trigger.py @@ -21,7 +21,7 @@ import json import socket import time -from typing import Any, Iterable, List, Mapping, Sequence, Union +from typing import Any, Iterable, Mapping, Sequence, Union from urllib.error import HTTPError, URLError import jenkins @@ -33,7 +33,7 @@ from airflow.providers.jenkins.hooks.jenkins import JenkinsHook JenkinsRequest = Mapping[str, Any] -ParamType = Union[str, dict, List, None] +ParamType = Union[str, dict, list, None] def jenkins_request_with_headers(jenkins_server: Jenkins, req: Request) -> JenkinsRequest | None: diff --git a/airflow/providers/microsoft/azure/hooks/adx.py b/airflow/providers/microsoft/azure/hooks/adx.py index 96518b7e0ad4..e3cda1d08f06 100644 --- a/airflow/providers/microsoft/azure/hooks/adx.py +++ b/airflow/providers/microsoft/azure/hooks/adx.py @@ -186,8 +186,9 @@ def get_required_param(name: str) -> str: def run_query(self, query: str, database: str, options: dict | None = None) -> KustoResponseDataSetV2: """ - Run KQL query using provided configuration, and return - `azure.kusto.data.response.KustoResponseDataSet` instance. + Run KQL query using provided configuration, and return KustoResponseDataSet instance. + + See: `azure.kusto.data.response.KustoResponseDataSet` If query is unsuccessful AirflowException is raised. :param query: KQL query to run diff --git a/airflow/providers/microsoft/azure/hooks/base_azure.py b/airflow/providers/microsoft/azure/hooks/base_azure.py index 4190a2bf03ee..2a4d250e1ca5 100644 --- a/airflow/providers/microsoft/azure/hooks/base_azure.py +++ b/airflow/providers/microsoft/azure/hooks/base_azure.py @@ -27,8 +27,10 @@ class AzureBaseHook(BaseHook): """ - This hook acts as a base hook for azure services. It offers several authentication mechanisms to - authenticate the client library used for upstream azure hooks. + This hook acts as a base hook for azure services. + + It offers several authentication mechanisms to authenticate + the client library used for upstream azure hooks. :param sdk_client: The SDKClient to use. :param conn_id: The :ref:`Azure connection id` diff --git a/airflow/providers/microsoft/azure/hooks/cosmos.py b/airflow/providers/microsoft/azure/hooks/cosmos.py index 238abc4c70f6..45b1b0dab2dc 100644 --- a/airflow/providers/microsoft/azure/hooks/cosmos.py +++ b/airflow/providers/microsoft/azure/hooks/cosmos.py @@ -237,10 +237,7 @@ def delete_collection(self, collection_name: str, database_name: str | None = No ) def upsert_document(self, document, database_name=None, collection_name=None, document_id=None): - """ - Inserts a new document (or updates an existing one) into an existing - collection in the CosmosDB database. - """ + """Insert or update a document into an existing collection in the CosmosDB database.""" # Assign unique ID if one isn't provided if document_id is None: document_id = str(uuid.uuid4()) diff --git a/airflow/providers/microsoft/azure/hooks/fileshare.py b/airflow/providers/microsoft/azure/hooks/fileshare.py index 8067907706fe..0244b7ce727e 100644 --- a/airflow/providers/microsoft/azure/hooks/fileshare.py +++ b/airflow/providers/microsoft/azure/hooks/fileshare.py @@ -28,6 +28,8 @@ def _ensure_prefixes(conn_type): """ + Deprecated. + Remove when provider min airflow version >= 2.5.0 since this is handled by provider manager from that version. """ diff --git a/airflow/providers/microsoft/azure/hooks/wasb.py b/airflow/providers/microsoft/azure/hooks/wasb.py index ad9c4754c525..794da8dd8575 100644 --- a/airflow/providers/microsoft/azure/hooks/wasb.py +++ b/airflow/providers/microsoft/azure/hooks/wasb.py @@ -53,6 +53,8 @@ def _ensure_prefixes(conn_type): """ + Deprecated. + Remove when provider min airflow version >= 2.5.0 since this is handled by provider manager from that version. """ diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 51c0c4dc182b..96c87219ca13 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -45,9 +45,9 @@ def get_default_delete_local_copy(): class WasbTaskHandler(FileTaskHandler, LoggingMixin): """ - WasbTaskHandler is a python log handler that handles and reads - task instance logs. It extends airflow FileTaskHandler and - uploads to and reads from Wasb remote storage. + WasbTaskHandler is a python log handler that handles and reads task instance logs. + + It extends airflow FileTaskHandler and uploads to and reads from Wasb remote storage. """ trigger_should_wrap = True @@ -171,6 +171,7 @@ def _read( ) -> tuple[str, dict[str, bool]]: """ Read logs of given task instance and try_number from Wasb remote storage. + If failed, read the log from task instance host machine. todo: when min airflow version >= 2.6, remove this method @@ -207,8 +208,7 @@ def wasb_log_exists(self, remote_log_location: str) -> bool: def wasb_read(self, remote_log_location: str, return_error: bool = False): """ - Returns the log found at the remote_log_location. Returns '' if no - logs are found or there is an error. + Return the log found at the remote_log_location. Returns '' if no logs are found or there is an error. :param remote_log_location: the log's location in remote storage :param return_error: if True, returns a string error message if an @@ -226,8 +226,7 @@ def wasb_read(self, remote_log_location: str, return_error: bool = False): def wasb_write(self, log: str, remote_log_location: str, append: bool = True) -> bool: """ - Writes the log to the remote_log_location. Fails silently if no hook - was created. + Writes the log to the remote_log_location. Fails silently if no hook was created. :param log: the log to write to the remote_log_location :param remote_log_location: the log's location in remote storage diff --git a/airflow/providers/microsoft/azure/operators/asb.py b/airflow/providers/microsoft/azure/operators/asb.py index 333fc82d2b4d..d9a460b77d23 100644 --- a/airflow/providers/microsoft/azure/operators/asb.py +++ b/airflow/providers/microsoft/azure/operators/asb.py @@ -155,10 +155,7 @@ def __init__( self.max_wait_time = max_wait_time def execute(self, context: Context) -> None: - """ - Receive Message in specific queue in Service Bus namespace, - by connecting to Service Bus client. - """ + """Receive Message in specific queue in Service Bus namespace by connecting to Service Bus client.""" # Create the hook hook = MessageHook(azure_service_bus_conn_id=self.azure_service_bus_conn_id) @@ -527,10 +524,7 @@ def __init__( self.azure_service_bus_conn_id = azure_service_bus_conn_id def execute(self, context: Context) -> None: - """ - Receive Message in specific queue in Service Bus namespace, - by connecting to Service Bus client. - """ + """Receive Message in specific queue in Service Bus namespace by connecting to Service Bus client.""" # Create the hook hook = MessageHook(azure_service_bus_conn_id=self.azure_service_bus_conn_id) diff --git a/airflow/providers/microsoft/azure/operators/data_factory.py b/airflow/providers/microsoft/azure/operators/data_factory.py index a2b2c528bf9b..1823212473f6 100644 --- a/airflow/providers/microsoft/azure/operators/data_factory.py +++ b/airflow/providers/microsoft/azure/operators/data_factory.py @@ -233,8 +233,8 @@ def execute(self, context: Context) -> None: def execute_complete(self, context: Context, event: dict[str, str]) -> None: """ Callback for when the trigger fires - returns immediately. - Relies on trigger to throw an exception, otherwise it assumes execution was - successful. + + Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ if event: if event["status"] == "error": diff --git a/airflow/providers/microsoft/azure/secrets/key_vault.py b/airflow/providers/microsoft/azure/secrets/key_vault.py index df696b05821b..177da5d23532 100644 --- a/airflow/providers/microsoft/azure/secrets/key_vault.py +++ b/airflow/providers/microsoft/azure/secrets/key_vault.py @@ -162,6 +162,7 @@ def get_config(self, key: str) -> str | None: def build_path(path_prefix: str, secret_id: str, sep: str = "-") -> str: """ Given a path_prefix and secret_id, build a valid secret name for the Azure Key Vault Backend. + Also replaces underscore in the path with dashes to support easy switching between environment variables, so ``connection_default`` becomes ``connection-default``. diff --git a/airflow/providers/microsoft/azure/sensors/data_factory.py b/airflow/providers/microsoft/azure/sensors/data_factory.py index b4ebedce698a..f0651494bfac 100644 --- a/airflow/providers/microsoft/azure/sensors/data_factory.py +++ b/airflow/providers/microsoft/azure/sensors/data_factory.py @@ -113,8 +113,8 @@ def execute(self, context: Context) -> None: def execute_complete(self, context: Context, event: dict[str, str]) -> None: """ Callback for when the trigger fires - returns immediately. - Relies on trigger to throw an exception, otherwise it assumes execution was - successful. + + Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ if event: if event["status"] == "error": diff --git a/airflow/providers/microsoft/azure/sensors/wasb.py b/airflow/providers/microsoft/azure/sensors/wasb.py index 0c227f2ea37b..18ac2331e018 100644 --- a/airflow/providers/microsoft/azure/sensors/wasb.py +++ b/airflow/providers/microsoft/azure/sensors/wasb.py @@ -97,8 +97,8 @@ def execute(self, context: Context) -> None: def execute_complete(self, context: Context, event: dict[str, str]) -> None: """ Callback for when the trigger fires - returns immediately. - Relies on trigger to throw an exception, otherwise it assumes execution was - successful. + + Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ if event: if event["status"] == "error": @@ -193,8 +193,8 @@ def execute(self, context: Context) -> None: def execute_complete(self, context: Context, event: dict[str, str]) -> None: """ Callback for when the trigger fires - returns immediately. - Relies on trigger to throw an exception, otherwise it assumes execution was - successful. + + Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ if event: if event["status"] == "error": diff --git a/airflow/providers/microsoft/azure/transfers/local_to_adls.py b/airflow/providers/microsoft/azure/transfers/local_to_adls.py index f39f837c9820..7eee5009afcc 100644 --- a/airflow/providers/microsoft/azure/transfers/local_to_adls.py +++ b/airflow/providers/microsoft/azure/transfers/local_to_adls.py @@ -102,6 +102,7 @@ def execute(self, context: Context) -> None: class LocalToAzureDataLakeStorageOperator(LocalFilesystemToADLSOperator): """ This class is deprecated. + Please use `airflow.providers.microsoft.azure.transfers.local_to_adls.LocalFilesystemToADLSOperator`. """ diff --git a/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py b/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py index 93e8a7b14d3f..f891e080a490 100644 --- a/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py +++ b/airflow/providers/microsoft/azure/transfers/oracle_to_azure_data_lake.py @@ -32,9 +32,7 @@ class OracleToAzureDataLakeOperator(BaseOperator): """ - Moves data from Oracle to Azure Data Lake. The operator runs the query against - Oracle and stores the file locally before loading it into Azure Data Lake. - + Runs the query against Oracle and stores the file locally before loading it into Azure Data Lake. :param filename: file name to be used by the csv file. :param azure_data_lake_conn_id: destination azure data lake connection. diff --git a/airflow/providers/microsoft/azure/triggers/wasb.py b/airflow/providers/microsoft/azure/triggers/wasb.py index 0bc75f065f3a..944c7ddae1d3 100644 --- a/airflow/providers/microsoft/azure/triggers/wasb.py +++ b/airflow/providers/microsoft/azure/triggers/wasb.py @@ -25,8 +25,9 @@ class WasbBlobSensorTrigger(BaseTrigger): """ - WasbBlobSensorTrigger is fired as deferred class with params to run the task in - trigger worker to check for existence of the given blob in the provided container. + Checks for existence of the given blob in the provided container. + + WasbBlobSensorTrigger is fired as deferred class with params to run the task in trigger worker. :param container_name: name of the container in which the blob should be searched for :param blob_name: name of the blob to check existence for @@ -90,8 +91,9 @@ async def run(self) -> AsyncIterator[TriggerEvent]: class WasbPrefixSensorTrigger(BaseTrigger): """ + Checks for the existence of a blob with the given prefix in the provided container. + WasbPrefixSensorTrigger is fired as a deferred class with params to run the task in trigger. - It checks for the existence of a blob with the given prefix in the provided container. :param container_name: name of the container in which the blob should be searched for :param prefix: prefix of the blob to check existence for diff --git a/airflow/providers/microsoft/azure/utils.py b/airflow/providers/microsoft/azure/utils.py index 8c0110046905..e4161a1b12bb 100644 --- a/airflow/providers/microsoft/azure/utils.py +++ b/airflow/providers/microsoft/azure/utils.py @@ -23,6 +23,8 @@ def _ensure_prefixes(conn_type): """ + Deprecated. + Remove when provider min airflow version >= 2.5.0 since this is handled by provider manager from that version. """ diff --git a/airflow/providers/microsoft/mssql/hooks/mssql.py b/airflow/providers/microsoft/mssql/hooks/mssql.py index 01e1c9e47b4a..7b21e77c7fab 100644 --- a/airflow/providers/microsoft/mssql/hooks/mssql.py +++ b/airflow/providers/microsoft/mssql/hooks/mssql.py @@ -26,7 +26,14 @@ class MsSqlHook(DbApiHook): - """Interact with Microsoft SQL Server.""" + """ + Interact with Microsoft SQL Server. + + :param args: passed to DBApiHook + :param sqlalchemy_scheme: Scheme sqlalchemy connection. Default is ``mssql+pymssql`` Only used for + ``get_sqlalchemy_engine`` and ``get_sqlalchemy_connection`` methods. + :param kwargs: passed to DbApiHook + """ conn_name_attr = "mssql_conn_id" default_conn_name = "mssql_default" @@ -41,12 +48,6 @@ def __init__( sqlalchemy_scheme: str | None = None, **kwargs, ) -> None: - """ - :param args: passed to DBApiHook - :param sqlalchemy_scheme: Scheme sqlalchemy connection. Default is ``mssql+pymssql`` Only used for - ``get_sqlalchemy_engine`` and ``get_sqlalchemy_connection`` methods. - :param kwargs: passed to DbApiHook - """ super().__init__(*args, **kwargs) self.schema = kwargs.pop("schema", None) self._sqlalchemy_scheme = sqlalchemy_scheme @@ -55,6 +56,7 @@ def __init__( def connection_extra_lower(self) -> dict: """ ``connection.extra_dejson`` but where keys are converted to lower case. + This is used internally for case-insensitive access of mssql params. """ conn = self.get_connection(self.mssql_conn_id) # type: ignore[attr-defined] diff --git a/airflow/providers/microsoft/psrp/hooks/psrp.py b/airflow/providers/microsoft/psrp/hooks/psrp.py index fe7f4c271fa7..06e3eb93a7fa 100644 --- a/airflow/providers/microsoft/psrp/hooks/psrp.py +++ b/airflow/providers/microsoft/psrp/hooks/psrp.py @@ -158,8 +158,9 @@ def apply_extra(d, keys): @contextmanager def invoke(self) -> Generator[PowerShell, None, None]: """ - Context manager that yields a PowerShell object to which commands can be - added. Upon exit, the commands will be invoked. + Yields a PowerShell object to which commands can be added. + + Upon exit, the commands will be invoked. """ logger = copy(self.log) logger.setLevel(self._logging_level) diff --git a/airflow/providers/mongo/hooks/mongo.py b/airflow/providers/mongo/hooks/mongo.py index b49a9365afcc..cfa34e0e7510 100644 --- a/airflow/providers/mongo/hooks/mongo.py +++ b/airflow/providers/mongo/hooks/mongo.py @@ -98,6 +98,7 @@ def get_conn(self) -> MongoClient: def _create_uri(self) -> str: """ Create URI string from the given credentials. + :return: URI string. """ srv = self.extras.pop("srv", False) diff --git a/airflow/providers/mysql/transfers/presto_to_mysql.py b/airflow/providers/mysql/transfers/presto_to_mysql.py index b38e6b8654d6..0d80b58bff8c 100644 --- a/airflow/providers/mysql/transfers/presto_to_mysql.py +++ b/airflow/providers/mysql/transfers/presto_to_mysql.py @@ -29,9 +29,10 @@ class PrestoToMySqlOperator(BaseOperator): """ - Moves data from Presto to MySQL, note that for now the data is loaded - into memory before being pushed to MySQL, so this operator should - be used for smallish amount of data. + Moves data from Presto to MySQL. + + Note that for now the data is loaded into memory before being pushed + to MySQL, so this operator should be used for smallish amount of data. :param sql: SQL query to execute against Presto. (templated) :param mysql_table: target MySQL table, use dot notation to target a diff --git a/airflow/providers/mysql/transfers/trino_to_mysql.py b/airflow/providers/mysql/transfers/trino_to_mysql.py index 8ff5ed0446b5..e96aaafd5bf0 100644 --- a/airflow/providers/mysql/transfers/trino_to_mysql.py +++ b/airflow/providers/mysql/transfers/trino_to_mysql.py @@ -29,9 +29,10 @@ class TrinoToMySqlOperator(BaseOperator): """ - Moves data from Trino to MySQL, note that for now the data is loaded - into memory before being pushed to MySQL, so this operator should - be used for smallish amount of data. + Moves data from Trino to MySQL. + + Note that for now the data is loaded into memory before being pushed + to MySQL, so this operator should be used for smallish amount of data. :param sql: SQL query to execute against Trino. (templated) :param mysql_table: target MySQL table, use dot notation to target a diff --git a/airflow/providers/neo4j/hooks/neo4j.py b/airflow/providers/neo4j/hooks/neo4j.py index ed1df254a87a..137b1e00c032 100644 --- a/airflow/providers/neo4j/hooks/neo4j.py +++ b/airflow/providers/neo4j/hooks/neo4j.py @@ -48,10 +48,7 @@ def __init__(self, conn_id: str = default_conn_name, *args, **kwargs) -> None: self.client: Driver | None = None def get_conn(self) -> Driver: - """ - Function that initiates a new Neo4j connection - with username, password and database schema. - """ + """Function that initiates a new Neo4j connection with username, password and database schema.""" if self.client is not None: return self.client @@ -112,8 +109,7 @@ def get_uri(self, conn: Connection) -> str: def run(self, query) -> list[Any]: """ - Function to create a neo4j session - and execute the query in the session. + Function to create a neo4j session and execute the query in the session. :param query: Neo4j query :return: Result diff --git a/airflow/providers/openlineage/extractors/bash.py b/airflow/providers/openlineage/extractors/bash.py index 5a2bc92f65bf..9d7c40b114c1 100644 --- a/airflow/providers/openlineage/extractors/bash.py +++ b/airflow/providers/openlineage/extractors/bash.py @@ -32,9 +32,11 @@ class BashExtractor(BaseExtractor): """ + Extract executed bash command and put it into SourceCodeJobFacet. + This extractor provides visibility on what bash task does by extracting - executed bash command and putting it into SourceCodeJobFacet. It does not extract - datasets. + executed bash command and putting it into SourceCodeJobFacet. It does + not extract datasets. :meta private: """ diff --git a/airflow/providers/openlineage/extractors/python.py b/airflow/providers/openlineage/extractors/python.py index 0c5720ac8672..50d84014fdee 100644 --- a/airflow/providers/openlineage/extractors/python.py +++ b/airflow/providers/openlineage/extractors/python.py @@ -35,6 +35,8 @@ class PythonExtractor(BaseExtractor): """ + Extract executed source code and put it into SourceCodeJobFacet. + This extractor provides visibility on what particular task does by extracting executed source code and putting it into SourceCodeJobFacet. It does not extract datasets yet. diff --git a/airflow/providers/openlineage/plugins/adapter.py b/airflow/providers/openlineage/plugins/adapter.py index aecd0a65314f..0e530e5c53fe 100644 --- a/airflow/providers/openlineage/plugins/adapter.py +++ b/airflow/providers/openlineage/plugins/adapter.py @@ -58,10 +58,7 @@ class OpenLineageAdapter(LoggingMixin): - """ - Adapter for translating Airflow metadata to OpenLineage events, - instead of directly creating them from Airflow code. - """ + """Translate Airflow metadata to OpenLineage events instead of creating them from Airflow code.""" def __init__(self, client: OpenLineageClient | None = None, secrets_masker: SecretsMasker | None = None): super().__init__() diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index 9b57bf919fb6..99394863f514 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -38,10 +38,7 @@ class OpenLineageListener: - """ - OpenLineage listener - Sends events on task instance and dag run starts, completes and failures. - """ + """OpenLineage listener sends events on task instance and dag run starts, completes and failures.""" def __init__(self): self.log = logging.getLogger(__name__) diff --git a/airflow/providers/openlineage/plugins/macros.py b/airflow/providers/openlineage/plugins/macros.py index 19cbb59f299e..61af81a1eb71 100644 --- a/airflow/providers/openlineage/plugins/macros.py +++ b/airflow/providers/openlineage/plugins/macros.py @@ -30,9 +30,9 @@ def lineage_run_id(task_instance: TaskInstance): """ - Macro function which returns the generated run id for a given task. This - can be used to forward the run id from a task to a child run so the job - hierarchy is preserved. + Macro function which returns the generated run id for a given task. + + This can be used to forward the run id from a task to a child run so the job hierarchy is preserved. .. seealso:: For more information on how to use this operator, take a look at the guide: @@ -45,8 +45,9 @@ def lineage_run_id(task_instance: TaskInstance): def lineage_parent_id(run_id: str, task_instance: TaskInstance): """ - Macro function which returns the generated job and run id for a given task. This - can be used to forward the ids from a task to a child run so the job + Macro function which returns the generated job and run id for a given task. + + This can be used to forward the ids from a task to a child run so the job hierarchy is preserved. Child run can create ParentRunFacet from those ids. .. seealso:: diff --git a/airflow/providers/openlineage/plugins/openlineage.py b/airflow/providers/openlineage/plugins/openlineage.py index 9aade083b482..2ec080114753 100644 --- a/airflow/providers/openlineage/plugins/openlineage.py +++ b/airflow/providers/openlineage/plugins/openlineage.py @@ -31,8 +31,11 @@ def _is_disabled() -> bool: class OpenLineageProviderPlugin(AirflowPlugin): - """OpenLineage Plugin provides listener that emits OL events on DAG start, complete and failure - and TaskInstances start, complete and failure. + """ + Listener that emits numerous Events. + + OpenLineage Plugin provides listener that emits OL events on DAG start, + complete and failure and TaskInstances start, complete and failure. """ name = "OpenLineageProviderPlugin" diff --git a/airflow/providers/openlineage/sqlparser.py b/airflow/providers/openlineage/sqlparser.py index ed3e92e58b5a..8e85d3706c52 100644 --- a/airflow/providers/openlineage/sqlparser.py +++ b/airflow/providers/openlineage/sqlparser.py @@ -207,7 +207,8 @@ def normalize_sql(cls, sql: list[str] | str) -> str: @classmethod def split_sql_string(cls, sql: list[str] | str) -> list[str]: """ - Split SQL string into list of statements + Split SQL string into list of statements. + Tries to use `DbApiHook.split_sql_string` if available. Otherwise, uses the same logic. """ diff --git a/airflow/providers/openlineage/utils/utils.py b/airflow/providers/openlineage/utils/utils.py index 84ad41e23701..9d1cab8eca87 100644 --- a/airflow/providers/openlineage/utils/utils.py +++ b/airflow/providers/openlineage/utils/utils.py @@ -101,6 +101,7 @@ def url_to_https(url) -> str | None: def redacted_connection_uri(conn: Connection, filtered_params=None, filtered_prefixes=None): """ Return the connection URI for the given Connection. + This method additionally filters URI by removing query parameters that are known to carry sensitive data like username, password, access key. """ @@ -324,7 +325,9 @@ def get_airflow_run_facet( class OpenLineageRedactor(SecretsMasker): - """This class redacts sensitive data similar to SecretsMasker in Airflow logs. + """ + This class redacts sensitive data similar to SecretsMasker in Airflow logs. + The difference is that our default max recursion depth is way higher - due to the structure of OL events we need more depth. Additionally, we allow data structures to specify data that needs not to be diff --git a/airflow/providers/opsgenie/hooks/opsgenie.py b/airflow/providers/opsgenie/hooks/opsgenie.py index 7c28b2a76e1e..239f67a7e212 100644 --- a/airflow/providers/opsgenie/hooks/opsgenie.py +++ b/airflow/providers/opsgenie/hooks/opsgenie.py @@ -33,6 +33,7 @@ class OpsgenieAlertHook(BaseHook): """ This hook allows you to post alerts to Opsgenie. + Accepts a connection that has an Opsgenie API key as the connection's password. This hook sets the domain to conn_id.host, and if not set will default to ``https://api.opsgenie.com``. diff --git a/airflow/providers/opsgenie/operators/opsgenie.py b/airflow/providers/opsgenie/operators/opsgenie.py index a4777a8eb72f..d12db2087356 100644 --- a/airflow/providers/opsgenie/operators/opsgenie.py +++ b/airflow/providers/opsgenie/operators/opsgenie.py @@ -29,6 +29,7 @@ class OpsgenieCreateAlertOperator(BaseOperator): """ This operator allows you to post alerts to Opsgenie. + Accepts a connection that has an Opsgenie API key as the connection's password. This operator sets the domain to conn_id.host, and if not set will default to ``https://api.opsgenie.com``. @@ -101,8 +102,9 @@ def __init__( def _build_opsgenie_payload(self) -> dict[str, Any]: """ - Construct the Opsgenie JSON payload. All relevant parameters are combined here - to a valid Opsgenie JSON payload. + Construct the Opsgenie JSON payload. + + All relevant parameters are combined here to a valid Opsgenie JSON payload. :return: Opsgenie payload (dict) to send """ @@ -137,6 +139,7 @@ def execute(self, context: Context) -> None: class OpsgenieCloseAlertOperator(BaseOperator): """ This operator allows you to close alerts to Opsgenie. + Accepts a connection that has an Opsgenie API key as the connection's password. This operator sets the domain to conn_id.host, and if not set will default to ``https://api.opsgenie.com``. @@ -183,8 +186,9 @@ def __init__( def _build_opsgenie_close_alert_payload(self) -> dict[str, Any]: """ - Construct the Opsgenie JSON payload. All relevant parameters are combined here - to a valid Opsgenie JSON payload. + Construct the Opsgenie JSON payload. + + All relevant parameters are combined here to a valid Opsgenie JSON payload. :return: Opsgenie close alert payload (dict) to send """ @@ -214,6 +218,7 @@ def execute(self, context: Context) -> None: class OpsgenieDeleteAlertOperator(BaseOperator): """ This operator allows you to delete alerts in Opsgenie. + Accepts a connection that has an Opsgenie API key as the connection's password. This operator sets the domain to conn_id.host, and if not set will default to ``https://api.opsgenie.com``. From cb4927a01887e2413c45d8d9cb63e74aa994ee74 Mon Sep 17 00:00:00 2001 From: eladkal <45845474+eladkal@users.noreply.github.com> Date: Wed, 5 Jul 2023 10:12:51 +0300 Subject: [PATCH 052/533] Prepare docs for July 2023 wave of Providers (#32298) * Prepare docs for July 2023 wave of Providers * Add common.sql and more changes to Amazon * Add amazon and microsoft.azure --- .../airbyte/.latest-doc-only-change.txt | 2 +- airflow/providers/alibaba/CHANGELOG.rst | 13 +++++ airflow/providers/alibaba/__init__.py | 2 +- airflow/providers/alibaba/provider.yaml | 1 + airflow/providers/amazon/CHANGELOG.rst | 50 +++++++++++++++++++ airflow/providers/amazon/__init__.py | 2 +- airflow/providers/amazon/provider.yaml | 1 + .../apache/beam/.latest-doc-only-change.txt | 2 +- .../cassandra/.latest-doc-only-change.txt | 2 +- .../apache/drill/.latest-doc-only-change.txt | 2 +- .../apache/druid/.latest-doc-only-change.txt | 2 +- .../apache/hdfs/.latest-doc-only-change.txt | 2 +- .../apache/hive/.latest-doc-only-change.txt | 2 +- airflow/providers/apache/kafka/CHANGELOG.rst | 13 +++++ airflow/providers/apache/kafka/__init__.py | 2 +- airflow/providers/apache/kafka/provider.yaml | 1 + .../apache/kylin/.latest-doc-only-change.txt | 2 +- airflow/providers/apache/livy/CHANGELOG.rst | 13 +++++ airflow/providers/apache/livy/__init__.py | 2 +- airflow/providers/apache/livy/provider.yaml | 1 + .../apache/pig/.latest-doc-only-change.txt | 2 +- .../apache/pinot/.latest-doc-only-change.txt | 2 +- .../apache/spark/.latest-doc-only-change.txt | 2 +- .../apache/sqoop/.latest-doc-only-change.txt | 2 +- airflow/providers/apprise/CHANGELOG.rst | 5 ++ .../arangodb/.latest-doc-only-change.txt | 2 +- .../asana/.latest-doc-only-change.txt | 2 +- .../celery/.latest-doc-only-change.txt | 2 +- .../cloudant/.latest-doc-only-change.txt | 2 +- .../providers/cncf/kubernetes/CHANGELOG.rst | 20 ++++++++ airflow/providers/cncf/kubernetes/__init__.py | 2 +- .../providers/cncf/kubernetes/provider.yaml | 1 + airflow/providers/common/sql/CHANGELOG.rst | 13 +++++ airflow/providers/common/sql/__init__.py | 2 +- airflow/providers/common/sql/provider.yaml | 1 + airflow/providers/databricks/CHANGELOG.rst | 13 +++++ airflow/providers/databricks/__init__.py | 2 +- airflow/providers/databricks/provider.yaml | 1 + .../datadog/.latest-doc-only-change.txt | 2 +- .../dbt/cloud/.latest-doc-only-change.txt | 2 +- .../dingding/.latest-doc-only-change.txt | 2 +- .../discord/.latest-doc-only-change.txt | 2 +- .../docker/.latest-doc-only-change.txt | 2 +- airflow/providers/elasticsearch/CHANGELOG.rst | 8 +++ airflow/providers/elasticsearch/__init__.py | 2 +- .../exasol/.latest-doc-only-change.txt | 2 +- .../facebook/.latest-doc-only-change.txt | 2 +- .../providers/ftp/.latest-doc-only-change.txt | 2 +- .../github/.latest-doc-only-change.txt | 2 +- airflow/providers/google/CHANGELOG.rst | 32 ++++++++++++ airflow/providers/google/__init__.py | 2 +- airflow/providers/google/provider.yaml | 1 + .../grpc/.latest-doc-only-change.txt | 2 +- airflow/providers/hashicorp/CHANGELOG.rst | 12 +++++ airflow/providers/hashicorp/__init__.py | 2 +- airflow/providers/hashicorp/provider.yaml | 1 + .../http/.latest-doc-only-change.txt | 2 +- .../imap/.latest-doc-only-change.txt | 2 +- .../influxdb/.latest-doc-only-change.txt | 2 +- .../jdbc/.latest-doc-only-change.txt | 2 +- .../jenkins/.latest-doc-only-change.txt | 2 +- .../providers/microsoft/azure/CHANGELOG.rst | 27 ++++++++++ airflow/providers/microsoft/azure/__init__.py | 2 +- .../providers/microsoft/azure/provider.yaml | 1 + .../mssql/.latest-doc-only-change.txt | 2 +- .../psrp/.latest-doc-only-change.txt | 2 +- .../winrm/.latest-doc-only-change.txt | 2 +- .../mongo/.latest-doc-only-change.txt | 2 +- .../mysql/.latest-doc-only-change.txt | 2 +- .../neo4j/.latest-doc-only-change.txt | 2 +- .../odbc/.latest-doc-only-change.txt | 2 +- .../openfaas/.latest-doc-only-change.txt | 2 +- .../opsgenie/.latest-doc-only-change.txt | 2 +- .../oracle/.latest-doc-only-change.txt | 2 +- .../pagerduty/.latest-doc-only-change.txt | 2 +- .../papermill/.latest-doc-only-change.txt | 2 +- .../plexus/.latest-doc-only-change.txt | 2 +- .../postgres/.latest-doc-only-change.txt | 2 +- .../presto/.latest-doc-only-change.txt | 2 +- .../qubole/.latest-doc-only-change.txt | 2 +- .../redis/.latest-doc-only-change.txt | 2 +- .../salesforce/.latest-doc-only-change.txt | 2 +- .../samba/.latest-doc-only-change.txt | 2 +- .../segment/.latest-doc-only-change.txt | 2 +- .../sendgrid/.latest-doc-only-change.txt | 2 +- .../sftp/.latest-doc-only-change.txt | 2 +- .../singularity/.latest-doc-only-change.txt | 2 +- .../slack/.latest-doc-only-change.txt | 2 +- .../snowflake/.latest-doc-only-change.txt | 2 +- .../sqlite/.latest-doc-only-change.txt | 2 +- .../providers/ssh/.latest-doc-only-change.txt | 2 +- .../tableau/.latest-doc-only-change.txt | 2 +- .../tabular/.latest-doc-only-change.txt | 2 +- .../telegram/.latest-doc-only-change.txt | 2 +- airflow/providers/trino/CHANGELOG.rst | 12 +++++ airflow/providers/trino/__init__.py | 2 +- airflow/providers/trino/provider.yaml | 1 + airflow/providers/vertica/CHANGELOG.rst | 12 +++++ airflow/providers/vertica/__init__.py | 2 +- airflow/providers/vertica/provider.yaml | 1 + .../zendesk/.latest-doc-only-change.txt | 2 +- .../prepare_provider_packages.py | 7 ++- .../commits.rst | 14 ++++++ .../index.rst | 2 +- .../commits.rst | 43 ++++++++++++++++ .../apache-airflow-providers-amazon/index.rst | 8 +-- .../commits.rst | 14 ++++++ .../index.rst | 2 +- .../commits.rst | 14 ++++++ .../index.rst | 6 +-- .../commits.rst | 23 ++++++++- .../commits.rst | 17 +++++++ .../index.rst | 2 +- .../commits.rst | 22 ++++++-- .../index.rst | 30 ++++++++++- .../commits.rst | 14 ++++++ .../index.rst | 6 +-- .../commits.rst | 14 ++++++ .../commits.rst | 24 +++++++++ .../apache-airflow-providers-google/index.rst | 6 +-- .../commits.rst | 13 +++++ .../index.rst | 6 +-- .../commits.rst | 20 ++++++++ .../index.rst | 6 +-- .../commits.rst | 13 +++++ docs/apache-airflow-providers-trino/index.rst | 6 +-- .../commits.rst | 13 +++++ .../index.rst | 6 +-- 128 files changed, 647 insertions(+), 109 deletions(-) diff --git a/airflow/providers/airbyte/.latest-doc-only-change.txt b/airflow/providers/airbyte/.latest-doc-only-change.txt index ff7136e07d74..370cc38c596d 100644 --- a/airflow/providers/airbyte/.latest-doc-only-change.txt +++ b/airflow/providers/airbyte/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +0bc689ee6d4b6967d7ae99a202031aac14d181a2 diff --git a/airflow/providers/alibaba/CHANGELOG.rst b/airflow/providers/alibaba/CHANGELOG.rst index be0b33e07a97..af267238d0f8 100644 --- a/airflow/providers/alibaba/CHANGELOG.rst +++ b/airflow/providers/alibaba/CHANGELOG.rst @@ -24,6 +24,19 @@ Changelog for ``apache-airflow-providers-alibaba`` -------------------------------------------------- +2.5.0 +..... + +Features +~~~~~~~~ + +* ``Add Alibaba Cloud AnalyticDB Spark Support (#31787)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Airbyte and Alibaba (#32214)`` + * ``Improve provider documentation and README structure (#32125)`` + 2.4.1 ..... diff --git a/airflow/providers/alibaba/__init__.py b/airflow/providers/alibaba/__init__.py index 2442a7218247..1d3407921c34 100644 --- a/airflow/providers/alibaba/__init__.py +++ b/airflow/providers/alibaba/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "2.4.1" +__version__ = "2.5.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/alibaba/provider.yaml b/airflow/providers/alibaba/provider.yaml index 660dccdcd648..367209552523 100644 --- a/airflow/providers/alibaba/provider.yaml +++ b/airflow/providers/alibaba/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 2.5.0 - 2.4.1 - 2.4.0 - 2.3.0 diff --git a/airflow/providers/amazon/CHANGELOG.rst b/airflow/providers/amazon/CHANGELOG.rst index ea6abb7c61ac..1b3b94fefc23 100644 --- a/airflow/providers/amazon/CHANGELOG.rst +++ b/airflow/providers/amazon/CHANGELOG.rst @@ -24,6 +24,56 @@ Changelog for ``apache-airflow-providers-amazon`` ------------------------------------------------- +8.3.0 +..... + +Features +~~~~~~~~ + +* ``Add 'ChimeWebhookHook' (#31939)`` +* ``Add 'ChimeNotifier' (#32222)`` +* ``Add deferrable mode to S3KeysUnchangedSensor (#31940)`` +* ``Add deferrable mode to 'RdsCreateDbInstanceOperator' and 'RdsDeleteDbInstanceOperator' (#32171)`` +* ``Add deferrable mode for 'AthenaOperator' (#32186)`` +* ``Add a deferrable mode to 'BatchCreateComputeEnvironmentOperator' (#32036)`` +* ``Add deferrable mode in EMR operator and sensor (#32029)`` +* ``add async wait method to the "with logging" aws utils (#32055)`` +* ``Add custom waiters to EMR Serverless (#30463)`` +* ``Add an option to 'GlueJobOperator' to stop the job run when the TI is killed (#32155)`` +* ``deferrable mode for 'SageMakerTuningOperator' and 'SageMakerEndpointOperator' (#32112)`` +* ``EKS Create/Delete Nodegroup Deferrable mode (#32165)`` +* ``Deferrable mode for ECS operators (#31881)`` +* ``feature: AWS - GlueJobOperator - job_poll_interval (#32147)`` +* ``Added 'AzureBlobStorageToS3Operator' transfer operator (#32270)`` + +Bug Fixes +~~~~~~~~~ + +* ``bugfix: break down run+wait method in ECS operator (#32104)`` +* ``Handle 'UnboundLocalError' while parsing invalid 's3_url' (#32120)`` +* ``Fix 'LambdaInvokeFunctionOperator' payload parameter type (#32259)`` + +Misc +~~~~ + +* ``Deprecate 'delimiter' param and source object's wildcards in GCS, introduce 'match_glob' param. (#31261)`` +* ``aws waiter util: log status info with error level on waiter error (#32247)`` +* ``rewrite method used in ecs to fetch less logs (#31786)`` +* ``Refactor Eks Create Cluster Operator code (#31960)`` +* ``Use a waiter in 'AthenaHook' (#31942)`` +* ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Revert "add deferrable mode for 'AthenaOperator' (#32110)" (#32172)`` + * ``add deferrable mode for 'AthenaOperator' (#32110)`` + * ``D205 Support - Auto-fixes and Stragglers (#32212)`` + * ``D205 Support - Providers: Amazon/AWS (#32224)`` + * ``Improve provider documentation and README structure (#32125)`` + * ``Minor name change for the util wait method. (#32152)`` + * ``Clean up string concatenation (#32129)`` + * ``cleanup Amazon CHANGELOG.rst (#32031)`` + 8.2.0 ..... diff --git a/airflow/providers/amazon/__init__.py b/airflow/providers/amazon/__init__.py index cf6ef41b5ebc..204607dbae69 100644 --- a/airflow/providers/amazon/__init__.py +++ b/airflow/providers/amazon/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "8.2.0" +__version__ = "8.3.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index 575a2b5cab1b..390a5866368a 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 8.3.0 - 8.2.0 - 8.1.0 - 8.0.0 diff --git a/airflow/providers/apache/beam/.latest-doc-only-change.txt b/airflow/providers/apache/beam/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/apache/beam/.latest-doc-only-change.txt +++ b/airflow/providers/apache/beam/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/cassandra/.latest-doc-only-change.txt b/airflow/providers/apache/cassandra/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/apache/cassandra/.latest-doc-only-change.txt +++ b/airflow/providers/apache/cassandra/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/drill/.latest-doc-only-change.txt b/airflow/providers/apache/drill/.latest-doc-only-change.txt index 8399fdc43bc0..33a07c855e0f 100644 --- a/airflow/providers/apache/drill/.latest-doc-only-change.txt +++ b/airflow/providers/apache/drill/.latest-doc-only-change.txt @@ -1 +1 @@ -428a43995390b3623a51aa7bac7e21da69a8db22 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/apache/druid/.latest-doc-only-change.txt b/airflow/providers/apache/druid/.latest-doc-only-change.txt index e7c3c940c9c7..4d0c8bcde51e 100644 --- a/airflow/providers/apache/druid/.latest-doc-only-change.txt +++ b/airflow/providers/apache/druid/.latest-doc-only-change.txt @@ -1 +1 @@ -602abe8394fafe7de54df7e73af56de848cdf617 +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/hdfs/.latest-doc-only-change.txt b/airflow/providers/apache/hdfs/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/apache/hdfs/.latest-doc-only-change.txt +++ b/airflow/providers/apache/hdfs/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/hive/.latest-doc-only-change.txt b/airflow/providers/apache/hive/.latest-doc-only-change.txt index e7c3c940c9c7..4d0c8bcde51e 100644 --- a/airflow/providers/apache/hive/.latest-doc-only-change.txt +++ b/airflow/providers/apache/hive/.latest-doc-only-change.txt @@ -1 +1 @@ -602abe8394fafe7de54df7e73af56de848cdf617 +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/kafka/CHANGELOG.rst b/airflow/providers/apache/kafka/CHANGELOG.rst index 3bde0c11ded1..95898617e6c7 100644 --- a/airflow/providers/apache/kafka/CHANGELOG.rst +++ b/airflow/providers/apache/kafka/CHANGELOG.rst @@ -23,6 +23,19 @@ Changelog for ``apache-airflow-providers-apache-kafka`` ------------------------------------------------------- +1.1.2 +..... + +Bug Fixes +~~~~~~~~~ + +* ``Break AwaitMessageTrigger execution when finding a message with the desired format (#31803)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` + 1.1.1 ..... diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 45b378cab2fd..d8fb72160e58 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "1.1.1" +__version__ = "1.1.2" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/apache/kafka/provider.yaml b/airflow/providers/apache/kafka/provider.yaml index c78c5952f078..7617440e82a1 100644 --- a/airflow/providers/apache/kafka/provider.yaml +++ b/airflow/providers/apache/kafka/provider.yaml @@ -23,6 +23,7 @@ suspended: false description: | `Apache Kafka `__ versions: + - 1.1.2 - 1.1.1 - 1.1.0 - 1.0.0 diff --git a/airflow/providers/apache/kylin/.latest-doc-only-change.txt b/airflow/providers/apache/kylin/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/apache/kylin/.latest-doc-only-change.txt +++ b/airflow/providers/apache/kylin/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/apache/livy/CHANGELOG.rst b/airflow/providers/apache/livy/CHANGELOG.rst index 40cf76463937..93dc82bbbfe8 100644 --- a/airflow/providers/apache/livy/CHANGELOG.rst +++ b/airflow/providers/apache/livy/CHANGELOG.rst @@ -24,6 +24,19 @@ Changelog for ``apache-airflow-providers-apache-livy`` ------------------------------------------------------ +3.5.2 +..... + +Bug Fixes +~~~~~~~~~ + +* ``Fix 'LivyHook' TypeError exception on 'session_id' log format (#32051)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` + 3.5.1 ..... diff --git a/airflow/providers/apache/livy/__init__.py b/airflow/providers/apache/livy/__init__.py index 5ce66cbc351b..fe23053440bc 100644 --- a/airflow/providers/apache/livy/__init__.py +++ b/airflow/providers/apache/livy/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.5.2" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/apache/livy/provider.yaml b/airflow/providers/apache/livy/provider.yaml index 7f06bf3093b9..1a22652df86e 100644 --- a/airflow/providers/apache/livy/provider.yaml +++ b/airflow/providers/apache/livy/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 3.5.2 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/apache/pig/.latest-doc-only-change.txt b/airflow/providers/apache/pig/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/apache/pig/.latest-doc-only-change.txt +++ b/airflow/providers/apache/pig/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/apache/pinot/.latest-doc-only-change.txt b/airflow/providers/apache/pinot/.latest-doc-only-change.txt index d2f4035cfe09..4d0c8bcde51e 100644 --- a/airflow/providers/apache/pinot/.latest-doc-only-change.txt +++ b/airflow/providers/apache/pinot/.latest-doc-only-change.txt @@ -1 +1 @@ -487b174073c01e03ae64760405a8d88f6a488ca6 +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/spark/.latest-doc-only-change.txt b/airflow/providers/apache/spark/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/apache/spark/.latest-doc-only-change.txt +++ b/airflow/providers/apache/spark/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/apache/sqoop/.latest-doc-only-change.txt b/airflow/providers/apache/sqoop/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/apache/sqoop/.latest-doc-only-change.txt +++ b/airflow/providers/apache/sqoop/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/apprise/CHANGELOG.rst b/airflow/providers/apprise/CHANGELOG.rst index 0b1586bbfafd..9b5d755cbec9 100644 --- a/airflow/providers/apprise/CHANGELOG.rst +++ b/airflow/providers/apprise/CHANGELOG.rst @@ -20,3 +20,8 @@ Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. + +1.0.0 +..... + +Initial version of the provider. diff --git a/airflow/providers/arangodb/.latest-doc-only-change.txt b/airflow/providers/arangodb/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/arangodb/.latest-doc-only-change.txt +++ b/airflow/providers/arangodb/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/asana/.latest-doc-only-change.txt b/airflow/providers/asana/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/asana/.latest-doc-only-change.txt +++ b/airflow/providers/asana/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/celery/.latest-doc-only-change.txt b/airflow/providers/celery/.latest-doc-only-change.txt index ff7136e07d74..4d0c8bcde51e 100644 --- a/airflow/providers/celery/.latest-doc-only-change.txt +++ b/airflow/providers/celery/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +8c37b74a208a808d905c1b86d081d69d7a1aa900 diff --git a/airflow/providers/cloudant/.latest-doc-only-change.txt b/airflow/providers/cloudant/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/cloudant/.latest-doc-only-change.txt +++ b/airflow/providers/cloudant/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/cncf/kubernetes/CHANGELOG.rst b/airflow/providers/cncf/kubernetes/CHANGELOG.rst index bae95c0297dd..f90802cbe402 100644 --- a/airflow/providers/cncf/kubernetes/CHANGELOG.rst +++ b/airflow/providers/cncf/kubernetes/CHANGELOG.rst @@ -24,6 +24,26 @@ Changelog for ``apache-airflow-providers-cncf-kubernetes`` ---------------------------------------------------------- +7.2.0 +..... + +Features +~~~~~~~~ + +* ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` + +Bug Fixes +~~~~~~~~~ + +* ``Fix KubernetesPodOperator validate xcom json and add retries (#32113)`` +* ``Fix 'KubernetesPodTrigger' waiting strategy (#31348)`` +* ``fix spark-kubernetes-operator compatibality (#31798)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` + 7.1.0 ..... diff --git a/airflow/providers/cncf/kubernetes/__init__.py b/airflow/providers/cncf/kubernetes/__init__.py index 3e3bd8ce3117..3235c02fd586 100644 --- a/airflow/providers/cncf/kubernetes/__init__.py +++ b/airflow/providers/cncf/kubernetes/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "7.1.0" +__version__ = "7.2.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/cncf/kubernetes/provider.yaml b/airflow/providers/cncf/kubernetes/provider.yaml index 7375b7b35276..f65527ded795 100644 --- a/airflow/providers/cncf/kubernetes/provider.yaml +++ b/airflow/providers/cncf/kubernetes/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 7.2.0 - 7.1.0 - 7.0.0 - 6.1.0 diff --git a/airflow/providers/common/sql/CHANGELOG.rst b/airflow/providers/common/sql/CHANGELOG.rst index 73400cd5c627..5710d8e39dda 100644 --- a/airflow/providers/common/sql/CHANGELOG.rst +++ b/airflow/providers/common/sql/CHANGELOG.rst @@ -24,6 +24,19 @@ Changelog for ``apache-airflow-providers-common-sql`` ----------------------------------------------------- +1.6.0 +..... + +Features +~~~~~~~~ + +* ``openlineage, common.sql: provide OL SQL parser as internal OpenLineage provider API (#31398)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` + * ``Improve provider documentation and README structure (#32125)`` + 1.5.2 ..... diff --git a/airflow/providers/common/sql/__init__.py b/airflow/providers/common/sql/__init__.py index 58ca3b0bc20f..6180b06a7bca 100644 --- a/airflow/providers/common/sql/__init__.py +++ b/airflow/providers/common/sql/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "1.5.2" +__version__ = "1.6.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/common/sql/provider.yaml b/airflow/providers/common/sql/provider.yaml index dc2630336055..e1064b37f51f 100644 --- a/airflow/providers/common/sql/provider.yaml +++ b/airflow/providers/common/sql/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 1.6.0 - 1.5.2 - 1.5.1 - 1.5.0 diff --git a/airflow/providers/databricks/CHANGELOG.rst b/airflow/providers/databricks/CHANGELOG.rst index de5a5df4cc7c..21bf6efebcf9 100644 --- a/airflow/providers/databricks/CHANGELOG.rst +++ b/airflow/providers/databricks/CHANGELOG.rst @@ -24,6 +24,19 @@ Changelog for ``apache-airflow-providers-databricks`` ----------------------------------------------------- +4.3.1 +..... + +Bug Fixes +~~~~~~~~~ + +* ``Modify 'template_fields' of 'DatabricksSqlOperator' to support parent class fields (#32253)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``D205 Support - Providers: Databricks to Github (inclusive) (#32243)`` + * ``Improve provider documentation and README structure (#32125)`` + 4.3.0 ..... diff --git a/airflow/providers/databricks/__init__.py b/airflow/providers/databricks/__init__.py index 78d3cd8c42cf..704af2e7674b 100644 --- a/airflow/providers/databricks/__init__.py +++ b/airflow/providers/databricks/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "4.3.0" +__version__ = "4.3.1" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/databricks/provider.yaml b/airflow/providers/databricks/provider.yaml index 9aad0384eea0..87ca83c99e47 100644 --- a/airflow/providers/databricks/provider.yaml +++ b/airflow/providers/databricks/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 4.3.1 - 4.3.0 - 4.2.0 - 4.1.0 diff --git a/airflow/providers/datadog/.latest-doc-only-change.txt b/airflow/providers/datadog/.latest-doc-only-change.txt index ff7136e07d74..4ca04d3bd48b 100644 --- a/airflow/providers/datadog/.latest-doc-only-change.txt +++ b/airflow/providers/datadog/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/dbt/cloud/.latest-doc-only-change.txt b/airflow/providers/dbt/cloud/.latest-doc-only-change.txt index ab24993f5713..4ca04d3bd48b 100644 --- a/airflow/providers/dbt/cloud/.latest-doc-only-change.txt +++ b/airflow/providers/dbt/cloud/.latest-doc-only-change.txt @@ -1 +1 @@ -8b6b0848a3cacf9999477d6af4d2a87463f03026 +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/dingding/.latest-doc-only-change.txt b/airflow/providers/dingding/.latest-doc-only-change.txt index ff7136e07d74..4ca04d3bd48b 100644 --- a/airflow/providers/dingding/.latest-doc-only-change.txt +++ b/airflow/providers/dingding/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/discord/.latest-doc-only-change.txt b/airflow/providers/discord/.latest-doc-only-change.txt index ff7136e07d74..4ca04d3bd48b 100644 --- a/airflow/providers/discord/.latest-doc-only-change.txt +++ b/airflow/providers/discord/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/docker/.latest-doc-only-change.txt b/airflow/providers/docker/.latest-doc-only-change.txt index 9ab50d6e01d3..4ca04d3bd48b 100644 --- a/airflow/providers/docker/.latest-doc-only-change.txt +++ b/airflow/providers/docker/.latest-doc-only-change.txt @@ -1 +1 @@ -6933022e94acf139b2dea9a589bb8b25c62a5d20 +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/elasticsearch/CHANGELOG.rst b/airflow/providers/elasticsearch/CHANGELOG.rst index bb489325a6e4..c104ed62ece0 100644 --- a/airflow/providers/elasticsearch/CHANGELOG.rst +++ b/airflow/providers/elasticsearch/CHANGELOG.rst @@ -27,9 +27,17 @@ Changelog for ``apache-airflow-providers-elasticsearch`` 5.0.0 ..... +Breaking changes +~~~~~~~~~~~~~~~~ + .. note:: Deprecate non-official elasticsearch libraries. Only the official elasticsearch library was used +* ``Deprecate the 2 non-official elasticsearch libraries (#31920)`` + +.. Review and move the new changes to one of the sections above: + * ``D205 Support - Providers: Databricks to Github (inclusive) (#32243)`` + * ``Improve provider documentation and README structure (#32125)`` 4.5.1 ..... diff --git a/airflow/providers/elasticsearch/__init__.py b/airflow/providers/elasticsearch/__init__.py index f7905972eeea..b5638d95b1cb 100644 --- a/airflow/providers/elasticsearch/__init__.py +++ b/airflow/providers/elasticsearch/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "4.5.1" +__version__ = "5.0.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/exasol/.latest-doc-only-change.txt b/airflow/providers/exasol/.latest-doc-only-change.txt index e7c3c940c9c7..33a07c855e0f 100644 --- a/airflow/providers/exasol/.latest-doc-only-change.txt +++ b/airflow/providers/exasol/.latest-doc-only-change.txt @@ -1 +1 @@ -602abe8394fafe7de54df7e73af56de848cdf617 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/facebook/.latest-doc-only-change.txt b/airflow/providers/facebook/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/facebook/.latest-doc-only-change.txt +++ b/airflow/providers/facebook/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/ftp/.latest-doc-only-change.txt b/airflow/providers/ftp/.latest-doc-only-change.txt index ff7136e07d74..4ca04d3bd48b 100644 --- a/airflow/providers/ftp/.latest-doc-only-change.txt +++ b/airflow/providers/ftp/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/github/.latest-doc-only-change.txt b/airflow/providers/github/.latest-doc-only-change.txt index ff7136e07d74..4ca04d3bd48b 100644 --- a/airflow/providers/github/.latest-doc-only-change.txt +++ b/airflow/providers/github/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +d1aa509bbd1941ceb3fe31789efeebbddd58d32f diff --git a/airflow/providers/google/CHANGELOG.rst b/airflow/providers/google/CHANGELOG.rst index 7c39d83e3c8d..630467f4b9e5 100644 --- a/airflow/providers/google/CHANGELOG.rst +++ b/airflow/providers/google/CHANGELOG.rst @@ -23,6 +23,38 @@ Changelog for ``apache-airflow-providers-google`` ------------------------------------------------- +10.3.0 +...... + +Features +~~~~~~~~ + +* ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` +* ``Add deferrable mode to CloudSQLExportInstanceOperator (#30852)`` +* ``Adding 'src_fmt_configs' to the list of template fields. (#32097)`` + +Bug Fixes +~~~~~~~~~ + +* ``[Issue-32069] Fix name format in the batch requests (#32070)`` +* ``Fix 'BigQueryInsertJobOperator' error handling in deferrable mode (#32034)`` +* ``Fix 'BIGQUERY_JOB_DETAILS_LINK_FMT' in 'BigQueryConsoleLink' (#31953)`` +* ``Make the deferrable version of DataprocCreateBatchOperator handle a batch_id that already exists (#32216)`` + + +Misc +~~~~ + +* ``Switch Google Ads API version from v13 to v14 (#32028)`` +* ``Deprecate 'delimiter' param and source object's wildcards in GCS, introduce 'match_glob' param. (#31261)`` +* ``Refactor GKECreateClusterOperator's body validation (#31923)`` +* ``Optimize deferrable mode execution for 'BigQueryValueCheckOperator' (#31872)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Improve provider documentation and README structure (#32125)`` + * ``Google provider docstring improvements (#31731)`` + 10.2.0 ...... diff --git a/airflow/providers/google/__init__.py b/airflow/providers/google/__init__.py index 245255b39e61..0bf8186b4a75 100644 --- a/airflow/providers/google/__init__.py +++ b/airflow/providers/google/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "10.2.0" +__version__ = "10.3.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml index f22ff420821d..02168a12c953 100644 --- a/airflow/providers/google/provider.yaml +++ b/airflow/providers/google/provider.yaml @@ -30,6 +30,7 @@ description: | suspended: false versions: + - 10.3.0 - 10.2.0 - 10.1.1 - 10.1.0 diff --git a/airflow/providers/grpc/.latest-doc-only-change.txt b/airflow/providers/grpc/.latest-doc-only-change.txt index 86f00ad558af..33a07c855e0f 100644 --- a/airflow/providers/grpc/.latest-doc-only-change.txt +++ b/airflow/providers/grpc/.latest-doc-only-change.txt @@ -1 +1 @@ -25bdbc8e6768712bad6043618242eec9c6632618 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/hashicorp/CHANGELOG.rst b/airflow/providers/hashicorp/CHANGELOG.rst index 88ca2da0256d..0b314faafb74 100644 --- a/airflow/providers/hashicorp/CHANGELOG.rst +++ b/airflow/providers/hashicorp/CHANGELOG.rst @@ -24,6 +24,18 @@ Changelog for ``apache-airflow-providers-hashicorp`` ---------------------------------------------------- +3.4.2 +..... + +Misc +~~~~ + +* ``Add default retry to hvac client requests (#31073)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Improve provider documentation and README structure (#32125)`` + 3.4.1 ..... diff --git a/airflow/providers/hashicorp/__init__.py b/airflow/providers/hashicorp/__init__.py index f58108903de8..d9d0ceaadc35 100644 --- a/airflow/providers/hashicorp/__init__.py +++ b/airflow/providers/hashicorp/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "3.4.1" +__version__ = "3.4.2" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/hashicorp/provider.yaml b/airflow/providers/hashicorp/provider.yaml index 2faa1009a245..46399874d9c5 100644 --- a/airflow/providers/hashicorp/provider.yaml +++ b/airflow/providers/hashicorp/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 3.4.2 - 3.4.1 - 3.4.0 - 3.3.1 diff --git a/airflow/providers/http/.latest-doc-only-change.txt b/airflow/providers/http/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/http/.latest-doc-only-change.txt +++ b/airflow/providers/http/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/imap/.latest-doc-only-change.txt b/airflow/providers/imap/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/imap/.latest-doc-only-change.txt +++ b/airflow/providers/imap/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/influxdb/.latest-doc-only-change.txt b/airflow/providers/influxdb/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/influxdb/.latest-doc-only-change.txt +++ b/airflow/providers/influxdb/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/jdbc/.latest-doc-only-change.txt b/airflow/providers/jdbc/.latest-doc-only-change.txt index ab24993f5713..33a07c855e0f 100644 --- a/airflow/providers/jdbc/.latest-doc-only-change.txt +++ b/airflow/providers/jdbc/.latest-doc-only-change.txt @@ -1 +1 @@ -8b6b0848a3cacf9999477d6af4d2a87463f03026 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/jenkins/.latest-doc-only-change.txt b/airflow/providers/jenkins/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/jenkins/.latest-doc-only-change.txt +++ b/airflow/providers/jenkins/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/microsoft/azure/CHANGELOG.rst b/airflow/providers/microsoft/azure/CHANGELOG.rst index 87424de9b624..223aea264997 100644 --- a/airflow/providers/microsoft/azure/CHANGELOG.rst +++ b/airflow/providers/microsoft/azure/CHANGELOG.rst @@ -24,6 +24,33 @@ Changelog for ``apache-airflow-providers-microsoft-azure`` ---------------------------------------------------------- +6.2.0 +..... + +Features +~~~~~~~~ + +* ``Adds connection test for ADLS Gen2 (#32126)`` +* ``Add option to pass extra configs to ClientSecretCredential (#31783)`` +* ``Added 'AzureBlobStorageToS3Operator' transfer operator (#32270)`` + +Bug Fixes +~~~~~~~~~ + +* ``Cancel pipeline if unexpected exception caught (#32238)`` +* ``Fix where account url is build if not provided using login (account name) (#32082)`` + +Misc +~~~~ + +* ``Doc changes: Added Transfers section in Azure provider docs (#32241)`` +* ``Adds Sensor section in the Azure providers docs (#32299)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Improve provider documentation and README structure (#32125)`` + * ``invalid args fix (#32326)`` + 6.1.2 ..... diff --git a/airflow/providers/microsoft/azure/__init__.py b/airflow/providers/microsoft/azure/__init__.py index 6a11fab18c2f..40f553f79fb7 100644 --- a/airflow/providers/microsoft/azure/__init__.py +++ b/airflow/providers/microsoft/azure/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "6.1.2" +__version__ = "6.2.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/microsoft/azure/provider.yaml b/airflow/providers/microsoft/azure/provider.yaml index c255da931522..a1070177bc11 100644 --- a/airflow/providers/microsoft/azure/provider.yaml +++ b/airflow/providers/microsoft/azure/provider.yaml @@ -22,6 +22,7 @@ description: | `Microsoft Azure `__ suspended: false versions: + - 6.2.0 - 6.1.2 - 6.1.1 - 6.1.0 diff --git a/airflow/providers/microsoft/mssql/.latest-doc-only-change.txt b/airflow/providers/microsoft/mssql/.latest-doc-only-change.txt index e7c3c940c9c7..33a07c855e0f 100644 --- a/airflow/providers/microsoft/mssql/.latest-doc-only-change.txt +++ b/airflow/providers/microsoft/mssql/.latest-doc-only-change.txt @@ -1 +1 @@ -602abe8394fafe7de54df7e73af56de848cdf617 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/microsoft/psrp/.latest-doc-only-change.txt b/airflow/providers/microsoft/psrp/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/microsoft/psrp/.latest-doc-only-change.txt +++ b/airflow/providers/microsoft/psrp/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/microsoft/winrm/.latest-doc-only-change.txt b/airflow/providers/microsoft/winrm/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/microsoft/winrm/.latest-doc-only-change.txt +++ b/airflow/providers/microsoft/winrm/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/mongo/.latest-doc-only-change.txt b/airflow/providers/mongo/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/mongo/.latest-doc-only-change.txt +++ b/airflow/providers/mongo/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/mysql/.latest-doc-only-change.txt b/airflow/providers/mysql/.latest-doc-only-change.txt index 13020f96f448..33a07c855e0f 100644 --- a/airflow/providers/mysql/.latest-doc-only-change.txt +++ b/airflow/providers/mysql/.latest-doc-only-change.txt @@ -1 +1 @@ -df00436569bb6fb79ce8c0b7ca71dddf02b854ef +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/neo4j/.latest-doc-only-change.txt b/airflow/providers/neo4j/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/neo4j/.latest-doc-only-change.txt +++ b/airflow/providers/neo4j/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/odbc/.latest-doc-only-change.txt b/airflow/providers/odbc/.latest-doc-only-change.txt index e7e8156d80b9..33a07c855e0f 100644 --- a/airflow/providers/odbc/.latest-doc-only-change.txt +++ b/airflow/providers/odbc/.latest-doc-only-change.txt @@ -1 +1 @@ -b916b7507921129dc48d6add1bdc4b923b60c9b9 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/openfaas/.latest-doc-only-change.txt b/airflow/providers/openfaas/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/openfaas/.latest-doc-only-change.txt +++ b/airflow/providers/openfaas/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/opsgenie/.latest-doc-only-change.txt b/airflow/providers/opsgenie/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/opsgenie/.latest-doc-only-change.txt +++ b/airflow/providers/opsgenie/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/oracle/.latest-doc-only-change.txt b/airflow/providers/oracle/.latest-doc-only-change.txt index ab24993f5713..a780a5d07a3c 100644 --- a/airflow/providers/oracle/.latest-doc-only-change.txt +++ b/airflow/providers/oracle/.latest-doc-only-change.txt @@ -1 +1 @@ -8b6b0848a3cacf9999477d6af4d2a87463f03026 +2794c4172e7b7655b813236fe47222cec46f57e4 diff --git a/airflow/providers/pagerduty/.latest-doc-only-change.txt b/airflow/providers/pagerduty/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/pagerduty/.latest-doc-only-change.txt +++ b/airflow/providers/pagerduty/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/papermill/.latest-doc-only-change.txt b/airflow/providers/papermill/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/papermill/.latest-doc-only-change.txt +++ b/airflow/providers/papermill/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/plexus/.latest-doc-only-change.txt b/airflow/providers/plexus/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/plexus/.latest-doc-only-change.txt +++ b/airflow/providers/plexus/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/postgres/.latest-doc-only-change.txt b/airflow/providers/postgres/.latest-doc-only-change.txt index 570fad6daee2..33a07c855e0f 100644 --- a/airflow/providers/postgres/.latest-doc-only-change.txt +++ b/airflow/providers/postgres/.latest-doc-only-change.txt @@ -1 +1 @@ -97496ba2b41063fa24393c58c5c648a0cdb5a7f8 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/presto/.latest-doc-only-change.txt b/airflow/providers/presto/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/presto/.latest-doc-only-change.txt +++ b/airflow/providers/presto/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/qubole/.latest-doc-only-change.txt b/airflow/providers/qubole/.latest-doc-only-change.txt index ab24993f5713..33a07c855e0f 100644 --- a/airflow/providers/qubole/.latest-doc-only-change.txt +++ b/airflow/providers/qubole/.latest-doc-only-change.txt @@ -1 +1 @@ -8b6b0848a3cacf9999477d6af4d2a87463f03026 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/redis/.latest-doc-only-change.txt b/airflow/providers/redis/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/redis/.latest-doc-only-change.txt +++ b/airflow/providers/redis/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/salesforce/.latest-doc-only-change.txt b/airflow/providers/salesforce/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/salesforce/.latest-doc-only-change.txt +++ b/airflow/providers/salesforce/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/samba/.latest-doc-only-change.txt b/airflow/providers/samba/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/samba/.latest-doc-only-change.txt +++ b/airflow/providers/samba/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/segment/.latest-doc-only-change.txt b/airflow/providers/segment/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/segment/.latest-doc-only-change.txt +++ b/airflow/providers/segment/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/sendgrid/.latest-doc-only-change.txt b/airflow/providers/sendgrid/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/sendgrid/.latest-doc-only-change.txt +++ b/airflow/providers/sendgrid/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/sftp/.latest-doc-only-change.txt b/airflow/providers/sftp/.latest-doc-only-change.txt index 28124098645c..33a07c855e0f 100644 --- a/airflow/providers/sftp/.latest-doc-only-change.txt +++ b/airflow/providers/sftp/.latest-doc-only-change.txt @@ -1 +1 @@ -6c3a67d4fccafe4ab6cd9ec8c7bacf2677f17038 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/singularity/.latest-doc-only-change.txt b/airflow/providers/singularity/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/singularity/.latest-doc-only-change.txt +++ b/airflow/providers/singularity/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/slack/.latest-doc-only-change.txt b/airflow/providers/slack/.latest-doc-only-change.txt index d947e4964a38..33a07c855e0f 100644 --- a/airflow/providers/slack/.latest-doc-only-change.txt +++ b/airflow/providers/slack/.latest-doc-only-change.txt @@ -1 +1 @@ -b5b1fae2dfe92751d6aaaace00009ce29625095b +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/snowflake/.latest-doc-only-change.txt b/airflow/providers/snowflake/.latest-doc-only-change.txt index 5bda81758205..33a07c855e0f 100644 --- a/airflow/providers/snowflake/.latest-doc-only-change.txt +++ b/airflow/providers/snowflake/.latest-doc-only-change.txt @@ -1 +1 @@ -814e471d137aad68bd64a21d20736e7b88403f97 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/sqlite/.latest-doc-only-change.txt b/airflow/providers/sqlite/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/sqlite/.latest-doc-only-change.txt +++ b/airflow/providers/sqlite/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/ssh/.latest-doc-only-change.txt b/airflow/providers/ssh/.latest-doc-only-change.txt index 570fad6daee2..33a07c855e0f 100644 --- a/airflow/providers/ssh/.latest-doc-only-change.txt +++ b/airflow/providers/ssh/.latest-doc-only-change.txt @@ -1 +1 @@ -97496ba2b41063fa24393c58c5c648a0cdb5a7f8 +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/tableau/.latest-doc-only-change.txt b/airflow/providers/tableau/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/tableau/.latest-doc-only-change.txt +++ b/airflow/providers/tableau/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/tabular/.latest-doc-only-change.txt b/airflow/providers/tabular/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/tabular/.latest-doc-only-change.txt +++ b/airflow/providers/tabular/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/telegram/.latest-doc-only-change.txt b/airflow/providers/telegram/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/telegram/.latest-doc-only-change.txt +++ b/airflow/providers/telegram/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/airflow/providers/trino/CHANGELOG.rst b/airflow/providers/trino/CHANGELOG.rst index 7e5b05538ebc..da3fd0695c17 100644 --- a/airflow/providers/trino/CHANGELOG.rst +++ b/airflow/providers/trino/CHANGELOG.rst @@ -24,6 +24,18 @@ Changelog for ``apache-airflow-providers-trino`` ------------------------------------------------ +5.2.0 +..... + +Features +~~~~~~~~ + +* ``Trino Hook: Add ability to read JWT from file (#31950)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Improve provider documentation and README structure (#32125)`` + 5.1.1 ..... diff --git a/airflow/providers/trino/__init__.py b/airflow/providers/trino/__init__.py index 37cb07d1a46e..7ecb43088fc6 100644 --- a/airflow/providers/trino/__init__.py +++ b/airflow/providers/trino/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "5.1.1" +__version__ = "5.2.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/trino/provider.yaml b/airflow/providers/trino/provider.yaml index 6c4c61a63ffe..97fe20574b11 100644 --- a/airflow/providers/trino/provider.yaml +++ b/airflow/providers/trino/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 5.2.0 - 5.1.1 - 5.1.0 - 5.0.0 diff --git a/airflow/providers/vertica/CHANGELOG.rst b/airflow/providers/vertica/CHANGELOG.rst index db9a5335b85c..36e06b2cf5a6 100644 --- a/airflow/providers/vertica/CHANGELOG.rst +++ b/airflow/providers/vertica/CHANGELOG.rst @@ -24,6 +24,18 @@ Changelog for ``apache-airflow-providers-vertica`` -------------------------------------------------- +3.5.0 +..... + +Features +~~~~~~~~ + +* ``Add various Vertica connection parameters (#32089)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Improve provider documentation and README structure (#32125)`` + 3.4.1 ..... diff --git a/airflow/providers/vertica/__init__.py b/airflow/providers/vertica/__init__.py index feba3e282653..94565d1472c8 100644 --- a/airflow/providers/vertica/__init__.py +++ b/airflow/providers/vertica/__init__.py @@ -28,7 +28,7 @@ __all__ = ["__version__"] -__version__ = "3.4.1" +__version__ = "3.5.0" try: from airflow import __version__ as airflow_version diff --git a/airflow/providers/vertica/provider.yaml b/airflow/providers/vertica/provider.yaml index 90498fa1a1cd..d9f30e37b014 100644 --- a/airflow/providers/vertica/provider.yaml +++ b/airflow/providers/vertica/provider.yaml @@ -23,6 +23,7 @@ description: | suspended: false versions: + - 3.5.0 - 3.4.1 - 3.4.0 - 3.3.1 diff --git a/airflow/providers/zendesk/.latest-doc-only-change.txt b/airflow/providers/zendesk/.latest-doc-only-change.txt index ff7136e07d74..33a07c855e0f 100644 --- a/airflow/providers/zendesk/.latest-doc-only-change.txt +++ b/airflow/providers/zendesk/.latest-doc-only-change.txt @@ -1 +1 @@ -06acf40a4337759797f666d5bb27a5a393b74fed +09d4718d3a46aecf3355d14d3d23022002f4a818 diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py index 8bf6e24852dc..cde0918381cf 100755 --- a/dev/provider_packages/prepare_provider_packages.py +++ b/dev/provider_packages/prepare_provider_packages.py @@ -2014,7 +2014,12 @@ def generate_new_changelog(package_id, provider_details, changelog_path, changes template_name="UPDATE_CHANGELOG", context=context, extension=".rst" ) else: - classified_changes = get_changes_classified(changes[0]) + if changes: + classified_changes = get_changes_classified(changes[0]) + else: + # change log exist but without version 1.0.0 entry + classified_changes = None + context = { "version": latest_version, "version_header": "." * len(latest_version), diff --git a/docs/apache-airflow-providers-alibaba/commits.rst b/docs/apache-airflow-providers-alibaba/commits.rst index f01e83594ba2..bb9559028c2a 100644 --- a/docs/apache-airflow-providers-alibaba/commits.rst +++ b/docs/apache-airflow-providers-alibaba/commits.rst @@ -28,6 +28,19 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-28 ``D205 Support - Providers: Airbyte and Alibaba (#32214)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`f05beb7219 `_ 2023-06-27 ``Add Alibaba Cloud AnalyticDB Spark Support (#31787)`` +================================================================================================= =========== ================================================================ + 2.4.1 ..... @@ -36,6 +49,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)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` `9fa75aaf7a `_ 2023-05-29 ``Remove Python 3.7 support (#30963)`` diff --git a/docs/apache-airflow-providers-alibaba/index.rst b/docs/apache-airflow-providers-alibaba/index.rst index 2b468623ff99..2c8f671b549a 100644 --- a/docs/apache-airflow-providers-alibaba/index.rst +++ b/docs/apache-airflow-providers-alibaba/index.rst @@ -77,7 +77,7 @@ Package apache-airflow-providers-alibaba Alibaba Cloud integration (including `Alibaba Cloud `__). -Release: 2.4.1 +Release: 2.5.0 Provider package ---------------- diff --git a/docs/apache-airflow-providers-amazon/commits.rst b/docs/apache-airflow-providers-amazon/commits.rst index 94b8c2bbfa60..fdceb0af6634 100644 --- a/docs/apache-airflow-providers-amazon/commits.rst +++ b/docs/apache-airflow-providers-amazon/commits.rst @@ -28,6 +28,48 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-07-04 ``Added 'AzureBlobStorageToS3Operator' transfer operator (#32270)`` +`5c72befcfd `_ 2023-07-03 ``Fix 'LambdaInvokeFunctionOperator' payload parameter type (#32259)`` +`d6e254db68 `_ 2023-06-30 ``Deprecate 'delimiter' param and source object's wildcards in GCS, introduce 'match_glob' param. (#31261)`` +`dd937e51fe `_ 2023-06-30 ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` +`e781aef1a7 `_ 2023-06-29 ``aws waiter util: log status info with error level on waiter error (#32247)`` +`a46b942f48 `_ 2023-06-29 ``Add deferrable mode to S3KeysUnchangedSensor (#31940)`` +`b1b69af88f `_ 2023-06-28 ``Add deferrable mode to 'RdsCreateDbInstanceOperator' and 'RdsDeleteDbInstanceOperator' (#32171)`` +`d029e043f4 `_ 2023-06-28 ``Add 'ChimeNotifier' (#32222)`` +`1d60332cf8 `_ 2023-06-28 ``Add an option to 'GlueJobOperator' to stop the job run when the TI is killed (#32155)`` +`bd2f156bc8 `_ 2023-06-28 ``D205 Support - Providers: Amazon/AWS (#32224)`` +`25526a112f `_ 2023-06-28 ``Add deferrable mode for 'AthenaOperator' (#32186)`` +`1fb2831239 `_ 2023-06-28 ``Add a deferrable mode to 'BatchCreateComputeEnvironmentOperator' (#32036)`` +`14785bc84c `_ 2023-06-27 ``deferrable mode for 'SageMakerTuningOperator' and 'SageMakerEndpointOperator' (#32112)`` +`2794c4172e `_ 2023-06-27 ``D205 Support - Auto-fixes and Stragglers (#32212)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`06b5a1e850 `_ 2023-06-27 ``Add deferrable mode in EMR operator and sensor (#32029)`` +`3a85d4e7e8 `_ 2023-06-27 ``Revert "add deferrable mode for 'AthenaOperator' (#32110)" (#32172)`` +`256438c3d6 `_ 2023-06-27 ``add deferrable mode for 'AthenaOperator' (#32110)`` +`e0f4de347f `_ 2023-06-27 ``EKS Create/Delete Nodegroup Deferrable mode (#32165)`` +`10df7436f3 `_ 2023-06-27 ``Add 'ChimeWebhookHook' (#31939)`` +`e4eb19866e `_ 2023-06-26 ``rewrite method used in ecs to fetch less logs (#31786)`` +`2f96127b9e `_ 2023-06-26 ``Minor name change for the util wait method. (#32152)`` +`cc87ae578e `_ 2023-06-26 ``feature: AWS - GlueJobOperator - job_poll_interval (#32147)`` +`e4468ed380 `_ 2023-06-26 ``Clean up string concatenation (#32129)`` +`8057659e96 `_ 2023-06-25 ``Handle 'UnboundLocalError' while parsing invalid 's3_url' (#32120)`` +`d49fa999a9 `_ 2023-06-23 ``bugfix: break down run+wait method in ECS operator (#32104)`` +`72d09a677f `_ 2023-06-23 ``Use a waiter in 'AthenaHook' (#31942)`` +`5c887988b0 `_ 2023-06-23 ``Refactor Eks Create Cluster Operator code (#31960)`` +`415e076761 `_ 2023-06-23 ``Deferrable mode for ECS operators (#31881)`` +`479719297f `_ 2023-06-22 ``add async wait method to the "with logging" aws utils (#32055)`` +`afe293d934 `_ 2023-06-21 ``cleanup Amazon CHANGELOG.rst (#32031)`` +`743bf5a0ae `_ 2023-06-20 ``Add custom waiters to EMR Serverless (#30463)`` +================================================================================================= =========== ============================================================================================================ + 8.2.0 ..... @@ -36,6 +78,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)`` `f0b91ac6a7 `_ 2023-06-20 ``Add 'deferrable' param in 'EmrContainerSensor' (#30945)`` `e01ff4749c `_ 2023-06-19 ``Add realtime container execution logs for BatchOperator (#31837)`` diff --git a/docs/apache-airflow-providers-amazon/index.rst b/docs/apache-airflow-providers-amazon/index.rst index 85d5542f8c00..b018d27cb0a7 100644 --- a/docs/apache-airflow-providers-amazon/index.rst +++ b/docs/apache-airflow-providers-amazon/index.rst @@ -82,7 +82,7 @@ Package apache-airflow-providers-amazon Amazon integration (including `Amazon Web Services (AWS) `__). -Release: 8.2.0 +Release: 8.3.0 Provider package ---------------- @@ -107,6 +107,7 @@ PIP package Version required ======================================= ================== ``apache-airflow`` ``>=2.4.0`` ``apache-airflow-providers-common-sql`` ``>=1.3.1`` +``apache-airflow-providers-http`` ``boto3`` ``>=1.24.0`` ``asgiref`` ``watchtower`` ``~=2.0.1`` @@ -144,6 +145,7 @@ Dependent package `apache-airflow-providers-google `_ ``google`` `apache-airflow-providers-http `_ ``http`` `apache-airflow-providers-imap `_ ``imap`` +`apache-airflow-providers-microsoft-azure `_ ``microsoft.azure`` `apache-airflow-providers-mongo `_ ``mongo`` `apache-airflow-providers-salesforce `_ ``salesforce`` `apache-airflow-providers-ssh `_ ``ssh`` @@ -155,5 +157,5 @@ 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-amazon 8.2.0 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-amazon 8.2.0 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-amazon 8.3.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-amazon 8.3.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-apache-kafka/commits.rst b/docs/apache-airflow-providers-apache-kafka/commits.rst index e15017bf58ee..94450c54b479 100644 --- a/docs/apache-airflow-providers-apache-kafka/commits.rst +++ b/docs/apache-airflow-providers-apache-kafka/commits.rst @@ -28,6 +28,19 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-30 ``Break AwaitMessageTrigger execution when finding a message with the desired format (#31803)`` +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== =============================================================================================== + 1.1.1 ..... @@ -36,6 +49,7 @@ Latest change: 2023-06-20 ================================================================================================= =========== ==================================================================== Commit Committed Subject ================================================================================================= =========== ==================================================================== +`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` `547f5846bf `_ 2023-06-20 ``Add note about dropping Python 3.7 for kafka and impala (#32017)`` `dc5bf3fd02 `_ 2023-06-02 ``Add discoverability for triggers in provider.yaml (#31576)`` `a473facf6c `_ 2023-06-01 ``Add D400 pydocstyle check - Apache providers only (#31424)`` diff --git a/docs/apache-airflow-providers-apache-kafka/index.rst b/docs/apache-airflow-providers-apache-kafka/index.rst index 2c56ce00460f..d3d7a8f66b34 100644 --- a/docs/apache-airflow-providers-apache-kafka/index.rst +++ b/docs/apache-airflow-providers-apache-kafka/index.rst @@ -82,7 +82,7 @@ Package apache-airflow-providers-apache-kafka `Apache Kafka `__ -Release: 1.1.1 +Release: 1.1.2 Provider package ---------------- diff --git a/docs/apache-airflow-providers-apache-livy/commits.rst b/docs/apache-airflow-providers-apache-livy/commits.rst index 3580de1bbf5b..f3685d6d9363 100644 --- a/docs/apache-airflow-providers-apache-livy/commits.rst +++ b/docs/apache-airflow-providers-apache-livy/commits.rst @@ -28,6 +28,19 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`58784ade88 `_ 2023-06-27 ``Fix 'LivyHook' TypeError exception on 'session_id' log format (#32051)`` +================================================================================================= =========== ========================================================================== + 3.5.1 ..... @@ -36,6 +49,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)`` `13890788ae `_ 2023-06-07 ``Apache provider docstring improvements (#31730)`` `361aa77d5d `_ 2023-06-05 ``Optimize deferred mode execution (#31685)`` diff --git a/docs/apache-airflow-providers-apache-livy/index.rst b/docs/apache-airflow-providers-apache-livy/index.rst index 4db63c3ae908..6b7704c37023 100644 --- a/docs/apache-airflow-providers-apache-livy/index.rst +++ b/docs/apache-airflow-providers-apache-livy/index.rst @@ -75,7 +75,7 @@ Package apache-airflow-providers-apache-livy `Apache Livy `__ -Release: 3.5.1 +Release: 3.5.2 Provider package ---------------- @@ -129,5 +129,5 @@ 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-livy 3.5.1 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-apache-livy 3.5.1 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-apache-livy 3.5.2 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-apache-livy 3.5.2 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-apprise/commits.rst b/docs/apache-airflow-providers-apprise/commits.rst index 9514a7d2d7be..af71b748389e 100644 --- a/docs/apache-airflow-providers-apprise/commits.rst +++ b/docs/apache-airflow-providers-apprise/commits.rst @@ -1,3 +1,4 @@ + .. 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,7 +16,27 @@ specific language governing permissions and limitations under the License. + Package apache-airflow-providers-apprise ----------------------------------------- +------------------------------------------------------ `Apprise `__ + + +This is detailed commit list of changes for versions provider package: ``apprise``. +For high-level changelog, see :doc:`package information including changelog `. + + + +1.0.0 +..... + +Latest change: 2023-06-28 + +================================================================================================= =========== =================================================================== +Commit Committed Subject +================================================================================================= =========== =================================================================== +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`f4c4b77486 `_ 2023-06-21 ``Add Apprise Provider (#31533)`` +================================================================================================= =========== =================================================================== diff --git a/docs/apache-airflow-providers-cncf-kubernetes/commits.rst b/docs/apache-airflow-providers-cncf-kubernetes/commits.rst index d0be3b36022e..070badf99a5a 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/commits.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/commits.rst @@ -28,6 +28,22 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-07-01 ``Fix KubernetesPodOperator validate xcom json and add retries (#32113)`` +`dd937e51fe `_ 2023-06-30 ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` +`8f5de83ee6 `_ 2023-06-29 ``Fix 'KubernetesPodTrigger' waiting strategy (#31348)`` +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`6693bdd72d `_ 2023-06-27 ``fix spark-kubernetes-operator compatibality (#31798)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== ========================================================================= + 7.1.0 ..... @@ -36,6 +52,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)`` `66299338eb `_ 2023-06-18 ``add a return when the event is yielded in a loop to stop the execution (#31985)`` `07ea574fed `_ 2023-06-07 ``Add D400 pydocstyle check (#31742)`` diff --git a/docs/apache-airflow-providers-cncf-kubernetes/index.rst b/docs/apache-airflow-providers-cncf-kubernetes/index.rst index f511ea5ecb58..d087d1d75759 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/index.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/index.rst @@ -77,7 +77,7 @@ Package apache-airflow-providers-cncf-kubernetes `Kubernetes `__ -Release: 7.1.0 +Release: 7.2.0 Provider package ---------------- diff --git a/docs/apache-airflow-providers-common-sql/commits.rst b/docs/apache-airflow-providers-common-sql/commits.rst index 385dfa45462e..d09b7c398104 100644 --- a/docs/apache-airflow-providers-common-sql/commits.rst +++ b/docs/apache-airflow-providers-common-sql/commits.rst @@ -28,18 +28,32 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-29 ``openlineage, common.sql: provide OL SQL parser as internal OpenLineage provider API (#31398)`` +`8c37b74a20 `_ 2023-06-28 ``D205 Support - Providers: Apache to Common (inclusive) (#32226)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== ================================================================================================= + 1.5.2 ..... -Latest change: 2023-06-05 +Latest change: 2023-06-20 -================================================================================================= =========== ================================================== +================================================================================================= =========== ============================================================= Commit Committed Subject -================================================================================================= =========== ================================================== +================================================================================================= =========== ============================================================= +`79bcc2e668 `_ 2023-06-20 ``Prepare RC1 docs for June 2023 wave of Providers (#32001)`` `9276310a43 `_ 2023-06-05 ``Improve docstrings in providers (#31681)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` `9fa75aaf7a `_ 2023-05-29 ``Remove Python 3.7 support (#30963)`` -================================================================================================= =========== ================================================== +================================================================================================= =========== ============================================================= 1.5.1 ..... diff --git a/docs/apache-airflow-providers-common-sql/index.rst b/docs/apache-airflow-providers-common-sql/index.rst index 027c9a436d3c..0e68b27f8957 100644 --- a/docs/apache-airflow-providers-common-sql/index.rst +++ b/docs/apache-airflow-providers-common-sql/index.rst @@ -76,7 +76,7 @@ Package apache-airflow-providers-common-sql `Common SQL Provider `__ -Release: 1.5.2 +Release: 1.6.0 Provider package ---------------- @@ -102,3 +102,31 @@ PIP package Version required ``apache-airflow`` ``>=2.4.0`` ``sqlparse`` ``>=0.4.2`` ================== ================== + +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-common-sql[openlineage] + + +============================================================================================================== =============== +Dependent package Extra +============================================================================================================== =============== +`apache-airflow-providers-openlineage `_ ``openlineage`` +============================================================================================================== =============== + +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-common-sql 1.6.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-common-sql 1.6.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-databricks/commits.rst b/docs/apache-airflow-providers-databricks/commits.rst index bfc05853edbd..b0df4f06c724 100644 --- a/docs/apache-airflow-providers-databricks/commits.rst +++ b/docs/apache-airflow-providers-databricks/commits.rst @@ -28,6 +28,19 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-29 ``Modify 'template_fields' of 'DatabricksSqlOperator' to support parent class fields (#32253)`` +`d1aa509bbd `_ 2023-06-28 ``D205 Support - Providers: Databricks to Github (inclusive) (#32243)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== =============================================================================================== + 4.3.0 ..... @@ -36,6 +49,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)`` `69bc90b824 `_ 2023-06-19 ``Fix type annotation (#31888)`` `66299338eb `_ 2023-06-18 ``add a return when the event is yielded in a loop to stop the execution (#31985)`` diff --git a/docs/apache-airflow-providers-databricks/index.rst b/docs/apache-airflow-providers-databricks/index.rst index c04b6d656c46..3d9e3672ec69 100644 --- a/docs/apache-airflow-providers-databricks/index.rst +++ b/docs/apache-airflow-providers-databricks/index.rst @@ -77,7 +77,7 @@ Package apache-airflow-providers-databricks `Databricks `__ -Release: 4.3.0 +Release: 4.3.1 Provider package ---------------- @@ -132,5 +132,5 @@ 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-databricks 4.3.0 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-databricks 4.3.0 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-databricks 4.3.1 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-databricks 4.3.1 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-elasticsearch/commits.rst b/docs/apache-airflow-providers-elasticsearch/commits.rst index 48b8cdbeefb8..774d5ea55f50 100644 --- a/docs/apache-airflow-providers-elasticsearch/commits.rst +++ b/docs/apache-airflow-providers-elasticsearch/commits.rst @@ -28,6 +28,19 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-28 ``D205 Support - Providers: Databricks to Github (inclusive) (#32243)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`b28c90354f `_ 2023-06-24 ``Deprecate the 2 non-official elasticsearch libraries (#31920)`` +================================================================================================= =========== ======================================================================= + 4.5.1 ..... @@ -36,6 +49,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)`` `9276310a43 `_ 2023-06-05 ``Improve docstrings in providers (#31681)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` diff --git a/docs/apache-airflow-providers-google/commits.rst b/docs/apache-airflow-providers-google/commits.rst index 22631f117d14..d69b4d009e78 100644 --- a/docs/apache-airflow-providers-google/commits.rst +++ b/docs/apache-airflow-providers-google/commits.rst @@ -35,6 +35,29 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-30 ``Deprecate 'delimiter' param and source object's wildcards in GCS, introduce 'match_glob' param. (#31261)`` +`dd937e51fe `_ 2023-06-30 ``Add 'on_finish_action' to 'KubernetesPodOperator' (#30718)`` +`c0eaa9b25d `_ 2023-06-29 ``Add deferrable mode to CloudSQLExportInstanceOperator (#30852)`` +`f3f69bf1e0 `_ 2023-06-29 ``Refactor GKECreateClusterOperator's body validation (#31923)`` +`7d2ec76c72 `_ 2023-06-29 ``Make the deferrable version of DataprocCreateBatchOperator handle a batch_id that already exists (#32216)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`b156db3a70 `_ 2023-06-23 ``Adding 'src_fmt_configs' to the list of template fields. (#32097)`` +`59d64d8f2e `_ 2023-06-22 ``[Issue-32069] Fix name format in the batch requests (#32070)`` +`fd116cc196 `_ 2023-06-22 ``Google provider docstring improvements (#31731)`` +`8f41584777 `_ 2023-06-21 ``Optimize deferrable mode execution for 'BigQueryValueCheckOperator' (#31872)`` +`fe7a1aa096 `_ 2023-06-21 ``Switch Google Ads API version from v13 to v14 (#32028)`` +`43fa157e51 `_ 2023-06-20 ``Fix 'BigQueryInsertJobOperator' error handling in deferrable mode (#32034)`` +`2a79fb74fd `_ 2023-06-20 ``Fix 'BIGQUERY_JOB_DETAILS_LINK_FMT' in 'BigQueryConsoleLink' (#31953)`` +================================================================================================= =========== ============================================================================================================= + 10.2.0 ...... @@ -43,6 +66,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)`` `f2ebc292fe `_ 2023-06-19 ``Provide missing project id and creds for TabularDataset (#31991)`` `66299338eb `_ 2023-06-18 ``add a return when the event is yielded in a loop to stop the execution (#31985)`` diff --git a/docs/apache-airflow-providers-google/index.rst b/docs/apache-airflow-providers-google/index.rst index 5e322f9d0054..6721dfc0399e 100644 --- a/docs/apache-airflow-providers-google/index.rst +++ b/docs/apache-airflow-providers-google/index.rst @@ -88,7 +88,7 @@ Google services including: - `Google Workspace `__ (formerly Google Suite) -Release: 10.2.0 +Release: 10.3.0 Provider package ---------------- @@ -206,5 +206,5 @@ 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-google 10.2.0 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-google 10.2.0 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-google 10.3.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-google 10.3.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-hashicorp/commits.rst b/docs/apache-airflow-providers-hashicorp/commits.rst index e6bfc9a669d4..8dff6dbb5bc4 100644 --- a/docs/apache-airflow-providers-hashicorp/commits.rst +++ b/docs/apache-airflow-providers-hashicorp/commits.rst @@ -28,6 +28,18 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`d136457686 `_ 2023-06-20 ``Add default retry to hvac client requests (#31073)`` +================================================================================================= =========== ================================================================ + 3.4.1 ..... @@ -36,6 +48,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)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` `41ea700cbd `_ 2023-05-30 ``Fix hashicorp some client authentication methods (#31593)`` diff --git a/docs/apache-airflow-providers-hashicorp/index.rst b/docs/apache-airflow-providers-hashicorp/index.rst index a1b4007398a7..36e122c58df5 100644 --- a/docs/apache-airflow-providers-hashicorp/index.rst +++ b/docs/apache-airflow-providers-hashicorp/index.rst @@ -69,7 +69,7 @@ Package apache-airflow-providers-hashicorp Hashicorp including `Hashicorp Vault `__ -Release: 3.4.1 +Release: 3.4.2 Provider package ---------------- @@ -121,5 +121,5 @@ 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-hashicorp 3.4.1 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-hashicorp 3.4.1 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-hashicorp 3.4.2 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-hashicorp 3.4.2 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-microsoft-azure/commits.rst b/docs/apache-airflow-providers-microsoft-azure/commits.rst index a812135ab721..04df0a44ec97 100644 --- a/docs/apache-airflow-providers-microsoft-azure/commits.rst +++ b/docs/apache-airflow-providers-microsoft-azure/commits.rst @@ -28,6 +28,25 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-07-04 ``Added 'AzureBlobStorageToS3Operator' transfer operator (#32270)`` +`985a6cd2a8 `_ 2023-07-03 ``invalid args fix (#32326)`` +`c03d7b7c33 `_ 2023-07-03 ``Adds Sensor section in the Azure providers docs (#32299)`` +`2ce51ac5d3 `_ 2023-06-29 ``Cancel pipeline if unexpected exception caught (#32238)`` +`57b7ba16a3 `_ 2023-06-29 ``Doc changes: Added Transfers section in Azure provider docs (#32241)`` +`46ee1c2c8d `_ 2023-06-28 ``Fix where account url is build if not provided using login (account name) (#32082)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`dc9b516494 `_ 2023-06-26 ``Adds connection test for ADLS Gen2 (#32126)`` +`c508b8e531 `_ 2023-06-21 ``Add option to pass extra configs to ClientSecretCredential (#31783)`` +================================================================================================= =========== ====================================================================================== + 6.1.2 ..... @@ -36,6 +55,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)`` `4266a545b0 `_ 2023-06-20 ``Microsoft provider docstring improvements (#31708)`` `94128303e1 `_ 2023-06-08 ``Removed unused variables in AzureBlobStorageToGCSOperator (#31765)`` diff --git a/docs/apache-airflow-providers-microsoft-azure/index.rst b/docs/apache-airflow-providers-microsoft-azure/index.rst index a10c5017314a..c2ee6d69df49 100644 --- a/docs/apache-airflow-providers-microsoft-azure/index.rst +++ b/docs/apache-airflow-providers-microsoft-azure/index.rst @@ -81,7 +81,7 @@ Package apache-airflow-providers-microsoft-azure `Microsoft Azure `__ -Release: 6.1.2 +Release: 6.2.0 Provider package ---------------- @@ -151,5 +151,5 @@ 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-microsoft-azure 6.1.2 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-microsoft-azure 6.1.2 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-microsoft-azure 6.2.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-microsoft-azure 6.2.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-trino/commits.rst b/docs/apache-airflow-providers-trino/commits.rst index 83c13556851c..3ddb3cd8ef9c 100644 --- a/docs/apache-airflow-providers-trino/commits.rst +++ b/docs/apache-airflow-providers-trino/commits.rst @@ -28,6 +28,18 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +`371833e076 `_ 2023-06-24 ``Trino Hook: Add ability to read JWT from file (#31950)`` +================================================================================================= =========== ================================================================ + 5.1.1 ..... @@ -36,6 +48,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)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` ================================================================================================= =========== ============================================================= diff --git a/docs/apache-airflow-providers-trino/index.rst b/docs/apache-airflow-providers-trino/index.rst index ebd3b2e0cd81..ea550a64b0ff 100644 --- a/docs/apache-airflow-providers-trino/index.rst +++ b/docs/apache-airflow-providers-trino/index.rst @@ -78,7 +78,7 @@ Package apache-airflow-providers-trino `Trino `__ -Release: 5.1.1 +Release: 5.2.0 Provider package ---------------- @@ -133,5 +133,5 @@ 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-trino 5.1.1 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-trino 5.1.1 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-trino 5.2.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-trino 5.2.0 wheel package `_ (`asc `__, `sha512 `__) diff --git a/docs/apache-airflow-providers-vertica/commits.rst b/docs/apache-airflow-providers-vertica/commits.rst index 35f09c68892c..489949baab86 100644 --- a/docs/apache-airflow-providers-vertica/commits.rst +++ b/docs/apache-airflow-providers-vertica/commits.rst @@ -28,6 +28,18 @@ For high-level changelog, see :doc:`package information including changelog `_ 2023-06-30 ``Add various Vertica connection parameters (#32089)`` +`09d4718d3a `_ 2023-06-27 ``Improve provider documentation and README structure (#32125)`` +================================================================================================= =========== ================================================================ + 3.4.1 ..... @@ -36,6 +48,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)`` `a59076eaee `_ 2023-06-02 ``Add D400 pydocstyle check - Providers (#31427)`` ================================================================================================= =========== ============================================================= diff --git a/docs/apache-airflow-providers-vertica/index.rst b/docs/apache-airflow-providers-vertica/index.rst index ae7c2457b811..78aa37666e11 100644 --- a/docs/apache-airflow-providers-vertica/index.rst +++ b/docs/apache-airflow-providers-vertica/index.rst @@ -68,7 +68,7 @@ Package apache-airflow-providers-vertica `Vertica `__ -Release: 3.4.1 +Release: 3.5.0 Provider package ---------------- @@ -121,5 +121,5 @@ 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-vertica 3.4.1 sdist package `_ (`asc `__, `sha512 `__) -* `The apache-airflow-providers-vertica 3.4.1 wheel package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-vertica 3.5.0 sdist package `_ (`asc `__, `sha512 `__) +* `The apache-airflow-providers-vertica 3.5.0 wheel package `_ (`asc `__, `sha512 `__) From 43f3e57bf162293b92154f16a8ce33e6922fbf4e Mon Sep 17 00:00:00 2001 From: Pankaj Koti Date: Wed, 5 Jul 2023 13:10:32 +0530 Subject: [PATCH 053/533] Update index.rst doc to correct grammar (#32315) * Update index.rst doc to correct grammar * Address @uranusjr's comments --- docs/apache-airflow/index.rst | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/docs/apache-airflow/index.rst b/docs/apache-airflow/index.rst index feaf2d388b50..adc5f47eff59 100644 --- a/docs/apache-airflow/index.rst +++ b/docs/apache-airflow/index.rst @@ -98,14 +98,22 @@ Rich scheduling and execution semantics enable you to easily define complex pipe intervals. Backfilling allows you to (re-)run pipelines on historical data after making changes to your logic. And the ability to rerun partial pipelines after resolving an error helps maximize efficiency. -Airflow's user interface provides both in-depth views of pipelines and individual tasks, and an overview of -pipelines over time. From the interface, you can inspect logs and manage tasks, for example retrying a task in +Airflow's user interface provides: + + 1. In-depth views of two things: + + i. Pipelines + ii. Tasks + + 2. Overview of your pipelines over time + +From the interface, you can inspect logs and manage tasks, for example retrying a task in case of failure. The open-source nature of Airflow ensures you work on components developed, tested, and used by many other `companies `_ around the world. In the active `community `_ you can find plenty of helpful resources in the form of -blogs posts, articles, conferences, books, and more. You can connect with other peers via several channels +blog posts, articles, conferences, books, and more. You can connect with other peers via several channels such as `Slack `_ and mailing lists. Airflow as a Platform is highly customizable. By utilizing :doc:`public-airflow-interface` you can extend @@ -115,7 +123,7 @@ Why not Airflow™? ================= Airflow™ was built for finite batch workflows. While the CLI and REST API do allow triggering workflows, -Airflow was not built for infinitely-running event-based workflows. Airflow is not a streaming solution. +Airflow was not built for infinitely running event-based workflows. Airflow is not a streaming solution. However, a streaming system such as Apache Kafka is often seen working together with Apache Airflow. Kafka can be used for ingestion and processing in real-time, event data is written to a storage location, and Airflow periodically starts a workflow processing a batch of data. From a8e4b8aee602e8c672ab879b7392a65b5c2bb34e Mon Sep 17 00:00:00 2001 From: Akash Sharma <35839624+Adaverse@users.noreply.github.com> Date: Wed, 5 Jul 2023 14:00:51 +0530 Subject: [PATCH 054/533] Fix try_number shown in the `task?task_id=&dag_id=&execution_date=` (#32361) * try_number fix * change test name --- airflow/www/views.py | 3 ++- tests/www/views/test_views_tasks.py | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index 3f0965da3889..d7059156f575 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1801,7 +1801,8 @@ def task(self, session: Session = NEW_SESSION): with warnings.catch_warnings(): warnings.simplefilter("ignore", RemovedInAirflow3Warning) all_ti_attrs = ( - (name, getattr(ti, name)) + # fetching the value of _try_number to be shown under name try_number in UI + (name, getattr(ti, "_try_number" if name == "try_number" else name)) for name in dir(ti) if not name.startswith("_") and name not in ti_attrs_to_skip ) diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 8ed0ba825a3e..21c573fe5049 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -323,6 +323,15 @@ def test_views_get(admin_client, url, contents): check_content_in_response(content, resp) +def test_rendered_task_view(admin_client): + url = f"task?task_id=runme_0&dag_id=example_bash_operator&execution_date={DEFAULT_VAL}" + resp = admin_client.get(url, follow_redirects=True) + resp_html = resp.data.decode("utf-8") + assert resp.status_code == 200 + assert "_try_number" not in resp_html + assert "try_number" in resp_html + + def test_rendered_k8s(admin_client): url = f"rendered-k8s?task_id=runme_0&dag_id=example_bash_operator&execution_date={DEFAULT_VAL}" with unittest.mock.patch.object(settings, "IS_K8S_OR_K8SCELERY_EXECUTOR", True): From 7722b6f226e9db3a89b01b89db5fdb7a1ab2256f Mon Sep 17 00:00:00 2001 From: Karthikeyan Singaravelan Date: Wed, 5 Jul 2023 14:08:29 +0530 Subject: [PATCH 055/533] Ensure dynamic tasks inside dynamic task group only marks the (#32354) corresponding EmptyOperator in downstream as success. --- airflow/models/dagrun.py | 4 ++-- tests/models/test_dagrun.py | 46 +++++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 2 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 8f3b3a33018a..b0b56f3c6c5a 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1338,7 +1338,7 @@ def schedule_tis( and not ti.task.on_success_callback and not ti.task.outlets ): - dummy_ti_ids.append(ti.task_id) + dummy_ti_ids.append((ti.task_id, ti.map_index)) else: schedulable_ti_ids.append((ti.task_id, ti.map_index)) @@ -1369,7 +1369,7 @@ def schedule_tis( .where( TI.dag_id == self.dag_id, TI.run_id == self.run_id, - TI.task_id.in_(dummy_ti_ids_chunk), + tuple_in_condition((TI.task_id, TI.map_index), dummy_ti_ids_chunk), ) .values( state=TaskInstanceState.SUCCESS, diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 5d7db547d31c..c90ec1aef328 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -1811,6 +1811,52 @@ def tg(x): ] +def test_mapped_task_group_empty_operator(dag_maker, session): + """ + Test that dynamic task inside a dynamic task group only marks + the corresponding downstream EmptyOperator as success. + """ + + literal = [1, 2, 3] + + with dag_maker(session=session) as dag: + + @task_group + def tg(x): + @task + def t1(x): + return x + + t2 = EmptyOperator(task_id="t2") + + @task + def t3(x): + return x + + t1(x) >> t2 >> t3(x) + + tg.expand(x=literal) + + dr = dag_maker.create_dagrun() + + t2_task = dag.get_task("tg.t2") + t2_0 = dr.get_task_instance(task_id="tg.t2", map_index=0) + t2_0.refresh_from_task(t2_task) + assert t2_0.state is None + + t2_1 = dr.get_task_instance(task_id="tg.t2", map_index=1) + t2_1.refresh_from_task(t2_task) + assert t2_1.state is None + + dr.schedule_tis([t2_0]) + + t2_0 = dr.get_task_instance(task_id="tg.t2", map_index=0) + assert t2_0.state == TaskInstanceState.SUCCESS + + t2_1 = dr.get_task_instance(task_id="tg.t2", map_index=1) + assert t2_1.state is None + + def test_ti_scheduling_mapped_zero_length(dag_maker, session): with dag_maker(session=session): task = BaseOperator(task_id="task_1") From 61f33304d587b3b0a48a876d3bfedab82e42bacc Mon Sep 17 00:00:00 2001 From: Phani Kumar <94376113+phanikumv@users.noreply.github.com> Date: Wed, 5 Jul 2023 14:26:36 +0530 Subject: [PATCH 056/533] Refactor Sqlalchemy queries to 2.0 style (Part 3) (#32350) --------- Co-authored-by: Ephraim Anierobi --- airflow/utils/db.py | 38 ++-- airflow/www/utils.py | 11 +- airflow/www/views.py | 461 ++++++++++++++++++++++--------------------- 3 files changed, 266 insertions(+), 244 deletions(-) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index a76f0d4f675d..46ddcbfb3453 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -92,7 +92,7 @@ def _format_airflow_moved_table_name(source_table, version, category): @provide_session def merge_conn(conn, session: Session = NEW_SESSION): """Add new Connection.""" - if not session.query(conn.__class__).filter_by(conn_id=conn.conn_id).first(): + if not session.scalar(select(conn.__class__).filter_by(conn_id=conn.conn_id).limit(1)): session.add(conn) session.commit() @@ -959,7 +959,9 @@ def check_conn_id_duplicates(session: Session) -> Iterable[str]: dups = [] try: - dups = session.query(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1).all() + dups = session.execute( + select(Connection.conn_id).group_by(Connection.conn_id).having(func.count() > 1) + ).all() except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -984,12 +986,11 @@ def check_username_duplicates(session: Session) -> Iterable[str]: for model in [User, RegisterUser]: dups = [] try: - dups = ( - session.query(model.username) # type: ignore[attr-defined] + dups = session.execute( + select(model.username) # type: ignore[attr-defined] .group_by(model.username) # type: ignore[attr-defined] .having(func.count() > 1) - .all() - ) + ).all() except (exc.OperationalError, exc.ProgrammingError): # fallback if tables hasn't been created yet session.rollback() @@ -1058,13 +1059,13 @@ def check_task_fail_for_duplicates(session): """ minimal_table_obj = table(table_name, *[column(x) for x in uniqueness]) try: - subquery = ( - session.query(minimal_table_obj, func.count().label("dupe_count")) + subquery = session.execute( + select(minimal_table_obj, func.count().label("dupe_count")) .group_by(*[text(x) for x in uniqueness]) .having(func.count() > text("1")) .subquery() ) - dupe_count = session.query(func.sum(subquery.c.dupe_count)).scalar() + dupe_count = session.scalar(select(func.sum(subquery.c.dupe_count))) if not dupe_count: # there are no duplicates; nothing to do. return @@ -1101,7 +1102,7 @@ def check_conn_type_null(session: Session) -> Iterable[str]: n_nulls = [] try: - n_nulls = session.query(Connection.conn_id).filter(Connection.conn_type.is_(None)).all() + n_nulls = session.scalars(select(Connection.conn_id).where(Connection.conn_type.is_(None))).all() except (exc.OperationalError, exc.ProgrammingError, exc.InternalError): # fallback if tables hasn't been created yet session.rollback() @@ -1143,7 +1144,7 @@ def check_run_id_null(session: Session) -> Iterable[str]: dagrun_table.c.run_id.is_(None), dagrun_table.c.execution_date.is_(None), ) - invalid_dagrun_count = session.query(func.count(dagrun_table.c.id)).filter(invalid_dagrun_filter).scalar() + invalid_dagrun_count = session.scalar(select(func.count(dagrun_table.c.id)).where(invalid_dagrun_filter)) if invalid_dagrun_count > 0: dagrun_dangling_table_name = _format_airflow_moved_table_name(dagrun_table.name, "2.2", "dangling") if dagrun_dangling_table_name in inspect(session.get_bind()).get_table_names(): @@ -1240,7 +1241,7 @@ def _move_dangling_data_to_new_table( pk_cols = source_table.primary_key.columns delete = source_table.delete().where( - tuple_(*pk_cols).in_(session.query(*target_table.primary_key.columns).subquery()) + tuple_(*pk_cols).in_(session.select(*target_table.primary_key.columns).subquery()) ) else: delete = source_table.delete().where( @@ -1262,10 +1263,11 @@ def _dangling_against_dag_run(session, source_table, dag_run): source_table.c.dag_id == dag_run.c.dag_id, source_table.c.execution_date == dag_run.c.execution_date, ) + return ( - session.query(*[c.label(c.name) for c in source_table.c]) + select(*[c.label(c.name) for c in source_table.c]) .join(dag_run, source_to_dag_run_join_cond, isouter=True) - .filter(dag_run.c.dag_id.is_(None)) + .where(dag_run.c.dag_id.is_(None)) ) @@ -1304,10 +1306,10 @@ def _dangling_against_task_instance(session, source_table, dag_run, task_instanc ) return ( - session.query(*[c.label(c.name) for c in source_table.c]) + select(*[c.label(c.name) for c in source_table.c]) .join(dag_run, dr_join_cond, isouter=True) .join(task_instance, ti_join_cond, isouter=True) - .filter(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) + .where(or_(task_instance.c.dag_id.is_(None), dag_run.c.dag_id.is_(None))) ) @@ -1331,9 +1333,9 @@ def _move_duplicate_data_to_new_table( """ bind = session.get_bind() dialect_name = bind.dialect.name + query = ( - session.query(source_table) - .with_entities(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) + select(*[getattr(source_table.c, x.name).label(str(x.name)) for x in source_table.columns]) .select_from(source_table) .join(subquery, and_(*[getattr(source_table.c, x) == getattr(subquery.c, x) for x in uniqueness])) ) diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 76914dd9cdcd..b31f9326d988 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -39,6 +39,7 @@ from pygments.lexer import Lexer from sqlalchemy import delete, func, types from sqlalchemy.ext.associationproxy import AssociationProxy +from sqlalchemy.sql import Select from airflow.exceptions import RemovedInAirflow3Warning from airflow.models import errors @@ -53,7 +54,6 @@ from airflow.www.widgets import AirflowDateTimePickerWidget if TYPE_CHECKING: - from sqlalchemy.orm.query import Query from sqlalchemy.orm.session import Session from sqlalchemy.sql.operators import ColumnOperators @@ -518,18 +518,21 @@ def _get_run_ordering_expr(name: str) -> ColumnOperators: return expr.desc() -def sorted_dag_runs(query: Query, *, ordering: Sequence[str], limit: int) -> Sequence[DagRun]: +def sorted_dag_runs( + query: Select, *, ordering: Sequence[str], limit: int, session: Session +) -> Sequence[DagRun]: """Produce DAG runs sorted by specified columns. - :param query: An ORM query object against *DagRun*. + :param query: An ORM select object against *DagRun*. :param ordering: Column names to sort the runs. should generally come from a timetable's ``run_ordering``. :param limit: Number of runs to limit to. + :param session: SQLAlchemy ORM session object :return: A list of DagRun objects ordered by the specified columns. The list contains only the *last* objects, but in *ascending* order. """ ordering_exprs = (_get_run_ordering_expr(name) for name in ordering) - runs = query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit).all() + runs = session.scalars(query.order_by(*ordering_exprs, DagRun.id.desc()).limit(limit)).all() runs.reverse() return runs diff --git a/airflow/www/views.py b/airflow/www/views.py index d7059156f575..b304dd0ddf02 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -69,7 +69,7 @@ from pendulum.parsing.exceptions import ParserError from pygments import highlight, lexers from pygments.formatters import HtmlFormatter -from sqlalchemy import Date, and_, case, desc, func, inspect, or_, union_all +from sqlalchemy import Date, and_, case, desc, func, inspect, or_, select, union_all from sqlalchemy.exc import IntegrityError from sqlalchemy.orm import Session, joinedload from wtforms import SelectField, validators @@ -95,7 +95,7 @@ from airflow.jobs.job import Job from airflow.jobs.scheduler_job_runner import SchedulerJobRunner from airflow.jobs.triggerer_job_runner import TriggererJobRunner -from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, XCom, errors +from airflow.models import Connection, DagModel, DagTag, Log, SlaMiss, TaskFail, Trigger, XCom, errors from airflow.models.abstractoperator import AbstractOperator from airflow.models.dag import DAG, get_dataset_triggered_next_run_info from airflow.models.dagcode import DagCode @@ -231,16 +231,15 @@ def get_date_time_num_runs_dag_runs_form_data(www_request, session, dag): # loaded and the actual requested run would be excluded by the limit(). Once # the user has changed base date to be anything else we want to use that instead. query_date = base_date - if date_time < base_date and date_time + datetime.timedelta(seconds=1) >= base_date: + if date_time < base_date <= date_time + datetime.timedelta(seconds=1): query_date = date_time - drs = ( - session.query(DagRun) - .filter(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) + drs = session.scalars( + select(DagRun) + .where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= query_date) .order_by(desc(DagRun.execution_date)) .limit(num_runs) - .all() - ) + ).all() dr_choices = [] dr_state = None for dr in drs: @@ -291,8 +290,8 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): Create a nested dict representation of the DAG's TaskGroup and its children used to construct the Graph and Grid views. """ - query = ( - session.query( + query = session.execute( + select( TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, @@ -303,7 +302,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): func.max(TaskInstance.end_date).label("end_date"), ) .join(TaskInstance.task_instance_note, isouter=True) - .filter( + .where( TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), ) @@ -426,11 +425,9 @@ def get_summary(dag_run: DagRun): } def get_mapped_group_summaries(): - mapped_ti_query = ( - session.query( - TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index - ) - .filter( + mapped_ti_query = session.execute( + select(TaskInstance.task_id, TaskInstance.state, TaskInstance.run_id, TaskInstance.map_index) + .where( TaskInstance.dag_id == dag.dag_id, TaskInstance.task_id.in_(child["id"] for child in children), TaskInstance.run_id.in_(r.run_id for r in dag_runs), @@ -738,21 +735,20 @@ def index(self): with create_session() as session: # read orm_dags from the db - dags_query = session.query(DagModel).filter(~DagModel.is_subdag, DagModel.is_active) + dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) if arg_search_query: escaped_arg_search_query = arg_search_query.replace("_", r"\_") - dags_query = dags_query.filter( + dags_query = dags_query.where( DagModel.dag_id.ilike("%" + escaped_arg_search_query + "%", escape="\\") | DagModel.owners.ilike("%" + escaped_arg_search_query + "%", escape="\\") ) if arg_tags_filter: - dags_query = dags_query.filter(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) + dags_query = dags_query.where(DagModel.tags.any(DagTag.name.in_(arg_tags_filter))) - dags_query = dags_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - - filtered_dag_count = dags_query.count() + dags_query = dags_query.where(DagModel.dag_id.in_(filter_dag_ids)) + filtered_dag_count = session.scalar(select(func.count()).select_from(dags_query)) if filtered_dag_count == 0 and len(arg_tags_filter): flash( "No matching DAG tags found.", @@ -762,28 +758,28 @@ def index(self): return redirect(url_for("Airflow.index")) all_dags = dags_query - active_dags = dags_query.filter(~DagModel.is_paused) - paused_dags = dags_query.filter(DagModel.is_paused) + active_dags = dags_query.where(~DagModel.is_paused) + paused_dags = dags_query.where(DagModel.is_paused) # find DAGs which have a RUNNING DagRun - running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).filter( + running_dags = dags_query.join(DagRun, DagModel.dag_id == DagRun.dag_id).where( DagRun.state == State.RUNNING ) # find DAGs for which the latest DagRun is FAILED subq_all = ( - session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) .group_by(DagRun.dag_id) .subquery() ) subq_failed = ( - session.query(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) - .filter(DagRun.state == State.FAILED) + select(DagRun.dag_id, func.max(DagRun.start_date).label("start_date")) + .where(DagRun.state == State.FAILED) .group_by(DagRun.dag_id) .subquery() ) subq_join = ( - session.query(subq_all.c.dag_id, subq_all.c.start_date) + select(subq_all.c.dag_id, subq_all.c.start_date) .join( subq_failed, and_( @@ -796,16 +792,18 @@ def index(self): failed_dags = dags_query.join(subq_join, DagModel.dag_id == subq_join.c.dag_id) is_paused_count = dict( - all_dags.with_entities(DagModel.is_paused, func.count(DagModel.dag_id)).group_by( - DagModel.is_paused - ) + session.execute( + select(DagModel.is_paused, func.count(DagModel.dag_id)) + .group_by(DagModel.is_paused) + .select_from(all_dags) + ).all() ) status_count_active = is_paused_count.get(False, 0) status_count_paused = is_paused_count.get(True, 0) - status_count_running = running_dags.count() - status_count_failed = failed_dags.count() + status_count_running = session.scalar(select(func.count()).select_from(running_dags)) + status_count_failed = session.scalar(select(func.count()).select_from(failed_dags)) all_dags_count = status_count_active + status_count_paused if arg_status_filter == "active": @@ -826,7 +824,7 @@ def index(self): if arg_sorting_key == "last_dagrun": dag_run_subquery = ( - session.query( + select( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) @@ -854,7 +852,13 @@ def index(self): else: current_dags = current_dags.order_by(null_case, sort_column) - dags = current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page).all() + dags = ( + session.scalars( + current_dags.options(joinedload(DagModel.tags)).offset(start).limit(dags_per_page) + ) + .unique() + .all() + ) user_permissions = g.user.perms can_create_dag_run = ( permissions.ACTION_CAN_CREATE, @@ -874,7 +878,7 @@ def index(self): dag.can_trigger = dag.can_edit and can_create_dag_run dag.can_delete = get_airflow_app().appbuilder.sm.can_delete_dag(dag.dag_id, g.user) - dagtags = session.query(func.distinct(DagTag.name)).order_by(DagTag.name).all() + dagtags = session.execute(select(func.distinct(DagTag.name)).order_by(DagTag.name)).all() tags = [ {"name": name, "selected": bool(arg_tags_filter and name in arg_tags_filter)} for name, in dagtags @@ -882,14 +886,15 @@ def index(self): owner_links_dict = DagOwnerAttributes.get_all(session) - import_errors = session.query(errors.ImportError).order_by(errors.ImportError.id) + import_errors = select(errors.ImportError).order_by(errors.ImportError.id) if (permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG) not in user_permissions: # if the user doesn't have access to all DAGs, only display errors from visible DAGs import_errors = import_errors.join( DagModel, DagModel.fileloc == errors.ImportError.filename - ).filter(DagModel.dag_id.in_(filter_dag_ids)) + ).where(DagModel.dag_id.in_(filter_dag_ids)) + import_errors = session.scalars(import_errors) for import_error in import_errors: flash( f"Broken DAG: [{import_error.filename}] {import_error.stacktrace}", @@ -933,10 +938,12 @@ def _iter_parsed_moved_data_table_names(): permissions.RESOURCE_ADMIN_MENU, ) in user_permissions and conf.getboolean("webserver", "warn_deployment_exposure"): robots_file_access_count = ( - session.query(Log) - .filter(Log.event == "robots") - .filter(Log.dttm > (utcnow() - datetime.timedelta(days=7))) - .count() + select(Log) + .where(Log.event == "robots") + .where(Log.dttm > (utcnow() - datetime.timedelta(days=7))) + ) + robots_file_access_count = session.scalar( + select(func.count()).select_from(robots_file_access_count) ) if robots_file_access_count > 0: flash( @@ -1038,9 +1045,11 @@ def next_run_datasets_summary(self, session: Session = NEW_SESSION): dataset_triggered_dag_ids = [ dag.dag_id for dag in ( - session.query(DagModel.dag_id) - .filter(DagModel.dag_id.in_(filter_dag_ids)) - .filter(DagModel.schedule_interval == "Dataset") + session.scalars( + select(DagModel.dag_id) + .where(DagModel.dag_id.in_(filter_dag_ids)) + .where(DagModel.schedule_interval == "Dataset") + ) ) ] @@ -1071,10 +1080,10 @@ def dag_stats(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify({}) - dag_state_stats = ( - session.query(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) + dag_state_stats = session.execute( + select(DagRun.dag_id, DagRun.state, sqla.func.count(DagRun.state)) .group_by(DagRun.dag_id, DagRun.state) - .filter(DagRun.dag_id.in_(filter_dag_ids)) + .where(DagRun.dag_id.in_(filter_dag_ids)) ) dag_state_data = {(dag_id, state): count for dag_id, state, count in dag_state_stats} @@ -1112,17 +1121,17 @@ def task_stats(self, session: Session = NEW_SESSION): filter_dag_ids = allowed_dag_ids running_dag_run_query_result = ( - session.query(DagRun.dag_id, DagRun.run_id) + select(DagRun.dag_id, DagRun.run_id) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .filter(DagRun.state == State.RUNNING, DagModel.is_active) + .where(DagRun.state == State.RUNNING, DagModel.is_active) ) - running_dag_run_query_result = running_dag_run_query_result.filter(DagRun.dag_id.in_(filter_dag_ids)) + running_dag_run_query_result = running_dag_run_query_result.where(DagRun.dag_id.in_(filter_dag_ids)) running_dag_run_query_result = running_dag_run_query_result.subquery("running_dag_run") # Select all task_instances from active dag_runs. - running_task_instance_query_result = session.query( + running_task_instance_query_result = select( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(True).label("is_dag_running"), @@ -1136,19 +1145,19 @@ def task_stats(self, session: Session = NEW_SESSION): if conf.getboolean("webserver", "SHOW_RECENT_STATS_FOR_COMPLETED_RUNS", fallback=True): last_dag_run = ( - session.query(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) + select(DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("execution_date")) .join(DagModel, DagModel.dag_id == DagRun.dag_id) - .filter(DagRun.state != State.RUNNING, DagModel.is_active) + .where(DagRun.state != State.RUNNING, DagModel.is_active) .group_by(DagRun.dag_id) ) - last_dag_run = last_dag_run.filter(DagRun.dag_id.in_(filter_dag_ids)) + last_dag_run = last_dag_run.where(DagRun.dag_id.in_(filter_dag_ids)) last_dag_run = last_dag_run.subquery("last_dag_run") # Select all task_instances from active dag_runs. # If no dag_run is active, return task instances from most recent dag_run. last_task_instance_query_result = ( - session.query( + select( TaskInstance.dag_id.label("dag_id"), TaskInstance.state.label("state"), sqla.literal(False).label("is_dag_running"), @@ -1169,8 +1178,8 @@ def task_stats(self, session: Session = NEW_SESSION): else: final_task_instance_query_result = running_task_instance_query_result.subquery("final_ti") - qry = ( - session.query( + qry = session.execute( + select( final_task_instance_query_result.c.dag_id, final_task_instance_query_result.c.state, final_task_instance_query_result.c.is_dag_running, @@ -1186,7 +1195,6 @@ def task_stats(self, session: Session = NEW_SESSION): final_task_instance_query_result.c.is_dag_running.desc(), ) ) - data = get_task_stats_from_query(qry) payload: dict[str, list[dict[str, Any]]] = collections.defaultdict(list) for dag_id in filter_dag_ids: @@ -1219,29 +1227,31 @@ def last_dagruns(self, session: Session = NEW_SESSION): return flask.json.jsonify({}) last_runs_subquery = ( - session.query( + select( DagRun.dag_id, sqla.func.max(DagRun.execution_date).label("max_execution_date"), ) .group_by(DagRun.dag_id) - .filter(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. + .where(DagRun.dag_id.in_(filter_dag_ids)) # Only include accessible/selected DAGs. .subquery("last_runs") ) - query = session.query( - DagRun.dag_id, - DagRun.start_date, - DagRun.end_date, - DagRun.state, - DagRun.execution_date, - DagRun.data_interval_start, - DagRun.data_interval_end, - ).join( - last_runs_subquery, - and_( - last_runs_subquery.c.dag_id == DagRun.dag_id, - last_runs_subquery.c.max_execution_date == DagRun.execution_date, - ), + query = session.execute( + select( + DagRun.dag_id, + DagRun.start_date, + DagRun.end_date, + DagRun.state, + DagRun.execution_date, + DagRun.data_interval_start, + DagRun.data_interval_end, + ).join( + last_runs_subquery, + and_( + last_runs_subquery.c.dag_id == DagRun.dag_id, + last_runs_subquery.c.max_execution_date == DagRun.execution_date, + ), + ) ) resp = { @@ -1340,19 +1350,18 @@ def dag_details(self, dag_id, session: Session = NEW_SESSION): title = "DAG Details" root = request.args.get("root", "") - states = ( - session.query(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) - .filter(TaskInstance.dag_id == dag_id) + states = session.execute( + select(TaskInstance.state, sqla.func.count(TaskInstance.dag_id)) + .where(TaskInstance.dag_id == dag_id) .group_by(TaskInstance.state) - .all() - ) + ).all() active_runs = models.DagRun.find(dag_id=dag_id, state=DagRunState.RUNNING, external_trigger=False) - tags = session.query(models.DagTag).filter(models.DagTag.dag_id == dag_id).all() + tags = session.scalars(select(models.DagTag).where(models.DagTag.dag_id == dag_id)).all() # TODO: convert this to a relationship - owner_links = session.query(DagOwnerAttributes).filter_by(dag_id=dag_id).all() + owner_links = session.execute(select(DagOwnerAttributes).filter_by(dag_id=dag_id)).all() attrs_to_avoid = [ "schedule_datasets", @@ -1613,18 +1622,17 @@ def get_logs_with_metadata(self, session: Session = NEW_SESSION): "metadata": {"end_of_log": True}, } - ti = ( - session.query(models.TaskInstance) - .filter( + ti = session.scalar( + select(models.TaskInstance) + .where( TaskInstance.task_id == task_id, TaskInstance.dag_id == dag_id, TaskInstance.execution_date == execution_date, TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) - .options(joinedload("trigger")) - .options(joinedload("trigger.triggerer_job")) - .first() + .options(joinedload(TaskInstance.trigger).joinedload(Trigger.triggerer_job)) + .limit(1) ) if ti is None: @@ -1682,10 +1690,10 @@ def log(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) dag_model = DagModel.get_dagmodel(dag_id) - ti = ( - session.query(models.TaskInstance) + ti = session.scalar( + select(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .first() + .limit(1) ) num_logs = 0 @@ -1726,10 +1734,10 @@ def redirect_to_external_log(self, session: Session = NEW_SESSION): map_index = request.args.get("map_index", -1, type=int) try_number = request.args.get("try_number", 1) - ti = ( - session.query(models.TaskInstance) + ti = session.scalar( + select(models.TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) - .first() + .limit(1) ) if not ti: @@ -1771,8 +1779,8 @@ def task(self, session: Session = NEW_SESSION): task = copy.copy(dag.get_task(task_id)) task.resolve_template_files() - ti: TaskInstance | None = ( - session.query(TaskInstance) + ti: TaskInstance | None = session.scalar( + select(TaskInstance) .options( # HACK: Eager-load relationships. This is needed because # multiple properties mis-use provide_session() that destroys @@ -1781,7 +1789,6 @@ def task(self, session: Session = NEW_SESSION): joinedload(TaskInstance.trigger, innerjoin=False), ) .filter_by(execution_date=dttm, dag_id=dag_id, task_id=task_id, map_index=map_index) - .one_or_none() ) if ti is None: ti_attrs: list[tuple[str, Any]] | None = None @@ -1899,17 +1906,19 @@ def xcom(self, session: Session = NEW_SESSION): form = DateTimeForm(data={"execution_date": dttm}) root = request.args.get("root", "") dag = DagModel.get_dagmodel(dag_id) - ti = session.query(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).first() + ti = session.scalar(select(TaskInstance).filter_by(dag_id=dag_id, task_id=task_id).limit(1)) if not ti: flash(f"Task [{dag_id}.{task_id}] doesn't seem to exist at the moment", "error") return redirect(url_for("Airflow.index")) - xcom_query = session.query(XCom.key, XCom.value).filter( - XCom.dag_id == dag_id, - XCom.task_id == task_id, - XCom.execution_date == dttm, - XCom.map_index == map_index, + xcom_query = session.execute( + select(XCom.key, XCom.value).where( + XCom.dag_id == dag_id, + XCom.task_id == task_id, + XCom.execution_date == dttm, + XCom.map_index == map_index, + ) ) attributes = [(k, v) for k, v in xcom_query if not k.startswith("_")] @@ -1979,7 +1988,7 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): request_execution_date = request.values.get("execution_date", default=timezone.utcnow().isoformat()) is_dag_run_conf_overrides_params = conf.getboolean("core", "dag_run_conf_overrides_params") dag = get_airflow_app().dag_bag.get_dag(dag_id) - dag_orm: DagModel = session.query(DagModel).filter(DagModel.dag_id == dag_id).first() + dag_orm: DagModel = session.scalar(select(DagModel).where(DagModel.dag_id == dag_id).limit(1)) # Prepare form fields with param struct details to render a proper form with schema information form_fields = {} @@ -2017,11 +2026,9 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): flash(f"Cannot create dagruns because the dag {dag_id} has import errors", "error") return redirect(origin) - recent_runs = ( - session.query( - DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date) - ) - .filter( + recent_runs = session.execute( + select(DagRun.conf, func.max(DagRun.run_id).label("run_id"), func.max(DagRun.execution_date)) + .where( DagRun.dag_id == dag_id, DagRun.run_type == DagRunType.MANUAL, DagRun.conf.isnot(None), @@ -2294,15 +2301,17 @@ def clear(self, *, session: Session = NEW_SESSION): # Lock the related dag runs to prevent from possible dead lock. # https://github.com/apache/airflow/pull/26658 - dag_runs_query = session.query(DagRun.id).filter(DagRun.dag_id == dag_id).with_for_update() + dag_runs_query = session.scalars( + select(DagRun.id).where(DagRun.dag_id == dag_id).with_for_update() + ) if start_date is None and end_date is None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date == start_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date == start_date) else: if start_date is not None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date >= start_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date >= start_date) if end_date is not None: - dag_runs_query = dag_runs_query.filter(DagRun.execution_date <= end_date) + dag_runs_query = dag_runs_query.where(DagRun.execution_date <= end_date) locked_dag_run_ids = dag_runs_query.all() elif task_id: @@ -2391,10 +2400,10 @@ def blocked(self, session: Session = NEW_SESSION): if not filter_dag_ids: return flask.json.jsonify([]) - dags = ( - session.query(DagRun.dag_id, sqla.func.count(DagRun.id)) - .filter(DagRun.state == DagRunState.RUNNING) - .filter(DagRun.dag_id.in_(filter_dag_ids)) + dags = session.execute( + select(DagRun.dag_id, sqla.func.count(DagRun.id)) + .where(DagRun.state == DagRunState.RUNNING) + .where(DagRun.dag_id.in_(filter_dag_ids)) .group_by(DagRun.dag_id) ) @@ -2475,9 +2484,11 @@ def _mark_dagrun_state_as_queued( # Identify tasks that will be queued up to run when confirmed all_task_ids = [task.task_id for task in dag.tasks] - existing_tis = session.query(TaskInstance.task_id).filter( - TaskInstance.dag_id == dag.dag_id, - TaskInstance.run_id == dag_run_id, + existing_tis = session.execute( + select(TaskInstance.task_id).where( + TaskInstance.dag_id == dag.dag_id, + TaskInstance.run_id == dag_run_id, + ) ) completed_tis_ids = [task_id for task_id, in existing_tis] @@ -2959,19 +2970,18 @@ def _convert_to_date(session, column): if root: dag = dag.partial_subset(task_ids_or_regex=root, include_downstream=False, include_upstream=True) - dag_states = ( - session.query( - (_convert_to_date(session, DagRun.execution_date)).label("date"), + dag_states = session.execute( + select( + _convert_to_date(session, DagRun.execution_date).label("date"), DagRun.state, func.max(DagRun.data_interval_start).label("data_interval_start"), func.max(DagRun.data_interval_end).label("data_interval_end"), func.count("*").label("count"), ) - .filter(DagRun.dag_id == dag.dag_id) + .where(DagRun.dag_id == dag.dag_id) .group_by(_convert_to_date(session, DagRun.execution_date), DagRun.state) .order_by(_convert_to_date(session, DagRun.execution_date).asc()) - .all() - ) + ).all() data_dag_states = [ { @@ -3242,16 +3252,17 @@ def duration(self, dag_id: str, session: Session = NEW_SESSION): else: min_date = timezone.utc_epoch() ti_fails = ( - session.query(TaskFail) + select(TaskFail) .join(TaskFail.dag_run) - .filter( + .where( TaskFail.dag_id == dag.dag_id, DagRun.execution_date >= min_date, DagRun.execution_date <= base_date, ) ) if dag.partial: - ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = session.scalars(ti_fails) fails_totals: dict[tuple[str, str, str], int] = defaultdict(int) for failed_task_instance in ti_fails: dict_key = ( @@ -3592,21 +3603,22 @@ def gantt(self, dag_id: str, session: Session = NEW_SESSION): form = DateTimeWithNumRunsWithDagRunsForm(data=dt_nr_dr_data) form.execution_date.choices = dt_nr_dr_data["dr_choices"] - tis = ( - session.query(TaskInstance) - .filter( + tis = session.scalars( + select(TaskInstance) + .where( TaskInstance.dag_id == dag_id, TaskInstance.run_id == dag_run_id, TaskInstance.start_date.is_not(None), TaskInstance.state.is_not(None), ) .order_by(TaskInstance.start_date) - ) + ).all() - ti_fails = session.query(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) + ti_fails = select(TaskFail).filter_by(run_id=dag_run_id, dag_id=dag_id) if dag.partial: - ti_fails = ti_fails.filter(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = ti_fails.where(TaskFail.task_id.in_([t.task_id for t in dag.tasks])) + ti_fails = session.scalars(ti_fails) tasks = [] for ti in tis: if not dag.has_task(ti.task_id): @@ -3712,12 +3724,13 @@ def extra_links(self, *, session: Session = NEW_SESSION): if link_name is None: return {"url": None, "error": "Link name not passed"}, 400 - ti = ( - session.query(TaskInstance) + ti = session.scalar( + select(TaskInstance) .filter_by(dag_id=dag_id, task_id=task_id, execution_date=dttm, map_index=map_index) .options(joinedload(TaskInstance.dag_run)) - .first() + .limit(1) ) + if not ti: return {"url": None, "error": "Task Instances not found"}, 404 try: @@ -3825,27 +3838,25 @@ def grid_data(self): base_date = dag.get_latest_execution_date() or timezone.utcnow() with create_session() as session: - query = session.query(DagRun).filter( - DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date - ) + query = select(DagRun).where(DagRun.dag_id == dag.dag_id, DagRun.execution_date <= base_date) - run_type = request.args.get("run_type") - if run_type: - query = query.filter(DagRun.run_type == run_type) + run_type = request.args.get("run_type") + if run_type: + query = query.where(DagRun.run_type == run_type) - run_state = request.args.get("run_state") - if run_state: - query = query.filter(DagRun.state == run_state) + run_state = request.args.get("run_state") + if run_state: + query = query.where(DagRun.state == run_state) - dag_runs = wwwutils.sorted_dag_runs(query, ordering=dag.timetable.run_ordering, limit=num_runs) - encoded_runs = [ - wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs - ] - data = { - "groups": dag_to_grid(dag, dag_runs, session), - "dag_runs": encoded_runs, - "ordering": dag.timetable.run_ordering, - } + dag_runs = wwwutils.sorted_dag_runs( + query, ordering=dag.timetable.run_ordering, limit=num_runs, session=session + ) + encoded_runs = [wwwutils.encode_dag_run(dr, json_encoder=utils_json.WebEncoder) for dr in dag_runs] + data = { + "groups": dag_to_grid(dag, dag_runs, session), + "dag_runs": encoded_runs, + "ordering": dag.timetable.run_ordering, + } # avoid spaces to reduce payload size return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -3864,37 +3875,34 @@ def historical_metrics_data(self): end_date = _safe_parse_datetime(request.args.get("end_date")) with create_session() as session: # DagRuns - dag_runs_type = ( - session.query(DagRun.run_type, func.count(DagRun.run_id)) - .filter( + dag_runs_type = session.execute( + select(DagRun.run_type, func.count(DagRun.run_id)) + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.run_type) - .all() - ) + ).all() - dag_run_states = ( - session.query(DagRun.state, func.count(DagRun.run_id)) - .filter( + dag_run_states = session.execute( + select(DagRun.state, func.count(DagRun.run_id)) + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(DagRun.state) - .all() - ) + ).all() # TaskInstances - task_instance_states = ( - session.query(TaskInstance.state, func.count(TaskInstance.run_id)) + task_instance_states = session.execute( + select(TaskInstance.state, func.count(TaskInstance.run_id)) .join(TaskInstance.dag_run) - .filter( + .where( DagRun.start_date >= start_date, or_(DagRun.end_date.is_(None), DagRun.end_date <= end_date), ) .group_by(TaskInstance.state) - .all() - ) + ).all() data = { "dag_run_types": { @@ -3928,28 +3936,32 @@ def next_run_datasets(self, dag_id): with create_session() as session: data = [ dict(info) - for info in session.query( - DatasetModel.id, - DatasetModel.uri, - func.max(DatasetEvent.timestamp).label("lastUpdate"), - ) - .join(DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id) - .join( - DatasetDagRunQueue, - and_( - DatasetDagRunQueue.dataset_id == DatasetModel.id, - DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, - ), - isouter=True, - ) - .join( - DatasetEvent, - DatasetEvent.dataset_id == DatasetModel.id, - isouter=True, + for info in session.execute( + select( + DatasetModel.id, + DatasetModel.uri, + func.max(DatasetEvent.timestamp).label("lastUpdate"), + ) + .join( + DagScheduleDatasetReference, DagScheduleDatasetReference.dataset_id == DatasetModel.id + ) + .join( + DatasetDagRunQueue, + and_( + DatasetDagRunQueue.dataset_id == DatasetModel.id, + DatasetDagRunQueue.target_dag_id == DagScheduleDatasetReference.dag_id, + ), + isouter=True, + ) + .join( + DatasetEvent, + DatasetEvent.dataset_id == DatasetModel.id, + isouter=True, + ) + .where(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) + .group_by(DatasetModel.id, DatasetModel.uri) + .order_by(DatasetModel.uri) ) - .filter(DagScheduleDatasetReference.dag_id == dag_id, ~DatasetModel.is_orphaned) - .group_by(DatasetModel.id, DatasetModel.uri) - .order_by(DatasetModel.uri) ] return ( htmlsafe_json_dumps(data, separators=(",", ":"), dumps=flask.json.dumps), @@ -4047,12 +4059,12 @@ def datasets_summary(self): if session.bind.dialect.name == "postgresql": order_by = (order_by[0].nulls_first(), *order_by[1:]) - count_query = session.query(func.count(DatasetModel.id)) + count_query = select(func.count(DatasetModel.id)) has_event_filters = bool(updated_before or updated_after) query = ( - session.query( + select( DatasetModel.id, DatasetModel.uri, func.max(DatasetEvent.timestamp).label("last_dataset_update"), @@ -4077,11 +4089,12 @@ def datasets_summary(self): if updated_before: filters.append(DatasetEvent.timestamp <= updated_before) - query = query.filter(*filters).offset(offset).limit(limit) - count_query = count_query.filter(*filters) + query = query.where(*filters).offset(offset).limit(limit) + count_query = count_query.where(*filters) + query = session.execute(query) datasets = [dict(dataset) for dataset in query] - data = {"datasets": datasets, "total_entries": count_query.scalar()} + data = {"datasets": datasets, "total_entries": session.scalar(count_query)} return ( htmlsafe_json_dumps(data, separators=(",", ":"), cls=utils_json.WebEncoder), @@ -4127,20 +4140,20 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): included_events_raw = conf.get("webserver", "audit_view_included_events", fallback=None) excluded_events_raw = conf.get("webserver", "audit_view_excluded_events", fallback=None) - query = session.query(Log).filter(Log.dag_id == dag_id) + query = select(Log).where(Log.dag_id == dag_id) if included_events_raw: included_events = {event.strip() for event in included_events_raw.split(",")} - query = query.filter(Log.event.in_(included_events)) + query = query.where(Log.event.in_(included_events)) elif excluded_events_raw: excluded_events = {event.strip() for event in excluded_events_raw.split(",")} - query = query.filter(Log.event.notin_(excluded_events)) + query = query.where(Log.event.notin_(excluded_events)) current_page = request.args.get("page", default=0, type=int) arg_sorting_key = request.args.get("sorting_key", "dttm") arg_sorting_direction = request.args.get("sorting_direction", default="desc") logs_per_page = PAGE_SIZE - audit_logs_count = query.count() + audit_logs_count = session.scalar(select(func.count()).select_from(query)) num_of_pages = int(math.ceil(audit_logs_count / float(logs_per_page))) start = current_page * logs_per_page @@ -4152,7 +4165,7 @@ def audit_log(self, dag_id: str, session: Session = NEW_SESSION): sort_column = sort_column.desc() query = query.order_by(sort_column) - dag_audit_logs = query.offset(start).limit(logs_per_page).all() + dag_audit_logs = session.scalars(query.offset(start).limit(logs_per_page)).all() return self.render_template( "airflow/dag_audit_log.html", dag=dag, @@ -4273,7 +4286,7 @@ def apply(self, query, func): if get_airflow_app().appbuilder.sm.has_all_dags_access(g.user): return query filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - return query.filter(self.model.dag_id.in_(filter_dag_ids)) + return query.where(self.model.dag_id.in_(filter_dag_ids)) class AirflowModelView(ModelView): @@ -4718,9 +4731,11 @@ def action_mulduplicate(self, connections, session: Session = NEW_SESSION): potential_connection_ids = [f"{base_conn_id}_copy{i}" for i in range(1, 11)] - query = session.query(Connection.conn_id).filter(Connection.conn_id.in_(potential_connection_ids)) + query = session.scalars( + select(Connection.conn_id).where(Connection.conn_id.in_(potential_connection_ids)) + ) - found_conn_id_set = {conn_id for conn_id, in query} + found_conn_id_set = {conn_id for conn_id in query} possible_conn_id_iter = ( connection_id @@ -5390,7 +5405,7 @@ def _set_dag_runs_to_active_state(self, drs: list[DagRun], state: str, session: """This routine only supports Running and Queued state.""" try: count = 0 - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 if state == State.RUNNING: dr.start_date = timezone.utcnow() @@ -5416,7 +5431,7 @@ def action_set_failed(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 altered_tis += set_dag_run_state_to_failed( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5444,7 +5459,7 @@ def action_set_success(self, drs: list[DagRun], session: Session = NEW_SESSION): try: count = 0 altered_tis = [] - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 altered_tis += set_dag_run_state_to_success( dag=get_airflow_app().dag_bag.get_dag(dr.dag_id), @@ -5468,7 +5483,7 @@ def action_clear(self, drs: list[DagRun], session: Session = NEW_SESSION): count = 0 cleared_ti_count = 0 dag_to_tis: dict[DAG, list[TaskInstance]] = {} - for dr in session.query(DagRun).filter(DagRun.id.in_(dagrun.id for dagrun in drs)): + for dr in session.scalars(select(DagRun).where(DagRun.id.in_(dagrun.id for dagrun in drs))): count += 1 dag = get_airflow_app().dag_bag.get_dag(dr.dag_id) tis_to_clear = dag_to_tis.setdefault(dag, []) @@ -5884,35 +5899,37 @@ def autocomplete(self, session: Session = NEW_SESSION): return flask.json.jsonify([]) # Provide suggestions of dag_ids and owners - dag_ids_query = session.query( + dag_ids_query = select( sqla.literal("dag").label("type"), DagModel.dag_id.label("name"), - ).filter(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) + ).where(~DagModel.is_subdag, DagModel.is_active, DagModel.dag_id.ilike(f"%{query}%")) owners_query = ( - session.query( + select( sqla.literal("owner").label("type"), DagModel.owners.label("name"), ) .distinct() - .filter(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) + .where(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) ) # Hide DAGs if not showing status: "all" status = flask_session.get(FILTER_STATUS_COOKIE) if status == "active": - dag_ids_query = dag_ids_query.filter(~DagModel.is_paused) - owners_query = owners_query.filter(~DagModel.is_paused) + dag_ids_query = dag_ids_query.where(~DagModel.is_paused) + owners_query = owners_query.where(~DagModel.is_paused) elif status == "paused": - dag_ids_query = dag_ids_query.filter(DagModel.is_paused) - owners_query = owners_query.filter(DagModel.is_paused) + dag_ids_query = dag_ids_query.where(DagModel.is_paused) + owners_query = owners_query.where(DagModel.is_paused) filter_dag_ids = get_airflow_app().appbuilder.sm.get_accessible_dag_ids(g.user) - dag_ids_query = dag_ids_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - owners_query = owners_query.filter(DagModel.dag_id.in_(filter_dag_ids)) - - payload = [row._asdict() for row in dag_ids_query.union(owners_query).order_by("name").limit(10)] + dag_ids_query = dag_ids_query.where(DagModel.dag_id.in_(filter_dag_ids)) + owners_query = owners_query.where(DagModel.dag_id.in_(filter_dag_ids)) + payload = [ + row._asdict() + for row in session.execute(dag_ids_query.union(owners_query).order_by("name").limit(10)) + ] return flask.json.jsonify(payload) From e53320d62030a53c6ffe896434bcf0fc85803f31 Mon Sep 17 00:00:00 2001 From: Sai Pragna Etikyala Date: Wed, 5 Jul 2023 02:22:30 -0700 Subject: [PATCH 057/533] Update python.rst with multi-Scheduler LocalExecutor info (#32310) Co-authored-by: Pankaj Koti --- docs/apache-airflow/core-concepts/executor/local.rst | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/apache-airflow/core-concepts/executor/local.rst b/docs/apache-airflow/core-concepts/executor/local.rst index 256c6125538c..94a3f512112a 100644 --- a/docs/apache-airflow/core-concepts/executor/local.rst +++ b/docs/apache-airflow/core-concepts/executor/local.rst @@ -47,3 +47,11 @@ Arguably, :class:`~airflow.executors.sequential_executor.SequentialExecutor` cou parallelism of just 1 worker, i.e. ``self.parallelism = 1``. This option could lead to the unification of the executor implementations, running locally, into just one :class:`~airflow.executors.local_executor.LocalExecutor` with multiple modes. + +.. note:: + + When multiple Schedulers are configured with ``executor = LocalExecutor`` in the ``[core]`` section of your ``airflow.cfg``, each Scheduler will run a LocalExecutor. This means tasks would be processed in a distributed fashion across the machines running the Schedulers. + + One consideration should be taken into account: + + - Restarting a Scheduler: If a Scheduler is restarted, it may take some time for other Schedulers to recognize the orphaned tasks and restart or fail them. From c8a3c112a7bae345d37bb8b90d68c8d6ff2ef8fc Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 5 Jul 2023 13:28:16 +0200 Subject: [PATCH 058/533] Retry transactions on occasional deadlocks for rendered fields (#32341) This is a follow-up on #18616 where we introduced retries on the occassional deadlocks when rendered task fields have been deleted by parallel threads (this is not a real deadlock, it's because MySQL locks too many things when queries are executed and will deadlock when one of those queries wait too much). Adding retry - while not perfect - should allow to handle the problem and significantly decrease the likelihood of such deadlocks. We can probably think about different approach for rendered fields, but for now retrying is - I think - acceptable short-term fix. Fixes: #32294 Fixes: #29687 --- airflow/models/renderedtifields.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/renderedtifields.py b/airflow/models/renderedtifields.py index 8cea81715f37..958683447004 100644 --- a/airflow/models/renderedtifields.py +++ b/airflow/models/renderedtifields.py @@ -175,6 +175,7 @@ def get_k8s_pod_yaml(cls, ti: TaskInstance, session: Session = NEW_SESSION) -> d return result.k8s_pod_yaml if result else None @provide_session + @retry_db_transaction def write(self, session: Session = None): """Write instance to database. From 41954fe194afcd316153ae18f9e7448af4a7f368 Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Wed, 5 Jul 2023 17:14:46 +0530 Subject: [PATCH 059/533] Improving LOCAL_VIRTUALENV.rst for ease of local venv setup (#32372) Co-authored-by: Amogh Desai --- LOCAL_VIRTUALENV.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/LOCAL_VIRTUALENV.rst b/LOCAL_VIRTUALENV.rst index 987a0df4bfa5..1f541270ec8e 100644 --- a/LOCAL_VIRTUALENV.rst +++ b/LOCAL_VIRTUALENV.rst @@ -58,12 +58,18 @@ Homebrew for macOS to install required software packages: Refer to the `Dockerfile.ci `__ for a comprehensive list of required packages. +.. note:: + + - MySql 2.2.0 needs pkgconf to be a pre requisite, refer `here `_ to install pkgconf + - MacOs with ARM architectures require graphviz for venv setup, refer `here `_ to install graphviz + Extra Packages -------------- .. note:: Only ``pip`` installation is currently officially supported. + Make sure you have the latest pip installed, reference `version `_ While there are some successes with using other tools like `poetry `_ or `pip-tools `_, they do not share the same workflow as From ee4a838d49461b3b053a9cbe660dbff06a17fff5 Mon Sep 17 00:00:00 2001 From: JDarDagran Date: Wed, 5 Jul 2023 15:36:10 +0200 Subject: [PATCH 060/533] Pass SQLAlchemy engine to construct information schema query. (#32371) Signed-off-by: Jakub Dardzinski --- airflow/providers/common/sql/hooks/sql.py | 2 +- airflow/providers/common/sql/operators/sql.py | 6 +++++- airflow/providers/openlineage/sqlparser.py | 20 +++++++++++++++++-- .../common/sql/operators/test_sql_execute.py | 2 +- 4 files changed, 25 insertions(+), 5 deletions(-) diff --git a/airflow/providers/common/sql/hooks/sql.py b/airflow/providers/common/sql/hooks/sql.py index 4ff6e62e2497..3f2f964a0bb6 100644 --- a/airflow/providers/common/sql/hooks/sql.py +++ b/airflow/providers/common/sql/hooks/sql.py @@ -538,7 +538,7 @@ def get_openlineage_database_dialect(self, connection) -> str: """ return "generic" - def get_openlineage_default_schema(self) -> str: + def get_openlineage_default_schema(self) -> str | None: """ Returns default schema specific to database. diff --git a/airflow/providers/common/sql/operators/sql.py b/airflow/providers/common/sql/operators/sql.py index 5f129e0817e6..53b499f50631 100644 --- a/airflow/providers/common/sql/operators/sql.py +++ b/airflow/providers/common/sql/operators/sql.py @@ -320,7 +320,11 @@ def get_openlineage_facets_on_start(self) -> OperatorLineage | None: return None operator_lineage = sql_parser.generate_openlineage_metadata_from_sql( - sql=self.sql, hook=hook, database_info=database_info, database=self.database + sql=self.sql, + hook=hook, + database_info=database_info, + database=self.database, + sqlalchemy_engine=hook.get_sqlalchemy_engine(), ) return operator_lineage diff --git a/airflow/providers/openlineage/sqlparser.py b/airflow/providers/openlineage/sqlparser.py index 8e85d3706c52..fdf36c94d37f 100644 --- a/airflow/providers/openlineage/sqlparser.py +++ b/airflow/providers/openlineage/sqlparser.py @@ -33,6 +33,8 @@ from openlineage.common.sql import DbTableMeta, SqlMeta, parse if TYPE_CHECKING: + from sqlalchemy.engine import Engine + from airflow.hooks.base import BaseHook DEFAULT_NAMESPACE = "default" @@ -113,6 +115,7 @@ def parse_table_schemas( database_info: DatabaseInfo, namespace: str = DEFAULT_NAMESPACE, database: str | None = None, + sqlalchemy_engine: Engine | None = None, ) -> tuple[list[Dataset], ...]: """Parse schemas for input and output tables.""" database_kwargs: GetTableSchemasParams = { @@ -128,8 +131,16 @@ def parse_table_schemas( namespace, self.default_schema, database or database_info.database, - self.create_information_schema_query(tables=inputs, **database_kwargs) if inputs else None, - self.create_information_schema_query(tables=outputs, **database_kwargs) if outputs else None, + self.create_information_schema_query( + tables=inputs, sqlalchemy_engine=sqlalchemy_engine, **database_kwargs + ) + if inputs + else None, + self.create_information_schema_query( + tables=outputs, sqlalchemy_engine=sqlalchemy_engine, **database_kwargs + ) + if outputs + else None, ) def generate_openlineage_metadata_from_sql( @@ -138,6 +149,7 @@ def generate_openlineage_metadata_from_sql( hook: BaseHook, database_info: DatabaseInfo, database: str | None = None, + sqlalchemy_engine: Engine | None = None, ) -> OperatorLineage: """Parses SQL statement(s) and generates OpenLineage metadata. @@ -152,6 +164,7 @@ def generate_openlineage_metadata_from_sql( :param hook: Airflow Hook used to connect to the database :param database_info: database specific information :param database: when passed it takes precedence over parsed database name + :param sqlalchemy_engine: when passed, engine's dialect is used to compile SQL queries """ job_facets: dict[str, BaseFacet] = {"sql": SqlJobFacet(query=self.normalize_sql(sql))} parse_result = self.parse(self.split_sql_string(sql)) @@ -182,6 +195,7 @@ def generate_openlineage_metadata_from_sql( namespace=namespace, database=database, database_info=database_info, + sqlalchemy_engine=sqlalchemy_engine, ) return OperatorLineage( @@ -236,6 +250,7 @@ def create_information_schema_query( information_schema_table, is_uppercase_names, database: str | None = None, + sqlalchemy_engine: Engine | None = None, ) -> str: """Creates SELECT statement to query information schema table.""" tables_hierarchy = cls._get_tables_hierarchy( @@ -249,6 +264,7 @@ def create_information_schema_query( information_schema_table_name=information_schema_table, tables_hierarchy=tables_hierarchy, uppercase_names=is_uppercase_names, + sqlalchemy_engine=sqlalchemy_engine, ) @staticmethod diff --git a/tests/providers/common/sql/operators/test_sql_execute.py b/tests/providers/common/sql/operators/test_sql_execute.py index 5bc2a932f878..48f59e0ffea4 100644 --- a/tests/providers/common/sql/operators/test_sql_execute.py +++ b/tests/providers/common/sql/operators/test_sql_execute.py @@ -316,7 +316,7 @@ def get_db_hook(self): (DB_SCHEMA_NAME, "popular_orders_day_of_week", "orders_placed", 3, "int4"), ] dbapi_hook.get_connection.return_value = Connection( - conn_id="sql_default", conn_type="postgres", host="host", port=1234 + conn_id="sql_default", conn_type="postgresql", host="host", port=1234 ) dbapi_hook.get_conn.return_value.cursor.return_value.fetchall.side_effect = [rows, []] From 3878fe6fab3ccc1461932b456c48996f2763139f Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Wed, 5 Jul 2023 16:40:00 +0200 Subject: [PATCH 061/533] Remove spurious headers for provider changelogs (#32373) --- airflow/providers/airbyte/CHANGELOG.rst | 8 +++++--- airflow/providers/alibaba/CHANGELOG.rst | 8 +++++--- airflow/providers/amazon/CHANGELOG.rst | 8 +++++--- airflow/providers/apache/beam/CHANGELOG.rst | 8 +++++--- airflow/providers/apache/cassandra/CHANGELOG.rst | 8 +++++--- airflow/providers/apache/drill/CHANGELOG.rst | 8 +++++--- airflow/providers/apache/druid/CHANGELOG.rst | 9 ++++++--- airflow/providers/apache/flink/CHANGELOG.rst | 8 +++++--- airflow/providers/apache/hdfs/CHANGELOG.rst | 9 ++++++--- airflow/providers/apache/hive/CHANGELOG.rst | 9 ++++++--- airflow/providers/apache/impala/CHANGELOG.rst | 9 ++++++--- airflow/providers/apache/kafka/CHANGELOG.rst | 10 +++++++--- airflow/providers/apache/kylin/CHANGELOG.rst | 11 ++++++++--- airflow/providers/apache/livy/CHANGELOG.rst | 10 +++++++--- airflow/providers/apache/pig/CHANGELOG.rst | 11 ++++++++--- airflow/providers/apache/pinot/CHANGELOG.rst | 11 ++++++++--- airflow/providers/apache/spark/CHANGELOG.rst | 11 ++++++++--- airflow/providers/apache/sqoop/CHANGELOG.rst | 10 +++++++--- airflow/providers/apprise/CHANGELOG.rst | 8 +++++++- airflow/providers/arangodb/CHANGELOG.rst | 9 ++++++--- airflow/providers/asana/CHANGELOG.rst | 9 ++++++--- airflow/providers/atlassian/jira/CHANGELOG.rst | 9 ++++++--- airflow/providers/celery/CHANGELOG.rst | 9 ++++++--- airflow/providers/cloudant/CHANGELOG.rst | 9 ++++++--- airflow/providers/cncf/kubernetes/CHANGELOG.rst | 9 ++++++--- airflow/providers/common/sql/CHANGELOG.rst | 5 ++--- airflow/providers/databricks/CHANGELOG.rst | 9 ++++++--- airflow/providers/datadog/CHANGELOG.rst | 9 ++++++--- airflow/providers/dbt/cloud/CHANGELOG.rst | 10 +++++++--- airflow/providers/dingding/CHANGELOG.rst | 9 ++++++--- airflow/providers/discord/CHANGELOG.rst | 9 ++++++--- airflow/providers/docker/CHANGELOG.rst | 9 ++++++--- airflow/providers/elasticsearch/CHANGELOG.rst | 9 ++++++--- airflow/providers/exasol/CHANGELOG.rst | 9 ++++++--- airflow/providers/facebook/CHANGELOG.rst | 9 ++++++--- airflow/providers/ftp/CHANGELOG.rst | 9 ++++++--- airflow/providers/github/CHANGELOG.rst | 10 +++++++--- airflow/providers/google/CHANGELOG.rst | 10 +++++++--- airflow/providers/grpc/CHANGELOG.rst | 9 ++++++--- airflow/providers/hashicorp/CHANGELOG.rst | 9 ++++++--- airflow/providers/http/CHANGELOG.rst | 9 ++++++--- airflow/providers/imap/CHANGELOG.rst | 8 +++++--- airflow/providers/influxdb/CHANGELOG.rst | 9 ++++++--- airflow/providers/jdbc/CHANGELOG.rst | 8 +++++--- airflow/providers/jenkins/CHANGELOG.rst | 9 ++++++--- airflow/providers/microsoft/azure/CHANGELOG.rst | 9 ++++++--- airflow/providers/microsoft/mssql/CHANGELOG.rst | 9 ++++++--- airflow/providers/microsoft/psrp/CHANGELOG.rst | 9 ++++++--- airflow/providers/microsoft/winrm/CHANGELOG.rst | 9 ++++++--- airflow/providers/mongo/CHANGELOG.rst | 9 ++++++--- airflow/providers/mysql/CHANGELOG.rst | 9 ++++++--- airflow/providers/neo4j/CHANGELOG.rst | 10 +++++++--- airflow/providers/odbc/CHANGELOG.rst | 9 ++++++--- airflow/providers/openfaas/CHANGELOG.rst | 9 ++++++--- airflow/providers/openlineage/CHANGELOG.rst | 9 ++++++--- airflow/providers/opsgenie/CHANGELOG.rst | 11 +++++++---- airflow/providers/oracle/CHANGELOG.rst | 9 ++++++--- airflow/providers/pagerduty/CHANGELOG.rst | 9 ++++++--- airflow/providers/papermill/CHANGELOG.rst | 9 ++++++--- airflow/providers/plexus/CHANGELOG.rst | 9 ++++++--- airflow/providers/postgres/CHANGELOG.rst | 9 ++++++--- airflow/providers/presto/CHANGELOG.rst | 9 ++++++--- airflow/providers/qubole/CHANGELOG.rst | 9 ++++++--- airflow/providers/redis/CHANGELOG.rst | 9 ++++++--- airflow/providers/salesforce/CHANGELOG.rst | 9 ++++++--- airflow/providers/samba/CHANGELOG.rst | 9 ++++++--- airflow/providers/segment/CHANGELOG.rst | 9 ++++++--- airflow/providers/sendgrid/CHANGELOG.rst | 9 ++++++--- airflow/providers/sftp/CHANGELOG.rst | 9 ++++++--- airflow/providers/singularity/CHANGELOG.rst | 9 ++++++--- airflow/providers/slack/CHANGELOG.rst | 9 ++++++--- airflow/providers/smtp/CHANGELOG.rst | 9 ++++++--- airflow/providers/snowflake/CHANGELOG.rst | 9 ++++++--- airflow/providers/sqlite/CHANGELOG.rst | 9 ++++++--- airflow/providers/ssh/CHANGELOG.rst | 9 ++++++--- airflow/providers/tableau/CHANGELOG.rst | 9 ++++++--- airflow/providers/tabular/CHANGELOG.rst | 9 ++++++--- airflow/providers/telegram/CHANGELOG.rst | 9 ++++++--- airflow/providers/trino/CHANGELOG.rst | 9 ++++++--- airflow/providers/vertica/CHANGELOG.rst | 10 +++++++--- airflow/providers/yandex/CHANGELOG.rst | 9 ++++++--- airflow/providers/zendesk/CHANGELOG.rst | 9 ++++++--- .../PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2 | 3 --- dev/provider_packages/prepare_provider_packages.py | 7 ++++--- docs/apache-airflow-providers-airbyte/changelog.rst | 3 +-- docs/apache-airflow-providers-alibaba/changelog.rst | 3 +-- docs/apache-airflow-providers-amazon/changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../apache-airflow-providers-apache-pig/changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-apprise/changelog.rst | 3 +-- docs/apache-airflow-providers-arangodb/changelog.rst | 3 +-- docs/apache-airflow-providers-asana/changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-celery/changelog.rst | 3 +-- docs/apache-airflow-providers-cloudant/changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../apache-airflow-providers-common-sql/changelog.rst | 3 +-- .../apache-airflow-providers-databricks/changelog.rst | 3 +-- docs/apache-airflow-providers-datadog/changelog.rst | 3 +-- docs/apache-airflow-providers-dbt-cloud/changelog.rst | 3 +-- docs/apache-airflow-providers-dingding/changelog.rst | 3 +-- docs/apache-airflow-providers-discord/changelog.rst | 3 +-- docs/apache-airflow-providers-docker/changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-exasol/changelog.rst | 3 +-- docs/apache-airflow-providers-facebook/changelog.rst | 3 +-- docs/apache-airflow-providers-ftp/changelog.rst | 3 +-- docs/apache-airflow-providers-github/changelog.rst | 3 +-- docs/apache-airflow-providers-google/changelog.rst | 3 +-- docs/apache-airflow-providers-grpc/changelog.rst | 3 +-- docs/apache-airflow-providers-hashicorp/changelog.rst | 3 +-- docs/apache-airflow-providers-http/changelog.rst | 3 +-- docs/apache-airflow-providers-imap/changelog.rst | 3 +-- docs/apache-airflow-providers-influxdb/changelog.rst | 3 +-- docs/apache-airflow-providers-jdbc/changelog.rst | 3 +-- docs/apache-airflow-providers-jenkins/changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-mongo/changelog.rst | 3 +-- docs/apache-airflow-providers-mysql/changelog.rst | 3 +-- docs/apache-airflow-providers-neo4j/changelog.rst | 3 +-- docs/apache-airflow-providers-odbc/changelog.rst | 3 +-- docs/apache-airflow-providers-openfaas/changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-opsgenie/changelog.rst | 3 +-- docs/apache-airflow-providers-oracle/changelog.rst | 3 +-- docs/apache-airflow-providers-pagerduty/changelog.rst | 3 +-- docs/apache-airflow-providers-papermill/changelog.rst | 3 +-- docs/apache-airflow-providers-plexus/changelog.rst | 3 +-- docs/apache-airflow-providers-postgres/changelog.rst | 3 +-- docs/apache-airflow-providers-presto/changelog.rst | 3 +-- docs/apache-airflow-providers-qubole/changelog.rst | 3 +-- docs/apache-airflow-providers-redis/changelog.rst | 3 +-- .../apache-airflow-providers-salesforce/changelog.rst | 3 +-- docs/apache-airflow-providers-samba/changelog.rst | 3 +-- docs/apache-airflow-providers-segment/changelog.rst | 3 +-- docs/apache-airflow-providers-sendgrid/changelog.rst | 3 +-- docs/apache-airflow-providers-sftp/changelog.rst | 3 +-- .../changelog.rst | 3 +-- docs/apache-airflow-providers-slack/changelog.rst | 3 +-- docs/apache-airflow-providers-smtp/changelog.rst | 3 +-- docs/apache-airflow-providers-snowflake/changelog.rst | 3 +-- docs/apache-airflow-providers-sqlite/changelog.rst | 3 +-- docs/apache-airflow-providers-ssh/changelog.rst | 3 +-- docs/apache-airflow-providers-tableau/changelog.rst | 3 +-- docs/apache-airflow-providers-tabular/changelog.rst | 3 +-- docs/apache-airflow-providers-telegram/changelog.rst | 3 +-- docs/apache-airflow-providers-trino/changelog.rst | 3 +-- docs/apache-airflow-providers-vertica/changelog.rst | 3 +-- docs/apache-airflow-providers-zendesk/changelog.rst | 3 +-- 165 files changed, 582 insertions(+), 413 deletions(-) diff --git a/airflow/providers/airbyte/CHANGELOG.rst b/airflow/providers/airbyte/CHANGELOG.rst index 6d8f34d5cd49..00a9cece44ac 100644 --- a/airflow/providers/airbyte/CHANGELOG.rst +++ b/airflow/providers/airbyte/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-airbyte`` --------------------------------------------------- +``apache-airflow-providers-airbyte`` + +Changelog +--------- 3.3.1 diff --git a/airflow/providers/alibaba/CHANGELOG.rst b/airflow/providers/alibaba/CHANGELOG.rst index af267238d0f8..b55e5aa6ea61 100644 --- a/airflow/providers/alibaba/CHANGELOG.rst +++ b/airflow/providers/alibaba/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-alibaba`` --------------------------------------------------- +``apache-airflow-providers-alibaba`` + +Changelog +--------- 2.5.0 ..... diff --git a/airflow/providers/amazon/CHANGELOG.rst b/airflow/providers/amazon/CHANGELOG.rst index 1b3b94fefc23..8af159f61104 100644 --- a/airflow/providers/amazon/CHANGELOG.rst +++ b/airflow/providers/amazon/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-amazon`` -------------------------------------------------- +``apache-airflow-providers-amazon`` + +Changelog +--------- 8.3.0 ..... diff --git a/airflow/providers/apache/beam/CHANGELOG.rst b/airflow/providers/apache/beam/CHANGELOG.rst index 495fba99e910..2176fc18c5e3 100644 --- a/airflow/providers/apache/beam/CHANGELOG.rst +++ b/airflow/providers/apache/beam/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-beam`` ------------------------------------------------------- +``apache-airflow-providers-apache-beam`` + +Changelog +--------- 5.1.1 ..... diff --git a/airflow/providers/apache/cassandra/CHANGELOG.rst b/airflow/providers/apache/cassandra/CHANGELOG.rst index 00bfcb83f816..10895c8d1507 100644 --- a/airflow/providers/apache/cassandra/CHANGELOG.rst +++ b/airflow/providers/apache/cassandra/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-cassandra`` ------------------------------------------------------------ +``apache-airflow-providers-apache-cassandra`` + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/apache/drill/CHANGELOG.rst b/airflow/providers/apache/drill/CHANGELOG.rst index 60df655ec462..37b6b3a5fdd0 100644 --- a/airflow/providers/apache/drill/CHANGELOG.rst +++ b/airflow/providers/apache/drill/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-drill`` -------------------------------------------------------- +``apache-airflow-providers-apache-drill`` + +Changelog +--------- 2.4.1 diff --git a/airflow/providers/apache/druid/CHANGELOG.rst b/airflow/providers/apache/druid/CHANGELOG.rst index 9a95a041ef71..8e7b0f67ee36 100644 --- a/airflow/providers/apache/druid/CHANGELOG.rst +++ b/airflow/providers/apache/druid/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-druid`` -------------------------------------------------------- +``apache-airflow-providers-apache-druid`` + + +Changelog +--------- 3.4.1 ..... diff --git a/airflow/providers/apache/flink/CHANGELOG.rst b/airflow/providers/apache/flink/CHANGELOG.rst index 40af6d54d452..25e177350b5c 100644 --- a/airflow/providers/apache/flink/CHANGELOG.rst +++ b/airflow/providers/apache/flink/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-flink`` -------------------------------------------------------- +``apache-airflow-providers-apache-flink`` + +Changelog +--------- 1.1.1 ..... diff --git a/airflow/providers/apache/hdfs/CHANGELOG.rst b/airflow/providers/apache/hdfs/CHANGELOG.rst index 7a8000b202a5..37de3a05cc49 100644 --- a/airflow/providers/apache/hdfs/CHANGELOG.rst +++ b/airflow/providers/apache/hdfs/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-hdfs`` ------------------------------------------------------- +``apache-airflow-providers-apache-hdfs`` + + +Changelog +--------- 4.1.0 ----- diff --git a/airflow/providers/apache/hive/CHANGELOG.rst b/airflow/providers/apache/hive/CHANGELOG.rst index 87c44ca909aa..a57689b1d857 100644 --- a/airflow/providers/apache/hive/CHANGELOG.rst +++ b/airflow/providers/apache/hive/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-hive`` ------------------------------------------------------- +``apache-airflow-providers-apache-hive`` + + +Changelog +--------- 6.1.1 diff --git a/airflow/providers/apache/impala/CHANGELOG.rst b/airflow/providers/apache/impala/CHANGELOG.rst index 95e6fca155f0..3dcd84dba83d 100644 --- a/airflow/providers/apache/impala/CHANGELOG.rst +++ b/airflow/providers/apache/impala/CHANGELOG.rst @@ -16,12 +16,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-impala`` --------------------------------------------------------- +``apache-airflow-providers-apache-impala`` + + +Changelog +--------- 1.1.1 ..... diff --git a/airflow/providers/apache/kafka/CHANGELOG.rst b/airflow/providers/apache/kafka/CHANGELOG.rst index 95898617e6c7..5c77dc4b349c 100644 --- a/airflow/providers/apache/kafka/CHANGELOG.rst +++ b/airflow/providers/apache/kafka/CHANGELOG.rst @@ -16,12 +16,16 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-kafka`` -------------------------------------------------------- + +``apache-airflow-providers-apache-kafka`` + + +Changelog +--------- 1.1.2 ..... diff --git a/airflow/providers/apache/kylin/CHANGELOG.rst b/airflow/providers/apache/kylin/CHANGELOG.rst index a40ddec0324c..48b03a0d9b15 100644 --- a/airflow/providers/apache/kylin/CHANGELOG.rst +++ b/airflow/providers/apache/kylin/CHANGELOG.rst @@ -17,12 +17,17 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-kylin`` -------------------------------------------------------- + +``apache-airflow-providers-apache-kylin`` + + + +Changelog +--------- 3.2.1 diff --git a/airflow/providers/apache/livy/CHANGELOG.rst b/airflow/providers/apache/livy/CHANGELOG.rst index 93dc82bbbfe8..36fb6f5cb330 100644 --- a/airflow/providers/apache/livy/CHANGELOG.rst +++ b/airflow/providers/apache/livy/CHANGELOG.rst @@ -17,12 +17,16 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-livy`` ------------------------------------------------------- + +``apache-airflow-providers-apache-livy`` + + +Changelog +--------- 3.5.2 ..... diff --git a/airflow/providers/apache/pig/CHANGELOG.rst b/airflow/providers/apache/pig/CHANGELOG.rst index 72ba0f4a3911..7279d51d7f6a 100644 --- a/airflow/providers/apache/pig/CHANGELOG.rst +++ b/airflow/providers/apache/pig/CHANGELOG.rst @@ -17,12 +17,17 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-pig`` ------------------------------------------------------ + +``apache-airflow-providers-apache-pig`` + + + +Changelog +--------- 4.1.1 ..... diff --git a/airflow/providers/apache/pinot/CHANGELOG.rst b/airflow/providers/apache/pinot/CHANGELOG.rst index eb4942319956..62fe3e9fa007 100644 --- a/airflow/providers/apache/pinot/CHANGELOG.rst +++ b/airflow/providers/apache/pinot/CHANGELOG.rst @@ -17,12 +17,17 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-pinot`` -------------------------------------------------------- + +``apache-airflow-providers-apache-pinot`` + + + +Changelog +--------- 4.1.1 diff --git a/airflow/providers/apache/spark/CHANGELOG.rst b/airflow/providers/apache/spark/CHANGELOG.rst index 2ec87940533d..8129b380a320 100644 --- a/airflow/providers/apache/spark/CHANGELOG.rst +++ b/airflow/providers/apache/spark/CHANGELOG.rst @@ -17,12 +17,17 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-spark`` -------------------------------------------------------- + +``apache-airflow-providers-apache-spark`` + + + +Changelog +--------- 4.1.1 ..... diff --git a/airflow/providers/apache/sqoop/CHANGELOG.rst b/airflow/providers/apache/sqoop/CHANGELOG.rst index a8a5dca3cd5f..d824b7dd1d9f 100644 --- a/airflow/providers/apache/sqoop/CHANGELOG.rst +++ b/airflow/providers/apache/sqoop/CHANGELOG.rst @@ -17,12 +17,16 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-apache-sqoop`` -------------------------------------------------------- + +``apache-airflow-providers-apache-sqoop`` + + +Changelog +--------- 3.2.1 diff --git a/airflow/providers/apprise/CHANGELOG.rst b/airflow/providers/apprise/CHANGELOG.rst index 9b5d755cbec9..bf8999d3a481 100644 --- a/airflow/providers/apprise/CHANGELOG.rst +++ b/airflow/providers/apprise/CHANGELOG.rst @@ -17,10 +17,16 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. +``apache-airflow-providers-apprise`` + + +Changelog +--------- + 1.0.0 ..... diff --git a/airflow/providers/arangodb/CHANGELOG.rst b/airflow/providers/arangodb/CHANGELOG.rst index e72e8963d432..d51c1f705eda 100644 --- a/airflow/providers/arangodb/CHANGELOG.rst +++ b/airflow/providers/arangodb/CHANGELOG.rst @@ -18,12 +18,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-arangodb`` ---------------------------------------------------- +``apache-airflow-providers-arangodb`` + + +Changelog +--------- 2.2.1 ..... diff --git a/airflow/providers/asana/CHANGELOG.rst b/airflow/providers/asana/CHANGELOG.rst index 0e8de8a3a558..1a811682b7e4 100644 --- a/airflow/providers/asana/CHANGELOG.rst +++ b/airflow/providers/asana/CHANGELOG.rst @@ -16,12 +16,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-asana`` ------------------------------------------------- +``apache-airflow-providers-asana`` + + +Changelog +--------- 2.2.1 ..... diff --git a/airflow/providers/atlassian/jira/CHANGELOG.rst b/airflow/providers/atlassian/jira/CHANGELOG.rst index 3358ac28cb85..f1a09c1b6768 100644 --- a/airflow/providers/atlassian/jira/CHANGELOG.rst +++ b/airflow/providers/atlassian/jira/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-atlassian-jira`` ---------------------------------------------------------- +``apache-airflow-providers-jira`` + + +Changelog +--------- 2.1.1 ..... diff --git a/airflow/providers/celery/CHANGELOG.rst b/airflow/providers/celery/CHANGELOG.rst index 1f3b37ac2d55..85ccbc5fc4be 100644 --- a/airflow/providers/celery/CHANGELOG.rst +++ b/airflow/providers/celery/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-celery`` -------------------------------------------------- +``apache-airflow-providers-celery`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/cloudant/CHANGELOG.rst b/airflow/providers/cloudant/CHANGELOG.rst index 93fea1598dbf..f1d30bb60620 100644 --- a/airflow/providers/cloudant/CHANGELOG.rst +++ b/airflow/providers/cloudant/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-cloudant`` ---------------------------------------------------- +``apache-airflow-providers-cloudant`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/cncf/kubernetes/CHANGELOG.rst b/airflow/providers/cncf/kubernetes/CHANGELOG.rst index f90802cbe402..c089b470adda 100644 --- a/airflow/providers/cncf/kubernetes/CHANGELOG.rst +++ b/airflow/providers/cncf/kubernetes/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-cncf-kubernetes`` ----------------------------------------------------------- +``apache-airflow-providers-cncf-kubernetes`` + + +Changelog +--------- 7.2.0 ..... diff --git a/airflow/providers/common/sql/CHANGELOG.rst b/airflow/providers/common/sql/CHANGELOG.rst index 5710d8e39dda..566c2e6b4266 100644 --- a/airflow/providers/common/sql/CHANGELOG.rst +++ b/airflow/providers/common/sql/CHANGELOG.rst @@ -16,13 +16,12 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-common-sql`` ------------------------------------------------------ +------ 1.6.0 ..... diff --git a/airflow/providers/databricks/CHANGELOG.rst b/airflow/providers/databricks/CHANGELOG.rst index 21bf6efebcf9..b21db94e25af 100644 --- a/airflow/providers/databricks/CHANGELOG.rst +++ b/airflow/providers/databricks/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-databricks`` ------------------------------------------------------ +``apache-airflow-providers-databricks`` + + +Changelog +--------- 4.3.1 ..... diff --git a/airflow/providers/datadog/CHANGELOG.rst b/airflow/providers/datadog/CHANGELOG.rst index d7e966661e17..59ebe25fdf23 100644 --- a/airflow/providers/datadog/CHANGELOG.rst +++ b/airflow/providers/datadog/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-datadog`` --------------------------------------------------- +``apache-airflow-providers-datadog`` + + +Changelog +--------- 3.3.1 ..... diff --git a/airflow/providers/dbt/cloud/CHANGELOG.rst b/airflow/providers/dbt/cloud/CHANGELOG.rst index e87ecaf4b8ee..7d6b8c1e4c01 100644 --- a/airflow/providers/dbt/cloud/CHANGELOG.rst +++ b/airflow/providers/dbt/cloud/CHANGELOG.rst @@ -17,12 +17,16 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-dbt-cloud`` ----------------------------------------------------- + +``apache-airflow-providers-dbt-cloud`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/dingding/CHANGELOG.rst b/airflow/providers/dingding/CHANGELOG.rst index 78620c282a9c..06c1954a0e6c 100644 --- a/airflow/providers/dingding/CHANGELOG.rst +++ b/airflow/providers/dingding/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-dingding`` ---------------------------------------------------- +``apache-airflow-providers-dingding`` + + +Changelog +--------- 3.2.1 diff --git a/airflow/providers/discord/CHANGELOG.rst b/airflow/providers/discord/CHANGELOG.rst index 23b9b970dc0f..80f50187507e 100644 --- a/airflow/providers/discord/CHANGELOG.rst +++ b/airflow/providers/discord/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-discord`` --------------------------------------------------- +``apache-airflow-providers-discord`` + + +Changelog +--------- 3.3.0 ..... diff --git a/airflow/providers/docker/CHANGELOG.rst b/airflow/providers/docker/CHANGELOG.rst index 8d52fb8577a1..58f996d69459 100644 --- a/airflow/providers/docker/CHANGELOG.rst +++ b/airflow/providers/docker/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-docker`` -------------------------------------------------- +``apache-airflow-providers-docker`` + + +Changelog +--------- 3.7.1 ..... diff --git a/airflow/providers/elasticsearch/CHANGELOG.rst b/airflow/providers/elasticsearch/CHANGELOG.rst index c104ed62ece0..b2838c1590aa 100644 --- a/airflow/providers/elasticsearch/CHANGELOG.rst +++ b/airflow/providers/elasticsearch/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-elasticsearch`` --------------------------------------------------------- +``apache-airflow-providers-elasticsearch`` + + +Changelog +--------- 5.0.0 ..... diff --git a/airflow/providers/exasol/CHANGELOG.rst b/airflow/providers/exasol/CHANGELOG.rst index a6bbf1659568..c0309305ca8a 100644 --- a/airflow/providers/exasol/CHANGELOG.rst +++ b/airflow/providers/exasol/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-exasol`` -------------------------------------------------- +``apache-airflow-providers-exasol`` + + +Changelog +--------- 4.2.1 ..... diff --git a/airflow/providers/facebook/CHANGELOG.rst b/airflow/providers/facebook/CHANGELOG.rst index 4aca578d134c..0e24be9cb1b2 100644 --- a/airflow/providers/facebook/CHANGELOG.rst +++ b/airflow/providers/facebook/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-facebook`` ---------------------------------------------------- +``apache-airflow-providers-facebook`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/ftp/CHANGELOG.rst b/airflow/providers/ftp/CHANGELOG.rst index d1377ea1152b..9316062acb1d 100644 --- a/airflow/providers/ftp/CHANGELOG.rst +++ b/airflow/providers/ftp/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-ftp`` ----------------------------------------------- +``apache-airflow-providers-ftp`` + + +Changelog +--------- 3.4.2 ..... diff --git a/airflow/providers/github/CHANGELOG.rst b/airflow/providers/github/CHANGELOG.rst index 6e0fcca472e6..b9da973f1796 100644 --- a/airflow/providers/github/CHANGELOG.rst +++ b/airflow/providers/github/CHANGELOG.rst @@ -18,12 +18,16 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-github`` -------------------------------------------------- + +``apache-airflow-providers-github`` + + +Changelog +--------- 2.3.1 ..... diff --git a/airflow/providers/google/CHANGELOG.rst b/airflow/providers/google/CHANGELOG.rst index 630467f4b9e5..a418a8d7b176 100644 --- a/airflow/providers/google/CHANGELOG.rst +++ b/airflow/providers/google/CHANGELOG.rst @@ -16,12 +16,16 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-google`` -------------------------------------------------- + +``apache-airflow-providers-google`` + + +Changelog +--------- 10.3.0 ...... diff --git a/airflow/providers/grpc/CHANGELOG.rst b/airflow/providers/grpc/CHANGELOG.rst index b65a9b0a7587..a752fbdaeb52 100644 --- a/airflow/providers/grpc/CHANGELOG.rst +++ b/airflow/providers/grpc/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-grpc`` ------------------------------------------------ +``apache-airflow-providers-grpc`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/hashicorp/CHANGELOG.rst b/airflow/providers/hashicorp/CHANGELOG.rst index 0b314faafb74..4f22ad96d0ab 100644 --- a/airflow/providers/hashicorp/CHANGELOG.rst +++ b/airflow/providers/hashicorp/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-hashicorp`` ----------------------------------------------------- +``apache-airflow-providers-hashicorp`` + + +Changelog +--------- 3.4.2 ..... diff --git a/airflow/providers/http/CHANGELOG.rst b/airflow/providers/http/CHANGELOG.rst index bab59fd533ba..d8d88f5f8088 100644 --- a/airflow/providers/http/CHANGELOG.rst +++ b/airflow/providers/http/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-http`` ------------------------------------------------ +``apache-airflow-providers-http`` + + +Changelog +--------- 4.4.2 ..... diff --git a/airflow/providers/imap/CHANGELOG.rst b/airflow/providers/imap/CHANGELOG.rst index e5ae86a416d1..24cf83e5b8f5 100644 --- a/airflow/providers/imap/CHANGELOG.rst +++ b/airflow/providers/imap/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-imap`` ------------------------------------------------ +``apache-airflow-providers-imap`` + +Changelog +--------- 3.2.2 ..... diff --git a/airflow/providers/influxdb/CHANGELOG.rst b/airflow/providers/influxdb/CHANGELOG.rst index 081456722f9a..432ff7fc7fd8 100644 --- a/airflow/providers/influxdb/CHANGELOG.rst +++ b/airflow/providers/influxdb/CHANGELOG.rst @@ -18,12 +18,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-influxdb`` ---------------------------------------------------- +``apache-airflow-providers-influxdb`` + + +Changelog +--------- 2.2.1 ..... diff --git a/airflow/providers/jdbc/CHANGELOG.rst b/airflow/providers/jdbc/CHANGELOG.rst index a8a079568ba8..c6c41c8d58bb 100644 --- a/airflow/providers/jdbc/CHANGELOG.rst +++ b/airflow/providers/jdbc/CHANGELOG.rst @@ -17,12 +17,14 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-jdbc`` ------------------------------------------------ +``apache-airflow-providers-jdbc`` + +Changelog +--------- 4.0.0 ..... diff --git a/airflow/providers/jenkins/CHANGELOG.rst b/airflow/providers/jenkins/CHANGELOG.rst index 1f9328f892aa..0e396f9269c0 100644 --- a/airflow/providers/jenkins/CHANGELOG.rst +++ b/airflow/providers/jenkins/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-jenkins`` --------------------------------------------------- +``apache-airflow-providers-jenkins`` + + +Changelog +--------- 3.3.1 ..... diff --git a/airflow/providers/microsoft/azure/CHANGELOG.rst b/airflow/providers/microsoft/azure/CHANGELOG.rst index 223aea264997..d82b4c128941 100644 --- a/airflow/providers/microsoft/azure/CHANGELOG.rst +++ b/airflow/providers/microsoft/azure/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-microsoft-azure`` ----------------------------------------------------------- +``apache-airflow-providers-microsoft-azure`` + + +Changelog +--------- 6.2.0 ..... diff --git a/airflow/providers/microsoft/mssql/CHANGELOG.rst b/airflow/providers/microsoft/mssql/CHANGELOG.rst index e161e6fa33e9..adf40ee6cbe7 100644 --- a/airflow/providers/microsoft/mssql/CHANGELOG.rst +++ b/airflow/providers/microsoft/mssql/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-microsoft-mssql`` ----------------------------------------------------------- +``apache-airflow-providers-microsoft-mssql`` + + +Changelog +--------- 3.4.1 ..... diff --git a/airflow/providers/microsoft/psrp/CHANGELOG.rst b/airflow/providers/microsoft/psrp/CHANGELOG.rst index f4fcf01bd10f..cad9578e332c 100644 --- a/airflow/providers/microsoft/psrp/CHANGELOG.rst +++ b/airflow/providers/microsoft/psrp/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-microsoft-psrp`` ---------------------------------------------------------- +``apache-airflow-providers-microsoft-psrp`` + + +Changelog +--------- 2.3.1 ..... diff --git a/airflow/providers/microsoft/winrm/CHANGELOG.rst b/airflow/providers/microsoft/winrm/CHANGELOG.rst index e30d84633268..19fef6082df0 100644 --- a/airflow/providers/microsoft/winrm/CHANGELOG.rst +++ b/airflow/providers/microsoft/winrm/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-microsoft-winrm`` ----------------------------------------------------------- +``apache-airflow-providers-microsoft-winrm`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/mongo/CHANGELOG.rst b/airflow/providers/mongo/CHANGELOG.rst index 2b07ba58d548..730b5286464a 100644 --- a/airflow/providers/mongo/CHANGELOG.rst +++ b/airflow/providers/mongo/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-mongo`` ------------------------------------------------- +``apache-airflow-providers-mongo`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/mysql/CHANGELOG.rst b/airflow/providers/mysql/CHANGELOG.rst index ec482a62ec96..c88a865f7113 100644 --- a/airflow/providers/mysql/CHANGELOG.rst +++ b/airflow/providers/mysql/CHANGELOG.rst @@ -16,12 +16,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-mysql`` ------------------------------------------------- +``apache-airflow-providers-mysql`` + + +Changelog +--------- 5.1.1 ..... diff --git a/airflow/providers/neo4j/CHANGELOG.rst b/airflow/providers/neo4j/CHANGELOG.rst index bf1ac7044607..57bbc05e1378 100644 --- a/airflow/providers/neo4j/CHANGELOG.rst +++ b/airflow/providers/neo4j/CHANGELOG.rst @@ -18,12 +18,16 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-neo4j`` ------------------------------------------------- + +``apache-airflow-providers-neo4j`` + + +Changelog +--------- 3.3.1 ..... diff --git a/airflow/providers/odbc/CHANGELOG.rst b/airflow/providers/odbc/CHANGELOG.rst index 9e2495ef292b..ae5cfcd35bb9 100644 --- a/airflow/providers/odbc/CHANGELOG.rst +++ b/airflow/providers/odbc/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-odbc`` ------------------------------------------------ +``apache-airflow-providers-odbc`` + + +Changelog +--------- 4.0.0 ..... diff --git a/airflow/providers/openfaas/CHANGELOG.rst b/airflow/providers/openfaas/CHANGELOG.rst index 5213b63000c8..ce7ee1fa4856 100644 --- a/airflow/providers/openfaas/CHANGELOG.rst +++ b/airflow/providers/openfaas/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-openfaas`` ---------------------------------------------------- +``apache-airflow-providers-openfaas`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/openlineage/CHANGELOG.rst b/airflow/providers/openlineage/CHANGELOG.rst index c54c6ebc0580..a8494511a54f 100644 --- a/airflow/providers/openlineage/CHANGELOG.rst +++ b/airflow/providers/openlineage/CHANGELOG.rst @@ -16,12 +16,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-openlineage`` ------------------------------------------------------- +``apache-airflow-providers-openlineage`` + + +Changelog +--------- 1.0.0 ..... diff --git a/airflow/providers/opsgenie/CHANGELOG.rst b/airflow/providers/opsgenie/CHANGELOG.rst index 69fd9383045e..3e29953b9a39 100644 --- a/airflow/providers/opsgenie/CHANGELOG.rst +++ b/airflow/providers/opsgenie/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-opsgenie`` ---------------------------------------------------- +``apache-airflow-providers-opsgenie`` + + +Changelog +--------- 5.1.1 ..... @@ -69,7 +72,7 @@ Misc Breaking changes ~~~~~~~~~~~~~~~~ -Remove 'OpsgenieAlertOperator' also removed hooks.opsgenie_alert path +Remove ``OpsgenieAlertOperator`` also removed hooks.opsgenie_alert path * ``Remove deprecated code from Opsgenie provider (#27252)`` diff --git a/airflow/providers/oracle/CHANGELOG.rst b/airflow/providers/oracle/CHANGELOG.rst index bc7e86d76c0b..c351ba422391 100644 --- a/airflow/providers/oracle/CHANGELOG.rst +++ b/airflow/providers/oracle/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-oracle`` -------------------------------------------------- +``apache-airflow-providers-oracle`` + + +Changelog +--------- 3.7.1 ..... diff --git a/airflow/providers/pagerduty/CHANGELOG.rst b/airflow/providers/pagerduty/CHANGELOG.rst index cddb9e34ee3d..27bdf89a56d6 100644 --- a/airflow/providers/pagerduty/CHANGELOG.rst +++ b/airflow/providers/pagerduty/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-pagerduty`` ----------------------------------------------------- +``apache-airflow-providers-pagerduty`` + + +Changelog +--------- 3.3.0 ..... diff --git a/airflow/providers/papermill/CHANGELOG.rst b/airflow/providers/papermill/CHANGELOG.rst index 933b456a49f4..1805d8aa6ded 100644 --- a/airflow/providers/papermill/CHANGELOG.rst +++ b/airflow/providers/papermill/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-papermill`` ----------------------------------------------------- +``apache-airflow-providers-papermill`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/plexus/CHANGELOG.rst b/airflow/providers/plexus/CHANGELOG.rst index d9ed577d220c..9d53bf994e09 100644 --- a/airflow/providers/plexus/CHANGELOG.rst +++ b/airflow/providers/plexus/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-plexus`` -------------------------------------------------- +``apache-airflow-providers-plexus`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/postgres/CHANGELOG.rst b/airflow/providers/postgres/CHANGELOG.rst index 46f391a0fad4..c86c013a64d6 100644 --- a/airflow/providers/postgres/CHANGELOG.rst +++ b/airflow/providers/postgres/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-postgres`` ---------------------------------------------------- +``apache-airflow-providers-postgres`` + + +Changelog +--------- 5.5.1 ..... diff --git a/airflow/providers/presto/CHANGELOG.rst b/airflow/providers/presto/CHANGELOG.rst index d35e42c51847..108bd30fda13 100644 --- a/airflow/providers/presto/CHANGELOG.rst +++ b/airflow/providers/presto/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-presto`` -------------------------------------------------- +``apache-airflow-providers-presto`` + + +Changelog +--------- 5.1.1 ..... diff --git a/airflow/providers/qubole/CHANGELOG.rst b/airflow/providers/qubole/CHANGELOG.rst index c8c7c76aa9dc..80dfff0f1699 100644 --- a/airflow/providers/qubole/CHANGELOG.rst +++ b/airflow/providers/qubole/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-qubole`` -------------------------------------------------- +``apache-airflow-providers-qubole`` + + +Changelog +--------- 3.4.1 ..... diff --git a/airflow/providers/redis/CHANGELOG.rst b/airflow/providers/redis/CHANGELOG.rst index 87b538ec7c07..2aab210268c8 100644 --- a/airflow/providers/redis/CHANGELOG.rst +++ b/airflow/providers/redis/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-redis`` ------------------------------------------------- +``apache-airflow-providers-redis`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/salesforce/CHANGELOG.rst b/airflow/providers/salesforce/CHANGELOG.rst index 78505ad7c870..319f82801a79 100644 --- a/airflow/providers/salesforce/CHANGELOG.rst +++ b/airflow/providers/salesforce/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-salesforce`` ------------------------------------------------------ +``apache-airflow-providers-salesforce`` + + +Changelog +--------- 5.4.1 ..... diff --git a/airflow/providers/samba/CHANGELOG.rst b/airflow/providers/samba/CHANGELOG.rst index 190b1f73f77b..28803623525c 100644 --- a/airflow/providers/samba/CHANGELOG.rst +++ b/airflow/providers/samba/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-samba`` ------------------------------------------------- +``apache-airflow-providers-samba`` + + +Changelog +--------- 4.2.1 ..... diff --git a/airflow/providers/segment/CHANGELOG.rst b/airflow/providers/segment/CHANGELOG.rst index 059ae0472def..b1856f183402 100644 --- a/airflow/providers/segment/CHANGELOG.rst +++ b/airflow/providers/segment/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-segment`` --------------------------------------------------- +``apache-airflow-providers-segment`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/sendgrid/CHANGELOG.rst b/airflow/providers/sendgrid/CHANGELOG.rst index 7e677678f5db..d49504984e8a 100644 --- a/airflow/providers/sendgrid/CHANGELOG.rst +++ b/airflow/providers/sendgrid/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-sendgrid`` ---------------------------------------------------- +``apache-airflow-providers-sendgrid`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/sftp/CHANGELOG.rst b/airflow/providers/sftp/CHANGELOG.rst index dcea4aad3171..951358a53b76 100644 --- a/airflow/providers/sftp/CHANGELOG.rst +++ b/airflow/providers/sftp/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-sftp`` ------------------------------------------------ +``apache-airflow-providers-sftp`` + + +Changelog +--------- 4.3.1 ..... diff --git a/airflow/providers/singularity/CHANGELOG.rst b/airflow/providers/singularity/CHANGELOG.rst index 1ad3b4bc1d54..0a543fa9f849 100644 --- a/airflow/providers/singularity/CHANGELOG.rst +++ b/airflow/providers/singularity/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-singularity`` ------------------------------------------------------- +``apache-airflow-providers-singularity`` + + +Changelog +--------- 3.2.1 ..... diff --git a/airflow/providers/slack/CHANGELOG.rst b/airflow/providers/slack/CHANGELOG.rst index ca1dc3895dfe..e43bbabb679a 100644 --- a/airflow/providers/slack/CHANGELOG.rst +++ b/airflow/providers/slack/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-slack`` ------------------------------------------------- +``apache-airflow-providers-slack`` + + +Changelog +--------- 7.3.1 ..... diff --git a/airflow/providers/smtp/CHANGELOG.rst b/airflow/providers/smtp/CHANGELOG.rst index 103e9a73c349..6e0a955e56b9 100644 --- a/airflow/providers/smtp/CHANGELOG.rst +++ b/airflow/providers/smtp/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-smtp`` ------------------------------------------------ +``apache-airflow-providers-smtp`` + + +Changelog +--------- 1.2.0 ..... diff --git a/airflow/providers/snowflake/CHANGELOG.rst b/airflow/providers/snowflake/CHANGELOG.rst index ee9fca09d8e4..2fa9008d8a9a 100644 --- a/airflow/providers/snowflake/CHANGELOG.rst +++ b/airflow/providers/snowflake/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-snowflake`` ----------------------------------------------------- +``apache-airflow-providers-snowflake`` + + +Changelog +--------- 4.2.0 ..... diff --git a/airflow/providers/sqlite/CHANGELOG.rst b/airflow/providers/sqlite/CHANGELOG.rst index f84ff50a8555..10b34065369e 100644 --- a/airflow/providers/sqlite/CHANGELOG.rst +++ b/airflow/providers/sqlite/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-sqlite`` -------------------------------------------------- +``apache-airflow-providers-sqlite`` + + +Changelog +--------- 3.4.2 ..... diff --git a/airflow/providers/ssh/CHANGELOG.rst b/airflow/providers/ssh/CHANGELOG.rst index 9accc14048aa..b434c6997c14 100644 --- a/airflow/providers/ssh/CHANGELOG.rst +++ b/airflow/providers/ssh/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-ssh`` ----------------------------------------------- +``apache-airflow-providers-ssh`` + + +Changelog +--------- 3.7.1 ..... diff --git a/airflow/providers/tableau/CHANGELOG.rst b/airflow/providers/tableau/CHANGELOG.rst index 41af7710de9b..1cd1289890cc 100644 --- a/airflow/providers/tableau/CHANGELOG.rst +++ b/airflow/providers/tableau/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-tableau`` --------------------------------------------------- +``apache-airflow-providers-tableau`` + + +Changelog +--------- 4.2.1 ..... diff --git a/airflow/providers/tabular/CHANGELOG.rst b/airflow/providers/tabular/CHANGELOG.rst index 9db36926d24e..5ebb94af1b44 100644 --- a/airflow/providers/tabular/CHANGELOG.rst +++ b/airflow/providers/tabular/CHANGELOG.rst @@ -16,12 +16,15 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-tabular`` --------------------------------------------------- +``apache-airflow-providers-tabular`` + + +Changelog +--------- 1.2.1 ..... diff --git a/airflow/providers/telegram/CHANGELOG.rst b/airflow/providers/telegram/CHANGELOG.rst index dcd21d242ff1..00ee3cc07409 100644 --- a/airflow/providers/telegram/CHANGELOG.rst +++ b/airflow/providers/telegram/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-telegram`` ---------------------------------------------------- +``apache-airflow-providers-telegram`` + + +Changelog +--------- 4.1.1 ..... diff --git a/airflow/providers/trino/CHANGELOG.rst b/airflow/providers/trino/CHANGELOG.rst index da3fd0695c17..3b6ee23a3054 100644 --- a/airflow/providers/trino/CHANGELOG.rst +++ b/airflow/providers/trino/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-trino`` ------------------------------------------------- +``apache-airflow-providers-trino`` + + +Changelog +--------- 5.2.0 ..... diff --git a/airflow/providers/vertica/CHANGELOG.rst b/airflow/providers/vertica/CHANGELOG.rst index 36e06b2cf5a6..7793a3dedeb2 100644 --- a/airflow/providers/vertica/CHANGELOG.rst +++ b/airflow/providers/vertica/CHANGELOG.rst @@ -17,12 +17,16 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-vertica`` --------------------------------------------------- +``apache-airflow-providers-vertica`` + + + +Changelog +--------- 3.5.0 ..... diff --git a/airflow/providers/yandex/CHANGELOG.rst b/airflow/providers/yandex/CHANGELOG.rst index 8eee1ab6c99e..03b7ee0f9c1e 100644 --- a/airflow/providers/yandex/CHANGELOG.rst +++ b/airflow/providers/yandex/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-yandex`` -------------------------------------------------- +``apache-airflow-providers-yandex`` + + +Changelog +--------- .. note:: This release dropped support for Python 3.7 diff --git a/airflow/providers/zendesk/CHANGELOG.rst b/airflow/providers/zendesk/CHANGELOG.rst index 3c9f67d9d042..09dc18fc77c0 100644 --- a/airflow/providers/zendesk/CHANGELOG.rst +++ b/airflow/providers/zendesk/CHANGELOG.rst @@ -17,12 +17,15 @@ .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there are some breaking changes + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. -Changelog for ``apache-airflow-providers-zendex`` -------------------------------------------------- +``apache-airflow-providers-zendesk`` + + +Changelog +--------- 4.3.1 ..... diff --git a/dev/provider_packages/PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2 b/dev/provider_packages/PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2 index 81d0e42a23ab..3ace4d579405 100644 --- a/dev/provider_packages/PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2 +++ b/dev/provider_packages/PROVIDER_CHANGELOG_TEMPLATE.rst.jinja2 @@ -40,7 +40,4 @@ specific language governing permissions and limitations under the License. -Package ``{{ PACKAGE_PIP_NAME }}`` -{{ '-' * (12 + PACKAGE_PIP_NAME | length) }} - .. include:: {{ CHANGELOG_RELATIVE_PATH }}/CHANGELOG.rst diff --git a/dev/provider_packages/prepare_provider_packages.py b/dev/provider_packages/prepare_provider_packages.py index cde0918381cf..82f3de998880 100755 --- a/dev/provider_packages/prepare_provider_packages.py +++ b/dev/provider_packages/prepare_provider_packages.py @@ -80,13 +80,14 @@ under the License. .. NOTE TO CONTRIBUTORS: - Please, only add notes to the Changelog just below the "Changelog for ..." header when there + Please, only add notes to the Changelog just below the "Changelog" header when there are some breaking changes and you want to add an explanation to the users on how they are supposed to deal with them. The changelog is updated and maintained semi-automatically by release manager. +``{{ package_name }}`` -Changelog for ``{{ package_name }}`` -{{ '-' * (18 + package_name | length) }} +Changelog +--------- 1.0.0 ..... diff --git a/docs/apache-airflow-providers-airbyte/changelog.rst b/docs/apache-airflow-providers-airbyte/changelog.rst index df72f14d050e..57b48befaed9 100644 --- a/docs/apache-airflow-providers-airbyte/changelog.rst +++ b/docs/apache-airflow-providers-airbyte/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-airbyte`` --------------------------------------------- + .. include:: ../../airflow/providers/airbyte/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-alibaba/changelog.rst b/docs/apache-airflow-providers-alibaba/changelog.rst index c032cb7d6a48..c4368e7ddbca 100644 --- a/docs/apache-airflow-providers-alibaba/changelog.rst +++ b/docs/apache-airflow-providers-alibaba/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-alibaba`` --------------------------------------------- + .. include:: ../../airflow/providers/alibaba/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-amazon/changelog.rst b/docs/apache-airflow-providers-amazon/changelog.rst index aed399da661e..ed6e5a92f2f7 100644 --- a/docs/apache-airflow-providers-amazon/changelog.rst +++ b/docs/apache-airflow-providers-amazon/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-amazon`` -------------------------------------------- + .. include:: ../../airflow/providers/amazon/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-beam/changelog.rst b/docs/apache-airflow-providers-apache-beam/changelog.rst index f8bf09df1c3a..aa8651cc9f41 100644 --- a/docs/apache-airflow-providers-apache-beam/changelog.rst +++ b/docs/apache-airflow-providers-apache-beam/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-beam`` ------------------------------------------------- + .. include:: ../../airflow/providers/apache/beam/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-cassandra/changelog.rst b/docs/apache-airflow-providers-apache-cassandra/changelog.rst index ff78c1158e3e..c792cbe570c7 100644 --- a/docs/apache-airflow-providers-apache-cassandra/changelog.rst +++ b/docs/apache-airflow-providers-apache-cassandra/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-cassandra`` ------------------------------------------------------ + .. include:: ../../airflow/providers/apache/cassandra/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-drill/changelog.rst b/docs/apache-airflow-providers-apache-drill/changelog.rst index d1cdd2f3040d..38e2ce1ea797 100644 --- a/docs/apache-airflow-providers-apache-drill/changelog.rst +++ b/docs/apache-airflow-providers-apache-drill/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-drill`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/drill/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-druid/changelog.rst b/docs/apache-airflow-providers-apache-druid/changelog.rst index a77d36598a8e..3e7dad97ff5f 100644 --- a/docs/apache-airflow-providers-apache-druid/changelog.rst +++ b/docs/apache-airflow-providers-apache-druid/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-druid`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/druid/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-flink/changelog.rst b/docs/apache-airflow-providers-apache-flink/changelog.rst index 5738f55e53d2..8b3fc75fa18d 100644 --- a/docs/apache-airflow-providers-apache-flink/changelog.rst +++ b/docs/apache-airflow-providers-apache-flink/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-flink`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/flink/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-hdfs/changelog.rst b/docs/apache-airflow-providers-apache-hdfs/changelog.rst index 85395a91c919..234612271e3f 100644 --- a/docs/apache-airflow-providers-apache-hdfs/changelog.rst +++ b/docs/apache-airflow-providers-apache-hdfs/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-hdfs`` ------------------------------------------------- + .. include:: ../../airflow/providers/apache/hdfs/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-hive/changelog.rst b/docs/apache-airflow-providers-apache-hive/changelog.rst index d93dba54e397..d8ad824aff3d 100644 --- a/docs/apache-airflow-providers-apache-hive/changelog.rst +++ b/docs/apache-airflow-providers-apache-hive/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-hive`` ------------------------------------------------- + .. include:: ../../airflow/providers/apache/hive/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-impala/changelog.rst b/docs/apache-airflow-providers-apache-impala/changelog.rst index 5f504622be8a..0f07c87feef3 100644 --- a/docs/apache-airflow-providers-apache-impala/changelog.rst +++ b/docs/apache-airflow-providers-apache-impala/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-impala`` --------------------------------------------------- + .. include:: ../../airflow/providers/apache/impala/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-kafka/changelog.rst b/docs/apache-airflow-providers-apache-kafka/changelog.rst index b2b294b36cc4..b1dafd854b3e 100644 --- a/docs/apache-airflow-providers-apache-kafka/changelog.rst +++ b/docs/apache-airflow-providers-apache-kafka/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-kafka`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/kafka/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-kylin/changelog.rst b/docs/apache-airflow-providers-apache-kylin/changelog.rst index 30e11d3bde01..b67c0f97e573 100644 --- a/docs/apache-airflow-providers-apache-kylin/changelog.rst +++ b/docs/apache-airflow-providers-apache-kylin/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-kylin`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/kylin/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-livy/changelog.rst b/docs/apache-airflow-providers-apache-livy/changelog.rst index 1e0af37c44c6..05f40d97d28b 100644 --- a/docs/apache-airflow-providers-apache-livy/changelog.rst +++ b/docs/apache-airflow-providers-apache-livy/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-livy`` ------------------------------------------------- + .. include:: ../../airflow/providers/apache/livy/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-pig/changelog.rst b/docs/apache-airflow-providers-apache-pig/changelog.rst index 355f67835bf7..f1cc7943293d 100644 --- a/docs/apache-airflow-providers-apache-pig/changelog.rst +++ b/docs/apache-airflow-providers-apache-pig/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-pig`` ------------------------------------------------ + .. include:: ../../airflow/providers/apache/pig/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-pinot/changelog.rst b/docs/apache-airflow-providers-apache-pinot/changelog.rst index 4e510c766ef0..55e366743c38 100644 --- a/docs/apache-airflow-providers-apache-pinot/changelog.rst +++ b/docs/apache-airflow-providers-apache-pinot/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-pinot`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/pinot/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-spark/changelog.rst b/docs/apache-airflow-providers-apache-spark/changelog.rst index 9817d195020d..1a8c7b1c3ea1 100644 --- a/docs/apache-airflow-providers-apache-spark/changelog.rst +++ b/docs/apache-airflow-providers-apache-spark/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-spark`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/spark/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apache-sqoop/changelog.rst b/docs/apache-airflow-providers-apache-sqoop/changelog.rst index b17af28da0e9..3e7b24a33909 100644 --- a/docs/apache-airflow-providers-apache-sqoop/changelog.rst +++ b/docs/apache-airflow-providers-apache-sqoop/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apache-sqoop`` -------------------------------------------------- + .. include:: ../../airflow/providers/apache/sqoop/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-apprise/changelog.rst b/docs/apache-airflow-providers-apprise/changelog.rst index 3dc9d66f9f63..75d15ffd0fb0 100644 --- a/docs/apache-airflow-providers-apprise/changelog.rst +++ b/docs/apache-airflow-providers-apprise/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-apprise`` --------------------------------------------- + .. include:: ../../airflow/providers/apprise/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-arangodb/changelog.rst b/docs/apache-airflow-providers-arangodb/changelog.rst index 53d58b062323..bb63458926a9 100644 --- a/docs/apache-airflow-providers-arangodb/changelog.rst +++ b/docs/apache-airflow-providers-arangodb/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-arangodb`` ---------------------------------------------- + .. include:: ../../airflow/providers/arangodb/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-asana/changelog.rst b/docs/apache-airflow-providers-asana/changelog.rst index f27de03b77d9..eba1b4ee7c6d 100644 --- a/docs/apache-airflow-providers-asana/changelog.rst +++ b/docs/apache-airflow-providers-asana/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-asana`` ------------------------------------------- + .. include:: ../../airflow/providers/asana/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-atlassian-jira/changelog.rst b/docs/apache-airflow-providers-atlassian-jira/changelog.rst index f5b77baa5dc9..30962a6d1e0c 100644 --- a/docs/apache-airflow-providers-atlassian-jira/changelog.rst +++ b/docs/apache-airflow-providers-atlassian-jira/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-atlassian-jira`` ---------------------------------------------------- + .. include:: ../../airflow/providers/atlassian/jira/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-celery/changelog.rst b/docs/apache-airflow-providers-celery/changelog.rst index 3a3395be9862..301ee64f5426 100644 --- a/docs/apache-airflow-providers-celery/changelog.rst +++ b/docs/apache-airflow-providers-celery/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-celery`` -------------------------------------------- + .. include:: ../../airflow/providers/celery/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-cloudant/changelog.rst b/docs/apache-airflow-providers-cloudant/changelog.rst index d92c198cfced..14c1a3715a4a 100644 --- a/docs/apache-airflow-providers-cloudant/changelog.rst +++ b/docs/apache-airflow-providers-cloudant/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-cloudant`` ---------------------------------------------- + .. include:: ../../airflow/providers/cloudant/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst b/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst index 284f10248847..758cd4c9abc7 100644 --- a/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst +++ b/docs/apache-airflow-providers-cncf-kubernetes/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-cncf-kubernetes`` ----------------------------------------------------- + .. include:: ../../airflow/providers/cncf/kubernetes/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-common-sql/changelog.rst b/docs/apache-airflow-providers-common-sql/changelog.rst index 2b8d6ae7e899..be8dae1c039b 100644 --- a/docs/apache-airflow-providers-common-sql/changelog.rst +++ b/docs/apache-airflow-providers-common-sql/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-common-sql`` ------------------------------------------------ + .. include:: ../../airflow/providers/common/sql/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-databricks/changelog.rst b/docs/apache-airflow-providers-databricks/changelog.rst index 9a262c2567c0..6c5cde733297 100644 --- a/docs/apache-airflow-providers-databricks/changelog.rst +++ b/docs/apache-airflow-providers-databricks/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-databricks`` ------------------------------------------------ + .. include:: ../../airflow/providers/databricks/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-datadog/changelog.rst b/docs/apache-airflow-providers-datadog/changelog.rst index 99ff68850f81..125edeffee29 100644 --- a/docs/apache-airflow-providers-datadog/changelog.rst +++ b/docs/apache-airflow-providers-datadog/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-datadog`` --------------------------------------------- + .. include:: ../../airflow/providers/datadog/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-dbt-cloud/changelog.rst b/docs/apache-airflow-providers-dbt-cloud/changelog.rst index 7d262eb397b0..ab244b93a10e 100644 --- a/docs/apache-airflow-providers-dbt-cloud/changelog.rst +++ b/docs/apache-airflow-providers-dbt-cloud/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-dbt-cloud`` ----------------------------------------------- + .. include:: ../../airflow/providers/dbt/cloud/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-dingding/changelog.rst b/docs/apache-airflow-providers-dingding/changelog.rst index 272bd9d017e0..6303afaa103e 100644 --- a/docs/apache-airflow-providers-dingding/changelog.rst +++ b/docs/apache-airflow-providers-dingding/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-dingding`` ---------------------------------------------- + .. include:: ../../airflow/providers/dingding/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-discord/changelog.rst b/docs/apache-airflow-providers-discord/changelog.rst index d26072461d1d..c9cfb5499faf 100644 --- a/docs/apache-airflow-providers-discord/changelog.rst +++ b/docs/apache-airflow-providers-discord/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-discord`` --------------------------------------------- + .. include:: ../../airflow/providers/discord/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-docker/changelog.rst b/docs/apache-airflow-providers-docker/changelog.rst index d92635e586b2..606d636484a0 100644 --- a/docs/apache-airflow-providers-docker/changelog.rst +++ b/docs/apache-airflow-providers-docker/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-docker`` -------------------------------------------- + .. include:: ../../airflow/providers/docker/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-elasticsearch/changelog.rst b/docs/apache-airflow-providers-elasticsearch/changelog.rst index 287030242def..78aa32de23aa 100644 --- a/docs/apache-airflow-providers-elasticsearch/changelog.rst +++ b/docs/apache-airflow-providers-elasticsearch/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-elasticsearch`` --------------------------------------------------- + .. include:: ../../airflow/providers/elasticsearch/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-exasol/changelog.rst b/docs/apache-airflow-providers-exasol/changelog.rst index 4dc893b56615..947bf8688e2f 100644 --- a/docs/apache-airflow-providers-exasol/changelog.rst +++ b/docs/apache-airflow-providers-exasol/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-exasol`` -------------------------------------------- + .. include:: ../../airflow/providers/exasol/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-facebook/changelog.rst b/docs/apache-airflow-providers-facebook/changelog.rst index d58f217b6488..62145f4fa244 100644 --- a/docs/apache-airflow-providers-facebook/changelog.rst +++ b/docs/apache-airflow-providers-facebook/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-facebook`` ---------------------------------------------- + .. include:: ../../airflow/providers/facebook/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-ftp/changelog.rst b/docs/apache-airflow-providers-ftp/changelog.rst index 6b7a72cefbe5..6dd0771a976b 100644 --- a/docs/apache-airflow-providers-ftp/changelog.rst +++ b/docs/apache-airflow-providers-ftp/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-ftp`` ----------------------------------------- + .. include:: ../../airflow/providers/ftp/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-github/changelog.rst b/docs/apache-airflow-providers-github/changelog.rst index 964dfec0fd24..eb1c4bfc5e00 100644 --- a/docs/apache-airflow-providers-github/changelog.rst +++ b/docs/apache-airflow-providers-github/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-github`` -------------------------------------------- + .. include:: ../../airflow/providers/github/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-google/changelog.rst b/docs/apache-airflow-providers-google/changelog.rst index abda3f330d61..f3c708e00f11 100644 --- a/docs/apache-airflow-providers-google/changelog.rst +++ b/docs/apache-airflow-providers-google/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-google`` -------------------------------------------- + .. include:: ../../airflow/providers/google/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-grpc/changelog.rst b/docs/apache-airflow-providers-grpc/changelog.rst index 051d23b8a818..55e970f047ca 100644 --- a/docs/apache-airflow-providers-grpc/changelog.rst +++ b/docs/apache-airflow-providers-grpc/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-grpc`` ------------------------------------------ + .. include:: ../../airflow/providers/grpc/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-hashicorp/changelog.rst b/docs/apache-airflow-providers-hashicorp/changelog.rst index 7ba4a83c0aa9..10da504065ad 100644 --- a/docs/apache-airflow-providers-hashicorp/changelog.rst +++ b/docs/apache-airflow-providers-hashicorp/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-hashicorp`` ----------------------------------------------- + .. include:: ../../airflow/providers/hashicorp/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-http/changelog.rst b/docs/apache-airflow-providers-http/changelog.rst index 30efed25e3a3..ae63287e513b 100644 --- a/docs/apache-airflow-providers-http/changelog.rst +++ b/docs/apache-airflow-providers-http/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-http`` ------------------------------------------ + .. include:: ../../airflow/providers/http/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-imap/changelog.rst b/docs/apache-airflow-providers-imap/changelog.rst index b635271658f9..cd51a7872892 100644 --- a/docs/apache-airflow-providers-imap/changelog.rst +++ b/docs/apache-airflow-providers-imap/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-imap`` ------------------------------------------ + .. include:: ../../airflow/providers/imap/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-influxdb/changelog.rst b/docs/apache-airflow-providers-influxdb/changelog.rst index 2c32bdae5a4a..1cd3fad0746b 100644 --- a/docs/apache-airflow-providers-influxdb/changelog.rst +++ b/docs/apache-airflow-providers-influxdb/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-influxdb`` ---------------------------------------------- + .. include:: ../../airflow/providers/influxdb/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-jdbc/changelog.rst b/docs/apache-airflow-providers-jdbc/changelog.rst index e8ff084db25d..00fe21958583 100644 --- a/docs/apache-airflow-providers-jdbc/changelog.rst +++ b/docs/apache-airflow-providers-jdbc/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-jdbc`` ------------------------------------------ + .. include:: ../../airflow/providers/jdbc/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-jenkins/changelog.rst b/docs/apache-airflow-providers-jenkins/changelog.rst index c8b1b0600a78..fab80a472c16 100644 --- a/docs/apache-airflow-providers-jenkins/changelog.rst +++ b/docs/apache-airflow-providers-jenkins/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-jenkins`` --------------------------------------------- + .. include:: ../../airflow/providers/jenkins/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-microsoft-azure/changelog.rst b/docs/apache-airflow-providers-microsoft-azure/changelog.rst index 5bf8a1b215de..11ea0a86b489 100644 --- a/docs/apache-airflow-providers-microsoft-azure/changelog.rst +++ b/docs/apache-airflow-providers-microsoft-azure/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-microsoft-azure`` ----------------------------------------------------- + .. include:: ../../airflow/providers/microsoft/azure/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-microsoft-mssql/changelog.rst b/docs/apache-airflow-providers-microsoft-mssql/changelog.rst index 739c6b8b62a7..7d66f519f89b 100644 --- a/docs/apache-airflow-providers-microsoft-mssql/changelog.rst +++ b/docs/apache-airflow-providers-microsoft-mssql/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-microsoft-mssql`` ----------------------------------------------------- + .. include:: ../../airflow/providers/microsoft/mssql/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-microsoft-psrp/changelog.rst b/docs/apache-airflow-providers-microsoft-psrp/changelog.rst index 07d8e032d52a..6a6ba838519a 100644 --- a/docs/apache-airflow-providers-microsoft-psrp/changelog.rst +++ b/docs/apache-airflow-providers-microsoft-psrp/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-microsoft-psrp`` ---------------------------------------------------- + .. include:: ../../airflow/providers/microsoft/psrp/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-microsoft-winrm/changelog.rst b/docs/apache-airflow-providers-microsoft-winrm/changelog.rst index 201bbe2df89b..23787e7a7a9d 100644 --- a/docs/apache-airflow-providers-microsoft-winrm/changelog.rst +++ b/docs/apache-airflow-providers-microsoft-winrm/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-microsoft-winrm`` ----------------------------------------------------- + .. include:: ../../airflow/providers/microsoft/winrm/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-mongo/changelog.rst b/docs/apache-airflow-providers-mongo/changelog.rst index 97e1e6c55d5b..2df335624949 100644 --- a/docs/apache-airflow-providers-mongo/changelog.rst +++ b/docs/apache-airflow-providers-mongo/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-mongo`` ------------------------------------------- + .. include:: ../../airflow/providers/mongo/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-mysql/changelog.rst b/docs/apache-airflow-providers-mysql/changelog.rst index 7221fd9fc662..e98c8b8a24ed 100644 --- a/docs/apache-airflow-providers-mysql/changelog.rst +++ b/docs/apache-airflow-providers-mysql/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-mysql`` ------------------------------------------- + .. include:: ../../airflow/providers/mysql/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-neo4j/changelog.rst b/docs/apache-airflow-providers-neo4j/changelog.rst index dd14f25ba628..15a9791980c1 100644 --- a/docs/apache-airflow-providers-neo4j/changelog.rst +++ b/docs/apache-airflow-providers-neo4j/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-neo4j`` ------------------------------------------- + .. include:: ../../airflow/providers/neo4j/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-odbc/changelog.rst b/docs/apache-airflow-providers-odbc/changelog.rst index 58166fe379ab..22cc2b961fcf 100644 --- a/docs/apache-airflow-providers-odbc/changelog.rst +++ b/docs/apache-airflow-providers-odbc/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-odbc`` ------------------------------------------ + .. include:: ../../airflow/providers/odbc/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-openfaas/changelog.rst b/docs/apache-airflow-providers-openfaas/changelog.rst index 9a2ff34f4895..0881442ad18b 100644 --- a/docs/apache-airflow-providers-openfaas/changelog.rst +++ b/docs/apache-airflow-providers-openfaas/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-openfaas`` ---------------------------------------------- + .. include:: ../../airflow/providers/openfaas/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-openlineage/changelog.rst b/docs/apache-airflow-providers-openlineage/changelog.rst index 8d5ab5278a60..7034f1b9e965 100644 --- a/docs/apache-airflow-providers-openlineage/changelog.rst +++ b/docs/apache-airflow-providers-openlineage/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-openlineage`` ------------------------------------------------- + .. include:: ../../airflow/providers/openlineage/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-opsgenie/changelog.rst b/docs/apache-airflow-providers-opsgenie/changelog.rst index 58a008331051..08f9e0ba2322 100644 --- a/docs/apache-airflow-providers-opsgenie/changelog.rst +++ b/docs/apache-airflow-providers-opsgenie/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-opsgenie`` ---------------------------------------------- + .. include:: ../../airflow/providers/opsgenie/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-oracle/changelog.rst b/docs/apache-airflow-providers-oracle/changelog.rst index ac5231a1d13b..6b05cb4fbf32 100644 --- a/docs/apache-airflow-providers-oracle/changelog.rst +++ b/docs/apache-airflow-providers-oracle/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-oracle`` -------------------------------------------- + .. include:: ../../airflow/providers/oracle/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-pagerduty/changelog.rst b/docs/apache-airflow-providers-pagerduty/changelog.rst index c463d381ad52..186bdeb52bac 100644 --- a/docs/apache-airflow-providers-pagerduty/changelog.rst +++ b/docs/apache-airflow-providers-pagerduty/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-pagerduty`` ----------------------------------------------- + .. include:: ../../airflow/providers/pagerduty/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-papermill/changelog.rst b/docs/apache-airflow-providers-papermill/changelog.rst index 4340d544f223..fc7fb2c86ec5 100644 --- a/docs/apache-airflow-providers-papermill/changelog.rst +++ b/docs/apache-airflow-providers-papermill/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-papermill`` ----------------------------------------------- + .. include:: ../../airflow/providers/papermill/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-plexus/changelog.rst b/docs/apache-airflow-providers-plexus/changelog.rst index 95c7f42b8d42..c9ccba5e642f 100644 --- a/docs/apache-airflow-providers-plexus/changelog.rst +++ b/docs/apache-airflow-providers-plexus/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-plexus`` -------------------------------------------- + .. include:: ../../airflow/providers/plexus/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-postgres/changelog.rst b/docs/apache-airflow-providers-postgres/changelog.rst index 6bbf23290870..4beb42c22798 100644 --- a/docs/apache-airflow-providers-postgres/changelog.rst +++ b/docs/apache-airflow-providers-postgres/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-postgres`` ---------------------------------------------- + .. include:: ../../airflow/providers/postgres/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-presto/changelog.rst b/docs/apache-airflow-providers-presto/changelog.rst index d1226474e2d5..0fb911447b88 100644 --- a/docs/apache-airflow-providers-presto/changelog.rst +++ b/docs/apache-airflow-providers-presto/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-presto`` -------------------------------------------- + .. include:: ../../airflow/providers/presto/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-qubole/changelog.rst b/docs/apache-airflow-providers-qubole/changelog.rst index 3566395943fe..5004b7cb1f2f 100644 --- a/docs/apache-airflow-providers-qubole/changelog.rst +++ b/docs/apache-airflow-providers-qubole/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-qubole`` -------------------------------------------- + .. include:: ../../airflow/providers/qubole/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-redis/changelog.rst b/docs/apache-airflow-providers-redis/changelog.rst index 8643b0621406..14e39687382f 100644 --- a/docs/apache-airflow-providers-redis/changelog.rst +++ b/docs/apache-airflow-providers-redis/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-redis`` ------------------------------------------- + .. include:: ../../airflow/providers/redis/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-salesforce/changelog.rst b/docs/apache-airflow-providers-salesforce/changelog.rst index a57d8371dfd8..aaa24b57c4a4 100644 --- a/docs/apache-airflow-providers-salesforce/changelog.rst +++ b/docs/apache-airflow-providers-salesforce/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-salesforce`` ------------------------------------------------ + .. include:: ../../airflow/providers/salesforce/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-samba/changelog.rst b/docs/apache-airflow-providers-samba/changelog.rst index c3a59099cfde..2d36cae155ee 100644 --- a/docs/apache-airflow-providers-samba/changelog.rst +++ b/docs/apache-airflow-providers-samba/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-samba`` ------------------------------------------- + .. include:: ../../airflow/providers/samba/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-segment/changelog.rst b/docs/apache-airflow-providers-segment/changelog.rst index 43e6324969ad..baca90b57523 100644 --- a/docs/apache-airflow-providers-segment/changelog.rst +++ b/docs/apache-airflow-providers-segment/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-segment`` --------------------------------------------- + .. include:: ../../airflow/providers/segment/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-sendgrid/changelog.rst b/docs/apache-airflow-providers-sendgrid/changelog.rst index 1963562bba92..ddb64b4ee8cd 100644 --- a/docs/apache-airflow-providers-sendgrid/changelog.rst +++ b/docs/apache-airflow-providers-sendgrid/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-sendgrid`` ---------------------------------------------- + .. include:: ../../airflow/providers/sendgrid/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-sftp/changelog.rst b/docs/apache-airflow-providers-sftp/changelog.rst index 3cab8a64516d..3f0ce545ca8c 100644 --- a/docs/apache-airflow-providers-sftp/changelog.rst +++ b/docs/apache-airflow-providers-sftp/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-sftp`` ------------------------------------------ + .. include:: ../../airflow/providers/sftp/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-singularity/changelog.rst b/docs/apache-airflow-providers-singularity/changelog.rst index 803e1201ab69..b8fd7c81138c 100644 --- a/docs/apache-airflow-providers-singularity/changelog.rst +++ b/docs/apache-airflow-providers-singularity/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-singularity`` ------------------------------------------------- + .. include:: ../../airflow/providers/singularity/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-slack/changelog.rst b/docs/apache-airflow-providers-slack/changelog.rst index cb130d2233cb..aa09de3fb0a5 100644 --- a/docs/apache-airflow-providers-slack/changelog.rst +++ b/docs/apache-airflow-providers-slack/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-slack`` ------------------------------------------- + .. include:: ../../airflow/providers/slack/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-smtp/changelog.rst b/docs/apache-airflow-providers-smtp/changelog.rst index e67bd0dcd652..d602eb0610a2 100644 --- a/docs/apache-airflow-providers-smtp/changelog.rst +++ b/docs/apache-airflow-providers-smtp/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-smtp`` ------------------------------------------ + .. include:: ../../airflow/providers/smtp/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-snowflake/changelog.rst b/docs/apache-airflow-providers-snowflake/changelog.rst index c3a0cdf18ef9..d0734ce6d7d6 100644 --- a/docs/apache-airflow-providers-snowflake/changelog.rst +++ b/docs/apache-airflow-providers-snowflake/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-snowflake`` ----------------------------------------------- + .. include:: ../../airflow/providers/snowflake/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-sqlite/changelog.rst b/docs/apache-airflow-providers-sqlite/changelog.rst index 99d35831de58..04dfe43c6020 100644 --- a/docs/apache-airflow-providers-sqlite/changelog.rst +++ b/docs/apache-airflow-providers-sqlite/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-sqlite`` -------------------------------------------- + .. include:: ../../airflow/providers/sqlite/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-ssh/changelog.rst b/docs/apache-airflow-providers-ssh/changelog.rst index d6ba0ea87983..2252e9bf0ce1 100644 --- a/docs/apache-airflow-providers-ssh/changelog.rst +++ b/docs/apache-airflow-providers-ssh/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-ssh`` ----------------------------------------- + .. include:: ../../airflow/providers/ssh/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-tableau/changelog.rst b/docs/apache-airflow-providers-tableau/changelog.rst index 0e60b899d701..39a7eec68ce4 100644 --- a/docs/apache-airflow-providers-tableau/changelog.rst +++ b/docs/apache-airflow-providers-tableau/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-tableau`` --------------------------------------------- + .. include:: ../../airflow/providers/tableau/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-tabular/changelog.rst b/docs/apache-airflow-providers-tabular/changelog.rst index 6257ac42c7f6..7c12450cea66 100644 --- a/docs/apache-airflow-providers-tabular/changelog.rst +++ b/docs/apache-airflow-providers-tabular/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-tabular`` --------------------------------------------- + .. include:: ../../airflow/providers/tabular/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-telegram/changelog.rst b/docs/apache-airflow-providers-telegram/changelog.rst index d4686e21b62d..86c7a9867260 100644 --- a/docs/apache-airflow-providers-telegram/changelog.rst +++ b/docs/apache-airflow-providers-telegram/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-telegram`` ---------------------------------------------- + .. include:: ../../airflow/providers/telegram/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-trino/changelog.rst b/docs/apache-airflow-providers-trino/changelog.rst index 97f28e55cd18..efd69bd600ed 100644 --- a/docs/apache-airflow-providers-trino/changelog.rst +++ b/docs/apache-airflow-providers-trino/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-trino`` ------------------------------------------- + .. include:: ../../airflow/providers/trino/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-vertica/changelog.rst b/docs/apache-airflow-providers-vertica/changelog.rst index 89424409bca9..447905b4c6c0 100644 --- a/docs/apache-airflow-providers-vertica/changelog.rst +++ b/docs/apache-airflow-providers-vertica/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-vertica`` --------------------------------------------- + .. include:: ../../airflow/providers/vertica/CHANGELOG.rst diff --git a/docs/apache-airflow-providers-zendesk/changelog.rst b/docs/apache-airflow-providers-zendesk/changelog.rst index fe70585d1efe..e32bcaee32f0 100644 --- a/docs/apache-airflow-providers-zendesk/changelog.rst +++ b/docs/apache-airflow-providers-zendesk/changelog.rst @@ -16,7 +16,6 @@ specific language governing permissions and limitations under the License. -Package ``apache-airflow-providers-zendesk`` --------------------------------------------- + .. include:: ../../airflow/providers/zendesk/CHANGELOG.rst From 68517c1765b4b5a87b7ab82129393f2fd7631e0c Mon Sep 17 00:00:00 2001 From: Amogh Desai Date: Wed, 5 Jul 2023 21:14:48 +0530 Subject: [PATCH 062/533] Updating pyenv instructions for quick start guide (#32374) Co-authored-by: Amogh Desai --- CONTRIBUTORS_QUICK_START.rst | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CONTRIBUTORS_QUICK_START.rst b/CONTRIBUTORS_QUICK_START.rst index b2aa30c9de72..20e6191b3db9 100644 --- a/CONTRIBUTORS_QUICK_START.rst +++ b/CONTRIBUTORS_QUICK_START.rst @@ -179,17 +179,27 @@ extra while installing airflow. .. code-block:: bash +For Architectures other than MacOS/ARM $ pyenv install --list $ pyenv install 3.8.5 $ pyenv versions +For MacOS/Arm (3.9.1 is the first version of Python to support MacOS/ARM, but 3.8.10 works too) + $ pyenv install --list + $ pyenv install 3.8.10 + $ pyenv versions + 5. Creating new virtual environment named ``airflow-env`` for installed version python. In next chapter virtual environment ``airflow-env`` will be used for installing airflow. .. code-block:: bash +For Architectures other than MacOS/ARM $ pyenv virtualenv 3.8.5 airflow-env +For MacOS/Arm (3.9.1 is the first version of Python to support MacOS/ARM, but 3.8.10 works too) + $ pyenv virtualenv 3.8.10 airflow-env + 6. Entering virtual environment ``airflow-env`` .. code-block:: bash From 383efbcbe00a0ce25251ba3a06a625f0ae40d806 Mon Sep 17 00:00:00 2001 From: Mats Holm <32451087+holmen1@users.noreply.github.com> Date: Wed, 5 Jul 2023 18:39:37 +0200 Subject: [PATCH 063/533] Update pipeline.rst (#32369) Typo in final dag, merge_data. --- docs/apache-airflow/tutorial/pipeline.rst | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/tutorial/pipeline.rst b/docs/apache-airflow/tutorial/pipeline.rst index c9bfcfa6013f..fca2418a701c 100644 --- a/docs/apache-airflow/tutorial/pipeline.rst +++ b/docs/apache-airflow/tutorial/pipeline.rst @@ -288,7 +288,10 @@ Putting all of the pieces together, we have our completed DAG. FROM employees_temp ) t ON CONFLICT ("Serial Number") DO UPDATE - SET "Serial Number" = excluded."Serial Number"; + SET + "Employee Markme" = excluded."Employee Markme", + "Description" = excluded."Description", + "Leave" = excluded."Leave"; """ try: postgres_hook = PostgresHook(postgres_conn_id="tutorial_pg_conn") From d3f0900bbb68bda36f81475bf64d83d51ee32b22 Mon Sep 17 00:00:00 2001 From: Vincent <97131062+vincbeck@users.noreply.github.com> Date: Wed, 5 Jul 2023 12:52:14 -0400 Subject: [PATCH 064/533] AIP-56 - Create auth manager interface and very basic implementation of FAB auth manager (#32217) --- airflow/auth/__init__.py | 17 +++++++ airflow/auth/managers/__init__.py | 17 +++++++ airflow/auth/managers/base_auth_manager.py | 35 ++++++++++++++ airflow/auth/managers/fab/__init__.py | 17 +++++++ airflow/auth/managers/fab/fab_auth_manager.py | 41 ++++++++++++++++ airflow/config_templates/config.yml | 7 +++ airflow/config_templates/default_airflow.cfg | 3 ++ airflow/configuration.py | 20 ++++++++ airflow/www/extensions/init_jinja_globals.py | 3 +- .../templates/appbuilder/navbar_right.html | 9 ++-- tests/auh/__init__.py | 16 +++++++ tests/auh/managers/__init__.py | 16 +++++++ tests/auh/managers/fab/__init__.py | 16 +++++++ .../auh/managers/fab/test_fab_auth_manager.py | 48 +++++++++++++++++++ tests/www/views/conftest.py | 1 + 15 files changed, 259 insertions(+), 7 deletions(-) create mode 100644 airflow/auth/__init__.py create mode 100644 airflow/auth/managers/__init__.py create mode 100644 airflow/auth/managers/base_auth_manager.py create mode 100644 airflow/auth/managers/fab/__init__.py create mode 100644 airflow/auth/managers/fab/fab_auth_manager.py create mode 100644 tests/auh/__init__.py create mode 100644 tests/auh/managers/__init__.py create mode 100644 tests/auh/managers/fab/__init__.py create mode 100644 tests/auh/managers/fab/test_fab_auth_manager.py diff --git a/airflow/auth/__init__.py b/airflow/auth/__init__.py new file mode 100644 index 000000000000..217e5db96078 --- /dev/null +++ b/airflow/auth/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/airflow/auth/managers/__init__.py b/airflow/auth/managers/__init__.py new file mode 100644 index 000000000000..217e5db96078 --- /dev/null +++ b/airflow/auth/managers/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/airflow/auth/managers/base_auth_manager.py b/airflow/auth/managers/base_auth_manager.py new file mode 100644 index 000000000000..6a4bb86f1666 --- /dev/null +++ b/airflow/auth/managers/base_auth_manager.py @@ -0,0 +1,35 @@ +# +# 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 abc import abstractmethod + +from airflow.utils.log.logging_mixin import LoggingMixin + + +class BaseAuthManager(LoggingMixin): + """ + Class to derive in order to implement concrete auth managers. + + Auth managers are responsible for any user management related operation such as login, logout, authz, ... + """ + + @abstractmethod + def get_user_name(self) -> str: + """Return the username associated to the user in session.""" + ... diff --git a/airflow/auth/managers/fab/__init__.py b/airflow/auth/managers/fab/__init__.py new file mode 100644 index 000000000000..217e5db96078 --- /dev/null +++ b/airflow/auth/managers/fab/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/airflow/auth/managers/fab/fab_auth_manager.py b/airflow/auth/managers/fab/fab_auth_manager.py new file mode 100644 index 000000000000..420385716822 --- /dev/null +++ b/airflow/auth/managers/fab/fab_auth_manager.py @@ -0,0 +1,41 @@ +# +# 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 flask_login import current_user + +from airflow.auth.managers.base_auth_manager import BaseAuthManager + + +class FabAuthManager(BaseAuthManager): + """ + Flask-AppBuilder auth manager. + + This auth manager is responsible for providing a backward compatible user management experience to users. + """ + + def get_user_name(self) -> str: + """ + Return the username associated to the user in session. + + For backward compatibility reasons, the username in FAB auth manager is the concatenation of the + first name and the last name. + """ + first_name = current_user.first_name or "" + last_name = current_user.last_name or "" + return f"{first_name} {last_name}".strip() diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index d588a7f26e3b..273f2a36c76d 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -70,6 +70,13 @@ core: type: string example: ~ default: "SequentialExecutor" + auth_manager: + description: | + The auth manager class that airflow should use. Full import path to the auth manager class. + version_added: ~ + type: string + example: ~ + default: "airflow.auth.managers.fab.fab_auth_manager.FabAuthManager" parallelism: description: | This defines the maximum number of task instances that can run concurrently per scheduler in diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index ae6bdec08524..439944022fe3 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -58,6 +58,9 @@ default_timezone = utc # full import path to the class when using a custom executor. executor = SequentialExecutor +# The auth manager class that airflow should use. Full import path to the auth manager class. +auth_manager = airflow.auth.managers.fab.fab_auth_manager.FabAuthManager + # This defines the maximum number of task instances that can run concurrently per scheduler in # Airflow, regardless of the worker count. Generally this value, multiplied by the number of # schedulers in your cluster, is the maximum number of task instances with the running diff --git a/airflow/configuration.py b/airflow/configuration.py index 288595efbd94..6a55a6c99cfc 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -42,6 +42,7 @@ from typing_extensions import overload +from airflow.auth.managers.base_auth_manager import BaseAuthManager from airflow.exceptions import AirflowConfigException from airflow.secrets import DEFAULT_SECRETS_SEARCH_PATH, BaseSecretsBackend from airflow.utils import yaml @@ -1743,6 +1744,24 @@ def initialize_secrets_backends() -> list[BaseSecretsBackend]: return backend_list +def initialize_auth_manager() -> BaseAuthManager: + """ + Initialize auth manager. + + * import user manager class + * instantiate it and return it + """ + auth_manager_cls = conf.getimport(section="core", key="auth_manager") + + if not auth_manager_cls: + raise AirflowConfigException( + "No auth manager defined in the config. " + "Please specify one using section/key [core/auth_manager]." + ) + + return auth_manager_cls() + + @functools.lru_cache(maxsize=None) def _DEFAULT_CONFIG() -> str: path = _default_config_file_path("default_airflow.cfg") @@ -1808,4 +1827,5 @@ def __getattr__(name): conf = initialize_config() secrets_backend_list = initialize_secrets_backends() +auth_manager = initialize_auth_manager() conf.validate() diff --git a/airflow/www/extensions/init_jinja_globals.py b/airflow/www/extensions/init_jinja_globals.py index 0674a8e4a3ea..9ef948084cc9 100644 --- a/airflow/www/extensions/init_jinja_globals.py +++ b/airflow/www/extensions/init_jinja_globals.py @@ -21,7 +21,7 @@ import pendulum import airflow -from airflow.configuration import conf +from airflow.configuration import auth_manager, conf from airflow.settings import IS_K8S_OR_K8SCELERY_EXECUTOR, STATE_COLORS from airflow.utils.net import get_hostname from airflow.utils.platform import get_airflow_git_version @@ -68,6 +68,7 @@ def prepare_jinja_globals(): "git_version": git_version, "k8s_or_k8scelery_executor": IS_K8S_OR_K8SCELERY_EXECUTOR, "rest_api_enabled": False, + "auth_manager": auth_manager, "config_test_connection": conf.get("core", "test_connection", fallback="Disabled"), } diff --git a/airflow/www/templates/appbuilder/navbar_right.html b/airflow/www/templates/appbuilder/navbar_right.html index 8e606c65b30b..1ade0847a65e 100644 --- a/airflow/www/templates/appbuilder/navbar_right.html +++ b/airflow/www/templates/appbuilder/navbar_right.html @@ -66,12 +66,9 @@ {% if not current_user.is_anonymous %}