diff --git a/.github/ISSUE_TEMPLATE/airflow_bug_report.yml b/.github/ISSUE_TEMPLATE/airflow_bug_report.yml index 612be280410a..2a482ffc7fbd 100644 --- a/.github/ISSUE_TEMPLATE/airflow_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/airflow_bug_report.yml @@ -25,7 +25,7 @@ body: the latest release or main to see if the issue is fixed before reporting it. multiple: false options: - - "2.10.0b1" + - "2.10.0rc1" - "2.9.3" - "main (development)" - "Other Airflow 2 version (please specify below)" diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml index 516fc2b457df..77380c7674e8 100644 --- a/.github/boring-cyborg.yml +++ b/.github/boring-cyborg.yml @@ -549,12 +549,9 @@ labelPRBasedOnFilePath: area:API: - airflow/api/**/* - airflow/api_connexion/**/* - - airflow/www/api/**/* - clients/**/* - - docs/apache-airflow/deprecated-rest-api-ref.rst - docs/apache-airflow/stable-rest-api-ref.rst - tests/api_connexion/**/* - - tests/www/api/**/* area:dev-tools: - scripts/**/* diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml index bd10e73aac65..1256fd2f0da6 100644 --- a/.github/workflows/build-images.yml +++ b/.github/workflows/build-images.yml @@ -241,14 +241,14 @@ jobs: pull-request-target: "true" is-committer-build: ${{ needs.build-info.outputs.is-committer-build }} push-image: "true" - use-uv: ${{ needs.build-info.outputs.default-branch == 'main' && 'true' || 'false' }} + use-uv: "true" image-tag: ${{ needs.build-info.outputs.image-tag }} platform: "linux/amd64" python-versions: ${{ needs.build-info.outputs.python-versions }} default-python-version: ${{ needs.build-info.outputs.default-python-version }} branch: ${{ needs.build-info.outputs.default-branch }} constraints-branch: ${{ needs.build-info.outputs.constraints-branch }} - build-provider-packages: "true" + build-provider-packages: ${{ needs.build-info.outputs.default-branch == 'main' }} upgrade-to-newer-dependencies: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }} chicken-egg-providers: ${{ needs.build-info.outputs.chicken-egg-providers }} docker-cache: ${{ needs.build-info.outputs.docker-cache }} diff --git a/.github/workflows/check-providers.yml b/.github/workflows/check-providers.yml index 622a67fea97a..e89d4a81faac 100644 --- a/.github/workflows/check-providers.yml +++ b/.github/workflows/check-providers.yml @@ -28,6 +28,10 @@ on: # yamllint disable-line rule:truthy description: "Tag to set for the image" required: true type: string + canary-run: + description: "Whether this is a canary run" + required: true + type: string default-python-version: description: "Which version of python should be used by default" required: true @@ -209,6 +213,7 @@ jobs: PYTHON_MAJOR_MINOR_VERSION: "${{ inputs.default-python-version }}" VERSION_SUFFIX_FOR_PYPI: "dev0" VERBOSE: "true" + CLEAN_AIRFLOW_INSTALLATION: "${{ inputs.canary-run }}" if: inputs.skip-provider-tests != 'true' steps: - name: "Cleanup repo" diff --git a/.github/workflows/ci-image-build.yml b/.github/workflows/ci-image-build.yml index 07ba028cf713..1c4b31b55a60 100644 --- a/.github/workflows/ci-image-build.yml +++ b/.github/workflows/ci-image-build.yml @@ -60,7 +60,7 @@ on: # yamllint disable-line rule:truthy default: "true" type: string debian-version: - description: "Base Debian distribution to use for the build (bookworm/bullseye)" + description: "Base Debian distribution to use for the build (bookworm)" type: string default: "bookworm" install-mysql-client-type: diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 68aa51bf860f..fda01f71a5c3 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -23,7 +23,7 @@ on: # yamllint disable-line rule:truthy push: branches: ['v[0-9]+-[0-9]+-test'] pull_request: - branches: ['main'] + branches: ['main', 'v[0-9]+-[0-9]+-test'] workflow_dispatch: permissions: # All other permissions are set to none @@ -319,6 +319,7 @@ jobs: with: runs-on-as-json-default: ${{ needs.build-info.outputs.runs-on-as-json-default }} image-tag: ${{ needs.build-info.outputs.image-tag }} + canary-run: ${{ needs.build-info.outputs.canary-run }} default-python-version: ${{ needs.build-info.outputs.default-python-version }} upgrade-to-newer-dependencies: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }} affected-providers-list-as-string: ${{ needs.build-info.outputs.affected-providers-list-as-string }} @@ -541,7 +542,7 @@ jobs: default-python-version: ${{ needs.build-info.outputs.default-python-version }} branch: ${{ needs.build-info.outputs.default-branch }} push-image: "true" - use-uv: ${{ needs.build-info.outputs.default-branch == 'main' && 'true' || 'false' }} + use-uv: "true" build-provider-packages: ${{ needs.build-info.outputs.default-branch == 'main' }} upgrade-to-newer-dependencies: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }} chicken-egg-providers: ${{ needs.build-info.outputs.chicken-egg-providers }} diff --git a/.github/workflows/news-fragment.yml b/.github/workflows/news-fragment.yml new file mode 100644 index 000000000000..6691bf65f9d1 --- /dev/null +++ b/.github/workflows/news-fragment.yml @@ -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. +# +--- +name: CI + +on: # yamllint disable-line rule:truthy + pull_request: + types: [labeled, unlabeled, opened, reopened, synchronize] + +jobs: + check-news-fragment: + name: Check News Fragment + runs-on: ubuntu-20.04 + if: "contains(github.event.pull_request.labels.*.name, 'airflow3.0:breaking')" + + steps: + - uses: actions/checkout@v4 + with: + persist-credentials: false + # `towncrier check` runs `git diff --name-only origin/main...`, which + # needs a non-shallow clone. + fetch-depth: 0 + + - name: Check news fragment + run: > + pipx run towncrier check + --dir . + --config newsfragments/config.toml + --compare-with origin/${{ github.base_ref }} + || + { + printf "\033[1;33mMissing significant newsfragment for PR labeled with + 'airflow3.0:breaking'.\nCheck + https://github.com/apache/airflow/blob/main/contributing-docs/16_contribution_workflow.rst + for guidance.\033[m\n" + && + false + ; } diff --git a/.github/workflows/prod-image-build.yml b/.github/workflows/prod-image-build.yml index c75701c4567d..75d9d0054ec7 100644 --- a/.github/workflows/prod-image-build.yml +++ b/.github/workflows/prod-image-build.yml @@ -63,7 +63,7 @@ on: # yamllint disable-line rule:truthy required: true type: string debian-version: - description: "Base Debian distribution to use for the build (bookworm/bullseye)" + description: "Base Debian distribution to use for the build (bookworm)" type: string default: "bookworm" install-mysql-client-type: diff --git a/.github/workflows/prod-image-extra-checks.yml b/.github/workflows/prod-image-extra-checks.yml index 380ecb5a67e6..82d327ba2f16 100644 --- a/.github/workflows/prod-image-extra-checks.yml +++ b/.github/workflows/prod-image-extra-checks.yml @@ -64,27 +64,6 @@ on: # yamllint disable-line rule:truthy required: true type: string jobs: - bullseye-image: - uses: ./.github/workflows/prod-image-build.yml - with: - runs-on-as-json-public: ${{ inputs.runs-on-as-json-public }} - build-type: "Bullseye" - upload-package-artifact: "false" - image-tag: bullseye-${{ inputs.image-tag }} - debian-version: "bullseye" - python-versions: ${{ inputs.python-versions }} - default-python-version: ${{ inputs.default-python-version }} - platform: "linux/amd64" - branch: ${{ inputs.branch }} - # Always build images during the extra checks and never push them - push-image: "false" - use-uv: ${{ inputs.use-uv }} - build-provider-packages: ${{ inputs.build-provider-packages }} - upgrade-to-newer-dependencies: ${{ inputs.upgrade-to-newer-dependencies }} - chicken-egg-providers: ${{ inputs.chicken-egg-providers }} - constraints-branch: ${{ inputs.constraints-branch }} - docker-cache: ${{ inputs.docker-cache }} - myssql-client-image: uses: ./.github/workflows/prod-image-build.yml with: diff --git a/.github/workflows/push-image-cache.yml b/.github/workflows/push-image-cache.yml index 1cdb5861e43a..0dc83a3fd66e 100644 --- a/.github/workflows/push-image-cache.yml +++ b/.github/workflows/push-image-cache.yml @@ -41,7 +41,7 @@ on: # yamllint disable-line rule:truthy required: true type: string debian-version: - description: "Base Debian distribution to use for the build (bookworm/bullseye)" + description: "Base Debian distribution to use for the build (bookworm)" type: string default: "bookworm" install-mysql-client-type: diff --git a/.github/workflows/static-checks-mypy-docs.yml b/.github/workflows/static-checks-mypy-docs.yml index 9a1e4ac4ac7f..c875c1667cf9 100644 --- a/.github/workflows/static-checks-mypy-docs.yml +++ b/.github/workflows/static-checks-mypy-docs.yml @@ -243,7 +243,9 @@ jobs: INCLUDE_SUCCESS_OUTPUTS: "${{ inputs.include-success-outputs }}" PYTHON_MAJOR_MINOR_VERSION: "${{ inputs.default-python-version }}" VERBOSE: "true" - if: inputs.canary-run == 'true' && inputs.branch == 'main' + # Temporary disabled it until self-hosted ASF runners will be working again + if: false + # if: inputs.canary-run == 'true' && inputs.branch == 'main' steps: - name: "Cleanup repo" shell: bash diff --git a/Dockerfile b/Dockerfile index 8c4a43274fd3..4aade68d5880 100644 --- a/Dockerfile +++ b/Dockerfile @@ -124,11 +124,7 @@ function get_runtime_apt_deps() { echo echo "DEBIAN CODENAME: ${debian_version}" echo - if [[ "${debian_version}" == "bullseye" ]]; then - debian_version_apt_deps="libffi7 libldap-2.4-2 libssl1.1 netcat" - else - debian_version_apt_deps="libffi8 libldap-2.5-0 libssl3 netcat-openbsd" - fi + debian_version_apt_deps="libffi8 libldap-2.5-0 libssl3 netcat-openbsd" echo echo "APPLIED INSTALLATION CONFIGURATION FOR DEBIAN VERSION: ${debian_version}" echo @@ -177,19 +173,6 @@ function install_debian_dev_dependencies() { echo echo "DEBIAN CODENAME: ${debian_version}" echo - if [[ "${debian_version}" == "bullseye" ]]; then - echo - echo "Bullseye detected - replacing dependencies in additional dev apt deps" - echo - # Replace dependencies in additional dev apt deps to be compatible with Bullseye - ADDITIONAL_DEV_APT_DEPS=${ADDITIONAL_DEV_APT_DEPS//libgcc-11-dev/libgcc-10-dev} - ADDITIONAL_DEV_APT_DEPS=${ADDITIONAL_DEV_APT_DEPS//netcat-openbsd/netcat} - echo - echo "Replaced bullseye dev apt dependencies" - echo "${ADDITIONAL_DEV_APT_COMMAND}" - echo - fi - # shellcheck disable=SC2086 apt-get install -y --no-install-recommends ${DEV_APT_DEPS} ${ADDITIONAL_DEV_APT_DEPS} } diff --git a/Dockerfile.ci b/Dockerfile.ci index 14ccb669f62a..fdf906978cac 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -70,11 +70,7 @@ function get_runtime_apt_deps() { echo echo "DEBIAN CODENAME: ${debian_version}" echo - if [[ "${debian_version}" == "bullseye" ]]; then - debian_version_apt_deps="libffi7 libldap-2.4-2 libssl1.1 netcat" - else - debian_version_apt_deps="libffi8 libldap-2.5-0 libssl3 netcat-openbsd" - fi + debian_version_apt_deps="libffi8 libldap-2.5-0 libssl3 netcat-openbsd" echo echo "APPLIED INSTALLATION CONFIGURATION FOR DEBIAN VERSION: ${debian_version}" echo @@ -123,19 +119,6 @@ function install_debian_dev_dependencies() { echo echo "DEBIAN CODENAME: ${debian_version}" echo - if [[ "${debian_version}" == "bullseye" ]]; then - echo - echo "Bullseye detected - replacing dependencies in additional dev apt deps" - echo - # Replace dependencies in additional dev apt deps to be compatible with Bullseye - ADDITIONAL_DEV_APT_DEPS=${ADDITIONAL_DEV_APT_DEPS//libgcc-11-dev/libgcc-10-dev} - ADDITIONAL_DEV_APT_DEPS=${ADDITIONAL_DEV_APT_DEPS//netcat-openbsd/netcat} - echo - echo "Replaced bullseye dev apt dependencies" - echo "${ADDITIONAL_DEV_APT_COMMAND}" - echo - fi - # shellcheck disable=SC2086 apt-get install -y --no-install-recommends ${DEV_APT_DEPS} ${ADDITIONAL_DEV_APT_DEPS} } @@ -997,15 +980,23 @@ function determine_airflow_to_use() { echo exit 0 fi + if [[ ${CLEAN_AIRFLOW_INSTALLATION=} == "true" ]]; then + echo + echo "${COLOR_BLUE}Uninstalling all packages first${COLOR_RESET}" + echo + pip freeze | grep -ve "^-e" | grep -ve "^#" | grep -ve "^uv" | xargs pip uninstall -y --root-user-action ignore + # Now install rich ad click first to use the installation script + uv pip install rich rich-click click --python "/usr/local/bin/python" \ + --constraint https://raw.githubusercontent.com/apache/airflow/constraints-main/constraints-${PYTHON_MAJOR_MINOR_VERSION}.txt + fi python "${IN_CONTAINER_DIR}/install_airflow_and_providers.py" + echo + echo "${COLOR_BLUE}Reinstalling all development dependencies${COLOR_RESET}" + echo + python "${IN_CONTAINER_DIR}/install_devel_deps.py" \ + --constraint https://raw.githubusercontent.com/apache/airflow/constraints-main/constraints-${PYTHON_MAJOR_MINOR_VERSION}.txt # Some packages might leave legacy typing module which causes test issues pip uninstall -y typing || true - # Upgrade pytest and pytest extensions to latest version if they have been accidentally - # downgraded by constraints - pip install --upgrade pytest pytest aiofiles aioresponses pytest-asyncio pytest-custom-exit-code \ - pytest-icdiff pytest-instafail pytest-mock pytest-rerunfailures pytest-timeouts \ - pytest-xdist pytest requests_mock time-machine \ - --constraint https://raw.githubusercontent.com/apache/airflow/constraints-main/constraints-${PYTHON_MAJOR_MINOR_VERSION}.txt fi if [[ "${USE_AIRFLOW_VERSION}" =~ ^2\.2\..*|^2\.1\..*|^2\.0\..* && "${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN=}" != "" ]]; then @@ -1068,8 +1059,9 @@ function check_pydantic() { echo echo "${COLOR_YELLOW}Downgrading Pydantic to < 2${COLOR_RESET}" echo + # Pydantic 1.10.17/1.10.15 conflicts with aws-sam-translator so we need to exclude it # shellcheck disable=SC2086 - ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade "pydantic<2.0.0" + ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade "pydantic<2.0.0,!=1.10.17,!=1.10.15" pip check else echo diff --git a/INSTALL b/INSTALL index 78506f9a571f..a875fd372811 100644 --- a/INSTALL +++ b/INSTALL @@ -256,9 +256,9 @@ Those extras are available as regular core airflow extras - they install optiona # START CORE EXTRAS HERE -aiobotocore, apache-atlas, apache-webhdfs, async, cgroups, cloudpickle, deprecated-api, github- -enterprise, google-auth, graphviz, kerberos, ldap, leveldb, otel, pandas, password, pydantic, -rabbitmq, s3fs, saml, sentry, statsd, uv, virtualenv +aiobotocore, apache-atlas, apache-webhdfs, async, cgroups, cloudpickle, github-enterprise, google- +auth, graphviz, kerberos, ldap, leveldb, otel, pandas, password, pydantic, rabbitmq, s3fs, saml, +sentry, statsd, uv, virtualenv # END CORE EXTRAS HERE diff --git a/README.md b/README.md index f72e5ff9f7c5..7878d4d9ff5f 100644 --- a/README.md +++ b/README.md @@ -122,9 +122,7 @@ The work to add Windows support is tracked via [#10388](https://github.com/apach it is not a high priority. You should only use Linux-based distros as "Production" execution environment as this is the only environment that is supported. The only distro that is used in our CI tests and that is used in the [Community managed DockerHub image](https://hub.docker.com/p/apache/airflow) is -`Debian Bookworm`. We also have support for legacy ``Debian Bullseye`` base image if you want to build a -custom image but it is deprecated and option to do it will be removed in the Dockerfile that -will accompany Airflow 2.9.3 so you are advised to switch to ``Debian Bookworm`` for your custom images. +`Debian Bookworm`. @@ -347,13 +345,9 @@ building and testing the OS version. Approximately 6 months before the end-of-re previous stable version of the OS, Airflow switches the images released to use the latest supported version of the OS. -For example since ``Debian Buster`` end-of-life was August 2022, Airflow switched the images in `main` branch -to use ``Debian Bullseye`` in February/March 2022. The version was used in the next MINOR release after -the switch happened. In case of the Bullseye switch - 2.3.0 version used ``Debian Bullseye``. -The images released in the previous MINOR version continue to use the version that all other releases -for the MINOR version used. Similar switch from ``Debian Bullseye`` to ``Debian Bookworm`` has been implemented +For example switch from ``Debian Bullseye`` to ``Debian Bookworm`` has been implemented before 2.8.0 release in October 2023 and ``Debian Bookworm`` will be the only option supported as of -Airflow 2.9.0. +Airflow 2.10.0. Users will continue to be able to build their images using stable Debian releases until the end of regular support and building and verifying of the images happens in our CI but no unit tests were executed using diff --git a/RELEASE_NOTES.rst b/RELEASE_NOTES.rst index ddd4e986c56f..da34dfc9bef0 100644 --- a/RELEASE_NOTES.rst +++ b/RELEASE_NOTES.rst @@ -3674,8 +3674,6 @@ Details in the `SQLAlchemy Changelog Client: """Return current API Client based on current Airflow configuration.""" - api_module = import_module(conf.get_mandatory_value("cli", "api_client")) auth_backends = api.load_auth() session = None for backend in auth_backends: session_factory = getattr(backend, "create_client_session", None) if session_factory: session = session_factory() - api_client = api_module.Client( - api_base_url=conf.get("cli", "endpoint_url"), + api_client = Client( auth=getattr(backend, "CLIENT_AUTH", None), session=session, ) diff --git a/airflow/api/client/api_client.py b/airflow/api/client/api_client.py deleted file mode 100644 index 7a61a7ac9a05..000000000000 --- a/airflow/api/client/api_client.py +++ /dev/null @@ -1,94 +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. -"""Client for all the API clients.""" - -from __future__ import annotations - -import httpx - - -class Client: - """Base API client for all API clients.""" - - def __init__(self, api_base_url, auth=None, session: httpx.Client | None = None): - self._api_base_url = api_base_url - self._session: httpx.Client = session or httpx.Client() - if auth: - self._session.auth = auth - - def trigger_dag(self, dag_id, run_id=None, conf=None, execution_date=None, replace_microseconds=True): - """ - Create a dag run for the specified dag. - - :param dag_id: - :param run_id: - :param conf: - :param execution_date: - :param replace_microseconds: - :return: - """ - raise NotImplementedError() - - def delete_dag(self, dag_id): - """ - Delete all DB records related to the specified dag. - - :param dag_id: - """ - raise NotImplementedError() - - def get_pool(self, name): - """ - Get pool. - - :param name: pool name - """ - raise NotImplementedError() - - def get_pools(self): - """Get all pools.""" - raise NotImplementedError() - - def create_pool(self, name, slots, description, include_deferred): - """ - Create a pool. - - :param name: pool name - :param slots: pool slots amount - :param description: pool description - :param include_deferred: include deferred tasks in pool calculations - """ - raise NotImplementedError() - - def delete_pool(self, name): - """ - Delete pool. - - :param name: pool name - """ - raise NotImplementedError() - - def get_lineage(self, dag_id: str, execution_date: str): - """ - Return the lineage information for the dag on this execution date. - - :param dag_id: - :param execution_date: - :return: - """ - raise NotImplementedError() diff --git a/airflow/api/client/json_client.py b/airflow/api/client/json_client.py deleted file mode 100644 index c2b090148083..000000000000 --- a/airflow/api/client/json_client.py +++ /dev/null @@ -1,164 +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. -"""JSON API Client.""" - -from __future__ import annotations - -from urllib.parse import urljoin - -from airflow.api.client import api_client - - -class Client(api_client.Client): - """ - Json API client implementation. - - This client is used to interact with a Json API server and perform various actions - such as triggering DAG runs,deleting DAGs, interacting with pools, and getting lineage information. - """ - - def _request(self, url: str, json=None, method: str = "GET") -> dict: - """ - Make a request to the Json API server. - - :param url: The URL to send the request to. - :param method: The HTTP method to use (e.g. "GET", "POST", "DELETE"). - :param json: A dictionary containing JSON data to send in the request body. - :return: A dictionary containing the JSON response from the server. - :raises OSError: If the server returns an error status. - """ - params = { - "url": url, - } - if json is not None: - params["json"] = json - resp = getattr(self._session, method.lower())(**params) - if resp.is_error: - # It is justified here because there might be many resp types. - try: - data = resp.json() - except Exception: - data = {} - raise OSError(data.get("error", "Server error")) - return resp.json() - - def trigger_dag(self, dag_id, run_id=None, conf=None, execution_date=None, replace_microseconds=True): - """ - Trigger a DAG run. - - :param dag_id: The ID of the DAG to trigger. - :param run_id: The ID of the DAG run to create. If not provided, a default ID will be generated. - :param conf: A dictionary containing configuration data to pass to the DAG run. - :param execution_date: The execution date for the DAG run, in the format "YYYY-MM-DDTHH:MM:SS". - :param replace_microseconds: Whether to replace microseconds in the execution date with zeros. - :return: A message indicating the status of the DAG run trigger. - """ - endpoint = f"/api/experimental/dags/{dag_id}/dag_runs" - url = urljoin(self._api_base_url, endpoint) - data = { - "run_id": run_id, - "conf": conf, - "execution_date": execution_date, - "replace_microseconds": replace_microseconds, - } - return self._request(url, method="POST", json=data)["message"] - - def delete_dag(self, dag_id: str): - """ - Delete a DAG. - - :param dag_id: The ID of the DAG to delete. - :return: A message indicating the status of the DAG delete operation. - """ - endpoint = f"/api/experimental/dags/{dag_id}/delete_dag" - url = urljoin(self._api_base_url, endpoint) - data = self._request(url, method="DELETE") - return data["message"] - - def get_pool(self, name: str): - """ - Get information about a specific pool. - - :param name: The name of the pool to retrieve information for. - :return: A tuple containing the name of the pool, the number of - slots in the pool, and a description of the pool. - """ - endpoint = f"/api/experimental/pools/{name}" - url = urljoin(self._api_base_url, endpoint) - pool = self._request(url) - return pool["pool"], pool["slots"], pool["description"] - - def get_pools(self): - """ - Get a list of all pools. - - :return: A list of tuples, each containing the name of a pool, - the number of slots in the pool, and a description of the pool. - """ - endpoint = "/api/experimental/pools" - url = urljoin(self._api_base_url, endpoint) - pools = self._request(url) - return [(p["pool"], p["slots"], p["description"]) for p in pools] - - def create_pool(self, name: str, slots: int, description: str, include_deferred: bool): - """ - Create a new pool. - - :param name: The name of the pool to create. - :param slots: The number of slots in the pool. - :param description: A description of the pool. - :param include_deferred: include deferred tasks in pool calculations - - :return: A tuple containing the name of the pool, the number of slots in the pool, - a description of the pool and the include_deferred flag. - """ - endpoint = "/api/experimental/pools" - data = { - "name": name, - "slots": slots, - "description": description, - "include_deferred": include_deferred, - } - response = self._request(urljoin(self._api_base_url, endpoint), method="POST", json=data) - return response["pool"], response["slots"], response["description"], response["include_deferred"] - - def delete_pool(self, name: str): - """ - Delete a pool. - - :param name: The name of the pool to delete. - :return: A tuple containing the name of the pool, the number - of slots in the pool, and a description of the pool. - """ - endpoint = f"/api/experimental/pools/{name}" - url = urljoin(self._api_base_url, endpoint) - pool = self._request(url, method="DELETE") - return pool["pool"], pool["slots"], pool["description"] - - def get_lineage(self, dag_id: str, execution_date: str): - """ - Get the lineage of a DAG run. - - :param dag_id: The ID of the DAG. - :param execution_date: The execution date of the DAG run, in the format "YYYY-MM-DDTHH:MM:SS". - :return: A message indicating the status of the lineage request. - """ - endpoint = f"/api/experimental/lineage/{dag_id}/{execution_date}" - url = urljoin(self._api_base_url, endpoint) - data = self._request(url, method="GET") - return data["message"] diff --git a/airflow/api/client/local_client.py b/airflow/api/client/local_client.py index 4fd0754218ec..bef99e318417 100644 --- a/airflow/api/client/local_client.py +++ b/airflow/api/client/local_client.py @@ -19,16 +19,21 @@ from __future__ import annotations -from airflow.api.client import api_client +import httpx + from airflow.api.common import delete_dag, trigger_dag -from airflow.api.common.experimental.get_lineage import get_lineage as get_lineage_api from airflow.exceptions import AirflowBadRequest, PoolNotFound from airflow.models.pool import Pool -class Client(api_client.Client): +class Client: """Local API client implementation.""" + def __init__(self, auth=None, session: httpx.Client | None = None): + self._session: httpx.Client = session or httpx.Client() + if auth: + self._session.auth = auth + def trigger_dag( self, dag_id, run_id=None, conf=None, execution_date=None, replace_microseconds=True ) -> dict | None: @@ -87,7 +92,3 @@ def create_pool(self, name, slots, description, include_deferred): def delete_pool(self, name): pool = Pool.delete_pool(name=name) return pool.pool, pool.slots, pool.description - - def get_lineage(self, dag_id, execution_date): - lineage = get_lineage_api(dag_id=dag_id, execution_date=execution_date) - return lineage diff --git a/airflow/api/common/delete_dag.py b/airflow/api/common/delete_dag.py index 1cf7ffec8b9e..11b046648c5a 100644 --- a/airflow/api/common/delete_dag.py +++ b/airflow/api/common/delete_dag.py @@ -22,11 +22,11 @@ import logging from typing import TYPE_CHECKING -from sqlalchemy import and_, delete, or_, select +from sqlalchemy import delete, select from airflow import models from airflow.exceptions import AirflowException, DagNotFound -from airflow.models import DagModel, TaskFail +from airflow.models import DagModel from airflow.models.errors import ParseImportError from airflow.models.serialized_dag import SerializedDagModel from airflow.utils.db import get_sqla_model_classes @@ -64,18 +64,6 @@ def delete_dag(dag_id: str, keep_records_in_log: bool = True, session: Session = if dag is None: raise DagNotFound(f"Dag id {dag_id} not found") - # deleting a DAG should also delete all of its subdags - dags_to_delete_query = session.execute( - select(DagModel.dag_id).where( - or_( - DagModel.dag_id == dag_id, - and_(DagModel.dag_id.like(f"{dag_id}.%"), DagModel.is_subdag), - ) - ) - ) - - dags_to_delete = [dag_id for (dag_id,) in dags_to_delete_query] - # Scheduler removes DAGs without files from serialized_dag table every dag_dir_list_interval. # There may be a lag, so explicitly removes serialized DAG here. if SerializedDagModel.has_dag(dag_id=dag_id, session=session): @@ -86,15 +74,7 @@ def delete_dag(dag_id: str, keep_records_in_log: bool = True, session: Session = for model in get_sqla_model_classes(): if hasattr(model, "dag_id") and (not keep_records_in_log or model.__name__ != "Log"): count += session.execute( - delete(model) - .where(model.dag_id.in_(dags_to_delete)) - .execution_options(synchronize_session="fetch") - ).rowcount - if dag.is_subdag: - parent_dag_id, task_id = dag_id.rsplit(".", 1) - for model in TaskFail, models.TaskInstance: - count += session.execute( - delete(model).where(model.dag_id == parent_dag_id, model.task_id == task_id) + delete(model).where(model.dag_id == dag_id).execution_options(synchronize_session="fetch") ).rowcount # Delete entries in Import Errors table for a deleted DAG diff --git a/airflow/api/common/experimental/__init__.py b/airflow/api/common/experimental/__init__.py deleted file mode 100644 index 1f0f0bc67db8..000000000000 --- a/airflow/api/common/experimental/__init__.py +++ /dev/null @@ -1,56 +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. -"""Experimental APIs.""" - -from __future__ import annotations - -from typing import TYPE_CHECKING - -from airflow.exceptions import DagNotFound, DagRunNotFound, TaskNotFound -from airflow.models import DagBag, DagModel - -if TYPE_CHECKING: - from datetime import datetime - - from airflow.models import DagRun - - -def check_and_get_dag(dag_id: str, task_id: str | None = None) -> DagModel: - """Check DAG existence and in case it is specified that Task exists.""" - dag_model = DagModel.get_current(dag_id) - if dag_model is None: - raise DagNotFound(f"Dag id {dag_id} not found in DagModel") - - dagbag = DagBag(dag_folder=dag_model.fileloc, read_dags_from_db=True) - dag = dagbag.get_dag(dag_id) - if not dag: - error_message = f"Dag id {dag_id} not found" - raise DagNotFound(error_message) - if task_id and not dag.has_task(task_id): - error_message = f"Task {task_id} not found in dag {dag_id}" - raise TaskNotFound(error_message) - return dag - - -def check_and_get_dagrun(dag: DagModel, execution_date: datetime) -> DagRun: - """Get DagRun object and check that it exists.""" - dagrun = dag.get_dagrun(execution_date=execution_date) - if not dagrun: - error_message = f"Dag Run for date {execution_date} not found in dag {dag.dag_id}" - raise DagRunNotFound(error_message) - return dagrun diff --git a/airflow/api/common/experimental/delete_dag.py b/airflow/api/common/experimental/delete_dag.py deleted file mode 100644 index 402b4cac7b1a..000000000000 --- a/airflow/api/common/experimental/delete_dag.py +++ /dev/null @@ -1,30 +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. -"""This module is deprecated. Please use :mod:`airflow.api.common.delete_dag` instead.""" - -from __future__ import annotations - -import warnings - -from airflow.api.common.delete_dag import * # noqa: F403 - -warnings.warn( - "This module is deprecated. Please use `airflow.api.common.delete_dag` instead.", - DeprecationWarning, - stacklevel=2, -) diff --git a/airflow/api/common/experimental/get_code.py b/airflow/api/common/experimental/get_code.py deleted file mode 100644 index f21e24b7751d..000000000000 --- a/airflow/api/common/experimental/get_code.py +++ /dev/null @@ -1,45 +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. -"""Get code APIs.""" - -from __future__ import annotations - -from deprecated import deprecated - -from airflow.api.common.experimental import check_and_get_dag -from airflow.exceptions import AirflowException, DagCodeNotFound, RemovedInAirflow3Warning -from airflow.models.dagcode import DagCode - - -@deprecated( - reason="Use DagCode().get_code_by_fileloc() instead", version="2.2.4", category=RemovedInAirflow3Warning -) -def get_code(dag_id: str) -> str: - """ - Return python code of a given dag_id. - - :param dag_id: DAG id - :return: code of the DAG - """ - dag = check_and_get_dag(dag_id=dag_id) - - try: - return DagCode.get_code_by_fileloc(dag.fileloc) - except (OSError, DagCodeNotFound) as exception: - error_message = f"Error {exception} while reading Dag id {dag_id} Code" - raise AirflowException(error_message, exception) diff --git a/airflow/api/common/experimental/get_dag_run_state.py b/airflow/api/common/experimental/get_dag_run_state.py deleted file mode 100644 index 0e344158eaf3..000000000000 --- a/airflow/api/common/experimental/get_dag_run_state.py +++ /dev/null @@ -1,46 +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. -"""DAG run APIs.""" - -from __future__ import annotations - -from typing import TYPE_CHECKING - -from deprecated import deprecated - -from airflow.api.common.experimental import check_and_get_dag, check_and_get_dagrun -from airflow.exceptions import RemovedInAirflow3Warning - -if TYPE_CHECKING: - from datetime import datetime - - -@deprecated(reason="Use DagRun().get_state() instead", version="2.2.4", category=RemovedInAirflow3Warning) -def get_dag_run_state(dag_id: str, execution_date: datetime) -> dict[str, str]: - """ - Return the Dag Run state identified by the given dag_id and execution_date. - - :param dag_id: DAG id - :param execution_date: execution date - :return: Dictionary storing state of the object - """ - dag = check_and_get_dag(dag_id=dag_id) - - dagrun = check_and_get_dagrun(dag, execution_date) - - return {"state": dagrun.get_state()} diff --git a/airflow/api/common/experimental/get_dag_runs.py b/airflow/api/common/experimental/get_dag_runs.py deleted file mode 100644 index 856214bd02cf..000000000000 --- a/airflow/api/common/experimental/get_dag_runs.py +++ /dev/null @@ -1,57 +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. -"""DAG runs APIs.""" - -from __future__ import annotations - -from typing import Any - -from flask import url_for - -from airflow.api.common.experimental import check_and_get_dag -from airflow.models import DagRun -from airflow.utils.state import DagRunState - - -def get_dag_runs(dag_id: str, state: str | None = None) -> list[dict[str, Any]]: - """ - Return a list of Dag Runs for a specific DAG ID. - - :param dag_id: String identifier of a DAG - :param state: queued|running|success... - :return: List of DAG runs of a DAG with requested state, - or all runs if the state is not specified - """ - check_and_get_dag(dag_id=dag_id) - - dag_runs = [] - state = DagRunState(state.lower()) if state else None - for run in DagRun.find(dag_id=dag_id, state=state): - dag_runs.append( - { - "id": run.id, - "run_id": run.run_id, - "state": run.state, - "dag_id": run.dag_id, - "execution_date": run.execution_date.isoformat(), - "start_date": ((run.start_date or "") and run.start_date.isoformat()), - "dag_run_url": url_for("Airflow.graph", dag_id=run.dag_id, execution_date=run.execution_date), - } - ) - - return dag_runs diff --git a/airflow/api/common/experimental/get_lineage.py b/airflow/api/common/experimental/get_lineage.py deleted file mode 100644 index d361ae66f309..000000000000 --- a/airflow/api/common/experimental/get_lineage.py +++ /dev/null @@ -1,53 +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. -"""Lineage APIs.""" - -from __future__ import annotations - -from collections import defaultdict -from typing import TYPE_CHECKING, Any - -from airflow.api.common.experimental import check_and_get_dag, check_and_get_dagrun -from airflow.lineage import PIPELINE_INLETS, PIPELINE_OUTLETS -from airflow.models.xcom import XCom -from airflow.utils.session import NEW_SESSION, provide_session - -if TYPE_CHECKING: - import datetime - - from sqlalchemy.orm import Session - - -@provide_session -def get_lineage( - dag_id: str, execution_date: datetime.datetime, *, session: Session = NEW_SESSION -) -> dict[str, dict[str, Any]]: - """Get lineage information for dag specified.""" - dag = check_and_get_dag(dag_id) - dagrun = check_and_get_dagrun(dag, execution_date) - - inlets = XCom.get_many(dag_ids=dag_id, run_id=dagrun.run_id, key=PIPELINE_INLETS, session=session) - outlets = XCom.get_many(dag_ids=dag_id, run_id=dagrun.run_id, key=PIPELINE_OUTLETS, session=session) - - lineage: dict[str, dict[str, Any]] = defaultdict(dict) - for meta in inlets: - lineage[meta.task_id]["inlets"] = meta.value - for meta in outlets: - lineage[meta.task_id]["outlets"] = meta.value - - return {"task_ids": dict(lineage)} diff --git a/airflow/api/common/experimental/get_task.py b/airflow/api/common/experimental/get_task.py deleted file mode 100644 index 7e25ff4ffd65..000000000000 --- a/airflow/api/common/experimental/get_task.py +++ /dev/null @@ -1,39 +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. -"""Task APIs.""" - -from __future__ import annotations - -from typing import TYPE_CHECKING - -from deprecated import deprecated - -from airflow.api.common.experimental import check_and_get_dag -from airflow.exceptions import RemovedInAirflow3Warning - -if TYPE_CHECKING: - from airflow.models import TaskInstance - - -@deprecated(reason="Use DAG().get_task", version="2.2.4", category=RemovedInAirflow3Warning) -def get_task(dag_id: str, task_id: str) -> TaskInstance: - """Return the task object identified by the given dag_id and task_id.""" - dag = check_and_get_dag(dag_id, task_id) - - # Return the task. - return dag.get_task(task_id) diff --git a/airflow/api/common/experimental/get_task_instance.py b/airflow/api/common/experimental/get_task_instance.py deleted file mode 100644 index 722120534d55..000000000000 --- a/airflow/api/common/experimental/get_task_instance.py +++ /dev/null @@ -1,48 +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. -"""Task instance APIs.""" - -from __future__ import annotations - -from typing import TYPE_CHECKING - -from deprecated import deprecated - -from airflow.api.common.experimental import check_and_get_dag, check_and_get_dagrun -from airflow.exceptions import RemovedInAirflow3Warning, TaskInstanceNotFound -from airflow.models import TaskInstance - -if TYPE_CHECKING: - from datetime import datetime - - -@deprecated(version="2.2.4", reason="Use DagRun.get_task_instance instead", category=RemovedInAirflow3Warning) -def get_task_instance(dag_id: str, task_id: str, execution_date: datetime) -> TaskInstance: - """Return the task instance identified by the given dag_id, task_id and execution_date.""" - dag = check_and_get_dag(dag_id, task_id) - - dagrun = check_and_get_dagrun(dag=dag, execution_date=execution_date) - # Get task instance object and check that it exists - task_instance = dagrun.get_task_instance(task_id) - if not task_instance: - error_message = f"Task {task_id} instance for date {execution_date} not found" - raise TaskInstanceNotFound(error_message) - # API methods has access to the database. - if isinstance(task_instance, TaskInstance): - return task_instance - raise ValueError("not a TaskInstance") diff --git a/airflow/api/common/experimental/mark_tasks.py b/airflow/api/common/experimental/mark_tasks.py deleted file mode 100644 index 753db8947f13..000000000000 --- a/airflow/api/common/experimental/mark_tasks.py +++ /dev/null @@ -1,36 +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. -"""Task Instance APIs. This module is deprecated. Please use :mod:`airflow.api.common.mark_tasks` instead.""" - -from __future__ import annotations - -import warnings - -from airflow.api.common.mark_tasks import ( # noqa: F401 - _create_dagruns, - set_dag_run_state_to_failed, - set_dag_run_state_to_running, - set_dag_run_state_to_success, - set_state, -) - -warnings.warn( - "This module is deprecated. Please use `airflow.api.common.mark_tasks` instead.", - DeprecationWarning, - stacklevel=2, -) diff --git a/airflow/api/common/experimental/pool.py b/airflow/api/common/experimental/pool.py deleted file mode 100644 index e3d99be8c164..000000000000 --- a/airflow/api/common/experimental/pool.py +++ /dev/null @@ -1,104 +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. -"""Pool APIs.""" - -from __future__ import annotations - -from typing import TYPE_CHECKING - -from deprecated import deprecated -from sqlalchemy import select - -from airflow.exceptions import AirflowBadRequest, PoolNotFound, RemovedInAirflow3Warning -from airflow.models import Pool -from airflow.utils.session import NEW_SESSION, provide_session - -if TYPE_CHECKING: - from sqlalchemy.orm import Session - - -@deprecated(reason="Use Pool.get_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning) -@provide_session -def get_pool(name, session: Session = NEW_SESSION): - """Get pool by a given name.""" - if not (name and name.strip()): - raise AirflowBadRequest("Pool name shouldn't be empty") - - pool = session.scalar(select(Pool).filter_by(pool=name).limit(1)) - if pool is None: - raise PoolNotFound(f"Pool '{name}' doesn't exist") - - return pool - - -@deprecated(reason="Use Pool.get_pools() instead", version="2.2.4", category=RemovedInAirflow3Warning) -@provide_session -def get_pools(session: Session = NEW_SESSION): - """Get all pools.""" - return session.scalars(select(Pool)).all() - - -@deprecated(reason="Use Pool.create_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning) -@provide_session -def create_pool(name, slots, description, session: Session = NEW_SESSION): - """Create a pool with given parameters.""" - if not (name and name.strip()): - raise AirflowBadRequest("Pool name shouldn't be empty") - - try: - slots = int(slots) - except ValueError: - raise AirflowBadRequest(f"Bad value for `slots`: {slots}") - - # Get the length of the pool column - pool_name_length = Pool.pool.property.columns[0].type.length - if len(name) > pool_name_length: - raise AirflowBadRequest(f"Pool name can't be more than {pool_name_length} characters") - - session.expire_on_commit = False - pool = session.scalar(select(Pool).filter_by(pool=name).limit(1)) - if pool is None: - pool = Pool(pool=name, slots=slots, description=description, include_deferred=False) - session.add(pool) - else: - pool.slots = slots - pool.description = description - - session.commit() - - return pool - - -@deprecated(reason="Use Pool.delete_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning) -@provide_session -def delete_pool(name, session: Session = NEW_SESSION): - """Delete pool by a given name.""" - if not (name and name.strip()): - raise AirflowBadRequest("Pool name shouldn't be empty") - - if name == Pool.DEFAULT_POOL_NAME: - raise AirflowBadRequest(f"{Pool.DEFAULT_POOL_NAME} cannot be deleted") - - pool = session.scalar(select(Pool).filter_by(pool=name).limit(1)) - if pool is None: - raise PoolNotFound(f"Pool '{name}' doesn't exist") - - session.delete(pool) - session.commit() - - return pool diff --git a/airflow/api/common/experimental/trigger_dag.py b/airflow/api/common/experimental/trigger_dag.py deleted file mode 100644 index c65b8a57c593..000000000000 --- a/airflow/api/common/experimental/trigger_dag.py +++ /dev/null @@ -1,30 +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. -"""This module is deprecated. Please use :mod:`airflow.api.common.trigger_dag` instead.""" - -from __future__ import annotations - -import warnings - -from airflow.api.common.trigger_dag import * # noqa: F403 - -warnings.warn( - "This module is deprecated. Please use `airflow.api.common.trigger_dag` instead.", - DeprecationWarning, - stacklevel=2, -) diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index fa6ce835a919..d0be6b86d21c 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -26,12 +26,10 @@ from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance -from airflow.operators.subdag import SubDagOperator from airflow.utils import timezone from airflow.utils.helpers import exactly_one from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import DagRunState, State, TaskInstanceState -from airflow.utils.types import DagRunType if TYPE_CHECKING: from datetime import datetime @@ -40,6 +38,7 @@ from airflow.models.dag import DAG from airflow.models.operator import Operator + from airflow.utils.types import DagRunType class _DagRunInfo(NamedTuple): @@ -101,14 +100,14 @@ def set_state( Can set state for future tasks (calculated from run_id) and retroactively for past tasks. Will verify integrity of past dag runs in order to create tasks that did not exist. It will not create dag runs that are missing - on the schedule (but it will, as for subdag, dag runs if needed). + on the schedule. :param tasks: the iterable of tasks or (task, map_index) tuples from which to work. ``task.dag`` needs to be set :param run_id: the run_id of the dagrun to start looking from :param execution_date: the execution date from which to start looking (deprecated) :param upstream: Mark all parents (upstream tasks) - :param downstream: Mark all siblings (downstream tasks) of task_id, including SubDags + :param downstream: Mark all siblings (downstream tasks) of task_id :param future: Mark all future tasks on the interval of the dag up until last execution date. :param past: Retroactively mark all tasks starting from start_date of the DAG @@ -140,54 +139,20 @@ def set_state( dag_run_ids = get_run_ids(dag, run_id, future, past, session=session) task_id_map_index_list = list(find_task_relatives(tasks, downstream, upstream)) - task_ids = [task_id if isinstance(task_id, str) else task_id[0] for task_id in task_id_map_index_list] - - confirmed_infos = list(_iter_existing_dag_run_infos(dag, dag_run_ids, session=session)) - confirmed_dates = [info.logical_date for info in confirmed_infos] - - sub_dag_run_ids = ( - list( - _iter_subdag_run_ids(dag, session, DagRunState(state), task_ids, commit, confirmed_infos), - ) - if not state == TaskInstanceState.SKIPPED - else [] - ) - # now look for the task instances that are affected qry_dag = get_all_dag_task_query(dag, session, state, task_id_map_index_list, dag_run_ids) if commit: tis_altered = session.scalars(qry_dag.with_for_update()).all() - if sub_dag_run_ids: - qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) - tis_altered += session.scalars(qry_sub_dag.with_for_update()).all() for task_instance in tis_altered: task_instance.set_state(state, session=session) session.flush() else: tis_altered = session.scalars(qry_dag).all() - if sub_dag_run_ids: - qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) - tis_altered += session.scalars(qry_sub_dag).all() return tis_altered -def all_subdag_tasks_query( - sub_dag_run_ids: list[str], - session: SASession, - state: TaskInstanceState, - confirmed_dates: Iterable[datetime], -): - """Get *all* tasks of the sub dags.""" - qry_sub_dag = ( - select(TaskInstance) - .where(TaskInstance.dag_id.in_(sub_dag_run_ids), TaskInstance.execution_date.in_(confirmed_dates)) - .where(or_(TaskInstance.state.is_(None), TaskInstance.state != state)) - ) - return qry_sub_dag - - def get_all_dag_task_query( dag: DAG, session: SASession, @@ -208,71 +173,6 @@ def get_all_dag_task_query( return qry_dag -def _iter_subdag_run_ids( - dag: DAG, - session: SASession, - state: DagRunState, - task_ids: list[str], - commit: bool, - confirmed_infos: Iterable[_DagRunInfo], -) -> Iterator[str]: - """ - Go through subdag operators and create dag runs. - - We only work within the scope of the subdag. A subdag does not propagate to - its parent DAG, but parent propagates to subdags. - """ - dags = [dag] - while dags: - current_dag = dags.pop() - for task_id in task_ids: - if not current_dag.has_task(task_id): - continue - - current_task = current_dag.get_task(task_id) - if isinstance(current_task, SubDagOperator) or current_task.task_type == "SubDagOperator": - # this works as a kind of integrity check - # it creates missing dag runs for subdag operators, - # maybe this should be moved to dagrun.verify_integrity - if TYPE_CHECKING: - assert current_task.subdag - dag_runs = _create_dagruns( - current_task.subdag, - infos=confirmed_infos, - state=DagRunState.RUNNING, - run_type=DagRunType.BACKFILL_JOB, - ) - - verify_dagruns(dag_runs, commit, state, session, current_task) - - dags.append(current_task.subdag) - yield current_task.subdag.dag_id - - -def verify_dagruns( - dag_runs: Iterable[DagRun], - commit: bool, - state: DagRunState, - session: SASession, - current_task: Operator, -): - """ - Verify integrity of dag_runs. - - :param dag_runs: dag runs to verify - :param commit: whether dag runs state should be updated - :param state: state of the dag_run to set if commit is True - :param session: session to use - :param current_task: current task - """ - for dag_run in dag_runs: - dag_run.dag = current_task.subdag - dag_run.verify_integrity() - if commit: - dag_run.state = state - session.merge(dag_run) - - def _iter_existing_dag_run_infos(dag: DAG, run_ids: list[str], session: SASession) -> Iterator[_DagRunInfo]: for dag_run in DagRun.find(dag_id=dag.dag_id, run_id=run_ids, session=session): dag_run.dag = dag diff --git a/airflow/api/common/trigger_dag.py b/airflow/api/common/trigger_dag.py index f22755ec640e..70fb999f542a 100644 --- a/airflow/api/common/trigger_dag.py +++ b/airflow/api/common/trigger_dag.py @@ -43,7 +43,7 @@ def _trigger_dag( conf: dict | str | None = None, execution_date: datetime | None = None, replace_microseconds: bool = True, -) -> list[DagRun | None]: +) -> DagRun | None: """ Triggers DAG run. @@ -90,21 +90,17 @@ def _trigger_dag( if conf: run_conf = conf if isinstance(conf, dict) else json.loads(conf) - dag_runs = [] - dags_to_run = [dag, *dag.subdags] - for _dag in dags_to_run: - dag_run = _dag.create_dagrun( - run_id=run_id, - execution_date=execution_date, - state=DagRunState.QUEUED, - conf=run_conf, - external_trigger=True, - dag_hash=dag_bag.dags_hash.get(dag_id), - data_interval=data_interval, - ) - dag_runs.append(dag_run) + dag_run = dag.create_dagrun( + run_id=run_id, + execution_date=execution_date, + state=DagRunState.QUEUED, + conf=run_conf, + external_trigger=True, + dag_hash=dag_bag.dags_hash.get(dag_id), + data_interval=data_interval, + ) - return dag_runs + return dag_run @internal_api_call @@ -133,7 +129,7 @@ def trigger_dag( raise DagNotFound(f"Dag id {dag_id} not found in DagModel") dagbag = DagBag(dag_folder=dag_model.fileloc, read_dags_from_db=True) - triggers = _trigger_dag( + dr = _trigger_dag( dag_id=dag_id, dag_bag=dagbag, run_id=run_id, @@ -142,4 +138,4 @@ def trigger_dag( replace_microseconds=replace_microseconds, ) - return triggers[0] if triggers else None + return dr if dr else None diff --git a/airflow/api_connexion/endpoints/dag_endpoint.py b/airflow/api_connexion/endpoints/dag_endpoint.py index 1895bfeaec76..749c3bf14ddf 100644 --- a/airflow/api_connexion/endpoints/dag_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_endpoint.py @@ -106,7 +106,7 @@ def get_dags( ) -> APIResponse: """Get all DAGs.""" allowed_attrs = ["dag_id"] - dags_query = select(DagModel).where(~DagModel.is_subdag) + dags_query = select(DagModel) if only_active: dags_query = dags_query.where(DagModel.is_active) if paused is not None: @@ -179,10 +179,9 @@ def patch_dags(limit, session, offset=0, only_active=True, tags=None, dag_id_pat update_mask = update_mask[0] patch_body_[update_mask] = patch_body[update_mask] patch_body = patch_body_ + dags_query = select(DagModel) if only_active: - dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) - else: - dags_query = select(DagModel).where(~DagModel.is_subdag) + dags_query = dags_query.where(DagModel.is_active) if dag_id_pattern == "~": dag_id_pattern = "%" diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py index 96fdd42fa03b..acf4b4449392 100644 --- a/airflow/api_connexion/endpoints/dag_run_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_run_endpoint.py @@ -425,8 +425,6 @@ def clear_dag_run(*, dag_id: str, dag_run_id: str, session: Session = NEW_SESSIO start_date=start_date, end_date=end_date, task_ids=None, - include_subdags=True, - include_parentdag=True, only_failed=False, dry_run=True, ) @@ -438,8 +436,6 @@ def clear_dag_run(*, dag_id: str, dag_run_id: str, session: Session = NEW_SESSIO start_date=start_date, end_date=end_date, task_ids=None, - include_subdags=True, - include_parentdag=True, only_failed=False, ) dag_run = session.execute(select(DagRun).where(DagRun.id == dag_run.id)).scalar_one() diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 0394da4f466c..fbd9a64eacd8 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3106,11 +3106,6 @@ components: Human centric display text for the DAG. *New in version 2.9.0* - root_dag_id: - type: string - readOnly: true - nullable: true - description: If the DAG is SubDAG then it is the top level DAG identifier. Otherwise, null. is_paused: type: boolean nullable: true @@ -3125,10 +3120,6 @@ components: nullable: true readOnly: true type: boolean - is_subdag: - description: Whether the DAG is SubDAG. - type: boolean - readOnly: true last_parsed_time: type: string format: date-time @@ -4903,14 +4894,6 @@ components: type: boolean default: false - include_subdags: - description: Clear tasks in subdags and clear external tasks indicated by ExternalTaskMarker. - type: boolean - - include_parentdag: - description: Clear tasks in the parent dag of the subdag. - type: boolean - reset_dag_runs: description: Set state of DAG runs to RUNNING. type: boolean diff --git a/airflow/api_connexion/schemas/dag_schema.py b/airflow/api_connexion/schemas/dag_schema.py index 799e4092ccae..32eca2f0b890 100644 --- a/airflow/api_connexion/schemas/dag_schema.py +++ b/airflow/api_connexion/schemas/dag_schema.py @@ -51,10 +51,8 @@ class Meta: dag_id = auto_field(dump_only=True) dag_display_name = fields.String(attribute="dag_display_name", dump_only=True) - root_dag_id = auto_field(dump_only=True) is_paused = auto_field() is_active = auto_field(dump_only=True) - is_subdag = auto_field(dump_only=True) last_parsed_time = auto_field(dump_only=True) last_pickled = auto_field(dump_only=True) last_expired = auto_field(dump_only=True) diff --git a/airflow/api_connexion/schemas/task_instance_schema.py b/airflow/api_connexion/schemas/task_instance_schema.py index 5d0eb7209127..74cd0585dcda 100644 --- a/airflow/api_connexion/schemas/task_instance_schema.py +++ b/airflow/api_connexion/schemas/task_instance_schema.py @@ -177,8 +177,6 @@ class ClearTaskInstanceFormSchema(Schema): end_date = fields.DateTime(load_default=None, validate=validate_istimezone) only_failed = fields.Boolean(load_default=True) only_running = fields.Boolean(load_default=False) - include_subdags = fields.Boolean(load_default=False) - include_parentdag = fields.Boolean(load_default=False) reset_dag_runs = fields.Boolean(load_default=False) task_ids = fields.List(fields.String(), validate=validate.Length(min=1)) dag_run_id = fields.Str(load_default=None) diff --git a/airflow/api_connexion/schemas/task_schema.py b/airflow/api_connexion/schemas/task_schema.py index 03bf4b59ef2e..e78c3ef4af1b 100644 --- a/airflow/api_connexion/schemas/task_schema.py +++ b/airflow/api_connexion/schemas/task_schema.py @@ -26,7 +26,6 @@ TimeDeltaSchema, WeightRuleField, ) -from airflow.api_connexion.schemas.dag_schema import DAGSchema from airflow.models.mappedoperator import MappedOperator if TYPE_CHECKING: @@ -61,7 +60,6 @@ class TaskSchema(Schema): ui_color = ColorField(dump_only=True) ui_fgcolor = ColorField(dump_only=True) template_fields = fields.List(fields.String(), dump_only=True) - sub_dag = fields.Nested(DAGSchema, dump_only=True) downstream_task_ids = fields.List(fields.String(), dump_only=True) params = fields.Method("_get_params", dump_only=True) is_mapped = fields.Method("_get_is_mapped", dump_only=True) diff --git a/airflow/api_internal/endpoints/rpc_api_endpoint.py b/airflow/api_internal/endpoints/rpc_api_endpoint.py index ad65157ef941..e4a5069b29bc 100644 --- a/airflow/api_internal/endpoints/rpc_api_endpoint.py +++ b/airflow/api_internal/endpoints/rpc_api_endpoint.py @@ -126,9 +126,9 @@ def initialize_method_map() -> dict[str, Callable]: # XCom.get_many, # Not supported because it returns query XCom.clear, XCom.set, - Variable.set, - Variable.update, - Variable.delete, + Variable._set, + Variable._update, + Variable._delete, DAG.fetch_callback, DAG.fetch_dagrun, DagRun.fetch_task_instances, @@ -237,7 +237,8 @@ def internal_airflow_api(body: dict[str, Any]) -> APIResponse: response = json.dumps(output_json) if output_json is not None else None log.info("Sending response: %s", response) return Response(response=response, headers={"Content-Type": "application/json"}) - except AirflowException as e: # In case of AirflowException transport the exception class back to caller + # In case of AirflowException or other selective known types, transport the exception class back to caller + except (KeyError, AttributeError, AirflowException) as e: exception_json = BaseSerialization.serialize(e, use_pydantic_models=True) response = json.dumps(exception_json) log.info("Sending exception response: %s", response) diff --git a/airflow/api_internal/internal_api_call.py b/airflow/api_internal/internal_api_call.py index fc0945b3c0fe..8838377877be 100644 --- a/airflow/api_internal/internal_api_call.py +++ b/airflow/api_internal/internal_api_call.py @@ -159,7 +159,7 @@ def wrapper(*args, **kwargs): if result is None or result == b"": return None result = BaseSerialization.deserialize(json.loads(result), use_pydantic_models=True) - if isinstance(result, AirflowException): + if isinstance(result, (KeyError, AttributeError, AirflowException)): raise result return result diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index 7814c86bd6d1..269916548401 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -449,12 +449,6 @@ def string_lower_type(val): ARG_ONLY_FAILED = Arg(("-f", "--only-failed"), help="Only failed jobs", action="store_true") ARG_ONLY_RUNNING = Arg(("-r", "--only-running"), help="Only running jobs", action="store_true") ARG_DOWNSTREAM = Arg(("-d", "--downstream"), help="Include downstream tasks", action="store_true") -ARG_EXCLUDE_SUBDAGS = Arg(("-x", "--exclude-subdags"), help="Exclude subdags", action="store_true") -ARG_EXCLUDE_PARENTDAG = Arg( - ("-X", "--exclude-parentdag"), - help="Exclude ParentDAGS if the task cleared is a part of a SubDAG", - action="store_true", -) ARG_DAG_REGEX = Arg( ("-R", "--dag-regex"), help="Search dag_id as regex instead of exact string", action="store_true" ) @@ -1330,8 +1324,6 @@ class GroupCommand(NamedTuple): ARG_YES, ARG_ONLY_FAILED, ARG_ONLY_RUNNING, - ARG_EXCLUDE_SUBDAGS, - ARG_EXCLUDE_PARENTDAG, ARG_DAG_REGEX, ARG_VERBOSE, ), diff --git a/airflow/cli/commands/dag_command.py b/airflow/cli/commands/dag_command.py index dac61d0da597..2f300ebef214 100644 --- a/airflow/cli/commands/dag_command.py +++ b/airflow/cli/commands/dag_command.py @@ -101,7 +101,6 @@ def _run_dag_backfill(dags: list[DAG], args) -> None: start_date=args.start_date, end_date=args.end_date, confirm_prompt=not args.yes, - include_subdags=True, dag_run_state=DagRunState.QUEUED, ) @@ -334,10 +333,8 @@ def _get_dagbag_dag_details(dag: DAG) -> dict: return { "dag_id": dag.dag_id, "dag_display_name": dag.dag_display_name, - "root_dag_id": dag.parent_dag.dag_id if dag.parent_dag else None, "is_paused": dag.get_is_paused(), "is_active": dag.get_is_active(), - "is_subdag": dag.is_subdag, "last_parsed_time": None, "last_pickled": None, "last_expired": None, diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 6e0fc80fbb30..c4c6db386a4e 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -764,8 +764,6 @@ def task_clear(args) -> None: only_failed=args.only_failed, only_running=args.only_running, confirm_prompt=not args.yes, - include_subdags=not args.exclude_subdags, - include_parentdag=not args.exclude_parentdag, ) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 782c5217c7dc..007ceadd6788 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -682,7 +682,7 @@ database: Improper configuration of sql_alchemy_session_maker can lead to serious issues, including data corruption, unrecoverable application crashes. Please review the SQLAlchemy documentation for detailed guidance on proper configuration and best practices. - version_added: 2.9.4 + version_added: 2.10.0 type: string example: 'airflow_local_settings._sessionmaker' default: ~ @@ -1348,27 +1348,6 @@ secrets: type: integer example: ~ default: "900" -cli: - description: ~ - options: - api_client: - description: | - In what way should the cli access the API. The LocalClient will use the - database directly, while the json_client will use the api running on the - webserver - version_added: ~ - type: string - example: ~ - default: "airflow.api.client.local_client" - endpoint_url: - description: | - If you set web_server_url_prefix, do NOT forget to append it here, ex: - ``endpoint_url = http://localhost:8080/myroot`` - So api will look like: ``http://localhost:8080/myroot/api/experimental/...`` - version_added: ~ - type: string - example: ~ - default: "http://localhost:8080" debug: description: ~ options: @@ -1383,24 +1362,6 @@ debug: api: description: ~ options: - enable_experimental_api: - description: | - Enables the deprecated experimental API. Please note that these API endpoints do not have - access control. An authenticated user has full access. - - .. warning:: - - This `Experimental REST API - `__ is - deprecated since version 2.0. Please consider using - `the Stable REST API - `__. - For more information on migration, see - `RELEASE_NOTES.rst `_ - version_added: 2.0.0 - type: boolean - example: ~ - default: "False" auth_backends: description: | Comma separated list of auth backends to authenticate users of the API. See @@ -2213,21 +2174,6 @@ smtp: type: string example: ~ default: "False" - smtp_user: - description: | - Username to authenticate when connecting to smtp server. - version_added: ~ - type: string - example: "airflow" - default: ~ - smtp_password: - description: | - Password to authenticate when connecting to smtp server. - version_added: ~ - type: string - sensitive: true - example: "airflow" - default: ~ smtp_port: description: | Defines the port number on which Airflow connects to the SMTP server to send email notifications. @@ -2472,11 +2418,8 @@ scheduler: default: "True" max_tis_per_query: description: | - This changes the batch size of queries in the scheduling main loop. - This should not be greater than ``[core] parallelism``. - If this is too high, SQL query performance may be impacted by - complexity of query predicate, and/or excessive locking. - Additionally, you may hit the maximum allowable query length for your db. + This determines the number of task instances to be evaluated for scheduling + during each scheduler loop. Set this to 0 to use the value of ``[core] parallelism`` version_added: ~ type: integer diff --git a/airflow/config_templates/unit_tests.cfg b/airflow/config_templates/unit_tests.cfg index 42055b9d9c7d..af32b79f4b0a 100644 --- a/airflow/config_templates/unit_tests.cfg +++ b/airflow/config_templates/unit_tests.cfg @@ -68,8 +68,6 @@ celery_logging_level = INFO [smtp] # Used as default values for SMTP unit tests -smtp_user = airflow -smtp_password = airflow smtp_mail_from = airflow@example.com [api] diff --git a/airflow/configuration.py b/airflow/configuration.py index 618f5185db7d..27534f78e81f 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -363,7 +363,6 @@ def sensitive_config_values(self) -> Set[tuple[str, str]]: # noqa: UP006 ("metrics", "statsd_datadog_metrics_tags"): ("scheduler", "statsd_datadog_metrics_tags", "2.6.0"), ("metrics", "statsd_custom_client_path"): ("scheduler", "statsd_custom_client_path", "2.0.0"), ("scheduler", "parsing_processes"): ("scheduler", "max_threads", "1.10.14"), - ("scheduler", "scheduler_idle_sleep_time"): ("scheduler", "processor_poll_interval", "2.2.0"), ("operators", "default_queue"): ("celery", "default_queue", "2.1.0"), ("core", "hide_sensitive_var_conn_fields"): ("admin", "hide_sensitive_variable_fields", "2.1.0"), ("core", "sensitive_var_conn_names"): ("admin", "sensitive_variable_fields", "2.1.0"), diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index c03bc074d0ab..1f12f3cb033b 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -53,7 +53,7 @@ from airflow.models.serialized_dag import SerializedDagModel from airflow.secrets.cache import SecretCache from airflow.stats import Stats -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano from airflow.utils.file import list_py_file_paths, might_contain_dag @@ -1210,7 +1210,7 @@ def _create_process(file_path, pickle_dags, dag_ids, dag_directory, callback_req callback_requests=callback_requests, ) - @span + @add_span def start_new_processes(self): """Start more processors if we have enough slots and files to process.""" # initialize cache to mutualize calls to Variable.get in DAGs @@ -1248,7 +1248,7 @@ def start_new_processes(self): Stats.gauge("dag_processing.file_path_queue_size", len(self._file_path_queue)) - @span + @add_span def add_new_file_path_to_queue(self): for file_path in self.file_paths: if file_path not in self._file_stats: diff --git a/airflow/dag_processing/processor.py b/airflow/dag_processing/processor.py index 86db0b5b881a..0b19d8f2db76 100644 --- a/airflow/dag_processing/processor.py +++ b/airflow/dag_processing/processor.py @@ -688,8 +688,6 @@ def get_pools(dag) -> dict[str, set[str]]: pool_dict: dict[str, set[str]] = {} for dag in dagbag.dags.values(): pool_dict.update(get_pools(dag)) - for subdag in dag.subdags: - pool_dict.update(get_pools(subdag)) dag_ids = {dag.dag_id for dag in dagbag.dags.values()} return DagFileProcessor._validate_task_pools_and_update_dag_warnings(pool_dict, dag_ids) diff --git a/airflow/decorators/__init__.pyi b/airflow/decorators/__init__.pyi index faf77e8240d6..7dd887431c6f 100644 --- a/airflow/decorators/__init__.pyi +++ b/airflow/decorators/__init__.pyi @@ -23,6 +23,7 @@ from __future__ import annotations from datetime import timedelta from typing import Any, Callable, Collection, Container, Iterable, Mapping, TypeVar, overload +from docker.types import Mount from kubernetes.client import models as k8s from airflow.decorators.base import FParams, FReturn, Task, TaskDecorator, _TaskDecorator @@ -125,6 +126,7 @@ class TaskDecoratorCollection: env_vars: dict[str, str] | None = None, inherit_env: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ) -> TaskDecorator: """Create a decorator to convert the decorated callable to a virtual environment task. @@ -176,6 +178,7 @@ class TaskDecoratorCollection: :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize the args and result (pickle is default). This allows more complex types but requires you to include dill in your requirements. + :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @overload def virtualenv(self, python_callable: Callable[FParams, FReturn]) -> Task[FParams, FReturn]: ... @@ -192,6 +195,7 @@ class TaskDecoratorCollection: env_vars: dict[str, str] | None = None, inherit_env: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ) -> TaskDecorator: """Create a decorator to convert the decorated callable to a virtual environment task. @@ -225,6 +229,7 @@ class TaskDecoratorCollection: :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize the args and result (pickle is default). This allows more complex types but requires you to include dill in your requirements. + :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @overload def branch( # type: ignore[misc] @@ -258,6 +263,7 @@ class TaskDecoratorCollection: venv_cache_path: None | str = None, show_return_value_in_logs: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ) -> TaskDecorator: """Create a decorator to wrap the decorated callable into a BranchPythonVirtualenvOperator. @@ -299,6 +305,7 @@ class TaskDecoratorCollection: :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize the args and result (pickle is default). This allows more complex types but requires you to include dill in your requirements. + :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @overload def branch_virtualenv(self, python_callable: Callable[FParams, FReturn]) -> Task[FParams, FReturn]: ... @@ -373,8 +380,9 @@ class TaskDecoratorCollection: self, *, multiple_outputs: bool | None = None, - use_dill: bool = False, # Added by _DockerDecoratedOperator. python_command: str = "python3", + serializer: Literal["pickle", "cloudpickle", "dill"] | None = None, + use_dill: bool = False, # Added by _DockerDecoratedOperator. # 'command', 'retrieve_output', and 'retrieve_output_path' are filled by # _DockerDecoratedOperator. image: str, @@ -398,7 +406,7 @@ class TaskDecoratorCollection: mount_tmp_dir: bool = True, tmp_dir: str = "/tmp/airflow", user: str | int | None = None, - mounts: list[str] | None = None, + mounts: list[Mount] | None = None, entrypoint: str | list[str] | None = None, working_dir: str | None = None, xcom_all: bool = False, @@ -426,8 +434,17 @@ class TaskDecoratorCollection: :param multiple_outputs: If set, function return value will be unrolled to multiple XCom values. Dict will unroll to XCom values with keys as XCom keys. Defaults to False. - :param use_dill: Whether to use dill or pickle for serialization :param python_command: Python command for executing functions, Default: python3 + :param serializer: Which serializer use to serialize the args and result. It can be one of the following: + + - ``"pickle"``: (default) Use pickle for serialization. Included in the Python Standard Library. + - ``"cloudpickle"``: Use cloudpickle for serialize more complex types, + this requires to include cloudpickle in your requirements. + - ``"dill"``: Use dill for serialize more complex types, + this requires to include dill in your requirements. + :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize + the args and result (pickle is default). This allows more complex types + but requires you to include dill in your requirements. :param image: Docker image from which to create the container. If image tag is omitted, "latest" will be used. :param api_version: Remote API version. Set to ``auto`` to automatically diff --git a/airflow/example_dags/example_dynamic_task_mapping.py b/airflow/example_dags/example_dynamic_task_mapping.py index 21f5a03ae8af..03a77b001850 100644 --- a/airflow/example_dags/example_dynamic_task_mapping.py +++ b/airflow/example_dags/example_dynamic_task_mapping.py @@ -24,7 +24,7 @@ from airflow.decorators import task from airflow.models.dag import DAG -with DAG(dag_id="example_dynamic_task_mapping", start_date=datetime(2022, 3, 4)) as dag: +with DAG(dag_id="example_dynamic_task_mapping", schedule=None, start_date=datetime(2022, 3, 4)) as dag: @task def add_one(x: int): diff --git a/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py b/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py index d639f345aa61..3d42ac47b565 100644 --- a/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py +++ b/airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py @@ -53,6 +53,7 @@ def execute(self, context): with DAG( dag_id="example_dynamic_task_mapping_with_no_taskflow_operators", + schedule=None, start_date=datetime(2022, 3, 4), catchup=False, ): diff --git a/airflow/example_dags/example_python_context_decorator.py b/airflow/example_dags/example_python_context_decorator.py new file mode 100644 index 000000000000..497ee08e17ce --- /dev/null +++ b/airflow/example_dags/example_python_context_decorator.py @@ -0,0 +1,92 @@ +# +# 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 DAG demonstrating the usage of the PythonOperator with `get_current_context()` to get the current context. + +Also, demonstrates the usage of the TaskFlow API. +""" + +from __future__ import annotations + +import sys + +import pendulum + +from airflow.decorators import dag, task + +SOME_EXTERNAL_PYTHON = sys.executable + + +@dag( + schedule=None, + start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), + catchup=False, + tags=["example"], +) +def example_python_context_decorator(): + # [START get_current_context] + @task(task_id="print_the_context") + def print_context() -> str: + """Print the Airflow context.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context = print_context() + # [END get_current_context] + + # [START get_current_context_venv] + @task.virtualenv(task_id="print_the_context_venv", use_airflow_context=True) + def print_context_venv() -> str: + """Print the Airflow context in venv.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context_venv = print_context_venv() + # [END get_current_context_venv] + + # [START get_current_context_external] + @task.external_python( + task_id="print_the_context_external", python=SOME_EXTERNAL_PYTHON, use_airflow_context=True + ) + def print_context_external() -> str: + """Print the Airflow context in external python.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context_external = print_context_external() + # [END get_current_context_external] + + _ = print_the_context >> [print_the_context_venv, print_the_context_external] + + +example_python_context_decorator() diff --git a/airflow/example_dags/example_python_context_operator.py b/airflow/example_dags/example_python_context_operator.py new file mode 100644 index 000000000000..f1b76c527cfd --- /dev/null +++ b/airflow/example_dags/example_python_context_operator.py @@ -0,0 +1,91 @@ +# +# 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 DAG demonstrating the usage of the PythonOperator with `get_current_context()` to get the current context. + +Also, demonstrates the usage of the classic Python operators. +""" + +from __future__ import annotations + +import sys + +import pendulum + +from airflow import DAG +from airflow.operators.python import ExternalPythonOperator, PythonOperator, PythonVirtualenvOperator + +SOME_EXTERNAL_PYTHON = sys.executable + +with DAG( + dag_id="example_python_context_operator", + schedule=None, + start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), + catchup=False, + tags=["example"], +) as dag: + # [START get_current_context] + def print_context() -> str: + """Print the Airflow context.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context = PythonOperator(task_id="print_the_context", python_callable=print_context) + # [END get_current_context] + + # [START get_current_context_venv] + def print_context_venv() -> str: + """Print the Airflow context in venv.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context_venv = PythonVirtualenvOperator( + task_id="print_the_context_venv", python_callable=print_context_venv, use_airflow_context=True + ) + # [END get_current_context_venv] + + # [START get_current_context_external] + def print_context_external() -> str: + """Print the Airflow context in external python.""" + from pprint import pprint + + from airflow.operators.python import get_current_context + + context = get_current_context() + pprint(context) + return "Whatever you return gets printed in the logs" + + print_the_context_external = ExternalPythonOperator( + task_id="print_the_context_external", + python_callable=print_context_external, + python=SOME_EXTERNAL_PYTHON, + use_airflow_context=True, + ) + # [END get_current_context_external] + + _ = print_the_context >> [print_the_context_venv, print_the_context_external] diff --git a/airflow/example_dags/example_setup_teardown.py b/airflow/example_dags/example_setup_teardown.py index dd61fcdc0197..9fab87df7568 100644 --- a/airflow/example_dags/example_setup_teardown.py +++ b/airflow/example_dags/example_setup_teardown.py @@ -27,6 +27,7 @@ with DAG( dag_id="example_setup_teardown", + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_setup_teardown_taskflow.py b/airflow/example_dags/example_setup_teardown_taskflow.py index 21c05e29c04a..6fec9f9a4787 100644 --- a/airflow/example_dags/example_setup_teardown_taskflow.py +++ b/airflow/example_dags/example_setup_teardown_taskflow.py @@ -26,6 +26,7 @@ with DAG( dag_id="example_setup_teardown_taskflow", + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_short_circuit_decorator.py b/airflow/example_dags/example_short_circuit_decorator.py index 00d6cd718675..2d82eeed069b 100644 --- a/airflow/example_dags/example_short_circuit_decorator.py +++ b/airflow/example_dags/example_short_circuit_decorator.py @@ -26,7 +26,7 @@ from airflow.utils.trigger_rule import TriggerRule -@dag(start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"]) +@dag(schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"]) def example_short_circuit_decorator(): # [START howto_operator_short_circuit] @task.short_circuit() diff --git a/airflow/example_dags/example_short_circuit_operator.py b/airflow/example_dags/example_short_circuit_operator.py index 9dfee6470724..3941ff17f95a 100644 --- a/airflow/example_dags/example_short_circuit_operator.py +++ b/airflow/example_dags/example_short_circuit_operator.py @@ -29,6 +29,7 @@ with DAG( dag_id="example_short_circuit_operator", + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_skip_dag.py b/airflow/example_dags/example_skip_dag.py index 72ff242831aa..2655394c6f6f 100644 --- a/airflow/example_dags/example_skip_dag.py +++ b/airflow/example_dags/example_skip_dag.py @@ -19,6 +19,7 @@ from __future__ import annotations +import datetime from typing import TYPE_CHECKING import pendulum @@ -63,6 +64,7 @@ def create_test_pipeline(suffix, trigger_rule): with DAG( dag_id="example_skip_dag", + schedule=datetime.timedelta(days=1), start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_subdag_operator.py b/airflow/example_dags/example_subdag_operator.py deleted file mode 100644 index 196012024c2c..000000000000 --- a/airflow/example_dags/example_subdag_operator.py +++ /dev/null @@ -1,70 +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. -"""Example DAG demonstrating the usage of the SubDagOperator.""" - -from __future__ import annotations - -import warnings - -with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message=r"This class is deprecated\. Please use `airflow\.utils\.task_group\.TaskGroup`\.", - ) - - # [START example_subdag_operator] - import datetime - - from airflow.example_dags.subdags.subdag import subdag - from airflow.models.dag import DAG - from airflow.operators.empty import EmptyOperator - from airflow.operators.subdag import SubDagOperator - - DAG_NAME = "example_subdag_operator" - - with DAG( - dag_id=DAG_NAME, - default_args={"retries": 2}, - start_date=datetime.datetime(2022, 1, 1), - schedule="@once", - tags=["example"], - ) as dag: - start = EmptyOperator( - task_id="start", - ) - - section_1 = SubDagOperator( - task_id="section-1", - subdag=subdag(DAG_NAME, "section-1", dag.default_args), - ) - - some_other_task = EmptyOperator( - task_id="some-other-task", - ) - - section_2 = SubDagOperator( - task_id="section-2", - subdag=subdag(DAG_NAME, "section-2", dag.default_args), - ) - - end = EmptyOperator( - task_id="end", - ) - - start >> section_1 >> some_other_task >> section_2 >> end - # [END example_subdag_operator] diff --git a/airflow/example_dags/example_task_group.py b/airflow/example_dags/example_task_group.py index 85a6f114ee37..6435a912cc41 100644 --- a/airflow/example_dags/example_task_group.py +++ b/airflow/example_dags/example_task_group.py @@ -29,6 +29,7 @@ # [START howto_task_group] with DAG( dag_id="example_task_group", + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/example_task_group_decorator.py b/airflow/example_dags/example_task_group_decorator.py index 56d4decf63a8..ce4a0e33b8c2 100644 --- a/airflow/example_dags/example_task_group_decorator.py +++ b/airflow/example_dags/example_task_group_decorator.py @@ -67,6 +67,7 @@ def task_group_function(value: int) -> None: # Executing Tasks and TaskGroups with DAG( dag_id="example_task_group_decorator", + schedule=None, start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), catchup=False, tags=["example"], diff --git a/airflow/example_dags/subdags/__init__.py b/airflow/example_dags/subdags/__init__.py deleted file mode 100644 index 217e5db96078..000000000000 --- a/airflow/example_dags/subdags/__init__.py +++ /dev/null @@ -1,17 +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. diff --git a/airflow/example_dags/subdags/subdag.py b/airflow/example_dags/subdags/subdag.py deleted file mode 100644 index 748582f4b84a..000000000000 --- a/airflow/example_dags/subdags/subdag.py +++ /dev/null @@ -1,56 +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. -"""Helper function to generate a DAG and operators given some arguments.""" - -from __future__ import annotations - -# [START subdag] -import pendulum - -from airflow.models.dag import DAG -from airflow.operators.empty import EmptyOperator - - -def subdag(parent_dag_name, child_dag_name, args) -> DAG: - """ - Generate a DAG to be used as a subdag. - - :param str parent_dag_name: Id of the parent DAG - :param str child_dag_name: Id of the child DAG - :param dict args: Default arguments to provide to the subdag - :return: DAG to use as a subdag - """ - dag_subdag = DAG( - dag_id=f"{parent_dag_name}.{child_dag_name}", - default_args=args, - start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, - schedule="@daily", - ) - - for i in range(5): - EmptyOperator( - task_id=f"{child_dag_name}-task-{i + 1}", - default_args=args, - dag=dag_subdag, - ) - - return dag_subdag - - -# [END subdag] diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py index 7c936a16b864..13a83393a912 100644 --- a/airflow/executors/__init__.py +++ b/airflow/executors/__init__.py @@ -14,52 +14,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -"""Executors.""" - -from __future__ import annotations - -from airflow.utils.deprecation_tools import add_deprecated_classes - -__deprecated_classes = { - "celery_executor": { - "app": "airflow.providers.celery.executors.celery_executor_utils.app", - "CeleryExecutor": "airflow.providers.celery.executors.celery_executor.CeleryExecutor", - }, - "celery_kubernetes_executor": { - "CeleryKubernetesExecutor": "airflow.providers.celery.executors." - "celery_kubernetes_executor.CeleryKubernetesExecutor", - }, - "dask_executor": { - "DaskExecutor": "airflow.providers.daskexecutor.executors.dask_executor.DaskExecutor", - }, - "kubernetes_executor": { - "KubernetesExecutor": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor.KubernetesExecutor", - }, - "kubernetes_executor_types": { - "ALL_NAMESPACES": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor_types.ALL_NAMESPACES", - "POD_EXECUTOR_DONE_KEY": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor_types.POD_EXECUTOR_DONE_KEY", - }, - "kubernetes_executor_utils": { - "AirflowKubernetesScheduler": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor_utils.AirflowKubernetesScheduler", - "KubernetesJobWatcher": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor_utils.KubernetesJobWatcher", - "ResourceVersion": "airflow.providers.cncf.kubernetes." - "executors.kubernetes_executor_utils.ResourceVersion", - }, - "local_kubernetes_executor": { - "LocalKubernetesExecutor": "airflow.providers.cncf.kubernetes.executors.LocalKubernetesExecutor", - }, -} - -add_deprecated_classes( - __deprecated_classes, - __name__, - {}, - "For Celery executors, the `celery` provider should be >= 3.3.0. " - "For Kubernetes executors, the `cncf.kubernetes` provider should be >= 7.4.0 for that. " - "For Dask executors, any version of `daskexecutor` provider is needed.", -) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index dd0b8a66d285..ad7690b3f6e1 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -20,7 +20,6 @@ import logging import sys -import warnings from collections import defaultdict, deque from dataclasses import dataclass, field from typing import TYPE_CHECKING, Any, List, Optional, Sequence, Tuple @@ -29,12 +28,11 @@ from airflow.cli.cli_config import DefaultHelpParser from airflow.configuration import conf -from airflow.exceptions import RemovedInAirflow3Warning from airflow.executors.executor_loader import ExecutorLoader from airflow.models import Log from airflow.stats import Stats from airflow.traces import NO_TRACE_ID -from airflow.traces.tracer import Trace, gen_context, span +from airflow.traces.tracer import Trace, add_span, gen_context from airflow.traces.utils import gen_span_id_from_ti_key, gen_trace_id from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState @@ -228,7 +226,7 @@ def sync(self) -> None: Executors should override this to perform gather statuses. """ - @span + @add_span def heartbeat(self) -> None: """Heartbeat sent to trigger new jobs.""" if not self.parallelism: @@ -321,7 +319,7 @@ def order_queued_tasks_by_priority(self) -> list[tuple[TaskInstanceKey, QueuedTa reverse=True, ) - @span + @add_span def trigger_tasks(self, open_slots: int) -> None: """ Initiate async execution of the queued tasks, up to the number of available slots. @@ -381,7 +379,7 @@ def trigger_tasks(self, open_slots: int) -> None: if task_tuples: self._process_tasks(task_tuples) - @span + @add_span def _process_tasks(self, task_tuples: list[TaskTuple]) -> None: for key, command, queue, executor_config in task_tuples: task_instance = self.queued_tasks[key][3] # TaskInstance in fourth element @@ -467,7 +465,7 @@ def success(self, key: TaskInstanceKey, info=None) -> None: span.set_attribute("dag_id", key.dag_id) span.set_attribute("run_id", key.run_id) span.set_attribute("task_id", key.task_id) - span.set_attribute("try_number", key.try_number - 1) + span.set_attribute("try_number", key.try_number) self.change_state(key, TaskInstanceState.SUCCESS, info) @@ -584,22 +582,6 @@ def slots_occupied(self): """Number of tasks this executor instance is currently managing.""" return len(self.running) + len(self.queued_tasks) - @staticmethod - def validate_command(command: list[str]) -> None: - """ - Back-compat method to Check if the command to execute is airflow command. - - :param command: command to check - """ - warnings.warn( - """ - The `validate_command` method is deprecated. Please use ``validate_airflow_tasks_run_command`` - """, - RemovedInAirflow3Warning, - stacklevel=2, - ) - BaseExecutor.validate_airflow_tasks_run_command(command) - @staticmethod def validate_airflow_tasks_run_command(command: list[str]) -> tuple[str | None, str | None]: """ diff --git a/airflow/executors/local_executor.py b/airflow/executors/local_executor.py index afa51b1d86bb..f28e525ec3ac 100644 --- a/airflow/executors/local_executor.py +++ b/airflow/executors/local_executor.py @@ -39,7 +39,7 @@ from airflow import settings from airflow.exceptions import AirflowException from airflow.executors.base_executor import PARALLELISM, BaseExecutor -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span from airflow.utils.dag_parsing_context import _airflow_parsing_context_manager from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState @@ -79,7 +79,7 @@ def run(self): setproctitle("airflow worker -- LocalExecutor") return super().run() - @span + @add_span def execute_work(self, key: TaskInstanceKey, command: CommandType) -> None: """ Execute command received and stores result state in queue. @@ -103,7 +103,7 @@ def execute_work(self, key: TaskInstanceKey, command: CommandType) -> None: # Remove the command since the worker is done executing the task setproctitle("airflow worker -- LocalExecutor") - @span + @add_span def _execute_work_in_subprocess(self, command: CommandType) -> TaskInstanceState: try: subprocess.check_call(command, close_fds=True) @@ -112,7 +112,7 @@ def _execute_work_in_subprocess(self, command: CommandType) -> TaskInstanceState self.log.error("Failed to execute task %s.", e) return TaskInstanceState.FAILED - @span + @add_span def _execute_work_in_fork(self, command: CommandType) -> TaskInstanceState: pid = os.fork() if pid: @@ -172,7 +172,7 @@ def __init__( self.key: TaskInstanceKey = key self.command: CommandType = command - @span + @add_span def do_work(self) -> None: self.execute_work(key=self.key, command=self.command) @@ -192,7 +192,7 @@ def __init__(self, task_queue: Queue[ExecutorWorkType], result_queue: Queue[Task super().__init__(result_queue=result_queue) self.task_queue = task_queue - @span + @add_span def do_work(self) -> None: while True: try: @@ -253,7 +253,7 @@ def start(self) -> None: self.executor.workers_used = 0 self.executor.workers_active = 0 - @span + @add_span def execute_async( self, key: TaskInstanceKey, @@ -277,7 +277,7 @@ def execute_async( span.set_attribute("dag_id", key.dag_id) span.set_attribute("run_id", key.run_id) span.set_attribute("task_id", key.task_id) - span.set_attribute("try_number", key.try_number - 1) + span.set_attribute("try_number", key.try_number) span.set_attribute("commands_to_run", str(command)) local_worker = LocalWorker(self.executor.result_queue, key=key, command=command) @@ -329,7 +329,7 @@ def start(self) -> None: for worker in self.executor.workers: worker.start() - @span + @add_span def execute_async( self, key: TaskInstanceKey, @@ -391,7 +391,7 @@ def start(self) -> None: self.impl.start() - @span + @add_span def execute_async( self, key: TaskInstanceKey, diff --git a/airflow/executors/sequential_executor.py b/airflow/executors/sequential_executor.py index 1b145892ebc7..0b4cbdea9dd4 100644 --- a/airflow/executors/sequential_executor.py +++ b/airflow/executors/sequential_executor.py @@ -29,7 +29,7 @@ from typing import TYPE_CHECKING, Any from airflow.executors.base_executor import BaseExecutor -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span if TYPE_CHECKING: from airflow.executors.base_executor import CommandType @@ -60,7 +60,7 @@ def __init__(self): super().__init__() self.commands_to_run = [] - @span + @add_span def execute_async( self, key: TaskInstanceKey, @@ -76,7 +76,7 @@ def execute_async( span.set_attribute("dag_id", key.dag_id) span.set_attribute("run_id", key.run_id) span.set_attribute("task_id", key.task_id) - span.set_attribute("try_number", key.try_number - 1) + span.set_attribute("try_number", key.try_number) span.set_attribute("commands_to_run", str(self.commands_to_run)) def sync(self) -> None: diff --git a/airflow/hooks/__init__.py b/airflow/hooks/__init__.py index 1baff07c0bb1..f1ad214d7148 100644 --- a/airflow/hooks/__init__.py +++ b/airflow/hooks/__init__.py @@ -16,76 +16,3 @@ # specific language governing permissions and limitations # under the License. # fmt:, off -"""Hooks.""" - -from __future__ import annotations - -from airflow.utils.deprecation_tools import add_deprecated_classes - -__deprecated_classes = { - "S3_hook": { - "S3Hook": "airflow.providers.amazon.aws.hooks.s3.S3Hook", - "provide_bucket_name": "airflow.providers.amazon.aws.hooks.s3.provide_bucket_name", - }, - "base_hook": { - "BaseHook": "airflow.hooks.base.BaseHook", - }, - "dbapi_hook": { - "DbApiHook": "airflow.providers.common.sql.hooks.sql.DbApiHook", - }, - "docker_hook": { - "DockerHook": "airflow.providers.docker.hooks.docker.DockerHook", - }, - "druid_hook": { - "DruidDbApiHook": "airflow.providers.apache.druid.hooks.druid.DruidDbApiHook", - "DruidHook": "airflow.providers.apache.druid.hooks.druid.DruidHook", - }, - "hive_hooks": { - "HIVE_QUEUE_PRIORITIES": "airflow.providers.apache.hive.hooks.hive.HIVE_QUEUE_PRIORITIES", - "HiveCliHook": "airflow.providers.apache.hive.hooks.hive.HiveCliHook", - "HiveMetastoreHook": "airflow.providers.apache.hive.hooks.hive.HiveMetastoreHook", - "HiveServer2Hook": "airflow.providers.apache.hive.hooks.hive.HiveServer2Hook", - }, - "http_hook": { - "HttpHook": "airflow.providers.http.hooks.http.HttpHook", - }, - "jdbc_hook": { - "JdbcHook": "airflow.providers.jdbc.hooks.jdbc.JdbcHook", - "jaydebeapi": "airflow.providers.jdbc.hooks.jdbc.jaydebeapi", - }, - "mssql_hook": { - "MsSqlHook": "airflow.providers.microsoft.mssql.hooks.mssql.MsSqlHook", - }, - "mysql_hook": { - "MySqlHook": "airflow.providers.mysql.hooks.mysql.MySqlHook", - }, - "oracle_hook": { - "OracleHook": "airflow.providers.oracle.hooks.oracle.OracleHook", - }, - "pig_hook": { - "PigCliHook": "airflow.providers.apache.pig.hooks.pig.PigCliHook", - }, - "postgres_hook": { - "PostgresHook": "airflow.providers.postgres.hooks.postgres.PostgresHook", - }, - "presto_hook": { - "PrestoHook": "airflow.providers.presto.hooks.presto.PrestoHook", - }, - "samba_hook": { - "SambaHook": "airflow.providers.samba.hooks.samba.SambaHook", - }, - "slack_hook": { - "SlackHook": "airflow.providers.slack.hooks.slack.SlackHook", - }, - "sqlite_hook": { - "SqliteHook": "airflow.providers.sqlite.hooks.sqlite.SqliteHook", - }, - "webhdfs_hook": { - "WebHDFSHook": "airflow.providers.apache.hdfs.hooks.webhdfs.WebHDFSHook", - }, - "zendesk_hook": { - "ZendeskHook": "airflow.providers.zendesk.hooks.zendesk.ZendeskHook", - }, -} - -add_deprecated_classes(__deprecated_classes, __name__) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 961c4b7e020b..028d49421967 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -65,7 +65,7 @@ class BackfillJobRunner(BaseJobRunner, LoggingMixin): """ - A backfill job runner consists of a dag or subdag for a specific time range. + A backfill job runner consists of a dag for a specific time range. It triggers a set of task instance runs, in the right order and lasts for as long as it takes for the set of task instance to be completed. @@ -327,7 +327,7 @@ def _manage_executor_state( def _iter_task_needing_expansion() -> Iterator[AbstractOperator]: from airflow.models.mappedoperator import AbstractOperator - for node in self.dag.get_task(ti.task_id, include_subdags=True).iter_mapped_dependants(): + for node in self.dag.get_task(ti.task_id).iter_mapped_dependants(): if isinstance(node, AbstractOperator): yield node else: # A (mapped) task group. All its children need expansion. @@ -359,8 +359,7 @@ def _get_dag_run( """ run_date = dagrun_info.logical_date - # consider max_active_runs but ignore when running subdags - respect_dag_max_active_limit = bool(dag.timetable.can_be_scheduled and not dag.is_subdag) + respect_dag_max_active_limit = bool(dag.timetable.can_be_scheduled) current_active_dag_count = dag.get_num_active_runs(external_trigger=False) @@ -500,7 +499,7 @@ def _process_backfill_task_instances( def _per_task_process(key, ti: TaskInstance, session): ti.refresh_from_db(lock_for_update=True, session=session) - task = self.dag.get_task(ti.task_id, include_subdags=True) + task = self.dag.get_task(ti.task_id) ti.task = task self.log.debug("Task instance to run %s state %s", ti, ti.state) @@ -636,7 +635,7 @@ def _per_task_process(key, ti: TaskInstance, session): ti_status.not_ready.add(key) try: - for task in self.dag.topological_sort(include_subdag_tasks=True): + for task in self.dag.topological_sort(): for key, ti in list(ti_status.to_run.items()): # Attempt to workaround deadlock on backfill by attempting to commit the transaction # state update few times before giving up @@ -839,9 +838,6 @@ def tabulate_ti_keys_set(ti_keys: Iterable[TaskInstanceKey]) -> str: yield "\n\nThese tasks are deadlocked:\n" yield tabulate_ti_keys_set([ti.key for ti in ti_status.deadlocked]) - def _get_dag_with_subdags(self) -> list[DAG]: - return [self.dag, *self.dag.subdags] - @provide_session def _execute_dagruns( self, @@ -863,12 +859,11 @@ def _execute_dagruns( :param session: the current session object """ for dagrun_info in dagrun_infos: - for dag in self._get_dag_with_subdags(): - dag_run = self._get_dag_run(dagrun_info, dag, session=session) - if dag_run is not None: - tis_map = self._task_instances_for_dag_run(dag, dag_run, session=session) - ti_status.active_runs.add(dag_run) - ti_status.to_run.update(tis_map or {}) + dag_run = self._get_dag_run(dagrun_info, self.dag, session=session) + if dag_run is not None: + tis_map = self._task_instances_for_dag_run(self.dag, dag_run, session=session) + ti_status.active_runs.add(dag_run) + ti_status.to_run.update(tis_map or {}) tis_missing_executor = [] for ti in ti_status.to_run.values(): @@ -948,9 +943,8 @@ def _execute(self, session: Session = NEW_SESSION) -> None: return dagrun_infos = [DagRunInfo.interval(dagrun_start_date, dagrun_end_date)] - dag_with_subdags_ids = [d.dag_id for d in self._get_dag_with_subdags()] running_dagruns = DagRun.find( - dag_id=dag_with_subdags_ids, + dag_id=self.dag.dag_id, execution_start_date=self.bf_start_date, execution_end_date=self.bf_end_date, no_backfills=True, diff --git a/airflow/jobs/job.py b/airflow/jobs/job.py index 938482180728..03bf92d4e3d5 100644 --- a/airflow/jobs/job.py +++ b/airflow/jobs/job.py @@ -34,7 +34,7 @@ from airflow.models.base import ID_LEN, Base from airflow.serialization.pydantic.job import JobPydantic from airflow.stats import Stats -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span from airflow.utils import timezone from airflow.utils.helpers import convert_camel_to_snake from airflow.utils.log.logging_mixin import LoggingMixin @@ -459,7 +459,7 @@ def execute_job(job: Job, execute_callable: Callable[[], int | None]) -> int | N return ret -@span +@add_span def perform_heartbeat( job: Job, heartbeat_callback: Callable[[Session], None], only_if_necessary: bool ) -> None: diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 163bf5b71449..baf91dc40ddd 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -40,7 +40,7 @@ from airflow.callbacks.callback_requests import DagCallbackRequest, SlaCallbackRequest, TaskCallbackRequest from airflow.callbacks.pipe_callback_sink import PipeCallbackSink from airflow.configuration import conf -from airflow.exceptions import RemovedInAirflow3Warning, UnknownExecutorException +from airflow.exceptions import UnknownExecutorException from airflow.executors.executor_loader import ExecutorLoader from airflow.jobs.base_job_runner import BaseJobRunner from airflow.jobs.job import Job, perform_heartbeat @@ -61,7 +61,7 @@ from airflow.ti_deps.dependencies_states import EXECUTION_STATES from airflow.timetables.simple import DatasetTriggeredTimetable from airflow.traces import utils as trace_utils -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano from airflow.utils.event_scheduler import EventScheduler @@ -165,7 +165,6 @@ def __init__( scheduler_idle_sleep_time: float = conf.getfloat("scheduler", "scheduler_idle_sleep_time"), do_pickle: bool = False, log: logging.Logger | None = None, - processor_poll_interval: float | None = None, ): super().__init__(job) self.subdir = subdir @@ -174,15 +173,6 @@ def __init__( # number of times. This is only to support testing, and isn't something a user is likely to want to # configure -- they'll want num_runs self.num_times_parse_dags = num_times_parse_dags - if processor_poll_interval: - # TODO: Remove in Airflow 3.0 - warnings.warn( - "The 'processor_poll_interval' parameter is deprecated. " - "Please use 'scheduler_idle_sleep_time'.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - scheduler_idle_sleep_time = processor_poll_interval self._scheduler_idle_sleep_time = scheduler_idle_sleep_time # How many seconds do we wait for tasks to heartbeat before mark them as zombies. self._zombie_threshold_secs = conf.getint("scheduler", "scheduler_zombie_task_threshold") @@ -837,7 +827,7 @@ def _process_executor_events(self, executor: BaseExecutor, session: Session) -> span.set_attribute("hostname", ti.hostname) span.set_attribute("log_url", ti.log_url) span.set_attribute("operator", str(ti.operator)) - span.set_attribute("try_number", ti.try_number - 1) + span.set_attribute("try_number", ti.try_number) span.set_attribute("executor_state", state) span.set_attribute("job_id", ti.job_id) span.set_attribute("pool", ti.pool) @@ -1304,7 +1294,7 @@ def _create_dagruns_for_dags(self, guard: CommitProhibitorGuard, session: Sessio guard.commit() # END: create dagruns - @span + @add_span def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) -> None: """Create a DAG run and update the dag_model to control if/when the next DAGRun should be created.""" # Bulk Fetch DagRuns with dag_id and execution_date same @@ -1512,7 +1502,7 @@ def _should_update_dag_next_dagruns( return False return True - @span + @add_span def _start_queued_dagruns(self, session: Session) -> None: """Find DagRuns in queued state and decide moving them to running state.""" # added all() to save runtime, otherwise query is executed more than once @@ -1522,13 +1512,13 @@ def _start_queued_dagruns(self, session: Session) -> None: DagRun.active_runs_of_dags((dr.dag_id for dr in dag_runs), only_running=True, session=session), ) - @span + @add_span def _update_state(dag: DAG, dag_run: DagRun): - __span = Trace.get_current_span() - __span.set_attribute("state", str(DagRunState.RUNNING)) - __span.set_attribute("run_id", dag_run.run_id) - __span.set_attribute("type", dag_run.run_type) - __span.set_attribute("dag_id", dag_run.dag_id) + span = Trace.get_current_span() + span.set_attribute("state", str(DagRunState.RUNNING)) + span.set_attribute("run_id", dag_run.run_id) + span.set_attribute("type", dag_run.run_type) + span.set_attribute("dag_id", dag_run.dag_id) dag_run.state = DagRunState.RUNNING dag_run.start_date = timezone.utcnow() @@ -1549,8 +1539,8 @@ def _update_state(dag: DAG, dag_run: DagRun): schedule_delay, tags={"dag_id": dag.dag_id}, ) - if __span.is_recording(): - __span.add_event( + if span.is_recording(): + span.add_event( name="schedule_delay", attributes={"dag_id": dag.dag_id, "schedule_delay": str(schedule_delay)}, ) @@ -1560,7 +1550,7 @@ def _update_state(dag: DAG, dag_run: DagRun): partial(self.dagbag.get_dag, session=session) ) - _span = Trace.get_current_span() + span = Trace.get_current_span() for dag_run in dag_runs: dag = dag_run.dag = cached_get_dag(dag_run.dag_id) @@ -1577,8 +1567,8 @@ def _update_state(dag: DAG, dag_run: DagRun): dag_run.execution_date, ) else: - if _span.is_recording(): - _span.add_event( + if span.is_recording(): + span.add_event( name="dag_run", attributes={ "run_id": dag_run.run_id, diff --git a/airflow/jobs/triggerer_job_runner.py b/airflow/jobs/triggerer_job_runner.py index 080323a1d161..b41af29f376b 100644 --- a/airflow/jobs/triggerer_job_runner.py +++ b/airflow/jobs/triggerer_job_runner.py @@ -37,7 +37,7 @@ from airflow.jobs.job import perform_heartbeat from airflow.models.trigger import Trigger from airflow.stats import Stats -from airflow.traces.tracer import Trace, span +from airflow.traces.tracer import Trace, add_span from airflow.triggers.base import TriggerEvent from airflow.typing_compat import TypedDict from airflow.utils import timezone @@ -392,14 +392,14 @@ def _run_trigger_loop(self) -> None: # Idle sleep time.sleep(1) - @span + @add_span def load_triggers(self): """Query the database for the triggers we're supposed to be running and update the runner.""" Trigger.assign_unassigned(self.job.id, self.capacity, self.health_check_threshold) ids = Trigger.ids_for_triggerer(self.job.id) self.trigger_runner.update_triggers(set(ids)) - @span + @add_span def handle_events(self): """Dispatch outbound events to the Trigger model which pushes them to the relevant task instances.""" while self.trigger_runner.events: @@ -410,7 +410,7 @@ def handle_events(self): # Emit stat event Stats.incr("triggers.succeeded") - @span + @add_span def handle_failed_triggers(self): """ Handle "failed" triggers. - ones that errored or exited before they sent an event. @@ -424,7 +424,7 @@ def handle_failed_triggers(self): # Emit stat event Stats.incr("triggers.failed") - @span + @add_span def emit_metrics(self): Stats.gauge(f"triggers.running.{self.job.hostname}", len(self.trigger_runner.triggers)) Stats.gauge( diff --git a/airflow/macros/__init__.py b/airflow/macros/__init__.py index b18984b94e5a..be4554818acf 100644 --- a/airflow/macros/__init__.py +++ b/airflow/macros/__init__.py @@ -29,20 +29,10 @@ from babel.dates import LC_TIME, format_datetime import airflow.utils.yaml as yaml # noqa: F401 -from airflow.utils.deprecation_tools import add_deprecated_classes if TYPE_CHECKING: from pendulum import DateTime -__deprecated_classes = { - "hive": { - "closest_ds_partition": "airflow.providers.apache.hive.macros.hive.closest_ds_partition", - "max_partition": "airflow.providers.apache.hive.macros.hive.max_partition", - }, -} - -add_deprecated_classes(__deprecated_classes, __name__) - def ds_add(ds: str, days: int) -> str: """ diff --git a/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py b/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py new file mode 100644 index 000000000000..eab9954b329a --- /dev/null +++ b/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py @@ -0,0 +1,76 @@ +# +# 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. + +""" +Remove SubDAGs: ``is_subdag`` & ``root_dag_id`` columns from DAG table. + +Revision ID: d0f1c55954fa +Revises: 044f740568ec +Create Date: 2024-08-11 21:32:40.576172 + +""" + +from __future__ import annotations + +import sqlalchemy as sa +from alembic import op + +from airflow.migrations.db_types import StringID + +# revision identifiers, used by Alembic. +revision = "d0f1c55954fa" +down_revision = "044f740568ec" +branch_labels = None +depends_on = None +airflow_version = "3.0.0" + + +def _column_exists(inspector, column_name): + return column_name in [col["name"] for col in inspector.get_columns("dag")] + + +def _index_exists(inspector, index_name): + return index_name in [index["name"] for index in inspector.get_indexes("dag")] + + +def upgrade(): + """Remove ``is_subdag`` column from DAGs table.""" + conn = op.get_bind() + inspector = sa.inspect(conn) + + with op.batch_alter_table("dag", schema=None) as batch_op: + if _index_exists(inspector, "idx_root_dag_id"): + batch_op.drop_index("idx_root_dag_id") + if _column_exists(inspector, "is_subdag"): + batch_op.drop_column("is_subdag") + if _column_exists(inspector, "root_dag_id"): + batch_op.drop_column("root_dag_id") + + +def downgrade(): + """Add ``is_subdag`` column in DAGs table.""" + conn = op.get_bind() + inspector = sa.inspect(conn) + + with op.batch_alter_table("dag", schema=None) as batch_op: + if not _column_exists(inspector, "is_subdag"): + batch_op.add_column(sa.Column("is_subdag", sa.BOOLEAN(), nullable=True)) + if not _column_exists(inspector, "root_dag_id"): + batch_op.add_column(sa.Column("root_dag_id", StringID(), nullable=True)) + if not _index_exists(inspector, "idx_root_dag_id"): + batch_op.create_index("idx_root_dag_id", ["root_dag_id"], unique=False) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 7ffa596ec67a..ea100cd4e2ab 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -854,10 +854,6 @@ def say_hello_world(**context): _dag: DAG | None = None task_group: TaskGroup | None = None - # subdag parameter is only set for SubDagOperator. - # Setting it to None by default as other Operators do not have that field - subdag: DAG | None = None - start_date: pendulum.DateTime | None = None end_date: pendulum.DateTime | None = None @@ -1724,7 +1720,6 @@ def get_serialized_fields(cls): "end_date", "_task_type", "_operator_name", - "subdag", "ui_color", "ui_fgcolor", "template_ext", diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 1c9d351c1d29..518b367067ef 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -131,7 +131,6 @@ from airflow.timetables.trigger import CronTriggerTimetable from airflow.utils import timezone from airflow.utils.dag_cycle_tester import check_cycle -from airflow.utils.dates import cron_presets, date_range as utils_date_range from airflow.utils.decorators import fixup_decorator_warning_stack from airflow.utils.helpers import at_most_one, exactly_one, validate_instance_args, validate_key from airflow.utils.log.logging_mixin import LoggingMixin @@ -497,7 +496,6 @@ class DAG(LoggingMixin): _comps = { "dag_id", "task_ids", - "parent_dag", "start_date", "end_date", "schedule_interval", @@ -510,14 +508,12 @@ class DAG(LoggingMixin): fileloc: str """ - File path that needs to be imported to load this DAG or subdag. + File path that needs to be imported to load this DAG. This may not be an actual file on disk in the case when this DAG is loaded from a ZIP file or other DAG distribution format. """ - parent_dag: DAG | None = None # Gets set when DAGs are loaded - # NOTE: When updating arguments here, please also keep arguments in @dag() # below in sync. (Search for 'def dag(' in this file.) def __init__( @@ -684,6 +680,12 @@ def __init__( self.timetable = DatasetTriggeredTimetable(DatasetAll(*schedule)) self.schedule_interval = self.timetable.summary elif isinstance(schedule, ArgNotSet): + warnings.warn( + "Creating a DAG with an implicit schedule is deprecated, and will stop working " + "in a future release. Set `schedule=datetime.timedelta(days=1)` explicitly.", + RemovedInAirflow3Warning, + stacklevel=2, + ) self.timetable = create_timetable(schedule, self.timezone) self.schedule_interval = DEFAULT_SCHEDULE_INTERVAL else: @@ -947,60 +949,6 @@ def update_old_perm(permission: str): return updated_access_control - def date_range( - self, - start_date: pendulum.DateTime, - num: int | None = None, - end_date: datetime | None = None, - ) -> list[datetime]: - message = "`DAG.date_range()` is deprecated." - if num is not None: - warnings.warn(message, category=RemovedInAirflow3Warning, stacklevel=2) - with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - return utils_date_range( - start_date=start_date, num=num, delta=self.normalized_schedule_interval - ) - message += " Please use `DAG.iter_dagrun_infos_between(..., align=False)` instead." - warnings.warn(message, category=RemovedInAirflow3Warning, stacklevel=2) - if end_date is None: - coerced_end_date = timezone.utcnow() - else: - coerced_end_date = end_date - it = self.iter_dagrun_infos_between(start_date, pendulum.instance(coerced_end_date), align=False) - return [info.logical_date for info in it] - - def is_fixed_time_schedule(self): - """ - Figures out if the schedule has a fixed time (e.g. 3 AM every day). - - Detection is done by "peeking" the next two cron trigger time; if the - two times have the same minute and hour value, the schedule is fixed, - and we *don't* need to perform the DST fix. - - This assumes DST happens on whole minute changes (e.g. 12:59 -> 12:00). - - Do not try to understand what this actually means. It is old logic that - should not be used anywhere. - """ - warnings.warn( - "`DAG.is_fixed_time_schedule()` is deprecated.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - - from airflow.timetables._cron import CronMixin - - if not isinstance(self.timetable, CronMixin): - return True - - from croniter import croniter - - cron = croniter(self.timetable._expression) - next_a = cron.get_next(datetime) - next_b = cron.get_next(datetime) - return next_b.minute == next_a.minute and next_b.hour == next_a.hour - def following_schedule(self, dttm): """ Calculate the following schedule for this dag in UTC. @@ -1135,10 +1083,6 @@ def next_dagrun_info( :return: DagRunInfo of the next dagrun, or None if a dagrun is not going to be scheduled. """ - # Never schedule a subdag. It will be scheduled by its parent dag. - if self.is_subdag: - return None - data_interval = None if isinstance(last_automated_dagrun, datetime): warnings.warn( @@ -1169,21 +1113,6 @@ def next_dagrun_info( info = None return info - def next_dagrun_after_date(self, date_last_automated_dagrun: pendulum.DateTime | None): - warnings.warn( - "`DAG.next_dagrun_after_date()` is deprecated. Please use `DAG.next_dagrun_info()` instead.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - if date_last_automated_dagrun is None: - data_interval = None - else: - data_interval = self.infer_automated_data_interval(date_last_automated_dagrun) - info = self.next_dagrun_info(data_interval) - if info is None: - return None - return info.run_after - @functools.cached_property def _time_restriction(self) -> TimeRestriction: start_dates = [t.start_date for t in self.tasks if t.start_date] @@ -1217,8 +1146,7 @@ def iter_dagrun_infos_between( If ``align`` is ``False``, the first run will happen immediately on ``earliest``, even if it does not fall on the logical timetable schedule. - The default is ``True``, but subdags will ignore this value and always - behave as if this is set to ``False`` for backward compatibility. + The default is ``True``. Example: A DAG is scheduled to run every midnight (``0 0 * * *``). If ``earliest`` is ``2021-06-03 23:00:00``, the first DagRunInfo would be @@ -1234,15 +1162,6 @@ def iter_dagrun_infos_between( restriction = TimeRestriction(earliest, latest, catchup=True) - # HACK: Sub-DAGs are currently scheduled differently. For example, say - # the schedule is @daily and start is 2021-06-03 22:16:00, a top-level - # DAG should be first scheduled to run on midnight 2021-06-04, but a - # sub-DAG should be first scheduled to run RIGHT NOW. We can change - # this, but since sub-DAGs are going away in 3.0 anyway, let's keep - # compatibility for now and remove this entirely later. - if self.is_subdag: - align = False - try: info = self.timetable.next_dagrun_info( last_automated_data_interval=None, @@ -1284,46 +1203,6 @@ def iter_dagrun_infos_between( ) break - def get_run_dates(self, start_date, end_date=None) -> list: - """ - Return a list of dates between the interval received as parameter using this dag's schedule interval. - - Returned dates can be used for execution dates. - - :param start_date: The start date of the interval. - :param end_date: The end date of the interval. Defaults to ``timezone.utcnow()``. - :return: A list of dates within the interval following the dag's schedule. - """ - warnings.warn( - "`DAG.get_run_dates()` is deprecated. Please use `DAG.iter_dagrun_infos_between()` instead.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - earliest = timezone.coerce_datetime(start_date) - if end_date is None: - latest = pendulum.now(timezone.utc) - else: - latest = timezone.coerce_datetime(end_date) - return [info.logical_date for info in self.iter_dagrun_infos_between(earliest, latest)] - - def normalize_schedule(self, dttm): - warnings.warn( - "`DAG.normalize_schedule()` is deprecated.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - following = self.following_schedule(dttm) - if not following: # in case of @once - return dttm - with warnings.catch_warnings(): - warnings.simplefilter("ignore", RemovedInAirflow3Warning) - previous_of_following = self.previous_schedule(following) - if previous_of_following != dttm: - return following - return dttm - @provide_session def get_last_dagrun(self, session=NEW_SESSION, include_externally_triggered=False): return get_last_dagrun( @@ -1347,47 +1226,6 @@ def dag_id(self) -> str: def dag_id(self, value: str) -> None: self._dag_id = value - @property - def is_subdag(self) -> bool: - return self.parent_dag is not None - - @property - def full_filepath(self) -> str: - """ - Full file path to the DAG. - - :meta private: - """ - warnings.warn( - "DAG.full_filepath is deprecated in favour of fileloc", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.fileloc - - @full_filepath.setter - def full_filepath(self, value) -> None: - warnings.warn( - "DAG.full_filepath is deprecated in favour of fileloc", - RemovedInAirflow3Warning, - stacklevel=2, - ) - self.fileloc = value - - @property - def concurrency(self) -> int: - # TODO: Remove in Airflow 3.0 - warnings.warn( - "The 'DAG.concurrency' attribute is deprecated. Please use 'DAG.max_active_tasks'.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self._max_active_tasks - - @concurrency.setter - def concurrency(self, value: int): - self._max_active_tasks = value - @property def max_active_tasks(self) -> int: return self._max_active_tasks @@ -1459,20 +1297,6 @@ def tasks_upstream_of_teardowns(self) -> list[Operator]: def task_group(self) -> TaskGroup: return self._task_group - @property - def filepath(self) -> str: - """ - Relative file path to the DAG. - - :meta private: - """ - warnings.warn( - "filepath is deprecated, use relative_fileloc instead", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return str(self.relative_fileloc) - @property def relative_fileloc(self) -> pathlib.Path: """File location of the importable dag 'file' relative to the configured DAGs folder.""" @@ -1517,16 +1341,6 @@ def get_concurrency_reached(self, session=NEW_SESSION) -> bool: ) return total_tasks >= self.max_active_tasks - @property - def concurrency_reached(self): - """Use `airflow.models.DAG.get_concurrency_reached`, this attribute is deprecated.""" - warnings.warn( - "This attribute is deprecated. Please use `airflow.models.DAG.get_concurrency_reached` method.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.get_concurrency_reached() - @provide_session def get_is_active(self, session=NEW_SESSION) -> None: """Return a boolean indicating whether this DAG is active.""" @@ -1547,21 +1361,6 @@ def is_paused(self): ) return self.get_is_paused() - @property - def normalized_schedule_interval(self) -> ScheduleInterval: - warnings.warn( - "DAG.normalized_schedule_interval() is deprecated.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - if isinstance(self.schedule_interval, str) and self.schedule_interval in cron_presets: - _schedule_interval: ScheduleInterval = cron_presets.get(self.schedule_interval) - elif self.schedule_interval == "@once": - _schedule_interval = None - else: - _schedule_interval = self.schedule_interval - return _schedule_interval - @staticmethod @internal_api_call @provide_session @@ -1745,35 +1544,6 @@ def get_latest_execution_date(self, session: Session = NEW_SESSION) -> pendulum. """Return the latest date for which at least one dag run exists.""" return session.scalar(select(func.max(DagRun.execution_date)).where(DagRun.dag_id == self.dag_id)) - @property - def latest_execution_date(self): - """Use `airflow.models.DAG.get_latest_execution_date`, this attribute is deprecated.""" - warnings.warn( - "This attribute is deprecated. Please use `airflow.models.DAG.get_latest_execution_date`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.get_latest_execution_date() - - @property - def subdags(self): - """Return a list of the subdag objects associated to this DAG.""" - # Check SubDag for class but don't check class directly - from airflow.operators.subdag import SubDagOperator - - subdag_lst = [] - for task in self.tasks: - if ( - isinstance(task, SubDagOperator) - or - # TODO remove in Airflow 2.0 - type(task).__name__ == "SubDagOperator" - or task.task_type == "SubDagOperator" - ): - subdag_lst.append(task.subdag) - subdag_lst += task.subdag.subdags - return subdag_lst - def resolve_template_files(self): for t in self.tasks: t.resolve_template_files() @@ -1866,8 +1636,6 @@ def get_task_instances( end_date=end_date, run_id=None, state=state or (), - include_subdags=False, - include_parentdag=False, include_dependent_dags=False, exclude_task_ids=(), session=session, @@ -1883,8 +1651,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1901,8 +1667,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1921,8 +1685,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1933,7 +1695,7 @@ def _get_task_instances( ) -> Iterable[TaskInstance] | set[TaskInstanceKey]: TI = TaskInstance - # If we are looking at subdags/dependent dags we want to avoid UNION calls + # If we are looking at dependent dags we want to avoid UNION calls # in SQL (it doesn't play nice with fields that have no equality operator, # like JSON types), we instead build our result set separately. # @@ -1948,15 +1710,7 @@ def _get_task_instances( tis = select(TaskInstance) tis = tis.join(TaskInstance.dag_run) - if include_subdags: - # Crafting the right filter for dag_id and task_ids combo - conditions = [] - for dag in [*self.subdags, self]: - conditions.append( - (TaskInstance.dag_id == dag.dag_id) & TaskInstance.task_id.in_(dag.task_ids) - ) - tis = tis.where(or_(*conditions)) - elif self.partial: + if self.partial: tis = tis.where(TaskInstance.dag_id == self.dag_id, TaskInstance.task_id.in_(self.task_ids)) else: tis = tis.where(TaskInstance.dag_id == self.dag_id) @@ -1990,36 +1744,6 @@ def _get_task_instances( else: tis = tis.where(TaskInstance.state.in_(state)) - # Next, get any of them from our parent DAG (if there is one) - if include_parentdag and self.parent_dag is not None: - if visited_external_tis is None: - visited_external_tis = set() - - p_dag = self.parent_dag.partial_subset( - task_ids_or_regex=r"^{}$".format(self.dag_id.split(".")[1]), - include_upstream=False, - include_downstream=True, - ) - result.update( - p_dag._get_task_instances( - task_ids=task_ids, - start_date=start_date, - end_date=end_date, - run_id=None, - state=state, - include_subdags=include_subdags, - include_parentdag=False, - include_dependent_dags=include_dependent_dags, - as_pk_tuple=True, - exclude_task_ids=exclude_task_ids, - session=session, - dag_bag=dag_bag, - recursion_depth=recursion_depth, - max_recursion_depth=max_recursion_depth, - visited_external_tis=visited_external_tis, - ) - ) - if include_dependent_dags: # Recursively find external tasks indicated by ExternalTaskMarker from airflow.sensors.external_task import ExternalTaskMarker @@ -2089,9 +1813,7 @@ def _get_task_instances( start_date=None, end_date=None, state=state, - include_subdags=include_subdags, include_dependent_dags=include_dependent_dags, - include_parentdag=False, as_pk_tuple=True, exclude_task_ids=exclude_task_ids, dag_bag=dag_bag, @@ -2103,7 +1825,7 @@ def _get_task_instances( ) if result or as_pk_tuple: - # Only execute the `ti` query if we have also collected some other results (i.e. subdags etc.) + # Only execute the `ti` query if we have also collected some other results if as_pk_tuple: tis_query = session.execute(tis).all() result.update(TaskInstanceKey(**cols._mapping) for cols in tis_query) @@ -2218,8 +1940,6 @@ def set_task_instance_state( subdag.clear( start_date=start_date, end_date=end_date, - include_subdags=True, - include_parentdag=True, only_failed=True, session=session, # Exclude the task itself from being cleared @@ -2319,8 +2039,6 @@ def set_task_group_state( task_subset.clear( start_date=start_date, end_date=end_date, - include_subdags=True, - include_parentdag=True, only_failed=True, session=session, # Exclude the task from the current group from being cleared @@ -2339,7 +2057,7 @@ def leaves(self) -> list[Operator]: """Return nodes with no children. These are last to execute and are called leaves or leaf nodes.""" return [task for task in self.tasks if not task.downstream_list] - def topological_sort(self, include_subdag_tasks: bool = False): + def topological_sort(self): """ Sorts tasks in topographical order, such that a task comes after any of its upstream dependencies. @@ -2348,7 +2066,7 @@ def topological_sort(self, include_subdag_tasks: bool = False): from airflow.utils.task_group import TaskGroup def nested_topo(group): - for node in group.topological_sort(_include_subdag_tasks=include_subdag_tasks): + for node in group.topological_sort(): if isinstance(node, TaskGroup): yield from nested_topo(node) else: @@ -2356,28 +2074,6 @@ def nested_topo(group): return tuple(nested_topo(self.task_group)) - @provide_session - def set_dag_runs_state( - self, - state: DagRunState = DagRunState.RUNNING, - session: Session = NEW_SESSION, - start_date: datetime | None = None, - end_date: datetime | None = None, - dag_ids: list[str] | None = None, - ) -> None: - warnings.warn( - "This method is deprecated and will be removed in a future version.", - RemovedInAirflow3Warning, - stacklevel=3, - ) - dag_ids = dag_ids or [self.dag_id] - query = update(DagRun).where(DagRun.dag_id.in_(dag_ids)) - if start_date: - query = query.where(DagRun.execution_date >= start_date) - if end_date: - query = query.where(DagRun.execution_date <= end_date) - session.execute(query.values(state=state).execution_options(synchronize_session="fetch")) - @provide_session def clear( self, @@ -2387,8 +2083,6 @@ def clear( only_failed: bool = False, only_running: bool = False, confirm_prompt: bool = False, - include_subdags: bool = True, - include_parentdag: bool = True, dag_run_state: DagRunState = DagRunState.QUEUED, dry_run: bool = False, session: Session = NEW_SESSION, @@ -2407,14 +2101,11 @@ def clear( :param only_failed: Only clear failed tasks :param only_running: Only clear running tasks. :param confirm_prompt: Ask for confirmation - :param include_subdags: Clear tasks in subdags and clear external tasks - indicated by ExternalTaskMarker - :param include_parentdag: Clear tasks in the parent dag of the subdag. :param dag_run_state: state to set DagRun to. If set to False, dagrun state will not be changed. :param dry_run: Find the tasks to clear but don't clear them. :param session: The sqlalchemy session to use - :param dag_bag: The DagBag used to find the dags subdags (Optional) + :param dag_bag: The DagBag used to find the dags (Optional) :param exclude_task_ids: A set of ``task_id`` or (``task_id``, ``map_index``) tuples that should not be cleared """ @@ -2452,9 +2143,7 @@ def clear( end_date=end_date, run_id=None, state=state, - include_subdags=include_subdags, - include_parentdag=include_parentdag, - include_dependent_dags=include_subdags, # compat, yes this is not a typo + include_dependent_dags=True, session=session, dag_bag=dag_bag, exclude_task_ids=exclude_task_ids, @@ -2497,8 +2186,6 @@ def clear_dags( only_failed=False, only_running=False, confirm_prompt=False, - include_subdags=True, - include_parentdag=False, dag_run_state=DagRunState.QUEUED, dry_run=False, ): @@ -2510,8 +2197,6 @@ def clear_dags( only_failed=only_failed, only_running=only_running, confirm_prompt=False, - include_subdags=include_subdags, - include_parentdag=include_parentdag, dag_run_state=dag_run_state, dry_run=True, ) @@ -2538,7 +2223,6 @@ def clear_dags( only_failed=only_failed, only_running=only_running, confirm_prompt=False, - include_subdags=include_subdags, dag_run_state=dag_run_state, dry_run=False, ) @@ -2563,15 +2247,6 @@ def __deepcopy__(self, memo): result._log = self._log return result - def sub_dag(self, *args, **kwargs): - """Use `airflow.models.DAG.partial_subset`, this method is deprecated.""" - warnings.warn( - "This method is deprecated and will be removed in a future version. Please use partial_subset", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.partial_subset(*args, **kwargs) - def partial_subset( self, task_ids_or_regex: str | Pattern | Iterable[str], @@ -2714,13 +2389,9 @@ def has_task_group(self, task_group_id: str) -> bool: def task_group_dict(self): return {k: v for k, v in self._task_group.get_task_group_dict().items() if k is not None} - def get_task(self, task_id: str, include_subdags: bool = False) -> Operator: + def get_task(self, task_id: str) -> Operator: if task_id in self.task_dict: return self.task_dict[task_id] - if include_subdags: - for dag in self.subdags: - if task_id in dag.task_dict: - return dag.task_dict[task_id] raise TaskNotFound(f"Task {task_id} not found") def pickle_info(self): @@ -3172,21 +2843,6 @@ def create_dagrun( ) return run - @classmethod - @provide_session - def bulk_sync_to_db( - cls, - dags: Collection[DAG], - session=NEW_SESSION, - ): - """Use `airflow.models.DAG.bulk_write_to_db`, this method is deprecated.""" - warnings.warn( - "This method is deprecated and will be removed in a future version. Please use bulk_write_to_db", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return cls.bulk_write_to_db(dags=dags, session=session) - @classmethod @provide_session def bulk_write_to_db( @@ -3198,8 +2854,6 @@ def bulk_write_to_db( """ Ensure the DagModel rows for the given dags are up-to-date in the dag table in the DB. - Note that this method can be called for both DAGs and SubDAGs. A SubDag is actually a SubDagOperator. - :param dags: the DAG objects to save to the DB :return: None """ @@ -3251,15 +2905,8 @@ def bulk_write_to_db( for orm_dag in sorted(orm_dags, key=lambda d: d.dag_id): dag = dag_by_ids[orm_dag.dag_id] filelocs.append(dag.fileloc) - if dag.is_subdag: - orm_dag.is_subdag = True - orm_dag.fileloc = dag.parent_dag.fileloc # type: ignore - orm_dag.root_dag_id = dag.parent_dag.dag_id # type: ignore - orm_dag.owners = dag.parent_dag.owner # type: ignore - else: - orm_dag.is_subdag = False - orm_dag.fileloc = dag.fileloc - orm_dag.owners = dag.owner + orm_dag.fileloc = dag.fileloc + orm_dag.owners = dag.owner orm_dag.is_active = True orm_dag.has_import_errors = False orm_dag.last_parsed_time = timezone.utcnow() @@ -3474,9 +3121,6 @@ def bulk_write_to_db( # decide when to commit session.flush() - for dag in dags: - cls.bulk_write_to_db(dag.subdags, processor_subdir=processor_subdir, session=session) - @classmethod def _get_latest_runs_stmt(cls, dags: list[str]) -> Select: """ @@ -3526,8 +3170,6 @@ def sync_to_db(self, processor_subdir: str | None = None, session=NEW_SESSION): """ Save attributes about this DAG to the DB. - Note that this method can be called for both DAGs and SubDAGs. A SubDag is actually a SubDagOperator. - :return: None """ self.bulk_write_to_db([self], processor_subdir=processor_subdir, session=session) @@ -3621,7 +3263,6 @@ def get_serialized_fields(cls): """Stringified DAGs and operators contain exactly these fields.""" if not cls.__serialized_fields: exclusion_list = { - "parent_dag", "schedule_dataset_references", "schedule_dataset_alias_references", "task_outlet_dataset_references", @@ -3647,7 +3288,7 @@ def get_serialized_fields(cls): "auto_register", "fail_stop", } - cls.__serialized_fields = frozenset(vars(DAG(dag_id="test"))) - exclusion_list + cls.__serialized_fields = frozenset(vars(DAG(dag_id="test", schedule=None))) - exclusion_list return cls.__serialized_fields def get_edge_info(self, upstream_task_id: str, downstream_task_id: str) -> EdgeInfoType: @@ -3753,13 +3394,10 @@ class DagModel(Base): These items are stored in the database for state related information """ dag_id = Column(StringID(), primary_key=True) - 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 = 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) # Whether that DAG was seen on the last DagBag load is_active = Column(Boolean, default=False) # Last time the scheduler started @@ -3818,14 +3456,8 @@ class DagModel(Base): # Earliest time at which this ``next_dagrun`` can be created. next_dagrun_create_after = Column(UtcDateTime) - __table_args__ = ( - Index("idx_root_dag_id", root_dag_id, unique=False), - Index("idx_next_dagrun_create_after", next_dagrun_create_after, unique=False), - ) + __table_args__ = (Index("idx_next_dagrun_create_after", next_dagrun_create_after, unique=False),) - parent_dag = relationship( - "DagModel", remote_side=[dag_id], primaryjoin=root_dag_id == dag_id, foreign_keys=[root_dag_id] - ) schedule_dataset_references = relationship( "DagScheduleDatasetReference", back_populates="dag", @@ -3898,7 +3530,6 @@ def get_dagmodel(dag_id: str, session: Session = NEW_SESSION) -> DagModel | None return session.get( DagModel, dag_id, - options=[joinedload(DagModel.parent_dag)], ) @classmethod @@ -3963,19 +3594,17 @@ def relative_fileloc(self) -> pathlib.Path | None: return path @provide_session - def set_is_paused(self, is_paused: bool, including_subdags: bool = True, session=NEW_SESSION) -> None: + def set_is_paused(self, is_paused: bool, session=NEW_SESSION) -> None: """ Pause/Un-pause a DAG. :param is_paused: Is the DAG paused - :param including_subdags: whether to include the DAG's subdags :param session: session """ filter_query = [ DagModel.dag_id == self.dag_id, ] - if including_subdags: - filter_query.append(DagModel.root_dag_id == self.dag_id) + session.execute( update(DagModel) .where(or_(*filter_query)) diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index f384bfcd84ea..561564a5e932 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -25,7 +25,6 @@ import sys import textwrap import traceback -import warnings import zipfile from datetime import datetime, timedelta from pathlib import Path @@ -47,7 +46,6 @@ AirflowDagCycleException, AirflowDagDuplicatedIdException, AirflowException, - RemovedInAirflow3Warning, ) from airflow.listeners.listener import get_listener_manager from airflow.models.base import Base @@ -113,7 +111,6 @@ class DagBag(LoggingMixin): to filter python modules to scan for dags. :param read_dags_from_db: Read DAGs from DB if ``True`` is passed. If ``False`` DAGs are read from python files. - :param store_serialized_dags: deprecated parameter, same effect as `read_dags_from_db` :param load_op_links: Should the extra operator link be loaded via plugins when de-serializing the DAG? This flag is set to False in Scheduler so that Extra Operator links are not loaded to not run User code in Scheduler. @@ -126,7 +123,6 @@ def __init__( include_examples: bool | ArgNotSet = NOTSET, safe_mode: bool | ArgNotSet = NOTSET, read_dags_from_db: bool = False, - store_serialized_dags: bool | None = None, load_op_links: bool = True, collect_dags: bool = True, ): @@ -143,15 +139,6 @@ def __init__( safe_mode if isinstance(safe_mode, bool) else conf.getboolean("core", "DAG_DISCOVERY_SAFE_MODE") ) - if store_serialized_dags: - warnings.warn( - "The store_serialized_dags parameter has been deprecated. " - "You should pass the read_dags_from_db parameter.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - read_dags_from_db = store_serialized_dags - dag_folder = dag_folder or settings.DAGS_FOLDER self.dag_folder = dag_folder self.dags: dict[str, DAG] = {} @@ -182,16 +169,6 @@ def size(self) -> int: """:return: the amount of dags contained in this dagbag""" return len(self.dags) - @property - def store_serialized_dags(self) -> bool: - """Whether to read dags from DB.""" - warnings.warn( - "The store_serialized_dags property has been deprecated. Use read_dags_from_db instead.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.read_dags_from_db - @property def dag_ids(self) -> list[str]: """ @@ -259,8 +236,6 @@ def get_dag(self, dag_id, session: Session = None): root_dag_id = dag_id if dag_id in self.dags: dag = self.dags[dag_id] - if dag.parent_dag: - root_dag_id = dag.parent_dag.dag_id # If DAG Model is absent, we can't check last_expired property. Is the DAG not yet synchronized? orm_dag = DagModel.get_current(root_dag_id, session=session) @@ -272,11 +247,7 @@ def get_dag(self, dag_id, session: Session = None): is_expired = orm_dag.last_expired and dag and dag.last_loaded < orm_dag.last_expired if is_expired: # Remove associated dags so we can re-add them. - self.dags = { - key: dag - for key, dag in self.dags.items() - if root_dag_id != key and not (dag.parent_dag and root_dag_id == dag.parent_dag.dag_id) - } + self.dags = {key: dag for key, dag in self.dags.items()} if is_missing or is_expired: # Reprocess source file. found_dags = self.process_file( @@ -300,8 +271,6 @@ def _add_dag_from_db(self, dag_id: str, session: Session): row.load_op_links = self.load_op_links dag = row.dag - for subdag in dag.subdags: - self.dags[subdag.dag_id] = subdag self.dags[dag.dag_id] = dag self.dags_last_fetched[dag.dag_id] = timezone.utcnow() self.dags_hash[dag.dag_id] = row.dag_hash @@ -476,7 +445,7 @@ def _process_modules(self, filepath, mods, file_last_changed_on_disk): dag.fileloc = mod.__file__ try: dag.validate() - self.bag_dag(dag=dag, root_dag=dag) + self.bag_dag(dag=dag) except AirflowClusterPolicySkipDag: pass except Exception as e: @@ -485,25 +454,15 @@ def _process_modules(self, filepath, mods, file_last_changed_on_disk): self.file_last_changed[dag.fileloc] = file_last_changed_on_disk else: found_dags.append(dag) - found_dags += dag.subdags return found_dags - def bag_dag(self, dag, root_dag): + def bag_dag(self, dag): """ - Add the DAG into the bag, recurses into sub dags. + Add the DAG into the bag. :raises: AirflowDagCycleException if a cycle is detected in this dag or its subdags. :raises: AirflowDagDuplicatedIdException if this dag or its subdags already exists in the bag. """ - self._bag_dag(dag=dag, root_dag=root_dag, recursive=True) - - def _bag_dag(self, *, dag, root_dag, recursive): - """ - Actual implementation of bagging a dag. - - The only purpose of this is to avoid exposing ``recursive`` in ``bag_dag()``, - intended to only be used by the ``_bag_dag()`` implementation. - """ check_cycle(dag) # throws if a task cycle is found dag.resolve_template_files() @@ -531,17 +490,7 @@ def _bag_dag(self, *, dag, root_dag, recursive): self.log.exception(e) raise AirflowClusterPolicyError(e) - subdags = dag.subdags - try: - # DAG.subdags automatically performs DFS search, so we don't recurse - # into further _bag_dag() calls. - if recursive: - for subdag in subdags: - subdag.fileloc = dag.fileloc - subdag.parent_dag = dag - self._bag_dag(dag=subdag, root_dag=root_dag, recursive=False) - prev_dag = self.dags.get(dag.dag_id) if prev_dag and prev_dag.fileloc != dag.fileloc: raise AirflowDagDuplicatedIdException( @@ -554,12 +503,6 @@ def _bag_dag(self, *, dag, root_dag, recursive): except (AirflowDagCycleException, AirflowDagDuplicatedIdException): # There was an error in bagging the dag. Remove it from the list of dags self.log.exception("Exception bagging dag: %s", dag.dag_id) - # Only necessary at the root level since DAG.subdags automatically - # performs DFS to search through all subdags - if recursive: - for subdag in subdags: - if subdag.dag_id in self.dags: - del self.dags[subdag.dag_id] raise def collect_dags( @@ -627,15 +570,6 @@ def collect_dags_from_db(self): # from the table by the scheduler job. self.dags = SerializedDagModel.read_all_dags() - # Adds subdags. - # DAG post-processing steps such as self.bag_dag and croniter are not needed as - # they are done by scheduler before serialization. - subdags = {} - for dag in self.dags.values(): - for subdag in dag.subdags: - subdags[subdag.dag_id] = subdag - self.dags.update(subdags) - def dagbag_report(self): """Print a report around DagBag loading stats.""" stats = self.dagbag_stats @@ -678,8 +612,6 @@ def _serialize_dag_capturing_errors(dag, session, processor_subdir): We can't place them directly in import_errors, as this may be retried, and work the next time """ - if dag.is_subdag: - return [] try: # We can't use bulk_write_to_db as we want to capture each error individually dag_was_updated = SerializedDagModel.write_dag( @@ -739,13 +671,13 @@ def sync_to_db(self, processor_subdir: str | None = None, session: Session = NEW @provide_session def _sync_perm_for_dag(cls, dag: DAG, session: Session = NEW_SESSION): """Sync DAG specific permissions.""" - root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id + dag_id = dag.dag_id - cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) + cls.logger().debug("Syncing DAG permissions: %s to the DB", dag_id) from airflow.www.security_appless import ApplessAirflowSecurityManager security_manager = ApplessAirflowSecurityManager(session=session) - security_manager.sync_perm_for_dag(root_dag_id, dag.access_control) + security_manager.sync_perm_for_dag(dag_id, dag.access_control) def generate_md5_hash(context): diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 523f94cded38..1ff2f6316a5a 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -590,7 +590,6 @@ def _check_last_n_dagruns_failed(self, dag_id, max_consecutive_failed_dag_runs, ) filter_query = [ DagModel.dag_id == self.dag_id, - DagModel.root_dag_id == self.dag_id, # for sub-dags ] session.execute( update(DagModel) diff --git a/airflow/models/mappedoperator.py b/airflow/models/mappedoperator.py index 2377fdab0075..2cb7d993fc9f 100644 --- a/airflow/models/mappedoperator.py +++ b/airflow/models/mappedoperator.py @@ -157,10 +157,6 @@ class OperatorPartial: _expand_called: bool = False # Set when expand() is called to ease user debugging. def __attrs_post_init__(self): - from airflow.operators.subdag import SubDagOperator - - if issubclass(self.operator_class, SubDagOperator): - raise TypeError("Mapping over deprecated SubDagOperator is not supported") validate_mapping_kwargs(self.operator_class, "partial", self.kwargs) def __repr__(self) -> str: @@ -306,7 +302,6 @@ class MappedOperator(AbstractOperator): This should be a name to call ``getattr()`` on. """ - subdag: None = None # Since we don't support SubDagOperator, this is always None. supports_lineage: bool = False HIDE_ATTRS_FROM_UI: ClassVar[frozenset[str]] = AbstractOperator.HIDE_ATTRS_FROM_UI | frozenset( @@ -347,7 +342,6 @@ def get_serialized_fields(cls): "dag", "deps", "expand_input", # This is needed to be able to accept XComArg. - "subdag", "task_group", "upstream_task_ids", "supports_lineage", diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py index 99495f81b2ca..dec843451a98 100644 --- a/airflow/models/serialized_dag.py +++ b/airflow/models/serialized_dag.py @@ -305,8 +305,6 @@ def get(cls, dag_id: str, session: Session = NEW_SESSION) -> SerializedDagModel """ Get the SerializedDAG for the given dag ID. - It will cope with being passed the ID of a subdag by looking up the root dag_id from the DAG table. - :param dag_id: the DAG to fetch :param session: ORM Session """ @@ -314,11 +312,7 @@ def get(cls, dag_id: str, session: Session = NEW_SESSION) -> SerializedDagModel if row: return row - # If we didn't find a matching DAG id then ask the DAG table to find - # out the root dag - root_dag_id = session.scalar(select(DagModel.root_dag_id).where(DagModel.dag_id == dag_id)) - - return session.scalar(select(cls).where(cls.dag_id == root_dag_id)) + return session.scalar(select(cls).where(cls.dag_id == dag_id)) @staticmethod @provide_session @@ -337,13 +331,12 @@ def bulk_sync_to_db( :return: None """ for dag in dags: - if not dag.is_subdag: - SerializedDagModel.write_dag( - dag=dag, - min_update_interval=MIN_SERIALIZED_DAG_UPDATE_INTERVAL, - processor_subdir=processor_subdir, - session=session, - ) + SerializedDagModel.write_dag( + dag=dag, + min_update_interval=MIN_SERIALIZED_DAG_UPDATE_INTERVAL, + processor_subdir=processor_subdir, + session=session, + ) @classmethod @provide_session diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 284b313cdae4..cedc25423900 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -2098,12 +2098,7 @@ def _command_as_list( should_pass_filepath = not pickle_id and dag path: PurePath | None = None if should_pass_filepath: - if dag.is_subdag: - if TYPE_CHECKING: - assert dag.parent_dag is not None - path = dag.parent_dag.relative_fileloc - else: - path = dag.relative_fileloc + path = dag.relative_fileloc if path: if not path.is_absolute(): diff --git a/airflow/models/variable.py b/airflow/models/variable.py index 63b71303bc80..563cac46e8c8 100644 --- a/airflow/models/variable.py +++ b/airflow/models/variable.py @@ -154,7 +154,6 @@ def get( @staticmethod @provide_session - @internal_api_call def set( key: str, value: Any, @@ -167,6 +166,35 @@ def set( This operation overwrites an existing variable. + :param key: Variable Key + :param value: Value to set for the Variable + :param description: Description of the Variable + :param serialize_json: Serialize the value to a JSON string + :param session: Session + """ + Variable._set( + key=key, value=value, description=description, serialize_json=serialize_json, session=session + ) + # invalidate key in cache for faster propagation + # we cannot save the value set because it's possible that it's shadowed by a custom backend + # (see call to check_for_write_conflict above) + SecretCache.invalidate_variable(key) + + @staticmethod + @provide_session + @internal_api_call + def _set( + key: str, + value: Any, + description: str | None = None, + serialize_json: bool = False, + session: Session = None, + ) -> None: + """ + Set a value for an Airflow Variable with a given Key. + + This operation overwrites an existing variable. + :param key: Variable Key :param value: Value to set for the Variable :param description: Description of the Variable @@ -190,7 +218,6 @@ def set( @staticmethod @provide_session - @internal_api_call def update( key: str, value: Any, @@ -200,6 +227,27 @@ def update( """ Update a given Airflow Variable with the Provided value. + :param key: Variable Key + :param value: Value to set for the Variable + :param serialize_json: Serialize the value to a JSON string + :param session: Session + """ + Variable._update(key=key, value=value, serialize_json=serialize_json, session=session) + # We need to invalidate the cache for internal API cases on the client side + SecretCache.invalidate_variable(key) + + @staticmethod + @provide_session + @internal_api_call + def _update( + key: str, + value: Any, + serialize_json: bool = False, + session: Session = None, + ) -> None: + """ + Update a given Airflow Variable with the Provided value. + :param key: Variable Key :param value: Value to set for the Variable :param serialize_json: Serialize the value to a JSON string @@ -219,11 +267,23 @@ def update( @staticmethod @provide_session - @internal_api_call def delete(key: str, session: Session = None) -> int: """ Delete an Airflow Variable for a given key. + :param key: Variable Keys + """ + rows = Variable._delete(key=key, session=session) + SecretCache.invalidate_variable(key) + return rows + + @staticmethod + @provide_session + @internal_api_call + def _delete(key: str, session: Session = None) -> int: + """ + Delete an Airflow Variable for a given key. + :param key: Variable Keys """ rows = session.execute(delete(Variable).where(Variable.key == key)).rowcount diff --git a/airflow/operators/__init__.py b/airflow/operators/__init__.py index 7fc63e1c8f50..7cedaa0ecfcf 100644 --- a/airflow/operators/__init__.py +++ b/airflow/operators/__init__.py @@ -21,179 +21,3 @@ :sphinx-autoapi-skip: """ - -from __future__ import annotations - -from airflow.utils.deprecation_tools import add_deprecated_classes - -__deprecated_classes = { - "bash_operator": { - "BashOperator": "airflow.operators.bash.BashOperator", - }, - "branch_operator": { - "BaseBranchOperator": "airflow.operators.branch.BaseBranchOperator", - }, - "check_operator": { - "SQLCheckOperator": "airflow.providers.common.sql.operators.sql.SQLCheckOperator", - "SQLIntervalCheckOperator": "airflow.providers.common.sql.operators.sql.SQLIntervalCheckOperator", - "SQLThresholdCheckOperator": "airflow.providers.common.sql.operators.sql.SQLThresholdCheckOperator", - "SQLValueCheckOperator": "airflow.providers.common.sql.operators.sql.SQLValueCheckOperator", - "CheckOperator": "airflow.providers.common.sql.operators.sql.SQLCheckOperator", - "IntervalCheckOperator": "airflow.providers.common.sql.operators.sql.SQLIntervalCheckOperator", - "ThresholdCheckOperator": "airflow.providers.common.sql.operators.sql.SQLThresholdCheckOperator", - "ValueCheckOperator": "airflow.providers.common.sql.operators.sql.SQLValueCheckOperator", - }, - "dagrun_operator": { - "TriggerDagRunLink": "airflow.operators.trigger_dagrun.TriggerDagRunLink", - "TriggerDagRunOperator": "airflow.operators.trigger_dagrun.TriggerDagRunOperator", - }, - "docker_operator": { - "DockerOperator": "airflow.providers.docker.operators.docker.DockerOperator", - }, - "druid_check_operator": { - "DruidCheckOperator": "airflow.providers.apache.druid.operators.druid_check.DruidCheckOperator", - }, - "dummy": { - "EmptyOperator": "airflow.operators.empty.EmptyOperator", - "DummyOperator": "airflow.operators.empty.EmptyOperator", - }, - "dummy_operator": { - "EmptyOperator": "airflow.operators.empty.EmptyOperator", - "DummyOperator": "airflow.operators.empty.EmptyOperator", - }, - "email_operator": { - "EmailOperator": "airflow.operators.email.EmailOperator", - }, - "gcs_to_s3": { - "GCSToS3Operator": "airflow.providers.amazon.aws.transfers.gcs_to_s3.GCSToS3Operator", - }, - "google_api_to_s3_transfer": { - "GoogleApiToS3Operator": ( - "airflow.providers.amazon.aws.transfers.google_api_to_s3.GoogleApiToS3Operator" - ), - "GoogleApiToS3Transfer": ( - "airflow.providers.amazon.aws.transfers.google_api_to_s3.GoogleApiToS3Operator" - ), - }, - "hive_operator": { - "HiveOperator": "airflow.providers.apache.hive.operators.hive.HiveOperator", - }, - "hive_stats_operator": { - "HiveStatsCollectionOperator": ( - "airflow.providers.apache.hive.operators.hive_stats.HiveStatsCollectionOperator" - ), - }, - "hive_to_druid": { - "HiveToDruidOperator": "airflow.providers.apache.druid.transfers.hive_to_druid.HiveToDruidOperator", - "HiveToDruidTransfer": "airflow.providers.apache.druid.transfers.hive_to_druid.HiveToDruidOperator", - }, - "hive_to_mysql": { - "HiveToMySqlOperator": "airflow.providers.apache.hive.transfers.hive_to_mysql.HiveToMySqlOperator", - "HiveToMySqlTransfer": "airflow.providers.apache.hive.transfers.hive_to_mysql.HiveToMySqlOperator", - }, - "hive_to_samba_operator": { - "HiveToSambaOperator": "airflow.providers.apache.hive.transfers.hive_to_samba.HiveToSambaOperator", - }, - "http_operator": { - "SimpleHttpOperator": "airflow.providers.http.operators.http.SimpleHttpOperator", - }, - "jdbc_operator": { - "JdbcOperator": "airflow.providers.jdbc.operators.jdbc.JdbcOperator", - }, - "latest_only_operator": { - "LatestOnlyOperator": "airflow.operators.latest_only.LatestOnlyOperator", - }, - "mssql_operator": { - "MsSqlOperator": "airflow.providers.microsoft.mssql.operators.mssql.MsSqlOperator", - }, - "mssql_to_hive": { - "MsSqlToHiveOperator": "airflow.providers.apache.hive.transfers.mssql_to_hive.MsSqlToHiveOperator", - "MsSqlToHiveTransfer": "airflow.providers.apache.hive.transfers.mssql_to_hive.MsSqlToHiveOperator", - }, - "mysql_operator": { - "MySqlOperator": "airflow.providers.mysql.operators.mysql.MySqlOperator", - }, - "mysql_to_hive": { - "MySqlToHiveOperator": "airflow.providers.apache.hive.transfers.mysql_to_hive.MySqlToHiveOperator", - "MySqlToHiveTransfer": "airflow.providers.apache.hive.transfers.mysql_to_hive.MySqlToHiveOperator", - }, - "oracle_operator": { - "OracleOperator": "airflow.providers.oracle.operators.oracle.OracleOperator", - }, - "papermill_operator": { - "PapermillOperator": "airflow.providers.papermill.operators.papermill.PapermillOperator", - }, - "pig_operator": { - "PigOperator": "airflow.providers.apache.pig.operators.pig.PigOperator", - }, - "postgres_operator": { - "Mapping": "airflow.providers.postgres.operators.postgres.Mapping", - "PostgresOperator": "airflow.providers.postgres.operators.postgres.PostgresOperator", - }, - "presto_check_operator": { - "SQLCheckOperator": "airflow.providers.common.sql.operators.sql.SQLCheckOperator", - "SQLIntervalCheckOperator": "airflow.providers.common.sql.operators.sql.SQLIntervalCheckOperator", - "SQLValueCheckOperator": "airflow.providers.common.sql.operators.sql.SQLValueCheckOperator", - "PrestoCheckOperator": "airflow.providers.common.sql.operators.sql.SQLCheckOperator", - "PrestoIntervalCheckOperator": "airflow.providers.common.sql.operators.sql.SQLIntervalCheckOperator", - "PrestoValueCheckOperator": "airflow.providers.common.sql.operators.sql.SQLValueCheckOperator", - }, - "presto_to_mysql": { - "PrestoToMySqlOperator": "airflow.providers.mysql.transfers.presto_to_mysql.PrestoToMySqlOperator", - "PrestoToMySqlTransfer": "airflow.providers.mysql.transfers.presto_to_mysql.PrestoToMySqlOperator", - }, - "python_operator": { - "BranchPythonOperator": "airflow.operators.python.BranchPythonOperator", - "PythonOperator": "airflow.operators.python.PythonOperator", - "PythonVirtualenvOperator": "airflow.operators.python.PythonVirtualenvOperator", - "ShortCircuitOperator": "airflow.operators.python.ShortCircuitOperator", - }, - "redshift_to_s3_operator": { - "RedshiftToS3Operator": "airflow.providers.amazon.aws.transfers.redshift_to_s3.RedshiftToS3Operator", - "RedshiftToS3Transfer": "airflow.providers.amazon.aws.transfers.redshift_to_s3.RedshiftToS3Operator", - }, - "s3_file_transform_operator": { - "S3FileTransformOperator": ( - "airflow.providers.amazon.aws.operators.s3_file_transform.S3FileTransformOperator" - ), - }, - "s3_to_hive_operator": { - "S3ToHiveOperator": "airflow.providers.apache.hive.transfers.s3_to_hive.S3ToHiveOperator", - "S3ToHiveTransfer": "airflow.providers.apache.hive.transfers.s3_to_hive.S3ToHiveOperator", - }, - "s3_to_redshift_operator": { - "S3ToRedshiftOperator": "airflow.providers.amazon.aws.transfers.s3_to_redshift.S3ToRedshiftOperator", - "S3ToRedshiftTransfer": "airflow.providers.amazon.aws.transfers.s3_to_redshift.S3ToRedshiftOperator", - }, - "slack_operator": { - "SlackAPIOperator": "airflow.providers.slack.operators.slack.SlackAPIOperator", - "SlackAPIPostOperator": "airflow.providers.slack.operators.slack.SlackAPIPostOperator", - }, - "sql": { - "BaseSQLOperator": "airflow.providers.common.sql.operators.sql.BaseSQLOperator", - "BranchSQLOperator": "airflow.providers.common.sql.operators.sql.BranchSQLOperator", - "SQLCheckOperator": "airflow.providers.common.sql.operators.sql.SQLCheckOperator", - "SQLColumnCheckOperator": "airflow.providers.common.sql.operators.sql.SQLColumnCheckOperator", - "SQLIntervalCheckOperator": "airflow.providers.common.sql.operators.sql.SQLIntervalCheckOperator", - "SQLTableCheckOperator": "airflow.providers.common.sql.operators.sql.SQLTableCheckOperator", - "SQLThresholdCheckOperator": "airflow.providers.common.sql.operators.sql.SQLThresholdCheckOperator", - "SQLValueCheckOperator": "airflow.providers.common.sql.operators.sql.SQLValueCheckOperator", - "_convert_to_float_if_possible": ( - "airflow.providers.common.sql.operators.sql._convert_to_float_if_possible" - ), - "parse_boolean": "airflow.providers.common.sql.operators.sql.parse_boolean", - }, - "sql_branch_operator": { - "BranchSQLOperator": "airflow.providers.common.sql.operators.sql.BranchSQLOperator", - "BranchSqlOperator": "airflow.providers.common.sql.operators.sql.BranchSQLOperator", - }, - "sqlite_operator": { - "SqliteOperator": "airflow.providers.sqlite.operators.sqlite.SqliteOperator", - }, - "subdag_operator": { - "SkippedStatePropagationOptions": "airflow.operators.subdag.SkippedStatePropagationOptions", - "SubDagOperator": "airflow.operators.subdag.SubDagOperator", - }, -} - -add_deprecated_classes(__deprecated_classes, __name__) diff --git a/airflow/operators/python.py b/airflow/operators/python.py index fdfe575fb927..09b2644beeee 100644 --- a/airflow/operators/python.py +++ b/airflow/operators/python.py @@ -49,19 +49,23 @@ from airflow.models.taskinstance import _CURRENT_CONTEXT from airflow.models.variable import Variable from airflow.operators.branch import BranchMixIn +from airflow.settings import _ENABLE_AIP_44 from airflow.typing_compat import Literal from airflow.utils import hashlib_wrapper from airflow.utils.context import context_copy_partial, context_get_outlet_events, context_merge from airflow.utils.file import get_unique_dag_module_name from airflow.utils.operator_helpers import ExecutionCallableRunner, KeywordParameters from airflow.utils.process_utils import execute_in_subprocess +from airflow.utils.pydantic import is_pydantic_2_installed from airflow.utils.python_virtualenv import prepare_virtualenv, write_python_script +from airflow.utils.session import create_session log = logging.getLogger(__name__) if TYPE_CHECKING: from pendulum.datetime import DateTime + from airflow.serialization.enums import Encoding from airflow.utils.context import Context @@ -442,6 +446,7 @@ def __init__( env_vars: dict[str, str] | None = None, inherit_env: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ): if ( @@ -481,6 +486,7 @@ def __init__( f"Expected one of {', '.join(map(repr, _SERIALIZERS))}" ) raise AirflowException(msg) + self.pickling_library = _SERIALIZERS[serializer] self.serializer: _SerializerTypeDef = serializer @@ -494,6 +500,7 @@ def __init__( ) self.env_vars = env_vars self.inherit_env = inherit_env + self.use_airflow_context = use_airflow_context @abstractmethod def _iter_serializable_context_keys(self): @@ -540,10 +547,15 @@ def _execute_python_callable_in_subprocess(self, python_path: Path): string_args_path = tmp_dir / "string_args.txt" script_path = tmp_dir / "script.py" termination_log_path = tmp_dir / "termination.log" + airflow_context_path = tmp_dir / "airflow_context.json" self._write_args(input_path) self._write_string_args(string_args_path) + if self.use_airflow_context and (not is_pydantic_2_installed() or not _ENABLE_AIP_44): + error_msg = "`get_current_context()` needs to be used with Pydantic 2 and AIP-44 enabled." + raise AirflowException(error_msg) + jinja_context = { "op_args": self.op_args, "op_kwargs": op_kwargs, @@ -551,6 +563,7 @@ def _execute_python_callable_in_subprocess(self, python_path: Path): "pickling_library": self.serializer, "python_callable": self.python_callable.__name__, "python_callable_source": self.get_python_source(), + "use_airflow_context": self.use_airflow_context, } if inspect.getfile(self.python_callable) == self.dag.fileloc: @@ -561,6 +574,19 @@ def _execute_python_callable_in_subprocess(self, python_path: Path): filename=os.fspath(script_path), render_template_as_native_obj=self.dag.render_template_as_native_obj, ) + if self.use_airflow_context: + from airflow.serialization.serialized_objects import BaseSerialization + + context = get_current_context() + with create_session() as session: + # FIXME: DetachedInstanceError + dag_run, task_instance = context["dag_run"], context["task_instance"] + session.add_all([dag_run, task_instance]) + serializable_context: dict[Encoding, Any] = BaseSerialization.serialize( + context, use_pydantic_models=True + ) + with airflow_context_path.open("w+") as file: + json.dump(serializable_context, file) env_vars = dict(os.environ) if self.inherit_env else {} if self.env_vars: @@ -575,6 +601,7 @@ def _execute_python_callable_in_subprocess(self, python_path: Path): os.fspath(output_path), os.fspath(string_args_path), os.fspath(termination_log_path), + os.fspath(airflow_context_path), ], env=env_vars, ) @@ -666,6 +693,7 @@ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator): :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize the args and result (pickle is default). This allows more complex types but requires you to include dill in your requirements. + :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ template_fields: Sequence[str] = tuple( @@ -694,6 +722,7 @@ def __init__( env_vars: dict[str, str] | None = None, inherit_env: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ): if ( @@ -715,6 +744,9 @@ def __init__( ) if not is_venv_installed(): raise AirflowException("PythonVirtualenvOperator requires virtualenv, please install it.") + if use_airflow_context and (not expect_airflow and not system_site_packages): + error_msg = "use_airflow_context is set to True, but expect_airflow and system_site_packages are set to False." + raise AirflowException(error_msg) if not requirements: self.requirements: list[str] = [] elif isinstance(requirements, str): @@ -744,6 +776,7 @@ def __init__( env_vars=env_vars, inherit_env=inherit_env, use_dill=use_dill, + use_airflow_context=use_airflow_context, **kwargs, ) @@ -962,6 +995,7 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator): :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize the args and result (pickle is default). This allows more complex types but requires you to include dill in your requirements. + :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ template_fields: Sequence[str] = tuple({"python"}.union(PythonOperator.template_fields)) @@ -983,10 +1017,14 @@ def __init__( env_vars: dict[str, str] | None = None, inherit_env: bool = True, use_dill: bool = False, + use_airflow_context: bool = False, **kwargs, ): if not python: raise ValueError("Python Path must be defined in ExternalPythonOperator") + if use_airflow_context and not expect_airflow: + error_msg = "use_airflow_context is set to True, but expect_airflow is set to False." + raise AirflowException(error_msg) self.python = python self.expect_pendulum = expect_pendulum super().__init__( @@ -1002,6 +1040,7 @@ def __init__( env_vars=env_vars, inherit_env=inherit_env, use_dill=use_dill, + use_airflow_context=use_airflow_context, **kwargs, ) diff --git a/airflow/operators/subdag.py b/airflow/operators/subdag.py deleted file mode 100644 index 7cbfa0319875..000000000000 --- a/airflow/operators/subdag.py +++ /dev/null @@ -1,247 +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. -""" -This module is deprecated. Please use :mod:`airflow.utils.task_group`. - -The module which provides a way to nest your DAGs and so your levels of complexity. -""" - -from __future__ import annotations - -import warnings -from enum import Enum -from typing import TYPE_CHECKING - -from sqlalchemy import select - -from airflow.api.common.experimental.get_task_instance import get_task_instance -from airflow.api_internal.internal_api_call import InternalApiConfig -from airflow.exceptions import AirflowException, RemovedInAirflow3Warning, TaskInstanceNotFound -from airflow.models import DagRun -from airflow.models.dag import DagContext -from airflow.models.pool import Pool -from airflow.models.taskinstance import TaskInstance -from airflow.sensors.base import BaseSensorOperator -from airflow.utils.session import NEW_SESSION, create_session, provide_session -from airflow.utils.state import DagRunState, TaskInstanceState -from airflow.utils.types import DagRunType - -if TYPE_CHECKING: - from datetime import datetime - - from sqlalchemy.orm.session import Session - - from airflow.models.dag import DAG - from airflow.utils.context import Context - - -class SkippedStatePropagationOptions(Enum): - """Available options for skipped state propagation of subdag's tasks to parent dag tasks.""" - - ALL_LEAVES = "all_leaves" - ANY_LEAF = "any_leaf" - - -class SubDagOperator(BaseSensorOperator): - """ - This class is deprecated, please use :class:`airflow.utils.task_group.TaskGroup`. - - This runs a sub dag. By convention, a sub dag's dag_id - should be prefixed by its parent and a dot. As in `parent.child`. - Although SubDagOperator can occupy a pool/concurrency slot, - user can specify the mode=reschedule so that the slot will be - released periodically to avoid potential deadlock. - - :param subdag: the DAG object to run as a subdag of the current DAG. - :param session: sqlalchemy session - :param conf: Configuration for the subdag - :param propagate_skipped_state: by setting this argument you can define - whether the skipped state of leaf task(s) should be propagated to the - parent dag's downstream task. - """ - - ui_color = "#555" - ui_fgcolor = "#fff" - - subdag: DAG - - @provide_session - def __init__( - self, - *, - subdag: DAG, - session: Session = NEW_SESSION, - conf: dict | None = None, - propagate_skipped_state: SkippedStatePropagationOptions | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.subdag = subdag - self.conf = conf - self.propagate_skipped_state = propagate_skipped_state - - self._validate_dag(kwargs) - if not InternalApiConfig.get_use_internal_api(): - self._validate_pool(session) - - warnings.warn( - """This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.""", - RemovedInAirflow3Warning, - stacklevel=4, - ) - - def _validate_dag(self, kwargs): - dag = kwargs.get("dag") or DagContext.get_current_dag() - - if not dag: - raise AirflowException("Please pass in the `dag` param or call within a DAG context manager") - - if dag.dag_id + "." + kwargs["task_id"] != self.subdag.dag_id: - raise AirflowException( - f"The subdag's dag_id should have the form '{{parent_dag_id}}.{{this_task_id}}'. " - f"Expected '{dag.dag_id}.{kwargs['task_id']}'; received '{self.subdag.dag_id}'." - ) - - def _validate_pool(self, session): - if self.pool: - conflicts = [t for t in self.subdag.tasks if t.pool == self.pool] - if conflicts: - # only query for pool conflicts if one may exist - pool = session.scalar(select(Pool).where(Pool.slots == 1, Pool.pool == self.pool)) - if pool and any(t.pool == self.pool for t in self.subdag.tasks): - raise AirflowException( - f"SubDagOperator {self.task_id} and subdag task{'s' if len(conflicts) > 1 else ''} " - f"{', '.join(t.task_id for t in conflicts)} both use pool {self.pool}, " - f"but the pool only has 1 slot. The subdag tasks will never run." - ) - - def _get_dagrun(self, execution_date): - dag_runs = DagRun.find( - dag_id=self.subdag.dag_id, - execution_date=execution_date, - ) - return dag_runs[0] if dag_runs else None - - def _reset_dag_run_and_task_instances(self, dag_run: DagRun, execution_date: datetime) -> None: - """ - Set task instance states to allow for execution. - - The state of the DAG run will be set to RUNNING, and failed task - instances to ``None`` for scheduler to pick up. - - :param dag_run: DAG run to reset. - :param execution_date: Execution date to select task instances. - """ - with create_session() as session: - dag_run.state = DagRunState.RUNNING - session.merge(dag_run) - failed_task_instances = session.scalars( - select(TaskInstance) - .where(TaskInstance.dag_id == self.subdag.dag_id) - .where(TaskInstance.execution_date == execution_date) - .where(TaskInstance.state.in_((TaskInstanceState.FAILED, TaskInstanceState.UPSTREAM_FAILED))) - ) - - for task_instance in failed_task_instances: - task_instance.state = None - session.merge(task_instance) - session.commit() - - def pre_execute(self, context): - super().pre_execute(context) - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date) - - if dag_run is None: - if context["data_interval_start"] is None or context["data_interval_end"] is None: - data_interval: tuple[datetime, datetime] | None = None - else: - data_interval = (context["data_interval_start"], context["data_interval_end"]) - dag_run = self.subdag.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=execution_date, - state=DagRunState.RUNNING, - conf=self.conf, - external_trigger=True, - data_interval=data_interval, - ) - self.log.info("Created DagRun: %s", dag_run.run_id) - else: - self.log.info("Found existing DagRun: %s", dag_run.run_id) - if dag_run.state == DagRunState.FAILED: - self._reset_dag_run_and_task_instances(dag_run, execution_date) - - def poke(self, context: Context): - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date=execution_date) - return dag_run.state != DagRunState.RUNNING - - def post_execute(self, context, result=None): - super().post_execute(context) - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date=execution_date) - self.log.info("Execution finished. State is %s", dag_run.state) - - if dag_run.state != DagRunState.SUCCESS: - raise AirflowException(f"Expected state: SUCCESS. Actual state: {dag_run.state}") - - if self.propagate_skipped_state and self._check_skipped_states(context): - self._skip_downstream_tasks(context) - - def _check_skipped_states(self, context): - leaves_tis = self._get_leaves_tis(context["execution_date"]) - - if self.propagate_skipped_state == SkippedStatePropagationOptions.ANY_LEAF: - return any(ti.state == TaskInstanceState.SKIPPED for ti in leaves_tis) - if self.propagate_skipped_state == SkippedStatePropagationOptions.ALL_LEAVES: - return all(ti.state == TaskInstanceState.SKIPPED for ti in leaves_tis) - raise AirflowException( - f"Unimplemented SkippedStatePropagationOptions {self.propagate_skipped_state} used." - ) - - def _get_leaves_tis(self, execution_date): - leaves_tis = [] - for leaf in self.subdag.leaves: - try: - ti = get_task_instance( - dag_id=self.subdag.dag_id, task_id=leaf.task_id, execution_date=execution_date - ) - leaves_tis.append(ti) - except TaskInstanceNotFound: - continue - return leaves_tis - - def _skip_downstream_tasks(self, context): - self.log.info( - "Skipping downstream tasks because propagate_skipped_state is set to %s " - "and skipped task(s) were found.", - self.propagate_skipped_state, - ) - - downstream_tasks = context["task"].downstream_list - self.log.debug("Downstream task_ids %s", downstream_tasks) - - if downstream_tasks: - self.skip( - context["dag_run"], - context["execution_date"], - downstream_tasks, - map_index=context["ti"].map_index, - ) - - self.log.info("Done.") diff --git a/airflow/providers/airbyte/CHANGELOG.rst b/airflow/providers/airbyte/CHANGELOG.rst index 9dbdc56724a6..854d5b1cc739 100644 --- a/airflow/providers/airbyte/CHANGELOG.rst +++ b/airflow/providers/airbyte/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +3.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.8.1 ..... diff --git a/airflow/providers/airbyte/__init__.py b/airflow/providers/airbyte/__init__.py index e54159bdbf31..91c578c0a19e 100644 --- a/airflow/providers/airbyte/__init__.py +++ b/airflow/providers/airbyte/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.8.1" +__version__ = "3.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/airbyte/provider.yaml b/airflow/providers/airbyte/provider.yaml index f703ff62be9c..d799066fe25b 100644 --- a/airflow/providers/airbyte/provider.yaml +++ b/airflow/providers/airbyte/provider.yaml @@ -22,9 +22,10 @@ description: | `Airbyte `__ state: ready -source-date-epoch: 1716286241 +source-date-epoch: 1723968864 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.9.0 - 3.8.1 - 3.8.0 - 3.7.0 diff --git a/airflow/providers/alibaba/CHANGELOG.rst b/airflow/providers/alibaba/CHANGELOG.rst index f8ac1fde7dcb..832b3018c4da 100644 --- a/airflow/providers/alibaba/CHANGELOG.rst +++ b/airflow/providers/alibaba/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +2.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.8.1 ..... diff --git a/airflow/providers/alibaba/__init__.py b/airflow/providers/alibaba/__init__.py index 2ba68d513824..86e9e067529b 100644 --- a/airflow/providers/alibaba/__init__.py +++ b/airflow/providers/alibaba/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.8.1" +__version__ = "2.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/alibaba/provider.yaml b/airflow/providers/alibaba/provider.yaml index 0c70ee72adde..b5dcfc258264 100644 --- a/airflow/providers/alibaba/provider.yaml +++ b/airflow/providers/alibaba/provider.yaml @@ -22,9 +22,10 @@ description: | Alibaba Cloud integration (including `Alibaba Cloud `__). state: ready -source-date-epoch: 1716286324 +source-date-epoch: 1723968880 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.9.0 - 2.8.1 - 2.8.0 - 2.7.3 diff --git a/airflow/providers/amazon/CHANGELOG.rst b/airflow/providers/amazon/CHANGELOG.rst index 1dfc37909cc4..32ac7cce53a7 100644 --- a/airflow/providers/amazon/CHANGELOG.rst +++ b/airflow/providers/amazon/CHANGELOG.rst @@ -26,19 +26,44 @@ Changelog --------- -Main +8.28.0 ...... +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + .. warning:: When deferrable mode was introduced for ``RedshiftDataOperator``, in version 8.17.0, tasks configured with ``deferrable=True`` and ``wait_for_completion=True`` wouldn't enter the deferred state. Instead, the task would occupy an executor slot until the statement was completed. A workaround may have been to set ``wait_for_completion=False``. In this version, tasks set up with ``wait_for_completion=False`` will not wait anymore, regardless of the value of ``deferrable``. +Features +~~~~~~~~ + +* ``Add incremental export and cross account export functionality in 'DynamoDBToS3Operator' (#41304)`` +* ``EKS Overrides for AWS Batch submit_job (#40718)`` + Bug Fixes ~~~~~~~~~ -* ``Fix deferred mode for 'RedshiftDataOperator' (#41206)`` +* ``Fix 'AwsTaskLogFetcher' missing logs (#41515)`` +* ``Fix the Exception name and unpin dependency in 'RdsHook' (#41256)`` +* ``Fix RedshiftDataOperator not running in deferred mode as expected (#41206)`` + +Misc +~~~~ + +* ``Partial fix for example_dynamodb_to_s3.py (#41517)`` +* ``Remove deprecated code is AWS provider (#41407)`` +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``Limit moto temporarily - 5.0.12 is breaking our tests (#41244)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``typo (#41381)`` 8.27.0 ...... diff --git a/airflow/providers/amazon/__init__.py b/airflow/providers/amazon/__init__.py index 4d83b19a6510..cdd5f2affc80 100644 --- a/airflow/providers/amazon/__init__.py +++ b/airflow/providers/amazon/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "8.27.0" +__version__ = "8.28.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/amazon/aws/hooks/rds.py b/airflow/providers/amazon/aws/hooks/rds.py index 588d78c78262..f58b089d7df1 100644 --- a/airflow/providers/amazon/aws/hooks/rds.py +++ b/airflow/providers/amazon/aws/hooks/rds.py @@ -155,7 +155,7 @@ def get_export_task_state(self, export_task_id: str) -> str: try: response = self.conn.describe_export_tasks(ExportTaskIdentifier=export_task_id) except self.conn.exceptions.ClientError as e: - if e.response["Error"]["Code"] == "ExportTaskNotFoundFault": + if e.response["Error"]["Code"] in ("ExportTaskNotFound", "ExportTaskNotFoundFault"): raise AirflowNotFoundException(e) raise e return response["ExportTasks"][0]["Status"].lower() @@ -196,7 +196,7 @@ def get_event_subscription_state(self, subscription_name: str) -> str: try: response = self.conn.describe_event_subscriptions(SubscriptionName=subscription_name) except self.conn.exceptions.ClientError as e: - if e.response["Error"]["Code"] == "SubscriptionNotFoundFault": + if e.response["Error"]["Code"] in ("SubscriptionNotFoundFault", "SubscriptionNotFound"): raise AirflowNotFoundException(e) raise e return response["EventSubscriptionsList"][0]["Status"].lower() diff --git a/airflow/providers/amazon/aws/sensors/athena.py b/airflow/providers/amazon/aws/sensors/athena.py index 38f2bb54f8b3..1dc8a4fe3335 100644 --- a/airflow/providers/amazon/aws/sensors/athena.py +++ b/airflow/providers/amazon/aws/sensors/athena.py @@ -25,7 +25,7 @@ if TYPE_CHECKING: from airflow.utils.context import Context -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.athena import AthenaHook @@ -88,11 +88,7 @@ def poke(self, context: Context) -> bool: state = self.hook.poll_query_status(self.query_execution_id, self.max_retries, self.sleep_time) if state in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = "Athena sensor failed" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException("Athena sensor failed") if state in self.INTERMEDIATE_STATES: return False diff --git a/airflow/providers/amazon/aws/sensors/batch.py b/airflow/providers/amazon/aws/sensors/batch.py index c5dcb0e46de6..9c1a29f8098f 100644 --- a/airflow/providers/amazon/aws/sensors/batch.py +++ b/airflow/providers/amazon/aws/sensors/batch.py @@ -86,18 +86,7 @@ def poke(self, context: Context) -> bool: if state in BatchClientHook.INTERMEDIATE_STATES: return False - if state == BatchClientHook.FAILURE_STATE: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Batch sensor failed. AWS Batch job status: {state}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) - - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Batch sensor failed. Unknown AWS Batch job status: {state}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Batch sensor failed. AWS Batch job status: {state}") def execute(self, context: Context) -> None: if not self.deferrable: @@ -127,12 +116,7 @@ def execute_complete(self, context: Context, event: dict[str, Any]) -> None: Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ if event["status"] != "success": - message = f"Error while running job: {event}" - # TODO: remove this if-else block when min_airflow_version is set to higher than the version that - # changed in https://github.com/apache/airflow/pull/33424 is released (2.7.1) - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Error while running job: {event}") job_id = event["job_id"] self.log.info("Batch Job %s complete", job_id) @@ -198,11 +182,7 @@ def poke(self, context: Context) -> bool: ) if not response["computeEnvironments"]: - message = f"AWS Batch compute environment {self.compute_environment} not found" - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"AWS Batch compute environment {self.compute_environment} not found") status = response["computeEnvironments"][0]["status"] @@ -212,11 +192,9 @@ def poke(self, context: Context) -> bool: if status in BatchClientHook.COMPUTE_ENVIRONMENT_INTERMEDIATE_STATUS: return False - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"AWS Batch compute environment failed. AWS Batch compute environment status: {status}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException( + f"AWS Batch compute environment failed. AWS Batch compute environment status: {status}" + ) class BatchJobQueueSensor(BaseSensorOperator): @@ -276,11 +254,7 @@ def poke(self, context: Context) -> bool: if self.treat_non_existing_as_deleted: return True else: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"AWS Batch job queue {self.job_queue} not found" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"AWS Batch job queue {self.job_queue} not found") status = response["jobQueues"][0]["status"] diff --git a/airflow/providers/amazon/aws/sensors/bedrock.py b/airflow/providers/amazon/aws/sensors/bedrock.py index 853288655486..e9157ab9c12a 100644 --- a/airflow/providers/amazon/aws/sensors/bedrock.py +++ b/airflow/providers/amazon/aws/sensors/bedrock.py @@ -21,7 +21,7 @@ from typing import TYPE_CHECKING, Any, Sequence, TypeVar from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.bedrock import BedrockAgentHook, BedrockHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.bedrock import ( @@ -76,9 +76,6 @@ def __init__( def poke(self, context: Context, **kwargs) -> bool: state = self.get_state() if state in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(self.FAILURE_MESSAGE) raise AirflowException(self.FAILURE_MESSAGE) return state not in self.INTERMEDIATE_STATES diff --git a/airflow/providers/amazon/aws/sensors/cloud_formation.py b/airflow/providers/amazon/aws/sensors/cloud_formation.py index f67278ecec38..ba07433bf0c8 100644 --- a/airflow/providers/amazon/aws/sensors/cloud_formation.py +++ b/airflow/providers/amazon/aws/sensors/cloud_formation.py @@ -27,7 +27,6 @@ if TYPE_CHECKING: from airflow.utils.context import Context -from airflow.exceptions import AirflowSkipException from airflow.providers.amazon.aws.hooks.cloud_formation import CloudFormationHook @@ -67,11 +66,7 @@ def poke(self, context: Context): if stack_status in ("CREATE_IN_PROGRESS", None): return False - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Stack {self.stack_name} in bad state: {stack_status}" - if self.soft_fail: - raise AirflowSkipException(message) - raise ValueError(message) + raise ValueError(f"Stack {self.stack_name} in bad state: {stack_status}") class CloudFormationDeleteStackSensor(AwsBaseSensor[CloudFormationHook]): @@ -119,8 +114,4 @@ def poke(self, context: Context): if stack_status == "DELETE_IN_PROGRESS": return False - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Stack {self.stack_name} in bad state: {stack_status}" - if self.soft_fail: - raise AirflowSkipException(message) - raise ValueError(message) + raise ValueError(f"Stack {self.stack_name} in bad state: {stack_status}") diff --git a/airflow/providers/amazon/aws/sensors/comprehend.py b/airflow/providers/amazon/aws/sensors/comprehend.py index 42344f65e983..545f7b02fc8d 100644 --- a/airflow/providers/amazon/aws/sensors/comprehend.py +++ b/airflow/providers/amazon/aws/sensors/comprehend.py @@ -20,7 +20,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.comprehend import ComprehendHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.comprehend import ( @@ -71,9 +71,6 @@ def __init__( def poke(self, context: Context, **kwargs) -> bool: state = self.get_state() if state in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(self.FAILURE_MESSAGE) raise AirflowException(self.FAILURE_MESSAGE) return state not in self.INTERMEDIATE_STATES @@ -241,9 +238,6 @@ def poke(self, context: Context, **kwargs) -> bool: ) if status in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(self.FAILURE_MESSAGE) raise AirflowException(self.FAILURE_MESSAGE) if status in self.SUCCESS_STATES: diff --git a/airflow/providers/amazon/aws/sensors/dms.py b/airflow/providers/amazon/aws/sensors/dms.py index 864a3b5276c3..2ea52ea0b5c3 100644 --- a/airflow/providers/amazon/aws/sensors/dms.py +++ b/airflow/providers/amazon/aws/sensors/dms.py @@ -21,7 +21,7 @@ from deprecated import deprecated -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.dms import DmsHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.utils.mixins import aws_template_fields @@ -75,11 +75,9 @@ def get_hook(self) -> DmsHook: def poke(self, context: Context): if not (status := self.hook.get_task_status(self.replication_task_arn)): - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Failed to read task status, task with ARN {self.replication_task_arn} not found" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException( + f"Failed to read task status, task with ARN {self.replication_task_arn} not found" + ) self.log.info("DMS Replication task (%s) has status: %s", self.replication_task_arn, status) @@ -87,11 +85,7 @@ def poke(self, context: Context): return True if status in self.termination_statuses: - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Unexpected status: {status}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Unexpected status: {status}") return False diff --git a/airflow/providers/amazon/aws/sensors/ec2.py b/airflow/providers/amazon/aws/sensors/ec2.py index 778bc49caa52..0736c63393ae 100644 --- a/airflow/providers/amazon/aws/sensors/ec2.py +++ b/airflow/providers/amazon/aws/sensors/ec2.py @@ -21,7 +21,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.ec2 import EC2Hook from airflow.providers.amazon.aws.triggers.ec2 import EC2StateSensorTrigger from airflow.providers.amazon.aws.utils import validate_execute_complete_event @@ -97,8 +97,4 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Error: {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Error: {event}") diff --git a/airflow/providers/amazon/aws/sensors/ecs.py b/airflow/providers/amazon/aws/sensors/ecs.py index 02a212fbde0d..aba3e5592268 100644 --- a/airflow/providers/amazon/aws/sensors/ecs.py +++ b/airflow/providers/amazon/aws/sensors/ecs.py @@ -19,7 +19,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Sequence -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.ecs import ( EcsClusterStates, EcsHook, @@ -37,11 +37,9 @@ def _check_failed(current_state, target_state, failure_states, soft_fail: bool) -> None: if (current_state != target_state) and (current_state in failure_states): - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Terminal state reached. Current state: {current_state}, Expected state: {target_state}" - if soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException( + f"Terminal state reached. Current state: {current_state}, Expected state: {target_state}" + ) class EcsBaseSensor(AwsBaseSensor[EcsHook]): diff --git a/airflow/providers/amazon/aws/sensors/eks.py b/airflow/providers/amazon/aws/sensors/eks.py index a5dcdeb0ef18..79e160b00740 100644 --- a/airflow/providers/amazon/aws/sensors/eks.py +++ b/airflow/providers/amazon/aws/sensors/eks.py @@ -22,7 +22,7 @@ from functools import cached_property from typing import TYPE_CHECKING, Sequence -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.eks import ( ClusterStates, EksHook, @@ -106,12 +106,10 @@ def poke(self, context: Context) -> bool: state = self.get_state() self.log.info("Current state: %s", state) if state in (self.get_terminal_states() - {self.target_state}): - # If we reach a terminal state which is not the target state: - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Terminal state reached. Current state: {state}, Expected state: {self.target_state}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + # If we reach a terminal state which is not the target state + raise AirflowException( + f"Terminal state reached. Current state: {state}, Expected state: {self.target_state}" + ) return state == self.target_state @abstractmethod diff --git a/airflow/providers/amazon/aws/sensors/emr.py b/airflow/providers/amazon/aws/sensors/emr.py index 19e026e7a6c4..e79642d35c69 100644 --- a/airflow/providers/amazon/aws/sensors/emr.py +++ b/airflow/providers/amazon/aws/sensors/emr.py @@ -27,7 +27,6 @@ from airflow.exceptions import ( AirflowException, AirflowProviderDeprecationWarning, - AirflowSkipException, ) from airflow.providers.amazon.aws.hooks.emr import EmrContainerHook, EmrHook, EmrServerlessHook from airflow.providers.amazon.aws.links.emr import EmrClusterLink, EmrLogsLink, get_log_uri @@ -91,11 +90,7 @@ def poke(self, context: Context): return True if state in self.failed_states: - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"EMR job failed: {self.failure_message_from_response(response)}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"EMR job failed: {self.failure_message_from_response(response)}") return False @@ -172,11 +167,9 @@ def poke(self, context: Context) -> bool: state = response["jobRun"]["state"] if state in EmrServerlessHook.JOB_FAILURE_STATES: - failure_message = f"EMR Serverless job failed: {self.failure_message_from_response(response)}" - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(failure_message) - raise AirflowException(failure_message) + raise AirflowException( + f"EMR Serverless job failed: {self.failure_message_from_response(response)}" + ) return state in self.target_states @@ -234,13 +227,9 @@ def poke(self, context: Context) -> bool: state = response["application"]["state"] if state in EmrServerlessHook.APPLICATION_FAILURE_STATES: - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - failure_message = ( + raise AirflowException( f"EMR Serverless application failed: {self.failure_message_from_response(response)}" ) - if self.soft_fail: - raise AirflowSkipException(failure_message) - raise AirflowException(failure_message) return state in self.target_states @@ -328,11 +317,7 @@ def poke(self, context: Context) -> bool: ) if state in self.FAILURE_STATES: - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = "EMR Containers sensor failed" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException("EMR Containers sensor failed") if state in self.INTERMEDIATE_STATES: return False @@ -370,11 +355,7 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Error while running job: {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Error while running job: {event}") self.log.info("Job completed.") @@ -563,11 +544,7 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Error while running job: {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Error while running job: {event}") self.log.info("Job completed.") @@ -696,10 +673,6 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if check when min_airflow_version is set to higher than 2.7.1 - message = f"Error while running job: {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Error while running job: {event}") self.log.info("Job %s completed.", self.job_flow_id) diff --git a/airflow/providers/amazon/aws/sensors/glacier.py b/airflow/providers/amazon/aws/sensors/glacier.py index 7a65fc6fc31e..eb87d8a51a21 100644 --- a/airflow/providers/amazon/aws/sensors/glacier.py +++ b/airflow/providers/amazon/aws/sensors/glacier.py @@ -20,7 +20,7 @@ from enum import Enum from typing import TYPE_CHECKING, Any, Sequence -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.glacier import GlacierHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.utils.mixins import aws_template_fields @@ -93,10 +93,6 @@ def poke(self, context: Context) -> bool: self.log.warning("Code status: %s", response["StatusCode"]) return False else: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = ( + raise AirflowException( f'Sensor failed. Job status: {response["Action"]}, code status: {response["StatusCode"]}' ) - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) diff --git a/airflow/providers/amazon/aws/sensors/glue.py b/airflow/providers/amazon/aws/sensors/glue.py index 8493c2fd4ab6..062e4ab3efd0 100644 --- a/airflow/providers/amazon/aws/sensors/glue.py +++ b/airflow/providers/amazon/aws/sensors/glue.py @@ -86,9 +86,6 @@ def poke(self, context: Context): elif job_state in self.errored_states: job_error_message = "Exiting Job %s Run State: %s", self.run_id, job_state self.log.info(job_error_message) - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(job_error_message) raise AirflowException(job_error_message) else: return False @@ -223,9 +220,6 @@ def poke(self, context: Context): f": {response.get('ErrorString')}" ) self.log.info(job_error_message) - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(job_error_message) raise AirflowException(job_error_message) else: return False @@ -343,9 +337,6 @@ def poke(self, context: Context) -> bool: f": {response.get('ErrorString')}" ) self.log.info(job_error_message) - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(job_error_message) raise AirflowException(job_error_message) else: return False diff --git a/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py b/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py index f397a446becb..af125e2dda6a 100644 --- a/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py +++ b/airflow/providers/amazon/aws/sensors/glue_catalog_partition.py @@ -23,7 +23,7 @@ from deprecated import deprecated from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.glue_catalog import GlueCatalogHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.glue import GlueCatalogPartitionTrigger @@ -127,11 +127,7 @@ def execute_complete(self, context: Context, event: dict | None = None) -> None: event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Trigger error: event is {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Trigger error: event is {event}") self.log.info("Partition exists in the Glue Catalog") @deprecated(reason="use `hook` property instead.", category=AirflowProviderDeprecationWarning) diff --git a/airflow/providers/amazon/aws/sensors/glue_crawler.py b/airflow/providers/amazon/aws/sensors/glue_crawler.py index ce35aef2cfb2..2d4396c010c3 100644 --- a/airflow/providers/amazon/aws/sensors/glue_crawler.py +++ b/airflow/providers/amazon/aws/sensors/glue_crawler.py @@ -21,7 +21,7 @@ from deprecated import deprecated -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.glue_crawler import GlueCrawlerHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.utils.mixins import aws_template_fields @@ -75,11 +75,7 @@ def poke(self, context: Context): self.log.info("Status: %s", crawler_status) return True else: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Status: {crawler_status}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Status: {crawler_status}") else: return False diff --git a/airflow/providers/amazon/aws/sensors/kinesis_analytics.py b/airflow/providers/amazon/aws/sensors/kinesis_analytics.py index 2c02e050c77f..673445e67d5f 100644 --- a/airflow/providers/amazon/aws/sensors/kinesis_analytics.py +++ b/airflow/providers/amazon/aws/sensors/kinesis_analytics.py @@ -19,7 +19,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.kinesis_analytics import KinesisAnalyticsV2Hook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.kinesis_analytics import ( @@ -80,9 +80,6 @@ def poke(self, context: Context, **kwargs) -> bool: ) if status in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(self.FAILURE_MESSAGE) raise AirflowException(self.FAILURE_MESSAGE) if status in self.SUCCESS_STATES: diff --git a/airflow/providers/amazon/aws/sensors/lambda_function.py b/airflow/providers/amazon/aws/sensors/lambda_function.py index c54dfbe8b71f..8e01d40235ec 100644 --- a/airflow/providers/amazon/aws/sensors/lambda_function.py +++ b/airflow/providers/amazon/aws/sensors/lambda_function.py @@ -19,7 +19,7 @@ from typing import TYPE_CHECKING, Any, Sequence -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.lambda_function import LambdaHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.utils import trim_none_values @@ -78,10 +78,8 @@ def poke(self, context: Context) -> bool: state = self.hook.conn.get_function(**trim_none_values(get_function_args))["Configuration"]["State"] if state in self.FAILURE_STATES: - message = "Lambda function state sensor failed because the Lambda is in a failed state" - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException( + "Lambda function state sensor failed because the Lambda is in a failed state" + ) return state in self.target_states diff --git a/airflow/providers/amazon/aws/sensors/opensearch_serverless.py b/airflow/providers/amazon/aws/sensors/opensearch_serverless.py index 7f5f650e0ee0..0f539f9bfb82 100644 --- a/airflow/providers/amazon/aws/sensors/opensearch_serverless.py +++ b/airflow/providers/amazon/aws/sensors/opensearch_serverless.py @@ -19,7 +19,7 @@ from typing import TYPE_CHECKING, Any, Sequence from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowSkipException +from airflow.exceptions import AirflowException from airflow.providers.amazon.aws.hooks.opensearch_serverless import OpenSearchServerlessHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.opensearch_serverless import ( @@ -104,9 +104,6 @@ def poke(self, context: Context, **kwargs) -> bool: state = self.hook.conn.batch_get_collection(**call_args)["collectionDetails"][0]["status"] if state in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(self.FAILURE_MESSAGE) raise AirflowException(self.FAILURE_MESSAGE) if state in self.INTERMEDIATE_STATES: diff --git a/airflow/providers/amazon/aws/sensors/quicksight.py b/airflow/providers/amazon/aws/sensors/quicksight.py index 321fa56dd235..848c0dc7048f 100644 --- a/airflow/providers/amazon/aws/sensors/quicksight.py +++ b/airflow/providers/amazon/aws/sensors/quicksight.py @@ -22,7 +22,7 @@ from deprecated import deprecated -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.quicksight import QuickSightHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor @@ -74,10 +74,7 @@ def poke(self, context: Context) -> bool: self.log.info("QuickSight Status: %s", quicksight_ingestion_state) if quicksight_ingestion_state in self.errored_statuses: error = self.hook.get_error_info(None, self.data_set_id, self.ingestion_id) - message = f"The QuickSight Ingestion failed. Error info: {error}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"The QuickSight Ingestion failed. Error info: {error}") return quicksight_ingestion_state == self.success_status @cached_property diff --git a/airflow/providers/amazon/aws/sensors/redshift_cluster.py b/airflow/providers/amazon/aws/sensors/redshift_cluster.py index 7d2f4ba4724b..243c71e61fe7 100644 --- a/airflow/providers/amazon/aws/sensors/redshift_cluster.py +++ b/airflow/providers/amazon/aws/sensors/redshift_cluster.py @@ -23,7 +23,7 @@ from deprecated import deprecated from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.redshift_cluster import RedshiftHook from airflow.providers.amazon.aws.triggers.redshift_cluster import RedshiftClusterTrigger from airflow.providers.amazon.aws.utils import validate_execute_complete_event @@ -93,11 +93,7 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None status = event["status"] if status == "error": - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"{event['status']}: {event['message']}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"{event['status']}: {event['message']}") elif status == "success": self.log.info("%s completed successfully.", self.task_id) self.log.info("Cluster Identifier %s is in %s state", self.cluster_identifier, self.target_status) diff --git a/airflow/providers/amazon/aws/sensors/s3.py b/airflow/providers/amazon/aws/sensors/s3.py index 9c524494cdeb..2f32fff3d30a 100644 --- a/airflow/providers/amazon/aws/sensors/s3.py +++ b/airflow/providers/amazon/aws/sensors/s3.py @@ -33,7 +33,7 @@ if TYPE_CHECKING: from airflow.utils.context import Context -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.amazon.aws.triggers.s3 import S3KeysUnchangedTrigger, S3KeyTrigger from airflow.sensors.base import BaseSensorOperator, poke_mode_only @@ -219,9 +219,6 @@ def execute_complete(self, context: Context, event: dict[str, Any]) -> None: if not found_keys: self._defer() elif event["status"] == "error": - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(event["message"]) raise AirflowException(event["message"]) @deprecated(reason="use `hook` property instead.", category=AirflowProviderDeprecationWarning) @@ -342,14 +339,9 @@ def is_keys_unchanged(self, current_objects: set[str]) -> bool: ) return False - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = ( - f"Illegal behavior: objects were deleted in" - f" {os.path.join(self.bucket_name, self.prefix)} between pokes." + raise AirflowException( + f"Illegal behavior: objects were deleted in {os.path.join(self.bucket_name, self.prefix)} between pokes." ) - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) if self.last_activity_time: self.inactivity_seconds = int((datetime.now() - self.last_activity_time).total_seconds()) @@ -411,8 +403,5 @@ def execute_complete(self, context: Context, event: dict[str, Any] | None = None event = validate_execute_complete_event(event) if event and event["status"] == "error": - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - if self.soft_fail: - raise AirflowSkipException(event["message"]) raise AirflowException(event["message"]) return None diff --git a/airflow/providers/amazon/aws/sensors/sagemaker.py b/airflow/providers/amazon/aws/sensors/sagemaker.py index 97ac8ad483c1..b01e24cd5b81 100644 --- a/airflow/providers/amazon/aws/sensors/sagemaker.py +++ b/airflow/providers/amazon/aws/sensors/sagemaker.py @@ -22,7 +22,7 @@ from deprecated import deprecated -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.sagemaker import LogState, SageMakerHook from airflow.sensors.base import BaseSensorOperator @@ -65,11 +65,9 @@ def poke(self, context: Context): return False if state in self.failed_states(): failed_reason = self.get_failed_reason_from_response(response) - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Sagemaker {self.resource_type} failed for the following reason: {failed_reason}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException( + f"Sagemaker {self.resource_type} failed for the following reason: {failed_reason}" + ) return True def non_terminal_states(self) -> set[str]: diff --git a/airflow/providers/amazon/aws/sensors/sqs.py b/airflow/providers/amazon/aws/sensors/sqs.py index 657c6d9599c7..d04a8cf820b0 100644 --- a/airflow/providers/amazon/aws/sensors/sqs.py +++ b/airflow/providers/amazon/aws/sensors/sqs.py @@ -25,7 +25,7 @@ from deprecated import deprecated from airflow.configuration import conf -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.sqs import SqsHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.triggers.sqs import SqsSensorTrigger @@ -160,11 +160,7 @@ def execute_complete(self, context: Context, event: dict | None = None) -> None: event = validate_execute_complete_event(event) if event["status"] != "success": - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Trigger error: event is {event}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Trigger error: event is {event}") context["ti"].xcom_push(key="messages", value=event["message_batch"]) def poll_sqs(self, sqs_conn: BaseAwsConnection) -> Collection: @@ -221,11 +217,7 @@ def poke(self, context: Context): response = self.hook.conn.delete_message_batch(QueueUrl=self.sqs_queue, Entries=entries) if "Successful" not in response: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - error_message = f"Delete SQS Messages failed {response} for messages {messages}" - if self.soft_fail: - raise AirflowSkipException(error_message) - raise AirflowException(error_message) + raise AirflowException(f"Delete SQS Messages failed {response} for messages {messages}") if message_batch: context["ti"].xcom_push(key="messages", value=message_batch) return True diff --git a/airflow/providers/amazon/aws/sensors/step_function.py b/airflow/providers/amazon/aws/sensors/step_function.py index 5e0d3cfcf79c..8af3bb6fe9c6 100644 --- a/airflow/providers/amazon/aws/sensors/step_function.py +++ b/airflow/providers/amazon/aws/sensors/step_function.py @@ -21,7 +21,7 @@ from deprecated import deprecated -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.amazon.aws.hooks.step_function import StepFunctionHook from airflow.providers.amazon.aws.sensors.base_aws import AwsBaseSensor from airflow.providers.amazon.aws.utils.mixins import aws_template_fields @@ -76,11 +76,7 @@ def poke(self, context: Context): output = json.loads(execution_status["output"]) if "output" in execution_status else None if state in self.FAILURE_STATES: - # TODO: remove this if block when min_airflow_version is set to higher than 2.7.1 - message = f"Step Function sensor failed. State Machine Output: {output}" - if self.soft_fail: - raise AirflowSkipException(message) - raise AirflowException(message) + raise AirflowException(f"Step Function sensor failed. State Machine Output: {output}") if state in self.INTERMEDIATE_STATES: return False diff --git a/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py b/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py index 81f0dd79b67b..f7670cfd8371 100644 --- a/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py +++ b/airflow/providers/amazon/aws/transfers/dynamodb_to_s3.py @@ -32,6 +32,7 @@ from airflow.providers.amazon.aws.hooks.dynamodb import DynamoDBHook from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.amazon.aws.transfers.base import AwsToAwsBaseOperator +from airflow.utils.helpers import prune_dict if TYPE_CHECKING: from airflow.utils.context import Context @@ -89,10 +90,13 @@ class DynamoDBToS3Operator(AwsToAwsBaseOperator): :param s3_key_prefix: Prefix of s3 object key :param process_func: How we transform a dynamodb item to bytes. By default, we dump the json + :param point_in_time_export: Boolean value indicating the operator to use 'scan' or 'point in time export' :param export_time: Time in the past from which to export table data, counted in seconds from the start of the Unix epoch. The table export will be a snapshot of the table's state at this point in time. :param export_format: The format for the exported data. Valid values for ExportFormat are DYNAMODB_JSON or ION. + :param export_table_to_point_in_time_kwargs: extra parameters for the boto3 + `export_table_to_point_in_time` function all. e.g. `ExportType`, `IncrementalExportSpecification` :param check_interval: The amount of time in seconds to wait between attempts. Only if ``export_time`` is provided. :param max_attempts: The maximum number of attempts to be made. Only if ``export_time`` is provided. @@ -107,12 +111,14 @@ class DynamoDBToS3Operator(AwsToAwsBaseOperator): "s3_key_prefix", "export_time", "export_format", + "export_table_to_point_in_time_kwargs", "check_interval", "max_attempts", ) template_fields_renderers = { "dynamodb_scan_kwargs": "json", + "export_table_to_point_in_time_kwargs": "json", } def __init__( @@ -120,12 +126,14 @@ def __init__( *, dynamodb_table_name: str, s3_bucket_name: str, - file_size: int, + file_size: int = 1000, dynamodb_scan_kwargs: dict[str, Any] | None = None, s3_key_prefix: str = "", process_func: Callable[[dict[str, Any]], bytes] = _convert_item_to_json_bytes, + point_in_time_export: bool = False, export_time: datetime | None = None, export_format: str = "DYNAMODB_JSON", + export_table_to_point_in_time_kwargs: dict | None = None, check_interval: int = 30, max_attempts: int = 60, **kwargs, @@ -137,8 +145,10 @@ def __init__( self.dynamodb_scan_kwargs = dynamodb_scan_kwargs self.s3_bucket_name = s3_bucket_name self.s3_key_prefix = s3_key_prefix + self.point_in_time_export = point_in_time_export self.export_time = export_time self.export_format = export_format + self.export_table_to_point_in_time_kwargs = export_table_to_point_in_time_kwargs or {} self.check_interval = check_interval self.max_attempts = max_attempts @@ -148,29 +158,50 @@ def hook(self): return DynamoDBHook(aws_conn_id=self.source_aws_conn_id) def execute(self, context: Context) -> None: - if self.export_time: + # There are 2 separate export to point in time configuration: + # 1. Full export, which takes the export_time arg. + # 2. Incremental export, which takes the incremental_export_... args + # Hence export time could not be used as the proper indicator for the `_export_table_to_point_in_time` + # function. This change introduces a new boolean, as the indicator for whether the operator scans + # and export entire data or using the point in time functionality. + if self.point_in_time_export or self.export_time: self._export_table_to_point_in_time() else: self._export_entire_data() def _export_table_to_point_in_time(self): """ - Export data from start of epoc till `export_time`. + Export data to point in time. + Full export exports data from start of epoc till `export_time`. Table export will be a snapshot of the table's state at this point in time. + + Incremental export exports the data from a specific datetime to a specific datetime + + + Note: S3BucketOwner is a required parameter when exporting to a S3 bucket in another account. """ if self.export_time and self.export_time > datetime.now(self.export_time.tzinfo): raise ValueError("The export_time parameter cannot be a future time.") client = self.hook.conn.meta.client table_description = client.describe_table(TableName=self.dynamodb_table_name) - response = client.export_table_to_point_in_time( - TableArn=table_description.get("Table", {}).get("TableArn"), - ExportTime=self.export_time, - S3Bucket=self.s3_bucket_name, - S3Prefix=self.s3_key_prefix, - ExportFormat=self.export_format, - ) + + export_table_to_point_in_time_base_args = { + "TableArn": table_description.get("Table", {}).get("TableArn"), + "ExportTime": self.export_time, + "S3Bucket": self.s3_bucket_name, + "S3Prefix": self.s3_key_prefix, + "ExportFormat": self.export_format, + } + export_table_to_point_in_time_args = { + **export_table_to_point_in_time_base_args, + **self.export_table_to_point_in_time_kwargs, + } + + args_filtered = prune_dict(export_table_to_point_in_time_args) + + response = client.export_table_to_point_in_time(**args_filtered) waiter = self.hook.get_waiter("export_table") export_arn = response.get("ExportDescription", {}).get("ExportArn") waiter.wait( diff --git a/airflow/providers/amazon/aws/transfers/s3_to_redshift.py b/airflow/providers/amazon/aws/transfers/s3_to_redshift.py index 161276b33cb0..653885b54111 100644 --- a/airflow/providers/amazon/aws/transfers/s3_to_redshift.py +++ b/airflow/providers/amazon/aws/transfers/s3_to_redshift.py @@ -121,6 +121,10 @@ def __init__( if arg in self.redshift_data_api_kwargs: raise AirflowException(f"Cannot include param '{arg}' in Redshift Data API kwargs") + @property + def use_redshift_data(self): + return bool(self.redshift_data_api_kwargs) + def _build_copy_query( self, copy_destination: str, credentials_block: str, region_info: str, copy_options: str ) -> str: @@ -138,11 +142,11 @@ def execute(self, context: Context) -> None: if self.method not in AVAILABLE_METHODS: raise AirflowException(f"Method not found! Available methods: {AVAILABLE_METHODS}") - redshift_hook: RedshiftDataHook | RedshiftSQLHook - if self.redshift_data_api_kwargs: - redshift_hook = RedshiftDataHook(aws_conn_id=self.redshift_conn_id) + if self.use_redshift_data: + redshift_data_hook = RedshiftDataHook(aws_conn_id=self.redshift_conn_id) else: - redshift_hook = RedshiftSQLHook(redshift_conn_id=self.redshift_conn_id) + redshift_sql_hook = RedshiftSQLHook(redshift_conn_id=self.redshift_conn_id) + conn = S3Hook.get_connection(conn_id=self.aws_conn_id) if self.aws_conn_id else None region_info = "" if conn and conn.extra_dejson.get("region", False): @@ -167,12 +171,12 @@ def execute(self, context: Context) -> None: if self.method == "REPLACE": sql = ["BEGIN;", f"DELETE FROM {destination};", copy_statement, "COMMIT"] elif self.method == "UPSERT": - if isinstance(redshift_hook, RedshiftDataHook): - keys = self.upsert_keys or redshift_hook.get_table_primary_key( + if self.use_redshift_data: + keys = self.upsert_keys or redshift_data_hook.get_table_primary_key( table=self.table, schema=self.schema, **self.redshift_data_api_kwargs ) else: - keys = self.upsert_keys or redshift_hook.get_table_primary_key(self.table, self.schema) + keys = self.upsert_keys or redshift_sql_hook.get_table_primary_key(self.table, self.schema) if not keys: raise AirflowException( f"No primary key on {self.schema}.{self.table}. Please provide keys on 'upsert_keys'" @@ -192,8 +196,85 @@ def execute(self, context: Context) -> None: sql = copy_statement self.log.info("Executing COPY command...") - if isinstance(redshift_hook, RedshiftDataHook): - redshift_hook.execute_query(sql=sql, **self.redshift_data_api_kwargs) + if self.use_redshift_data: + redshift_data_hook.execute_query(sql=sql, **self.redshift_data_api_kwargs) else: - redshift_hook.run(sql, autocommit=self.autocommit) + redshift_sql_hook.run(sql, autocommit=self.autocommit) self.log.info("COPY command complete...") + + def get_openlineage_facets_on_complete(self, task_instance): + """Implement on_complete as we will query destination table.""" + from pathlib import Path + + from airflow.providers.amazon.aws.utils.openlineage import ( + get_facets_from_redshift_table, + get_identity_column_lineage_facet, + ) + from airflow.providers.common.compat.openlineage.facet import ( + Dataset, + Identifier, + LifecycleStateChange, + LifecycleStateChangeDatasetFacet, + SymlinksDatasetFacet, + ) + from airflow.providers.openlineage.extractors import OperatorLineage + + if self.use_redshift_data: + redshift_data_hook = RedshiftDataHook(aws_conn_id=self.redshift_conn_id) + database = self.redshift_data_api_kwargs.get("database") + identifier = self.redshift_data_api_kwargs.get( + "cluster_identifier", self.redshift_data_api_kwargs.get("workgroup_name") + ) + port = self.redshift_data_api_kwargs.get("port", "5439") + authority = f"{identifier}.{redshift_data_hook.region_name}:{port}" + output_dataset_facets = get_facets_from_redshift_table( + redshift_data_hook, self.table, self.redshift_data_api_kwargs, self.schema + ) + else: + redshift_sql_hook = RedshiftSQLHook(redshift_conn_id=self.redshift_conn_id) + database = redshift_sql_hook.conn.schema + authority = redshift_sql_hook.get_openlineage_database_info(redshift_sql_hook.conn).authority + output_dataset_facets = get_facets_from_redshift_table( + redshift_sql_hook, self.table, self.redshift_data_api_kwargs, self.schema + ) + + input_dataset_facets = {} + if not self.column_list: + # If column_list is not specified, then we know that input file matches columns of output table. + input_dataset_facets["schema"] = output_dataset_facets["schema"] + + dataset_name = self.s3_key + if "*" in dataset_name: + # If wildcard ("*") is used in s3 path, we want the name of dataset to be directory name, + # but we create a symlink to the full object path with wildcard. + input_dataset_facets["symlink"] = SymlinksDatasetFacet( + identifiers=[Identifier(namespace=f"s3://{self.s3_bucket}", name=dataset_name, type="file")] + ) + dataset_name = Path(dataset_name).parent.as_posix() + if dataset_name == ".": + # blob path does not have leading slash, but we need root dataset name to be "/" + dataset_name = "/" + + input_dataset = Dataset( + namespace=f"s3://{self.s3_bucket}", + name=dataset_name, + facets=input_dataset_facets, + ) + + output_dataset_facets["columnLineage"] = get_identity_column_lineage_facet( + field_names=[field.name for field in output_dataset_facets["schema"].fields], + input_datasets=[input_dataset], + ) + + if self.method == "REPLACE": + output_dataset_facets["lifecycleStateChange"] = LifecycleStateChangeDatasetFacet( + lifecycleStateChange=LifecycleStateChange.OVERWRITE + ) + + output_dataset = Dataset( + namespace=f"redshift://{authority}", + name=f"{database}.{self.schema}.{self.table}", + facets=output_dataset_facets, + ) + + return OperatorLineage(inputs=[input_dataset], outputs=[output_dataset]) diff --git a/airflow/providers/amazon/aws/utils/openlineage.py b/airflow/providers/amazon/aws/utils/openlineage.py new file mode 100644 index 000000000000..db472a3e46c5 --- /dev/null +++ b/airflow/providers/amazon/aws/utils/openlineage.py @@ -0,0 +1,136 @@ +# +# 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 typing import TYPE_CHECKING, Any + +from airflow.providers.amazon.aws.hooks.redshift_sql import RedshiftSQLHook +from airflow.providers.common.compat.openlineage.facet import ( + ColumnLineageDatasetFacet, + DocumentationDatasetFacet, + Fields, + InputField, + SchemaDatasetFacet, + SchemaDatasetFacetFields, +) + +if TYPE_CHECKING: + from airflow.providers.amazon.aws.hooks.redshift_data import RedshiftDataHook + + +def get_facets_from_redshift_table( + redshift_hook: RedshiftDataHook | RedshiftSQLHook, + table: str, + redshift_data_api_kwargs: dict, + schema: str = "public", +) -> dict[Any, Any]: + """ + Query redshift for table metadata. + + SchemaDatasetFacet and DocumentationDatasetFacet (if table has description) will be created. + """ + sql = f""" + SELECT + cols.column_name, + cols.data_type, + col_des.description as column_description, + tbl_des.description as table_description + FROM + information_schema.columns cols + LEFT JOIN + pg_catalog.pg_description col_des + ON + cols.ordinal_position = col_des.objsubid + AND col_des.objoid = (SELECT oid FROM pg_class WHERE relnamespace = + (SELECT oid FROM pg_namespace WHERE nspname = cols.table_schema) AND relname = cols.table_name) + LEFT JOIN + pg_catalog.pg_class tbl + ON + tbl.relname = cols.table_name + AND tbl.relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = cols.table_schema) + LEFT JOIN + pg_catalog.pg_description tbl_des + ON + tbl.oid = tbl_des.objoid + AND tbl_des.objsubid = 0 + WHERE + cols.table_name = '{table}' + AND cols.table_schema = '{schema}'; + """ + if isinstance(redshift_hook, RedshiftSQLHook): + records = redshift_hook.get_records(sql) + if records: + table_description = records[0][-1] # Assuming the table description is the same for all rows + else: + table_description = None + documentation = DocumentationDatasetFacet(description=table_description or "") + table_schema = SchemaDatasetFacet( + fields=[ + SchemaDatasetFacetFields(name=field[0], type=field[1], description=field[2]) + for field in records + ] + ) + else: + statement_id = redshift_hook.execute_query(sql=sql, poll_interval=1, **redshift_data_api_kwargs) + response = redshift_hook.conn.get_statement_result(Id=statement_id) + + table_schema = SchemaDatasetFacet( + fields=[ + SchemaDatasetFacetFields( + name=field[0]["stringValue"], + type=field[1]["stringValue"], + description=field[2].get("stringValue"), + ) + for field in response["Records"] + ] + ) + # Table description will be the same for all fields, so we retrieve it from first field. + documentation = DocumentationDatasetFacet( + description=response["Records"][0][3].get("stringValue") or "" + ) + + return {"schema": table_schema, "documentation": documentation} + + +def get_identity_column_lineage_facet( + field_names, + input_datasets, +) -> ColumnLineageDatasetFacet: + """ + Get column lineage facet. + + Simple lineage will be created, where each source column corresponds to single destination column + in each input dataset and there are no transformations made. + """ + if field_names and not input_datasets: + raise ValueError("When providing `field_names` You must provide at least one `input_dataset`.") + + column_lineage_facet = ColumnLineageDatasetFacet( + fields={ + field: Fields( + inputFields=[ + InputField(namespace=dataset.namespace, name=dataset.name, field=field) + for dataset in input_datasets + ], + transformationType="IDENTITY", + transformationDescription="identical", + ) + for field in field_names + } + ) + return column_lineage_facet diff --git a/airflow/providers/amazon/aws/utils/task_log_fetcher.py b/airflow/providers/amazon/aws/utils/task_log_fetcher.py index a4cad6c099c2..83c42f685792 100644 --- a/airflow/providers/amazon/aws/utils/task_log_fetcher.py +++ b/airflow/providers/amazon/aws/utils/task_log_fetcher.py @@ -59,8 +59,20 @@ def run(self) -> None: while not self.is_stopped(): time.sleep(self.fetch_interval.total_seconds()) log_events = self._get_log_events(continuation_token) + prev_timestamp_event = None for log_event in log_events: + current_timestamp_event = datetime.fromtimestamp( + log_event["timestamp"] / 1000.0, tz=timezone.utc + ) + if current_timestamp_event == prev_timestamp_event: + # When multiple events have the same timestamp, somehow, only one event is logged + # As a consequence, some logs are missed in the log group (in case they have the same + # timestamp) + # When a slight delay is added before logging the event, that solves the issue + # See https://github.com/apache/airflow/issues/40875 + time.sleep(0.1) self.logger.info(self.event_to_str(log_event)) + prev_timestamp_event = current_timestamp_event def _get_log_events(self, skip_token: AwsLogsHook.ContinuationToken | None = None) -> Generator: if skip_token is None: diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index ffd6b47a43ee..b993a7c4f95b 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -22,9 +22,10 @@ description: | Amazon integration (including `Amazon Web Services (AWS) `__). state: ready -source-date-epoch: 1722145222 +source-date-epoch: 1723968989 # note that those versions are maintained by release manager - do not update them manually versions: + - 8.28.0 - 8.27.0 - 8.26.0 - 8.25.0 @@ -139,9 +140,7 @@ additional-extras: devel-dependencies: - aiobotocore>=2.13.0 - aws_xray_sdk>=2.12.0 - # The latest 5.0.12 moto breaks some of our tests https://github.com/apache/airflow/issues/41243 - # Until we fix it, we limit moto to <5.0.12 - - moto[cloudformation,glue]>=5.0.0,<5.0.12 + - moto[cloudformation,glue]>=5.0.0 - mypy-boto3-appflow>=1.34.0 - mypy-boto3-rds>=1.34.90 - mypy-boto3-redshift-data>=1.34.0 diff --git a/airflow/providers/apache/beam/CHANGELOG.rst b/airflow/providers/apache/beam/CHANGELOG.rst index b1abfdc77348..85f0ac8f7268 100644 --- a/airflow/providers/apache/beam/CHANGELOG.rst +++ b/airflow/providers/apache/beam/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +5.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.7.2 ..... diff --git a/airflow/providers/apache/beam/__init__.py b/airflow/providers/apache/beam/__init__.py index 3de40bbc54fc..f88dd1a9909c 100644 --- a/airflow/providers/apache/beam/__init__.py +++ b/airflow/providers/apache/beam/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.7.2" +__version__ = "5.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/beam/provider.yaml b/airflow/providers/apache/beam/provider.yaml index 1be48e908407..8011f564d719 100644 --- a/airflow/providers/apache/beam/provider.yaml +++ b/airflow/providers/apache/beam/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Beam `__. state: ready -source-date-epoch: 1721545598 +source-date-epoch: 1723969156 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.8.0 - 5.7.2 - 5.7.1 - 5.7.0 diff --git a/airflow/providers/apache/cassandra/CHANGELOG.rst b/airflow/providers/apache/cassandra/CHANGELOG.rst index efa5bbe7e4ad..1bd8eebf0559 100644 --- a/airflow/providers/apache/cassandra/CHANGELOG.rst +++ b/airflow/providers/apache/cassandra/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/apache/cassandra/__init__.py b/airflow/providers/apache/cassandra/__init__.py index 1facc5167d5a..1e3027d34b3b 100644 --- a/airflow/providers/apache/cassandra/__init__.py +++ b/airflow/providers/apache/cassandra/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/cassandra/provider.yaml b/airflow/providers/apache/cassandra/provider.yaml index c31ff6eceff1..7f2ea49fbbfd 100644 --- a/airflow/providers/apache/cassandra/provider.yaml +++ b/airflow/providers/apache/cassandra/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Cassandra `__. state: ready -source-date-epoch: 1716286425 +source-date-epoch: 1723969167 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.2 diff --git a/airflow/providers/apache/drill/CHANGELOG.rst b/airflow/providers/apache/drill/CHANGELOG.rst index 6c40bf111884..252d57c9ea33 100644 --- a/airflow/providers/apache/drill/CHANGELOG.rst +++ b/airflow/providers/apache/drill/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +2.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.7.3 ..... diff --git a/airflow/providers/apache/drill/__init__.py b/airflow/providers/apache/drill/__init__.py index fae683d663e9..f831798a8ff4 100644 --- a/airflow/providers/apache/drill/__init__.py +++ b/airflow/providers/apache/drill/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.7.3" +__version__ = "2.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/drill/provider.yaml b/airflow/providers/apache/drill/provider.yaml index 3207f2059b35..8512112bf092 100644 --- a/airflow/providers/apache/drill/provider.yaml +++ b/airflow/providers/apache/drill/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Drill `__. state: ready -source-date-epoch: 1722662634 +source-date-epoch: 1723969173 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.8.0 - 2.7.3 - 2.7.2 - 2.7.1 diff --git a/airflow/providers/apache/druid/CHANGELOG.rst b/airflow/providers/apache/druid/CHANGELOG.rst index c84ece5a443a..9a56e15acbea 100644 --- a/airflow/providers/apache/druid/CHANGELOG.rst +++ b/airflow/providers/apache/druid/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.11.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.10.2 ...... diff --git a/airflow/providers/apache/druid/__init__.py b/airflow/providers/apache/druid/__init__.py index 57b55c39d57b..5a80ed9dd36a 100644 --- a/airflow/providers/apache/druid/__init__.py +++ b/airflow/providers/apache/druid/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.10.2" +__version__ = "3.11.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/druid/provider.yaml b/airflow/providers/apache/druid/provider.yaml index 7b48fee1dfc5..773cddddd4fe 100644 --- a/airflow/providers/apache/druid/provider.yaml +++ b/airflow/providers/apache/druid/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Druid `__. state: ready -source-date-epoch: 1722662771 +source-date-epoch: 1723969177 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.11.0 - 3.10.2 - 3.10.1 - 3.10.0 diff --git a/airflow/providers/apache/flink/CHANGELOG.rst b/airflow/providers/apache/flink/CHANGELOG.rst index 420ca8ad5c1a..2c0c8084f3d9 100644 --- a/airflow/providers/apache/flink/CHANGELOG.rst +++ b/airflow/providers/apache/flink/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +1.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.4.2 ..... diff --git a/airflow/providers/apache/flink/__init__.py b/airflow/providers/apache/flink/__init__.py index 0126b1aef076..1fecaccfc14b 100644 --- a/airflow/providers/apache/flink/__init__.py +++ b/airflow/providers/apache/flink/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.4.2" +__version__ = "1.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/flink/provider.yaml b/airflow/providers/apache/flink/provider.yaml index cdcd63e95a32..81f0e6414dc2 100644 --- a/airflow/providers/apache/flink/provider.yaml +++ b/airflow/providers/apache/flink/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Flink `__ state: ready -source-date-epoch: 1718603786 +source-date-epoch: 1723969183 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.5.0 - 1.4.2 - 1.4.1 - 1.4.0 diff --git a/airflow/providers/apache/hdfs/CHANGELOG.rst b/airflow/providers/apache/hdfs/CHANGELOG.rst index b029bccfaaf5..072ae9f230cf 100644 --- a/airflow/providers/apache/hdfs/CHANGELOG.rst +++ b/airflow/providers/apache/hdfs/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +4.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 4.4.2 ..... diff --git a/airflow/providers/apache/hdfs/__init__.py b/airflow/providers/apache/hdfs/__init__.py index 0e0772fe65ab..7f8458092820 100644 --- a/airflow/providers/apache/hdfs/__init__.py +++ b/airflow/providers/apache/hdfs/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.4.2" +__version__ = "4.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/hdfs/provider.yaml b/airflow/providers/apache/hdfs/provider.yaml index 8578d44e7f0f..8cb0f0451521 100644 --- a/airflow/providers/apache/hdfs/provider.yaml +++ b/airflow/providers/apache/hdfs/provider.yaml @@ -23,9 +23,10 @@ description: | and `WebHDFS `__. state: ready -source-date-epoch: 1718603811 +source-date-epoch: 1723969196 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.5.0 - 4.4.2 - 4.4.1 - 4.4.0 diff --git a/airflow/providers/apache/hive/CHANGELOG.rst b/airflow/providers/apache/hive/CHANGELOG.rst index bfcce4db2fb1..50e6cecc9b53 100644 --- a/airflow/providers/apache/hive/CHANGELOG.rst +++ b/airflow/providers/apache/hive/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +8.2.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 8.1.2 ..... diff --git a/airflow/providers/apache/hive/__init__.py b/airflow/providers/apache/hive/__init__.py index 10c658b78188..68cfa84d0f8c 100644 --- a/airflow/providers/apache/hive/__init__.py +++ b/airflow/providers/apache/hive/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "8.1.2" +__version__ = "8.2.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/hive/provider.yaml b/airflow/providers/apache/hive/provider.yaml index fd0c37866bd7..1ee8e62f4caf 100644 --- a/airflow/providers/apache/hive/provider.yaml +++ b/airflow/providers/apache/hive/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Hive `__ state: ready -source-date-epoch: 1718603836 +source-date-epoch: 1723969208 # note that those versions are maintained by release manager - do not update them manually versions: + - 8.2.0 - 8.1.2 - 8.1.1 - 8.1.0 diff --git a/airflow/providers/apache/iceberg/CHANGELOG.rst b/airflow/providers/apache/iceberg/CHANGELOG.rst index 797bb9161074..9149ff43d739 100644 --- a/airflow/providers/apache/iceberg/CHANGELOG.rst +++ b/airflow/providers/apache/iceberg/CHANGELOG.rst @@ -26,6 +26,26 @@ Changelog --------- +1.1.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Prepare docs 2nd wave June 2024 (#40273)`` + * ``Update Iceberg provider name in provider.yaml (#39846)`` + 1.0.0 ..... diff --git a/airflow/providers/apache/iceberg/__init__.py b/airflow/providers/apache/iceberg/__init__.py index f0ccedf8ac48..e71ea486fc91 100644 --- a/airflow/providers/apache/iceberg/__init__.py +++ b/airflow/providers/apache/iceberg/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.0.0" +__version__ = "1.1.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/iceberg/provider.yaml b/airflow/providers/apache/iceberg/provider.yaml index 8ccd8b7b771f..3dd930b0eaba 100644 --- a/airflow/providers/apache/iceberg/provider.yaml +++ b/airflow/providers/apache/iceberg/provider.yaml @@ -22,9 +22,10 @@ description: | `Iceberg `__ state: ready -source-date-epoch: 1715384402 +source-date-epoch: 1723969223 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.1.0 - 1.0.0 dependencies: diff --git a/airflow/providers/apache/impala/CHANGELOG.rst b/airflow/providers/apache/impala/CHANGELOG.rst index ac7633067c7e..bb49a8a37aaa 100644 --- a/airflow/providers/apache/impala/CHANGELOG.rst +++ b/airflow/providers/apache/impala/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +1.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.4.2 ..... diff --git a/airflow/providers/apache/impala/__init__.py b/airflow/providers/apache/impala/__init__.py index 550f506346da..4bd11212b3ac 100644 --- a/airflow/providers/apache/impala/__init__.py +++ b/airflow/providers/apache/impala/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.4.2" +__version__ = "1.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/impala/provider.yaml b/airflow/providers/apache/impala/provider.yaml index c62726871249..e2cdfd13329a 100644 --- a/airflow/providers/apache/impala/provider.yaml +++ b/airflow/providers/apache/impala/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Impala `__. state: ready -source-date-epoch: 1722662858 +source-date-epoch: 1723969232 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.5.0 - 1.4.2 - 1.4.1 - 1.4.0 diff --git a/airflow/providers/apache/kafka/CHANGELOG.rst b/airflow/providers/apache/kafka/CHANGELOG.rst index e7bd081e92be..045475e755b5 100644 --- a/airflow/providers/apache/kafka/CHANGELOG.rst +++ b/airflow/providers/apache/kafka/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +1.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 1.5.0 ..... diff --git a/airflow/providers/apache/kafka/__init__.py b/airflow/providers/apache/kafka/__init__.py index 45c1d97aa687..862c4c19b49a 100644 --- a/airflow/providers/apache/kafka/__init__.py +++ b/airflow/providers/apache/kafka/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.5.0" +__version__ = "1.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/kafka/provider.yaml b/airflow/providers/apache/kafka/provider.yaml index fcbd77efcbe0..8a577ba4cdad 100644 --- a/airflow/providers/apache/kafka/provider.yaml +++ b/airflow/providers/apache/kafka/provider.yaml @@ -20,11 +20,12 @@ package-name: apache-airflow-providers-apache-kafka name: Apache Kafka state: ready -source-date-epoch: 1718603924 +source-date-epoch: 1723969246 description: | `Apache Kafka `__ # note that those versions are maintained by release manager - do not update them manually versions: + - 1.6.0 - 1.5.0 - 1.4.1 - 1.4.0 diff --git a/airflow/providers/apache/kylin/CHANGELOG.rst b/airflow/providers/apache/kylin/CHANGELOG.rst index 5233ffaf617c..32179c629f66 100644 --- a/airflow/providers/apache/kylin/CHANGELOG.rst +++ b/airflow/providers/apache/kylin/CHANGELOG.rst @@ -29,6 +29,24 @@ Changelog --------- +3.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` 3.6.2 ..... diff --git a/airflow/providers/apache/kylin/__init__.py b/airflow/providers/apache/kylin/__init__.py index e88f1c741960..fa7850f8d160 100644 --- a/airflow/providers/apache/kylin/__init__.py +++ b/airflow/providers/apache/kylin/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.6.2" +__version__ = "3.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/kylin/provider.yaml b/airflow/providers/apache/kylin/provider.yaml index 2fc34b44a625..610934f9ca84 100644 --- a/airflow/providers/apache/kylin/provider.yaml +++ b/airflow/providers/apache/kylin/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Kylin `__ state: ready -source-date-epoch: 1718603957 +source-date-epoch: 1723969259 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.7.0 - 3.6.2 - 3.6.1 - 3.6.0 diff --git a/airflow/providers/apache/livy/CHANGELOG.rst b/airflow/providers/apache/livy/CHANGELOG.rst index 28aaeab5d9ba..1e9746433dca 100644 --- a/airflow/providers/apache/livy/CHANGELOG.rst +++ b/airflow/providers/apache/livy/CHANGELOG.rst @@ -28,6 +28,25 @@ Changelog --------- +3.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.8.1 ..... diff --git a/airflow/providers/apache/livy/__init__.py b/airflow/providers/apache/livy/__init__.py index 8210bbd4861c..a36dd8ba5b2c 100644 --- a/airflow/providers/apache/livy/__init__.py +++ b/airflow/providers/apache/livy/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.8.1" +__version__ = "3.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/livy/provider.yaml b/airflow/providers/apache/livy/provider.yaml index 7eb5e22e39a4..8933ff4074a8 100644 --- a/airflow/providers/apache/livy/provider.yaml +++ b/airflow/providers/apache/livy/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Livy `__ state: ready -source-date-epoch: 1716286713 +source-date-epoch: 1723969297 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.9.0 - 3.8.1 - 3.8.0 - 3.7.3 diff --git a/airflow/providers/apache/pig/CHANGELOG.rst b/airflow/providers/apache/pig/CHANGELOG.rst index fa3d2168cc53..d4e8dfad2563 100644 --- a/airflow/providers/apache/pig/CHANGELOG.rst +++ b/airflow/providers/apache/pig/CHANGELOG.rst @@ -29,6 +29,25 @@ Changelog --------- +4.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 4.4.1 ..... diff --git a/airflow/providers/apache/pig/__init__.py b/airflow/providers/apache/pig/__init__.py index 1977687b6975..544cea765761 100644 --- a/airflow/providers/apache/pig/__init__.py +++ b/airflow/providers/apache/pig/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.4.1" +__version__ = "4.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/pig/provider.yaml b/airflow/providers/apache/pig/provider.yaml index 8fcd0813383a..3ce8397d2d44 100644 --- a/airflow/providers/apache/pig/provider.yaml +++ b/airflow/providers/apache/pig/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Pig `__ state: ready -source-date-epoch: 1716286733 +source-date-epoch: 1723969310 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.5.0 - 4.4.1 - 4.4.0 - 4.3.0 diff --git a/airflow/providers/apache/pinot/CHANGELOG.rst b/airflow/providers/apache/pinot/CHANGELOG.rst index d7ac3705f8f1..9e7151b61941 100644 --- a/airflow/providers/apache/pinot/CHANGELOG.rst +++ b/airflow/providers/apache/pinot/CHANGELOG.rst @@ -29,6 +29,22 @@ Changelog --------- +4.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.4.2 ..... diff --git a/airflow/providers/apache/pinot/__init__.py b/airflow/providers/apache/pinot/__init__.py index 2619867a7f75..0671894e6377 100644 --- a/airflow/providers/apache/pinot/__init__.py +++ b/airflow/providers/apache/pinot/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.4.2" +__version__ = "4.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/pinot/provider.yaml b/airflow/providers/apache/pinot/provider.yaml index f47d5ac430b4..befaf091e276 100644 --- a/airflow/providers/apache/pinot/provider.yaml +++ b/airflow/providers/apache/pinot/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Pinot `__ state: ready -source-date-epoch: 1722662903 +source-date-epoch: 1723969316 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.5.0 - 4.4.2 - 4.4.1 - 4.4.0 diff --git a/airflow/providers/apache/spark/CHANGELOG.rst b/airflow/providers/apache/spark/CHANGELOG.rst index e6e1d2650660..3e8ff93174bb 100644 --- a/airflow/providers/apache/spark/CHANGELOG.rst +++ b/airflow/providers/apache/spark/CHANGELOG.rst @@ -29,6 +29,19 @@ Changelog --------- +4.10.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``Resolve 'AirflowProviderDeprecationWarning' in 'SparkSqlOperator' (#41358)`` + 4.9.0 ..... diff --git a/airflow/providers/apache/spark/__init__.py b/airflow/providers/apache/spark/__init__.py index 9465c24d52ba..2c75022d95dd 100644 --- a/airflow/providers/apache/spark/__init__.py +++ b/airflow/providers/apache/spark/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.9.0" +__version__ = "4.10.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apache/spark/provider.yaml b/airflow/providers/apache/spark/provider.yaml index 9834e4dca291..89e89db47823 100644 --- a/airflow/providers/apache/spark/provider.yaml +++ b/airflow/providers/apache/spark/provider.yaml @@ -22,9 +22,10 @@ description: | `Apache Spark `__ state: ready -source-date-epoch: 1721545745 +source-date-epoch: 1723969336 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.10.0 - 4.9.0 - 4.8.2 - 4.8.1 diff --git a/airflow/providers/apprise/CHANGELOG.rst b/airflow/providers/apprise/CHANGELOG.rst index 1dee1be737cb..aa148ff1290e 100644 --- a/airflow/providers/apprise/CHANGELOG.rst +++ b/airflow/providers/apprise/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +1.4.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.3.2 ..... diff --git a/airflow/providers/apprise/__init__.py b/airflow/providers/apprise/__init__.py index 746095d5e7fe..d0a3b4257983 100644 --- a/airflow/providers/apprise/__init__.py +++ b/airflow/providers/apprise/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.3.2" +__version__ = "1.4.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/apprise/provider.yaml b/airflow/providers/apprise/provider.yaml index edd55f489d48..a05020b3807b 100644 --- a/airflow/providers/apprise/provider.yaml +++ b/airflow/providers/apprise/provider.yaml @@ -24,10 +24,11 @@ description: | `Apprise `__ state: ready -source-date-epoch: 1722662912 +source-date-epoch: 1723969344 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.4.0 - 1.3.2 - 1.3.1 - 1.3.0 diff --git a/airflow/providers/arangodb/CHANGELOG.rst b/airflow/providers/arangodb/CHANGELOG.rst index b0f748b10540..d498ceed876d 100644 --- a/airflow/providers/arangodb/CHANGELOG.rst +++ b/airflow/providers/arangodb/CHANGELOG.rst @@ -28,6 +28,22 @@ Changelog --------- +2.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.5.1 ..... diff --git a/airflow/providers/arangodb/__init__.py b/airflow/providers/arangodb/__init__.py index e6ac1087c9c0..cc7e24472614 100644 --- a/airflow/providers/arangodb/__init__.py +++ b/airflow/providers/arangodb/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.5.1" +__version__ = "2.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/arangodb/provider.yaml b/airflow/providers/arangodb/provider.yaml index 17f18f84f84a..e5a480bc6a89 100644 --- a/airflow/providers/arangodb/provider.yaml +++ b/airflow/providers/arangodb/provider.yaml @@ -26,9 +26,10 @@ dependencies: - python-arango>=7.3.2 state: ready -source-date-epoch: 1716286828 +source-date-epoch: 1723969354 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.6.0 - 2.5.1 - 2.5.0 - 2.4.1 diff --git a/airflow/providers/asana/CHANGELOG.rst b/airflow/providers/asana/CHANGELOG.rst index e49cd690be3a..0e830d9ff003 100644 --- a/airflow/providers/asana/CHANGELOG.rst +++ b/airflow/providers/asana/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +2.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.5.1 ..... diff --git a/airflow/providers/asana/__init__.py b/airflow/providers/asana/__init__.py index 87392fa4b5ad..f8b8eda494fa 100644 --- a/airflow/providers/asana/__init__.py +++ b/airflow/providers/asana/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.5.1" +__version__ = "2.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/asana/provider.yaml b/airflow/providers/asana/provider.yaml index a0d57a2fe5be..40f7131462cc 100644 --- a/airflow/providers/asana/provider.yaml +++ b/airflow/providers/asana/provider.yaml @@ -22,9 +22,10 @@ description: | `Asana `__ state: ready -source-date-epoch: 1716286843 +source-date-epoch: 1723969361 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.6.0 - 2.5.1 - 2.5.0 - 2.4.1 diff --git a/airflow/providers/atlassian/jira/CHANGELOG.rst b/airflow/providers/atlassian/jira/CHANGELOG.rst index f965d5b8d902..4dd96107c15c 100644 --- a/airflow/providers/atlassian/jira/CHANGELOG.rst +++ b/airflow/providers/atlassian/jira/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +2.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 2.6.1 ..... diff --git a/airflow/providers/atlassian/jira/__init__.py b/airflow/providers/atlassian/jira/__init__.py index f7d02a85f308..750a0383b84a 100644 --- a/airflow/providers/atlassian/jira/__init__.py +++ b/airflow/providers/atlassian/jira/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.6.1" +__version__ = "2.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/atlassian/jira/provider.yaml b/airflow/providers/atlassian/jira/provider.yaml index a4e63e008a8e..aba080660f80 100644 --- a/airflow/providers/atlassian/jira/provider.yaml +++ b/airflow/providers/atlassian/jira/provider.yaml @@ -22,9 +22,10 @@ description: | `Atlassian Jira `__ state: ready -source-date-epoch: 1716286859 +source-date-epoch: 1723969369 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.7.0 - 2.6.1 - 2.6.0 - 2.5.1 diff --git a/airflow/providers/celery/CHANGELOG.rst b/airflow/providers/celery/CHANGELOG.rst index 978b5d39fddc..48343927c140 100644 --- a/airflow/providers/celery/CHANGELOG.rst +++ b/airflow/providers/celery/CHANGELOG.rst @@ -27,6 +27,23 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``Remove deprecated SubDags (#41390)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.3 ..... diff --git a/airflow/providers/celery/__init__.py b/airflow/providers/celery/__init__.py index c3eec53c7079..26be5e5fc445 100644 --- a/airflow/providers/celery/__init__.py +++ b/airflow/providers/celery/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.3" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/celery/executors/celery_executor_utils.py b/airflow/providers/celery/executors/celery_executor_utils.py index 5dd2d59ab0ea..8f25f040c90a 100644 --- a/airflow/providers/celery/executors/celery_executor_utils.py +++ b/airflow/providers/celery/executors/celery_executor_utils.py @@ -112,8 +112,7 @@ def on_celery_import_modules(*args, **kwargs): import airflow.jobs.local_task_job_runner import airflow.macros import airflow.operators.bash - import airflow.operators.python - import airflow.operators.subdag # noqa: F401 + import airflow.operators.python # noqa: F401 with contextlib.suppress(ImportError): import numpy # noqa: F401 diff --git a/airflow/providers/celery/provider.yaml b/airflow/providers/celery/provider.yaml index 6d79ae1ef36e..ba78272ba6b8 100644 --- a/airflow/providers/celery/provider.yaml +++ b/airflow/providers/celery/provider.yaml @@ -22,9 +22,10 @@ description: | `Celery `__ state: ready -source-date-epoch: 1722662983 +source-date-epoch: 1723969853 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.3 - 3.7.2 - 3.7.1 diff --git a/airflow/providers/cloudant/CHANGELOG.rst b/airflow/providers/cloudant/CHANGELOG.rst index 5de11778aa31..ceb4d95ed632 100644 --- a/airflow/providers/cloudant/CHANGELOG.rst +++ b/airflow/providers/cloudant/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.2 ..... diff --git a/airflow/providers/cloudant/__init__.py b/airflow/providers/cloudant/__init__.py index 6820e4b0b794..586634cbc807 100644 --- a/airflow/providers/cloudant/__init__.py +++ b/airflow/providers/cloudant/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.2" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/cloudant/provider.yaml b/airflow/providers/cloudant/provider.yaml index acf3c375fcd9..3f645d808ba9 100644 --- a/airflow/providers/cloudant/provider.yaml +++ b/airflow/providers/cloudant/provider.yaml @@ -22,9 +22,10 @@ description: | `IBM Cloudant `__ state: ready -source-date-epoch: 1718603992 +source-date-epoch: 1723969866 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.2 - 3.5.1 - 3.5.0 diff --git a/airflow/providers/cncf/kubernetes/CHANGELOG.rst b/airflow/providers/cncf/kubernetes/CHANGELOG.rst index ae5ee6716a50..62972f7631ad 100644 --- a/airflow/providers/cncf/kubernetes/CHANGELOG.rst +++ b/airflow/providers/cncf/kubernetes/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +8.4.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``Describe behaviour in docstring correctly (#41458)`` +* ``Remove deprecated SubDags (#41390)`` +* ``reorder docstring of 'SparkKubernetesOperator' (#41372)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 8.3.4 ..... diff --git a/airflow/providers/cncf/kubernetes/__init__.py b/airflow/providers/cncf/kubernetes/__init__.py index 203a4c01699b..e4d9114b5a75 100644 --- a/airflow/providers/cncf/kubernetes/__init__.py +++ b/airflow/providers/cncf/kubernetes/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "8.3.4" +__version__ = "8.4.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index 921fdaa92762..352c0773e54d 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -495,8 +495,9 @@ def _get_ti_pod_labels(context: Context | None = None, include_try_number: bool if include_try_number: labels.update(try_number=ti.try_number) # In the case of sub dags this is just useful - if context["dag"].parent_dag: - labels["parent_dag_id"] = context["dag"].parent_dag.dag_id + if getattr(context["dag"], "parent_dag", False): + labels["parent_dag_id"] = context["dag"].parent_dag.dag_id # type: ignore[attr-defined] + # Ensure that label is valid for Kube, # and if not truncate/remove invalid chars and replace with short hash. for label_id, label in labels.items(): @@ -1003,7 +1004,7 @@ def _set_name(name: str | None) -> str | None: return None def patch_already_checked(self, pod: k8s.V1Pod, *, reraise=True): - """Add an "already checked" annotation to ensure we don't reattach on retries.""" + """Add an "already checked" label to ensure we don't reattach on retries.""" with _optionally_suppress(reraise=reraise): self.client.patch_namespaced_pod( name=pod.metadata.name, diff --git a/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py b/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py index 82df0a2ec90c..d8a2867c64ac 100644 --- a/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py +++ b/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py @@ -202,7 +202,7 @@ def create_labels_for_pod(context: dict | None = None, include_try_number: bool labels.update(try_number=ti.try_number) # In the case of sub dags this is just useful - if context["dag"].is_subdag: + if getattr(context["dag"], "is_subdag", False): labels["parent_dag_id"] = context["dag"].parent_dag.dag_id # Ensure that label is valid for Kube, # and if not truncate/remove invalid chars and replace with short hash. diff --git a/airflow/providers/cncf/kubernetes/provider.yaml b/airflow/providers/cncf/kubernetes/provider.yaml index 8c6a0d01f2f0..cbbef370a7df 100644 --- a/airflow/providers/cncf/kubernetes/provider.yaml +++ b/airflow/providers/cncf/kubernetes/provider.yaml @@ -22,9 +22,10 @@ description: | `Kubernetes `__ state: ready -source-date-epoch: 1722663036 +source-date-epoch: 1723969891 # note that those versions are maintained by release manager - do not update them manually versions: + - 8.4.0 - 8.3.4 - 8.3.3 - 8.3.2 diff --git a/airflow/providers/cohere/CHANGELOG.rst b/airflow/providers/cohere/CHANGELOG.rst index 224669abae26..4b777c165b68 100644 --- a/airflow/providers/cohere/CHANGELOG.rst +++ b/airflow/providers/cohere/CHANGELOG.rst @@ -20,6 +20,25 @@ Changelog --------- +1.3.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 1.2.1 ..... diff --git a/airflow/providers/cohere/__init__.py b/airflow/providers/cohere/__init__.py index 88157e9c03b0..3c375df277e3 100644 --- a/airflow/providers/cohere/__init__.py +++ b/airflow/providers/cohere/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.2.1" +__version__ = "1.3.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/cohere/provider.yaml b/airflow/providers/cohere/provider.yaml index 4cd6f1a5686b..e5fa6ebb0271 100644 --- a/airflow/providers/cohere/provider.yaml +++ b/airflow/providers/cohere/provider.yaml @@ -24,10 +24,11 @@ description: | `Cohere `__ state: ready -source-date-epoch: 1716287173 +source-date-epoch: 1723969926 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.3.0 - 1.2.1 - 1.2.0 - 1.1.3 diff --git a/airflow/providers/common/compat/CHANGELOG.rst b/airflow/providers/common/compat/CHANGELOG.rst index 1f5fbe841f8e..27770f785cc7 100644 --- a/airflow/providers/common/compat/CHANGELOG.rst +++ b/airflow/providers/common/compat/CHANGELOG.rst @@ -25,6 +25,22 @@ Changelog --------- +1.2.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.1.0 ..... diff --git a/airflow/providers/common/compat/__init__.py b/airflow/providers/common/compat/__init__.py index e5b95b245656..ffe1acbcce74 100644 --- a/airflow/providers/common/compat/__init__.py +++ b/airflow/providers/common/compat/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.1.0" +__version__ = "1.2.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/common/compat/provider.yaml b/airflow/providers/common/compat/provider.yaml index a26b2bd37a6c..7ac3a0290092 100644 --- a/airflow/providers/common/compat/provider.yaml +++ b/airflow/providers/common/compat/provider.yaml @@ -22,9 +22,10 @@ description: | ``Common Compatibility Provider - providing compatibility code for previous Airflow versions.`` state: ready -source-date-epoch: 1716287191 +source-date-epoch: 1723970013 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.2.0 - 1.1.0 - 1.0.0 diff --git a/airflow/providers/common/sql/CHANGELOG.rst b/airflow/providers/common/sql/CHANGELOG.rst index 213a060b03a9..8cd190482c23 100644 --- a/airflow/providers/common/sql/CHANGELOG.rst +++ b/airflow/providers/common/sql/CHANGELOG.rst @@ -25,6 +25,23 @@ Changelog --------- +1.16.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Bug Fixes +~~~~~~~~~ + +* ``fix: rm deprecated import (#41461)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + 1.15.0 ...... diff --git a/airflow/providers/common/sql/__init__.py b/airflow/providers/common/sql/__init__.py index 6cbd75fe13ae..06d667928a2a 100644 --- a/airflow/providers/common/sql/__init__.py +++ b/airflow/providers/common/sql/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.15.0" +__version__ = "1.16.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/common/sql/operators/sql.py b/airflow/providers/common/sql/operators/sql.py index 262af5293d7b..f8b4080d6670 100644 --- a/airflow/providers/common/sql/operators/sql.py +++ b/airflow/providers/common/sql/operators/sql.py @@ -153,22 +153,6 @@ def _hook(self): conn = BaseHook.get_connection(conn_id) hook = conn.get_hook(hook_params=self.hook_params) if not isinstance(hook, DbApiHook): - from airflow.hooks.dbapi_hook import DbApiHook as _DbApiHook - - if isinstance(hook, _DbApiHook): - # This case might happen if user installed common.sql provider but did not upgrade the - # Other provider's versions to a version that supports common.sql provider - class_module = hook.__class__.__module__ - match = _PROVIDERS_MATCHER.match(class_module) - if match: - provider = match.group(1) - min_version = _MIN_SUPPORTED_PROVIDERS_VERSION.get(provider) - if min_version: - raise AirflowException( - f"You are trying to use common-sql with {hook.__class__.__name__}," - f" but the Hook class comes from provider {provider} that does not support it." - f" Please upgrade provider {provider} to at least {min_version}." - ) raise AirflowException( f"You are trying to use `common-sql` with {hook.__class__.__name__}," " but its provider does not support it. Please upgrade the provider to a version that" diff --git a/airflow/providers/common/sql/provider.yaml b/airflow/providers/common/sql/provider.yaml index d320c0dc7789..06c43af1d8db 100644 --- a/airflow/providers/common/sql/provider.yaml +++ b/airflow/providers/common/sql/provider.yaml @@ -22,9 +22,10 @@ description: | `Common SQL Provider `__ state: ready -source-date-epoch: 1722663378 +source-date-epoch: 1723970051 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.16.0 - 1.15.0 - 1.14.2 - 1.14.1 diff --git a/airflow/providers/databricks/CHANGELOG.rst b/airflow/providers/databricks/CHANGELOG.rst index 854193ee3191..ce52b4543a47 100644 --- a/airflow/providers/databricks/CHANGELOG.rst +++ b/airflow/providers/databricks/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +6.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 6.8.0 ..... diff --git a/airflow/providers/databricks/__init__.py b/airflow/providers/databricks/__init__.py index b6dc7bb41336..1638a00f3b52 100644 --- a/airflow/providers/databricks/__init__.py +++ b/airflow/providers/databricks/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "6.8.0" +__version__ = "6.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/databricks/provider.yaml b/airflow/providers/databricks/provider.yaml index f815b8a3c8e4..c5200fa2f80d 100644 --- a/airflow/providers/databricks/provider.yaml +++ b/airflow/providers/databricks/provider.yaml @@ -22,9 +22,10 @@ description: | `Databricks `__ state: ready -source-date-epoch: 1722663644 +source-date-epoch: 1723970061 # note that those versions are maintained by release manager - do not update them manually versions: + - 6.9.0 - 6.8.0 - 6.7.0 - 6.6.0 diff --git a/airflow/providers/datadog/CHANGELOG.rst b/airflow/providers/datadog/CHANGELOG.rst index 9768f5da5f95..e8fcaaf8d527 100644 --- a/airflow/providers/datadog/CHANGELOG.rst +++ b/airflow/providers/datadog/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.6.1 ..... diff --git a/airflow/providers/datadog/__init__.py b/airflow/providers/datadog/__init__.py index ffcc5680f65a..bd991e53a0c6 100644 --- a/airflow/providers/datadog/__init__.py +++ b/airflow/providers/datadog/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.6.1" +__version__ = "3.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/datadog/provider.yaml b/airflow/providers/datadog/provider.yaml index 454aba20d4af..f315e2524e92 100644 --- a/airflow/providers/datadog/provider.yaml +++ b/airflow/providers/datadog/provider.yaml @@ -22,9 +22,10 @@ description: | `Datadog `__ state: ready -source-date-epoch: 1716287288 +source-date-epoch: 1723970068 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.7.0 - 3.6.1 - 3.6.0 - 3.5.1 diff --git a/airflow/providers/dbt/cloud/CHANGELOG.rst b/airflow/providers/dbt/cloud/CHANGELOG.rst index cd5968a45124..1b0764c50bbc 100644 --- a/airflow/providers/dbt/cloud/CHANGELOG.rst +++ b/airflow/providers/dbt/cloud/CHANGELOG.rst @@ -28,6 +28,25 @@ Changelog --------- +3.10.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.9.0 ..... diff --git a/airflow/providers/dbt/cloud/__init__.py b/airflow/providers/dbt/cloud/__init__.py index 20b087747aa7..bf3c3e415fac 100644 --- a/airflow/providers/dbt/cloud/__init__.py +++ b/airflow/providers/dbt/cloud/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.9.0" +__version__ = "3.10.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/dbt/cloud/provider.yaml b/airflow/providers/dbt/cloud/provider.yaml index f2b9284864f3..fde92d54f262 100644 --- a/airflow/providers/dbt/cloud/provider.yaml +++ b/airflow/providers/dbt/cloud/provider.yaml @@ -22,9 +22,10 @@ description: | `dbt Cloud `__ state: ready -source-date-epoch: 1718604259 +source-date-epoch: 1723970074 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.10.0 - 3.9.0 - 3.8.1 - 3.8.0 diff --git a/airflow/providers/dingding/CHANGELOG.rst b/airflow/providers/dingding/CHANGELOG.rst index 9b5ce5810b29..943df8c619cf 100644 --- a/airflow/providers/dingding/CHANGELOG.rst +++ b/airflow/providers/dingding/CHANGELOG.rst @@ -28,6 +28,25 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.5.1 ..... diff --git a/airflow/providers/dingding/__init__.py b/airflow/providers/dingding/__init__.py index 38e476f58a92..568e2f3d8aed 100644 --- a/airflow/providers/dingding/__init__.py +++ b/airflow/providers/dingding/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/dingding/provider.yaml b/airflow/providers/dingding/provider.yaml index 7d36ea42365f..caa0f2b93dc4 100644 --- a/airflow/providers/dingding/provider.yaml +++ b/airflow/providers/dingding/provider.yaml @@ -22,9 +22,10 @@ description: | `DingTalk `__ state: ready -source-date-epoch: 1716287570 +source-date-epoch: 1723970083 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/discord/CHANGELOG.rst b/airflow/providers/discord/CHANGELOG.rst index cf3f02df4e7a..35376ae8ed56 100644 --- a/airflow/providers/discord/CHANGELOG.rst +++ b/airflow/providers/discord/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.1 ..... diff --git a/airflow/providers/discord/__init__.py b/airflow/providers/discord/__init__.py index 9944545421d2..94a385dee1d4 100644 --- a/airflow/providers/discord/__init__.py +++ b/airflow/providers/discord/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.1" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/discord/provider.yaml b/airflow/providers/discord/provider.yaml index c485218ebe9e..250e3ac87476 100644 --- a/airflow/providers/discord/provider.yaml +++ b/airflow/providers/discord/provider.yaml @@ -22,9 +22,10 @@ description: | `Discord `__ state: ready -source-date-epoch: 1716287589 +source-date-epoch: 1723970089 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.1 - 3.7.0 - 3.6.0 diff --git a/airflow/providers/docker/CHANGELOG.rst b/airflow/providers/docker/CHANGELOG.rst index 1d0ce6b267e8..c5e3b7734682 100644 --- a/airflow/providers/docker/CHANGELOG.rst +++ b/airflow/providers/docker/CHANGELOG.rst @@ -27,6 +27,23 @@ Changelog --------- +3.13.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``feat(docker): Replace 'use_dill' with 'serializer' (#41356)`` +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.12.3 ...... diff --git a/airflow/providers/docker/__init__.py b/airflow/providers/docker/__init__.py index 4424f1150286..a5cf7e4e4010 100644 --- a/airflow/providers/docker/__init__.py +++ b/airflow/providers/docker/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.12.3" +__version__ = "3.13.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/docker/decorators/docker.py b/airflow/providers/docker/decorators/docker.py index d851c98aca5d..9812e5fc5748 100644 --- a/airflow/providers/docker/decorators/docker.py +++ b/airflow/providers/docker/decorators/docker.py @@ -18,13 +18,12 @@ import base64 import os -import pickle +import warnings from tempfile import TemporaryDirectory -from typing import TYPE_CHECKING, Callable, Sequence - -import dill +from typing import TYPE_CHECKING, Any, Callable, Literal, Sequence from airflow.decorators.base import DecoratedOperator, task_decorator_factory +from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning from airflow.providers.docker.operators.docker import DockerOperator from airflow.utils.python_virtualenv import write_python_script @@ -32,6 +31,47 @@ from airflow.decorators.base import TaskDecorator from airflow.utils.context import Context + Serializer = Literal["pickle", "dill", "cloudpickle"] + +try: + from airflow.operators.python import _SERIALIZERS +except ImportError: + import logging + + import lazy_object_proxy + + log = logging.getLogger(__name__) + + def _load_pickle(): + import pickle + + return pickle + + def _load_dill(): + try: + import dill + except ModuleNotFoundError: + log.error("Unable to import `dill` module. Please please make sure that it installed.") + raise + return dill + + def _load_cloudpickle(): + try: + import cloudpickle + except ModuleNotFoundError: + log.error( + "Unable to import `cloudpickle` module. " + "Please install it with: pip install 'apache-airflow[cloudpickle]'" + ) + raise + return cloudpickle + + _SERIALIZERS: dict[Serializer, Any] = { # type: ignore[no-redef] + "pickle": lazy_object_proxy.Proxy(_load_pickle), + "dill": lazy_object_proxy.Proxy(_load_dill), + "cloudpickle": lazy_object_proxy.Proxy(_load_cloudpickle), + } + def _generate_decode_command(env_var, file, python_command): # We don't need `f.close()` as the interpreter is about to exit anyway @@ -53,7 +93,6 @@ class _DockerDecoratedOperator(DecoratedOperator, DockerOperator): :param python_callable: A reference to an object that is callable :param python: Python binary name to use - :param use_dill: Whether dill should be used to serialize the callable :param expect_airflow: whether to expect airflow to be installed in the docker environment. if this one is specified, the script to run callable will attempt to load Airflow macros. :param op_kwargs: a dictionary of keyword arguments that will get unpacked @@ -63,6 +102,16 @@ class _DockerDecoratedOperator(DecoratedOperator, DockerOperator): :param multiple_outputs: if set, function return value will be unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys. Defaults to False. + :param serializer: Which serializer use to serialize the args and result. It can be one of the following: + + - ``"pickle"``: (default) Use pickle for serialization. Included in the Python Standard Library. + - ``"cloudpickle"``: Use cloudpickle for serialize more complex types, + this requires to include cloudpickle in your requirements. + - ``"dill"``: Use dill for serialize more complex types, + this requires to include dill in your requirements. + :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize + the args and result (pickle is default). This allows more complex types + but requires you to include dill in your requirements. """ custom_operator_name = "@task.docker" @@ -74,12 +123,35 @@ def __init__( use_dill=False, python_command="python3", expect_airflow: bool = True, + serializer: Serializer | None = None, **kwargs, ) -> None: + if use_dill: + warnings.warn( + "`use_dill` is deprecated and will be removed in a future version. " + "Please provide serializer='dill' instead.", + AirflowProviderDeprecationWarning, + stacklevel=3, + ) + if serializer: + raise AirflowException( + "Both 'use_dill' and 'serializer' parameters are set. Please set only one of them" + ) + serializer = "dill" + serializer = serializer or "pickle" + if serializer not in _SERIALIZERS: + msg = ( + f"Unsupported serializer {serializer!r}. " + f"Expected one of {', '.join(map(repr, _SERIALIZERS))}" + ) + raise AirflowException(msg) + command = "placeholder command" self.python_command = python_command self.expect_airflow = expect_airflow - self.use_dill = use_dill + self.use_dill = serializer == "dill" + self.serializer: Serializer = serializer + super().__init__( command=command, retrieve_output=True, retrieve_output_path="/tmp/script.out", **kwargs ) @@ -128,9 +200,7 @@ def execute(self, context: Context): @property def pickling_library(self): - if self.use_dill: - return dill - return pickle + return _SERIALIZERS[self.serializer] def docker_task( diff --git a/airflow/providers/docker/provider.yaml b/airflow/providers/docker/provider.yaml index 23f5548656ea..014720029360 100644 --- a/airflow/providers/docker/provider.yaml +++ b/airflow/providers/docker/provider.yaml @@ -22,9 +22,10 @@ description: | `Docker `__ state: ready -source-date-epoch: 1722663826 +source-date-epoch: 1723970104 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.13.0 - 3.12.3 - 3.12.2 - 3.12.1 diff --git a/airflow/providers/elasticsearch/CHANGELOG.rst b/airflow/providers/elasticsearch/CHANGELOG.rst index 03692d91818e..c63ab7d97b5d 100644 --- a/airflow/providers/elasticsearch/CHANGELOG.rst +++ b/airflow/providers/elasticsearch/CHANGELOG.rst @@ -27,6 +27,26 @@ Changelog --------- +5.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Bug Fixes +~~~~~~~~~ + +* ``Fix 'ElasticsearchSQLHook' fails with 'AttributeError: __enter__' (#41537)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.4.2 ..... diff --git a/airflow/providers/elasticsearch/__init__.py b/airflow/providers/elasticsearch/__init__.py index 21aabca87dc5..106358c8b7c7 100644 --- a/airflow/providers/elasticsearch/__init__.py +++ b/airflow/providers/elasticsearch/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.4.2" +__version__ = "5.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/elasticsearch/hooks/elasticsearch.py b/airflow/providers/elasticsearch/hooks/elasticsearch.py index 444ad3220662..11039651c0ed 100644 --- a/airflow/providers/elasticsearch/hooks/elasticsearch.py +++ b/airflow/providers/elasticsearch/hooks/elasticsearch.py @@ -23,12 +23,15 @@ from deprecated import deprecated from elasticsearch import Elasticsearch +from elasticsearch.client import SqlClient from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.hooks.base import BaseHook from airflow.providers.common.sql.hooks.sql import DbApiHook if TYPE_CHECKING: + from elastic_transport import ObjectApiResponse + from airflow.models.connection import Connection as AirflowConnection @@ -67,6 +70,11 @@ def __init__( self.es = Elasticsearch(self.url, http_auth=(user, password), **self.kwargs) else: self.es = Elasticsearch(self.url, **self.kwargs) + self.es_sql_client = SqlClient(self.es) + + def execute_sql(self, query: str) -> ObjectApiResponse: + sql_query = {"query": query} + return self.es_sql_client.query(body=sql_query) class ElasticsearchSQLHook(DbApiHook): diff --git a/airflow/providers/elasticsearch/provider.yaml b/airflow/providers/elasticsearch/provider.yaml index 559e0bfa8f3b..f0417b7e6d94 100644 --- a/airflow/providers/elasticsearch/provider.yaml +++ b/airflow/providers/elasticsearch/provider.yaml @@ -22,9 +22,10 @@ description: | `Elasticsearch `__ state: ready -source-date-epoch: 1722663865 +source-date-epoch: 1723970114 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.5.0 - 5.4.2 - 5.4.1 - 5.4.0 diff --git a/airflow/providers/exasol/CHANGELOG.rst b/airflow/providers/exasol/CHANGELOG.rst index e3365bf6bcfe..c9e8847dc498 100644 --- a/airflow/providers/exasol/CHANGELOG.rst +++ b/airflow/providers/exasol/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +4.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.5.3 ..... diff --git a/airflow/providers/exasol/__init__.py b/airflow/providers/exasol/__init__.py index e78be37f1c9a..315d4a216a70 100644 --- a/airflow/providers/exasol/__init__.py +++ b/airflow/providers/exasol/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.5.3" +__version__ = "4.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/exasol/provider.yaml b/airflow/providers/exasol/provider.yaml index 6ffc07821b7d..bca92be13770 100644 --- a/airflow/providers/exasol/provider.yaml +++ b/airflow/providers/exasol/provider.yaml @@ -22,9 +22,10 @@ description: | `Exasol `__ state: ready -source-date-epoch: 1722663904 +source-date-epoch: 1723970121 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.6.0 - 4.5.3 - 4.5.2 - 4.5.1 diff --git a/airflow/providers/fab/CHANGELOG.rst b/airflow/providers/fab/CHANGELOG.rst index 9c9e29412793..8ca6b42b0559 100644 --- a/airflow/providers/fab/CHANGELOG.rst +++ b/airflow/providers/fab/CHANGELOG.rst @@ -20,6 +20,23 @@ Changelog --------- +1.3.0 +..... + +Features +~~~~~~~~ + +* ``Feature: Allow set Dag Run resource into Dag Level permission (#40703)`` + +Misc +~~~~ + +* ``Remove deprecated SubDags (#41390)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.2.2 ..... diff --git a/airflow/providers/fab/__init__.py b/airflow/providers/fab/__init__.py index c59168fb92ad..a5fcd56ccdcd 100644 --- a/airflow/providers/fab/__init__.py +++ b/airflow/providers/fab/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.2.2" +__version__ = "1.3.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.9.0" diff --git a/airflow/providers/fab/auth_manager/fab_auth_manager.py b/airflow/providers/fab/auth_manager/fab_auth_manager.py index 344df7588de7..ceec5c0e37bd 100644 --- a/airflow/providers/fab/auth_manager/fab_auth_manager.py +++ b/airflow/providers/fab/auth_manager/fab_auth_manager.py @@ -503,7 +503,7 @@ def _get_root_dag_id(self, dag_id: str) -> str: :meta private: """ - if "." in dag_id: + if "." in dag_id and hasattr(DagModel, "root_dag_id"): return self.appbuilder.get_session.scalar( select(DagModel.dag_id, DagModel.root_dag_id).where(DagModel.dag_id == dag_id).limit(1) ) diff --git a/airflow/providers/fab/auth_manager/security_manager/override.py b/airflow/providers/fab/auth_manager/security_manager/override.py index e2208e5fb409..86d76de76ff2 100644 --- a/airflow/providers/fab/auth_manager/security_manager/override.py +++ b/airflow/providers/fab/auth_manager/security_manager/override.py @@ -1073,7 +1073,7 @@ def create_dag_specific_permissions(self) -> None: dags = dagbag.dags.values() for dag in dags: - root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id + root_dag_id = (getattr(dag, "parent_dag", None) or dag).dag_id for resource_name, resource_values in self.RESOURCE_DETAILS_MAP.items(): dag_resource_name = self._resource_name(root_dag_id, resource_name) for action_name in resource_values["actions"]: @@ -2828,7 +2828,7 @@ def filter_roles_by_perm_with_action(self, action_name: str, role_ids: list[int] ).all() def _get_root_dag_id(self, dag_id: str) -> str: - if "." in dag_id: + if "." in dag_id and hasattr(DagModel, "root_dag_id"): dm = self.appbuilder.get_session.execute( select(DagModel.dag_id, DagModel.root_dag_id).where(DagModel.dag_id == dag_id) ).one() diff --git a/airflow/providers/fab/provider.yaml b/airflow/providers/fab/provider.yaml index 57cce91c754f..741297d4466c 100644 --- a/airflow/providers/fab/provider.yaml +++ b/airflow/providers/fab/provider.yaml @@ -28,10 +28,11 @@ description: | # For providers until we think it should be released. state: ready -source-date-epoch: 1722149665 +source-date-epoch: 1723970140 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.3.0 - 1.2.2 - 1.2.1 - 1.2.0 diff --git a/airflow/providers/facebook/CHANGELOG.rst b/airflow/providers/facebook/CHANGELOG.rst index d89db1bf7d64..e2cb82864a60 100644 --- a/airflow/providers/facebook/CHANGELOG.rst +++ b/airflow/providers/facebook/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.5.2 ..... diff --git a/airflow/providers/facebook/__init__.py b/airflow/providers/facebook/__init__.py index 0ff9788e7ef5..5e2c2e7e419d 100644 --- a/airflow/providers/facebook/__init__.py +++ b/airflow/providers/facebook/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.2" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/facebook/provider.yaml b/airflow/providers/facebook/provider.yaml index fa6ee6a02423..6bcaa88404d3 100644 --- a/airflow/providers/facebook/provider.yaml +++ b/airflow/providers/facebook/provider.yaml @@ -22,9 +22,10 @@ description: | `Facebook Ads `__ state: ready -source-date-epoch: 1718604396 +source-date-epoch: 1723970150 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.2 - 3.5.1 - 3.5.0 diff --git a/airflow/providers/ftp/CHANGELOG.rst b/airflow/providers/ftp/CHANGELOG.rst index d1b452cddd57..8c854ed2a14d 100644 --- a/airflow/providers/ftp/CHANGELOG.rst +++ b/airflow/providers/ftp/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.11.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.10.1 ...... diff --git a/airflow/providers/ftp/__init__.py b/airflow/providers/ftp/__init__.py index 1d691e34bb05..35607463e5b3 100644 --- a/airflow/providers/ftp/__init__.py +++ b/airflow/providers/ftp/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.10.1" +__version__ = "3.11.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/ftp/provider.yaml b/airflow/providers/ftp/provider.yaml index de1023c59ff1..f2b80d9b5fbf 100644 --- a/airflow/providers/ftp/provider.yaml +++ b/airflow/providers/ftp/provider.yaml @@ -22,8 +22,9 @@ description: | `File Transfer Protocol (FTP) `__ state: ready -source-date-epoch: 1722663922 +source-date-epoch: 1723970162 versions: + - 3.11.0 - 3.10.1 - 3.10.0 - 3.9.1 diff --git a/airflow/providers/github/CHANGELOG.rst b/airflow/providers/github/CHANGELOG.rst index 6787e13ff400..8095f5840eb8 100644 --- a/airflow/providers/github/CHANGELOG.rst +++ b/airflow/providers/github/CHANGELOG.rst @@ -29,6 +29,25 @@ Changelog --------- +2.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 2.6.2 ..... diff --git a/airflow/providers/github/__init__.py b/airflow/providers/github/__init__.py index 3b0aef4eb14f..453a74a4c4f9 100644 --- a/airflow/providers/github/__init__.py +++ b/airflow/providers/github/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.6.2" +__version__ = "2.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/github/provider.yaml b/airflow/providers/github/provider.yaml index 5f9ab2d9a737..eaa025adcba4 100644 --- a/airflow/providers/github/provider.yaml +++ b/airflow/providers/github/provider.yaml @@ -27,9 +27,10 @@ dependencies: - PyGithub>=2.1.1 state: ready -source-date-epoch: 1718604440 +source-date-epoch: 1723970175 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.7.0 - 2.6.2 - 2.6.1 - 2.6.0 diff --git a/airflow/providers/google/CHANGELOG.rst b/airflow/providers/google/CHANGELOG.rst index 6127e8c7a8c9..64ae3b4e919e 100644 --- a/airflow/providers/google/CHANGELOG.rst +++ b/airflow/providers/google/CHANGELOG.rst @@ -27,6 +27,34 @@ Changelog --------- +10.22.0 +....... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Features +~~~~~~~~ + +* ``Add 'CloudRunServiceHook' and 'CloudRunCreateServiceOperator' (#40008)`` + +Bug Fixes +~~~~~~~~~ + +* ``fix(providers/google): add missing sync_hook_class to CloudDataTransferServiceAsyncHook (#41417)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``Refactor 'DataprocCreateBatchOperator' (#41527)`` +* ``Upgrade package gcloud-aio-auth>=5.2.0 (#41262)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 10.21.1 ....... diff --git a/airflow/providers/google/__init__.py b/airflow/providers/google/__init__.py index a32a368212ad..654b99496b99 100644 --- a/airflow/providers/google/__init__.py +++ b/airflow/providers/google/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "10.21.1" +__version__ = "10.22.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/google/cloud/operators/dataproc.py b/airflow/providers/google/cloud/operators/dataproc.py index efaf0d6326d0..2384bfbd6251 100644 --- a/airflow/providers/google/cloud/operators/dataproc.py +++ b/airflow/providers/google/cloud/operators/dataproc.py @@ -30,6 +30,7 @@ from dataclasses import dataclass from datetime import datetime, timedelta from enum import Enum +from functools import cached_property from typing import TYPE_CHECKING, Any, Sequence from deprecated import deprecated @@ -638,7 +639,7 @@ def __init__( request_id: str | None = None, delete_on_error: bool = True, use_if_exists: bool = True, - retry: AsyncRetry | _MethodDefault = DEFAULT, + retry: AsyncRetry | _MethodDefault | Retry = DEFAULT, timeout: float = 1 * 60 * 60, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", @@ -1184,7 +1185,7 @@ def __init__( project_id: str = PROVIDE_PROJECT_ID, cluster_uuid: str | None = None, request_id: str | None = None, - retry: AsyncRetry | _MethodDefault = DEFAULT, + retry: AsyncRetry | _MethodDefault | Retry = DEFAULT, timeout: float = 1 * 60 * 60, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", @@ -2712,7 +2713,7 @@ def __init__( region: str, request_id: str | None = None, project_id: str = PROVIDE_PROJECT_ID, - retry: AsyncRetry | _MethodDefault = DEFAULT, + retry: AsyncRetry | _MethodDefault | Retry = DEFAULT, timeout: float | None = None, metadata: Sequence[tuple[str, str]] = (), gcp_conn_id: str = "google_cloud_default", @@ -2985,10 +2986,10 @@ class DataprocCreateBatchOperator(GoogleCloudBaseOperator): def __init__( self, *, - region: str | None = None, + region: str, project_id: str = PROVIDE_PROJECT_ID, batch: dict | Batch, - batch_id: str, + batch_id: str | None = None, request_id: str | None = None, retry: Retry | _MethodDefault = DEFAULT, timeout: float | None = None, @@ -3021,20 +3022,20 @@ def __init__( self.polling_interval_seconds = polling_interval_seconds def execute(self, context: Context): - hook = DataprocHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain) - # batch_id might not be set and will be generated - if self.batch_id: - link = DATAPROC_BATCH_LINK.format( - region=self.region, project_id=self.project_id, batch_id=self.batch_id + if self.asynchronous and self.deferrable: + raise AirflowException( + "Both asynchronous and deferrable parameters were passed. Please, provide only one." ) - self.log.info("Creating batch %s", self.batch_id) - self.log.info("Once started, the batch job will be available at %s", link) + + batch_id: str = "" + if self.batch_id: + batch_id = self.batch_id + self.log.info("Starting batch %s", batch_id) else: - self.log.info("Starting batch job. The batch ID will be generated since it was not provided.") - if self.region is None: - raise AirflowException("Region should be set here") + self.log.info("Starting batch. The batch ID will be generated since it was not provided.") + try: - self.operation = hook.create_batch( + self.operation = self.hook.create_batch( region=self.region, project_id=self.project_id, batch=self.batch, @@ -3044,85 +3045,62 @@ def execute(self, context: Context): timeout=self.timeout, metadata=self.metadata, ) - if self.operation is None: - raise RuntimeError("The operation should be set here!") - - if not self.deferrable: - if not self.asynchronous: - result = hook.wait_for_operation( - timeout=self.timeout, result_retry=self.result_retry, operation=self.operation - ) - self.log.info("Batch %s created", self.batch_id) - - else: - DataprocBatchLink.persist( - context=context, - operator=self, - project_id=self.project_id, - region=self.region, - batch_id=self.batch_id, - ) - return self.operation.operation.name - - else: - # processing ends in execute_complete - self.defer( - trigger=DataprocBatchTrigger( - batch_id=self.batch_id, - project_id=self.project_id, - region=self.region, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - polling_interval_seconds=self.polling_interval_seconds, - ), - method_name="execute_complete", - ) - except AlreadyExists: - self.log.info("Batch with given id already exists") - # This is only likely to happen if batch_id was provided - # Could be running if Airflow was restarted after task started - # poll until a final state is reached - - self.log.info("Attaching to the job %s if it is still running.", self.batch_id) + self.log.info("Batch with given id already exists.") + self.log.info("Attaching to the job %s if it is still running.", batch_id) + else: + batch_id = self.operation.metadata.batch.split("/")[-1] + self.log.info("The batch %s was created.", batch_id) - # deferrable handling of a batch_id that already exists - processing ends in execute_complete - if self.deferrable: - self.defer( - trigger=DataprocBatchTrigger( - batch_id=self.batch_id, - project_id=self.project_id, - region=self.region, - gcp_conn_id=self.gcp_conn_id, - impersonation_chain=self.impersonation_chain, - polling_interval_seconds=self.polling_interval_seconds, - ), - method_name="execute_complete", - ) + DataprocBatchLink.persist( + context=context, + operator=self, + project_id=self.project_id, + region=self.region, + batch_id=batch_id, + ) - # non-deferrable handling of a batch_id that already exists - result = hook.wait_for_batch( - batch_id=self.batch_id, + if self.asynchronous: + batch = self.hook.get_batch( + batch_id=batch_id, region=self.region, project_id=self.project_id, retry=self.retry, timeout=self.timeout, metadata=self.metadata, - wait_check_interval=self.polling_interval_seconds, ) - batch_id = self.batch_id or result.name.split("/")[-1] + self.log.info("The batch %s was created asynchronously. Exiting.", batch_id) + return Batch.to_dict(batch) - self.handle_batch_status(context, result.state, batch_id) - project_id = self.project_id or hook.project_id - if project_id: - DataprocBatchLink.persist( - context=context, - operator=self, - project_id=project_id, - region=self.region, - batch_id=batch_id, + if self.deferrable: + self.defer( + trigger=DataprocBatchTrigger( + batch_id=batch_id, + project_id=self.project_id, + region=self.region, + gcp_conn_id=self.gcp_conn_id, + impersonation_chain=self.impersonation_chain, + polling_interval_seconds=self.polling_interval_seconds, + ), + method_name="execute_complete", ) - return Batch.to_dict(result) + + self.log.info("Waiting for the completion of batch job %s", batch_id) + batch = self.hook.wait_for_batch( + batch_id=batch_id, + region=self.region, + project_id=self.project_id, + retry=self.retry, + timeout=self.timeout, + metadata=self.metadata, + ) + + self.handle_batch_status(context, batch.state, batch_id, batch.state_message) + return Batch.to_dict(batch) + + @cached_property + def hook(self) -> DataprocHook: + return DataprocHook(gcp_conn_id=self.gcp_conn_id, impersonation_chain=self.impersonation_chain) def execute_complete(self, context, event=None) -> None: """ @@ -3135,23 +3113,27 @@ def execute_complete(self, context, event=None) -> None: raise AirflowException("Batch failed.") state = event["batch_state"] batch_id = event["batch_id"] - self.handle_batch_status(context, state, batch_id) + self.handle_batch_status(context, state, batch_id, state_message=event["batch_state_message"]) def on_kill(self): if self.operation: self.operation.cancel() - def handle_batch_status(self, context: Context, state: Batch.State, batch_id: str) -> None: + def handle_batch_status( + self, context: Context, state: Batch.State, batch_id: str, state_message: str | None = None + ) -> None: # The existing batch may be a number of states other than 'SUCCEEDED'\ # wait_for_operation doesn't fail if the job is cancelled, so we will check for it here which also # finds a cancelling|canceled|unspecified job from wait_for_batch or the deferred trigger link = DATAPROC_BATCH_LINK.format(region=self.region, project_id=self.project_id, batch_id=batch_id) if state == Batch.State.FAILED: - raise AirflowException("Batch job %s failed. Driver Logs: %s", batch_id, link) + raise AirflowException( + f"Batch job {batch_id} failed with error: {state_message}\nDriver Logs: {link}" + ) if state in (Batch.State.CANCELLED, Batch.State.CANCELLING): - raise AirflowException("Batch job %s was cancelled. Driver logs: %s", batch_id, link) + raise AirflowException(f"Batch job {batch_id} was cancelled. Driver logs: {link}") if state == Batch.State.STATE_UNSPECIFIED: - raise AirflowException("Batch job %s unspecified. Driver logs: %s", batch_id, link) + raise AirflowException(f"Batch job {batch_id} unspecified. Driver logs: {link}") self.log.info("Batch job %s completed. Driver logs: %s", batch_id, link) diff --git a/airflow/providers/google/cloud/triggers/dataproc.py b/airflow/providers/google/cloud/triggers/dataproc.py index 99800d266a86..508b0444c05a 100644 --- a/airflow/providers/google/cloud/triggers/dataproc.py +++ b/airflow/providers/google/cloud/triggers/dataproc.py @@ -371,7 +371,10 @@ async def run(self): self.log.info("Current state is %s", state) self.log.info("Sleeping for %s seconds.", self.polling_interval_seconds) await asyncio.sleep(self.polling_interval_seconds) - yield TriggerEvent({"batch_id": self.batch_id, "batch_state": state}) + + yield TriggerEvent( + {"batch_id": self.batch_id, "batch_state": state, "batch_state_message": batch.state_message} + ) class DataprocDeleteClusterTrigger(DataprocBaseTrigger): diff --git a/airflow/providers/google/provider.yaml b/airflow/providers/google/provider.yaml index 28c0112712d8..cd49f2cb6e42 100644 --- a/airflow/providers/google/provider.yaml +++ b/airflow/providers/google/provider.yaml @@ -29,9 +29,10 @@ description: | - `Google Workspace `__ (formerly Google Suite) state: ready -source-date-epoch: 1722145669 +source-date-epoch: 1723970253 # note that those versions are maintained by release manager - do not update them manually versions: + - 10.22.0 - 10.21.1 - 10.21.0 - 10.20.0 diff --git a/airflow/providers/grpc/CHANGELOG.rst b/airflow/providers/grpc/CHANGELOG.rst index 4d5aa699d6ff..88761f776c81 100644 --- a/airflow/providers/grpc/CHANGELOG.rst +++ b/airflow/providers/grpc/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.2 ..... diff --git a/airflow/providers/grpc/__init__.py b/airflow/providers/grpc/__init__.py index 6ad5c8d74197..0806e492d06c 100644 --- a/airflow/providers/grpc/__init__.py +++ b/airflow/providers/grpc/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.2" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/grpc/provider.yaml b/airflow/providers/grpc/provider.yaml index 68b6d4fdc0c6..66d44073114b 100644 --- a/airflow/providers/grpc/provider.yaml +++ b/airflow/providers/grpc/provider.yaml @@ -22,9 +22,10 @@ description: | `gRPC `__ state: ready -source-date-epoch: 1718604847 +source-date-epoch: 1723970286 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.2 - 3.5.1 - 3.5.0 diff --git a/airflow/providers/hashicorp/CHANGELOG.rst b/airflow/providers/hashicorp/CHANGELOG.rst index 4d83bc77871a..032f9d565c85 100644 --- a/airflow/providers/hashicorp/CHANGELOG.rst +++ b/airflow/providers/hashicorp/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.1 ..... diff --git a/airflow/providers/hashicorp/__init__.py b/airflow/providers/hashicorp/__init__.py index b71b1c480a3e..9ae09d6f00b3 100644 --- a/airflow/providers/hashicorp/__init__.py +++ b/airflow/providers/hashicorp/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.1" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/hashicorp/provider.yaml b/airflow/providers/hashicorp/provider.yaml index 700dae276138..74a979c20e21 100644 --- a/airflow/providers/hashicorp/provider.yaml +++ b/airflow/providers/hashicorp/provider.yaml @@ -22,9 +22,10 @@ description: | Hashicorp including `Hashicorp Vault `__ state: ready -source-date-epoch: 1716287935 +source-date-epoch: 1723970290 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.1 - 3.7.0 - 3.6.4 diff --git a/airflow/providers/http/CHANGELOG.rst b/airflow/providers/http/CHANGELOG.rst index de56023cbd04..810381bdb1eb 100644 --- a/airflow/providers/http/CHANGELOG.rst +++ b/airflow/providers/http/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +4.13.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 4.12.0 ...... diff --git a/airflow/providers/http/__init__.py b/airflow/providers/http/__init__.py index 90576be9a103..9ceb1d545281 100644 --- a/airflow/providers/http/__init__.py +++ b/airflow/providers/http/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.12.0" +__version__ = "4.13.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/http/provider.yaml b/airflow/providers/http/provider.yaml index d3fae7c036bc..9baa1870dc4c 100644 --- a/airflow/providers/http/provider.yaml +++ b/airflow/providers/http/provider.yaml @@ -22,9 +22,10 @@ description: | `Hypertext Transfer Protocol (HTTP) `__ state: ready -source-date-epoch: 1718604867 +source-date-epoch: 1723970300 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.13.0 - 4.12.0 - 4.11.1 - 4.11.0 diff --git a/airflow/providers/imap/CHANGELOG.rst b/airflow/providers/imap/CHANGELOG.rst index 44915a256828..ec5267fc0af9 100644 --- a/airflow/providers/imap/CHANGELOG.rst +++ b/airflow/providers/imap/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +3.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.6.1 ..... diff --git a/airflow/providers/imap/__init__.py b/airflow/providers/imap/__init__.py index b76247aabe17..b4b17d91c6ce 100644 --- a/airflow/providers/imap/__init__.py +++ b/airflow/providers/imap/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.6.1" +__version__ = "3.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/imap/provider.yaml b/airflow/providers/imap/provider.yaml index 998c033d0ad6..988a29341f87 100644 --- a/airflow/providers/imap/provider.yaml +++ b/airflow/providers/imap/provider.yaml @@ -23,9 +23,10 @@ description: | `Internet Message Access Protocol (IMAP) `__ state: ready -source-date-epoch: 1716287988 +source-date-epoch: 1723970306 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.7.0 - 3.6.1 - 3.6.0 - 3.5.0 diff --git a/airflow/providers/influxdb/CHANGELOG.rst b/airflow/providers/influxdb/CHANGELOG.rst index c9f1ced04fce..98b8d35de53b 100644 --- a/airflow/providers/influxdb/CHANGELOG.rst +++ b/airflow/providers/influxdb/CHANGELOG.rst @@ -28,6 +28,22 @@ Changelog --------- +2.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.6.0 ..... diff --git a/airflow/providers/influxdb/__init__.py b/airflow/providers/influxdb/__init__.py index 13e31d7e3774..3386ba166c96 100644 --- a/airflow/providers/influxdb/__init__.py +++ b/airflow/providers/influxdb/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.6.0" +__version__ = "2.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/influxdb/provider.yaml b/airflow/providers/influxdb/provider.yaml index cbc13bc3ea96..5f7d5f272e1a 100644 --- a/airflow/providers/influxdb/provider.yaml +++ b/airflow/providers/influxdb/provider.yaml @@ -29,9 +29,10 @@ dependencies: - requests>=2.27.0,<3 state: ready -source-date-epoch: 1720423610 +source-date-epoch: 1723970310 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.7.0 - 2.6.0 - 2.5.1 - 2.5.0 diff --git a/airflow/providers/jdbc/CHANGELOG.rst b/airflow/providers/jdbc/CHANGELOG.rst index c5c5de5cbc83..9edc93c7320e 100644 --- a/airflow/providers/jdbc/CHANGELOG.rst +++ b/airflow/providers/jdbc/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +4.5.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.4.0 ..... diff --git a/airflow/providers/jdbc/__init__.py b/airflow/providers/jdbc/__init__.py index 92f3fca45acf..27363ab52f71 100644 --- a/airflow/providers/jdbc/__init__.py +++ b/airflow/providers/jdbc/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.4.0" +__version__ = "4.5.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/jdbc/provider.yaml b/airflow/providers/jdbc/provider.yaml index c812420161b6..96a0e0f84ff2 100644 --- a/airflow/providers/jdbc/provider.yaml +++ b/airflow/providers/jdbc/provider.yaml @@ -22,9 +22,10 @@ description: | `Java Database Connectivity (JDBC) `__ state: ready -source-date-epoch: 1722664072 +source-date-epoch: 1723970315 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.5.0 - 4.4.0 - 4.3.1 - 4.3.0 diff --git a/airflow/providers/jenkins/CHANGELOG.rst b/airflow/providers/jenkins/CHANGELOG.rst index 194427b3521f..ec6eead85ccd 100644 --- a/airflow/providers/jenkins/CHANGELOG.rst +++ b/airflow/providers/jenkins/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +3.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.6.1 ..... diff --git a/airflow/providers/jenkins/__init__.py b/airflow/providers/jenkins/__init__.py index 124cb461b11b..2b9e0ed25808 100644 --- a/airflow/providers/jenkins/__init__.py +++ b/airflow/providers/jenkins/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.6.1" +__version__ = "3.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/jenkins/provider.yaml b/airflow/providers/jenkins/provider.yaml index 90a5c2bccad4..0995bd551231 100644 --- a/airflow/providers/jenkins/provider.yaml +++ b/airflow/providers/jenkins/provider.yaml @@ -22,9 +22,10 @@ description: | `Jenkins `__ state: ready -source-date-epoch: 1716288028 +source-date-epoch: 1723970321 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.7.0 - 3.6.1 - 3.6.0 - 3.5.1 diff --git a/airflow/providers/microsoft/azure/CHANGELOG.rst b/airflow/providers/microsoft/azure/CHANGELOG.rst index 72ae5f167a7f..371fce31895d 100644 --- a/airflow/providers/microsoft/azure/CHANGELOG.rst +++ b/airflow/providers/microsoft/azure/CHANGELOG.rst @@ -27,6 +27,33 @@ Changelog --------- +10.4.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Features +~~~~~~~~ + +* ``Microsoft Power BI operator to refresh the dataset (#40356)`` +* ``Export Azure Container Instance log messages to XCOM (#41142)`` + +Bug Fixes +~~~~~~~~~ + +* ``Fix mypy checks for new azure libraries (#41386)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 10.3.0 ...... diff --git a/airflow/providers/microsoft/azure/__init__.py b/airflow/providers/microsoft/azure/__init__.py index 94b8d7c5f7b4..4f35757124d0 100644 --- a/airflow/providers/microsoft/azure/__init__.py +++ b/airflow/providers/microsoft/azure/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "10.3.0" +__version__ = "10.4.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/microsoft/azure/hooks/msgraph.py b/airflow/providers/microsoft/azure/hooks/msgraph.py index 56abfa155da7..8410d8d7077c 100644 --- a/airflow/providers/microsoft/azure/hooks/msgraph.py +++ b/airflow/providers/microsoft/azure/hooks/msgraph.py @@ -110,12 +110,16 @@ def __init__( conn_id: str = default_conn_name, timeout: float | None = None, proxies: dict | None = None, + host: str = NationalClouds.Global.value, + scopes: list[str] | None = None, api_version: APIVersion | str | None = None, ): super().__init__() self.conn_id = conn_id self.timeout = timeout self.proxies = proxies + self.host = host + self.scopes = scopes or ["https://graph.microsoft.com/.default"] self._api_version = self.resolve_api_version_from_value(api_version) @property @@ -141,11 +145,10 @@ def get_api_version(self, config: dict) -> APIVersion: ) return self._api_version - @staticmethod - def get_host(connection: Connection) -> str: + def get_host(self, connection: Connection) -> str: if connection.schema and connection.host: return f"{connection.schema}://{connection.host}" - return NationalClouds.Global.value + return self.host @staticmethod def format_no_proxy_url(url: str) -> str: @@ -198,7 +201,7 @@ def get_conn(self) -> RequestAdapter: proxies = self.proxies or config.get("proxies", {}) msal_proxies = self.to_msal_proxies(authority=authority, proxies=proxies) httpx_proxies = self.to_httpx_proxies(proxies=proxies) - scopes = config.get("scopes", ["https://graph.microsoft.com/.default"]) + scopes = config.get("scopes", self.scopes) verify = config.get("verify", True) trust_env = config.get("trust_env", False) disable_instance_discovery = config.get("disable_instance_discovery", False) diff --git a/airflow/providers/microsoft/azure/hooks/powerbi.py b/airflow/providers/microsoft/azure/hooks/powerbi.py new file mode 100644 index 000000000000..04326f4fecee --- /dev/null +++ b/airflow/providers/microsoft/azure/hooks/powerbi.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 enum import Enum +from typing import TYPE_CHECKING, Any + +from airflow.exceptions import AirflowException +from airflow.providers.microsoft.azure.hooks.msgraph import KiotaRequestAdapterHook + +if TYPE_CHECKING: + from msgraph_core import APIVersion + + +class PowerBIDatasetRefreshFields(Enum): + """Power BI refresh dataset details.""" + + REQUEST_ID = "request_id" + STATUS = "status" + ERROR = "error" + + +class PowerBIDatasetRefreshStatus: + """Power BI refresh dataset statuses.""" + + IN_PROGRESS = "In Progress" + FAILED = "Failed" + COMPLETED = "Completed" + DISABLED = "Disabled" + + TERMINAL_STATUSES = {FAILED, COMPLETED} + + +class PowerBIDatasetRefreshException(AirflowException): + """An exception that indicates a dataset refresh failed to complete.""" + + +class PowerBIHook(KiotaRequestAdapterHook): + """ + A async hook to interact with Power BI. + + :param conn_id: The Power BI connection id. + """ + + conn_type: str = "powerbi" + conn_name_attr: str = "conn_id" + default_conn_name: str = "powerbi_default" + hook_name: str = "Power BI" + + def __init__( + self, + conn_id: str = default_conn_name, + proxies: dict | None = None, + timeout: float = 60 * 60 * 24 * 7, + api_version: APIVersion | str | None = None, + ): + super().__init__( + conn_id=conn_id, + proxies=proxies, + timeout=timeout, + host="https://api.powerbi.com", + scopes=["https://analysis.windows.net/powerbi/api/.default"], + api_version=api_version, + ) + + @classmethod + def get_connection_form_widgets(cls) -> dict[str, Any]: + """Return connection widgets to add to connection form.""" + from flask_appbuilder.fieldwidgets import BS3TextFieldWidget + from flask_babel import lazy_gettext + from wtforms import StringField + + return { + "tenant_id": StringField(lazy_gettext("Tenant ID"), widget=BS3TextFieldWidget()), + } + + @classmethod + def get_ui_field_behaviour(cls) -> dict[str, Any]: + """Return custom field behaviour.""" + return { + "hidden_fields": ["schema", "port", "host", "extra"], + "relabeling": { + "login": "Client ID", + "password": "Client Secret", + }, + } + + async def get_refresh_history( + self, + dataset_id: str, + group_id: str, + ) -> list[dict[str, str]]: + """ + Retrieve the refresh history of the specified dataset from the given group ID. + + :param dataset_id: The dataset ID. + :param group_id: The workspace ID. + + :return: Dictionary containing all the refresh histories of the dataset. + """ + try: + response = await self.run( + url="myorg/groups/{group_id}/datasets/{dataset_id}/refreshes", + path_parameters={ + "group_id": group_id, + "dataset_id": dataset_id, + }, + ) + + refresh_histories = response.get("value") + return [self.raw_to_refresh_details(refresh_history) for refresh_history in refresh_histories] + + except AirflowException: + raise PowerBIDatasetRefreshException("Failed to retrieve refresh history") + + @classmethod + def raw_to_refresh_details(cls, refresh_details: dict) -> dict[str, str]: + """ + Convert raw refresh details into a dictionary containing required fields. + + :param refresh_details: Raw object of refresh details. + """ + return { + PowerBIDatasetRefreshFields.REQUEST_ID.value: str(refresh_details.get("requestId")), + PowerBIDatasetRefreshFields.STATUS.value: ( + "In Progress" + if str(refresh_details.get("status")) == "Unknown" + else str(refresh_details.get("status")) + ), + PowerBIDatasetRefreshFields.ERROR.value: str(refresh_details.get("serviceExceptionJson")), + } + + async def get_refresh_details_by_refresh_id( + self, dataset_id: str, group_id: str, refresh_id: str + ) -> dict[str, str]: + """ + Get the refresh details of the given request Id. + + :param refresh_id: Request Id of the Dataset refresh. + """ + refresh_histories = await self.get_refresh_history(dataset_id=dataset_id, group_id=group_id) + + if len(refresh_histories) == 0: + raise PowerBIDatasetRefreshException( + f"Unable to fetch the details of dataset refresh with Request Id: {refresh_id}" + ) + + refresh_ids = [ + refresh_history.get(PowerBIDatasetRefreshFields.REQUEST_ID.value) + for refresh_history in refresh_histories + ] + + if refresh_id not in refresh_ids: + raise PowerBIDatasetRefreshException( + f"Unable to fetch the details of dataset refresh with Request Id: {refresh_id}" + ) + + refresh_details = refresh_histories[refresh_ids.index(refresh_id)] + + return refresh_details + + async def trigger_dataset_refresh(self, *, dataset_id: str, group_id: str) -> str: + """ + Triggers a refresh for the specified dataset from the given group id. + + :param dataset_id: The dataset id. + :param group_id: The workspace id. + + :return: Request id of the dataset refresh request. + """ + try: + response = await self.run( + url="myorg/groups/{group_id}/datasets/{dataset_id}/refreshes", + method="POST", + path_parameters={ + "group_id": group_id, + "dataset_id": dataset_id, + }, + ) + + request_id = response.get("requestid") + return request_id + except AirflowException: + raise PowerBIDatasetRefreshException("Failed to trigger dataset refresh.") + + async def cancel_dataset_refresh(self, dataset_id: str, group_id: str, dataset_refresh_id: str) -> None: + """ + Cancel the dataset refresh. + + :param dataset_id: The dataset Id. + :param group_id: The workspace Id. + :param dataset_refresh_id: The dataset refresh Id. + """ + await self.run( + url="myorg/groups/{group_id}/datasets/{dataset_id}/refreshes/{dataset_refresh_id}", + response_type=None, + path_parameters={ + "group_id": group_id, + "dataset_id": dataset_id, + "dataset_refresh_id": dataset_refresh_id, + }, + method="DELETE", + ) diff --git a/airflow/providers/microsoft/azure/operators/container_instances.py b/airflow/providers/microsoft/azure/operators/container_instances.py index 605c0e1608f2..b47dec1c703e 100644 --- a/airflow/providers/microsoft/azure/operators/container_instances.py +++ b/airflow/providers/microsoft/azure/operators/container_instances.py @@ -86,6 +86,12 @@ class AzureContainerInstancesOperator(BaseOperator): :param container_timeout: max time allowed for the execution of the container instance. :param tags: azure tags as dict of str:str + :param xcom_all: Control if logs are pushed to XCOM similarly to how DockerOperator does. + Possible values include: 'None', 'True', 'False'. Defaults to 'None', meaning no logs + are pushed to XCOM which is the historical behaviour. 'True' means push all logs to XCOM + which may run the risk of hitting XCOM size limits. 'False' means push only the last line + of the logs to XCOM. However, the logs are pushed into XCOM under "logs", not return_value + to avoid breaking the existing behaviour. :param os_type: The operating system type required by the containers in the container group. Possible values include: 'Windows', 'Linux' :param restart_policy: Restart policy for all containers within the container group. @@ -158,6 +164,7 @@ def __init__( remove_on_error: bool = True, fail_if_exists: bool = True, tags: dict[str, str] | None = None, + xcom_all: bool | None = None, os_type: str = "Linux", restart_policy: str = "Never", ip_address: IpAddress | None = None, @@ -187,6 +194,7 @@ def __init__( self.fail_if_exists = fail_if_exists self._ci_hook: Any = None self.tags = tags + self.xcom_all = xcom_all self.os_type = os_type if self.os_type not in ["Linux", "Windows"]: raise AirflowException( @@ -296,6 +304,16 @@ def execute(self, context: Context) -> int: self.log.info("Container group started %s/%s", self.resource_group, self.name) exit_code = self._monitor_logging(self.resource_group, self.name) + if self.xcom_all is not None: + logs = self._ci_hook.get_logs(self.resource_group, self.name) + if logs is None: + context["ti"].xcom_push(key="logs", value=[]) + else: + if self.xcom_all: + context["ti"].xcom_push(key="logs", value=logs) + else: + # slice off the last entry in the list logs and return it as a list + context["ti"].xcom_push(key="logs", value=logs[-1:]) self.log.info("Container had exit code: %s", exit_code) if exit_code != 0: diff --git a/airflow/providers/microsoft/azure/operators/powerbi.py b/airflow/providers/microsoft/azure/operators/powerbi.py new file mode 100644 index 000000000000..e54ad250bde7 --- /dev/null +++ b/airflow/providers/microsoft/azure/operators/powerbi.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 + +from typing import TYPE_CHECKING, Any, Sequence + +from airflow.exceptions import AirflowException +from airflow.models import BaseOperator, BaseOperatorLink +from airflow.providers.microsoft.azure.hooks.powerbi import ( + PowerBIHook, +) +from airflow.providers.microsoft.azure.triggers.powerbi import PowerBITrigger + +if TYPE_CHECKING: + from msgraph_core import APIVersion + + from airflow.models.taskinstancekey import TaskInstanceKey + from airflow.utils.context import Context + + +class PowerBILink(BaseOperatorLink): + """Construct a link to monitor a dataset in Power BI.""" + + name = "Monitor PowerBI Dataset" + + def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey): + url = ( + "https://app.powerbi.com" # type: ignore[attr-defined] + f"/groups/{operator.group_id}/datasets/{operator.dataset_id}" # type: ignore[attr-defined] + "/details?experience=power-bi" + ) + + return url + + +class PowerBIDatasetRefreshOperator(BaseOperator): + """ + Refreshes a Power BI dataset. + + :param dataset_id: The dataset id. + :param group_id: The workspace id. + :param conn_id: Airflow Connection ID that contains the connection information for the Power BI account used for authentication. + :param timeout: Time in seconds to wait for a dataset to reach a terminal status for asynchronous waits. Used only if ``wait_for_termination`` is True. + :param check_interval: Number of seconds to wait before rechecking the + refresh status. + """ + + template_fields: Sequence[str] = ( + "dataset_id", + "group_id", + ) + template_fields_renderers = {"parameters": "json"} + + operator_extra_links = (PowerBILink(),) + + def __init__( + self, + *, + dataset_id: str, + group_id: str, + conn_id: str = PowerBIHook.default_conn_name, + timeout: float = 60 * 60 * 24 * 7, + proxies: dict | None = None, + api_version: APIVersion | None = None, + check_interval: int = 60, + **kwargs, + ) -> None: + super().__init__(**kwargs) + self.hook = PowerBIHook(conn_id=conn_id, proxies=proxies, api_version=api_version, timeout=timeout) + self.dataset_id = dataset_id + self.group_id = group_id + self.wait_for_termination = True + self.conn_id = conn_id + self.timeout = timeout + self.check_interval = check_interval + + def execute(self, context: Context): + """Refresh the Power BI Dataset.""" + if self.wait_for_termination: + self.defer( + trigger=PowerBITrigger( + conn_id=self.conn_id, + group_id=self.group_id, + dataset_id=self.dataset_id, + timeout=self.timeout, + check_interval=self.check_interval, + wait_for_termination=self.wait_for_termination, + ), + method_name=self.execute_complete.__name__, + ) + + def execute_complete(self, context: Context, event: dict[str, str]) -> Any: + """ + Return immediately - callback for when the trigger fires. + + Relies on trigger to throw an exception, otherwise it assumes execution was successful. + """ + if event: + if event["status"] == "error": + raise AirflowException(event["message"]) + + self.xcom_push( + context=context, key="powerbi_dataset_refresh_Id", value=event["dataset_refresh_id"] + ) + self.xcom_push(context=context, key="powerbi_dataset_refresh_status", value=event["status"]) diff --git a/airflow/providers/microsoft/azure/provider.yaml b/airflow/providers/microsoft/azure/provider.yaml index 04e7311b44b0..08124a733f47 100644 --- a/airflow/providers/microsoft/azure/provider.yaml +++ b/airflow/providers/microsoft/azure/provider.yaml @@ -20,9 +20,10 @@ name: Microsoft Azure description: | `Microsoft Azure `__ state: ready -source-date-epoch: 1722664136 +source-date-epoch: 1723970344 # note that those versions are maintained by release manager - do not update them manually versions: + - 10.4.0 - 10.3.0 - 10.2.0 - 10.1.2 @@ -176,6 +177,9 @@ integrations: how-to-guide: - /docs/apache-airflow-providers-microsoft-azure/operators/msgraph.rst tags: [azure] + - integration-name: Microsoft Power BI + external-doc-url: https://learn.microsoft.com/en-us/rest/api/power-bi/ + tags: [azure] operators: - integration-name: Microsoft Azure Data Lake Storage @@ -208,6 +212,9 @@ operators: - integration-name: Microsoft Graph API python-modules: - airflow.providers.microsoft.azure.operators.msgraph + - integration-name: Microsoft Power BI + python-modules: + - airflow.providers.microsoft.azure.operators.powerbi sensors: - integration-name: Microsoft Azure Cosmos DB @@ -268,6 +275,9 @@ hooks: - integration-name: Microsoft Graph API python-modules: - airflow.providers.microsoft.azure.hooks.msgraph + - integration-name: Microsoft Power BI + python-modules: + - airflow.providers.microsoft.azure.hooks.powerbi triggers: - integration-name: Microsoft Azure Data Factory @@ -279,6 +289,9 @@ triggers: - integration-name: Microsoft Graph API python-modules: - airflow.providers.microsoft.azure.triggers.msgraph + - integration-name: Microsoft Power BI + python-modules: + - airflow.providers.microsoft.azure.triggers.powerbi transfers: - source-integration-name: Local @@ -334,6 +347,8 @@ connection-types: connection-type: azure_synapse - hook-class-name: airflow.providers.microsoft.azure.hooks.data_lake.AzureDataLakeStorageV2Hook connection-type: adls + - hook-class-name: airflow.providers.microsoft.azure.hooks.powerbi.PowerBIHook + connection-type: powerbi secrets-backends: - airflow.providers.microsoft.azure.secrets.key_vault.AzureKeyVaultBackend @@ -344,6 +359,7 @@ logging: extra-links: - airflow.providers.microsoft.azure.operators.data_factory.AzureDataFactoryPipelineRunLink - airflow.providers.microsoft.azure.operators.synapse.AzureSynapsePipelineRunLink + - airflow.providers.microsoft.azure.operators.powerbi.PowerBILink config: azure_remote_logging: diff --git a/airflow/providers/microsoft/azure/triggers/powerbi.py b/airflow/providers/microsoft/azure/triggers/powerbi.py new file mode 100644 index 000000000000..d25802b84fb7 --- /dev/null +++ b/airflow/providers/microsoft/azure/triggers/powerbi.py @@ -0,0 +1,181 @@ +# 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 +import time +from typing import TYPE_CHECKING, AsyncIterator + +from airflow.providers.microsoft.azure.hooks.powerbi import ( + PowerBIDatasetRefreshStatus, + PowerBIHook, +) +from airflow.triggers.base import BaseTrigger, TriggerEvent + +if TYPE_CHECKING: + from msgraph_core import APIVersion + + +class PowerBITrigger(BaseTrigger): + """ + Triggers when Power BI dataset refresh is completed. + + Wait for termination will always be True. + + :param conn_id: The connection Id to connect to PowerBI. + :param timeout: The HTTP timeout being used by the `KiotaRequestAdapter` (default is None). + When no timeout is specified or set to None then there is no HTTP timeout on each request. + :param proxies: A dict defining the HTTP proxies to be used (default is None). + :param api_version: The API version of the Microsoft Graph API to be used (default is v1). + You can pass an enum named APIVersion which has 2 possible members v1 and beta, + or you can pass a string as `v1.0` or `beta`. + :param dataset_id: The dataset Id to refresh. + :param group_id: The workspace Id where dataset is located. + :param end_time: Time in seconds when trigger should stop polling. + :param check_interval: Time in seconds to wait between each poll. + :param wait_for_termination: Wait for the dataset refresh to complete or fail. + """ + + def __init__( + self, + conn_id: str, + dataset_id: str, + group_id: str, + timeout: float = 60 * 60 * 24 * 7, + proxies: dict | None = None, + api_version: APIVersion | None = None, + check_interval: int = 60, + wait_for_termination: bool = True, + ): + super().__init__() + self.hook = PowerBIHook(conn_id=conn_id, proxies=proxies, api_version=api_version, timeout=timeout) + self.dataset_id = dataset_id + self.timeout = timeout + self.group_id = group_id + self.check_interval = check_interval + self.wait_for_termination = wait_for_termination + + def serialize(self): + """Serialize the trigger instance.""" + api_version = self.api_version.value if self.api_version else None + return ( + "airflow.providers.microsoft.azure.triggers.powerbi.PowerBITrigger", + { + "conn_id": self.conn_id, + "proxies": self.proxies, + "api_version": api_version, + "dataset_id": self.dataset_id, + "group_id": self.group_id, + "timeout": self.timeout, + "check_interval": self.check_interval, + "wait_for_termination": self.wait_for_termination, + }, + ) + + @property + def conn_id(self) -> str: + return self.hook.conn_id + + @property + def proxies(self) -> dict | None: + return self.hook.proxies + + @property + def api_version(self) -> APIVersion: + return self.hook.api_version + + async def run(self) -> AsyncIterator[TriggerEvent]: + """Make async connection to the PowerBI and polls for the dataset refresh status.""" + self.dataset_refresh_id = await self.hook.trigger_dataset_refresh( + dataset_id=self.dataset_id, + group_id=self.group_id, + ) + try: + dataset_refresh_status = None + start_time = time.monotonic() + while start_time + self.timeout > time.monotonic(): + refresh_details = await self.hook.get_refresh_details_by_refresh_id( + dataset_id=self.dataset_id, + group_id=self.group_id, + refresh_id=self.dataset_refresh_id, + ) + dataset_refresh_status = refresh_details.get("status") + + if dataset_refresh_status == PowerBIDatasetRefreshStatus.COMPLETED: + yield TriggerEvent( + { + "status": dataset_refresh_status, + "message": f"The dataset refresh {self.dataset_refresh_id} has {dataset_refresh_status}.", + "dataset_refresh_id": self.dataset_refresh_id, + } + ) + return + elif dataset_refresh_status == PowerBIDatasetRefreshStatus.FAILED: + yield TriggerEvent( + { + "status": dataset_refresh_status, + "message": f"The dataset refresh {self.dataset_refresh_id} has {dataset_refresh_status}.", + "dataset_refresh_id": self.dataset_refresh_id, + } + ) + return + + self.log.info( + "Sleeping for %s. The dataset refresh status is %s.", + self.check_interval, + dataset_refresh_status, + ) + await asyncio.sleep(self.check_interval) + + yield TriggerEvent( + { + "status": "error", + "message": f"Timeout occurred while waiting for dataset refresh to complete: The dataset refresh {self.dataset_refresh_id} has status {dataset_refresh_status}.", + "dataset_refresh_id": self.dataset_refresh_id, + } + ) + return + except Exception as error: + if self.dataset_refresh_id: + try: + self.log.info( + "Unexpected error %s caught. Canceling dataset refresh %s", + error, + self.dataset_refresh_id, + ) + await self.hook.cancel_dataset_refresh( + dataset_id=self.dataset_id, + group_id=self.group_id, + dataset_refresh_id=self.dataset_refresh_id, + ) + except Exception as e: + yield TriggerEvent( + { + "status": "error", + "message": f"An error occurred while canceling dataset: {e}", + "dataset_refresh_id": self.dataset_refresh_id, + } + ) + return + yield TriggerEvent( + { + "status": "error", + "message": f"An error occurred: {error}", + "dataset_refresh_id": self.dataset_refresh_id, + } + ) diff --git a/airflow/providers/microsoft/mssql/CHANGELOG.rst b/airflow/providers/microsoft/mssql/CHANGELOG.rst index 5e2084bd1a6e..f4e74a441dec 100644 --- a/airflow/providers/microsoft/mssql/CHANGELOG.rst +++ b/airflow/providers/microsoft/mssql/CHANGELOG.rst @@ -27,6 +27,27 @@ Changelog --------- +3.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Bug Fixes +~~~~~~~~~ + +* ``Fix missing methodtools as dependency to MsSqlHook (#41392)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Add comment on methodtools in mssql provider.yaml (#41402)`` + 3.8.0 ..... diff --git a/airflow/providers/microsoft/mssql/__init__.py b/airflow/providers/microsoft/mssql/__init__.py index ef80207c0485..e3fc1c80e078 100644 --- a/airflow/providers/microsoft/mssql/__init__.py +++ b/airflow/providers/microsoft/mssql/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.8.0" +__version__ = "3.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/microsoft/mssql/provider.yaml b/airflow/providers/microsoft/mssql/provider.yaml index 0ab0fc65f498..3ae91673b598 100644 --- a/airflow/providers/microsoft/mssql/provider.yaml +++ b/airflow/providers/microsoft/mssql/provider.yaml @@ -22,9 +22,10 @@ description: | `Microsoft SQL Server (MSSQL) `__ state: ready -source-date-epoch: 1722664157 +source-date-epoch: 1723970390 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.9.0 - 3.8.0 - 3.7.2 - 3.7.1 diff --git a/airflow/providers/microsoft/psrp/CHANGELOG.rst b/airflow/providers/microsoft/psrp/CHANGELOG.rst index e33e5a24d160..794ba1d6cb6a 100644 --- a/airflow/providers/microsoft/psrp/CHANGELOG.rst +++ b/airflow/providers/microsoft/psrp/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +2.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 2.7.1 ..... diff --git a/airflow/providers/microsoft/psrp/__init__.py b/airflow/providers/microsoft/psrp/__init__.py index babae20cedaa..2b06cb0e15bc 100644 --- a/airflow/providers/microsoft/psrp/__init__.py +++ b/airflow/providers/microsoft/psrp/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.7.1" +__version__ = "2.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/microsoft/psrp/provider.yaml b/airflow/providers/microsoft/psrp/provider.yaml index bed15c84d347..ae29eb92fa20 100644 --- a/airflow/providers/microsoft/psrp/provider.yaml +++ b/airflow/providers/microsoft/psrp/provider.yaml @@ -24,9 +24,10 @@ description: | `__. state: ready -source-date-epoch: 1716288139 +source-date-epoch: 1723970427 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.8.0 - 2.7.1 - 2.7.0 - 2.6.1 diff --git a/airflow/providers/microsoft/winrm/CHANGELOG.rst b/airflow/providers/microsoft/winrm/CHANGELOG.rst index cc68260f5df5..e52d385abafc 100644 --- a/airflow/providers/microsoft/winrm/CHANGELOG.rst +++ b/airflow/providers/microsoft/winrm/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/microsoft/winrm/__init__.py b/airflow/providers/microsoft/winrm/__init__.py index fe75ee6648f6..e37c38707694 100644 --- a/airflow/providers/microsoft/winrm/__init__.py +++ b/airflow/providers/microsoft/winrm/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/microsoft/winrm/provider.yaml b/airflow/providers/microsoft/winrm/provider.yaml index 7a409fe60209..155bceba41a4 100644 --- a/airflow/providers/microsoft/winrm/provider.yaml +++ b/airflow/providers/microsoft/winrm/provider.yaml @@ -22,9 +22,10 @@ description: | `Windows Remote Management (WinRM) `__ state: ready -source-date-epoch: 1716288147 +source-date-epoch: 1723970432 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/mongo/CHANGELOG.rst b/airflow/providers/mongo/CHANGELOG.rst index 80afb4115805..8ef97e096a9d 100644 --- a/airflow/providers/mongo/CHANGELOG.rst +++ b/airflow/providers/mongo/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +4.2.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.1.2 ..... diff --git a/airflow/providers/mongo/__init__.py b/airflow/providers/mongo/__init__.py index 4d503191155d..f3c23d039a44 100644 --- a/airflow/providers/mongo/__init__.py +++ b/airflow/providers/mongo/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.1.2" +__version__ = "4.2.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/mongo/provider.yaml b/airflow/providers/mongo/provider.yaml index 2014f459ca57..82f6c325633d 100644 --- a/airflow/providers/mongo/provider.yaml +++ b/airflow/providers/mongo/provider.yaml @@ -22,9 +22,10 @@ description: | `MongoDB `__ state: ready -source-date-epoch: 1718605092 +source-date-epoch: 1723970436 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.2.0 - 4.1.2 - 4.1.1 - 4.1.0 diff --git a/airflow/providers/mysql/CHANGELOG.rst b/airflow/providers/mysql/CHANGELOG.rst index 4a94411cb4e6..50b633f1a331 100644 --- a/airflow/providers/mysql/CHANGELOG.rst +++ b/airflow/providers/mysql/CHANGELOG.rst @@ -26,6 +26,22 @@ Changelog --------- +5.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.6.3 ..... diff --git a/airflow/providers/mysql/__init__.py b/airflow/providers/mysql/__init__.py index e33ab923a2cf..90db473ccb50 100644 --- a/airflow/providers/mysql/__init__.py +++ b/airflow/providers/mysql/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.6.3" +__version__ = "5.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/mysql/provider.yaml b/airflow/providers/mysql/provider.yaml index 40199b6ec8c4..309b4dc727fc 100644 --- a/airflow/providers/mysql/provider.yaml +++ b/airflow/providers/mysql/provider.yaml @@ -22,9 +22,10 @@ description: | `MySQL `__ state: ready -source-date-epoch: 1722664188 +source-date-epoch: 1723970439 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.7.0 - 5.6.3 - 5.6.2 - 5.6.1 diff --git a/airflow/providers/neo4j/CHANGELOG.rst b/airflow/providers/neo4j/CHANGELOG.rst index 91f35c53933e..7b2df96f7553 100644 --- a/airflow/providers/neo4j/CHANGELOG.rst +++ b/airflow/providers/neo4j/CHANGELOG.rst @@ -29,6 +29,22 @@ Changelog --------- +3.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.6.1 ..... diff --git a/airflow/providers/neo4j/__init__.py b/airflow/providers/neo4j/__init__.py index ac5c55cef8a8..99d14644d4a5 100644 --- a/airflow/providers/neo4j/__init__.py +++ b/airflow/providers/neo4j/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.6.1" +__version__ = "3.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/neo4j/provider.yaml b/airflow/providers/neo4j/provider.yaml index 6efd96efadf7..990bb45b514a 100644 --- a/airflow/providers/neo4j/provider.yaml +++ b/airflow/providers/neo4j/provider.yaml @@ -22,9 +22,10 @@ description: | `Neo4j `__ state: ready -source-date-epoch: 1716288178 +source-date-epoch: 1723970444 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.7.0 - 3.6.1 - 3.6.0 - 3.5.0 diff --git a/airflow/providers/odbc/CHANGELOG.rst b/airflow/providers/odbc/CHANGELOG.rst index d6c8e881e91b..d026a0d6ee7c 100644 --- a/airflow/providers/odbc/CHANGELOG.rst +++ b/airflow/providers/odbc/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +4.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.6.3 ..... diff --git a/airflow/providers/odbc/__init__.py b/airflow/providers/odbc/__init__.py index 493a2469a493..571b18d698b1 100644 --- a/airflow/providers/odbc/__init__.py +++ b/airflow/providers/odbc/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.6.3" +__version__ = "4.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/odbc/provider.yaml b/airflow/providers/odbc/provider.yaml index 655a8fba8902..2f3a0b418f84 100644 --- a/airflow/providers/odbc/provider.yaml +++ b/airflow/providers/odbc/provider.yaml @@ -22,9 +22,10 @@ description: | `ODBC `__ state: ready -source-date-epoch: 1722664215 +source-date-epoch: 1723970448 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.7.0 - 4.6.3 - 4.6.2 - 4.6.1 diff --git a/airflow/providers/openai/CHANGELOG.rst b/airflow/providers/openai/CHANGELOG.rst index b2515245591b..1a01a27b8b6d 100644 --- a/airflow/providers/openai/CHANGELOG.rst +++ b/airflow/providers/openai/CHANGELOG.rst @@ -20,6 +20,22 @@ Changelog --------- +1.3.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.2.2 ..... diff --git a/airflow/providers/openai/__init__.py b/airflow/providers/openai/__init__.py index 2eb3378ab4fe..a28803407866 100644 --- a/airflow/providers/openai/__init__.py +++ b/airflow/providers/openai/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.2.2" +__version__ = "1.3.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/openai/provider.yaml b/airflow/providers/openai/provider.yaml index 33ec615146db..a78338f7ce61 100644 --- a/airflow/providers/openai/provider.yaml +++ b/airflow/providers/openai/provider.yaml @@ -24,10 +24,11 @@ description: | `OpenAI `__ state: ready -source-date-epoch: 1718605162 +source-date-epoch: 1723970452 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.3.0 - 1.2.2 - 1.2.1 - 1.2.0 diff --git a/airflow/providers/openfaas/CHANGELOG.rst b/airflow/providers/openfaas/CHANGELOG.rst index c4db1bea6a47..ff59dc1531e3 100644 --- a/airflow/providers/openfaas/CHANGELOG.rst +++ b/airflow/providers/openfaas/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/openfaas/__init__.py b/airflow/providers/openfaas/__init__.py index 467a5671d767..23c23fb4876f 100644 --- a/airflow/providers/openfaas/__init__.py +++ b/airflow/providers/openfaas/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/openfaas/provider.yaml b/airflow/providers/openfaas/provider.yaml index dcf9d2246ca0..3c647be58aae 100644 --- a/airflow/providers/openfaas/provider.yaml +++ b/airflow/providers/openfaas/provider.yaml @@ -22,9 +22,10 @@ description: | `OpenFaaS `__ state: ready -source-date-epoch: 1716288185 +source-date-epoch: 1723970455 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/openlineage/CHANGELOG.rst b/airflow/providers/openlineage/CHANGELOG.rst index 6eaf9771f2b2..2ddd91423be5 100644 --- a/airflow/providers/openlineage/CHANGELOG.rst +++ b/airflow/providers/openlineage/CHANGELOG.rst @@ -26,6 +26,35 @@ Changelog --------- +1.11.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Features +~~~~~~~~ + +* ``feat: add debug facet to all OpenLineage events (#41217)`` +* ``feat: add fileloc to DAG info in AirflowRunFacet (#41311)`` +* ``feat: remove openlineage client deprecated from_environment() method (#41310)`` + +Bug Fixes +~~~~~~~~~ + +* ``fix: get task dependencies without serializing task tree to string (#41494)`` +* ``fix: return empty data instead of None when OpenLineage on_start method is missing (#41268)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``chore: remove openlineage deprecation warnings (#41284)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.10.0 ...... diff --git a/airflow/providers/openlineage/__init__.py b/airflow/providers/openlineage/__init__.py index 3c02e1e1d136..cb4452cc457d 100644 --- a/airflow/providers/openlineage/__init__.py +++ b/airflow/providers/openlineage/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.10.0" +__version__ = "1.11.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/openlineage/provider.yaml b/airflow/providers/openlineage/provider.yaml index f38b49035d83..82fff7b8837f 100644 --- a/airflow/providers/openlineage/provider.yaml +++ b/airflow/providers/openlineage/provider.yaml @@ -22,9 +22,10 @@ description: | `OpenLineage `__ state: ready -source-date-epoch: 1722664661 +source-date-epoch: 1723970474 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.11.0 - 1.10.0 - 1.9.1 - 1.9.0 diff --git a/airflow/providers/openlineage/utils/utils.py b/airflow/providers/openlineage/utils/utils.py index dd070d6fa501..f4b5c0223417 100644 --- a/airflow/providers/openlineage/utils/utils.py +++ b/airflow/providers/openlineage/utils/utils.py @@ -20,11 +20,9 @@ import datetime import json import logging -import re -from contextlib import redirect_stdout, suppress +from contextlib import suppress from functools import wraps from importlib import metadata -from io import StringIO from typing import TYPE_CHECKING, Any, Callable, Iterable import attrs @@ -35,7 +33,7 @@ from airflow import __version__ as AIRFLOW_VERSION from airflow.datasets import Dataset from airflow.exceptions import AirflowProviderDeprecationWarning # TODO: move this maybe to Airflow's logic? -from airflow.models import DAG, BaseOperator, MappedOperator +from airflow.models import DAG, BaseOperator, MappedOperator, Operator from airflow.providers.openlineage import conf from airflow.providers.openlineage.plugins.facets import ( AirflowDagRunFacet, @@ -441,16 +439,6 @@ def get_airflow_state_run_facet(dag_run: DagRun) -> dict[str, AirflowStateRunFac } -def _safe_get_dag_tree_view(dag: DAG) -> list[str]: - # get_tree_view() has been added in Airflow 2.8.2 - if hasattr(dag, "get_tree_view"): - return dag.get_tree_view().splitlines() - - with redirect_stdout(StringIO()) as stdout: - dag.tree_view() - return stdout.getvalue().splitlines() - - def _get_parsed_dag_tree(dag: DAG) -> dict: """ Get DAG's tasks hierarchy representation. @@ -476,37 +464,15 @@ def _get_parsed_dag_tree(dag: DAG) -> dict: "task_6": {} } """ - lines = _safe_get_dag_tree_view(dag) - task_dict: dict[str, dict] = {} - parent_map: dict[int, tuple[str, dict]] = {} - - for line in lines: - stripped_line = line.strip() - if not stripped_line: - continue - - # Determine the level by counting the leading spaces, assuming 4 spaces per level - # as defined in airflow.models.dag.DAG._generate_tree_view() - level = (len(line) - len(stripped_line)) // 4 - # airflow.models.baseoperator.BaseOperator.__repr__ or - # airflow.models.mappedoperator.MappedOperator.__repr__ is used in DAG tree - # or - match = re.match(r"^<(?:Task|Mapped)\(.+\): (.+)>$", stripped_line) - if not match: - return {} - current_task_id = match[1] - - if level == 0: # It's a root task - task_dict[current_task_id] = {} - parent_map[level] = (current_task_id, task_dict[current_task_id]) - else: - # Find the immediate parent task - parent_task, parent_dict = parent_map[(level - 1)] - # Create new dict for the current task - parent_dict[current_task_id] = {} - # Update this task in the parent map - parent_map[level] = (current_task_id, parent_dict[current_task_id]) + def get_downstream(task: Operator, current_dict: dict): + current_dict[task.task_id] = {} + for tmp_task in sorted(task.downstream_list, key=lambda x: x.task_id): + get_downstream(tmp_task, current_dict[task.task_id]) + + task_dict: dict = {} + for t in sorted(dag.roots, key=lambda x: x.task_id): + get_downstream(t, task_dict) return task_dict diff --git a/airflow/providers/opensearch/CHANGELOG.rst b/airflow/providers/opensearch/CHANGELOG.rst index 108d5d7435cd..4bc3235a79cf 100644 --- a/airflow/providers/opensearch/CHANGELOG.rst +++ b/airflow/providers/opensearch/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +1.4.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.3.0 ..... diff --git a/airflow/providers/opensearch/__init__.py b/airflow/providers/opensearch/__init__.py index f174a52fe3c5..b98bfd8b5a51 100644 --- a/airflow/providers/opensearch/__init__.py +++ b/airflow/providers/opensearch/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.3.0" +__version__ = "1.4.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/opensearch/provider.yaml b/airflow/providers/opensearch/provider.yaml index e17b7aa3f2a3..3902727c8a62 100644 --- a/airflow/providers/opensearch/provider.yaml +++ b/airflow/providers/opensearch/provider.yaml @@ -22,9 +22,10 @@ description: | `OpenSearch `__ state: ready -source-date-epoch: 1718605236 +source-date-epoch: 1723970482 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.4.0 - 1.3.0 - 1.2.1 - 1.2.0 diff --git a/airflow/providers/opsgenie/CHANGELOG.rst b/airflow/providers/opsgenie/CHANGELOG.rst index 4559c569574b..28144270e9fd 100644 --- a/airflow/providers/opsgenie/CHANGELOG.rst +++ b/airflow/providers/opsgenie/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +5.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.6.1 ..... diff --git a/airflow/providers/opsgenie/__init__.py b/airflow/providers/opsgenie/__init__.py index c4587da15525..32fd3e62d3c9 100644 --- a/airflow/providers/opsgenie/__init__.py +++ b/airflow/providers/opsgenie/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.6.1" +__version__ = "5.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/opsgenie/provider.yaml b/airflow/providers/opsgenie/provider.yaml index cded1a189fd2..0e6fdf9f0413 100644 --- a/airflow/providers/opsgenie/provider.yaml +++ b/airflow/providers/opsgenie/provider.yaml @@ -22,9 +22,10 @@ description: | `Opsgenie `__ state: ready -source-date-epoch: 1716288876 +source-date-epoch: 1723970486 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.7.0 - 5.6.1 - 5.6.0 - 5.5.0 diff --git a/airflow/providers/oracle/CHANGELOG.rst b/airflow/providers/oracle/CHANGELOG.rst index 2d510d3a4c76..2a4bba934c0b 100644 --- a/airflow/providers/oracle/CHANGELOG.rst +++ b/airflow/providers/oracle/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.11.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.10.3 ...... diff --git a/airflow/providers/oracle/__init__.py b/airflow/providers/oracle/__init__.py index 3a21260e2287..8bb5a0ae6677 100644 --- a/airflow/providers/oracle/__init__.py +++ b/airflow/providers/oracle/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.10.3" +__version__ = "3.11.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/oracle/provider.yaml b/airflow/providers/oracle/provider.yaml index c08b55a675c4..87b13c3a8a64 100644 --- a/airflow/providers/oracle/provider.yaml +++ b/airflow/providers/oracle/provider.yaml @@ -22,9 +22,10 @@ description: | `Oracle `__ state: ready -source-date-epoch: 1720423764 +source-date-epoch: 1723970489 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.11.0 - 3.10.3 - 3.10.2 - 3.10.1 diff --git a/airflow/providers/pagerduty/CHANGELOG.rst b/airflow/providers/pagerduty/CHANGELOG.rst index 7829841d891b..a55539e65e63 100644 --- a/airflow/providers/pagerduty/CHANGELOG.rst +++ b/airflow/providers/pagerduty/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.2 ..... diff --git a/airflow/providers/pagerduty/__init__.py b/airflow/providers/pagerduty/__init__.py index 8b2d439f6797..6c24ad31e9b0 100644 --- a/airflow/providers/pagerduty/__init__.py +++ b/airflow/providers/pagerduty/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.2" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/pagerduty/provider.yaml b/airflow/providers/pagerduty/provider.yaml index 077ecb728c64..0e4a0c9c2a1e 100644 --- a/airflow/providers/pagerduty/provider.yaml +++ b/airflow/providers/pagerduty/provider.yaml @@ -22,9 +22,10 @@ description: | `Pagerduty `__ state: ready -source-date-epoch: 1718605271 +source-date-epoch: 1723970493 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.2 - 3.7.1 - 3.7.0 diff --git a/airflow/providers/papermill/CHANGELOG.rst b/airflow/providers/papermill/CHANGELOG.rst index b63af0a016f8..bb833b953bc4 100644 --- a/airflow/providers/papermill/CHANGELOG.rst +++ b/airflow/providers/papermill/CHANGELOG.rst @@ -27,6 +27,23 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` +* ``restore python 3.12 support for papermill (#41548)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.2 ..... diff --git a/airflow/providers/papermill/__init__.py b/airflow/providers/papermill/__init__.py index 0ad957507d57..9309b22636fe 100644 --- a/airflow/providers/papermill/__init__.py +++ b/airflow/providers/papermill/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.2" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/papermill/provider.yaml b/airflow/providers/papermill/provider.yaml index 9e074d846576..afd273a69e1b 100644 --- a/airflow/providers/papermill/provider.yaml +++ b/airflow/providers/papermill/provider.yaml @@ -21,15 +21,11 @@ name: Papermill description: | `Papermill `__ -# Papermill is technically compliant with 3.12, but it's 2.5.0 version that is compliant, requires pinned -# version of aiohttp which conflicts with other providers. The fix for that is implemented extra-links: -# https://github.com/nteract/papermill/pull/771 and waits for new Papermill release -excluded-python-versions: ['3.12'] - state: ready -source-date-epoch: 1718605283 +source-date-epoch: 1723970506 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.2 - 3.7.1 - 3.7.0 @@ -56,13 +52,12 @@ versions: dependencies: - apache-airflow>=2.8.0 - - papermill[all]>=2.4.0 + - papermill[all]>=2.6.0 - scrapbook[all] - ipykernel - pandas>=2.1.2,<2.2;python_version>="3.9" - pandas>=1.5.3,<2.2;python_version<"3.9" - integrations: - integration-name: Papermill external-doc-url: https://github.com/nteract/papermill diff --git a/airflow/providers/pgvector/CHANGELOG.rst b/airflow/providers/pgvector/CHANGELOG.rst index 7cb2cf440799..dcbe59ae9063 100644 --- a/airflow/providers/pgvector/CHANGELOG.rst +++ b/airflow/providers/pgvector/CHANGELOG.rst @@ -20,6 +20,22 @@ Changelog --------- +1.3.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.2.2 ..... diff --git a/airflow/providers/pgvector/__init__.py b/airflow/providers/pgvector/__init__.py index 236f24dd2226..fbfb782e94e1 100644 --- a/airflow/providers/pgvector/__init__.py +++ b/airflow/providers/pgvector/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.2.2" +__version__ = "1.3.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/pgvector/provider.yaml b/airflow/providers/pgvector/provider.yaml index 2139f987cd5b..eb992edf078b 100644 --- a/airflow/providers/pgvector/provider.yaml +++ b/airflow/providers/pgvector/provider.yaml @@ -24,10 +24,11 @@ description: | `pgvector `__ state: ready -source-date-epoch: 1722664705 +source-date-epoch: 1723970510 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.3.0 - 1.2.2 - 1.2.1 - 1.2.0 diff --git a/airflow/providers/pinecone/CHANGELOG.rst b/airflow/providers/pinecone/CHANGELOG.rst index 86cc9b614bb5..51e9ea1eaaae 100644 --- a/airflow/providers/pinecone/CHANGELOG.rst +++ b/airflow/providers/pinecone/CHANGELOG.rst @@ -20,6 +20,22 @@ Changelog --------- +2.1.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.0.1 ..... diff --git a/airflow/providers/pinecone/__init__.py b/airflow/providers/pinecone/__init__.py index a2ce126f881d..25b6883af301 100644 --- a/airflow/providers/pinecone/__init__.py +++ b/airflow/providers/pinecone/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.0.1" +__version__ = "2.1.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/pinecone/provider.yaml b/airflow/providers/pinecone/provider.yaml index 537eea9f1ea4..630e16724205 100644 --- a/airflow/providers/pinecone/provider.yaml +++ b/airflow/providers/pinecone/provider.yaml @@ -24,10 +24,11 @@ description: | `Pinecone `__ state: ready -source-date-epoch: 1718605302 +source-date-epoch: 1723970514 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.1.0 - 2.0.1 - 2.0.0 - 1.1.2 diff --git a/airflow/providers/postgres/CHANGELOG.rst b/airflow/providers/postgres/CHANGELOG.rst index 01a960c4c4ef..01e350f9499d 100644 --- a/airflow/providers/postgres/CHANGELOG.rst +++ b/airflow/providers/postgres/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +5.12.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.11.3 ...... diff --git a/airflow/providers/postgres/__init__.py b/airflow/providers/postgres/__init__.py index df53fd617b29..bf2a05b4fd81 100644 --- a/airflow/providers/postgres/__init__.py +++ b/airflow/providers/postgres/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.11.3" +__version__ = "5.12.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/postgres/provider.yaml b/airflow/providers/postgres/provider.yaml index 1c28bd133e72..3de77272484f 100644 --- a/airflow/providers/postgres/provider.yaml +++ b/airflow/providers/postgres/provider.yaml @@ -22,9 +22,10 @@ description: | `PostgreSQL `__ state: ready -source-date-epoch: 1722664743 +source-date-epoch: 1723970520 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.12.0 - 5.11.3 - 5.11.2 - 5.11.1 diff --git a/airflow/providers/presto/CHANGELOG.rst b/airflow/providers/presto/CHANGELOG.rst index ace20a6e8f1a..9396141e83f8 100644 --- a/airflow/providers/presto/CHANGELOG.rst +++ b/airflow/providers/presto/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +5.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.5.2 ..... diff --git a/airflow/providers/presto/__init__.py b/airflow/providers/presto/__init__.py index 28d4cbcbe601..c293f325f4da 100644 --- a/airflow/providers/presto/__init__.py +++ b/airflow/providers/presto/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.5.2" +__version__ = "5.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/presto/provider.yaml b/airflow/providers/presto/provider.yaml index 6704910c6618..0cc9bc5a44d0 100644 --- a/airflow/providers/presto/provider.yaml +++ b/airflow/providers/presto/provider.yaml @@ -22,9 +22,10 @@ description: | `Presto `__ state: ready -source-date-epoch: 1718605325 +source-date-epoch: 1723970527 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.6.0 - 5.5.2 - 5.5.1 - 5.5.0 diff --git a/airflow/providers/qdrant/CHANGELOG.rst b/airflow/providers/qdrant/CHANGELOG.rst index 0d7898aae3f9..e5a28bb54d29 100644 --- a/airflow/providers/qdrant/CHANGELOG.rst +++ b/airflow/providers/qdrant/CHANGELOG.rst @@ -20,6 +20,22 @@ Changelog --------- +1.2.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.1.2 ..... diff --git a/airflow/providers/qdrant/__init__.py b/airflow/providers/qdrant/__init__.py index dd4ed48500e8..ebab80607746 100644 --- a/airflow/providers/qdrant/__init__.py +++ b/airflow/providers/qdrant/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.1.2" +__version__ = "1.2.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/qdrant/provider.yaml b/airflow/providers/qdrant/provider.yaml index 637789847a3e..3f86123faf67 100644 --- a/airflow/providers/qdrant/provider.yaml +++ b/airflow/providers/qdrant/provider.yaml @@ -24,10 +24,11 @@ description: | `Qdrant `__ state: ready -source-date-epoch: 1722665002 +source-date-epoch: 1723970532 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.2.0 - 1.1.2 - 1.1.1 - 1.1.0 diff --git a/airflow/providers/redis/CHANGELOG.rst b/airflow/providers/redis/CHANGELOG.rst index 30a0f20bd1bf..2f53fb18b0c3 100644 --- a/airflow/providers/redis/CHANGELOG.rst +++ b/airflow/providers/redis/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.7.1 ..... diff --git a/airflow/providers/redis/__init__.py b/airflow/providers/redis/__init__.py index 4a1c9cc179c7..9d71e73648c3 100644 --- a/airflow/providers/redis/__init__.py +++ b/airflow/providers/redis/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.7.1" +__version__ = "3.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/redis/provider.yaml b/airflow/providers/redis/provider.yaml index f6a001005fda..63b019f274c1 100644 --- a/airflow/providers/redis/provider.yaml +++ b/airflow/providers/redis/provider.yaml @@ -22,9 +22,10 @@ description: | `Redis `__ state: ready -source-date-epoch: 1716288988 +source-date-epoch: 1723970535 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.8.0 - 3.7.1 - 3.7.0 - 3.6.1 diff --git a/airflow/providers/salesforce/CHANGELOG.rst b/airflow/providers/salesforce/CHANGELOG.rst index f0f786bc2da0..6bac18774923 100644 --- a/airflow/providers/salesforce/CHANGELOG.rst +++ b/airflow/providers/salesforce/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +5.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.7.2 ..... diff --git a/airflow/providers/salesforce/__init__.py b/airflow/providers/salesforce/__init__.py index 7134a11051c3..0cf2ba8f97a1 100644 --- a/airflow/providers/salesforce/__init__.py +++ b/airflow/providers/salesforce/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.7.2" +__version__ = "5.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/salesforce/provider.yaml b/airflow/providers/salesforce/provider.yaml index a544e33a52ec..dd5147d2ffee 100644 --- a/airflow/providers/salesforce/provider.yaml +++ b/airflow/providers/salesforce/provider.yaml @@ -22,9 +22,10 @@ description: | `Salesforce `__ state: ready -source-date-epoch: 1718605344 +source-date-epoch: 1723970538 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.8.0 - 5.7.2 - 5.7.1 - 5.7.0 diff --git a/airflow/providers/samba/CHANGELOG.rst b/airflow/providers/samba/CHANGELOG.rst index 9ee1d0ede898..18613651f763 100644 --- a/airflow/providers/samba/CHANGELOG.rst +++ b/airflow/providers/samba/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +4.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 4.7.1 ..... diff --git a/airflow/providers/samba/__init__.py b/airflow/providers/samba/__init__.py index 13c375f8f959..4f4d0ee2915d 100644 --- a/airflow/providers/samba/__init__.py +++ b/airflow/providers/samba/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.7.1" +__version__ = "4.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/samba/provider.yaml b/airflow/providers/samba/provider.yaml index d11a257670f6..4e681228bffe 100644 --- a/airflow/providers/samba/provider.yaml +++ b/airflow/providers/samba/provider.yaml @@ -22,9 +22,10 @@ description: | `Samba `__ state: ready -source-date-epoch: 1716288992 +source-date-epoch: 1723970544 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.8.0 - 4.7.1 - 4.7.0 - 4.6.0 diff --git a/airflow/providers/segment/CHANGELOG.rst b/airflow/providers/segment/CHANGELOG.rst index 02dabbf345a8..54b64a813361 100644 --- a/airflow/providers/segment/CHANGELOG.rst +++ b/airflow/providers/segment/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/segment/__init__.py b/airflow/providers/segment/__init__.py index 4f46f708d383..a51e77748eec 100644 --- a/airflow/providers/segment/__init__.py +++ b/airflow/providers/segment/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/segment/provider.yaml b/airflow/providers/segment/provider.yaml index 33336478daf3..30c5e75beb64 100644 --- a/airflow/providers/segment/provider.yaml +++ b/airflow/providers/segment/provider.yaml @@ -22,9 +22,10 @@ description: | `Segment `__ state: ready -source-date-epoch: 1716288995 +source-date-epoch: 1723970548 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/sendgrid/CHANGELOG.rst b/airflow/providers/sendgrid/CHANGELOG.rst index a631a49faa4c..5928b42144aa 100644 --- a/airflow/providers/sendgrid/CHANGELOG.rst +++ b/airflow/providers/sendgrid/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/sendgrid/__init__.py b/airflow/providers/sendgrid/__init__.py index dcf0d4072144..c6342ee4144d 100644 --- a/airflow/providers/sendgrid/__init__.py +++ b/airflow/providers/sendgrid/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/sendgrid/provider.yaml b/airflow/providers/sendgrid/provider.yaml index 911336ee2ea6..7a64ef4c2de1 100644 --- a/airflow/providers/sendgrid/provider.yaml +++ b/airflow/providers/sendgrid/provider.yaml @@ -26,9 +26,10 @@ dependencies: - sendgrid>=6.0.0 state: ready -source-date-epoch: 1716288997 +source-date-epoch: 1723970552 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/sftp/CHANGELOG.rst b/airflow/providers/sftp/CHANGELOG.rst index c9d2eeac3a07..aa65fd25d0cf 100644 --- a/airflow/providers/sftp/CHANGELOG.rst +++ b/airflow/providers/sftp/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +4.11.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.10.3 ...... diff --git a/airflow/providers/sftp/__init__.py b/airflow/providers/sftp/__init__.py index 2a542ffd7380..ea1424754e70 100644 --- a/airflow/providers/sftp/__init__.py +++ b/airflow/providers/sftp/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.10.3" +__version__ = "4.11.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/sftp/provider.yaml b/airflow/providers/sftp/provider.yaml index 4d9480537b20..048ddf1ad6b9 100644 --- a/airflow/providers/sftp/provider.yaml +++ b/airflow/providers/sftp/provider.yaml @@ -22,9 +22,10 @@ description: | `SSH File Transfer Protocol (SFTP) `__ state: ready -source-date-epoch: 1722664831 +source-date-epoch: 1723970557 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.11.0 - 4.10.3 - 4.10.2 - 4.10.1 diff --git a/airflow/providers/singularity/CHANGELOG.rst b/airflow/providers/singularity/CHANGELOG.rst index d4b01b287650..2d79b236f3ca 100644 --- a/airflow/providers/singularity/CHANGELOG.rst +++ b/airflow/providers/singularity/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.5.1 ..... diff --git a/airflow/providers/singularity/__init__.py b/airflow/providers/singularity/__init__.py index 85e86d53c59e..366f0cc44518 100644 --- a/airflow/providers/singularity/__init__.py +++ b/airflow/providers/singularity/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.5.1" +__version__ = "3.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/singularity/provider.yaml b/airflow/providers/singularity/provider.yaml index 55147e7d537c..3d36879f50ef 100644 --- a/airflow/providers/singularity/provider.yaml +++ b/airflow/providers/singularity/provider.yaml @@ -22,9 +22,10 @@ description: | `Singularity `__ state: ready -source-date-epoch: 1716289081 +source-date-epoch: 1723970561 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.6.0 - 3.5.1 - 3.5.0 - 3.4.0 diff --git a/airflow/providers/slack/CHANGELOG.rst b/airflow/providers/slack/CHANGELOG.rst index c2f6c60ab152..9aff8bc725e7 100644 --- a/airflow/providers/slack/CHANGELOG.rst +++ b/airflow/providers/slack/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +8.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 8.8.0 ..... diff --git a/airflow/providers/slack/__init__.py b/airflow/providers/slack/__init__.py index 2313cf342915..019f78256f8a 100644 --- a/airflow/providers/slack/__init__.py +++ b/airflow/providers/slack/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "8.8.0" +__version__ = "8.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/slack/provider.yaml b/airflow/providers/slack/provider.yaml index 510045a8e66f..2a1cda7b5dc7 100644 --- a/airflow/providers/slack/provider.yaml +++ b/airflow/providers/slack/provider.yaml @@ -25,9 +25,10 @@ description: | - `Slack Incoming Webhook `__ state: ready -source-date-epoch: 1722664847 +source-date-epoch: 1723970565 # note that those versions are maintained by release manager - do not update them manually versions: + - 8.9.0 - 8.8.0 - 8.7.1 - 8.7.0 diff --git a/airflow/providers/smtp/CHANGELOG.rst b/airflow/providers/smtp/CHANGELOG.rst index 68346de22ef6..4f508ea07919 100644 --- a/airflow/providers/smtp/CHANGELOG.rst +++ b/airflow/providers/smtp/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +1.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 1.7.1 ..... diff --git a/airflow/providers/smtp/__init__.py b/airflow/providers/smtp/__init__.py index 4368754d17cd..6d557201505b 100644 --- a/airflow/providers/smtp/__init__.py +++ b/airflow/providers/smtp/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.7.1" +__version__ = "1.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/smtp/provider.yaml b/airflow/providers/smtp/provider.yaml index a4b6664728ff..495e15e815b3 100644 --- a/airflow/providers/smtp/provider.yaml +++ b/airflow/providers/smtp/provider.yaml @@ -23,9 +23,10 @@ description: | `Simple Mail Transfer Protocol (SMTP) `__ state: ready -source-date-epoch: 1716289086 +source-date-epoch: 1723970570 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.8.0 - 1.7.1 - 1.7.0 - 1.6.1 diff --git a/airflow/providers/snowflake/CHANGELOG.rst b/airflow/providers/snowflake/CHANGELOG.rst index 7124ee717247..1ed5c705a6dd 100644 --- a/airflow/providers/snowflake/CHANGELOG.rst +++ b/airflow/providers/snowflake/CHANGELOG.rst @@ -27,6 +27,27 @@ Changelog --------- +5.7.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Bug Fixes +~~~~~~~~~ + +* ``Fix: Pass hook parameters to SnowflakeSqlApiHook and prep them for API call (#41150)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.6.1 ..... diff --git a/airflow/providers/snowflake/__init__.py b/airflow/providers/snowflake/__init__.py index 2dc2cce0e212..e01da744804f 100644 --- a/airflow/providers/snowflake/__init__.py +++ b/airflow/providers/snowflake/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.6.1" +__version__ = "5.7.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/snowflake/hooks/snowflake_sql_api.py b/airflow/providers/snowflake/hooks/snowflake_sql_api.py index 3f52a43a1d88..7b7e22c228ff 100644 --- a/airflow/providers/snowflake/hooks/snowflake_sql_api.py +++ b/airflow/providers/snowflake/hooks/snowflake_sql_api.py @@ -158,10 +158,13 @@ def execute_query( data = { "statement": sql, "resultSetMetaData": {"format": "json"}, - "database": conn_config["database"], - "schema": conn_config["schema"], - "warehouse": conn_config["warehouse"], - "role": conn_config["role"], + # If database, schema, warehouse, role parameters have been provided set them accordingly + # If either of them has been not (Parent class initializes them to None in that case) + # set them to what in the Airflow connection configuration + "database": self.database or conn_config["database"], + "schema": self.schema or conn_config["schema"], + "warehouse": self.warehouse or conn_config["warehouse"], + "role": self.role or conn_config["role"], "bindings": bindings, "parameters": { "MULTI_STATEMENT_COUNT": statement_count, diff --git a/airflow/providers/snowflake/operators/snowflake.py b/airflow/providers/snowflake/operators/snowflake.py index 70b3dfda4347..89af8fbb6fdc 100644 --- a/airflow/providers/snowflake/operators/snowflake.py +++ b/airflow/providers/snowflake/operators/snowflake.py @@ -505,6 +505,7 @@ def execute(self, context: Context) -> None: token_life_time=self.token_life_time, token_renewal_delta=self.token_renewal_delta, deferrable=self.deferrable, + **self.hook_params, ) self.query_ids = self._hook.execute_query( self.sql, # type: ignore[arg-type] diff --git a/airflow/providers/snowflake/provider.yaml b/airflow/providers/snowflake/provider.yaml index eddc132285ec..a0f7ec026175 100644 --- a/airflow/providers/snowflake/provider.yaml +++ b/airflow/providers/snowflake/provider.yaml @@ -22,9 +22,10 @@ description: | `Snowflake `__ state: ready -source-date-epoch: 1722664874 +source-date-epoch: 1723970579 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.7.0 - 5.6.1 - 5.6.0 - 5.5.2 diff --git a/airflow/providers/sqlite/CHANGELOG.rst b/airflow/providers/sqlite/CHANGELOG.rst index 0632e50513b1..0445c5bcf63f 100644 --- a/airflow/providers/sqlite/CHANGELOG.rst +++ b/airflow/providers/sqlite/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.8.2 ..... diff --git a/airflow/providers/sqlite/__init__.py b/airflow/providers/sqlite/__init__.py index c108fffd088c..973ce0f369e7 100644 --- a/airflow/providers/sqlite/__init__.py +++ b/airflow/providers/sqlite/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.8.2" +__version__ = "3.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/sqlite/provider.yaml b/airflow/providers/sqlite/provider.yaml index 13e0bf330279..586551285d83 100644 --- a/airflow/providers/sqlite/provider.yaml +++ b/airflow/providers/sqlite/provider.yaml @@ -22,9 +22,10 @@ description: | `SQLite `__ state: ready -source-date-epoch: 1722664885 +source-date-epoch: 1723970587 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.9.0 - 3.8.2 - 3.8.1 - 3.8.0 diff --git a/airflow/providers/ssh/CHANGELOG.rst b/airflow/providers/ssh/CHANGELOG.rst index e5745b4bc373..3bca18f52377 100644 --- a/airflow/providers/ssh/CHANGELOG.rst +++ b/airflow/providers/ssh/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +3.13.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.12.0 ...... diff --git a/airflow/providers/ssh/__init__.py b/airflow/providers/ssh/__init__.py index b07b23cfa201..98a034552f89 100644 --- a/airflow/providers/ssh/__init__.py +++ b/airflow/providers/ssh/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.12.0" +__version__ = "3.13.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/ssh/provider.yaml b/airflow/providers/ssh/provider.yaml index dd49cbae5b42..ae5d8d2cabc6 100644 --- a/airflow/providers/ssh/provider.yaml +++ b/airflow/providers/ssh/provider.yaml @@ -22,9 +22,10 @@ description: | `Secure Shell (SSH) `__ state: ready -source-date-epoch: 1722664896 +source-date-epoch: 1723970592 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.13.0 - 3.12.0 - 3.11.2 - 3.11.1 diff --git a/airflow/providers/tableau/CHANGELOG.rst b/airflow/providers/tableau/CHANGELOG.rst index b3966587b9b1..0f34016d1ed3 100644 --- a/airflow/providers/tableau/CHANGELOG.rst +++ b/airflow/providers/tableau/CHANGELOG.rst @@ -27,6 +27,25 @@ Changelog --------- +4.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 4.5.2 ..... diff --git a/airflow/providers/tableau/__init__.py b/airflow/providers/tableau/__init__.py index de418f2d73a2..a0ee8d4fad15 100644 --- a/airflow/providers/tableau/__init__.py +++ b/airflow/providers/tableau/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.5.2" +__version__ = "4.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/tableau/provider.yaml b/airflow/providers/tableau/provider.yaml index 93f7e338eab8..165b86b56da6 100644 --- a/airflow/providers/tableau/provider.yaml +++ b/airflow/providers/tableau/provider.yaml @@ -22,9 +22,10 @@ description: | `Tableau `__ state: ready -source-date-epoch: 1718605499 +source-date-epoch: 1723970597 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.6.0 - 4.5.2 - 4.5.1 - 4.5.0 diff --git a/airflow/providers/tabular/CHANGELOG.rst b/airflow/providers/tabular/CHANGELOG.rst index e65e1b752f2f..65cee2a3d911 100644 --- a/airflow/providers/tabular/CHANGELOG.rst +++ b/airflow/providers/tabular/CHANGELOG.rst @@ -26,6 +26,18 @@ Changelog --------- +1.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + 1.5.1 ..... diff --git a/airflow/providers/tabular/__init__.py b/airflow/providers/tabular/__init__.py index 352ab48a1d36..b08fb7c6e844 100644 --- a/airflow/providers/tabular/__init__.py +++ b/airflow/providers/tabular/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.5.1" +__version__ = "1.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/tabular/provider.yaml b/airflow/providers/tabular/provider.yaml index fffa4b26c17f..2e0530313d64 100644 --- a/airflow/providers/tabular/provider.yaml +++ b/airflow/providers/tabular/provider.yaml @@ -22,9 +22,10 @@ description: | `Tabular `__ state: ready -source-date-epoch: 1715450969 +source-date-epoch: 1723970601 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.6.0 - 1.5.1 - 1.5.0 - 1.4.1 diff --git a/airflow/providers/telegram/CHANGELOG.rst b/airflow/providers/telegram/CHANGELOG.rst index 217d3e4ed7d9..a7f8779f82b4 100644 --- a/airflow/providers/telegram/CHANGELOG.rst +++ b/airflow/providers/telegram/CHANGELOG.rst @@ -27,6 +27,18 @@ Changelog --------- +4.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + 4.5.2 ..... diff --git a/airflow/providers/telegram/__init__.py b/airflow/providers/telegram/__init__.py index d34b69d3aa51..be38ca0d2fc7 100644 --- a/airflow/providers/telegram/__init__.py +++ b/airflow/providers/telegram/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.5.2" +__version__ = "4.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/telegram/provider.yaml b/airflow/providers/telegram/provider.yaml index ad5563491e4a..3bbcf3753b96 100644 --- a/airflow/providers/telegram/provider.yaml +++ b/airflow/providers/telegram/provider.yaml @@ -22,9 +22,10 @@ description: | `Telegram `__ state: ready -source-date-epoch: 1718605519 +source-date-epoch: 1723970613 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.6.0 - 4.5.2 - 4.5.1 - 4.5.0 diff --git a/airflow/providers/teradata/CHANGELOG.rst b/airflow/providers/teradata/CHANGELOG.rst index 660c124ef104..ef183bc27fda 100644 --- a/airflow/providers/teradata/CHANGELOG.rst +++ b/airflow/providers/teradata/CHANGELOG.rst @@ -25,6 +25,22 @@ Changelog --------- +2.6.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.5.0 ..... diff --git a/airflow/providers/teradata/__init__.py b/airflow/providers/teradata/__init__.py index e418afd3e478..0898af3bc7ac 100644 --- a/airflow/providers/teradata/__init__.py +++ b/airflow/providers/teradata/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.5.0" +__version__ = "2.6.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/teradata/provider.yaml b/airflow/providers/teradata/provider.yaml index 9f5c1d5580cd..3e304b7334b4 100644 --- a/airflow/providers/teradata/provider.yaml +++ b/airflow/providers/teradata/provider.yaml @@ -22,9 +22,10 @@ description: | `Teradata `__ state: ready -source-date-epoch: 1722664913 +source-date-epoch: 1723970618 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.6.0 - 2.5.0 - 2.4.0 - 2.3.0 diff --git a/airflow/providers/trino/CHANGELOG.rst b/airflow/providers/trino/CHANGELOG.rst index 48d4f5823d0e..a0f32bc76a3c 100644 --- a/airflow/providers/trino/CHANGELOG.rst +++ b/airflow/providers/trino/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +5.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 5.7.2 ..... diff --git a/airflow/providers/trino/__init__.py b/airflow/providers/trino/__init__.py index bb4973dd091e..e729510404c1 100644 --- a/airflow/providers/trino/__init__.py +++ b/airflow/providers/trino/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "5.7.2" +__version__ = "5.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/trino/provider.yaml b/airflow/providers/trino/provider.yaml index 076f90d84697..d4000baaa063 100644 --- a/airflow/providers/trino/provider.yaml +++ b/airflow/providers/trino/provider.yaml @@ -22,9 +22,10 @@ description: | `Trino `__ state: ready -source-date-epoch: 1718605541 +source-date-epoch: 1723970622 # note that those versions are maintained by release manager - do not update them manually versions: + - 5.8.0 - 5.7.2 - 5.7.1 - 5.7.0 diff --git a/airflow/providers/vertica/CHANGELOG.rst b/airflow/providers/vertica/CHANGELOG.rst index 4520e14c2815..59ee35fe9298 100644 --- a/airflow/providers/vertica/CHANGELOG.rst +++ b/airflow/providers/vertica/CHANGELOG.rst @@ -28,6 +28,22 @@ Changelog --------- +3.9.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 3.8.2 ..... diff --git a/airflow/providers/vertica/__init__.py b/airflow/providers/vertica/__init__.py index ce8ac24bcb5c..c050eb77f3f6 100644 --- a/airflow/providers/vertica/__init__.py +++ b/airflow/providers/vertica/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.8.2" +__version__ = "3.9.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/vertica/provider.yaml b/airflow/providers/vertica/provider.yaml index ba5c19533dc7..c33d202fb56b 100644 --- a/airflow/providers/vertica/provider.yaml +++ b/airflow/providers/vertica/provider.yaml @@ -22,9 +22,10 @@ description: | `Vertica `__ state: ready -source-date-epoch: 1718605559 +source-date-epoch: 1723970627 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.9.0 - 3.8.2 - 3.8.1 - 3.8.0 diff --git a/airflow/providers/weaviate/CHANGELOG.rst b/airflow/providers/weaviate/CHANGELOG.rst index 7059fdd4e8fb..3565b781d22d 100644 --- a/airflow/providers/weaviate/CHANGELOG.rst +++ b/airflow/providers/weaviate/CHANGELOG.rst @@ -20,6 +20,22 @@ Changelog --------- +2.1.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 2.0.0 ...... diff --git a/airflow/providers/weaviate/__init__.py b/airflow/providers/weaviate/__init__.py index 778af16b4cf5..5547395b7cb5 100644 --- a/airflow/providers/weaviate/__init__.py +++ b/airflow/providers/weaviate/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "2.0.0" +__version__ = "2.1.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/weaviate/provider.yaml b/airflow/providers/weaviate/provider.yaml index 6c02e54ee4a1..8328d2c61474 100644 --- a/airflow/providers/weaviate/provider.yaml +++ b/airflow/providers/weaviate/provider.yaml @@ -24,10 +24,11 @@ description: | `Weaviate `__ state: ready -source-date-epoch: 1718605569 +source-date-epoch: 1723970637 # note that those versions are maintained by release manager - do not update them manually versions: + - 2.1.0 - 2.0.0 - 1.4.1 - 1.4.0 diff --git a/airflow/providers/yandex/CHANGELOG.rst b/airflow/providers/yandex/CHANGELOG.rst index 94635fb3bb87..a34d53265e92 100644 --- a/airflow/providers/yandex/CHANGELOG.rst +++ b/airflow/providers/yandex/CHANGELOG.rst @@ -27,6 +27,29 @@ Changelog --------- +3.12.0 +...... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Bug Fixes +~~~~~~~~~ + +* ``providers/yandex: fix typing (#40997)`` + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + * ``Prepare docs for Aug 1st wave of providers (#41230)`` + * ``Prepare docs 1st wave July 2024 (#40644)`` + * ``Enable enforcing pydocstyle rule D213 in ruff. (#40448)`` + 3.11.2 ...... diff --git a/airflow/providers/yandex/__init__.py b/airflow/providers/yandex/__init__.py index e8305c8ef4f2..0cfe8989d883 100644 --- a/airflow/providers/yandex/__init__.py +++ b/airflow/providers/yandex/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "3.11.2" +__version__ = "3.12.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/yandex/provider.yaml b/airflow/providers/yandex/provider.yaml index b3717347f6fc..000781f28de1 100644 --- a/airflow/providers/yandex/provider.yaml +++ b/airflow/providers/yandex/provider.yaml @@ -23,9 +23,10 @@ description: | - `Yandex.Cloud `__ state: ready -source-date-epoch: 1718605606 +source-date-epoch: 1723970668 # note that those versions are maintained by release manager - do not update them manually versions: + - 3.12.0 - 3.11.2 - 3.11.1 - 3.11.0 diff --git a/airflow/providers/ydb/CHANGELOG.rst b/airflow/providers/ydb/CHANGELOG.rst index c8e653ad4f72..a6ee9aa6b104 100644 --- a/airflow/providers/ydb/CHANGELOG.rst +++ b/airflow/providers/ydb/CHANGELOG.rst @@ -27,6 +27,23 @@ Changelog --------- +1.3.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Add database to table name in bulk upsert, use bulk upsert in system test (#41303)`` +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 1.2.0 ..... diff --git a/airflow/providers/ydb/__init__.py b/airflow/providers/ydb/__init__.py index 0bfd52dbda3f..b7b03b98626b 100644 --- a/airflow/providers/ydb/__init__.py +++ b/airflow/providers/ydb/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "1.2.0" +__version__ = "1.3.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/ydb/hooks/ydb.py b/airflow/providers/ydb/hooks/ydb.py index fa82483e1ca3..e4580212d8d6 100644 --- a/airflow/providers/ydb/hooks/ydb.py +++ b/airflow/providers/ydb/hooks/ydb.py @@ -138,7 +138,7 @@ def __init__(self, *args, is_ddl: bool = False, **kwargs) -> None: super().__init__(*args, **kwargs) self.is_ddl = is_ddl - conn: Connection = self.get_connection(getattr(self, self.conn_name_attr)) + conn: Connection = self.get_connection(self.get_conn_id()) host: str | None = conn.host if not host: raise ValueError("YDB host must be specified") @@ -148,6 +148,7 @@ def __init__(self, *args, is_ddl: bool = False, **kwargs) -> None: database: str | None = connection_extra.get("database") if not database: raise ValueError("YDB database must be specified") + self.database: str = database endpoint = f"{host}:{port}" credentials = get_credentials_from_connection( @@ -222,15 +223,13 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]: @property def sqlalchemy_url(self) -> URL: conn: Connection = self.get_connection(self.get_conn_id()) - connection_extra: dict[str, Any] = conn.extra_dejson - database: str | None = connection_extra.get("database") return URL.create( drivername="ydb", username=conn.login, password=conn.password, host=conn.host, port=conn.port, - query={"database": database}, + query={"database": self.database}, ) def get_conn(self) -> YDBConnection: @@ -249,7 +248,7 @@ def bulk_upsert(self, table_name: str, rows: Sequence, column_types: ydb.BulkUps https://ydb.tech/docs/en/recipes/ydb-sdk/bulk-upsert """ - self.get_conn().bulk_upsert(table_name, rows, column_types) + self.get_conn().bulk_upsert(f"{self.database}/{table_name}", rows, column_types) @staticmethod def _get_table_client_settings() -> ydb.TableClientSettings: diff --git a/airflow/providers/ydb/provider.yaml b/airflow/providers/ydb/provider.yaml index 2b7397393a7b..a956100dc3f6 100644 --- a/airflow/providers/ydb/provider.yaml +++ b/airflow/providers/ydb/provider.yaml @@ -22,9 +22,10 @@ description: | `YDB `__ state: ready -source-date-epoch: 1722664926 +source-date-epoch: 1723970719 # note that those versions are maintained by release manager - do not update them manually versions: + - 1.3.0 - 1.2.0 - 1.1.0 - 1.0.0 diff --git a/airflow/providers/zendesk/CHANGELOG.rst b/airflow/providers/zendesk/CHANGELOG.rst index b15d5afc619a..6b45ce426889 100644 --- a/airflow/providers/zendesk/CHANGELOG.rst +++ b/airflow/providers/zendesk/CHANGELOG.rst @@ -27,6 +27,22 @@ Changelog --------- +4.8.0 +..... + +.. note:: + This release of provider is only available for Airflow 2.8+ as explained in the + `Apache Airflow providers support policy `_. + +Misc +~~~~ + +* ``Bump minimum Airflow version in providers to Airflow 2.8.0 (#41396)`` + + +.. Below changes are excluded from the changelog. Move them to + appropriate section above if needed. Do not delete the lines(!): + 4.7.1 ..... diff --git a/airflow/providers/zendesk/__init__.py b/airflow/providers/zendesk/__init__.py index bd45239bb8c9..1f4fb6efc489 100644 --- a/airflow/providers/zendesk/__init__.py +++ b/airflow/providers/zendesk/__init__.py @@ -29,7 +29,7 @@ __all__ = ["__version__"] -__version__ = "4.7.1" +__version__ = "4.8.0" if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse( "2.8.0" diff --git a/airflow/providers/zendesk/provider.yaml b/airflow/providers/zendesk/provider.yaml index a9bad9cea67f..eb552c323bbb 100644 --- a/airflow/providers/zendesk/provider.yaml +++ b/airflow/providers/zendesk/provider.yaml @@ -22,9 +22,10 @@ description: | `Zendesk `__ state: ready -source-date-epoch: 1716289258 +source-date-epoch: 1723970723 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.8.0 - 4.7.1 - 4.7.0 - 4.6.0 diff --git a/airflow/reproducible_build.yaml b/airflow/reproducible_build.yaml index b6f92ca9a470..775aaa70ae9b 100644 --- a/airflow/reproducible_build.yaml +++ b/airflow/reproducible_build.yaml @@ -1,2 +1,2 @@ -release-notes-hash: fdd42ae58b946146d51d09ea6e5c28cd -source-date-epoch: 1721131067 +release-notes-hash: 62779daa28762b157d814861dc91adcf +source-date-epoch: 1723560829 diff --git a/airflow/secrets/local_filesystem.py b/airflow/secrets/local_filesystem.py index f9ea4ad3554c..60e78f59e5f8 100644 --- a/airflow/secrets/local_filesystem.py +++ b/airflow/secrets/local_filesystem.py @@ -22,7 +22,6 @@ import json import logging import os -import warnings from collections import defaultdict from inspect import signature from json import JSONDecodeError @@ -33,7 +32,6 @@ AirflowFileParseException, ConnectionNotUnique, FileSyntaxError, - RemovedInAirflow3Warning, ) from airflow.secrets.base_secrets import BaseSecretsBackend from airflow.utils import yaml @@ -243,16 +241,6 @@ def load_variables(file_path: str) -> dict[str, str]: return variables -def load_connections(file_path) -> dict[str, list[Any]]: - """Use `airflow.secrets.local_filesystem.load_connections_dict`, this is deprecated.""" - warnings.warn( - "This function is deprecated. Please use `airflow.secrets.local_filesystem.load_connections_dict`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return {k: [v] for k, v in load_connections_dict(file_path).values()} - - def load_connections_dict(file_path: str) -> dict[str, Any]: """ Load connection from text file. @@ -318,17 +306,5 @@ def get_connection(self, conn_id: str) -> Connection | None: return self._local_connections[conn_id] return None - def get_connections(self, conn_id: str) -> list[Any]: - warnings.warn( - "This method is deprecated. Please use " - "`airflow.secrets.local_filesystem.LocalFilesystemBackend.get_connection`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - conn = self.get_connection(conn_id=conn_id) - if conn: - return [conn] - return [] - def get_variable(self, key: str) -> str | None: return self._local_variables.get(key) diff --git a/airflow/sensors/__init__.py b/airflow/sensors/__init__.py index 3d1627cc036a..bc76ab08fc65 100644 --- a/airflow/sensors/__init__.py +++ b/airflow/sensors/__init__.py @@ -21,53 +21,3 @@ :sphinx-autoapi-skip: """ -from __future__ import annotations - -from airflow.utils.deprecation_tools import add_deprecated_classes - -__deprecated_classes = { - "base_sensor_operator": { - "BaseSensorOperator": "airflow.sensors.base.BaseSensorOperator", - }, - "date_time_sensor": { - "DateTimeSensor": "airflow.sensors.date_time.DateTimeSensor", - }, - "external_task_sensor": { - "ExternalTaskMarker": "airflow.sensors.external_task.ExternalTaskMarker", - "ExternalTaskSensor": "airflow.sensors.external_task.ExternalTaskSensor", - "ExternalTaskSensorLink": "airflow.sensors.external_task.ExternalTaskSensorLink", - }, - "hive_partition_sensor": { - "HivePartitionSensor": "airflow.providers.apache.hive.sensors.hive_partition.HivePartitionSensor", - }, - "http_sensor": { - "HttpSensor": "airflow.providers.http.sensors.http.HttpSensor", - }, - "metastore_partition_sensor": { - "MetastorePartitionSensor": ( - "airflow.providers.apache.hive.sensors.metastore_partition.MetastorePartitionSensor" - ), - }, - "named_hive_partition_sensor": { - "NamedHivePartitionSensor": ( - "airflow.providers.apache.hive.sensors.named_hive_partition.NamedHivePartitionSensor" - ), - }, - "s3_key_sensor": { - "S3KeySensor": "airflow.providers.amazon.aws.sensors.s3.S3KeySensor", - }, - "sql": { - "SqlSensor": "airflow.providers.common.sql.sensors.sql.SqlSensor", - }, - "sql_sensor": { - "SqlSensor": "airflow.providers.common.sql.sensors.sql.SqlSensor", - }, - "time_delta_sensor": { - "TimeDeltaSensor": "airflow.sensors.time_delta.TimeDeltaSensor", - }, - "web_hdfs_sensor": { - "WebHdfsSensor": "airflow.providers.apache.hdfs.sensors.web_hdfs.WebHdfsSensor", - }, -} - -add_deprecated_classes(__deprecated_classes, __name__) diff --git a/airflow/serialization/enums.py b/airflow/serialization/enums.py index a5bd5e3646e8..f216ce731610 100644 --- a/airflow/serialization/enums.py +++ b/airflow/serialization/enums.py @@ -46,6 +46,7 @@ class DagAttributeTypes(str, Enum): RELATIVEDELTA = "relativedelta" BASE_TRIGGER = "base_trigger" AIRFLOW_EXC_SER = "airflow_exc_ser" + BASE_EXC_SER = "base_exc_ser" DICT = "dict" SET = "set" TUPLE = "tuple" diff --git a/airflow/serialization/pydantic/dag.py b/airflow/serialization/pydantic/dag.py index fa1cac535f60..a1fea6384aad 100644 --- a/airflow/serialization/pydantic/dag.py +++ b/airflow/serialization/pydantic/dag.py @@ -108,10 +108,8 @@ class DagModelPydantic(BaseModelPydantic): """Serializable representation of the DagModel ORM SqlAlchemyModel used by internal API.""" dag_id: str - root_dag_id: Optional[str] is_paused_at_creation: bool = airflow_conf.getboolean("core", "dags_are_paused_at_creation") is_paused: bool = is_paused_at_creation - is_subdag: Optional[bool] = False is_active: Optional[bool] = False last_parsed_time: Optional[datetime] last_pickled: Optional[datetime] @@ -127,7 +125,6 @@ class DagModelPydantic(BaseModelPydantic): timetable_description: Optional[str] tags: List[DagTagPydantic] # noqa: UP006 dag_owner_links: List[DagOwnerAttributesPydantic] # noqa: UP006 - parent_dag: Optional[PydanticDag] max_active_tasks: int max_active_runs: Optional[int] diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 84b2e2ed4a75..d76bfcb1a40b 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -157,7 +157,6 @@ } }, "catchup": { "type": "boolean" }, - "is_subdag": { "type": "boolean" }, "fileloc": { "type" : "string"}, "_processor_dags_folder": { "anyOf": [ diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index d110271c3da0..4004b83a991b 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -692,6 +692,15 @@ def serialize( ), type_=DAT.AIRFLOW_EXC_SER, ) + elif isinstance(var, (KeyError, AttributeError)): + return cls._encode( + cls.serialize( + {"exc_cls_name": var.__class__.__name__, "args": [var.args], "kwargs": {}}, + use_pydantic_models=use_pydantic_models, + strict=strict, + ), + type_=DAT.BASE_EXC_SER, + ) elif isinstance(var, BaseTrigger): return cls._encode( cls.serialize(var.serialize(), use_pydantic_models=use_pydantic_models, strict=strict), @@ -834,13 +843,16 @@ def deserialize(cls, encoded_var: Any, use_pydantic_models=False) -> Any: return decode_timezone(var) elif type_ == DAT.RELATIVEDELTA: return decode_relativedelta(var) - elif type_ == DAT.AIRFLOW_EXC_SER: + elif type_ == DAT.AIRFLOW_EXC_SER or type_ == DAT.BASE_EXC_SER: deser = cls.deserialize(var, use_pydantic_models=use_pydantic_models) exc_cls_name = deser["exc_cls_name"] args = deser["args"] kwargs = deser["kwargs"] del deser - exc_cls = import_string(exc_cls_name) + if type_ == DAT.AIRFLOW_EXC_SER: + exc_cls = import_string(exc_cls_name) + else: + exc_cls = import_string(f"builtins.{exc_cls_name}") return exc_cls(*args, **kwargs) elif type_ == DAT.BASE_TRIGGER: tr_cls_name, kwargs = cls.deserialize(var, use_pydantic_models=use_pydantic_models) @@ -1273,8 +1285,6 @@ def populate_operator(cls, op: Operator, encoded_op: dict[str, Any]) -> None: continue elif k == "downstream_task_ids": v = set(v) - elif k == "subdag": - v = SerializedDAG.deserialize_dag(v) elif k in {"retry_delay", "execution_timeout", "sla", "max_retry_delay"}: v = cls._deserialize_timedelta(v) elif k in encoded_op["template_fields"]: @@ -1359,9 +1369,6 @@ def set_task_dag_references(task: Operator, dag: DAG) -> None: if getattr(task, date_attr, None) is None: setattr(task, date_attr, getattr(dag, date_attr, None)) - if task.subdag is not None: - task.subdag.parent_dag = dag - # Dereference expand_input and op_kwargs_expand_input. for k in ("expand_input", "op_kwargs_expand_input"): if isinstance(kwargs_ref := getattr(task, k, None), _ExpandInputRef): @@ -1663,7 +1670,7 @@ def serialize_dag(cls, dag: DAG) -> dict: @classmethod def deserialize_dag(cls, encoded_dag: dict[str, Any]) -> SerializedDAG: """Deserializes a DAG from a JSON object.""" - dag = SerializedDAG(dag_id=encoded_dag["_dag_id"]) + dag = SerializedDAG(dag_id=encoded_dag["_dag_id"], schedule=None) for k, v in encoded_dag.items(): if k == "_downstream_task_ids": diff --git a/airflow/settings.py b/airflow/settings.py index c780ccd6c5d0..49385dcd0652 100644 --- a/airflow/settings.py +++ b/airflow/settings.py @@ -36,7 +36,7 @@ from airflow import __version__ as airflow_version, policies from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf # noqa: F401 -from airflow.exceptions import AirflowInternalRuntimeError, RemovedInAirflow3Warning +from airflow.exceptions import AirflowInternalRuntimeError from airflow.executors import executor_constants from airflow.logging_config import configure_logging from airflow.utils.orm_event_handlers import setup_event_handlers @@ -671,25 +671,7 @@ def prepare_syspath(): def get_session_lifetime_config(): """Get session timeout configs and handle outdated configs gracefully.""" session_lifetime_minutes = conf.get("webserver", "session_lifetime_minutes", fallback=None) - session_lifetime_days = conf.get("webserver", "session_lifetime_days", fallback=None) - uses_deprecated_lifetime_configs = session_lifetime_days or conf.get( - "webserver", "force_log_out_after", fallback=None - ) - minutes_per_day = 24 * 60 - default_lifetime_minutes = "43200" - if uses_deprecated_lifetime_configs and session_lifetime_minutes == default_lifetime_minutes: - warnings.warn( - "`session_lifetime_days` option from `[webserver]` section has been " - "renamed to `session_lifetime_minutes`. The new option allows to configure " - "session lifetime in minutes. The `force_log_out_after` option has been removed " - "from `[webserver]` section. Please update your configuration.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - if session_lifetime_days: - session_lifetime_minutes = minutes_per_day * int(session_lifetime_days) - if not session_lifetime_minutes: session_lifetime_days = 30 session_lifetime_minutes = minutes_per_day * session_lifetime_days @@ -721,16 +703,6 @@ def import_local_settings(): else: names = {n for n in airflow_local_settings.__dict__ if not n.startswith("__")} - if "policy" in names and "task_policy" not in names: - warnings.warn( - "Using `policy` in airflow_local_settings.py is deprecated. " - "Please rename your `policy` to `task_policy`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - setattr(airflow_local_settings, "task_policy", airflow_local_settings.policy) - names.remove("policy") - plugin_functions = policies.make_plugin_from_local_settings( POLICY_PLUGIN_MANAGER, airflow_local_settings, names ) diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 1d58717287c7..79272d43d00b 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -42,7 +42,7 @@ def gen_links_from_kv_list(list): return gen_links_from_kv_list(list) -def span(func): +def add_span(func): """Decorate a function with span.""" def wrapper(*args, **kwargs): diff --git a/airflow/traces/utils.py b/airflow/traces/utils.py index afab2591d514..9932c249f077 100644 --- a/airflow/traces/utils.py +++ b/airflow/traces/utils.py @@ -22,7 +22,6 @@ from airflow.traces import NO_TRACE_ID from airflow.utils.hashlib_wrapper import md5 -from airflow.utils.state import TaskInstanceState if TYPE_CHECKING: from airflow.models import DagRun, TaskInstance @@ -75,12 +74,8 @@ def gen_dag_span_id(dag_run: DagRun, as_int: bool = False) -> str | int: def gen_span_id(ti: TaskInstance, as_int: bool = False) -> str | int: """Generate span id from the task instance.""" dag_run = ti.dag_run - if ti.state == TaskInstanceState.SUCCESS or ti.state == TaskInstanceState.FAILED: - try_number = ti.try_number - 1 - else: - try_number = ti.try_number return _gen_id( - [dag_run.dag_id, dag_run.run_id, ti.task_id, str(try_number)], + [dag_run.dag_id, dag_run.run_id, ti.task_id, str(ti.try_number)], as_int, SPAN_ID, ) diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py index f3eeddb916f5..7a9ac2d80316 100644 --- a/airflow/utils/dates.py +++ b/airflow/utils/dates.py @@ -17,14 +17,11 @@ # under the License. from __future__ import annotations -import warnings from datetime import datetime, timedelta -from typing import Collection +from typing import TYPE_CHECKING, Collection from croniter import croniter -from dateutil.relativedelta import relativedelta # for doctest -from airflow.exceptions import RemovedInAirflow3Warning from airflow.typing_compat import Literal from airflow.utils import timezone @@ -37,103 +34,8 @@ "@yearly": "0 0 1 1 *", } - -def date_range( - start_date: datetime, - end_date: datetime | None = None, - num: int | None = None, - delta: str | timedelta | relativedelta | None = None, -) -> list[datetime]: - """ - Get a list of dates in the specified range, separated by delta. - - .. code-block:: pycon - >>> from airflow.utils.dates import date_range - >>> from datetime import datetime, timedelta - >>> date_range(datetime(2016, 1, 1), datetime(2016, 1, 3), delta=timedelta(1)) - [datetime.datetime(2016, 1, 1, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 1, 2, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 1, 3, 0, 0, tzinfo=Timezone('UTC'))] - >>> date_range(datetime(2016, 1, 1), datetime(2016, 1, 3), delta="0 0 * * *") - [datetime.datetime(2016, 1, 1, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 1, 2, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 1, 3, 0, 0, tzinfo=Timezone('UTC'))] - >>> date_range(datetime(2016, 1, 1), datetime(2016, 3, 3), delta="0 0 0 * *") - [datetime.datetime(2016, 1, 1, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 2, 1, 0, 0, tzinfo=Timezone('UTC')), - datetime.datetime(2016, 3, 1, 0, 0, tzinfo=Timezone('UTC'))] - - :param start_date: anchor date to start the series from - :param end_date: right boundary for the date range - :param num: alternatively to end_date, you can specify the number of - number of entries you want in the range. This number can be negative, - output will always be sorted regardless - :param delta: step length. It can be datetime.timedelta or cron expression as string - """ - warnings.warn( - "`airflow.utils.dates.date_range()` is deprecated. Please use `airflow.timetables`.", - category=RemovedInAirflow3Warning, - stacklevel=2, - ) - - if not delta: - return [] - if end_date: - if start_date > end_date: - raise ValueError("Wait. start_date needs to be before end_date") - if num: - raise ValueError("Wait. Either specify end_date OR num") - if not end_date and not num: - end_date = timezone.utcnow() - - delta_iscron = False - time_zone = start_date.tzinfo - - abs_delta: timedelta | relativedelta - if isinstance(delta, str): - delta_iscron = True - if timezone.is_localized(start_date): - start_date = timezone.make_naive(start_date, time_zone) - cron = croniter(cron_presets.get(delta, delta), start_date) - elif isinstance(delta, timedelta): - abs_delta = abs(delta) - elif isinstance(delta, relativedelta): - abs_delta = abs(delta) - else: - raise TypeError("Wait. delta must be either datetime.timedelta or cron expression as str") - - dates = [] - if end_date: - if timezone.is_naive(start_date) and not timezone.is_naive(end_date): - end_date = timezone.make_naive(end_date, time_zone) - while start_date <= end_date: # type: ignore - if timezone.is_naive(start_date): - dates.append(timezone.make_aware(start_date, time_zone)) - else: - dates.append(start_date) - - if delta_iscron: - start_date = cron.get_next(datetime) - else: - start_date += abs_delta - else: - num_entries: int = num # type: ignore - for _ in range(abs(num_entries)): - if timezone.is_naive(start_date): - dates.append(timezone.make_aware(start_date, time_zone)) - else: - dates.append(start_date) - - if delta_iscron and num_entries > 0: - start_date = cron.get_next(datetime) - elif delta_iscron: - start_date = cron.get_prev(datetime) - elif num_entries > 0: - start_date += abs_delta - else: - start_date -= abs_delta - - return sorted(dates) +if TYPE_CHECKING: + from dateutil.relativedelta import relativedelta # for doctest def round_time( @@ -256,23 +158,6 @@ def scale_time_units(time_seconds_arr: Collection[float], unit: TimeUnit) -> Col return [x / factor for x in time_seconds_arr] -def days_ago(n, hour=0, minute=0, second=0, microsecond=0): - """ - Get a datetime object representing *n* days ago. - - By default the time is set to midnight. - """ - warnings.warn( - "Function `days_ago` is deprecated and will be removed in Airflow 3.0. " - "You can achieve equivalent behavior with `pendulum.today('UTC').add(days=-N, ...)`", - RemovedInAirflow3Warning, - stacklevel=2, - ) - - today = timezone.utcnow().replace(hour=hour, minute=minute, second=second, microsecond=microsecond) - return today - timedelta(days=n) - - def parse_execution_date(execution_date_str): """Parse execution date string to datetime object.""" return timezone.parse(execution_date_str) diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b5e722cb5005..a86ca5dbf562 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -95,7 +95,7 @@ class MappedClassProtocol(Protocol): _REVISION_HEADS_MAP = { "2.10.0": "22ed7efa9da2", - "3.0.0": "044f740568ec", + "3.0.0": "d0f1c55954fa", } diff --git a/airflow/utils/email.py b/airflow/utils/email.py index 31a33337f97d..3a63b4180472 100644 --- a/airflow/utils/email.py +++ b/airflow/utils/email.py @@ -22,7 +22,6 @@ import os import smtplib import ssl -import warnings from email.mime.application import MIMEApplication from email.mime.multipart import MIMEMultipart from email.mime.text import MIMEText @@ -32,7 +31,7 @@ import re2 from airflow.configuration import conf -from airflow.exceptions import AirflowConfigException, AirflowException, RemovedInAirflow3Warning +from airflow.exceptions import AirflowException log = logging.getLogger(__name__) @@ -255,17 +254,7 @@ def send_mime_email( except AirflowException: pass if smtp_user is None or smtp_password is None: - warnings.warn( - "Fetching SMTP credentials from configuration variables will be deprecated in a future " - "release. Please set credentials using a connection instead.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - try: - smtp_user = conf.get("smtp", "SMTP_USER") - smtp_password = conf.get("smtp", "SMTP_PASSWORD") - except AirflowConfigException: - log.debug("No user/password found for SMTP, so logging in with no authentication.") + log.debug("No user/password found for SMTP, so logging in with no authentication.") if not dryrun: for attempt in range(1, smtp_retry_limit + 1): diff --git a/airflow/utils/helpers.py b/airflow/utils/helpers.py index 801cda03f017..8c08acc66be2 100644 --- a/airflow/utils/helpers.py +++ b/airflow/utils/helpers.py @@ -21,7 +21,6 @@ import itertools import re import signal -import warnings from datetime import datetime from functools import reduce from typing import TYPE_CHECKING, Any, Callable, Generator, Iterable, Mapping, MutableMapping, TypeVar, cast @@ -29,8 +28,7 @@ from lazy_object_proxy import Proxy from airflow.configuration import conf -from airflow.exceptions import AirflowException, RemovedInAirflow3Warning -from airflow.utils.module_loading import import_string +from airflow.exceptions import AirflowException from airflow.utils.types import NOTSET if TYPE_CHECKING: @@ -231,26 +229,6 @@ def partition(pred: Callable[[T], bool], iterable: Iterable[T]) -> tuple[Iterabl return itertools.filterfalse(pred, iter_1), filter(pred, iter_2) -def chain(*args, **kwargs): - """Use `airflow.models.baseoperator.chain`, this function is deprecated.""" - warnings.warn( - "This function is deprecated. Please use `airflow.models.baseoperator.chain`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return import_string("airflow.models.baseoperator.chain")(*args, **kwargs) - - -def cross_downstream(*args, **kwargs): - """Use `airflow.models.baseoperator.cross_downstream`, this function is deprecated.""" - warnings.warn( - "This function is deprecated. Please use `airflow.models.baseoperator.cross_downstream`.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return import_string("airflow.models.baseoperator.cross_downstream")(*args, **kwargs) - - def build_airflow_url_with_query(query: dict[str, Any]) -> str: """ Build airflow url using base_url and default_view and provided query. diff --git a/airflow/utils/python_virtualenv_script.jinja2 b/airflow/utils/python_virtualenv_script.jinja2 index 2ff417985e88..22d68acd755b 100644 --- a/airflow/utils/python_virtualenv_script.jinja2 +++ b/airflow/utils/python_virtualenv_script.jinja2 @@ -64,6 +64,29 @@ with open(sys.argv[3], "r") as file: virtualenv_string_args = list(map(lambda x: x.strip(), list(file))) {% endif %} +{% if use_airflow_context | default(false) -%} +if len(sys.argv) > 5: + import json + from types import ModuleType + + from airflow.operators import python as airflow_python + from airflow.serialization.serialized_objects import BaseSerialization + + + class _MockPython(ModuleType): + @staticmethod + def get_current_context(): + with open(sys.argv[5]) as file: + context = json.load(file) + return BaseSerialization.deserialize(context, use_pydantic_models=True) + + def __getattr__(self, name: str): + return getattr(airflow_python, name) + + + MockPython = _MockPython("MockPython") + sys.modules["airflow.operators.python"] = MockPython +{% endif %} try: res = {{ python_callable }}(*arg_dict["args"], **arg_dict["kwargs"]) diff --git a/airflow/utils/task_group.py b/airflow/utils/task_group.py index f3d46ea6eac7..69a5d015bd42 100644 --- a/airflow/utils/task_group.py +++ b/airflow/utils/task_group.py @@ -491,7 +491,7 @@ def hierarchical_alphabetical_sort(self): self.children.values(), key=lambda node: (not isinstance(node, TaskGroup), node.node_id) ) - def topological_sort(self, _include_subdag_tasks: bool = False): + def topological_sort(self): """ Sorts children in topographical order, such that a task comes after any of its upstream dependencies. @@ -499,8 +499,6 @@ def topological_sort(self, _include_subdag_tasks: bool = False): """ # This uses a modified version of Kahn's Topological Sort algorithm to # not have to pre-compute the "in-degree" of the nodes. - from airflow.operators.subdag import SubDagOperator # Avoid circular import - graph_unsorted = copy.copy(self.children) graph_sorted: list[DAGNode] = [] @@ -539,10 +537,6 @@ def topological_sort(self, _include_subdag_tasks: bool = False): acyclic = True del graph_unsorted[node.node_id] graph_sorted.append(node) - if _include_subdag_tasks and isinstance(node, SubDagOperator): - graph_sorted.extend( - node.subdag.task_group.topological_sort(_include_subdag_tasks=True) - ) if not acyclic: raise AirflowDagCycleException(f"A cyclic dependency occurred in dag: {self.dag_id}") diff --git a/airflow/www/api/__init__.py b/airflow/www/api/__init__.py deleted file mode 100644 index 217e5db96078..000000000000 --- a/airflow/www/api/__init__.py +++ /dev/null @@ -1,17 +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. diff --git a/airflow/www/api/experimental/__init__.py b/airflow/www/api/experimental/__init__.py deleted file mode 100644 index 217e5db96078..000000000000 --- a/airflow/www/api/experimental/__init__.py +++ /dev/null @@ -1,17 +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. diff --git a/airflow/www/api/experimental/endpoints.py b/airflow/www/api/experimental/endpoints.py deleted file mode 100644 index cba955d4da3a..000000000000 --- a/airflow/www/api/experimental/endpoints.py +++ /dev/null @@ -1,434 +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. -from __future__ import annotations - -import logging -from functools import wraps -from typing import TYPE_CHECKING, Callable, TypeVar, cast - -from flask import Blueprint, current_app, g, jsonify, request, url_for - -from airflow import models -from airflow.api.common import delete_dag as delete, trigger_dag as trigger -from airflow.api.common.experimental import pool as pool_api -from airflow.api.common.experimental.get_code import get_code -from airflow.api.common.experimental.get_dag_run_state import get_dag_run_state -from airflow.api.common.experimental.get_dag_runs import get_dag_runs -from airflow.api.common.experimental.get_lineage import get_lineage as get_lineage_api -from airflow.api.common.experimental.get_task import get_task -from airflow.api.common.experimental.get_task_instance import get_task_instance -from airflow.exceptions import AirflowException -from airflow.utils import timezone -from airflow.utils.docs import get_docs_url -from airflow.utils.strings import to_boolean -from airflow.version import version - -if TYPE_CHECKING: - from flask import Response - -log = logging.getLogger(__name__) - -T = TypeVar("T", bound=Callable) - - -def requires_authentication(function: T): - """Mark a function as requiring authentication.""" - - @wraps(function) - def decorated(*args, **kwargs): - auth = current_app.api_auth[0] - return auth.requires_authentication(function)(*args, **kwargs) - - return cast(T, decorated) - - -api_experimental = Blueprint("api_experimental", __name__) - - -def add_deprecation_headers(response: Response): - """ - Add Deprecation HTTP Header Field. - - .. seealso:: IETF proposal for the header field - `here `_. - """ - response.headers["Deprecation"] = "true" - doc_url = get_docs_url("upgrading-to-2.html#migration-guide-from-experimental-api-to-stable-api-v1") - deprecation_link = f'<{doc_url}>; rel="deprecation"; type="text/html"' - if "link" in response.headers: - response.headers["Link"] += f", {deprecation_link}" - else: - response.headers["Link"] = f"{deprecation_link}" - - return response - - -# This API is deprecated. We do not care too much about typing here -api_experimental.after_request(add_deprecation_headers) # type: ignore[arg-type] - - -@api_experimental.route("/dags//dag_runs", methods=["POST"]) -@requires_authentication -def trigger_dag(dag_id): - """Trigger a new dag run for a Dag with an execution date of now unless specified in the data.""" - data = request.get_json(force=True) - - run_id = None - if "run_id" in data: - run_id = data["run_id"] - - conf = None - if "conf" in data: - conf = data["conf"] - if not isinstance(conf, dict): - error_message = "Dag Run conf must be a dictionary object, other types are not supported" - log.error(error_message) - response = jsonify({"error": error_message}) - response.status_code = 400 - return response - - execution_date = None - if "execution_date" in data and data["execution_date"] is not None: - execution_date = data["execution_date"] - - # Convert string datetime into actual datetime - try: - execution_date = timezone.parse(execution_date) - except ValueError: - log.error("Given execution date could not be identified as a date.") - error_message = ( - f"Given execution date, {execution_date}, could not be identified as a date. " - f"Example date format: 2015-11-16T14:34:15+00:00" - ) - response = jsonify({"error": error_message}) - response.status_code = 400 - - return response - - replace_microseconds = execution_date is None - if "replace_microseconds" in data: - replace_microseconds = to_boolean(data["replace_microseconds"]) - - try: - dr = trigger.trigger_dag(dag_id, run_id, conf, execution_date, replace_microseconds) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - - if getattr(g, "user", None): - log.info("User %s created %s", g.user, dr) - - response = jsonify( - message=f"Created {dr}", execution_date=dr.execution_date.isoformat(), run_id=dr.run_id - ) - return response - - -@api_experimental.route("/dags/", methods=["DELETE"]) -@requires_authentication -def delete_dag(dag_id): - """Delete all DB records related to the specified Dag.""" - try: - count = delete.delete_dag(dag_id) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - return jsonify(message=f"Removed {count} record(s)", count=count) - - -@api_experimental.route("/dags//dag_runs", methods=["GET"]) -@requires_authentication -def dag_runs(dag_id): - """ - Return a list of Dag Runs for a specific DAG ID. - - :query param state: a query string parameter '?state=queued|running|success...' - - :param dag_id: String identifier of a DAG - - :return: List of DAG runs of a DAG with requested state, - or all runs if the state is not specified - """ - try: - state = request.args.get("state") - dagruns = get_dag_runs(dag_id, state) - except AirflowException as err: - log.info(err) - response = jsonify(error=f"{err}") - response.status_code = 400 - return response - - return jsonify(dagruns) - - -@api_experimental.route("/test", methods=["GET"]) -@requires_authentication -def test(): - """Test endpoint to check authentication.""" - return jsonify(status="OK") - - -@api_experimental.route("/info", methods=["GET"]) -@requires_authentication -def info(): - """Get Airflow Version.""" - return jsonify(version=version) - - -@api_experimental.route("/dags//code", methods=["GET"]) -@requires_authentication -def get_dag_code(dag_id): - """Return python code of a given dag_id.""" - try: - return get_code(dag_id) - except AirflowException as err: - log.info(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - - -@api_experimental.route("/dags//tasks/", methods=["GET"]) -@requires_authentication -def task_info(dag_id, task_id): - """Return a JSON with a task's public instance variables.""" - try: - t_info = get_task(dag_id, task_id) - except AirflowException as err: - log.info(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - - # JSONify and return. - fields = {k: str(v) for k, v in vars(t_info).items() if not k.startswith("_")} - return jsonify(fields) - - -# ToDo: Shouldn't this be a PUT method? -@api_experimental.route("/dags//paused/", methods=["GET"]) -@requires_authentication -def dag_paused(dag_id, paused): - """(Un)pause a dag.""" - is_paused = bool(paused == "true") - - models.DagModel.get_dagmodel(dag_id).set_is_paused( - is_paused=is_paused, - ) - - return jsonify({"response": "ok"}) - - -@api_experimental.route("/dags//paused", methods=["GET"]) -@requires_authentication -def dag_is_paused(dag_id): - """Get paused state of a dag.""" - is_paused = models.DagModel.get_dagmodel(dag_id).is_paused - - return jsonify({"is_paused": is_paused}) - - -@api_experimental.route( - "/dags//dag_runs//tasks/", methods=["GET"] -) -@requires_authentication -def task_instance_info(dag_id, execution_date, task_id): - """ - Return a JSON with a task instance's public instance variables. - - The format for the exec_date is expected to be - "YYYY-mm-DDTHH:MM:SS", for example: "2016-11-16T11:34:15". This will - of course need to have been encoded for URL in the request. - """ - # Convert string datetime into actual datetime - try: - execution_date = timezone.parse(execution_date) - except ValueError: - log.error("Given execution date could not be identified as a date.") - error_message = ( - f"Given execution date, {execution_date}, could not be identified as a date. " - f"Example date format: 2015-11-16T14:34:15+00:00" - ) - response = jsonify({"error": error_message}) - response.status_code = 400 - - return response - - try: - ti_info = get_task_instance(dag_id, task_id, execution_date) - except AirflowException as err: - log.info(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - - # JSONify and return. - fields = {k: str(v) for k, v in vars(ti_info).items() if not k.startswith("_")} - return jsonify(fields) - - -@api_experimental.route("/dags//dag_runs/", methods=["GET"]) -@requires_authentication -def dag_run_status(dag_id, execution_date): - """ - Return a JSON with a dag_run's public instance variables. - - The format for the exec_date is expected to be - "YYYY-mm-DDTHH:MM:SS", for example: "2016-11-16T11:34:15". This will - of course need to have been encoded for URL in the request. - """ - # Convert string datetime into actual datetime - try: - execution_date = timezone.parse(execution_date) - except ValueError: - log.error("Given execution date could not be identified as a date.") - error_message = ( - f"Given execution date, {execution_date}, could not be identified as a date. " - f"Example date format: 2015-11-16T14:34:15+00:00" - ) - response = jsonify({"error": error_message}) - response.status_code = 400 - - return response - - try: - dr_info = get_dag_run_state(dag_id, execution_date) - except AirflowException as err: - log.info(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - - return jsonify(dr_info) - - -@api_experimental.route("/latest_runs", methods=["GET"]) -@requires_authentication -def latest_dag_runs(): - """Return the latest DagRun for each DAG formatted for the UI.""" - from airflow.models import DagRun - - dagruns = DagRun.get_latest_runs() - payload = [] - for dagrun in dagruns: - if dagrun.execution_date: - payload.append( - { - "dag_id": dagrun.dag_id, - "execution_date": dagrun.execution_date.isoformat(), - "start_date": ((dagrun.start_date or "") and dagrun.start_date.isoformat()), - "dag_run_url": url_for( - "Airflow.graph", dag_id=dagrun.dag_id, execution_date=dagrun.execution_date - ), - } - ) - return jsonify(items=payload) # old flask versions don't support jsonifying arrays - - -@api_experimental.route("/pools/", methods=["GET"]) -@requires_authentication -def get_pool(name): - """Get pool by a given name.""" - try: - pool = pool_api.get_pool(name=name) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - else: - return jsonify(pool.to_json()) - - -@api_experimental.route("/pools", methods=["GET"]) -@requires_authentication -def get_pools(): - """Get all pools.""" - try: - pools = pool_api.get_pools() - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - else: - return jsonify([p.to_json() for p in pools]) - - -@api_experimental.route("/pools", methods=["POST"]) -@requires_authentication -def create_pool(): - """Create a pool.""" - params = request.get_json(force=True) - try: - pool = pool_api.create_pool(**params) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - else: - return jsonify(pool.to_json()) - - -@api_experimental.route("/pools/", methods=["DELETE"]) -@requires_authentication -def delete_pool(name): - """Delete pool.""" - try: - pool = pool_api.delete_pool(name=name) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - else: - return jsonify(pool.to_json()) - - -@api_experimental.route("/lineage//", methods=["GET"]) -@requires_authentication -def get_lineage(dag_id: str, execution_date: str): - """Get Lineage details for a DagRun.""" - # Convert string datetime into actual datetime - try: - execution_dt = timezone.parse(execution_date) - except ValueError: - log.error("Given execution date could not be identified as a date.") - error_message = ( - f"Given execution date, {execution_date}, could not be identified as a date. " - f"Example date format: 2015-11-16T14:34:15+00:00" - ) - response = jsonify({"error": error_message}) - response.status_code = 400 - - return response - - try: - lineage = get_lineage_api(dag_id=dag_id, execution_date=execution_dt) - except AirflowException as err: - log.error(err) - response = jsonify(error=f"{err}") - response.status_code = err.status_code - return response - else: - return jsonify(lineage) diff --git a/airflow/www/app.py b/airflow/www/app.py index e093e66cfd88..93c4e91d6d2a 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -44,7 +44,7 @@ from airflow.www.extensions.init_manifest_files import configure_manifest_files from airflow.www.extensions.init_robots import init_robots from airflow.www.extensions.init_security import ( - init_api_experimental_auth, + init_api_auth, init_cache_control, init_check_user_active, init_xframe_protection, @@ -54,7 +54,6 @@ init_api_auth_provider, init_api_connexion, init_api_error_handlers, - init_api_experimental, init_api_internal, init_appbuilder_views, init_error_handlers, @@ -150,7 +149,7 @@ def create_app(config=None, testing=False): init_dagbag(flask_app) - init_api_experimental_auth(flask_app) + init_api_auth(flask_app) init_robots(flask_app) @@ -175,7 +174,6 @@ def create_app(config=None, testing=False): if not _ENABLE_AIP_44: raise RuntimeError("The AIP_44 is not enabled so you cannot use it.") init_api_internal(flask_app) - init_api_experimental(flask_app) init_api_auth_provider(flask_app) init_api_error_handlers(flask_app) # needs to be after all api inits to let them add their path first diff --git a/airflow/www/extensions/init_security.py b/airflow/www/extensions/init_security.py index 8bf2c29fbfbe..6dc30a41a4cb 100644 --- a/airflow/www/extensions/init_security.py +++ b/airflow/www/extensions/init_security.py @@ -47,7 +47,7 @@ def apply_caching(response): app.after_request(apply_caching) -def init_api_experimental_auth(app): +def init_api_auth(app): """Load authentication backends.""" auth_backends = "airflow.api.auth.backend.default" try: diff --git a/airflow/www/extensions/init_views.py b/airflow/www/extensions/init_views.py index cc4e1141be70..a04116a7c7d4 100644 --- a/airflow/www/extensions/init_views.py +++ b/airflow/www/extensions/init_views.py @@ -17,7 +17,6 @@ from __future__ import annotations import logging -import warnings from functools import cached_property from pathlib import Path from typing import TYPE_CHECKING @@ -29,7 +28,6 @@ from airflow.api_connexion.exceptions import common_error_handler from airflow.configuration import conf -from airflow.exceptions import RemovedInAirflow3Warning from airflow.security import permissions from airflow.utils.yaml import safe_load from airflow.www.constants import SWAGGER_BUNDLE, SWAGGER_ENABLED @@ -314,24 +312,6 @@ def init_api_internal(app: Flask, standalone_api: bool = False) -> None: app.extensions["csrf"].exempt(api_bp) -def init_api_experimental(app): - """Initialize Experimental API.""" - if not conf.getboolean("api", "enable_experimental_api", fallback=False): - return - from airflow.www.api.experimental import endpoints - - warnings.warn( - "The experimental REST API is deprecated. Please migrate to the stable REST API. " - "Please note that the experimental API do not have access control. " - "The authenticated user has full access.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - base_paths.append("/api/experimental") - app.register_blueprint(endpoints.api_experimental, url_prefix="/api/experimental") - app.extensions["csrf"].exempt(endpoints.api_experimental) - - def init_api_auth_provider(app): """Initialize the API offered by the auth manager.""" auth_mgr = get_auth_manager() diff --git a/airflow/www/package.json b/airflow/www/package.json index 7e494cadd4cf..0abf0f5f569a 100644 --- a/airflow/www/package.json +++ b/airflow/www/package.json @@ -107,7 +107,7 @@ "@visx/group": "^2.10.0", "@visx/shape": "^2.12.2", "ansi_up": "^6.0.2", - "axios": "^1.6.0", + "axios": "^1.7.4", "bootstrap-3-typeahead": "^4.0.2", "camelcase-keys": "^7.0.0", "chakra-react-select": "^4.0.0", diff --git a/airflow/www/static/js/connection_form.js b/airflow/www/static/js/connection_form.js index 57cee44a6aba..886ff208d71f 100644 --- a/airflow/www/static/js/connection_form.js +++ b/airflow/www/static/js/connection_form.js @@ -234,11 +234,11 @@ $(document).ready(() => { /** * Displays the Flask style alert on UI via JS * - * @param {boolean} status - true for success, false for error + * @param {string} status - Status can be either success, error, or warning * @param {string} message - The text message to show in alert box */ function displayAlert(status, message) { - const alertClass = status ? "alert-success" : "alert-error"; + const alertClass = `alert-${status}`; let alertBox = $(".container .row .alert"); if (alertBox.length) { alertBox.removeClass("alert-success").removeClass("alert-error"); @@ -255,6 +255,11 @@ $(document).ready(() => { } } + displayAlert( + "warning", + "Warning: Fields that are currently populated can be modified but cannot be deleted. To delete data from a field, delete the Connection object and create a new one." + ); + function hideAlert() { const alertBox = $(".container .row .alert"); alertBox.hide(); @@ -303,7 +308,7 @@ $(document).ready(() => { extra = JSON.parse(this.value); } catch (e) { if (e instanceof SyntaxError) { - displayAlert(false, "Extra field value is not valid JSON."); + displayAlert("error", "Extra field value is not valid JSON."); } throw e; } @@ -345,10 +350,10 @@ $(document).ready(() => { dataType: "json", data: getSerializedFormData("form#model_form"), success(data) { - displayAlert(data.status, data.message); + displayAlert("success", data.message); }, error(jq, err, msg) { - displayAlert(false, msg); + displayAlert("error", msg); }, }); }); diff --git a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx index e24088085a9f..22d7fdd696fa 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx @@ -20,7 +20,7 @@ import React, { forwardRef } from "react"; import { Flex } from "@chakra-ui/react"; -import { getMetaValue, appendSearchParams } from "src/utils"; +import { getMetaValue } from "src/utils"; import LinkButton from "src/components/LinkButton"; import type { Task } from "src/types"; import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; @@ -28,18 +28,16 @@ import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; const dagId = getMetaValue("dag_id"); const taskInstancesUrl = getMetaValue("task_instances_list_url"); const taskUrl = getMetaValue("task_url"); -const gridUrl = getMetaValue("grid_url"); interface Props { taskId: Task["id"]; executionDate: string; - operator?: string; isMapped?: boolean; mapIndex?: number; } const Nav = forwardRef( - ({ taskId, executionDate, operator, isMapped = false, mapIndex }, ref) => { + ({ taskId, executionDate, isMapped = false, mapIndex }, ref) => { if (!taskId) return null; const params = new URLSearchParamsWrapper({ task_id: taskId, @@ -52,32 +50,16 @@ const Nav = forwardRef( _flt_3_task_id: taskId, _oc_TaskInstanceModelView: "dag_run.execution_date", }); - const subDagParams = new URLSearchParamsWrapper({ - execution_date: executionDate, - }).toString(); if (mapIndex !== undefined && mapIndex >= 0) listParams.append("_flt_0_map_index", mapIndex.toString()); const allInstancesLink = `${taskInstancesUrl}?${listParams.toString()}`; - const subDagLink = appendSearchParams( - gridUrl.replace(dagId, `${dagId}.${taskId}`), - subDagParams - ); - - // TODO: base subdag zooming as its own attribute instead of via operator name - const isSubDag = operator === "SubDagOperator"; - return ( {(!isMapped || mapIndex !== undefined) && ( - <> - More Details - {isSubDag && ( - Zoom into SubDag - )} - + More Details )} List All Instances diff --git a/airflow/www/static/js/dag/details/taskInstance/TrySelector.tsx b/airflow/www/static/js/dag/details/taskInstance/TrySelector.tsx index 5fc96803c85d..31d894fcaaa4 100644 --- a/airflow/www/static/js/dag/details/taskInstance/TrySelector.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/TrySelector.tsx @@ -61,63 +61,10 @@ const TrySelector = ({ const logAttemptDropdownLimit = 10; const showDropdown = finalTryNumber > logAttemptDropdownLimit; - const tries = (tiHistory?.taskInstances || []).filter( - (t) => t?.startDate !== taskInstance?.startDate - ); - tries?.push(taskInstance); - return ( Task Tries - {!showDropdown && ( - - {/* Even without try history showing up we should still show all try numbers */} - {Array.from({ length: finalTryNumber }, (_, i) => i + 1).map( - (tryNumber, i) => { - let attempt; - if (tries.length) { - attempt = tries[i]; - } - return ( - - Status: {attempt.state} - - Duration:{" "} - {formatDuration( - getDuration(attempt.startDate, attempt.endDate) - )} - - - ) - } - hasArrow - portalProps={{ containerRef }} - placement="top" - isDisabled={!attempt} - > - - - ); - } - )} - - )} - {showDropdown && ( + {showDropdown ? ( + ) : ( + + {tiHistory?.taskInstances?.map((ti) => ( + + Status: {ti.state} + + Duration:{" "} + {formatDuration(getDuration(ti.startDate, ti.endDate))} + + + } + hasArrow + portalProps={{ containerRef }} + placement="top" + isDisabled={!ti} + > + + + ))} + )} ); diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 74f317867aea..44311249b0ce 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -54,7 +54,6 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { const children = group?.children; const isMapped = group?.isMapped; - const operator = group?.operator; const isMappedTaskSummary = !!isMapped && !isMapIndexDefined && taskId; const isGroup = !!children; @@ -90,7 +89,6 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { isMapped={isMapped} mapIndex={mapIndex} executionDate={run?.executionDate} - operator={operator} /> )} {!isGroupOrMappedTaskSummary && } diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index a892e327ace0..30948df332bb 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1031,8 +1031,6 @@ export interface components { * *New in version 2.9.0* */ dag_display_name?: string; - /** @description If the DAG is SubDAG then it is the top level DAG identifier. Otherwise, null. */ - root_dag_id?: string | null; /** @description Whether the DAG is paused. */ is_paused?: boolean | null; /** @@ -1043,8 +1041,6 @@ export interface components { * *Changed in version 2.2.0*: Field is read-only. */ is_active?: boolean | null; - /** @description Whether the DAG is SubDAG. */ - is_subdag?: boolean; /** * Format: date-time * @description The last time the DAG was parsed. @@ -2090,10 +2086,6 @@ export interface components { * @default false */ only_running?: boolean; - /** @description Clear tasks in subdags and clear external tasks indicated by ExternalTaskMarker. */ - include_subdags?: boolean; - /** @description Clear tasks in the parent dag of the subdag. */ - include_parentdag?: boolean; /** @description Set state of DAG runs to RUNNING. */ reset_dag_runs?: boolean; /** @description The DagRun ID for this task instance */ diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index d3a7995440c0..973ca812e3fb 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -110,35 +110,25 @@ {% endblock %} {% block content %} - {% if dag.parent_dag is defined and dag.parent_dag %} - - - DAG: {{ dag.parent_dag.dag_display_name }} - {% endif %} -

- {% if dag.parent_dag is defined and dag.parent_dag %} - SUBDAG: {{ dag.dag_id }} + {% if can_edit_dag %} + {% set switch_tooltip = 'Pause/Unpause DAG' %} {% else %} - {% if can_edit_dag %} - {% set switch_tooltip = 'Pause/Unpause DAG' %} - {% else %} - {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} - {% endif %} - - DAG: {{ dag.dag_display_name }} - {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} - {% if dag_model is defined and dag_model.max_consecutive_failed_dag_runs is defined and dag_model.max_consecutive_failed_dag_runs > 0 %} - - {% endif %} + {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} + {% endif %} + + DAG: {{ dag.dag_display_name }} + {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} + {% if dag_model is defined and dag_model.max_consecutive_failed_dag_runs is defined and dag_model.max_consecutive_failed_dag_runs > 0 %} + {% endif %}

diff --git a/airflow/www/views.py b/airflow/www/views.py index a485f84ed4b1..d7b670303c86 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -844,7 +844,7 @@ 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 = select(DagModel).where(DagModel.is_active) if arg_search_query: escaped_arg_search_query = arg_search_query.replace("_", r"\_") @@ -2341,8 +2341,6 @@ def _clear_dag_tis( start_date=start_date, end_date=end_date, task_ids=task_ids, - include_subdags=recursive, - include_parentdag=recursive, only_failed=only_failed, session=session, ) @@ -2355,8 +2353,6 @@ def _clear_dag_tis( start_date=start_date, end_date=end_date, task_ids=task_ids, - include_subdags=recursive, - include_parentdag=recursive, only_failed=only_failed, dry_run=True, session=session, @@ -5488,8 +5484,6 @@ def _clear_task_instances( start_date=dag_run.execution_date, end_date=dag_run.execution_date, task_ids=downstream_task_ids_to_clear, - include_subdags=False, - include_parentdag=False, session=session, dry_run=True, ) @@ -5638,7 +5632,6 @@ def autocomplete(self, session: Session = NEW_SESSION): DagModel.dag_id.label("name"), DagModel._dag_display_property_value.label("dag_display_name"), ).where( - ~DagModel.is_subdag, DagModel.is_active, or_( DagModel.dag_id.ilike(f"%{query}%"), @@ -5653,7 +5646,7 @@ def autocomplete(self, session: Session = NEW_SESSION): sqla.literal(None).label("dag_display_name"), ) .distinct() - .where(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) + .where(DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) ) # Hide DAGs if not showing status: "all" diff --git a/airflow/www/yarn.lock b/airflow/www/yarn.lock index 20d9ce752095..1aa3e725bb8c 100644 --- a/airflow/www/yarn.lock +++ b/airflow/www/yarn.lock @@ -4269,12 +4269,12 @@ axe-core@^4.3.5: resolved "https://registry.yarnpkg.com/axe-core/-/axe-core-4.4.2.tgz#dcf7fb6dea866166c3eab33d68208afe4d5f670c" integrity sha512-LVAaGp/wkkgYJcjmHsoKx4juT1aQvJyPcW09MLCjVTh3V2cc6PnyempiLMNH5iMdfIX/zdbjUx2KDjMLCTdPeA== -axios@^1.6.0: - version "1.6.1" - resolved "https://registry.yarnpkg.com/axios/-/axios-1.6.1.tgz#76550d644bf0a2d469a01f9244db6753208397d7" - integrity sha512-vfBmhDpKafglh0EldBEbVuoe7DyAavGSLWhuSm5ZSEKQnHhBf0xAAwybbNH1IkrJNGnS/VG4I5yxig1pCEXE4g== +axios@^1.7.4: + version "1.7.4" + resolved "https://registry.yarnpkg.com/axios/-/axios-1.7.4.tgz#4c8ded1b43683c8dd362973c393f3ede24052aa2" + integrity sha512-DukmaFRnY6AzAALSH4J2M3k6PkaC+MfaAGdEERRWcC9q3/TWQwLpHR8ZRLKTdQ3aBDL64EdluRDjJqKw+BPZEw== dependencies: - follow-redirects "^1.15.0" + follow-redirects "^1.15.6" form-data "^4.0.0" proxy-from-env "^1.1.0" @@ -6592,7 +6592,7 @@ focus-lock@^0.11.6: dependencies: tslib "^2.0.3" -follow-redirects@^1.15.0: +follow-redirects@^1.15.6: version "1.15.6" resolved "https://registry.yarnpkg.com/follow-redirects/-/follow-redirects-1.15.6.tgz#7f815c0cda4249c74ff09e95ef97c23b5fd0399b" integrity sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA== diff --git a/chart/Chart.yaml b/chart/Chart.yaml index 1bba08b73b93..7e1d5f5c54da 100644 --- a/chart/Chart.yaml +++ b/chart/Chart.yaml @@ -20,7 +20,7 @@ apiVersion: v2 name: airflow version: 1.16.0-dev -appVersion: 2.9.3 +appVersion: 2.10.0 description: The official Helm chart to deploy Apache Airflow, a platform to programmatically author, schedule, and monitor workflows home: https://airflow.apache.org/ @@ -47,23 +47,23 @@ annotations: url: https://airflow.apache.org/docs/helm-chart/1.16.0/ artifacthub.io/screenshots: | - title: DAGs View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/dags.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/dags.png - title: Datasets View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/datasets.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/datasets.png - title: Grid View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/grid.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/grid.png - title: Graph View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/graph.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/graph.png - title: Calendar View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/calendar.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/calendar.png - title: Variable View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/variable_hidden.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/variable_hidden.png - title: Gantt Chart - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/gantt.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/gantt.png - title: Task Duration - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/duration.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/duration.png - title: Code View - url: https://airflow.apache.org/docs/apache-airflow/2.9.3/_images/code.png + url: https://airflow.apache.org/docs/apache-airflow/2.10.0/_images/code.png artifacthub.io/changes: | - description: Add git-sync container lifecycle hooks kind: added @@ -132,7 +132,7 @@ annotations: links: - name: '#40318' url: https://github.com/apache/airflow/pull/40318 - - description: 'Misc: Default airflow version to 2.9.3' + - description: 'Misc: Default airflow version to 2.10.0' kind: changed links: - name: '#40816' diff --git a/chart/newsfragments/41529.significant.rst b/chart/newsfragments/41529.significant.rst new file mode 100644 index 000000000000..9eb0ae8573f9 --- /dev/null +++ b/chart/newsfragments/41529.significant.rst @@ -0,0 +1,3 @@ +Default Airflow image is updated to ``2.10.0`` + +The default Airflow image that is used with the Chart is now ``2.10.0``, previously it was ``2.9.3``. diff --git a/chart/templates/NOTES.txt b/chart/templates/NOTES.txt index 2e96cfa80ca4..846159d6ac65 100644 --- a/chart/templates/NOTES.txt +++ b/chart/templates/NOTES.txt @@ -78,7 +78,7 @@ Airflow Webserver: {{- $hostname = .name -}} {{- end }} {{- end }} - http{{ if $tlsEnabled }}s{{ end }}://{{ $hostname }}{{ $.Values.ingress.web.path }}/ + http{{ if $tlsEnabled }}s{{ end }}://{{ (tpl $hostname $) }}{{ $.Values.ingress.web.path }}/ {{- end }} {{- end }} {{- if and (or .Values.ingress.flower.enabled .Values.ingress.enabled) (or (eq .Values.executor "CeleryExecutor") (eq .Values.executor "CeleryKubernetesExecutor")) }} @@ -92,7 +92,7 @@ Flower dashboard: {{- $hostname = .name -}} {{- end }} {{- end }} - http{{ if $tlsEnabled }}s{{ end }}://{{ $hostname }}{{ $.Values.ingress.flower.path }}/ + http{{ if $tlsEnabled }}s{{ end }}://{{ (tpl $hostname $) }}{{ $.Values.ingress.flower.path }}/ {{- end }} {{- end }} {{- else }} diff --git a/chart/templates/redis/redis-statefulset.yaml b/chart/templates/redis/redis-statefulset.yaml index cbc504e0a584..db98f0bf4d51 100644 --- a/chart/templates/redis/redis-statefulset.yaml +++ b/chart/templates/redis/redis-statefulset.yaml @@ -77,6 +77,7 @@ spec: affinity: {{- toYaml $affinity | nindent 8 }} tolerations: {{- toYaml $tolerations | nindent 8 }} topologySpreadConstraints: {{- toYaml $topologySpreadConstraints | nindent 8 }} + terminationGracePeriodSeconds: {{ .Values.redis.terminationGracePeriodSeconds }} serviceAccountName: {{ include "redis.serviceAccountName" . }} {{- if .Values.schedulerName }} schedulerName: {{ .Values.schedulerName }} diff --git a/chart/templates/scheduler/scheduler-deployment.yaml b/chart/templates/scheduler/scheduler-deployment.yaml index ac1e2a824087..634c3a41f7de 100644 --- a/chart/templates/scheduler/scheduler-deployment.yaml +++ b/chart/templates/scheduler/scheduler-deployment.yaml @@ -131,7 +131,7 @@ spec: tolerations: {{- toYaml $tolerations | nindent 8 }} topologySpreadConstraints: {{- toYaml $topologySpreadConstraints | nindent 8 }} restartPolicy: Always - terminationGracePeriodSeconds: 10 + terminationGracePeriodSeconds: {{ .Values.scheduler.terminationGracePeriodSeconds }} serviceAccountName: {{ include "scheduler.serviceAccountName" . }} securityContext: {{ $securityContext | nindent 8 }} {{- if or .Values.registry.secretName .Values.registry.connection }} diff --git a/chart/templates/statsd/statsd-deployment.yaml b/chart/templates/statsd/statsd-deployment.yaml index dd7723f30283..b97c98d2afdd 100644 --- a/chart/templates/statsd/statsd-deployment.yaml +++ b/chart/templates/statsd/statsd-deployment.yaml @@ -82,6 +82,7 @@ spec: schedulerName: {{ .Values.schedulerName }} {{- end }} topologySpreadConstraints: {{- toYaml $topologySpreadConstraints | nindent 8 }} + terminationGracePeriodSeconds: {{ .Values.statsd.terminationGracePeriodSeconds }} serviceAccountName: {{ include "statsd.serviceAccountName" . }} securityContext: {{ $securityContext | nindent 8 }} restartPolicy: Always diff --git a/chart/templates/webserver/webserver-deployment.yaml b/chart/templates/webserver/webserver-deployment.yaml index 45095a28e652..5e1af9401280 100644 --- a/chart/templates/webserver/webserver-deployment.yaml +++ b/chart/templates/webserver/webserver-deployment.yaml @@ -127,6 +127,7 @@ spec: tolerations: {{- toYaml $tolerations | nindent 8 }} topologySpreadConstraints: {{- toYaml $topologySpreadConstraints | nindent 8 }} restartPolicy: Always + terminationGracePeriodSeconds: {{ .Values.webserver.terminationGracePeriodSeconds }} securityContext: {{ $securityContext | nindent 8 }} {{- if or .Values.registry.secretName .Values.registry.connection }} imagePullSecrets: diff --git a/chart/values.schema.json b/chart/values.schema.json index bec09a84bf14..2d2515013f7a 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -78,7 +78,7 @@ "defaultAirflowTag": { "description": "Default airflow tag to deploy.", "type": "string", - "default": "2.9.3", + "default": "2.10.0", "x-docsSection": "Common" }, "defaultAirflowDigest": { @@ -93,7 +93,7 @@ "airflowVersion": { "description": "Airflow version (Used to make some decisions based on Airflow Version being deployed).", "type": "string", - "default": "2.9.3", + "default": "2.10.0", "x-docsSection": "Common" }, "securityContext": { @@ -2404,6 +2404,11 @@ ], "default": null }, + "terminationGracePeriodSeconds": { + "description": "Grace period for scheduler to finish after SIGTERM is sent from Kubernetes.", + "type": "integer", + "default": 10 + }, "serviceAccount": { "description": "Create ServiceAccount.", "type": "object", @@ -5363,6 +5368,11 @@ ], "default": null }, + "terminationGracePeriodSeconds": { + "description": "Grace period for webserver to finish after SIGTERM is sent from Kubernetes.", + "type": "integer", + "default": 30 + }, "serviceAccount": { "description": "Create ServiceAccount.", "type": "object", @@ -6546,6 +6556,11 @@ } } }, + "terminationGracePeriodSeconds": { + "description": "Grace period for statsd to finish after SIGTERM is sent from Kubernetes.", + "type": "integer", + "default": 30 + }, "serviceAccount": { "description": "Create ServiceAccount.", "type": "object", diff --git a/chart/values.yaml b/chart/values.yaml index 9d8794f3db93..fe059cab9654 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -68,13 +68,13 @@ airflowHome: /opt/airflow defaultAirflowRepository: apache/airflow # Default airflow tag to deploy -defaultAirflowTag: "2.9.3" +defaultAirflowTag: "2.10.0" # Default airflow digest. If specified, it takes precedence over tag defaultAirflowDigest: ~ # Airflow version (Used to make some decisions based on Airflow Version being deployed) -airflowVersion: "2.9.3" +airflowVersion: "2.10.0" # Images images: @@ -863,6 +863,9 @@ scheduler: # container level lifecycle hooks containerLifecycleHooks: {} + # Grace period for tasks to finish after SIGTERM is sent from kubernetes + terminationGracePeriodSeconds: 10 + # Create ServiceAccount serviceAccount: # default value is true @@ -1324,6 +1327,9 @@ webserver: # Args to use when running the Airflow webserver (templated). args: ["bash", "-c", "exec airflow webserver"] + # Grace period for webserver to finish after SIGTERM is sent from kubernetes + terminationGracePeriodSeconds: 30 + # Create ServiceAccount serviceAccount: # default value is true @@ -1992,6 +1998,9 @@ statsd: # Annotations to add to the StatsD Deployment. annotations: {} + # Grace period for statsd to finish after SIGTERM is sent from kubernetes + terminationGracePeriodSeconds: 30 + # Create ServiceAccount serviceAccount: # default value is true diff --git a/contributing-docs/12_airflow_dependencies_and_extras.rst b/contributing-docs/12_airflow_dependencies_and_extras.rst index 50854799ca8a..2bd2c1f95303 100644 --- a/contributing-docs/12_airflow_dependencies_and_extras.rst +++ b/contributing-docs/12_airflow_dependencies_and_extras.rst @@ -164,9 +164,9 @@ Those extras are available as regular core airflow extras - they install optiona .. START CORE EXTRAS HERE -aiobotocore, apache-atlas, apache-webhdfs, async, cgroups, cloudpickle, deprecated-api, github- -enterprise, google-auth, graphviz, kerberos, ldap, leveldb, otel, pandas, password, pydantic, -rabbitmq, s3fs, saml, sentry, statsd, uv, virtualenv +aiobotocore, apache-atlas, apache-webhdfs, async, cgroups, cloudpickle, github-enterprise, google- +auth, graphviz, kerberos, ldap, leveldb, otel, pandas, password, pydantic, rabbitmq, s3fs, saml, +sentry, statsd, uv, virtualenv .. END CORE EXTRAS HERE diff --git a/contributing-docs/testing/unit_tests.rst b/contributing-docs/testing/unit_tests.rst index c83f391e5281..8d27db19c2d5 100644 --- a/contributing-docs/testing/unit_tests.rst +++ b/contributing-docs/testing/unit_tests.rst @@ -96,7 +96,7 @@ test types you want to use in various ``breeze testing`` sub-commands in three w Those test types are defined: * ``Always`` - those are tests that should be always executed (always sub-folder) -* ``API`` - Tests for the Airflow API (api, api_connexion, api_experimental and api_internal sub-folders) +* ``API`` - Tests for the Airflow API (api, api_connexion and api_internal sub-folders) * ``CLI`` - Tests for the Airflow CLI (cli folder) * ``Core`` - for the core Airflow functionality (core, executors, jobs, models, ti_deps, utils sub-folders) * ``Operators`` - tests for the operators (operators folder with exception of Virtualenv Operator tests and @@ -1226,6 +1226,12 @@ Running provider compatibility tests in CI In CI those tests are run in a slightly more complex way because we want to run them against the build provider packages, rather than mounted from sources. +In case of canary runs we add ``--clean-airflow-installation`` flag that removes all packages before +installing older airflow version, and then installs development dependencies +from latest airflow - in order to avoid case where a provider depends on a new dependency added in latest +version of Airflow. This clean removal and re-installation takes quite some time though and in order to +speed up the tests in regular PRs we only do that in the canary runs. + The exact way CI tests are run can be reproduced locally building providers from selected tag/commit and using them to install and run tests against the selected airflow version. @@ -1262,6 +1268,14 @@ Herr id how to reproduce it. breeze shell --use-packages-from-dist --package-format wheel --use-airflow-version 2.9.1 \ --install-airflow-with-constraints --providers-skip-constraints --mount-sources tests +In case you want to reproduce canary run, you need to add ``--clean-airflow-installation`` flag: + +.. code-block:: bash + + breeze shell --use-packages-from-dist --package-format wheel --use-airflow-version 2.9.1 \ + --install-airflow-with-constraints --providers-skip-constraints --mount-sources tests --clean-airflow-installation + + 6. You can then run tests as usual: .. code-block:: bash diff --git a/dev/README_RELEASE_AIRFLOW.md b/dev/README_RELEASE_AIRFLOW.md index 19d2bcec04be..e0e014cfcb13 100644 --- a/dev/README_RELEASE_AIRFLOW.md +++ b/dev/README_RELEASE_AIRFLOW.md @@ -549,7 +549,7 @@ Optionally you can use `check_files.py` script to verify that all expected files present in SVN. This script may help also with verifying installation of the packages. ```shell script -python check_files.py airflow -v {VERSION} -p {PATH_TO_SVN} +python check_files.py airflow -v ${VERSION} -p {PATH_TO_SVN} ``` ## Licence check diff --git a/dev/breeze/doc/ci/05_workflows.md b/dev/breeze/doc/ci/05_workflows.md index 9ea39709c943..b70a81ef64ce 100644 --- a/dev/breeze/doc/ci/05_workflows.md +++ b/dev/breeze/doc/ci/05_workflows.md @@ -227,7 +227,6 @@ code. | Build CI images | Builds images in-workflow (not in the build images) | | Yes | Yes (1) | Yes (4) | | Generate constraints/CI verify | Generate constraints for the build and verify CI image | Yes (2) | Yes (2) | Yes (2) | Yes (2) | | Build PROD images | Builds images in-workflow (not in the build images) | | Yes | Yes (1) | Yes (4) | -| Build Bullseye PROD images | Builds images based on Bullseye debian | | Yes | Yes | Yes | | Run breeze tests | Run unit tests for Breeze | Yes | Yes | Yes | Yes | | Test OpenAPI client gen | Tests if OpenAPIClient continues to generate | Yes | Yes | Yes | Yes | | React WWW tests | React UI tests for new Airflow UI | Yes | Yes | Yes | Yes | diff --git a/dev/breeze/doc/ci/06_diagrams.md b/dev/breeze/doc/ci/06_diagrams.md index 89d6fc772c86..afe51a309e8e 100644 --- a/dev/breeze/doc/ci/06_diagrams.md +++ b/dev/breeze/doc/ci/06_diagrams.md @@ -379,7 +379,6 @@ sequenceDiagram Tests ->> GitHub Registry: Push PROD Images
[COMMIT_SHA] and Artifacts ->> Tests: Download source constraints - Note over Tests: Build Bullseye PROD Images
[COMMIT_SHA] and GitHub Registry ->> Tests: Pull CI Images
[COMMIT_SHA] Note over Tests: Run static checks diff --git a/dev/breeze/doc/images/output_ci-image_build.svg b/dev/breeze/doc/images/output_ci-image_build.svg index d2089f482e0e..131b618e403c 100644 --- a/dev/breeze/doc/images/output_ci-image_build.svg +++ b/dev/breeze/doc/images/output_ci-image_build.svg @@ -1,4 +1,4 @@ - +