diff --git a/.github/actions/migration_tests/action.yml b/.github/actions/migration_tests/action.yml index 34951bac5ffe0..93e966de8a847 100644 --- a/.github/actions/migration_tests/action.yml +++ b/.github/actions/migration_tests/action.yml @@ -18,6 +18,10 @@ --- name: 'Run migration tests' description: 'Runs migration tests' +inputs: + python-version: + description: "Python version to run the tests on" + required: true runs: using: "composite" steps: @@ -38,12 +42,18 @@ runs: airflow db migrate --to-revision heads && airflow db downgrade -n 2.7.0 -y && airflow db migrate - if: env.BACKEND != 'sqlite' + # migration tests cannot be run with Python 3.13 now - currently we have no FAB and no FABDBManager - + # and airflow (correctly) refuses to migrate things to Airflow 2 when there is no "ab_user" + # table created. So migration tests for now will have to be excluded for Python 3.13 until + # we start working on 3.2 (with migration to 3.1) or until FAB is supported in 3.13 (FAB 5) + # TODO(potiuk) bring migration tests back for Python 3.13 when one of the two conditions are fulfilled + if: env.BACKEND != 'sqlite' && inputs.python-version != '3.13' - name: "Bring composer down" shell: bash run: breeze down env: COMPOSE_PROJECT_NAME: "docker-compose" + if: inputs.python-version != '3.13' - name: "Test ORM migration 2 to 3: ${{env.BACKEND}}" shell: bash run: > @@ -60,12 +70,13 @@ runs: airflow db migrate --to-revision heads && airflow db downgrade -n 2.7.0 -y && airflow db migrate - if: env.BACKEND != 'sqlite' + if: env.BACKEND != 'sqlite' && inputs.python-version != '3.13' - name: "Bring compose down again" shell: bash run: breeze down env: COMPOSE_PROJECT_NAME: "docker-compose" + if: inputs.python-version != '3.13' - name: "Test ORM migration ${{env.BACKEND}}" shell: bash run: > @@ -77,11 +88,13 @@ runs: env: COMPOSE_PROJECT_NAME: "docker-compose" DB_MANAGERS: "airflow.providers.fab.auth_manager.models.db.FABDBManager" + if: inputs.python-version != '3.13' - name: "Bring compose down again" shell: bash run: breeze down env: COMPOSE_PROJECT_NAME: "docker-compose" + if: inputs.python-version != '3.13' - name: "Test offline migration ${{env.BACKEND}}" shell: bash run: > @@ -93,12 +106,13 @@ runs: env: COMPOSE_PROJECT_NAME: "docker-compose" DB_MANAGERS: "airflow.providers.fab.auth_manager.models.db.FABDBManager" - if: env.BACKEND != 'sqlite' + if: env.BACKEND != 'sqlite' && inputs.python-version != '3.13' - name: "Bring any containers left down" shell: bash run: breeze down env: COMPOSE_PROJECT_NAME: "docker-compose" + if: inputs.python-version != '3.13' - name: "Dump logs on failure ${{env.BACKEND}}" shell: bash run: docker ps -q | xargs docker logs diff --git a/.github/actions/post_tests_success/action.yml b/.github/actions/post_tests_success/action.yml index 36ee429477733..234cde900e4d8 100644 --- a/.github/actions/post_tests_success/action.yml +++ b/.github/actions/post_tests_success/action.yml @@ -47,7 +47,9 @@ runs: uses: codecov/codecov-action@b9fd7d16f6d7d1b5d2bec1a2887e65ceed900238 env: CODECOV_TOKEN: ${{ inputs.codecov-token }} - if: env.ENABLE_COVERAGE == 'true' && env.TEST_TYPES != 'Helm' && inputs.python-version != '3.12' + if: > + env.ENABLE_COVERAGE == 'true' && env.TEST_TYPES != 'Helm' && inputs.python-version != '3.12' + && inputs.python-version != '3.13' with: name: coverage-${{env.JOB_ID}} flags: python-${{ env.PYTHON_MAJOR_MINOR_VERSION }},${{ env.BACKEND }}-${{ env.BACKEND_VERSION }} diff --git a/.github/actions/prepare_all_ci_images/action.yml b/.github/actions/prepare_all_ci_images/action.yml index f245d5081d5d8..7e2e8395636eb 100644 --- a/.github/actions/prepare_all_ci_images/action.yml +++ b/.github/actions/prepare_all_ci_images/action.yml @@ -61,3 +61,9 @@ runs: platform: ${{ inputs.platform }} python: "3.12" python-versions-list-as-string: ${{ inputs.python-versions-list-as-string }} + - name: "Restore CI docker image ${{ inputs.platform }}:3.13" + uses: ./.github/actions/prepare_single_ci_image + with: + platform: ${{ inputs.platform }} + python: "3.13" + python-versions-list-as-string: ${{ inputs.python-versions-list-as-string }} diff --git a/.github/workflows/run-unit-tests.yml b/.github/workflows/run-unit-tests.yml index 035248113dac1..aa7d124648c6e 100644 --- a/.github/workflows/run-unit-tests.yml +++ b/.github/workflows/run-unit-tests.yml @@ -125,8 +125,11 @@ permissions: jobs: tests: timeout-minutes: 65 + # yamllint disable rule:line-length name: "\ - ${{ inputs.test-scope }}-${{ inputs.test-group }}:\ + ${{ inputs.test-scope == 'All' && '' || inputs.test-scope == 'Quarantined' && 'Qrnt' || inputs.test-scope }}\ + ${{ inputs.test-scope == 'All' && '' || '-' }}\ + ${{ inputs.test-group == 'providers' && 'prov' || inputs.test-group}}:\ ${{ inputs.test-name }}${{ inputs.test-name-separator }}${{ matrix.backend-version }}:\ ${{ matrix.python-version}}:${{ matrix.test-types.description }}" runs-on: ${{ fromJSON(inputs.runners) }} @@ -179,6 +182,8 @@ jobs: - name: > Migration Tests: ${{ matrix.python-version }}:${{ env.PARALLEL_TEST_TYPES }} uses: ./.github/actions/migration_tests + with: + python-version: ${{ matrix.python-version }} if: inputs.run-migration-tests == 'true' && inputs.test-group == 'core' - name: > ${{ inputs.test-group }}:${{ inputs.test-scope }} Tests ${{ inputs.test-name }} ${{ matrix.backend-version }} diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 6cbe623cd57f4..2072fc7bbd6a1 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -513,7 +513,7 @@ repos: entry: ./scripts/ci/pre_commit/check_common_sql_dependency.py language: python files: ^providers/.*/src/airflow/providers/.*/hooks/.*\.py$ - additional_dependencies: ['rich>=12.4.4', 'pyyaml>=6.0.2', 'packaging>=23.2'] + additional_dependencies: ['rich>=12.4.4', 'pyyaml>=6.0.2', 'packaging>=25'] - id: check-extra-packages-references name: Checks setup extra packages description: Checks if all the extras defined in hatch_build.py are listed in extra-packages-ref.rst file @@ -576,7 +576,7 @@ repos: ^providers/fab/src/airflow/providers/fab/migrations/versions/.*$|^providers/fab/src/airflow/providers/fab/migrations/versions| ^airflow-core/src/airflow/utils/db\.py$| ^providers/fab/src/airflow/providers/fab/auth_manager/models/db\.py$ - additional_dependencies: ['packaging>=23.2', 'rich>=12.4.4'] + additional_dependencies: ['packaging>=25', 'rich>=12.4.4'] - id: update-version name: Update versions in docs entry: ./scripts/ci/pre_commit/update_versions.py @@ -1014,10 +1014,15 @@ repos: name: Update Airflow's meta-package pyproject.toml language: python entry: ./scripts/ci/pre_commit/update_airflow_pyproject_toml.py - files: ^.*/pyproject\.toml$|^scripts/ci/pre_commit/update_airflow_pyproject_toml\.py$ + files: > + (?x) + ^.*/pyproject\.toml$| + ^scripts/ci/pre_commit/update_airflow_pyproject_toml\.py$| + ^providers/.*/pyproject\.toml$| + ^providers/.*/provider\.yaml$ pass_filenames: false require_serial: true - additional_dependencies: ['rich>=12.4.4', 'tomli>=2.0.1', 'packaging>=23.2' ] + additional_dependencies: ['rich>=12.4.4', 'tomli>=2.0.1', 'packaging>=25' ] - id: update-reproducible-source-date-epoch name: Update Source Date Epoch for reproducible builds language: python diff --git a/Dockerfile.ci b/Dockerfile.ci index eafd6dcc4722e..8c55e55bfdf66 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -871,10 +871,15 @@ EOF # The content below is automatically copied from scripts/docker/entrypoint_ci.sh COPY <<"EOF" /entrypoint_ci.sh #!/usr/bin/env bash -if [[ ${VERBOSE_COMMANDS:="false"} == "true" ]]; then - set -x -fi +function set_verbose() { + if [[ ${VERBOSE_COMMANDS:="false"} == "true" ]]; then + set -x + else + set +x + fi +} +set_verbose . "${AIRFLOW_SOURCES:-/opt/airflow}"/scripts/in_container/_in_container_script_init.sh LD_PRELOAD="/usr/lib/$(uname -m)-linux-gnu/libstdc++.so.6" @@ -1103,12 +1108,10 @@ function check_boto_upgrade() { echo echo "${COLOR_BLUE}Upgrading boto3, botocore to latest version to run Amazon tests with them${COLOR_RESET}" echo - set -x # shellcheck disable=SC2086 ${PACKAGING_TOOL_CMD} uninstall ${EXTRA_UNINSTALL_FLAGS} aiobotocore s3fs || true # shellcheck disable=SC2086 ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade "boto3<1.38.3" "botocore<1.38.3" - set +x } function check_upgrade_sqlalchemy() { @@ -1118,10 +1121,8 @@ function check_upgrade_sqlalchemy() { echo echo "${COLOR_BLUE}Upgrading sqlalchemy to the latest version to run tests with it${COLOR_RESET}" echo - set -x # shellcheck disable=SC2086 ${PACKAGING_TOOL_CMD} install ${EXTRA_INSTALL_FLAGS} --upgrade "sqlalchemy[asyncio]<2.1" "databricks-sqlalchemy>=2" - set +x } function check_downgrade_sqlalchemy() { @@ -1185,7 +1186,7 @@ function check_force_lowest_dependencies() { # --no-binary is needed in order to avoid libxml and xmlsec using different version of libxml2 # (binary lxml embeds its own libxml2, while xmlsec uses system one). # See https://bugs.launchpad.net/lxml/+bug/2110068 - uv sync --resolution lowest-direct --no-binary-package lxml --no-binary-package xmlsec + uv sync --resolution lowest-direct --no-binary-package lxml --no-binary-package xmlsec --all-extras else echo echo "${COLOR_BLUE}Forcing dependencies to lowest versions for Airflow.${COLOR_RESET}" @@ -1194,7 +1195,7 @@ function check_force_lowest_dependencies() { # --no-binary is needed in order to avoid libxml and xmlsec using different version of libxml2 # (binary lxml embeds its own libxml2, while xmlsec uses system one). # See https://bugs.launchpad.net/lxml/+bug/2110068 - uv sync --resolution lowest-direct --no-binary-package lxml --no-binary-package xmlsec + uv sync --resolution lowest-direct --no-binary-package lxml --no-binary-package xmlsec --all-extras fi } @@ -1474,8 +1475,8 @@ ENV AIRFLOW_PIP_VERSION=${AIRFLOW_PIP_VERSION} \ UV_LINK_MODE=copy \ AIRFLOW_PRE_COMMIT_VERSION=${AIRFLOW_PRE_COMMIT_VERSION} -# The PATH is needed for PIPX to find the tools installed -ENV PATH="/root/.local/bin:${PATH}" +# The PATH is needed for PIPX to find the tools installed and cargo to build the wheels +ENV PATH="/root/.local/bin:/root/.cargo/bin:${PATH}" # Useful for creating a cache id based on the underlying architecture, preventing the use of cached python packages from # an incorrect architecture. diff --git a/README.md b/README.md index 2c538d90a179b..65e7f7348ecd9 100644 --- a/README.md +++ b/README.md @@ -101,7 +101,7 @@ Apache Airflow is tested with: | | Main version (dev) | Stable version (3.0.3) | |------------|------------------------|------------------------| -| Python | 3.10, 3.11, 3.12 | 3.9, 3.10, 3.11, 3.12 | +| Python | 3.10, 3.11, 3.12, 3.13 | 3.9, 3.10, 3.11, 3.12 | | Platform | AMD64/ARM64(\*) | AMD64/ARM64(\*) | | Kubernetes | 1.30, 1.31, 1.32, 1.33 | 1.30, 1.31, 1.32, 1.33 | | PostgreSQL | 13, 14, 15, 16, 17 | 13, 14, 15, 16, 17 | diff --git a/airflow-core/docs/installation/prerequisites.rst b/airflow-core/docs/installation/prerequisites.rst index 6859918d9c65c..44e706d7af68b 100644 --- a/airflow-core/docs/installation/prerequisites.rst +++ b/airflow-core/docs/installation/prerequisites.rst @@ -20,7 +20,7 @@ Prerequisites Airflow® is tested with: -* Python: 3.10, 3.11, 3.12 +* Python: 3.10, 3.11, 3.12, 3.13 * Databases: diff --git a/airflow-core/docs/start.rst b/airflow-core/docs/start.rst index e992e94ac8f95..d40165001619c 100644 --- a/airflow-core/docs/start.rst +++ b/airflow-core/docs/start.rst @@ -24,7 +24,7 @@ This quick start guide will help you bootstrap an Airflow standalone instance on .. note:: - Successful installation requires a Python 3 environment. Starting with Airflow 3.1.0, Airflow supports Python 3.10, 3.11, and 3.12. + Successful installation requires a Python 3 environment. Starting with Airflow 3.1.0, Airflow supports Python 3.10, 3.11, 3.12, 3.13. Officially supported installation methods is with``pip`. diff --git a/airflow-core/pyproject.toml b/airflow-core/pyproject.toml index 3646ccabdea7b..e36e1fa40bb49 100644 --- a/airflow-core/pyproject.toml +++ b/airflow-core/pyproject.toml @@ -35,7 +35,12 @@ name = "apache-airflow-core" description = "Core packages for Apache Airflow, schedule and API server" readme = { file = "README.md", content-type = "text/markdown" } license-files.globs = ["LICENSE", "3rd-party-licenses/*.txt", "NOTICE"] -requires-python = ">=3.10,!=3.13" +# We know that it will take a while before we can support Python 3.14 because of all our dependencies +# It takes about 4-7 months after Python release before we can support it, so we limit it to <3.14 +# proactively. This way we also have a chance to test it with Python 3.14 and bump the upper binding +# and manually mark providers that do not support it yet with !-3.14 - until they support it - which will +# also exclude resolving uv workspace dependencies for those providers. +requires-python = ">=3.10,<3.14" authors = [ { name = "Apache Software Foundation", email = "dev@airflow.apache.org" }, ] @@ -80,6 +85,7 @@ dependencies = [ # 0.115.10 fastapi was a bad release that broke our API's and static checks. # Related fastapi issue here: https://github.com/fastapi/fastapi/discussions/13431 "fastapi[standard]>=0.115.0,!=0.115.10", + "starlette>=0.45.0", "httpx>=0.25.0", 'importlib_metadata>=6.5;python_version<"3.12"', 'importlib_metadata>=7.0;python_version>="3.12"', @@ -87,7 +93,7 @@ dependencies = [ "jinja2>=3.1.5", "jsonschema>=4.19.1", "lazy-object-proxy>=1.2.0", - 'libcst >=1.1.0', + 'libcst >=1.8.2', "linkify-it-py>=2.0.0", "lockfile>=0.12.2", "methodtools>=0.4.7", @@ -102,11 +108,10 @@ dependencies = [ # dependency and should be resolved as early as possible. # This may be removed when future versions of pip are able # to handle this dependency resolution automatically. - "opentelemetry-proto<9999", + "opentelemetry-proto<9999,>=1.27.0", "packaging>=25.0", "pathspec>=0.9.0", - 'pendulum>=2.1.2,<4.0;python_version<"3.12"', - 'pendulum>=3.0.0,<4.0;python_version>="3.12"', + 'pendulum>=3.1.0', "pluggy>=1.5.0", "psutil>=5.8.0", "pydantic>=2.11.0", @@ -133,9 +138,7 @@ dependencies = [ "tabulate>=0.9.0", "tenacity>=8.3.0", "termcolor>=3.0.0", - # temporarily exclude 4.14.0 due to its broken compat with cadwyn - # See https://github.com/zmievsa/cadwyn/issues/283 - "typing-extensions!=4.14.0", + "typing-extensions>=4.14.1", # Universal Pathlib 0.2.4 adds extra validation for Paths and our integration with local file paths # Does not work with it Tracked in https://github.com/fsspec/universal_pathlib/issues/276 "universal-pathlib>=0.2.2,!=0.2.4", @@ -153,14 +156,14 @@ dependencies = [ [project.optional-dependencies] "async" = [ "eventlet>=0.37.0", - "gevent>=24.2.1", - "greenlet>=0.4.9", + "gevent>=25.4.1", + "greenlet>=3.1.0", ] "graphviz" = [ # The graphviz package creates friction when installing on MacOS as it needs graphviz system package to # be installed, and it's really only used for very obscure features of Airflow, so we can skip it on MacOS # Instead, if someone attempts to use it on MacOS, they will get explanatory error on how to install it - "graphviz>=0.12; sys_platform != 'darwin'", + "graphviz>=0.20; sys_platform != 'darwin'", ] "kerberos" = [ "pykerberos>=1.1.13", @@ -172,9 +175,8 @@ dependencies = [ ] "sentry" = [ "blinker>=1.1", - # Sentry SDK 1.33 is broken when greenlets are installed and fails to import - # See https://github.com/getsentry/sentry-python/issues/2473 - "sentry-sdk>=1.32.0,!=1.33.0", + # Apparently sentry needs flask to be installed to work properly + "sentry-sdk[flask]>=2.30.0", ] "statsd" = [ "statsd>=3.3.0", @@ -199,7 +201,6 @@ Mastodon = "https://fosstodon.org/@airflow" Bluesky = "https://bsky.app/profile/apache-airflow.bsky.social" YouTube = "https://www.youtube.com/channel/UCSXwxpWZQ7XZ1WL3wqevChA/" - [tool.hatch.version] path = "src/airflow/__init__.py" @@ -249,7 +250,7 @@ dev = [ "apache-airflow-providers-amazon", "apache-airflow-providers-celery", "apache-airflow-providers-cncf-kubernetes", - "apache-airflow-providers-fab", + "apache-airflow-providers-fab>=2.2.0; python_version < '3.13'", "apache-airflow-providers-git", "apache-airflow-providers-ftp", ] diff --git a/airflow-core/src/airflow/api_fastapi/core_api/app.py b/airflow-core/src/airflow/api_fastapi/core_api/app.py index bafa3add823c8..67a3223fbfa8f 100644 --- a/airflow-core/src/airflow/api_fastapi/core_api/app.py +++ b/airflow-core/src/airflow/api_fastapi/core_api/app.py @@ -18,6 +18,7 @@ import logging import os +import sys import warnings from pathlib import Path @@ -35,6 +36,8 @@ log = logging.getLogger(__name__) +PY313 = sys.version_info >= (3, 13) + def init_views(app: FastAPI) -> None: """Init views by registering the different routers.""" @@ -124,6 +127,13 @@ def init_flask_plugins(app: FastAPI) -> None: try: from airflow.providers.fab.www.app import create_app except ImportError: + if PY313: + log.info( + "Some Airflow 2 plugins have been detected in your environment. Currently FAB provider " + "does not support Python 3.13, so you cannot use Airflow 2 plugins with Airflow 3 until " + "FAB provider will be Python 3.13 compatible." + ) + return raise AirflowException( "Some Airflow 2 plugins have been detected in your environment. " "To run them with Airflow 3, you must install the FAB provider in your Airflow environment." diff --git a/airflow-core/src/airflow/models/connection.py b/airflow-core/src/airflow/models/connection.py index 8628c592584be..5548f7726ec4e 100644 --- a/airflow-core/src/airflow/models/connection.py +++ b/airflow-core/src/airflow/models/connection.py @@ -44,7 +44,7 @@ # the symbols #,!,-,_,.,:,\,/ and () requiring at least one match. # # You can try the regex here: https://regex101.com/r/69033B/1 -RE_SANITIZE_CONN_ID = re.compile(r"^[\w\#\!\(\)\-\.\:\/\\]{1,}$") +RE_SANITIZE_CONN_ID = re.compile(r"^[\w#!()\-.:/\\]{1,}$") # the conn ID max len should be 250 CONN_ID_MAX_LEN: int = 250 diff --git a/airflow-core/src/airflow/models/dag.py b/airflow-core/src/airflow/models/dag.py index 4e1eac0ff7c8a..a141d96a1c30f 100644 --- a/airflow-core/src/airflow/models/dag.py +++ b/airflow-core/src/airflow/models/dag.py @@ -39,7 +39,6 @@ import pendulum import sqlalchemy_jsonfield from dateutil.relativedelta import relativedelta -from packaging import version as packaging_version from sqlalchemy import ( Boolean, Column, @@ -461,28 +460,14 @@ def _upgrade_outdated_dag_access_control(access_control=None): """Look for outdated dag level actions in DAG access_controls and replace them with updated actions.""" if access_control is None: return None - - from airflow.providers.fab import __version__ as FAB_VERSION - from airflow.providers.fab.www.security import permissions - updated_access_control = {} for role, perms in access_control.items(): - if packaging_version.parse(FAB_VERSION) >= packaging_version.parse("1.3.0"): - updated_access_control[role] = updated_access_control.get(role, {}) - if isinstance(perms, (set, list)): - # Support for old-style access_control where only the actions are specified - updated_access_control[role][permissions.RESOURCE_DAG] = set(perms) - else: - updated_access_control[role] = perms - elif isinstance(perms, dict): - # Not allow new access control format with old FAB versions - raise AirflowException( - "Please upgrade the FAB provider to a version >= 1.3.0 to allow " - "use the Dag Level Access Control new format." - ) + updated_access_control[role] = updated_access_control.get(role, {}) + if isinstance(perms, (set, list)): + # Support for old-style access_control where only the actions are specified + updated_access_control[role]["DAGs"] = set(perms) else: - updated_access_control[role] = set(perms) - + updated_access_control[role] = perms return updated_access_control def get_next_data_interval(self, dag_model: DagModel) -> DataInterval | None: diff --git a/airflow-core/src/airflow/plugins_manager.py b/airflow-core/src/airflow/plugins_manager.py index d750ff7c405cd..20377e6165cd8 100644 --- a/airflow-core/src/airflow/plugins_manager.py +++ b/airflow-core/src/airflow/plugins_manager.py @@ -133,7 +133,7 @@ class EntryPointSource(AirflowPluginSource): """Class used to define Plugins loaded from entrypoint.""" def __init__(self, entrypoint: metadata.EntryPoint, dist: metadata.Distribution): - self.dist = dist.metadata["Name"] + self.dist = dist.metadata["Name"] # type: ignore[index] self.version = dist.version self.entrypoint = str(entrypoint) diff --git a/airflow-core/src/airflow/providers_manager.py b/airflow-core/src/airflow/providers_manager.py index 0c99707a32d45..20fc79bd7fac5 100644 --- a/airflow-core/src/airflow/providers_manager.py +++ b/airflow-core/src/airflow/providers_manager.py @@ -586,6 +586,8 @@ def _discover_all_providers_from_packages(self) -> None: and verifies only the subset of fields that are needed at runtime. """ for entry_point, dist in entry_points_with_dist("apache_airflow_provider"): + if not dist.metadata: + continue package_name = canonicalize_name(dist.metadata["name"]) if package_name in self._provider_dict: continue diff --git a/airflow-core/src/airflow/traces/otel_tracer.py b/airflow-core/src/airflow/traces/otel_tracer.py index 34ee543a56d6a..982e8826e279f 100644 --- a/airflow-core/src/airflow/traces/otel_tracer.py +++ b/airflow-core/src/airflow/traces/otel_tracer.py @@ -268,13 +268,13 @@ def _new_span( start_time=datetime_to_nano(start_time), ) - span = tracer.start_span( + span = tracer.start_span( # type: ignore[assignment] name=span_name, context=parent_context, links=links, start_time=datetime_to_nano(start_time), ) - current_span_ctx = trace.set_span_in_context(NonRecordingSpan(span.get_span_context())) + current_span_ctx = trace.set_span_in_context(NonRecordingSpan(span.get_span_context())) # type: ignore[attr-defined] # We have to manually make the span context as the active context. # If the span needs to be injected into the carrier, then this is needed to make sure # that the injected context will point to the span context that was just created. diff --git a/airflow-core/tests/unit/always/test_example_dags.py b/airflow-core/tests/unit/always/test_example_dags.py index 01ea301f319b9..11c1fd879ac20 100644 --- a/airflow-core/tests/unit/always/test_example_dags.py +++ b/airflow-core/tests/unit/always/test_example_dags.py @@ -82,12 +82,7 @@ def get_suspended_providers_folders() -> list[str]: for provider_path in AIRFLOW_PROVIDERS_ROOT_PATH.rglob("provider.yaml"): provider_yaml = yaml.safe_load(provider_path.read_text()) if provider_yaml["state"] == "suspended": - suspended_providers.append( - provider_path.parent.relative_to(AIRFLOW_ROOT_PATH) - .as_posix() - # TODO(potiuk): check - .replace("providers/src/airflow/providers/", "") - ) + suspended_providers.append(provider_path.parent.resolve().as_posix()) return suspended_providers @@ -101,12 +96,7 @@ def get_python_excluded_providers_folders() -> list[str]: provider_yaml = yaml.safe_load(provider_path.read_text()) excluded_python_versions = provider_yaml.get("excluded-python-versions", []) if CURRENT_PYTHON_VERSION in excluded_python_versions: - excluded_providers.append( - provider_path.parent.relative_to(AIRFLOW_ROOT_PATH) - .as_posix() - # TODO(potiuk): check - .replace("providers/src/airflow/providers/", "") - ) + excluded_providers.append(provider_path.parent.resolve().as_posix()) return excluded_providers @@ -122,16 +112,6 @@ def example_not_excluded_dags(xfail_db_exception: bool = False): suspended_providers_folders = get_suspended_providers_folders() current_python_excluded_providers_folders = get_python_excluded_providers_folders() - suspended_providers_folders = [ - AIRFLOW_ROOT_PATH.joinpath(prefix, provider).as_posix() - for prefix in PROVIDERS_PREFIXES - for provider in suspended_providers_folders - ] - current_python_excluded_providers_folders = [ - AIRFLOW_ROOT_PATH.joinpath(prefix, provider).as_posix() - for prefix in PROVIDERS_PREFIXES - for provider in current_python_excluded_providers_folders - ] providers_folders = tuple([AIRFLOW_ROOT_PATH.joinpath(pp).as_posix() for pp in PROVIDERS_PREFIXES]) for example_dir in example_dirs: candidates = glob(f"{AIRFLOW_ROOT_PATH.as_posix()}/{example_dir}", recursive=True) @@ -187,6 +167,12 @@ def test_should_be_importable(example: str): dag_folder=example, include_examples=False, ) + if len(dagbag.import_errors) == 1 and "AirflowOptionalProviderFeatureException" in str( + dagbag.import_errors + ): + pytest.skip( + f"Skipping {example} because it requires an optional provider feature that is not installed." + ) assert len(dagbag.import_errors) == 0, f"import_errors={str(dagbag.import_errors)}" assert len(dagbag.dag_ids) >= 1 diff --git a/airflow-core/tests/unit/always/test_providers_manager.py b/airflow-core/tests/unit/always/test_providers_manager.py index d0a2037f9bdce..0f14fc399d6e6 100644 --- a/airflow-core/tests/unit/always/test_providers_manager.py +++ b/airflow-core/tests/unit/always/test_providers_manager.py @@ -21,6 +21,8 @@ import logging import re import sys + +PY313 = sys.version_info >= (3, 13) import warnings from unittest.mock import patch @@ -253,6 +255,19 @@ def test_hook_values(self): print(record.exc_info, file=sys.stderr) real_warning_count += 1 if real_warning_count: + if PY313: + only_ydb_and_yandexcloud_warnings = True + for record in warning_records: + if "ydb" in str(record.message) or "yandexcloud" in str(record.message): + continue + only_ydb_and_yandexcloud_warnings = False + if only_ydb_and_yandexcloud_warnings: + print( + "Only warnings from ydb and yandexcloud providers are generated, " + "which is expected in Python 3.13+", + file=sys.stderr, + ) + return raise AssertionError("There are warnings generated during hook imports. Please fix them") assert [w.message for w in warning_records if "hook-class-names" in str(w.message)] == [] diff --git a/airflow-core/tests/unit/always/test_secrets_backends.py b/airflow-core/tests/unit/always/test_secrets_backends.py index 87cbd18f0478b..45f81baa8d72e 100644 --- a/airflow-core/tests/unit/always/test_secrets_backends.py +++ b/airflow-core/tests/unit/always/test_secrets_backends.py @@ -45,6 +45,7 @@ def __init__(self, conn_id, variation: str): class TestBaseSecretsBackend: def setup_method(self) -> None: + clear_db_connections() clear_db_variables() def teardown_method(self) -> None: diff --git a/airflow-core/tests/unit/api_fastapi/conftest.py b/airflow-core/tests/unit/api_fastapi/conftest.py index b39f4c4f743c9..30cef3aa71e10 100644 --- a/airflow-core/tests/unit/api_fastapi/conftest.py +++ b/airflow-core/tests/unit/api_fastapi/conftest.py @@ -135,7 +135,10 @@ def configure_git_connection_for_dag_bundle(session): } ): yield - + # in case no flush or commit was executed after the "session.add" above, we need to flush the session + # manually here to make sure that the added connection will be deleted by query(Connection).delete() + # in the`clear_db_connections` function below + session.flush() clear_db_connections(False) diff --git a/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dag_run.py b/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dag_run.py index 6ce599e0c675b..dbe55a92e7893 100644 --- a/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dag_run.py +++ b/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dag_run.py @@ -38,7 +38,13 @@ from airflow.utils.types import DagRunTriggeredByType, DagRunType from tests_common.test_utils.api_fastapi import _check_dag_run_note, _check_last_log -from tests_common.test_utils.db import clear_db_dags, clear_db_logs, clear_db_runs, clear_db_serialized_dags +from tests_common.test_utils.db import ( + clear_db_connections, + clear_db_dags, + clear_db_logs, + clear_db_runs, + clear_db_serialized_dags, +) from tests_common.test_utils.format_datetime import from_datetime_to_zulu, from_datetime_to_zulu_without_ms if TYPE_CHECKING: @@ -82,6 +88,7 @@ @pytest.fixture(autouse=True) @provide_session def setup(request, dag_maker, session=None): + clear_db_connections() clear_db_runs() clear_db_dags() clear_db_serialized_dags() diff --git a/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dags.py b/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dags.py index 34f8e0df4477a..ed0e9c147ea31 100644 --- a/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dags.py +++ b/airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_dags.py @@ -31,7 +31,12 @@ from airflow.utils.state import DagRunState, TaskInstanceState from airflow.utils.types import DagRunTriggeredByType, DagRunType -from tests_common.test_utils.db import clear_db_dags, clear_db_runs, clear_db_serialized_dags +from tests_common.test_utils.db import ( + clear_db_connections, + clear_db_dags, + clear_db_runs, + clear_db_serialized_dags, +) from tests_common.test_utils.logs import check_last_log pytestmark = pytest.mark.db_test @@ -58,6 +63,7 @@ class TestDagEndpoint: @staticmethod def _clear_db(): + clear_db_connections() clear_db_runs() clear_db_dags() clear_db_serialized_dags() diff --git a/airflow-core/tests/unit/cli/commands/test_api_server_command.py b/airflow-core/tests/unit/cli/commands/test_api_server_command.py index 2f3dec30f3dbb..1e30f8d701c94 100644 --- a/airflow-core/tests/unit/cli/commands/test_api_server_command.py +++ b/airflow-core/tests/unit/cli/commands/test_api_server_command.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import sys from unittest import mock import pytest @@ -296,16 +297,31 @@ def test_run_command_daemon( ) ] mock_pid_file.assert_has_calls([mock.call(mock_setup_locations.return_value[0], -1)]) - assert mock_open.mock_calls == [ - mock.call(mock_setup_locations.return_value[1], "a"), - mock.call().__enter__(), - mock.call(mock_setup_locations.return_value[2], "a"), - mock.call().__enter__(), - mock.call().truncate(0), - mock.call().truncate(0), - mock.call().__exit__(None, None, None), - mock.call().__exit__(None, None, None), - ] + if sys.version_info >= (3, 13): + # extra close is called in Python 3.13+ to close the file descriptors + assert mock_open.mock_calls == [ + mock.call(mock_setup_locations.return_value[1], "a"), + mock.call().__enter__(), + mock.call(mock_setup_locations.return_value[2], "a"), + mock.call().__enter__(), + mock.call().truncate(0), + mock.call().truncate(0), + mock.call().__exit__(None, None, None), + mock.call().close(), + mock.call().__exit__(None, None, None), + mock.call().close(), + ] + else: + assert mock_open.mock_calls == [ + mock.call(mock_setup_locations.return_value[1], "a"), + mock.call().__enter__(), + mock.call(mock_setup_locations.return_value[2], "a"), + mock.call().__enter__(), + mock.call().truncate(0), + mock.call().truncate(0), + mock.call().__exit__(None, None, None), + mock.call().__exit__(None, None, None), + ] else: assert mock_daemon.mock_calls == [] mock_setup_locations.mock_calls == [] diff --git a/airflow-core/tests/unit/cli/commands/test_kerberos_command.py b/airflow-core/tests/unit/cli/commands/test_kerberos_command.py index 39695ff51a857..ac8ba911ddab7 100644 --- a/airflow-core/tests/unit/cli/commands/test_kerberos_command.py +++ b/airflow-core/tests/unit/cli/commands/test_kerberos_command.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import sys from unittest import mock import pytest @@ -28,6 +29,8 @@ pytestmark = pytest.mark.db_test +PY313 = sys.version_info >= (3, 13) + class TestKerberosCommand: @classmethod @@ -100,6 +103,8 @@ def test_run_command_daemon(self, mock_krb, mock_daemon, mock_setup_locations, m log="/tmp/kerberos.log", ) + python_3_13_close_calls = [mock.call().close()] if PY313 else [] + mock_pid_file.mock_calls[0] = mock.call(mock_setup_locations.return_value[0], -1) assert mock_open.mock_calls == [ mock.call(mock_setup_locations.return_value[1], "a"), @@ -109,7 +114,9 @@ def test_run_command_daemon(self, mock_krb, mock_daemon, mock_setup_locations, m mock.call().truncate(0), mock.call().truncate(0), mock.call().__exit__(None, None, None), + *python_3_13_close_calls, mock.call().__exit__(None, None, None), + *python_3_13_close_calls, ] @mock.patch("airflow.cli.commands.kerberos_command.krb") diff --git a/airflow-core/tests/unit/cli/commands/test_plugins_command.py b/airflow-core/tests/unit/cli/commands/test_plugins_command.py index c6362935c1315..8a6944483a838 100644 --- a/airflow-core/tests/unit/cli/commands/test_plugins_command.py +++ b/airflow-core/tests/unit/cli/commands/test_plugins_command.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import importlib import io import json import textwrap @@ -30,7 +31,13 @@ from airflow.sdk import BaseOperatorLink from tests_common.test_utils.mock_plugins import mock_plugin_manager -from unit.plugins.test_plugin import AirflowTestPlugin as ComplexAirflowPlugin + +if importlib.util.find_spec("flask_appbuilder"): + flask_appbuilder_installed = True + from unit.plugins.test_plugin import AirflowTestPlugin as ComplexAirflowPlugin +else: + ComplexAirflowPlugin = None # type: ignore [misc, assignment] + flask_appbuilder_installed = False pytestmark = pytest.mark.db_test @@ -61,6 +68,7 @@ def test_should_display_no_plugins(self): stdout = temp_stdout.getvalue() assert "No plugins loaded" in stdout + @pytest.mark.skipif(not flask_appbuilder_installed, reason="Flask AppBuilder is not installed") @mock_plugin_manager(plugins=[ComplexAirflowPlugin]) def test_should_display_one_plugin(self): with redirect_stdout(io.StringIO()) as temp_stdout: diff --git a/airflow-core/tests/unit/cli/commands/test_task_command.py b/airflow-core/tests/unit/cli/commands/test_task_command.py index 5ce864667027d..6e25253eb4067 100644 --- a/airflow-core/tests/unit/cli/commands/test_task_command.py +++ b/airflow-core/tests/unit/cli/commands/test_task_command.py @@ -84,7 +84,6 @@ class TestCliTasks: dag_run: DagRun @classmethod - @pytest.fixture(autouse=True) def setup_class(cls): logging.config.dictConfig(DEFAULT_LOGGING_CONFIG) parse_and_sync_to_db(os.devnull, include_examples=True) diff --git a/airflow-core/tests/unit/jobs/test_triggerer_job.py b/airflow-core/tests/unit/jobs/test_triggerer_job.py index 2f01f63677c90..194c1c8752ba7 100644 --- a/airflow-core/tests/unit/jobs/test_triggerer_job.py +++ b/airflow-core/tests/unit/jobs/test_triggerer_job.py @@ -76,17 +76,17 @@ @pytest.fixture(autouse=True) def clean_database(): """Fixture that cleans the database before and after every test.""" + clear_db_connections() clear_db_runs() clear_db_dags() clear_db_xcom() clear_db_variables() - clear_db_connections() yield # Test runs here + clear_db_connections() clear_db_runs() clear_db_dags() clear_db_xcom() clear_db_variables() - clear_db_connections() def create_trigger_in_db(session, trigger, operator=None): diff --git a/airflow-core/tests/unit/models/test_dag.py b/airflow-core/tests/unit/models/test_dag.py index 1feb7ebd2eb58..9a99eba5974a2 100644 --- a/airflow-core/tests/unit/models/test_dag.py +++ b/airflow-core/tests/unit/models/test_dag.py @@ -70,7 +70,6 @@ from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetAll, AssetAny from airflow.sdk.definitions.deadline import DeadlineAlert, DeadlineReference from airflow.sdk.definitions.param import Param -from airflow.security import permissions from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable from airflow.timetables.simple import ( AssetTriggeredTimetable, @@ -1907,63 +1906,6 @@ def test_next_dagrun_info_on_29_feb(self): assert next_info.data_interval.start == timezone.datetime(2028, 2, 29) assert next_info.data_interval.end == timezone.datetime(2032, 2, 29) - @pytest.mark.parametrize( - "fab_version, perms, expected_exception, expected_perms", - [ - pytest.param( - "1.2.0", - { - "role1": {permissions.ACTION_CAN_READ, permissions.ACTION_CAN_EDIT}, - "role3": {permissions.RESOURCE_DAG_RUN: {permissions.ACTION_CAN_CREATE}}, - # will raise error in old FAB with new access control format - }, - AirflowException, - None, - id="old_fab_new_access_control_format", - ), - pytest.param( - "1.2.0", - { - "role1": [ - permissions.ACTION_CAN_READ, - permissions.ACTION_CAN_EDIT, - permissions.ACTION_CAN_READ, - ], - }, - None, - {"role1": {permissions.ACTION_CAN_READ, permissions.ACTION_CAN_EDIT}}, - id="old_fab_old_access_control_format", - ), - pytest.param( - "1.3.0", - { - "role1": {permissions.ACTION_CAN_READ, permissions.ACTION_CAN_EDIT}, # old format - "role3": {permissions.RESOURCE_DAG_RUN: {permissions.ACTION_CAN_CREATE}}, # new format - }, - None, - { - "role1": { - permissions.RESOURCE_DAG: {permissions.ACTION_CAN_READ, permissions.ACTION_CAN_EDIT} - }, - "role3": {permissions.RESOURCE_DAG_RUN: {permissions.ACTION_CAN_CREATE}}, - }, - id="new_fab_mixed_access_control_format", - ), - ], - ) - def test_access_control_format(self, fab_version, perms, expected_exception, expected_perms): - if expected_exception: - with patch("airflow.providers.fab.__version__", fab_version): - with pytest.raises( - expected_exception, - match="Please upgrade the FAB provider to a version >= 1.3.0 to allow use the Dag Level Access Control new format.", - ): - DAG(dag_id="dag_test", schedule=None, access_control=perms) - else: - with patch("airflow.providers.fab.__version__", fab_version): - dag = DAG(dag_id="dag_test", schedule=None, access_control=perms) - assert dag.access_control == expected_perms - def test_validate_executor_field_executor_not_configured(self): dag = DAG("test-dag", schedule=None) EmptyOperator(task_id="t1", dag=dag, executor="test.custom.executor") diff --git a/airflow-core/tests/unit/models/test_dagbag.py b/airflow-core/tests/unit/models/test_dagbag.py index 0d46f180e1a33..00aa6151f9728 100644 --- a/airflow-core/tests/unit/models/test_dagbag.py +++ b/airflow-core/tests/unit/models/test_dagbag.py @@ -55,6 +55,7 @@ example_dags_folder = AIRFLOW_ROOT_PATH / "airflow-core" / "src" / "airflow" / "example_dags" / "standard" PY311 = sys.version_info >= (3, 11) +PY313 = sys.version_info >= (3, 13) # Include the words "airflow" and "dag" in the file contents, # tricking airflow into thinking these @@ -651,10 +652,12 @@ def f(): @staticmethod def _make_test_traceback(unparseable_filename: str, depth=None) -> str: - marker = " ^^^^^^^^^^^\n" if PY311 else "" + python_311_marker = " ^^^^^^^^^^^\n" if PY311 else "" + python_313_marker = [" ~~~~~~~~~^^\n"] if PY313 else [] frames = ( f' File "{unparseable_filename}", line 3, in \n something()\n', - f' File "{unparseable_filename}", line 2, in something\n return airflow_DAG\n{marker}', + *python_313_marker, + f' File "{unparseable_filename}", line 2, in something\n return airflow_DAG\n{python_311_marker}', ) depth = 0 if depth is None else -depth return ( diff --git a/airflow-core/tests/unit/plugins/test_plugin.py b/airflow-core/tests/unit/plugins/test_plugin.py index c4c1fac515016..def446620e94b 100644 --- a/airflow-core/tests/unit/plugins/test_plugin.py +++ b/airflow-core/tests/unit/plugins/test_plugin.py @@ -19,7 +19,52 @@ from fastapi import FastAPI from flask import Blueprint -from flask_appbuilder import BaseView as AppBuilderBaseView, expose + +try: + # if flask_appbuilder is installed, we can use AppBuilderBaseView + from flask_appbuilder import BaseView as AppBuilderBaseView, expose + + # Creating a flask appbuilder BaseView + class PluginTestAppBuilderBaseView(AppBuilderBaseView): + default_view = "test" + + @expose("/") + def test(self): + return self.render_template("test_plugin/test.html", content="Hello galaxy!") + + v_appbuilder_view = PluginTestAppBuilderBaseView() + v_appbuilder_package = { + "name": "Test View", + "category": "Test Plugin", + "view": v_appbuilder_view, + "label": "Test Label", + } + + v_nomenu_appbuilder_package = {"view": v_appbuilder_view} + + # Creating flask appbuilder Menu Items + appbuilder_mitem = { + "name": "Google", + "href": "https://www.google.com", + "category": "Search", + } + appbuilder_mitem_toplevel = { + "name": "apache", + "href": "https://www.apache.org/", + "label": "The Apache Software Foundation", + } + is_flask_appbuilder_installed = True + v_appbuilder_packages = [v_appbuilder_package] + v_appbuilder_menu_items = [appbuilder_mitem, appbuilder_mitem_toplevel] +except ImportError as e: + if "flask_appbuilder" not in str(e): + raise Exception("The import error should be about flask_appbuilder") from e + # flask_appbuilder is not installed, so we cannot use AppBuilderBaseView + AppBuilderBaseView = None + is_flask_appbuilder_installed = False + v_appbuilder_packages = [] + v_appbuilder_menu_items = [] + from starlette.middleware.base import BaseHTTPMiddleware # This is the class you derive to create a plugin @@ -44,37 +89,6 @@ def plugin_macro(): pass -# Creating a flask appbuilder BaseView -class PluginTestAppBuilderBaseView(AppBuilderBaseView): - default_view = "test" - - @expose("/") - def test(self): - return self.render_template("test_plugin/test.html", content="Hello galaxy!") - - -v_appbuilder_view = PluginTestAppBuilderBaseView() -v_appbuilder_package = { - "name": "Test View", - "category": "Test Plugin", - "view": v_appbuilder_view, - "label": "Test Label", -} - -v_nomenu_appbuilder_package = {"view": v_appbuilder_view} - -# Creating flask appbuilder Menu Items -appbuilder_mitem = { - "name": "Google", - "href": "https://www.google.com", - "category": "Search", -} -appbuilder_mitem_toplevel = { - "name": "apache", - "href": "https://www.apache.org/", - "label": "The Apache Software Foundation", -} - # Creating a flask blueprint to integrate the templates and static folder bp = Blueprint( "test_plugin", @@ -140,8 +154,8 @@ class AirflowTestPlugin(AirflowPlugin): fastapi_root_middlewares = [middleware_with_metadata] external_views = [external_view_with_metadata] react_apps = [react_app_with_metadata] - appbuilder_views = [v_appbuilder_package] - appbuilder_menu_items = [appbuilder_mitem, appbuilder_mitem_toplevel] + appbuilder_views = v_appbuilder_packages + appbuilder_menu_items = v_appbuilder_menu_items global_operator_extra_links = [ AirflowLink(), GithubLink(), diff --git a/airflow-core/tests/unit/serialization/test_dag_serialization.py b/airflow-core/tests/unit/serialization/test_dag_serialization.py index 3e11e91499472..0331b325ac1dd 100644 --- a/airflow-core/tests/unit/serialization/test_dag_serialization.py +++ b/airflow-core/tests/unit/serialization/test_dag_serialization.py @@ -93,7 +93,6 @@ AirflowLink2, CustomOperator, GithubLink, - GoogleLink, MockOperator, ) from tests_common.test_utils.timetables import ( @@ -385,6 +384,12 @@ def get_excluded_patterns() -> Generator[str, None, None]: if python_version in provider_info.get("excluded-python-versions"): provider_path = provider.replace(".", "/") yield f"providers/{provider_path}" + current_python_version = sys.version_info[:2] + if current_python_version >= (3, 13): + # We should remove google when ray is fixed to work with Python 3.13 + # and yandex when it is fixed to work with Python 3.13 + yield "providers/google/tests/system/google/" + yield "providers/yandex/tests/system/yandex/" def collect_dags(dag_folder=None): @@ -598,9 +603,10 @@ def sorted_serialized_dag(dag_dict: dict): task["__var"] = dict(sorted(task["__var"].items(), key=lambda x: x[0])) tasks.append(task) dag_dict["dag"]["tasks"] = tasks - dag_dict["dag"]["access_control"]["__var"]["test_role"]["__var"] = sorted( - dag_dict["dag"]["access_control"]["__var"]["test_role"]["__var"] - ) + if "access_control" in dag_dict["dag"]: + dag_dict["dag"]["access_control"]["__var"]["test_role"]["__var"] = sorted( + dag_dict["dag"]["access_control"]["__var"]["test_role"]["__var"] + ) return dag_dict expected = copy.deepcopy(expected) @@ -1228,10 +1234,14 @@ def test_extra_serialized_field_and_operator_links( link = simple_task.get_extra_links(ti, name) assert link == expected - - # Test Deserialized link registered via Airflow Plugin - link = simple_task.get_extra_links(ti, GoogleLink.name) - assert link == "https://www.google.com" + current_python_version = sys.version_info[:2] + if current_python_version >= (3, 13): + # TODO(potiuk) We should bring it back when ray is supported on Python 3.13 + # Test Deserialized link registered via Airflow Plugin + from tests_common.test_utils.mock_operators import GoogleLink + + link = simple_task.get_extra_links(ti, GoogleLink.name) + assert link == "https://www.google.com" class ClassWithCustomAttributes: """ diff --git a/airflow-core/tests/unit/utils/test_db.py b/airflow-core/tests/unit/utils/test_db.py index cf69275025330..aa24ad87e94cb 100644 --- a/airflow-core/tests/unit/utils/test_db.py +++ b/airflow-core/tests/unit/utils/test_db.py @@ -24,7 +24,6 @@ from contextlib import redirect_stdout from io import StringIO from unittest import mock -from unittest.mock import MagicMock import pytest from alembic.autogenerate import compare_metadata @@ -36,7 +35,6 @@ from airflow.exceptions import AirflowException from airflow.models import Base as airflow_base -from airflow.providers.fab.auth_manager.models.db import FABDBManager from airflow.settings import engine from airflow.utils.db import ( _REVISION_HEADS_MAP, @@ -53,6 +51,7 @@ from airflow.utils.db_manager import RunDBManager from tests_common.test_utils.config import conf_vars +from unit.cli.commands.test_kerberos_command import PY313 pytestmark = pytest.mark.db_test @@ -71,9 +70,16 @@ def test_database_schema_and_sqlalchemy_model_are_in_sync(self): for dbmanager in external_db_managers._managers: for table_name, table in dbmanager.metadata.tables.items(): all_meta_data._add_table(table_name, table.schema, table) - # test FAB models - for table_name, table in FABDBManager.metadata.tables.items(): - all_meta_data._add_table(table_name, table.schema, table) + skip_fab = PY313 + if not skip_fab: + # FAB DB Manager + from airflow.providers.fab.auth_manager.models.db import FABDBManager + + # test FAB models + for table_name, table in FABDBManager.metadata.tables.items(): + all_meta_data._add_table(table_name, table.schema, table) + else: + print("Ignoring FAB models in Python 3.13+ as FAB is not compatible with 3.13+ yet.") # create diff between database schema and SQLAlchemy model mctx = MigrationContext.configure( engine.connect(), @@ -103,10 +109,19 @@ def test_database_schema_and_sqlalchemy_model_are_in_sync(self): lambda t: (t[0] == "remove_table" and t[1].name == "_xcom_archive"), ] + if skip_fab: + ignores.append(lambda t: (t[1].name.startswith("ab_"))) + ignores.append( + lambda t: (t[0] == "remove_index" and t[1].columns[0].table.name.startswith("ab_")) + ) + for ignore in ignores: diff = [d for d in diff if not ignore(d)] - - assert not diff, "Database schema and SQLAlchemy model are not in sync: " + str(diff) + if diff: + print("Database schema and SQLAlchemy model are not in sync: ") + for single_diff in diff: + print(f"Diff: {single_diff}") + pytest.fail("Database schema and SQLAlchemy model are not in sync") def test_only_single_head_revision_in_migrations(self): config = Config() @@ -161,6 +176,10 @@ def test_check_migrations(self): ) @mock.patch("alembic.command") def test_upgradedb(self, mock_alembic_command, auth, expected): + if PY313 and "airflow.providers.fab.auth_manager.fab_auth_manager.FabAuthManager" in str(auth): + pytest.skip( + "Skipping test for FAB Auth Manager on Python 3.13+ as FAB is not compatible with 3.13+ yet." + ) with conf_vars(auth): upgradedb() mock_alembic_command.upgrade.assert_called_with(mock.ANY, revision="heads") @@ -232,34 +251,6 @@ def test_downgrade_with_from(self, mock_om): actual = mock_om.call_args.kwargs["revision"] assert actual == "abc" - @pytest.mark.parametrize("skip_init", [False, True]) - @conf_vars( - {("database", "external_db_managers"): "airflow.providers.fab.auth_manager.models.db.FABDBManager"} - ) - @mock.patch("airflow.providers.fab.auth_manager.models.db.FABDBManager") - @mock.patch("airflow.utils.db.create_global_lock", new=MagicMock) - @mock.patch("airflow.utils.db.drop_airflow_models") - @mock.patch("airflow.utils.db.drop_airflow_moved_tables") - @mock.patch("airflow.utils.db.initdb") - @mock.patch("airflow.settings.engine.connect") - def test_resetdb( - self, - mock_connect, - mock_init, - mock_drop_moved, - mock_drop_airflow, - mock_fabdb_manager, - skip_init, - ): - session_mock = MagicMock() - resetdb(session_mock, skip_init=skip_init) - mock_drop_airflow.assert_called_once_with(mock_connect.return_value) - mock_drop_moved.assert_called_once_with(mock_connect.return_value) - if skip_init: - mock_init.assert_not_called() - else: - mock_init.assert_called_once_with(session=session_mock) - def test_resetdb_logging_level(self): unset_logging_level = logging.root.level logging.root.setLevel(logging.DEBUG) diff --git a/airflow-core/tests/unit/utils/test_db_cleanup.py b/airflow-core/tests/unit/utils/test_db_cleanup.py index 7b4fc055405d5..48a0c8486219b 100644 --- a/airflow-core/tests/unit/utils/test_db_cleanup.py +++ b/airflow-core/tests/unit/utils/test_db_cleanup.py @@ -39,7 +39,6 @@ from airflow.utils import timezone from airflow.utils.db_cleanup import ( ARCHIVE_TABLE_PREFIX, - ARCHIVED_TABLES_FROM_DB_MIGRATIONS, CreateTableAs, _build_query, _cleanup_table, @@ -353,7 +352,7 @@ def test_run_cleanup_archival_integration(self, table_name, expected_archived): @pytest.mark.parametrize( "skip_archive, expected_archives", - [pytest.param(True, 1, id="skip_archive"), pytest.param(False, 2, id="do_archive")], + [pytest.param(True, 0, id="skip_archive"), pytest.param(False, 1, id="do_archive")], ) def test__skip_archive(self, skip_archive, expected_archives): """ @@ -368,6 +367,9 @@ def test__skip_archive(self, skip_archive, expected_archives): num_tis=num_tis, ) with create_session() as session: + # cleanup any existing archived tables + for name in _get_archived_table_names(["dag_run"], session): + session.execute(text(f"DROP TABLE IF EXISTS {name}")) clean_before_date = base_date.add(days=5) _cleanup_table( **config_dict["dag_run"].__dict__, @@ -397,6 +399,9 @@ def test_skip_archive_failure_will_remove_table(self, reflect_tables_mock): ) try: with create_session() as session: + # cleanup any existing archived tables + for name in _get_archived_table_names(["dag_run"], session): + session.execute(text(f"DROP TABLE IF EXISTS {name}")) clean_before_date = base_date.add(days=5) _cleanup_table( **config_dict["dag_run"].__dict__, @@ -409,8 +414,7 @@ def test_skip_archive_failure_will_remove_table(self, reflect_tables_mock): except SQLAlchemyError: pass archived_table_names = _get_archived_table_names(["dag_run"], session) - assert len(archived_table_names) == 1 - assert archived_table_names[0] in ARCHIVED_TABLES_FROM_DB_MIGRATIONS + assert len(archived_table_names) == 0 def test_no_models_missing(self): """ diff --git a/airflow-core/tests/unit/utils/test_db_manager.py b/airflow-core/tests/unit/utils/test_db_manager.py index 075e87c0cbe37..e46ac4d4265e1 100644 --- a/airflow-core/tests/unit/utils/test_db_manager.py +++ b/airflow-core/tests/unit/utils/test_db_manager.py @@ -19,89 +19,13 @@ from unittest import mock import pytest -from sqlalchemy import Table -from airflow.exceptions import AirflowException from airflow.models import Base -from airflow.utils.db import downgrade, initdb -from airflow.utils.db_manager import BaseDBManager, RunDBManager - -from tests_common.test_utils.config import conf_vars +from airflow.utils.db_manager import BaseDBManager pytestmark = [pytest.mark.db_test] -class TestRunDBManager: - @conf_vars( - {("database", "external_db_managers"): "airflow.providers.fab.auth_manager.models.db.FABDBManager"} - ) - def test_db_manager_uses_config(self): - from airflow.providers.fab.auth_manager.models.db import FABDBManager - - run_db_manager = RunDBManager() - assert run_db_manager._managers == [FABDBManager] - - @conf_vars( - {("database", "external_db_managers"): "airflow.providers.fab.auth_manager.models.db.FABDBManager"} - ) - def test_defining_table_same_name_as_airflow_table_name_raises(self): - from sqlalchemy import Column, Integer, String - - run_db_manager = RunDBManager() - metadata = run_db_manager._managers[0].metadata - # Add dag_run table to metadata - mytable = Table( - "dag_run", metadata, Column("id", Integer, primary_key=True), Column("name", String(50)) - ) - metadata._add_table("dag_run", None, mytable) - with pytest.raises(AirflowException, match="Table 'dag_run' already exists in the Airflow metadata"): - run_db_manager.validate() - metadata._remove_table("dag_run", None) - - @mock.patch.object(RunDBManager, "downgrade") - @mock.patch.object(RunDBManager, "upgradedb") - @mock.patch.object(RunDBManager, "initdb") - def test_init_db_calls_rundbmanager(self, mock_initdb, mock_upgrade_db, mock_downgrade_db, session): - initdb(session=session) - mock_initdb.assert_called() - mock_initdb.assert_called_once_with(session) - mock_downgrade_db.assert_not_called() - - @mock.patch.object(RunDBManager, "downgrade") - @mock.patch.object(RunDBManager, "upgradedb") - @mock.patch.object(RunDBManager, "initdb") - @mock.patch("alembic.command") - def test_downgrade_dont_call_rundbmanager( - self, mock_alembic_command, mock_initdb, mock_upgrade_db, mock_downgrade_db, session - ): - downgrade(to_revision="base") - mock_alembic_command.downgrade.assert_called_once_with(mock.ANY, revision="base", sql=False) - mock_upgrade_db.assert_not_called() - mock_initdb.assert_not_called() - mock_downgrade_db.assert_not_called() - - @conf_vars( - {("database", "external_db_managers"): "airflow.providers.fab.auth_manager.models.db.FABDBManager"} - ) - @mock.patch("airflow.providers.fab.auth_manager.models.db.FABDBManager") - def test_rundbmanager_calls_dbmanager_methods(self, mock_fabdb_manager, session): - mock_fabdb_manager.supports_table_dropping = True - fabdb_manager = mock_fabdb_manager.return_value - ext_db = RunDBManager() - # initdb - ext_db.initdb(session=session) - fabdb_manager.initdb.assert_called_once() - # upgradedb - ext_db.upgradedb(session=session) - fabdb_manager.upgradedb.assert_called_once() - # downgrade - ext_db.downgrade(session=session) - mock_fabdb_manager.return_value.downgrade.assert_called_once() - connection = mock.MagicMock() - ext_db.drop_tables(session, connection) - mock_fabdb_manager.return_value.drop_tables.assert_called_once_with(connection) - - class MockDBManager(BaseDBManager): metadata = Base.metadata version_table_name = "mock_alembic_version" diff --git a/airflow-ctl/pyproject.toml b/airflow-ctl/pyproject.toml index 734f43465afe9..63482c05a5536 100644 --- a/airflow-ctl/pyproject.toml +++ b/airflow-ctl/pyproject.toml @@ -20,7 +20,11 @@ name = "apache-airflow-ctl" dynamic = ["version"] description = "Apache Airflow command line tool for communicating with an Apache Airflow, using the API." readme = { file = "README.md", content-type = "text/markdown" } -requires-python = ">=3.10, !=3.13" +# We do not want to upper-bind Python version, as we do not know if we will support Python 3.14+ +# out-of-the box. Airflow-ctl is a small tool that does not have many dependencies and does not use +# sophisticated features of Python, so it should work with Python 3.14+ once all it's dependencies are +# updated to support it. +requires-python = ">=3.10" dependencies = [ # TODO there could be still missing deps such as airflow-core "argcomplete>=1.10", diff --git a/chart/pyproject.toml b/chart/pyproject.toml index a03b08a74694e..6e9b1773bccd1 100644 --- a/chart/pyproject.toml +++ b/chart/pyproject.toml @@ -24,7 +24,7 @@ build-backend = "hatchling.build" [project] name = "apache-airflow-helm-chart" description = "Programmatically author, schedule and monitor data pipelines" -requires-python = ">=3.10,!=3.13" +requires-python = ">=3.10" authors = [ { name = "Apache Software Foundation", email = "dev@airflow.apache.org" }, ] diff --git a/contributing-docs/07_local_virtualenv.rst b/contributing-docs/07_local_virtualenv.rst index adb97fe99fa9c..35ddb4b6f1a1d 100644 --- a/contributing-docs/07_local_virtualenv.rst +++ b/contributing-docs/07_local_virtualenv.rst @@ -34,7 +34,7 @@ Required Software Packages Use system-level package managers like yum, apt-get for Linux, or Homebrew for macOS to install required software packages: -* Python (One of: 3.10, 3.11, 3.12) +* Python (One of: 3.10, 3.11, 3.12, 3.13) * MySQL 5.7+ * libxml * helm (only for helm chart tests) diff --git a/dev/README_RELEASE_AIRFLOW.md b/dev/README_RELEASE_AIRFLOW.md index 037ab5464241b..de201c44f093c 100644 --- a/dev/README_RELEASE_AIRFLOW.md +++ b/dev/README_RELEASE_AIRFLOW.md @@ -918,7 +918,7 @@ the older branches, you should set the "skip" field to true. ## Verify production images ```shell script -for PYTHON in 3.10 3.11 3.12 +for PYTHON in 3.10 3.11 3.12 3.13 do docker pull apache/airflow:${VERSION}-python${PYTHON} breeze prod-image verify --image-name apache/airflow:${VERSION}-python${PYTHON} diff --git a/dev/breeze/README.md b/dev/breeze/README.md index e9a18fe9aac24..b80e3a60c3db2 100644 --- a/dev/breeze/README.md +++ b/dev/breeze/README.md @@ -135,6 +135,6 @@ PLEASE DO NOT MODIFY THE HASH BELOW! IT IS AUTOMATICALLY UPDATED BY PRE-COMMIT. --------------------------------------------------------------------------------------------------------- -Package config hash: 8f6ecf8ab58351038013b3b89ffe07914b16d012e426899d04d1c821c20e0d13da7aa5fa8517786d21df6b1f367a44ac32cf7200f44d4cf579b4ce5cd77a804f +Package config hash: ca0dde3dc813baf255b7f645c73025b9ce0729d24acf33499f15ba62a6d94cb6e665fa84c575fc88b5a20f3a1c73ded735ff2dcfad353d2b91434b03e11a4467 --------------------------------------------------------------------------------------------------------- diff --git a/dev/breeze/doc/ci/02_images.md b/dev/breeze/doc/ci/02_images.md index 430c41df24701..3514b6a0e51cf 100644 --- a/dev/breeze/doc/ci/02_images.md +++ b/dev/breeze/doc/ci/02_images.md @@ -567,7 +567,7 @@ percent-encoded when you access them via UI (/ = %2F) | PROD image | airflow/\/prod/python\ | faster to build or pull. Production image optimized for size. | - \ might be either "main" or "v2-\*-test" -- \ - Python version (Major + Minor).Should be one of \["3.10", "3.11", "3.12" \]. +- \ - Python version (Major + Minor).Should be one of \["3.10", "3.11", "3.12", "3.13" \]. ---- diff --git a/dev/breeze/doc/images/output-commands.svg b/dev/breeze/doc/images/output-commands.svg index 9b5bd8dd912c6..58c3c6d499765 100644 --- a/dev/breeze/doc/images/output-commands.svg +++ b/dev/breeze/doc/images/output-commands.svg @@ -327,7 +327,7 @@ ╭─ Execution mode ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ --python-pPython major/minor version used in Airflow image for    images.                                                 -(>3.10< | 3.11 | 3.12)                                  +(>3.10< | 3.11 | 3.12 | 3.13)                           [default: 3.10]                                         --integrationCore Integrations to enable when running (can be more   than one).                                              diff --git a/dev/breeze/doc/images/output_ci-image_build.svg b/dev/breeze/doc/images/output_ci-image_build.svg index 7e8b2d9b429ec..ec16a59fb4bb8 100644 --- a/dev/breeze/doc/images/output_ci-image_build.svg +++ b/dev/breeze/doc/images/output_ci-image_build.svg @@ -328,7 +328,7 @@ ╭─ Basic usage ────────────────────────────────────────────────────────────────────────────────────────────────────────╮ --python-pPython major/minor version used in Airflow image for images. -(>3.10< | 3.11 | 3.12)                                       +(>3.10< | 3.11 | 3.12 | 3.13)                                [default: 3.10]                                              --upgrade-to-newer-dependencies-uWhen set, upgrade all PIP packages to latest. --upgrade-on-failure/--no-upgrade-on-failureWhen set, attempt to run upgrade to newer dependencies when        @@ -354,7 +354,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --run-in-parallelRun the operation in parallel on all or selected subset of parameters. --skip-cleanupSkip cleanup of temporary files created during parallel run. ╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_ci-image_build.txt b/dev/breeze/doc/images/output_ci-image_build.txt index f1d7c7a717a1a..5f2f3dfc1f009 100644 --- a/dev/breeze/doc/images/output_ci-image_build.txt +++ b/dev/breeze/doc/images/output_ci-image_build.txt @@ -1 +1 @@ -c075a5d7ad1134165d16d1c938161fc9 +f143beb241b9f46078f6f295de8ac1a2 diff --git a/dev/breeze/doc/images/output_ci-image_load.svg b/dev/breeze/doc/images/output_ci-image_load.svg index aa8e15d30fec9..bcf39f4a31b75 100644 --- a/dev/breeze/doc/images/output_ci-image_load.svg +++ b/dev/breeze/doc/images/output_ci-image_load.svg @@ -171,7 +171,7 @@ ╭─ Load image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ --python-pPython major/minor version used in Airflow image for images. -(>3.10< | 3.11 | 3.12)                                       +(>3.10< | 3.11 | 3.12 | 3.13)                                [default: 3.10]                                              --platformPlatform for Airflow image.                                (linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64) diff --git a/dev/breeze/doc/images/output_ci-image_load.txt b/dev/breeze/doc/images/output_ci-image_load.txt index d7d932a7463cb..6df709bb34984 100644 --- a/dev/breeze/doc/images/output_ci-image_load.txt +++ b/dev/breeze/doc/images/output_ci-image_load.txt @@ -1 +1 @@ -d86e2e217fe99bd74580f751dc4c51bd +d21c3b7345c0680cd0adc5b71a444b8d diff --git a/dev/breeze/doc/images/output_ci-image_pull.svg b/dev/breeze/doc/images/output_ci-image_pull.svg index b6f6da58aa94c..fdecba3640527 100644 --- a/dev/breeze/doc/images/output_ci-image_pull.svg +++ b/dev/breeze/doc/images/output_ci-image_pull.svg @@ -153,7 +153,7 @@ Pull and optionally verify CI images - possibly in parallel for all Python versions. ╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --verifyVerify image. --wait-for-imageWait until image is available. @@ -164,7 +164,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --skip-cleanupSkip cleanup of temporary files created during parallel run. --debug-resourcesWhether to show resource information while running in parallel. --include-success-outputsWhether to include outputs of successful runs (not shown by default). diff --git a/dev/breeze/doc/images/output_ci-image_pull.txt b/dev/breeze/doc/images/output_ci-image_pull.txt index 0de3a5b933769..8e973f27359f0 100644 --- a/dev/breeze/doc/images/output_ci-image_pull.txt +++ b/dev/breeze/doc/images/output_ci-image_pull.txt @@ -1 +1 @@ -c29b9a5e5bf4b503f4e0310006414597 +06adeccc3bd68200e745f2df98e7cfb2 diff --git a/dev/breeze/doc/images/output_ci-image_save.svg b/dev/breeze/doc/images/output_ci-image_save.svg index e803d2b341142..f39fa22b7f0b3 100644 --- a/dev/breeze/doc/images/output_ci-image_save.svg +++ b/dev/breeze/doc/images/output_ci-image_save.svg @@ -117,7 +117,7 @@ Save CI image to a file. ╭─ Save image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64) --github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] diff --git a/dev/breeze/doc/images/output_ci-image_save.txt b/dev/breeze/doc/images/output_ci-image_save.txt index 5e457d7c315e1..9332259fd6b98 100644 --- a/dev/breeze/doc/images/output_ci-image_save.txt +++ b/dev/breeze/doc/images/output_ci-image_save.txt @@ -1 +1 @@ -807d082ba3edd987a78c50608eb280c3 +d399fd57269abeeb164c8e4365ee6cf0 diff --git a/dev/breeze/doc/images/output_ci-image_verify.svg b/dev/breeze/doc/images/output_ci-image_verify.svg index ae6639d9a7167..d3c592587fab7 100644 --- a/dev/breeze/doc/images/output_ci-image_verify.svg +++ b/dev/breeze/doc/images/output_ci-image_verify.svg @@ -154,7 +154,7 @@ ╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ --image-name-nName of the image to verify (overrides --python).(TEXT) ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --pullPull image is missing before attempting to verify it. ╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ @@ -164,7 +164,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --skip-cleanupSkip cleanup of temporary files created during parallel run. --debug-resourcesWhether to show resource information while running in parallel. --include-success-outputsWhether to include outputs of successful runs (not shown by default). diff --git a/dev/breeze/doc/images/output_ci-image_verify.txt b/dev/breeze/doc/images/output_ci-image_verify.txt index e1300b3163f00..21bce565e3467 100644 --- a/dev/breeze/doc/images/output_ci-image_verify.txt +++ b/dev/breeze/doc/images/output_ci-image_verify.txt @@ -1 +1 @@ -af0c91ebd8bb2ed858bdad86c43c579a +e79316e9ece043436f959f94158d3648 diff --git a/dev/breeze/doc/images/output_k8s_build-k8s-image.svg b/dev/breeze/doc/images/output_k8s_build-k8s-image.svg index b810baa3752f0..92ee8f8e11f9a 100644 --- a/dev/breeze/doc/images/output_k8s_build-k8s-image.svg +++ b/dev/breeze/doc/images/output_k8s_build-k8s-image.svg @@ -154,7 +154,7 @@ ╭─ Build image flags ──────────────────────────────────────────────────────────────────────────────────────────────────╮ --python-pPython major/minor version used in Airflow image for images. -(>3.10< | 3.11 | 3.12)                                       +(>3.10< | 3.11 | 3.12 | 3.13)                                [default: 3.10]                                              --rebuild-base-imageRebuilds base Airflow image before building K8S image. --copy-local-sources/--no-copy-local-sourcesCopy local sources to the image.[default: copy-local-sources] @@ -167,7 +167,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --skip-cleanupSkip cleanup of temporary files created during parallel run. --debug-resourcesWhether to show resource information while running in parallel. --include-success-outputsWhether to include outputs of successful runs (not shown by default). diff --git a/dev/breeze/doc/images/output_k8s_build-k8s-image.txt b/dev/breeze/doc/images/output_k8s_build-k8s-image.txt index 67f0160b50209..43aef8ba419d5 100644 --- a/dev/breeze/doc/images/output_k8s_build-k8s-image.txt +++ b/dev/breeze/doc/images/output_k8s_build-k8s-image.txt @@ -1 +1 @@ -40442787a9fafb301236e1821099092a +28e63d037b162335336e2414ea35f8b8 diff --git a/dev/breeze/doc/images/output_k8s_configure-cluster.svg b/dev/breeze/doc/images/output_k8s_configure-cluster.svg index e35f2713ef075..994b13646ce18 100644 --- a/dev/breeze/doc/images/output_k8s_configure-cluster.svg +++ b/dev/breeze/doc/images/output_k8s_configure-cluster.svg @@ -157,7 +157,7 @@ parallel). ╭─ Configure cluster flags ────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             @@ -170,7 +170,7 @@ (INTEGER RANGE)                                                                         [default: 2; 1<=x<=4]                                                                   --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) [default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    --skip-cleanupSkip cleanup of temporary files created during parallel run. diff --git a/dev/breeze/doc/images/output_k8s_configure-cluster.txt b/dev/breeze/doc/images/output_k8s_configure-cluster.txt index 7ecd42d760681..eddcdfb23e715 100644 --- a/dev/breeze/doc/images/output_k8s_configure-cluster.txt +++ b/dev/breeze/doc/images/output_k8s_configure-cluster.txt @@ -1 +1 @@ -6e93a011facfa3eef2cf3daccdbac9bd +c8f706b2b033bdb923cff390c2c79fe2 diff --git a/dev/breeze/doc/images/output_k8s_create-cluster.svg b/dev/breeze/doc/images/output_k8s_create-cluster.svg index 4e825353c2e09..7e0cdb58e7d3e 100644 --- a/dev/breeze/doc/images/output_k8s_create-cluster.svg +++ b/dev/breeze/doc/images/output_k8s_create-cluster.svg @@ -1,4 +1,4 @@ - + Create a KinD Cluster for Python and Kubernetes version specified (optionally create all clusters in parallel). ╭─ K8S cluster creation flags ─────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) -[default: 3.10]                                              ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             -[default: v1.30.10]                                    ---force-recreate-clusterForce recreation of the cluster even if it is already created. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of parameters. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful runs (not shown by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +--python-pPython major/minor version used in Airflow image for images. +(>3.10< | 3.11 | 3.12 | 3.13)                                +[default: 3.10]                                              +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             +[default: v1.30.10]                                    +--force-recreate-clusterForce recreation of the cluster even if it is already created. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of parameters. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.10 3.11 3.12 3.13]                                                 +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful runs (not shown by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_k8s_create-cluster.txt b/dev/breeze/doc/images/output_k8s_create-cluster.txt index 8d6330198c8b5..44c6c746a62bb 100644 --- a/dev/breeze/doc/images/output_k8s_create-cluster.txt +++ b/dev/breeze/doc/images/output_k8s_create-cluster.txt @@ -1 +1 @@ -10b1e7982d783b404e854b6bccec5787 +9bcbf201e9966346691d7c0f1e0f91d7 diff --git a/dev/breeze/doc/images/output_k8s_delete-cluster.svg b/dev/breeze/doc/images/output_k8s_delete-cluster.svg index 0dbc5d38b1aea..498129efc97fd 100644 --- a/dev/breeze/doc/images/output_k8s_delete-cluster.svg +++ b/dev/breeze/doc/images/output_k8s_delete-cluster.svg @@ -114,7 +114,7 @@ Delete the current KinD Cluster (optionally all clusters). ╭─ K8S cluster delete flags ───────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             diff --git a/dev/breeze/doc/images/output_k8s_delete-cluster.txt b/dev/breeze/doc/images/output_k8s_delete-cluster.txt index 169719432d2ff..b08616491cca3 100644 --- a/dev/breeze/doc/images/output_k8s_delete-cluster.txt +++ b/dev/breeze/doc/images/output_k8s_delete-cluster.txt @@ -1 +1 @@ -ad0eea9be87ef3d5451698aab8a25358 +ce317cf63c3bf6bb7c91e65ba0fb7f70 diff --git a/dev/breeze/doc/images/output_k8s_deploy-airflow.svg b/dev/breeze/doc/images/output_k8s_deploy-airflow.svg index e517993e03a1c..160c88efa8f99 100644 --- a/dev/breeze/doc/images/output_k8s_deploy-airflow.svg +++ b/dev/breeze/doc/images/output_k8s_deploy-airflow.svg @@ -1,4 +1,4 @@ - + Deploy airflow image to the current KinD cluster (or all clusters). ╭─ Airflow deploy flags ───────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) -[default: 3.10]                                              ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             -[default: v1.30.10]                                    ---executorExecutor to use for a kubernetes cluster.                                                -(>LocalExecutor< | KubernetesExecutor | CeleryExecutor | CeleryKubernetesExecutor |      -EdgeExecutor)                                                                            -[default: LocalExecutor]                                                                 ---upgradeUpgrade Helm Chart rather than installing it. ---wait-time-in-secondsWait for Airflow api-server for specified number of seconds.(INTEGER RANGE) ---use-standard-namingUse standard naming. ---multi-namespace-modeUse multi namespace mode. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of parameters. ---parallelismMaximum number of processes to use while running the operation in parallel for cluster  -operations.                                                                             -(INTEGER RANGE)                                                                         -[default: 2; 1<=x<=4]                                                                   ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      ---kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) -[default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful runs (not shown by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +--python-pPython major/minor version used in Airflow image for images. +(>3.10< | 3.11 | 3.12 | 3.13)                                +[default: 3.10]                                              +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             +[default: v1.30.10]                                    +--executorExecutor to use for a kubernetes cluster.                                                +(>LocalExecutor< | KubernetesExecutor | CeleryExecutor | CeleryKubernetesExecutor |      +EdgeExecutor)                                                                            +[default: LocalExecutor]                                                                 +--upgradeUpgrade Helm Chart rather than installing it. +--wait-time-in-secondsWait for Airflow api-server for specified number of seconds.(INTEGER RANGE) +--use-standard-namingUse standard naming. +--multi-namespace-modeUse multi namespace mode. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Parallel options ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of parameters. +--parallelismMaximum number of processes to use while running the operation in parallel for cluster  +operations.                                                                             +(INTEGER RANGE)                                                                         +[default: 2; 1<=x<=4]                                                                   +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.10 3.11 3.12 3.13]                                                 +--kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) +[default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful runs (not shown by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_k8s_deploy-airflow.txt b/dev/breeze/doc/images/output_k8s_deploy-airflow.txt index 9b82b31144831..0507986238b13 100644 --- a/dev/breeze/doc/images/output_k8s_deploy-airflow.txt +++ b/dev/breeze/doc/images/output_k8s_deploy-airflow.txt @@ -1 +1 @@ -c31bffdf3bba32ed1278cd59f20c2faf +12183422cdea79b28d2885ebcd41cf62 diff --git a/dev/breeze/doc/images/output_k8s_k9s.svg b/dev/breeze/doc/images/output_k8s_k9s.svg index c3b3058d3fbc9..5ba17f2d03279 100644 --- a/dev/breeze/doc/images/output_k8s_k9s.svg +++ b/dev/breeze/doc/images/output_k8s_k9s.svg @@ -119,7 +119,7 @@ ╭─ K8S k9s flags ──────────────────────────────────────────────────────────────────────────────────────────────────────╮ --use-dockerUse Docker to start k8s executor (otherwise k9s from PATH is used and only run with docker if not found on PATH).                                                                     ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             diff --git a/dev/breeze/doc/images/output_k8s_k9s.txt b/dev/breeze/doc/images/output_k8s_k9s.txt index 7191c67d24e31..9ba65cd1c4afc 100644 --- a/dev/breeze/doc/images/output_k8s_k9s.txt +++ b/dev/breeze/doc/images/output_k8s_k9s.txt @@ -1 +1 @@ -efedbc9d3a96d716fa0f560e8dc5ea7e +fe0b5aae376d9c9936f8a6817ab038e8 diff --git a/dev/breeze/doc/images/output_k8s_logs.svg b/dev/breeze/doc/images/output_k8s_logs.svg index 6e1399a36dc3d..5d60ba7ad5215 100644 --- a/dev/breeze/doc/images/output_k8s_logs.svg +++ b/dev/breeze/doc/images/output_k8s_logs.svg @@ -115,7 +115,7 @@ Dump k8s logs to ${TMP_DIR}/kind_logs_<cluster_name> directory (optionally all clusters). ╭─ K8S logs flags ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             diff --git a/dev/breeze/doc/images/output_k8s_logs.txt b/dev/breeze/doc/images/output_k8s_logs.txt index 46cec2dd8bb7e..f505f1e2011c4 100644 --- a/dev/breeze/doc/images/output_k8s_logs.txt +++ b/dev/breeze/doc/images/output_k8s_logs.txt @@ -1 +1 @@ -cf00d923f26a0d331dd7fe7251c56f37 +47e33fd09941211b2c3f5092a2bbc832 diff --git a/dev/breeze/doc/images/output_k8s_run-complete-tests.svg b/dev/breeze/doc/images/output_k8s_run-complete-tests.svg index 80014a73460c5..896d2ee1a340d 100644 --- a/dev/breeze/doc/images/output_k8s_run-complete-tests.svg +++ b/dev/breeze/doc/images/output_k8s_run-complete-tests.svg @@ -228,7 +228,7 @@ [default: use-uv]                                           ╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ ╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             @@ -246,7 +246,7 @@ (INTEGER RANGE)                                                                         [default: 2; 1<=x<=4]                                                                   --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) [default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    --skip-cleanupSkip cleanup of temporary files created during parallel run. diff --git a/dev/breeze/doc/images/output_k8s_run-complete-tests.txt b/dev/breeze/doc/images/output_k8s_run-complete-tests.txt index 128524bb500e8..e0007b1ce7ea9 100644 --- a/dev/breeze/doc/images/output_k8s_run-complete-tests.txt +++ b/dev/breeze/doc/images/output_k8s_run-complete-tests.txt @@ -1 +1 @@ -90311a31704a88dbeadc5955a4b26452 +0ac32eea997eab41404062f20e11fe5b diff --git a/dev/breeze/doc/images/output_k8s_shell.svg b/dev/breeze/doc/images/output_k8s_shell.svg index 84449a4aaf6df..b79866cfabeed 100644 --- a/dev/breeze/doc/images/output_k8s_shell.svg +++ b/dev/breeze/doc/images/output_k8s_shell.svg @@ -126,7 +126,7 @@ Run shell environment for the current KinD cluster. ╭─ K8S shell flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             diff --git a/dev/breeze/doc/images/output_k8s_shell.txt b/dev/breeze/doc/images/output_k8s_shell.txt index 16014bad6d105..471ef4a1e5696 100644 --- a/dev/breeze/doc/images/output_k8s_shell.txt +++ b/dev/breeze/doc/images/output_k8s_shell.txt @@ -1 +1 @@ -b364a237eaa04d37db3b51620566ac85 +3e9d5bed9a63a9b13d066d44a9a00a96 diff --git a/dev/breeze/doc/images/output_k8s_status.svg b/dev/breeze/doc/images/output_k8s_status.svg index 5d4f080a5ebe9..f275d8646ebfb 100644 --- a/dev/breeze/doc/images/output_k8s_status.svg +++ b/dev/breeze/doc/images/output_k8s_status.svg @@ -1,4 +1,4 @@ - + Check status of the current cluster and airflow deployed to it (optionally all clusters). ╭─ K8S cluster status flags ───────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) -[default: 3.10]                                              ---kubernetes-versionKubernetes version used to create the KinD cluster of. -(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             -[default: v1.30.10]                                    ---wait-time-in-secondsWait for Airflow api-server for specified number of seconds.(INTEGER RANGE) ---allApply it to all created clusters -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +--python-pPython major/minor version used in Airflow image for images. +(>3.10< | 3.11 | 3.12 | 3.13)                                +[default: 3.10]                                              +--kubernetes-versionKubernetes version used to create the KinD cluster of. +(>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             +[default: v1.30.10]                                    +--wait-time-in-secondsWait for Airflow api-server for specified number of seconds.(INTEGER RANGE) +--allApply it to all created clusters +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_k8s_status.txt b/dev/breeze/doc/images/output_k8s_status.txt index e9c655c06abeb..d23f1228d55b7 100644 --- a/dev/breeze/doc/images/output_k8s_status.txt +++ b/dev/breeze/doc/images/output_k8s_status.txt @@ -1 +1 @@ -45ae1dbd57e2d8e7c512bf32f5ab4193 +7e417976f692dd3ebbe4247e4d8da3fd diff --git a/dev/breeze/doc/images/output_k8s_tests.svg b/dev/breeze/doc/images/output_k8s_tests.svg index 1e15dd87efd79..148af2adc4e6b 100644 --- a/dev/breeze/doc/images/output_k8s_tests.svg +++ b/dev/breeze/doc/images/output_k8s_tests.svg @@ -168,7 +168,7 @@ Run tests against the current KinD cluster (optionally for all clusters in parallel). ╭─ K8S tests flags ────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             @@ -186,7 +186,7 @@ (INTEGER RANGE)                                                                         [default: 2; 1<=x<=4]                                                                   --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) [default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    --skip-cleanupSkip cleanup of temporary files created during parallel run. diff --git a/dev/breeze/doc/images/output_k8s_tests.txt b/dev/breeze/doc/images/output_k8s_tests.txt index f5a8137e26795..899187612e931 100644 --- a/dev/breeze/doc/images/output_k8s_tests.txt +++ b/dev/breeze/doc/images/output_k8s_tests.txt @@ -1 +1 @@ -73297fbd99e02da431cb693f8bd69049 +82800ea41d36592ffae10dfbdb8cd0f9 diff --git a/dev/breeze/doc/images/output_k8s_upload-k8s-image.svg b/dev/breeze/doc/images/output_k8s_upload-k8s-image.svg index 5aff919e8c664..74ed3d7e23f5b 100644 --- a/dev/breeze/doc/images/output_k8s_upload-k8s-image.svg +++ b/dev/breeze/doc/images/output_k8s_upload-k8s-image.svg @@ -150,7 +150,7 @@ Upload k8s-ready airflow image to the KinD cluster (optionally to all clusters in parallel) ╭─ Upload image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --kubernetes-versionKubernetes version used to create the KinD cluster of. (>v1.30.10< | v1.31.6 | v1.32.3 | v1.33.0)             @@ -162,7 +162,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --kubernetes-versionsKubernetes versions used to run in parallel (space separated).(TEXT) [default: v1.30.10 v1.31.6 v1.32.3 v1.33.0]                    --skip-cleanupSkip cleanup of temporary files created during parallel run. diff --git a/dev/breeze/doc/images/output_k8s_upload-k8s-image.txt b/dev/breeze/doc/images/output_k8s_upload-k8s-image.txt index c50b134b778b4..873c26da617a6 100644 --- a/dev/breeze/doc/images/output_k8s_upload-k8s-image.txt +++ b/dev/breeze/doc/images/output_k8s_upload-k8s-image.txt @@ -1 +1 @@ -7b05f647b8a0009a1a18471678dbd14d +1c7133c33d3f1842ed5de2d51267dc6b diff --git a/dev/breeze/doc/images/output_prod-image_build.svg b/dev/breeze/doc/images/output_prod-image_build.svg index 94e6a33e2c482..7754b23f1767b 100644 --- a/dev/breeze/doc/images/output_prod-image_build.svg +++ b/dev/breeze/doc/images/output_prod-image_build.svg @@ -1,4 +1,4 @@ - + option is ignored when building images.                                               (TEXT)                                                                                --install-airflow-version-VInstall version of Airflow from PyPI.(TEXT) ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) -[default: 3.10]                                              ---version-suffixVersion suffix used for PyPI packages (a1, a2, b1, rc1, rc2, .dev0, .dev1, .post1,    -.post2 etc.). Note the `.` is need in `.dev0` and `.post`. Might be followed with     -+local_version                                                                        -(TEXT)                                                                                -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Building images in parallel ────────────────────────────────────────────────────────────────────────────────────────╮ ---run-in-parallelRun the operation in parallel on all or selected subset of parameters. ---parallelismMaximum number of processes to use while running the operation in parallel. -(INTEGER RANGE)                                                             -[default: 4; 1<=x<=8]                                                       ---python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      ---skip-cleanupSkip cleanup of temporary files created during parallel run. ---debug-resourcesWhether to show resource information while running in parallel. ---include-success-outputsWhether to include outputs of successful runs (not shown by default). -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced build options (for power users) ───────────────────────────────────────────────────────────────────────────╮ ---additional-pip-install-flagsAdditional flags added to `pip install` commands (except reinstalling `pip`        -itself).                                                                           -(TEXT)                                                                             ---commit-shaCommit SHA that is used to build the images.(TEXT) ---debian-versionDebian version used in Airflow image as base for building images.(bookworm) -[default: bookworm]                                               ---python-imageIf specified this is the base python image used to build the image. Should be      -something like: python:VERSION-slim-bookworm.                                      -(TEXT)                                                                             ---use-uv/--no-use-uvUse uv instead of pip as packaging tool to build the image.[default: no-use-uv] ---uv-http-timeoutTimeout for requests that UV makes (only used in case of UV builds). -(INTEGER RANGE)                                                      -[default: 300; x>=1]                                                 -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Selecting constraint location (for power users) ────────────────────────────────────────────────────────────────────╮ ---airflow-constraints-locationLocation of airflow constraints to use (remote URL or local context file).(TEXT) ---airflow-constraints-modeMode of constraints for Airflow for PROD image building.                -(constraints | constraints-no-providers | constraints-source-providers) -[default: constraints]                                                  ---airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Choosing dependencies and extras (for power users) ─────────────────────────────────────────────────────────────────╮ ---airflow-extrasExtras to install by default.                                                    -(TEXT)                                                                           -[default:                                                                        -aiobotocore,amazon,async,celery,cncf-kubernetes,common-io,common-messaging,dock… ---additional-airflow-extrasAdditional extra package while installing Airflow in the image.(TEXT) ---additional-python-depsAdditional python dependencies to use when building the images.(TEXT) ---dev-apt-depsApt dev dependencies to use when building the images.(TEXT) ---additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) ---dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) ---additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) ---additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) ---runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) ---additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) ---runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) ---additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) ---additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Advanced customization options (for specific customization needs) ──────────────────────────────────────────────────╮ ---installation-methodInstall Airflow from: sources or PyPI.(. | apache-airflow) -[default: .]                           ---install-airflow-referenceInstall Airflow using GitHub tag or branch.(TEXT) ---install-distributions-from-contextInstall distributions from local docker-context-files when building   -image. Implies --disable-airflow-repo-cache ---install-mysql-client-typeWhich client to choose when installing.(mariadb | mysql) ---cleanup-contextClean up docker context files before running build (cannot be used    -together with --install-distributions-from-context).                  ---use-constraints-for-context-distributionsUses constraints for context distributions installation - either from -constraints store in docker-context-files or from github.             ---disable-airflow-repo-cacheDisable cache from Airflow repository during building. ---disable-mysql-client-installationDo not install MySQL client. ---disable-mssql-client-installationDo not install MsSQl client. ---disable-postgres-client-installationDo not install Postgres client. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT) -[default: autodetect]                                          ---platformPlatform for Airflow image.                                                          -(linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64 | linux/amd64,linux/arm64) ---pushPush image after building it. ---prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  -image).                                                                                    -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Github authentication ──────────────────────────────────────────────────────────────────────────────────────────────╮ ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---github-tokenThe token used to authenticate to GitHub.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---answer-aForce answer to questions.(y | n | q | yes | no | quit) ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---verbose-vPrint verbose information about performed steps. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +--python-pPython major/minor version used in Airflow image for images. +(>3.10< | 3.11 | 3.12 | 3.13)                                +[default: 3.10]                                              +--version-suffixVersion suffix used for PyPI packages (a1, a2, b1, rc1, rc2, .dev0, .dev1, .post1,    +.post2 etc.). Note the `.` is need in `.dev0` and `.post`. Might be followed with     ++local_version                                                                        +(TEXT)                                                                                +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Building images in parallel ────────────────────────────────────────────────────────────────────────────────────────╮ +--run-in-parallelRun the operation in parallel on all or selected subset of parameters. +--parallelismMaximum number of processes to use while running the operation in parallel. +(INTEGER RANGE)                                                             +[default: 4; 1<=x<=8]                                                       +--python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) +[default: 3.10 3.11 3.12 3.13]                                                 +--skip-cleanupSkip cleanup of temporary files created during parallel run. +--debug-resourcesWhether to show resource information while running in parallel. +--include-success-outputsWhether to include outputs of successful runs (not shown by default). +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced build options (for power users) ───────────────────────────────────────────────────────────────────────────╮ +--additional-pip-install-flagsAdditional flags added to `pip install` commands (except reinstalling `pip`        +itself).                                                                           +(TEXT)                                                                             +--commit-shaCommit SHA that is used to build the images.(TEXT) +--debian-versionDebian version used in Airflow image as base for building images.(bookworm) +[default: bookworm]                                               +--python-imageIf specified this is the base python image used to build the image. Should be      +something like: python:VERSION-slim-bookworm.                                      +(TEXT)                                                                             +--use-uv/--no-use-uvUse uv instead of pip as packaging tool to build the image.[default: no-use-uv] +--uv-http-timeoutTimeout for requests that UV makes (only used in case of UV builds). +(INTEGER RANGE)                                                      +[default: 300; x>=1]                                                 +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Selecting constraint location (for power users) ────────────────────────────────────────────────────────────────────╮ +--airflow-constraints-locationLocation of airflow constraints to use (remote URL or local context file).(TEXT) +--airflow-constraints-modeMode of constraints for Airflow for PROD image building.                +(constraints | constraints-no-providers | constraints-source-providers) +[default: constraints]                                                  +--airflow-constraints-referenceConstraint reference to use when building the image.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Choosing dependencies and extras (for power users) ─────────────────────────────────────────────────────────────────╮ +--airflow-extrasExtras to install by default.                                                    +(TEXT)                                                                           +[default:                                                                        +aiobotocore,amazon,async,celery,cncf-kubernetes,common-io,common-messaging,dock… +--additional-airflow-extrasAdditional extra package while installing Airflow in the image.(TEXT) +--additional-python-depsAdditional python dependencies to use when building the images.(TEXT) +--dev-apt-depsApt dev dependencies to use when building the images.(TEXT) +--additional-dev-apt-depsAdditional apt dev dependencies to use when building the images.(TEXT) +--dev-apt-commandCommand executed before dev apt deps are installed.(TEXT) +--additional-dev-apt-commandAdditional command executed before dev apt deps are installed.(TEXT) +--additional-dev-apt-envAdditional environment variables set when adding dev dependencies.(TEXT) +--runtime-apt-depsApt runtime dependencies to use when building the images.(TEXT) +--additional-runtime-apt-depsAdditional apt runtime dependencies to use when building the images.(TEXT) +--runtime-apt-commandCommand executed before runtime apt deps are installed.(TEXT) +--additional-runtime-apt-commandAdditional command executed before runtime apt deps are installed.(TEXT) +--additional-runtime-apt-envAdditional environment variables set when adding runtime dependencies.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Advanced customization options (for specific customization needs) ──────────────────────────────────────────────────╮ +--installation-methodInstall Airflow from: sources or PyPI.(. | apache-airflow) +[default: .]                           +--install-airflow-referenceInstall Airflow using GitHub tag or branch.(TEXT) +--install-distributions-from-contextInstall distributions from local docker-context-files when building   +image. Implies --disable-airflow-repo-cache +--install-mysql-client-typeWhich client to choose when installing.(mariadb | mysql) +--cleanup-contextClean up docker context files before running build (cannot be used    +together with --install-distributions-from-context).                  +--use-constraints-for-context-distributionsUses constraints for context distributions installation - either from +constraints store in docker-context-files or from github.             +--disable-airflow-repo-cacheDisable cache from Airflow repository during building. +--disable-mysql-client-installationDo not install MySQL client. +--disable-mssql-client-installationDo not install MsSQl client. +--disable-postgres-client-installationDo not install Postgres client. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Preparing cache and push (for maintainers and CI) ──────────────────────────────────────────────────────────────────╮ +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT) +[default: autodetect]                                          +--platformPlatform for Airflow image.                                                          +(linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64 | linux/amd64,linux/arm64) +--pushPush image after building it. +--prepare-buildx-cachePrepares build cache (this is done as separate per-platform steps instead of building the  +image).                                                                                    +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Github authentication ──────────────────────────────────────────────────────────────────────────────────────────────╮ +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--github-tokenThe token used to authenticate to GitHub.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--answer-aForce answer to questions.(y | n | q | yes | no | quit) +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--verbose-vPrint verbose information about performed steps. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_prod-image_build.txt b/dev/breeze/doc/images/output_prod-image_build.txt index 58f9fe1ecf939..59283bee87590 100644 --- a/dev/breeze/doc/images/output_prod-image_build.txt +++ b/dev/breeze/doc/images/output_prod-image_build.txt @@ -1 +1 @@ -f0627afea3a75fad4886afaa9a517fdf +ddf60c0b290bbf8e75d5f41bfd33d7ca diff --git a/dev/breeze/doc/images/output_prod-image_load.svg b/dev/breeze/doc/images/output_prod-image_load.svg index d40cc1db81ba9..d91a8b66d7058 100644 --- a/dev/breeze/doc/images/output_prod-image_load.svg +++ b/dev/breeze/doc/images/output_prod-image_load.svg @@ -156,7 +156,7 @@ ╭─ Load image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ --python-pPython major/minor version used in Airflow image for images. -(>3.10< | 3.11 | 3.12)                                       +(>3.10< | 3.11 | 3.12 | 3.13)                                [default: 3.10]                                              --platformPlatform for Airflow image.                                (linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64) diff --git a/dev/breeze/doc/images/output_prod-image_load.txt b/dev/breeze/doc/images/output_prod-image_load.txt index e229179e97f35..0b860cab3aac8 100644 --- a/dev/breeze/doc/images/output_prod-image_load.txt +++ b/dev/breeze/doc/images/output_prod-image_load.txt @@ -1 +1 @@ -de986e6f731cef102f8f2f40dd2e8b01 +c300967f9093cc56e02032bfd1f610dc diff --git a/dev/breeze/doc/images/output_prod-image_pull.svg b/dev/breeze/doc/images/output_prod-image_pull.svg index ccddaeaf3a14e..4cb8a79e5ca85 100644 --- a/dev/breeze/doc/images/output_prod-image_pull.svg +++ b/dev/breeze/doc/images/output_prod-image_pull.svg @@ -153,7 +153,7 @@ Pull and optionally verify Production images - possibly in parallel for all Python versions. ╭─ Pull image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --verifyVerify image. --wait-for-imageWait until image is available. @@ -164,7 +164,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --skip-cleanupSkip cleanup of temporary files created during parallel run. --debug-resourcesWhether to show resource information while running in parallel. --include-success-outputsWhether to include outputs of successful runs (not shown by default). diff --git a/dev/breeze/doc/images/output_prod-image_pull.txt b/dev/breeze/doc/images/output_prod-image_pull.txt index 6e0a79486fc8e..289b75c8a8553 100644 --- a/dev/breeze/doc/images/output_prod-image_pull.txt +++ b/dev/breeze/doc/images/output_prod-image_pull.txt @@ -1 +1 @@ -a1c4d0b647670b696293bde6e97898bd +0637801b0d6f4cd81f5d808611315066 diff --git a/dev/breeze/doc/images/output_prod-image_save.svg b/dev/breeze/doc/images/output_prod-image_save.svg index 992d5ce22e9b0..11fc0cf730406 100644 --- a/dev/breeze/doc/images/output_prod-image_save.svg +++ b/dev/breeze/doc/images/output_prod-image_save.svg @@ -117,7 +117,7 @@ Save PROD image to a file. ╭─ Save image flags ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --platformPlatform for Airflow image.(linux/amd64 | linux/arm64 | linux/x86_64 | linux/aarch64) --github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] diff --git a/dev/breeze/doc/images/output_prod-image_save.txt b/dev/breeze/doc/images/output_prod-image_save.txt index b39bbfdb68c8a..b007d41ba7452 100644 --- a/dev/breeze/doc/images/output_prod-image_save.txt +++ b/dev/breeze/doc/images/output_prod-image_save.txt @@ -1 +1 @@ -b4b74446ac624e2114a03773996968fc +62e5d8920705b7b7497ee65566c67171 diff --git a/dev/breeze/doc/images/output_prod-image_verify.svg b/dev/breeze/doc/images/output_prod-image_verify.svg index 0c4c68ad8ec87..740805fd2b200 100644 --- a/dev/breeze/doc/images/output_prod-image_verify.svg +++ b/dev/breeze/doc/images/output_prod-image_verify.svg @@ -160,7 +160,7 @@ ╭─ Verify image flags ─────────────────────────────────────────────────────────────────────────────────────────────────╮ --image-name-nName of the image to verify (overrides --python).(TEXT) ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) +--python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12 | 3.13) [default: 3.10]                                              --slim-imageThe image to verify is slim and non-slim tests should be skipped. --pullPull image is missing before attempting to verify it. @@ -172,7 +172,7 @@ (INTEGER RANGE)                                                             [default: 4; 1<=x<=8]                                                       --python-versionsSpace separated list of python versions used for build with multiple versions.(TEXT) -[default: 3.10 3.11 3.12]                                                      +[default: 3.10 3.11 3.12 3.13]                                                 --skip-cleanupSkip cleanup of temporary files created during parallel run. --debug-resourcesWhether to show resource information while running in parallel. --include-success-outputsWhether to include outputs of successful runs (not shown by default). diff --git a/dev/breeze/doc/images/output_prod-image_verify.txt b/dev/breeze/doc/images/output_prod-image_verify.txt index 857a970c85745..0da020b62fed0 100644 --- a/dev/breeze/doc/images/output_prod-image_verify.txt +++ b/dev/breeze/doc/images/output_prod-image_verify.txt @@ -1 +1 @@ -8fcf39b4153549e82cddcf3746126801 +66800328fa7aa6c42705c56182c8c83c diff --git a/dev/breeze/doc/images/output_release-management_constraints-version-check.svg b/dev/breeze/doc/images/output_release-management_constraints-version-check.svg index 9f6af4fd413f3..0a965790d9393 100644 --- a/dev/breeze/doc/images/output_release-management_constraints-version-check.svg +++ b/dev/breeze/doc/images/output_release-management_constraints-version-check.svg @@ -1,4 +1,4 @@ - + Check constraints against released versions of packages. ╭─ Constraints options. ───────────────────────────────────────────────────────────────────────────────────────────────╮ ---python-pPython major/minor version used in Airflow image for images.(>3.10< | 3.11 | 3.12) -[default: 3.10]                                              ---airflow-constraints-modeMode of constraints for Airflow for CI image building.                  -(constraints-source-providers | constraints | constraints-no-providers) -[default: constraints-source-providers]                                 ---github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] ---github-tokenThe token used to authenticate to GitHub.(TEXT) -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Comparison mode. ───────────────────────────────────────────────────────────────────────────────────────────────────╮ ---diff-modeReport mode: full, diff-all, diff-constraints.(full|diff-all|diff-constraints) -[default: full]                                ---packageOnly check specific package(s). Can be used multiple times.(TEXT) ---explain-why/--no-explain-whyShow explanations for outdated packages. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Build options. ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ -╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ ---verbose-vPrint verbose information about performed steps. ---dry-run-DIf dry-run is set, commands are only printed, not executed. ---help-hShow this message and exit. -╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +--python-pPython major/minor version used in Airflow image for images. +(>3.10< | 3.11 | 3.12 | 3.13)                                +[default: 3.10]                                              +--airflow-constraints-modeMode of constraints for Airflow for CI image building.                  +(constraints-source-providers | constraints | constraints-no-providers) +[default: constraints-source-providers]                                 +--github-repository-gGitHub repository used to pull, push run images.(TEXT)[default: apache/airflow] +--github-tokenThe token used to authenticate to GitHub.(TEXT) +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Comparison mode. ───────────────────────────────────────────────────────────────────────────────────────────────────╮ +--diff-modeReport mode: full, diff-all, diff-constraints.(full|diff-all|diff-constraints) +[default: full]                                +--packageOnly check specific package(s). Can be used multiple times.(TEXT) +--explain-why/--no-explain-whyShow explanations for outdated packages. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Build options. ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--builderBuildx builder used to perform `docker buildx build` commands.(TEXT)[default: autodetect] +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ +╭─ Common options ─────────────────────────────────────────────────────────────────────────────────────────────────────╮ +--verbose-vPrint verbose information about performed steps. +--dry-run-DIf dry-run is set, commands are only printed, not executed. +--help-hShow this message and exit. +╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯ diff --git a/dev/breeze/doc/images/output_release-management_constraints-version-check.txt b/dev/breeze/doc/images/output_release-management_constraints-version-check.txt index 3723c26b6a66e..3824d26bd3157 100644 --- a/dev/breeze/doc/images/output_release-management_constraints-version-check.txt +++ b/dev/breeze/doc/images/output_release-management_constraints-version-check.txt @@ -1 +1 @@ -22ed78e3466a4dfd72a94f168d148e7d +4615897d941992b576a908d3939d8e7b diff --git a/dev/breeze/doc/images/output_release-management_generate-constraints.svg b/dev/breeze/doc/images/output_release-management_generate-constraints.svg index bc0ea443f1c77..274d8e957150d 100644 --- a/dev/breeze/doc/images/output_release-management_generate-constraints.svg +++ b/dev/breeze/doc/images/output_release-management_generate-constraints.svg @@ -1,4 +1,4 @@ - +