You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2024/02/12 23:22:59 UTC

(airflow) 23/23: Turn Pydantic into an optional dependency (#37320)

This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-8-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit eb00f003de3c66020cbddaa28ff2daccccd0b80a
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Feb 13 00:08:37 2024 +0100

    Turn Pydantic into an optional dependency (#37320)
    
    We've been internally using pydantic for internal API and it caused
    some compatibility issues, because Pydantic is so popular and currently
    still users of Pydantic are somewhat split between Pydantic 1 and
    Pydantic 2.  The popularity of Pydantic works against us, and since we
    are not yet using it in "production" (and in the future we will only
    actually use it for Internal API), it seems that turning Pydantic into
    an optional dependency is the best way we can proceed.
    
    It's as simple as converting all the direct imports into a common util
    imports that have a fallback mechanism when import is not found.
    
    This should enable less conflicts when installing 3rd-party libraries
    with Airflow.
    
    Added test where pydantic is removed. Also made sure that the special
    cases we have tests for run full suite of tests - non-db and db.
    
    (cherry picked from commit c3f48eeda0618a0c1bab8842d26155ee9b1f634d)
---
 .github/workflows/ci.yml                           |  97 +++++++++---
 Dockerfile.ci                                      |  34 +++++
 INSTALL                                            |   7 +-
 airflow/providers/apache/hdfs/sensors/hdfs.py      |   2 +
 airflow/providers/papermill/hooks/kernel.py        |   5 +-
 airflow/providers_manager.py                       |  23 +++
 airflow/serialization/pydantic/dag.py              |  10 +-
 airflow/serialization/pydantic/dag_run.py          |   6 +-
 airflow/serialization/pydantic/dataset.py          |   2 +-
 airflow/serialization/pydantic/job.py              |   3 +-
 airflow/serialization/pydantic/taskinstance.py     |  13 +-
 airflow/serialization/pydantic/tasklog.py          |   2 +-
 airflow/serialization/serialized_objects.py        |   3 +-
 airflow/utils/pydantic.py                          |  69 +++++++++
 .../12_airflow_dependencies_and_extras.rst         |   7 +-
 contributing-docs/testing/unit_tests.rst           |   2 -
 dev/breeze/doc/images/output_shell.svg             |  46 +++---
 dev/breeze/doc/images/output_shell.txt             |   2 +-
 dev/breeze/doc/images/output_testing_db-tests.svg  |  68 +++++----
 dev/breeze/doc/images/output_testing_db-tests.txt  |   2 +-
 .../doc/images/output_testing_non-db-tests.svg     |  68 +++++----
 .../doc/images/output_testing_non-db-tests.txt     |   2 +-
 dev/breeze/doc/images/output_testing_tests.svg     |  68 +++++----
 dev/breeze/doc/images/output_testing_tests.txt     |   2 +-
 .../src/airflow_breeze/commands/common_options.py  |   9 ++
 .../airflow_breeze/commands/developer_commands.py  |   4 +
 .../commands/developer_commands_config.py          |   3 +-
 .../airflow_breeze/commands/testing_commands.py    |  42 ++---
 .../commands/testing_commands_config.py            |  42 +++--
 dev/breeze/src/airflow_breeze/global_constants.py  |   2 +-
 .../src/airflow_breeze/params/shell_params.py      |   3 +
 docs/apache-airflow/extra-packages-ref.rst         |   2 +
 pyproject.toml                                     |  10 +-
 scripts/ci/docker-compose/devcontainer.env         |   5 +-
 scripts/docker/entrypoint_ci.sh                    |  35 +++++
 tests/always/test_example_dags.py                  |   5 +
 tests/api/common/test_airflow_health.py            |  86 ++++++-----
 .../endpoints/test_rpc_api_endpoint.py             |   2 +
 tests/api_internal/test_internal_api_call.py       |   2 +
 tests/conftest.py                                  |   1 +
 tests/core/test_configuration.py                   | 169 +++++++++++----------
 tests/core/test_settings.py                        |  12 +-
 tests/providers/openai/hooks/test_openai.py        |   3 +
 tests/providers/openai/operators/test_openai.py    |   2 +
 .../serialization/serializers/test_serializers.py  |  41 ++---
 tests/serialization/test_pydantic_models.py        |   2 +
 tests/serialization/test_serde.py                  |   8 +-
 tests/serialization/test_serialized_objects.py     |   1 +
 .../example_cloud_storage_transfer_service_aws.py  |   3 +-
 49 files changed, 698 insertions(+), 339 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 5dac5ce49f..9d3f5c41ac 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -1181,14 +1181,63 @@ jobs:
         uses: ./.github/actions/post_tests_failure
         if: failure()
 
-  tests-postgres-boto:
+  tests-min-sqlalchemy:
     timeout-minutes: 130
     name: >
-      DB:LatestBoto${{needs.build-info.outputs.default-postgres-version}},
+      DB:MinSQLAlchemy${{needs.build-info.outputs.default-postgres-version}},
       Py${{needs.build-info.outputs.default-python-version}}:
       ${{needs.build-info.outputs.parallel-test-types-list-as-string}}
     runs-on: ${{fromJSON(needs.build-info.outputs.runs-on)}}
     needs: [build-info, wait-for-ci-images]
+    env:
+      RUNS_ON: "${{needs.build-info.outputs.runs-on}}"
+      PARALLEL_TEST_TYPES: "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
+      PR_LABELS: "${{needs.build-info.outputs.pull-request-labels}}"
+      FULL_TESTS_NEEDED: "${{needs.build-info.outputs.full-tests-needed}}"
+      DEBUG_RESOURCES: "${{needs.build-info.outputs.debug-resources}}"
+      BACKEND: "postgres"
+      ENABLE_COVERAGE: "${{needs.build-info.outputs.run-coverage}}"
+      PYTHON_MAJOR_MINOR_VERSION: "${{needs.build-info.outputs.default-python-version}}"
+      PYTHON_VERSION: "${needs.build-info.outputs.default-python-version}}"
+      POSTGRES_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
+      BACKEND_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
+      DOWNGRADE_SQLALCHEMY: "true"
+      JOB_ID: >
+        min-sqlalchemy-${{needs.build-info.outputs.default-python-version}}-
+        ${{needs.build-info.outputs.default-postgres-version}}
+    if: needs.build-info.outputs.run-tests == 'true'
+    steps:
+      - name: Cleanup repo
+        shell: bash
+        run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*"
+      - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
+        uses: actions/checkout@v4
+        with:
+          persist-credentials: false
+      - name: >
+          Prepare breeze & CI image: ${{needs.build-info.outputs.default-python-version}}:${{env.IMAGE_TAG}}
+        uses: ./.github/actions/prepare_breeze_and_image
+      - name: >
+          Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.parallel-test-types-list-as-string}}
+        run: >
+          breeze testing db-tests
+          --parallel-test-types "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
+      - name: >
+          Post Tests success: ${{needs.build-info.outputs.default-python-version}}:MinSQLAlchemy"
+        uses: ./.github/actions/post_tests_success
+        if: success()
+      - name: >
+          Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:MinSQLAlchemy"
+        uses: ./.github/actions/post_tests_failure
+        if: failure()
+
+  tests-boto:
+    timeout-minutes: 130
+    name: >
+      LatestBoto-Py${{needs.build-info.outputs.default-python-version}}:
+      ${{needs.build-info.outputs.parallel-test-types-list-as-string}}
+    runs-on: ${{fromJSON(needs.build-info.outputs.runs-on)}}
+    needs: [build-info, wait-for-ci-images]
     env:
       RUNS_ON: "${{needs.build-info.outputs.runs-on}}"
       PARALLEL_TEST_TYPES: "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
@@ -1203,7 +1252,7 @@ jobs:
       BACKEND_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
       UPGRADE_BOTO: "true"
       JOB_ID: >
-        postgres-boto-${{needs.build-info.outputs.default-python-version}}-
+        boto-${{needs.build-info.outputs.default-python-version}}-
         ${{needs.build-info.outputs.default-postgres-version}}
     if: needs.build-info.outputs.run-tests == 'true' && needs.build-info.outputs.run-amazon-tests == 'true'
     steps:
@@ -1220,24 +1269,26 @@ jobs:
       - name: >
           Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.parallel-test-types-list-as-string}}
         run: >
-          breeze testing db-tests
+          breeze testing tests --run-in-parallel
           --parallel-test-types "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
       - name: >
-          Post Tests success: ${{needs.build-info.outputs.default-python-version}}:Boto"
+          Post Tests success: ${{needs.build-info.outputs.default-python-version}}:LatestBoto"
         uses: ./.github/actions/post_tests_success
         if: success()
       - name: >
-          Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:Boto"
+          Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:LatestBoto"
         uses: ./.github/actions/post_tests_failure
         if: failure()
 
-  tests-postgres-min-sqlalchemy:
+  tests-pydantic:
     timeout-minutes: 130
     name: >
-      DB:MinSQLAlchemy${{needs.build-info.outputs.default-postgres-version}},
-      Py${{needs.build-info.outputs.default-python-version}}:
+      Pydantic-${{ matrix.pydantic }}-Py${{needs.build-info.outputs.default-python-version}}:
       ${{needs.build-info.outputs.parallel-test-types-list-as-string}}
     runs-on: ${{fromJSON(needs.build-info.outputs.runs-on)}}
+    strategy:
+      matrix:
+        pydantic: ["v1", "none"]
     needs: [build-info, wait-for-ci-images]
     env:
       RUNS_ON: "${{needs.build-info.outputs.runs-on}}"
@@ -1251,9 +1302,9 @@ jobs:
       PYTHON_VERSION: "${needs.build-info.outputs.default-python-version}}"
       POSTGRES_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
       BACKEND_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
-      DOWNGRADE_SQLALCHEMY: "true"
+      PYDANTIC: ${{ matrix.pydantic }}
       JOB_ID: >
-        postgres-min-sqlalchemy-${{needs.build-info.outputs.default-python-version}}-
+        pydantic-${{ matrix.pydantic }}-${{needs.build-info.outputs.default-python-version}}-
         ${{needs.build-info.outputs.default-postgres-version}}
     if: needs.build-info.outputs.run-tests == 'true'
     steps:
@@ -1270,22 +1321,21 @@ jobs:
       - name: >
           Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.parallel-test-types-list-as-string}}
         run: >
-          breeze testing db-tests
+          breeze testing tests --run-in-parallel
           --parallel-test-types "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
       - name: >
-          Post Tests success: ${{needs.build-info.outputs.default-python-version}}:MinSQLAlchemy"
+          Post Tests success: ${{needs.build-info.outputs.default-python-version}}:NoPydantic"
         uses: ./.github/actions/post_tests_success
         if: success()
       - name: >
-          Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:MinSQLAlchemy"
+          Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:NoPydantic"
         uses: ./.github/actions/post_tests_failure
         if: failure()
 
-  tests-postgres-pendulum-2:
+  tests-pendulum-2:
     timeout-minutes: 130
     name: >
-      DB:Postgres${{needs.build-info.outputs.default-postgres-version}},
-      Pendulum2,Py${{needs.build-info.outputs.default-python-version}}:
+      Pendulum2-Py${{needs.build-info.outputs.default-python-version}}:
       ${{needs.build-info.outputs.parallel-test-types-list-as-string}}
     runs-on: ${{fromJSON(needs.build-info.outputs.runs-on)}}
     needs: [build-info, wait-for-ci-images]
@@ -1303,7 +1353,7 @@ jobs:
       BACKEND_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
       DOWNGRADE_PENDULUM: "true"
       JOB_ID: >
-        postgres-pendulum-2-${{needs.build-info.outputs.default-python-version}}-
+        pendulum-2-${{needs.build-info.outputs.default-python-version}}-
         ${{needs.build-info.outputs.default-postgres-version}}
     if: needs.build-info.outputs.run-tests == 'true'
     steps:
@@ -1320,7 +1370,7 @@ jobs:
       - name: >
           Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.parallel-test-types-list-as-string}}
         run: >
-          breeze testing db-tests
+          breeze testing tests --run-in-parallel
           --parallel-test-types "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
       - name: >
           Post Tests success: ${{needs.build-info.outputs.default-python-version}}:Pendulum2"
@@ -1331,11 +1381,10 @@ jobs:
         uses: ./.github/actions/post_tests_failure
         if: failure()
 
-  tests-postgres-in-progress-features-disabled:
+  tests-in-progress-features-disabled:
     timeout-minutes: 130
     name: >
-      DB:InProgressDisabledPostgres${{needs.build-info.outputs.default-postgres-version}},
-      Py${{needs.build-info.outputs.default-python-version}}:
+      InProgressDisabled-Py${{needs.build-info.outputs.default-python-version}}:
       ${{needs.build-info.outputs.parallel-test-types-list-as-string}}
     runs-on: ${{fromJSON(needs.build-info.outputs.runs-on)}}
     needs: [build-info, wait-for-ci-images]
@@ -1353,7 +1402,7 @@ jobs:
       BACKEND_VERSION: "${{needs.build-info.outputs.default-postgres-version}}"
       AIRFLOW_ENABLE_AIP_44: "false"
       JOB_ID: >
-        postgres-in-progress-disabled-${{needs.build-info.outputs.default-python-version}}-
+        in-progress-disabled-${{needs.build-info.outputs.default-python-version}}-
         ${{needs.build-info.outputs.default-postgres-version}}
     if: needs.build-info.outputs.run-tests == 'true' && needs.build-info.outputs.run-amazon-tests == 'true'
     steps:
@@ -1370,7 +1419,7 @@ jobs:
       - name: >
           Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.parallel-test-types-list-as-string}}
         run: >
-          breeze testing db-tests
+          breeze testing tests --run-in-parallel
           --parallel-test-types "${{needs.build-info.outputs.parallel-test-types-list-as-string}}"
       - name: >
           Post Tests success: ${{needs.build-info.outputs.default-python-version}}:FeaturesDisabled"
diff --git a/Dockerfile.ci b/Dockerfile.ci
index d818dc18e8..09c4fe45c5 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -899,6 +899,39 @@ function check_boto_upgrade() {
     pip check
 }
 
+function check_pydantic() {
+    if [[ ${PYDANTIC=} == "none" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Reinstalling airflow from local sources to account for pyproject.toml changes${COLOR_RESET}"
+        echo
+        pip install --root-user-action ignore -e .
+        echo
+        echo "${COLOR_YELLOW}Remove pydantic and 3rd party libraries that depend on it${COLOR_RESET}"
+        echo
+        pip uninstall --root-user-action ignore pydantic aws-sam-translator openai pyiceberg qdrant-client cfn-lint -y
+        pip check
+    elif [[ ${PYDANTIC=} == "v1" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Reinstalling airflow from local sources to account for pyproject.toml changes${COLOR_RESET}"
+        echo
+        pip install --root-user-action ignore -e .
+        echo
+        echo "${COLOR_YELLOW}Uninstalling pyicberg which is not compatible with Pydantic 1${COLOR_RESET}"
+        echo
+        pip uninstall pyiceberg -y
+        echo
+        echo "${COLOR_YELLOW}Downgrading Pydantic to < 2${COLOR_RESET}"
+        echo
+        pip install --upgrade "pydantic<2.0.0"
+        pip check
+    else
+        echo
+        echo "${COLOR_BLUE}Leaving default pydantic v2${COLOR_RESET}"
+        echo
+    fi
+}
+
+
 function check_download_sqlalchemy() {
     if [[ ${DOWNGRADE_SQLALCHEMY=} != "true" ]]; then
         return
@@ -951,6 +984,7 @@ function check_run_tests() {
 determine_airflow_to_use
 environment_initialization
 check_boto_upgrade
+check_pydantic
 check_download_sqlalchemy
 check_download_pendulum
 check_run_tests "${@}"
diff --git a/INSTALL b/INSTALL
index ddd40256dd..ee63ff12df 100644
--- a/INSTALL
+++ b/INSTALL
@@ -253,9 +253,10 @@ gcp_api, github, github-enterprise, google, google-auth, graphviz, grpc, hashico
 http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, leveldb, microsoft-azure,
 microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, odbc, openai,
 openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password,
-pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml,
-segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd,
-tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
+pgvector, pinecone, pinot, postgres, presto, pydantic, rabbitmq, redis, s3, s3fs, salesforce, samba,
+saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh,
+statsd, tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex,
+zendesk
 
 # END REGULAR EXTRAS HERE
 
diff --git a/airflow/providers/apache/hdfs/sensors/hdfs.py b/airflow/providers/apache/hdfs/sensors/hdfs.py
index 49b12d740b..9e2551c4e8 100644
--- a/airflow/providers/apache/hdfs/sensors/hdfs.py
+++ b/airflow/providers/apache/hdfs/sensors/hdfs.py
@@ -18,6 +18,8 @@ from __future__ import annotations
 
 from airflow.sensors.base import BaseSensorOperator
 
+# Ignore missing docstring
+
 _EXCEPTION_MESSAGE = """The old HDFS Sensors have been removed in 4.0.0 version of the apache.hdfs provider.
 Please convert your DAGs to use the WebHdfsSensor or downgrade the provider to below 4.*
 if you want to continue using it.
diff --git a/airflow/providers/papermill/hooks/kernel.py b/airflow/providers/papermill/hooks/kernel.py
index 0bac65dc98..9e6bb12a80 100644
--- a/airflow/providers/papermill/hooks/kernel.py
+++ b/airflow/providers/papermill/hooks/kernel.py
@@ -16,7 +16,7 @@
 # under the License.
 from __future__ import annotations
 
-from typing import TYPE_CHECKING
+import typing
 
 from jupyter_client import AsyncKernelManager
 from papermill.clientwrap import PapermillNotebookClient
@@ -24,9 +24,6 @@ from papermill.engines import NBClientEngine
 from papermill.utils import merge_kwargs, remove_args
 from traitlets import Unicode
 
-if TYPE_CHECKING:
-    from pydantic import typing
-
 from airflow.hooks.base import BaseHook
 
 JUPYTER_KERNEL_SHELL_PORT = 60316
diff --git a/airflow/providers_manager.py b/airflow/providers_manager.py
index 01bc9bc39a..a1c5ca59e5 100644
--- a/airflow/providers_manager.py
+++ b/airflow/providers_manager.py
@@ -1247,3 +1247,26 @@ class ProvidersManager(LoggingMixin, metaclass=Singleton):
     @property
     def already_initialized_provider_configs(self) -> list[tuple[str, dict[str, Any]]]:
         return sorted(self._provider_configs.items(), key=lambda x: x[0])
+
+    def _cleanup(self):
+        self._initialized_cache.clear()
+        self._provider_dict.clear()
+        self._hooks_dict.clear()
+        self._fs_set.clear()
+        self._taskflow_decorators.clear()
+        self._hook_provider_dict.clear()
+        self._hooks_lazy_dict.clear()
+        self._connection_form_widgets.clear()
+        self._field_behaviours.clear()
+        self._extra_link_class_name_set.clear()
+        self._logging_class_name_set.clear()
+        self._auth_manager_class_name_set.clear()
+        self._secrets_backend_class_name_set.clear()
+        self._executor_class_name_set.clear()
+        self._provider_configs.clear()
+        self._api_auth_backend_module_names.clear()
+        self._trigger_info_set.clear()
+        self._notification_info_set.clear()
+        self._plugins_set.clear()
+        self._initialized = False
+        self._initialization_stack_trace = None
diff --git a/airflow/serialization/pydantic/dag.py b/airflow/serialization/pydantic/dag.py
index 04b2472355..03a49f5663 100644
--- a/airflow/serialization/pydantic/dag.py
+++ b/airflow/serialization/pydantic/dag.py
@@ -21,17 +21,17 @@ from datetime import datetime, timedelta
 from typing import Any, List, Optional
 
 from dateutil import relativedelta
-from pydantic import (
+from typing_extensions import Annotated
+
+from airflow import DAG, settings
+from airflow.configuration import conf as airflow_conf
+from airflow.utils.pydantic import (
     BaseModel as BaseModelPydantic,
     ConfigDict,
     PlainSerializer,
     PlainValidator,
     ValidationInfo,
 )
-from typing_extensions import Annotated
-
-from airflow import DAG, settings
-from airflow.configuration import conf as airflow_conf
 from airflow.utils.sqlalchemy import Interval
 
 
diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py
index 8faabc5ee4..3de072cf90 100644
--- a/airflow/serialization/pydantic/dag_run.py
+++ b/airflow/serialization/pydantic/dag_run.py
@@ -19,10 +19,9 @@ from __future__ import annotations
 from datetime import datetime
 from typing import TYPE_CHECKING, Iterable, List, Optional
 
-from pydantic import BaseModel as BaseModelPydantic, ConfigDict
-
 from airflow.serialization.pydantic.dag import PydanticDag
 from airflow.serialization.pydantic.dataset import DatasetEventPydantic
+from airflow.utils.pydantic import BaseModel as BaseModelPydantic, ConfigDict, is_pydantic_2_installed
 from airflow.utils.session import NEW_SESSION, provide_session
 
 if TYPE_CHECKING:
@@ -101,4 +100,5 @@ class DagRunPydantic(BaseModelPydantic):
         )
 
 
-DagRunPydantic.model_rebuild()
+if is_pydantic_2_installed():
+    DagRunPydantic.model_rebuild()
diff --git a/airflow/serialization/pydantic/dataset.py b/airflow/serialization/pydantic/dataset.py
index 0c233a3fd6..44822c546d 100644
--- a/airflow/serialization/pydantic/dataset.py
+++ b/airflow/serialization/pydantic/dataset.py
@@ -17,7 +17,7 @@
 from datetime import datetime
 from typing import List, Optional
 
-from pydantic import BaseModel as BaseModelPydantic, ConfigDict
+from airflow.utils.pydantic import BaseModel as BaseModelPydantic, ConfigDict
 
 
 class DagScheduleDatasetReferencePydantic(BaseModelPydantic):
diff --git a/airflow/serialization/pydantic/job.py b/airflow/serialization/pydantic/job.py
index 2db30ab2c8..fd80528425 100644
--- a/airflow/serialization/pydantic/job.py
+++ b/airflow/serialization/pydantic/job.py
@@ -18,10 +18,9 @@ import datetime
 from functools import cached_property
 from typing import Optional
 
-from pydantic import BaseModel as BaseModelPydantic, ConfigDict
-
 from airflow.executors.executor_loader import ExecutorLoader
 from airflow.jobs.base_job_runner import BaseJobRunner
+from airflow.utils.pydantic import BaseModel as BaseModelPydantic, ConfigDict
 
 
 def check_runner_initialized(job_runner: Optional[BaseJobRunner], job_type: str) -> BaseJobRunner:
diff --git a/airflow/serialization/pydantic/taskinstance.py b/airflow/serialization/pydantic/taskinstance.py
index 106a31186e..72fad7612d 100644
--- a/airflow/serialization/pydantic/taskinstance.py
+++ b/airflow/serialization/pydantic/taskinstance.py
@@ -19,7 +19,6 @@ from __future__ import annotations
 from datetime import datetime
 from typing import TYPE_CHECKING, Any, Iterable, Optional
 
-from pydantic import BaseModel as BaseModelPydantic, ConfigDict, PlainSerializer, PlainValidator
 from typing_extensions import Annotated
 
 from airflow.models import Operator
@@ -29,16 +28,23 @@ from airflow.serialization.pydantic.dag import DagModelPydantic
 from airflow.serialization.pydantic.dag_run import DagRunPydantic
 from airflow.utils.log.logging_mixin import LoggingMixin
 from airflow.utils.net import get_hostname
+from airflow.utils.pydantic import (
+    BaseModel as BaseModelPydantic,
+    ConfigDict,
+    PlainSerializer,
+    PlainValidator,
+    is_pydantic_2_installed,
+)
 from airflow.utils.session import NEW_SESSION, provide_session
 from airflow.utils.xcom import XCOM_RETURN_KEY
 
 if TYPE_CHECKING:
     import pendulum
-    from pydantic_core.core_schema import ValidationInfo
     from sqlalchemy.orm import Session
 
     from airflow.models.dagrun import DagRun
     from airflow.utils.context import Context
+    from airflow.utils.pydantic import ValidationInfo
     from airflow.utils.state import DagRunState
 
 
@@ -430,4 +436,5 @@ class TaskInstancePydantic(BaseModelPydantic, LoggingMixin):
         )
 
 
-TaskInstancePydantic.model_rebuild()
+if is_pydantic_2_installed():
+    TaskInstancePydantic.model_rebuild()
diff --git a/airflow/serialization/pydantic/tasklog.py b/airflow/serialization/pydantic/tasklog.py
index a23204400c..3fbbf872b8 100644
--- a/airflow/serialization/pydantic/tasklog.py
+++ b/airflow/serialization/pydantic/tasklog.py
@@ -16,7 +16,7 @@
 # under the License.
 from datetime import datetime
 
-from pydantic import BaseModel as BaseModelPydantic, ConfigDict
+from airflow.utils.pydantic import BaseModel as BaseModelPydantic, ConfigDict
 
 
 class LogTemplatePydantic(BaseModelPydantic):
diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py
index 85822f3eec..7c8906d697 100644
--- a/airflow/serialization/serialized_objects.py
+++ b/airflow/serialization/serialized_objects.py
@@ -70,8 +70,6 @@ from airflow.utils.types import NOTSET, ArgNotSet
 if TYPE_CHECKING:
     from inspect import Parameter
 
-    from pydantic import BaseModel
-
     from airflow.models.baseoperatorlink import BaseOperatorLink
     from airflow.models.expandinput import ExpandInput
     from airflow.models.operator import Operator
@@ -79,6 +77,7 @@ if TYPE_CHECKING:
     from airflow.serialization.json_schema import Validator
     from airflow.ti_deps.deps.base_ti_dep import BaseTIDep
     from airflow.timetables.base import Timetable
+    from airflow.utils.pydantic import BaseModel
 
     HAS_KUBERNETES: bool
     try:
diff --git a/airflow/utils/pydantic.py b/airflow/utils/pydantic.py
new file mode 100644
index 0000000000..0ec184672c
--- /dev/null
+++ b/airflow/utils/pydantic.py
@@ -0,0 +1,69 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# This is an util module that makes Pydantic use optional. While we are using Pydantic in the airflow core
+# codebase, we don't want to make it a hard dependency for all the users of the core codebase, because
+# it is only used in the serialization and deserialization of the models for Internal API and for nothing
+# else, and since Pydantic is a very popular library, we don't want to force the users of the core codebase
+# to install specific Pydantic version - especially that a lot of libraries out there still depend on
+# Pydantic 1 and our internal API uses Pydantic 2+
+
+from __future__ import annotations
+
+
+def is_pydantic_2_installed() -> bool:
+    import sys
+
+    from packaging.version import Version
+
+    if sys.version_info >= (3, 9):
+        from importlib.metadata import distribution
+    else:
+        from importlib_metadata import distribution
+    try:
+        return Version(distribution("pydantic").version) >= Version("2.0.0")
+    except ImportError:
+        return False
+
+
+if is_pydantic_2_installed():
+    from pydantic import BaseModel, ConfigDict, PlainSerializer, PlainValidator, ValidationInfo
+else:
+
+    class BaseModel:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
+
+    class ConfigDict:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
+
+    class PlainSerializer:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
+
+    class PlainSerializer:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
+
+    class PlainValidator:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
+
+    class ValidationInfo:  # type: ignore[no-redef]  # noqa
+        def __init__(self, *args, **kwargs):
+            pass
diff --git a/contributing-docs/12_airflow_dependencies_and_extras.rst b/contributing-docs/12_airflow_dependencies_and_extras.rst
index 1b651154b2..bb67185881 100644
--- a/contributing-docs/12_airflow_dependencies_and_extras.rst
+++ b/contributing-docs/12_airflow_dependencies_and_extras.rst
@@ -209,9 +209,10 @@ gcp_api, github, github-enterprise, google, google-auth, graphviz, grpc, hashico
 http, imap, influxdb, jdbc, jenkins, kerberos, kubernetes, ldap, leveldb, microsoft-azure,
 microsoft-mssql, microsoft-psrp, microsoft-winrm, mongo, mssql, mysql, neo4j, odbc, openai,
 openfaas, openlineage, opensearch, opsgenie, oracle, otel, pagerduty, pandas, papermill, password,
-pgvector, pinecone, pinot, postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml,
-segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd,
-tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
+pgvector, pinecone, pinot, postgres, presto, pydantic, rabbitmq, redis, s3, s3fs, salesforce, samba,
+saml, segment, sendgrid, sentry, sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh,
+statsd, tableau, tabular, telegram, trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex,
+zendesk
 
   .. END REGULAR EXTRAS HERE
 
diff --git a/contributing-docs/testing/unit_tests.rst b/contributing-docs/testing/unit_tests.rst
index 4b84cf92c7..53bf3a1210 100644
--- a/contributing-docs/testing/unit_tests.rst
+++ b/contributing-docs/testing/unit_tests.rst
@@ -484,8 +484,6 @@ the test is marked as DB test:
 
   .. code-block:: python
 
-     pytestmark = pytest.mark.db_test
-
      TI = TaskInstance(
          task=BashOperator(task_id="test", bash_command="true", dag=DAG(dag_id="id"), start_date=datetime.now()),
          run_id="fake_run",
diff --git a/dev/breeze/doc/images/output_shell.svg b/dev/breeze/doc/images/output_shell.svg
index 3728b64243..3e267c31be 100644
--- a/dev/breeze/doc/images/output_shell.svg
+++ b/dev/breeze/doc/images/output_shell.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 3051.2" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 3075.6" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -43,7 +43,7 @@
 
     <defs>
     <clipPath id="breeze-shell-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="3000.2" />
+      <rect x="0" y="0" width="1463.0" height="3024.6" />
     </clipPath>
     <clipPath id="breeze-shell-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
@@ -411,9 +411,12 @@
 <clipPath id="breeze-shell-line-121">
     <rect x="0" y="2953.9" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="breeze-shell-line-122">
+    <rect x="0" y="2978.3" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="3049.2" rx="8"/><text class="breeze-shell-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;shell</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="3073.6" rx="8"/><text class="breeze-shell-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;shell</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
@@ -525,27 +528,28 @@
 </text><text class="breeze-shell-r5" x="0" y="2435.6" textLength="12.2" clip-path="url(#breeze-shell-line-99)">│</text><text class="breeze-shell-r4" x="24.4" y="2435.6" textLength="12.2" clip-path="url(#breeze-shell-line-99)">-</text><text class="breeze-shell-r4" x="36.6" y="2435.6" textLength="48.8" clip-path="url(#breeze-shell-line-99)">-use</text><text class="breeze-shell-r4" x="85.4" y="2435.6" textLength="231.8" clip-path="url(#breeze-shell-line-99)">-packages-from-dist</text><text  [...]
 </text><text class="breeze-shell-r5" x="0" y="2460" textLength="12.2" clip-path="url(#breeze-shell-line-100)">│</text><text class="breeze-shell-r1" x="475.8" y="2460" textLength="963.8" clip-path="url(#breeze-shell-line-100)">folder&#160;when&#160;entering&#160;breeze.&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
 </text><text class="breeze-shell-r5" x="0" y="2484.4" textLength="1464" clip-path="url(#breeze-shell-line-101)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2484.4" textLength="12.2" clip-path="url(#breeze-shell-line-101)">
-</text><text class="breeze-shell-r5" x="0" y="2508.8" textLength="24.4" clip-path="url(#breeze-shell-line-102)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2508.8" textLength="500.2" clip-path="url(#breeze-shell-line-102)">&#160;Upgrading/downgrading&#160;selected&#160;packages&#160;</text><text class="breeze-shell-r5" x="524.6" y="2508.8" textLength="915" clip-path="url(#breeze-shell-line-102)">───────────────────────────────────────────────────────────────────────────</text><tex [...]
+</text><text class="breeze-shell-r5" x="0" y="2508.8" textLength="24.4" clip-path="url(#breeze-shell-line-102)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2508.8" textLength="610" clip-path="url(#breeze-shell-line-102)">&#160;Upgrading/downgrading/removing&#160;selected&#160;packages&#160;</text><text class="breeze-shell-r5" x="634.4" y="2508.8" textLength="805.2" clip-path="url(#breeze-shell-line-102)">──────────────────────────────────────────────────────────────────</text><tex [...]
 </text><text class="breeze-shell-r5" x="0" y="2533.2" textLength="12.2" clip-path="url(#breeze-shell-line-103)">│</text><text class="breeze-shell-r4" x="24.4" y="2533.2" textLength="12.2" clip-path="url(#breeze-shell-line-103)">-</text><text class="breeze-shell-r4" x="36.6" y="2533.2" textLength="97.6" clip-path="url(#breeze-shell-line-103)">-upgrade</text><text class="breeze-shell-r4" x="134.2" y="2533.2" textLength="61" clip-path="url(#breeze-shell-line-103)">-boto</text><text class="b [...]
 </text><text class="breeze-shell-r5" x="0" y="2557.6" textLength="12.2" clip-path="url(#breeze-shell-line-104)">│</text><text class="breeze-shell-r4" x="24.4" y="2557.6" textLength="12.2" clip-path="url(#breeze-shell-line-104)">-</text><text class="breeze-shell-r4" x="36.6" y="2557.6" textLength="122" clip-path="url(#breeze-shell-line-104)">-downgrade</text><text class="breeze-shell-r4" x="158.6" y="2557.6" textLength="134.2" clip-path="url(#breeze-shell-line-104)">-sqlalchemy</text><tex [...]
 </text><text class="breeze-shell-r5" x="0" y="2582" textLength="12.2" clip-path="url(#breeze-shell-line-105)">│</text><text class="breeze-shell-r4" x="24.4" y="2582" textLength="12.2" clip-path="url(#breeze-shell-line-105)">-</text><text class="breeze-shell-r4" x="36.6" y="2582" textLength="122" clip-path="url(#breeze-shell-line-105)">-downgrade</text><text class="breeze-shell-r4" x="158.6" y="2582" textLength="109.8" clip-path="url(#breeze-shell-line-105)">-pendulum</text><text class="b [...]
-</text><text class="breeze-shell-r5" x="0" y="2606.4" textLength="1464" clip-path="url(#breeze-shell-line-106)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2606.4" textLength="12.2" clip-path="url(#breeze-shell-line-106)">
-</text><text class="breeze-shell-r5" x="0" y="2630.8" textLength="24.4" clip-path="url(#breeze-shell-line-107)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2630.8" textLength="183" clip-path="url(#breeze-shell-line-107)">&#160;DB&#160;test&#160;flags&#160;</text><text class="breeze-shell-r5" x="207.4" y="2630.8" textLength="1232.2" clip-path="url(#breeze-shell-line-107)">─────────────────────────────────────────────────────────────────────────────────────────────────────</text><te [...]
-</text><text class="breeze-shell-r5" x="0" y="2655.2" textLength="12.2" clip-path="url(#breeze-shell-line-108)">│</text><text class="breeze-shell-r4" x="24.4" y="2655.2" textLength="12.2" clip-path="url(#breeze-shell-line-108)">-</text><text class="breeze-shell-r4" x="36.6" y="2655.2" textLength="48.8" clip-path="url(#breeze-shell-line-108)">-run</text><text class="breeze-shell-r4" x="85.4" y="2655.2" textLength="170.8" clip-path="url(#breeze-shell-line-108)">-db-tests-only</text><text c [...]
-</text><text class="breeze-shell-r5" x="0" y="2679.6" textLength="12.2" clip-path="url(#breeze-shell-line-109)">│</text><text class="breeze-shell-r4" x="24.4" y="2679.6" textLength="12.2" clip-path="url(#breeze-shell-line-109)">-</text><text class="breeze-shell-r4" x="36.6" y="2679.6" textLength="61" clip-path="url(#breeze-shell-line-109)">-skip</text><text class="breeze-shell-r4" x="97.6" y="2679.6" textLength="109.8" clip-path="url(#breeze-shell-line-109)">-db-tests</text><text class=" [...]
-</text><text class="breeze-shell-r5" x="0" y="2704" textLength="1464" clip-path="url(#breeze-shell-line-110)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2704" textLength="12.2" clip-path="url(#breeze-shell-line-110)">
-</text><text class="breeze-shell-r5" x="0" y="2728.4" textLength="24.4" clip-path="url(#breeze-shell-line-111)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2728.4" textLength="183" clip-path="url(#breeze-shell-line-111)">&#160;Other&#160;options&#160;</text><text class="breeze-shell-r5" x="207.4" y="2728.4" textLength="1232.2" clip-path="url(#breeze-shell-line-111)">─────────────────────────────────────────────────────────────────────────────────────────────────────</text><text cl [...]
-</text><text class="breeze-shell-r5" x="0" y="2752.8" textLength="12.2" clip-path="url(#breeze-shell-line-112)">│</text><text class="breeze-shell-r4" x="24.4" y="2752.8" textLength="12.2" clip-path="url(#breeze-shell-line-112)">-</text><text class="breeze-shell-r4" x="36.6" y="2752.8" textLength="97.6" clip-path="url(#breeze-shell-line-112)">-forward</text><text class="breeze-shell-r4" x="134.2" y="2752.8" textLength="146.4" clip-path="url(#breeze-shell-line-112)">-credentials</text><tex [...]
-</text><text class="breeze-shell-r5" x="0" y="2777.2" textLength="12.2" clip-path="url(#breeze-shell-line-113)">│</text><text class="breeze-shell-r4" x="24.4" y="2777.2" textLength="12.2" clip-path="url(#breeze-shell-line-113)">-</text><text class="breeze-shell-r4" x="36.6" y="2777.2" textLength="48.8" clip-path="url(#breeze-shell-line-113)">-max</text><text class="breeze-shell-r4" x="85.4" y="2777.2" textLength="61" clip-path="url(#breeze-shell-line-113)">-time</text><text class="breeze [...]
-</text><text class="breeze-shell-r5" x="0" y="2801.6" textLength="12.2" clip-path="url(#breeze-shell-line-114)">│</text><text class="breeze-shell-r7" x="353.8" y="2801.6" textLength="1049.2" clip-path="url(#breeze-shell-line-114)">(INTEGER&#160;RANGE)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="breeze-shell-r5" x="0" y="2826" textLength="12.2" clip-path="url(#breeze-shell-line-115)">│</text><text class="breeze-shell-r4" x="24.4" y="2826" textLength="12.2" clip-path="url(#breeze-shell-line-115)">-</text><text class="breeze-shell-r4" x="36.6" y="2826" textLength="97.6" clip-path="url(#breeze-shell-line-115)">-verbose</text><text class="breeze-shell-r4" x="134.2" y="2826" textLength="109.8" clip-path="url(#breeze-shell-line-115)">-commands</text><text class="br [...]
-</text><text class="breeze-shell-r5" x="0" y="2850.4" textLength="1464" clip-path="url(#breeze-shell-line-116)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2850.4" textLength="12.2" clip-path="url(#breeze-shell-line-116)">
-</text><text class="breeze-shell-r5" x="0" y="2874.8" textLength="24.4" clip-path="url(#breeze-shell-line-117)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2874.8" textLength="195.2" clip-path="url(#breeze-shell-line-117)">&#160;Common&#160;options&#160;</text><text class="breeze-shell-r5" x="219.6" y="2874.8" textLength="1220" clip-path="url(#breeze-shell-line-117)">────────────────────────────────────────────────────────────────────────────────────────────────────</text><text cl [...]
-</text><text class="breeze-shell-r5" x="0" y="2899.2" textLength="12.2" clip-path="url(#breeze-shell-line-118)">│</text><text class="breeze-shell-r4" x="24.4" y="2899.2" textLength="12.2" clip-path="url(#breeze-shell-line-118)">-</text><text class="breeze-shell-r4" x="36.6" y="2899.2" textLength="85.4" clip-path="url(#breeze-shell-line-118)">-answer</text><text class="breeze-shell-r6" x="158.6" y="2899.2" textLength="24.4" clip-path="url(#breeze-shell-line-118)">-a</text><text class="bre [...]
-</text><text class="breeze-shell-r5" x="0" y="2923.6" textLength="12.2" clip-path="url(#breeze-shell-line-119)">│</text><text class="breeze-shell-r4" x="24.4" y="2923.6" textLength="12.2" clip-path="url(#breeze-shell-line-119)">-</text><text class="breeze-shell-r4" x="36.6" y="2923.6" textLength="48.8" clip-path="url(#breeze-shell-line-119)">-dry</text><text class="breeze-shell-r4" x="85.4" y="2923.6" textLength="48.8" clip-path="url(#breeze-shell-line-119)">-run</text><text class="breez [...]
-</text><text class="breeze-shell-r5" x="0" y="2948" textLength="12.2" clip-path="url(#breeze-shell-line-120)">│</text><text class="breeze-shell-r4" x="24.4" y="2948" textLength="12.2" clip-path="url(#breeze-shell-line-120)">-</text><text class="breeze-shell-r4" x="36.6" y="2948" textLength="97.6" clip-path="url(#breeze-shell-line-120)">-verbose</text><text class="breeze-shell-r6" x="158.6" y="2948" textLength="24.4" clip-path="url(#breeze-shell-line-120)">-v</text><text class="breeze-she [...]
-</text><text class="breeze-shell-r5" x="0" y="2972.4" textLength="12.2" clip-path="url(#breeze-shell-line-121)">│</text><text class="breeze-shell-r4" x="24.4" y="2972.4" textLength="12.2" clip-path="url(#breeze-shell-line-121)">-</text><text class="breeze-shell-r4" x="36.6" y="2972.4" textLength="61" clip-path="url(#breeze-shell-line-121)">-help</text><text class="breeze-shell-r6" x="158.6" y="2972.4" textLength="24.4" clip-path="url(#breeze-shell-line-121)">-h</text><text class="breeze- [...]
-</text><text class="breeze-shell-r5" x="0" y="2996.8" textLength="1464" clip-path="url(#breeze-shell-line-122)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2996.8" textLength="12.2" clip-path="url(#breeze-shell-line-122)">
+</text><text class="breeze-shell-r5" x="0" y="2606.4" textLength="12.2" clip-path="url(#breeze-shell-line-106)">│</text><text class="breeze-shell-r4" x="24.4" y="2606.4" textLength="12.2" clip-path="url(#breeze-shell-line-106)">-</text><text class="breeze-shell-r4" x="36.6" y="2606.4" textLength="109.8" clip-path="url(#breeze-shell-line-106)">-pydantic</text><text class="breeze-shell-r1" x="341.6" y="2606.4" textLength="658.8" clip-path="url(#breeze-shell-line-106)">Determines&#160;which [...]
+</text><text class="breeze-shell-r5" x="0" y="2630.8" textLength="1464" clip-path="url(#breeze-shell-line-107)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2630.8" textLength="12.2" clip-path="url(#breeze-shell-line-107)">
+</text><text class="breeze-shell-r5" x="0" y="2655.2" textLength="24.4" clip-path="url(#breeze-shell-line-108)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2655.2" textLength="183" clip-path="url(#breeze-shell-line-108)">&#160;DB&#160;test&#160;flags&#160;</text><text class="breeze-shell-r5" x="207.4" y="2655.2" textLength="1232.2" clip-path="url(#breeze-shell-line-108)">─────────────────────────────────────────────────────────────────────────────────────────────────────</text><te [...]
+</text><text class="breeze-shell-r5" x="0" y="2679.6" textLength="12.2" clip-path="url(#breeze-shell-line-109)">│</text><text class="breeze-shell-r4" x="24.4" y="2679.6" textLength="12.2" clip-path="url(#breeze-shell-line-109)">-</text><text class="breeze-shell-r4" x="36.6" y="2679.6" textLength="48.8" clip-path="url(#breeze-shell-line-109)">-run</text><text class="breeze-shell-r4" x="85.4" y="2679.6" textLength="170.8" clip-path="url(#breeze-shell-line-109)">-db-tests-only</text><text c [...]
+</text><text class="breeze-shell-r5" x="0" y="2704" textLength="12.2" clip-path="url(#breeze-shell-line-110)">│</text><text class="breeze-shell-r4" x="24.4" y="2704" textLength="12.2" clip-path="url(#breeze-shell-line-110)">-</text><text class="breeze-shell-r4" x="36.6" y="2704" textLength="61" clip-path="url(#breeze-shell-line-110)">-skip</text><text class="breeze-shell-r4" x="97.6" y="2704" textLength="109.8" clip-path="url(#breeze-shell-line-110)">-db-tests</text><text class="breeze-s [...]
+</text><text class="breeze-shell-r5" x="0" y="2728.4" textLength="1464" clip-path="url(#breeze-shell-line-111)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2728.4" textLength="12.2" clip-path="url(#breeze-shell-line-111)">
+</text><text class="breeze-shell-r5" x="0" y="2752.8" textLength="24.4" clip-path="url(#breeze-shell-line-112)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2752.8" textLength="183" clip-path="url(#breeze-shell-line-112)">&#160;Other&#160;options&#160;</text><text class="breeze-shell-r5" x="207.4" y="2752.8" textLength="1232.2" clip-path="url(#breeze-shell-line-112)">─────────────────────────────────────────────────────────────────────────────────────────────────────</text><text cl [...]
+</text><text class="breeze-shell-r5" x="0" y="2777.2" textLength="12.2" clip-path="url(#breeze-shell-line-113)">│</text><text class="breeze-shell-r4" x="24.4" y="2777.2" textLength="12.2" clip-path="url(#breeze-shell-line-113)">-</text><text class="breeze-shell-r4" x="36.6" y="2777.2" textLength="97.6" clip-path="url(#breeze-shell-line-113)">-forward</text><text class="breeze-shell-r4" x="134.2" y="2777.2" textLength="146.4" clip-path="url(#breeze-shell-line-113)">-credentials</text><tex [...]
+</text><text class="breeze-shell-r5" x="0" y="2801.6" textLength="12.2" clip-path="url(#breeze-shell-line-114)">│</text><text class="breeze-shell-r4" x="24.4" y="2801.6" textLength="12.2" clip-path="url(#breeze-shell-line-114)">-</text><text class="breeze-shell-r4" x="36.6" y="2801.6" textLength="48.8" clip-path="url(#breeze-shell-line-114)">-max</text><text class="breeze-shell-r4" x="85.4" y="2801.6" textLength="61" clip-path="url(#breeze-shell-line-114)">-time</text><text class="breeze [...]
+</text><text class="breeze-shell-r5" x="0" y="2826" textLength="12.2" clip-path="url(#breeze-shell-line-115)">│</text><text class="breeze-shell-r7" x="353.8" y="2826" textLength="1049.2" clip-path="url(#breeze-shell-line-115)">(INTEGER&#160;RANGE)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-shell-r5" x="0" y="2850.4" textLength="12.2" clip-path="url(#breeze-shell-line-116)">│</text><text class="breeze-shell-r4" x="24.4" y="2850.4" textLength="12.2" clip-path="url(#breeze-shell-line-116)">-</text><text class="breeze-shell-r4" x="36.6" y="2850.4" textLength="97.6" clip-path="url(#breeze-shell-line-116)">-verbose</text><text class="breeze-shell-r4" x="134.2" y="2850.4" textLength="109.8" clip-path="url(#breeze-shell-line-116)">-commands</text><text c [...]
+</text><text class="breeze-shell-r5" x="0" y="2874.8" textLength="1464" clip-path="url(#breeze-shell-line-117)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="2874.8" textLength="12.2" clip-path="url(#breeze-shell-line-117)">
+</text><text class="breeze-shell-r5" x="0" y="2899.2" textLength="24.4" clip-path="url(#breeze-shell-line-118)">╭─</text><text class="breeze-shell-r5" x="24.4" y="2899.2" textLength="195.2" clip-path="url(#breeze-shell-line-118)">&#160;Common&#160;options&#160;</text><text class="breeze-shell-r5" x="219.6" y="2899.2" textLength="1220" clip-path="url(#breeze-shell-line-118)">────────────────────────────────────────────────────────────────────────────────────────────────────</text><text cl [...]
+</text><text class="breeze-shell-r5" x="0" y="2923.6" textLength="12.2" clip-path="url(#breeze-shell-line-119)">│</text><text class="breeze-shell-r4" x="24.4" y="2923.6" textLength="12.2" clip-path="url(#breeze-shell-line-119)">-</text><text class="breeze-shell-r4" x="36.6" y="2923.6" textLength="85.4" clip-path="url(#breeze-shell-line-119)">-answer</text><text class="breeze-shell-r6" x="158.6" y="2923.6" textLength="24.4" clip-path="url(#breeze-shell-line-119)">-a</text><text class="bre [...]
+</text><text class="breeze-shell-r5" x="0" y="2948" textLength="12.2" clip-path="url(#breeze-shell-line-120)">│</text><text class="breeze-shell-r4" x="24.4" y="2948" textLength="12.2" clip-path="url(#breeze-shell-line-120)">-</text><text class="breeze-shell-r4" x="36.6" y="2948" textLength="48.8" clip-path="url(#breeze-shell-line-120)">-dry</text><text class="breeze-shell-r4" x="85.4" y="2948" textLength="48.8" clip-path="url(#breeze-shell-line-120)">-run</text><text class="breeze-shell- [...]
+</text><text class="breeze-shell-r5" x="0" y="2972.4" textLength="12.2" clip-path="url(#breeze-shell-line-121)">│</text><text class="breeze-shell-r4" x="24.4" y="2972.4" textLength="12.2" clip-path="url(#breeze-shell-line-121)">-</text><text class="breeze-shell-r4" x="36.6" y="2972.4" textLength="97.6" clip-path="url(#breeze-shell-line-121)">-verbose</text><text class="breeze-shell-r6" x="158.6" y="2972.4" textLength="24.4" clip-path="url(#breeze-shell-line-121)">-v</text><text class="br [...]
+</text><text class="breeze-shell-r5" x="0" y="2996.8" textLength="12.2" clip-path="url(#breeze-shell-line-122)">│</text><text class="breeze-shell-r4" x="24.4" y="2996.8" textLength="12.2" clip-path="url(#breeze-shell-line-122)">-</text><text class="breeze-shell-r4" x="36.6" y="2996.8" textLength="61" clip-path="url(#breeze-shell-line-122)">-help</text><text class="breeze-shell-r6" x="158.6" y="2996.8" textLength="24.4" clip-path="url(#breeze-shell-line-122)">-h</text><text class="breeze- [...]
+</text><text class="breeze-shell-r5" x="0" y="3021.2" textLength="1464" clip-path="url(#breeze-shell-line-123)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-shell-r1" x="1464" y="3021.2" textLength="12.2" clip-path="url(#breeze-shell-line-123)">
 </text>
     </g>
     </g>
diff --git a/dev/breeze/doc/images/output_shell.txt b/dev/breeze/doc/images/output_shell.txt
index 1dedc9a19b..6d20efe12d 100644
--- a/dev/breeze/doc/images/output_shell.txt
+++ b/dev/breeze/doc/images/output_shell.txt
@@ -1 +1 @@
-050121fd818915f32906829b43a8a28b
+bd15318a0ff034461180cd902116b6f6
diff --git a/dev/breeze/doc/images/output_testing_db-tests.svg b/dev/breeze/doc/images/output_testing_db-tests.svg
index ca7569d576..02c79ab846 100644
--- a/dev/breeze/doc/images/output_testing_db-tests.svg
+++ b/dev/breeze/doc/images/output_testing_db-tests.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 1855.6" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 1928.8" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -43,7 +43,7 @@
 
     <defs>
     <clipPath id="breeze-testing-db-tests-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="1804.6" />
+      <rect x="0" y="0" width="1463.0" height="1877.8" />
     </clipPath>
     <clipPath id="breeze-testing-db-tests-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
@@ -264,9 +264,18 @@
 <clipPath id="breeze-testing-db-tests-line-72">
     <rect x="0" y="1758.3" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="breeze-testing-db-tests-line-73">
+    <rect x="0" y="1782.7" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-db-tests-line-74">
+    <rect x="0" y="1807.1" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-db-tests-line-75">
+    <rect x="0" y="1831.5" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1853.6" rx="8"/><text class="breeze-testing-db-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;db-tests</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1926.8" rx="8"/><text class="breeze-testing-db-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;db-tests</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
@@ -325,31 +334,34 @@
 </text><text class="breeze-testing-db-tests-r5" x="0" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-46)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-46)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1142.4" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-46)">-debug</text><text class="breeze-testing-db-tests-r4" x="109.8" y="1142.4" textLength [...]
 </text><text class="breeze-testing-db-tests-r5" x="0" y="1166.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-47)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-47)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1166.8" textLength="97.6" clip-path="url(#breeze-testing-db-tests-line-47)">-include</text><text class="breeze-testing-db-tests-r4" x="134.2" y="1166.8" textLeng [...]
 </text><text class="breeze-testing-db-tests-r5" x="0" y="1191.2" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-48)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1191.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-48)">
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1215.6" textLength="24.4" clip-path="url(#breeze-testing-db-tests-line-49)">╭─</text><text class="breeze-testing-db-tests-r5" x="24.4" y="1215.6" textLength="402.6" clip-path="url(#breeze-testing-db-tests-line-49)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-db-tests-r5" x="427" y="1215.6" textLength="1012.6" clip-path="url(#breeze-testing-db-tests-line-49)">──────────────────── [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1240" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-50)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1240" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-50)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1240" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-50)">-image</text><text class="breeze-testing-db-tests-r4" x="109.8" y="1240" textLength="48.8"  [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-51)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1264.4" textLength="963.8" clip-path="url(#breeze-testing-db-tests-line-51)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-52)">│</text><text class="breeze-testing-db-tests-r5" x="414.8" y="1288.8" textLength="963.8" clip-path="url(#breeze-testing-db-tests-line-52)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-53)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-53)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1313.2" textLength="85.4" clip-path="url(#breeze-testing-db-tests-line-53)">-github</text><text class="breeze-testing-db-tests-r4" x="122" y="1313.2" textLength= [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-54)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-54)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1337.6" textLength="48.8" clip-path="url(#breeze-testing-db-tests-line-54)">-use</text><text class="breeze-testing-db-tests-r4" x="85.4" y="1337.6" textLength="1 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1362" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-55)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1362" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-55)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-db-tests-r5" x="1451.8" y="13 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1386.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-56)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1386.4" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-56)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-db-tests-r5" x="1451.8" y="1386.4" textLength="12.2" clip-path="url(#b [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-57)">│</text><text class="breeze-testing-db-tests-r4" x="414.8" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-57)">-</text><text class="breeze-testing-db-tests-r4" x="427" y="1410.8" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-57)">-mount</text><text class="breeze-testing-db-tests-r4" x="500.2" y="1410.8" textLength [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-58)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1435.2" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-58)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-59)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-59)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1459.6" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-59)">-mount</text><text class="breeze-testing-db-tests-r4" x="109.8" y="1459.6" textLength [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1484" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-60)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1484" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-60)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1508.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-61)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1508.4" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-61)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-62)">│</text><text class="breeze-testing-db-tests-r5" x="414.8" y="1532.8" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-62)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-63)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-63)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1557.2" textLength="97.6" clip-path="url(#breeze-testing-db-tests-line-63)">-upgrade</text><text class="breeze-testing-db-tests-r4" x="134.2" y="1557.2" textLeng [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-64)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-64)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1581.6" textLength="122" clip-path="url(#breeze-testing-db-tests-line-64)">-downgrade</text><text class="breeze-testing-db-tests-r4" x="158.6" y="1581.6" textLen [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1606" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-65)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1606" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-65)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1606" textLength="122" clip-path="url(#breeze-testing-db-tests-line-65)">-downgrade</text><text class="breeze-testing-db-tests-r4" x="158.6" y="1606" textLength="109 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-66)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-66)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1630.4" textLength="85.4" clip-path="url(#breeze-testing-db-tests-line-66)">-remove</text><text class="breeze-testing-db-tests-r4" x="122" y="1630.4" textLength= [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-67)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-67)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1654.8" textLength="61" clip-path="url(#breeze-testing-db-tests-line-67)">-skip</text><text class="breeze-testing-db-tests-r4" x="97.6" y="1654.8" textLength="24 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1679.2" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-68)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-68)">
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1703.6" textLength="24.4" clip-path="url(#breeze-testing-db-tests-line-69)">╭─</text><text class="breeze-testing-db-tests-r5" x="24.4" y="1703.6" textLength="195.2" clip-path="url(#breeze-testing-db-tests-line-69)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-db-tests-r5" x="219.6" y="1703.6" textLength="1220" clip-path="url(#breeze-testing-db-tests-line-69)">──────────────────────────────────────────────────── [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1728" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-70)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1728" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-70)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1728" textLength="97.6" clip-path="url(#breeze-testing-db-tests-line-70)">-verbose</text><text class="breeze-testing-db-tests-r7" x="158.6" y="1728" textLength="24.4 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-71)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-71)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1752.4" textLength="48.8" clip-path="url(#breeze-testing-db-tests-line-71)">-dry</text><text class="breeze-testing-db-tests-r4" x="85.4" y="1752.4" textLength="4 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1776.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-72)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1776.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-72)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1776.8" textLength="61" clip-path="url(#breeze-testing-db-tests-line-72)">-help</text><text class="breeze-testing-db-tests-r7" x="158.6" y="1776.8" textLength="2 [...]
-</text><text class="breeze-testing-db-tests-r5" x="0" y="1801.2" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-73)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1801.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-73)">
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1215.6" textLength="24.4" clip-path="url(#breeze-testing-db-tests-line-49)">╭─</text><text class="breeze-testing-db-tests-r5" x="24.4" y="1215.6" textLength="610" clip-path="url(#breeze-testing-db-tests-line-49)">&#160;Upgrading/downgrading/removing&#160;selected&#160;packages&#160;</text><text class="breeze-testing-db-tests-r5" x="634.4" y="1215.6" textLength="805.2" clip-path="url(#breeze-testing-db-tests-line-49)">────────────── [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1240" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-50)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1240" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-50)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1240" textLength="97.6" clip-path="url(#breeze-testing-db-tests-line-50)">-upgrade</text><text class="breeze-testing-db-tests-r4" x="134.2" y="1240" textLength="61"  [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-51)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-51)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1264.4" textLength="122" clip-path="url(#breeze-testing-db-tests-line-51)">-downgrade</text><text class="breeze-testing-db-tests-r4" x="158.6" y="1264.4" textLen [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-52)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-52)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1288.8" textLength="122" clip-path="url(#breeze-testing-db-tests-line-52)">-downgrade</text><text class="breeze-testing-db-tests-r4" x="158.6" y="1288.8" textLen [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-53)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-53)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1313.2" textLength="109.8" clip-path="url(#breeze-testing-db-tests-line-53)">-pydantic</text><text class="breeze-testing-db-tests-r1" x="341.6" y="1313.2" textLe [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-54)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-54)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1337.6" textLength="85.4" clip-path="url(#breeze-testing-db-tests-line-54)">-remove</text><text class="breeze-testing-db-tests-r4" x="122" y="1337.6" textLength= [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1362" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-55)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1362" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-55)">
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1386.4" textLength="24.4" clip-path="url(#breeze-testing-db-tests-line-56)">╭─</text><text class="breeze-testing-db-tests-r5" x="24.4" y="1386.4" textLength="402.6" clip-path="url(#breeze-testing-db-tests-line-56)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-db-tests-r5" x="427" y="1386.4" textLength="1012.6" clip-path="url(#breeze-testing-db-tests-line-56)">──────────────────── [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-57)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-57)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1410.8" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-57)">-image</text><text class="breeze-testing-db-tests-r4" x="109.8" y="1410.8" textLength [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-58)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1435.2" textLength="963.8" clip-path="url(#breeze-testing-db-tests-line-58)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-59)">│</text><text class="breeze-testing-db-tests-r5" x="414.8" y="1459.6" textLength="963.8" clip-path="url(#breeze-testing-db-tests-line-59)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1484" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-60)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1484" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-60)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1484" textLength="85.4" clip-path="url(#breeze-testing-db-tests-line-60)">-github</text><text class="breeze-testing-db-tests-r4" x="122" y="1484" textLength="134.2"  [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1508.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-61)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1508.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-61)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1508.4" textLength="48.8" clip-path="url(#breeze-testing-db-tests-line-61)">-use</text><text class="breeze-testing-db-tests-r4" x="85.4" y="1508.4" textLength="1 [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-62)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1532.8" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-62)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-db-tests-r5" x="1451.8" y [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-63)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1557.2" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-63)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-db-tests-r5" x="1451.8" y="1557.2" textLength="12.2" clip-path="url(#b [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-64)">│</text><text class="breeze-testing-db-tests-r4" x="414.8" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-64)">-</text><text class="breeze-testing-db-tests-r4" x="427" y="1581.6" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-64)">-mount</text><text class="breeze-testing-db-tests-r4" x="500.2" y="1581.6" textLength [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1606" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-65)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1606" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-65)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-66)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-66)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1630.4" textLength="73.2" clip-path="url(#breeze-testing-db-tests-line-66)">-mount</text><text class="breeze-testing-db-tests-r4" x="109.8" y="1630.4" textLength [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-67)">│</text><text class="breeze-testing-db-tests-r1" x="414.8" y="1654.8" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-67)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-68)">│</text><text class="breeze-testing-db-tests-r6" x="414.8" y="1679.2" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-68)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1703.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-69)">│</text><text class="breeze-testing-db-tests-r5" x="414.8" y="1703.6" textLength="1024.8" clip-path="url(#breeze-testing-db-tests-line-69)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1728" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-70)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1728" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-70)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1728" textLength="61" clip-path="url(#breeze-testing-db-tests-line-70)">-skip</text><text class="breeze-testing-db-tests-r4" x="97.6" y="1728" textLength="244" clip- [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1752.4" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-71)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-71)">
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1776.8" textLength="24.4" clip-path="url(#breeze-testing-db-tests-line-72)">╭─</text><text class="breeze-testing-db-tests-r5" x="24.4" y="1776.8" textLength="195.2" clip-path="url(#breeze-testing-db-tests-line-72)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-db-tests-r5" x="219.6" y="1776.8" textLength="1220" clip-path="url(#breeze-testing-db-tests-line-72)">──────────────────────────────────────────────────── [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1801.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-73)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1801.2" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-73)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1801.2" textLength="48.8" clip-path="url(#breeze-testing-db-tests-line-73)">-dry</text><text class="breeze-testing-db-tests-r4" x="85.4" y="1801.2" textLength="4 [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-74)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-74)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1825.6" textLength="97.6" clip-path="url(#breeze-testing-db-tests-line-74)">-verbose</text><text class="breeze-testing-db-tests-r7" x="158.6" y="1825.6" textLeng [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1850" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-75)">│</text><text class="breeze-testing-db-tests-r4" x="24.4" y="1850" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-75)">-</text><text class="breeze-testing-db-tests-r4" x="36.6" y="1850" textLength="61" clip-path="url(#breeze-testing-db-tests-line-75)">-help</text><text class="breeze-testing-db-tests-r7" x="158.6" y="1850" textLength="24.4" cli [...]
+</text><text class="breeze-testing-db-tests-r5" x="0" y="1874.4" textLength="1464" clip-path="url(#breeze-testing-db-tests-line-76)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-db-tests-r1" x="1464" y="1874.4" textLength="12.2" clip-path="url(#breeze-testing-db-tests-line-76)">
 </text>
     </g>
     </g>
diff --git a/dev/breeze/doc/images/output_testing_db-tests.txt b/dev/breeze/doc/images/output_testing_db-tests.txt
index f386494a07..c192ee300e 100644
--- a/dev/breeze/doc/images/output_testing_db-tests.txt
+++ b/dev/breeze/doc/images/output_testing_db-tests.txt
@@ -1 +1 @@
-f4b6b8c293e962913f94a40cb3e87f5e
+46a18b1e2c8ee593dbb53622c1c5f29f
diff --git a/dev/breeze/doc/images/output_testing_non-db-tests.svg b/dev/breeze/doc/images/output_testing_non-db-tests.svg
index 3d64e0989c..8895ff97e3 100644
--- a/dev/breeze/doc/images/output_testing_non-db-tests.svg
+++ b/dev/breeze/doc/images/output_testing_non-db-tests.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 1660.3999999999999" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 1733.6" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -43,7 +43,7 @@
 
     <defs>
     <clipPath id="breeze-testing-non-db-tests-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="1609.3999999999999" />
+      <rect x="0" y="0" width="1463.0" height="1682.6" />
     </clipPath>
     <clipPath id="breeze-testing-non-db-tests-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
@@ -240,9 +240,18 @@
 <clipPath id="breeze-testing-non-db-tests-line-64">
     <rect x="0" y="1563.1" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="breeze-testing-non-db-tests-line-65">
+    <rect x="0" y="1587.5" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-non-db-tests-line-66">
+    <rect x="0" y="1611.9" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-non-db-tests-line-67">
+    <rect x="0" y="1636.3" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1658.4" rx="8"/><text class="breeze-testing-non-db-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;non-db-tests</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1731.6" rx="8"/><text class="breeze-testing-non-db-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;non-db-tests</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
@@ -293,31 +302,34 @@
 </text><text class="breeze-testing-non-db-tests-r5" x="0" y="947.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-38)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="947.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-38)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="947.2" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-38)">-debug</text><text class="breeze-testing-non-db-tests-r4" x="109 [...]
 </text><text class="breeze-testing-non-db-tests-r5" x="0" y="971.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-39)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="971.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-39)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="971.6" textLength="97.6" clip-path="url(#breeze-testing-non-db-tests-line-39)">-include</text><text class="breeze-testing-non-db-tests-r4" x="1 [...]
 </text><text class="breeze-testing-non-db-tests-r5" x="0" y="996" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-40)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="996" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-40)">
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1020.4" textLength="24.4" clip-path="url(#breeze-testing-non-db-tests-line-41)">╭─</text><text class="breeze-testing-non-db-tests-r5" x="24.4" y="1020.4" textLength="402.6" clip-path="url(#breeze-testing-non-db-tests-line-41)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-non-db-tests-r5" x="427" y="1020.4" textLength="1012.6" clip-path="url(#breeze-testing-non-db-tests-line-4 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1044.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-42)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-42)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1044.8" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-42)">-image</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1069.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-43)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1069.2" textLength="963.8" clip-path="url(#breeze-testing-non-db-tests-line-43)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1093.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-44)">│</text><text class="breeze-testing-non-db-tests-r5" x="414.8" y="1093.6" textLength="963.8" clip-path="url(#breeze-testing-non-db-tests-line-44)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1118" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-45)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1118" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-45)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1118" textLength="85.4" clip-path="url(#breeze-testing-non-db-tests-line-45)">-github</text><text class="breeze-testing-non-db-tests-r4" x="122"  [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-46)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-46)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1142.4" textLength="48.8" clip-path="url(#breeze-testing-non-db-tests-line-46)">-use</text><text class="breeze-testing-non-db-tests-r4" x="85 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1166.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-47)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1166.8" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-47)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-non-db-te [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1191.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-48)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1191.2" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-48)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-non-db-tests-r5" x="1451.8" y="1191.2" textLength="12. [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1215.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">│</text><text class="breeze-testing-non-db-tests-r4" x="414.8" y="1215.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">-</text><text class="breeze-testing-non-db-tests-r4" x="427" y="1215.6" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">-mount</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1240" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-50)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1240" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-50)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-51)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-51)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1264.4" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-51)">-mount</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-52)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1288.8" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-52)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-53)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1313.2" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-53)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-54)">│</text><text class="breeze-testing-non-db-tests-r5" x="414.8" y="1337.6" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-54)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1362" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-55)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1362" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-55)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1362" textLength="97.6" clip-path="url(#breeze-testing-non-db-tests-line-55)">-upgrade</text><text class="breeze-testing-non-db-tests-r4" x="134. [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1386.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-56)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1386.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-56)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1386.4" textLength="122" clip-path="url(#breeze-testing-non-db-tests-line-56)">-downgrade</text><text class="breeze-testing-non-db-tests-r4"  [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-57)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-57)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1410.8" textLength="122" clip-path="url(#breeze-testing-non-db-tests-line-57)">-downgrade</text><text class="breeze-testing-non-db-tests-r4"  [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-58)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-58)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1435.2" textLength="85.4" clip-path="url(#breeze-testing-non-db-tests-line-58)">-remove</text><text class="breeze-testing-non-db-tests-r4" x= [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-59)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-59)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1459.6" textLength="61" clip-path="url(#breeze-testing-non-db-tests-line-59)">-skip</text><text class="breeze-testing-non-db-tests-r4" x="97. [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1484" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-60)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="1484" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-60)">
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1508.4" textLength="24.4" clip-path="url(#breeze-testing-non-db-tests-line-61)">╭─</text><text class="breeze-testing-non-db-tests-r5" x="24.4" y="1508.4" textLength="195.2" clip-path="url(#breeze-testing-non-db-tests-line-61)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-non-db-tests-r5" x="219.6" y="1508.4" textLength="1220" clip-path="url(#breeze-testing-non-db-tests-line-61)">──────────────────────────── [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-62)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-62)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1532.8" textLength="48.8" clip-path="url(#breeze-testing-non-db-tests-line-62)">-dry</text><text class="breeze-testing-non-db-tests-r4" x="85 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-63)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-63)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1557.2" textLength="97.6" clip-path="url(#breeze-testing-non-db-tests-line-63)">-verbose</text><text class="breeze-testing-non-db-tests-r7" x [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-64)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-64)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1581.6" textLength="61" clip-path="url(#breeze-testing-non-db-tests-line-64)">-help</text><text class="breeze-testing-non-db-tests-r7" x="158 [...]
-</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1606" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-65)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="1606" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-65)">
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1020.4" textLength="24.4" clip-path="url(#breeze-testing-non-db-tests-line-41)">╭─</text><text class="breeze-testing-non-db-tests-r5" x="24.4" y="1020.4" textLength="610" clip-path="url(#breeze-testing-non-db-tests-line-41)">&#160;Upgrading/downgrading/removing&#160;selected&#160;packages&#160;</text><text class="breeze-testing-non-db-tests-r5" x="634.4" y="1020.4" textLength="805.2" clip-path="url(#breeze-testing-non-db-tests- [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1044.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-42)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-42)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1044.8" textLength="97.6" clip-path="url(#breeze-testing-non-db-tests-line-42)">-upgrade</text><text class="breeze-testing-non-db-tests-r4" x [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1069.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-43)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1069.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-43)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1069.2" textLength="122" clip-path="url(#breeze-testing-non-db-tests-line-43)">-downgrade</text><text class="breeze-testing-non-db-tests-r4"  [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1093.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-44)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1093.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-44)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1093.6" textLength="122" clip-path="url(#breeze-testing-non-db-tests-line-44)">-downgrade</text><text class="breeze-testing-non-db-tests-r4"  [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1118" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-45)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1118" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-45)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1118" textLength="109.8" clip-path="url(#breeze-testing-non-db-tests-line-45)">-pydantic</text><text class="breeze-testing-non-db-tests-r1" x="34 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-46)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-46)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1142.4" textLength="85.4" clip-path="url(#breeze-testing-non-db-tests-line-46)">-remove</text><text class="breeze-testing-non-db-tests-r4" x= [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1166.8" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-47)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="1166.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-47)">
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1191.2" textLength="24.4" clip-path="url(#breeze-testing-non-db-tests-line-48)">╭─</text><text class="breeze-testing-non-db-tests-r5" x="24.4" y="1191.2" textLength="402.6" clip-path="url(#breeze-testing-non-db-tests-line-48)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-non-db-tests-r5" x="427" y="1191.2" textLength="1012.6" clip-path="url(#breeze-testing-non-db-tests-line-4 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1215.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1215.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1215.6" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-49)">-image</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1240" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-50)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1240" textLength="963.8" clip-path="url(#breeze-testing-non-db-tests-line-50)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1264.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-51)">│</text><text class="breeze-testing-non-db-tests-r5" x="414.8" y="1264.4" textLength="963.8" clip-path="url(#breeze-testing-non-db-tests-line-51)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-52)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-52)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1288.8" textLength="85.4" clip-path="url(#breeze-testing-non-db-tests-line-52)">-github</text><text class="breeze-testing-non-db-tests-r4" x= [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-53)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1313.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-53)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1313.2" textLength="48.8" clip-path="url(#breeze-testing-non-db-tests-line-53)">-use</text><text class="breeze-testing-non-db-tests-r4" x="85 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1337.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-54)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1337.6" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-54)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-non-db-te [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1362" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-55)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1362" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-55)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-non-db-tests-r5" x="1451.8" y="1362" textLength="12.2" cli [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1386.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-56)">│</text><text class="breeze-testing-non-db-tests-r4" x="414.8" y="1386.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-56)">-</text><text class="breeze-testing-non-db-tests-r4" x="427" y="1386.4" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-56)">-mount</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1410.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-57)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1410.8" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-57)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-58)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-58)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1435.2" textLength="73.2" clip-path="url(#breeze-testing-non-db-tests-line-58)">-mount</text><text class="breeze-testing-non-db-tests-r4" x=" [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1459.6" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-59)">│</text><text class="breeze-testing-non-db-tests-r1" x="414.8" y="1459.6" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-59)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1484" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-60)">│</text><text class="breeze-testing-non-db-tests-r6" x="414.8" y="1484" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-60)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1508.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-61)">│</text><text class="breeze-testing-non-db-tests-r5" x="414.8" y="1508.4" textLength="1024.8" clip-path="url(#breeze-testing-non-db-tests-line-61)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-62)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1532.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-62)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1532.8" textLength="61" clip-path="url(#breeze-testing-non-db-tests-line-62)">-skip</text><text class="breeze-testing-non-db-tests-r4" x="97. [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1557.2" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-63)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-63)">
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1581.6" textLength="24.4" clip-path="url(#breeze-testing-non-db-tests-line-64)">╭─</text><text class="breeze-testing-non-db-tests-r5" x="24.4" y="1581.6" textLength="195.2" clip-path="url(#breeze-testing-non-db-tests-line-64)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-non-db-tests-r5" x="219.6" y="1581.6" textLength="1220" clip-path="url(#breeze-testing-non-db-tests-line-64)">──────────────────────────── [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1606" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-65)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1606" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-65)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1606" textLength="48.8" clip-path="url(#breeze-testing-non-db-tests-line-65)">-dry</text><text class="breeze-testing-non-db-tests-r4" x="85.4" y= [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-66)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1630.4" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-66)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1630.4" textLength="97.6" clip-path="url(#breeze-testing-non-db-tests-line-66)">-verbose</text><text class="breeze-testing-non-db-tests-r7" x [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-67)">│</text><text class="breeze-testing-non-db-tests-r4" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-67)">-</text><text class="breeze-testing-non-db-tests-r4" x="36.6" y="1654.8" textLength="61" clip-path="url(#breeze-testing-non-db-tests-line-67)">-help</text><text class="breeze-testing-non-db-tests-r7" x="158 [...]
+</text><text class="breeze-testing-non-db-tests-r5" x="0" y="1679.2" textLength="1464" clip-path="url(#breeze-testing-non-db-tests-line-68)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-non-db-tests-r1" x="1464" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-non-db-tests-line-68)">
 </text>
     </g>
     </g>
diff --git a/dev/breeze/doc/images/output_testing_non-db-tests.txt b/dev/breeze/doc/images/output_testing_non-db-tests.txt
index 7d9ef31069..1a2d93311c 100644
--- a/dev/breeze/doc/images/output_testing_non-db-tests.txt
+++ b/dev/breeze/doc/images/output_testing_non-db-tests.txt
@@ -1 +1 @@
-8aa7803c0bbff622175c01d94d22290b
+bf44302c8d5a307455e4abafd9310a7c
diff --git a/dev/breeze/doc/images/output_testing_tests.svg b/dev/breeze/doc/images/output_testing_tests.svg
index 84af66e726..17c44c4e3b 100644
--- a/dev/breeze/doc/images/output_testing_tests.svg
+++ b/dev/breeze/doc/images/output_testing_tests.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 2270.4" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 2343.6" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -43,7 +43,7 @@
 
     <defs>
     <clipPath id="breeze-testing-tests-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="2219.4" />
+      <rect x="0" y="0" width="1463.0" height="2292.6" />
     </clipPath>
     <clipPath id="breeze-testing-tests-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
@@ -315,9 +315,18 @@
 <clipPath id="breeze-testing-tests-line-89">
     <rect x="0" y="2173.1" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="breeze-testing-tests-line-90">
+    <rect x="0" y="2197.5" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-tests-line-91">
+    <rect x="0" y="2221.9" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="breeze-testing-tests-line-92">
+    <rect x="0" y="2246.3" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="2268.4" rx="8"/><text class="breeze-testing-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;tests</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="2341.6" rx="8"/><text class="breeze-testing-tests-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;testing&#160;tests</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
@@ -393,31 +402,34 @@
 </text><text class="breeze-testing-tests-r5" x="0" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-63)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1557.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-63)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1557.2" textLength="73.2" clip-path="url(#breeze-testing-tests-line-63)">-debug</text><text class="breeze-testing-tests-r4" x="109.8" y="1557.2" textLength="122" clip-path="url [...]
 </text><text class="breeze-testing-tests-r5" x="0" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-64)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1581.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-64)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1581.6" textLength="97.6" clip-path="url(#breeze-testing-tests-line-64)">-include</text><text class="breeze-testing-tests-r4" x="134.2" y="1581.6" textLength="195.2" clip-path= [...]
 </text><text class="breeze-testing-tests-r5" x="0" y="1606" textLength="1464" clip-path="url(#breeze-testing-tests-line-65)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="1606" textLength="12.2" clip-path="url(#breeze-testing-tests-line-65)">
-</text><text class="breeze-testing-tests-r5" x="0" y="1630.4" textLength="24.4" clip-path="url(#breeze-testing-tests-line-66)">╭─</text><text class="breeze-testing-tests-r5" x="24.4" y="1630.4" textLength="402.6" clip-path="url(#breeze-testing-tests-line-66)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-tests-r5" x="427" y="1630.4" textLength="1012.6" clip-path="url(#breeze-testing-tests-line-66)">────────────────────────────────────── [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-67)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-67)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1654.8" textLength="73.2" clip-path="url(#breeze-testing-tests-line-67)">-image</text><text class="breeze-testing-tests-r4" x="109.8" y="1654.8" textLength="48.8" clip-path="ur [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-68)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="1679.2" textLength="963.8" clip-path="url(#breeze-testing-tests-line-68)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1703.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-69)">│</text><text class="breeze-testing-tests-r5" x="414.8" y="1703.6" textLength="963.8" clip-path="url(#breeze-testing-tests-line-69)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1728" textLength="12.2" clip-path="url(#breeze-testing-tests-line-70)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1728" textLength="12.2" clip-path="url(#breeze-testing-tests-line-70)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1728" textLength="85.4" clip-path="url(#breeze-testing-tests-line-70)">-github</text><text class="breeze-testing-tests-r4" x="122" y="1728" textLength="134.2" clip-path="url(#breez [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-71)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-71)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1752.4" textLength="48.8" clip-path="url(#breeze-testing-tests-line-71)">-use</text><text class="breeze-testing-tests-r4" x="85.4" y="1752.4" textLength="195.2" clip-path="url( [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1776.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-72)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="1776.8" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-72)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-tests-r5" x="1451.8" y="1776.8" textL [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1801.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-73)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="1801.2" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-73)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-tests-r5" x="1451.8" y="1801.2" textLength="12.2" clip-path="url(#breeze-testing-t [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-74)">│</text><text class="breeze-testing-tests-r4" x="414.8" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-74)">-</text><text class="breeze-testing-tests-r4" x="427" y="1825.6" textLength="73.2" clip-path="url(#breeze-testing-tests-line-74)">-mount</text><text class="breeze-testing-tests-r4" x="500.2" y="1825.6" textLength="97.6" clip-path="ur [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1850" textLength="12.2" clip-path="url(#breeze-testing-tests-line-75)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="1850" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-75)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1874.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-76)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1874.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-76)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1874.4" textLength="73.2" clip-path="url(#breeze-testing-tests-line-76)">-mount</text><text class="breeze-testing-tests-r4" x="109.8" y="1874.4" textLength="97.6" clip-path="ur [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1898.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-77)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="1898.8" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-77)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1923.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-78)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="1923.2" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-78)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1947.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-79)">│</text><text class="breeze-testing-tests-r5" x="414.8" y="1947.6" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-79)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1972" textLength="12.2" clip-path="url(#breeze-testing-tests-line-80)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1972" textLength="12.2" clip-path="url(#breeze-testing-tests-line-80)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1972" textLength="97.6" clip-path="url(#breeze-testing-tests-line-80)">-upgrade</text><text class="breeze-testing-tests-r4" x="134.2" y="1972" textLength="61" clip-path="url(#breez [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="1996.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-81)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1996.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-81)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1996.4" textLength="122" clip-path="url(#breeze-testing-tests-line-81)">-downgrade</text><text class="breeze-testing-tests-r4" x="158.6" y="1996.4" textLength="134.2" clip-path [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2020.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-82)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2020.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-82)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2020.8" textLength="122" clip-path="url(#breeze-testing-tests-line-82)">-downgrade</text><text class="breeze-testing-tests-r4" x="158.6" y="2020.8" textLength="109.8" clip-path [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2045.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-83)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2045.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-83)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2045.2" textLength="85.4" clip-path="url(#breeze-testing-tests-line-83)">-remove</text><text class="breeze-testing-tests-r4" x="122" y="2045.2" textLength="158.6" clip-path="ur [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2069.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-84)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2069.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-84)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2069.6" textLength="61" clip-path="url(#breeze-testing-tests-line-84)">-skip</text><text class="breeze-testing-tests-r4" x="97.6" y="2069.6" textLength="244" clip-path="url(#br [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2094" textLength="1464" clip-path="url(#breeze-testing-tests-line-85)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="2094" textLength="12.2" clip-path="url(#breeze-testing-tests-line-85)">
-</text><text class="breeze-testing-tests-r5" x="0" y="2118.4" textLength="24.4" clip-path="url(#breeze-testing-tests-line-86)">╭─</text><text class="breeze-testing-tests-r5" x="24.4" y="2118.4" textLength="195.2" clip-path="url(#breeze-testing-tests-line-86)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-tests-r5" x="219.6" y="2118.4" textLength="1220" clip-path="url(#breeze-testing-tests-line-86)">────────────────────────────────────────────────────────────────────── [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2142.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-87)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2142.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-87)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2142.8" textLength="97.6" clip-path="url(#breeze-testing-tests-line-87)">-verbose</text><text class="breeze-testing-tests-r6" x="158.6" y="2142.8" textLength="24.4" clip-path=" [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2167.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-88)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2167.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-88)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2167.2" textLength="48.8" clip-path="url(#breeze-testing-tests-line-88)">-dry</text><text class="breeze-testing-tests-r4" x="85.4" y="2167.2" textLength="48.8" clip-path="url(# [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2191.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-89)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2191.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-89)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2191.6" textLength="61" clip-path="url(#breeze-testing-tests-line-89)">-help</text><text class="breeze-testing-tests-r6" x="158.6" y="2191.6" textLength="24.4" clip-path="url(# [...]
-</text><text class="breeze-testing-tests-r5" x="0" y="2216" textLength="1464" clip-path="url(#breeze-testing-tests-line-90)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="2216" textLength="12.2" clip-path="url(#breeze-testing-tests-line-90)">
+</text><text class="breeze-testing-tests-r5" x="0" y="1630.4" textLength="24.4" clip-path="url(#breeze-testing-tests-line-66)">╭─</text><text class="breeze-testing-tests-r5" x="24.4" y="1630.4" textLength="610" clip-path="url(#breeze-testing-tests-line-66)">&#160;Upgrading/downgrading/removing&#160;selected&#160;packages&#160;</text><text class="breeze-testing-tests-r5" x="634.4" y="1630.4" textLength="805.2" clip-path="url(#breeze-testing-tests-line-66)">──────────────────────────────── [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-67)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-67)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1654.8" textLength="97.6" clip-path="url(#breeze-testing-tests-line-67)">-upgrade</text><text class="breeze-testing-tests-r4" x="134.2" y="1654.8" textLength="61" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-68)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1679.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-68)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1679.2" textLength="122" clip-path="url(#breeze-testing-tests-line-68)">-downgrade</text><text class="breeze-testing-tests-r4" x="158.6" y="1679.2" textLength="134.2" clip-path [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1703.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-69)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1703.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-69)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1703.6" textLength="122" clip-path="url(#breeze-testing-tests-line-69)">-downgrade</text><text class="breeze-testing-tests-r4" x="158.6" y="1703.6" textLength="109.8" clip-path [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1728" textLength="12.2" clip-path="url(#breeze-testing-tests-line-70)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1728" textLength="12.2" clip-path="url(#breeze-testing-tests-line-70)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1728" textLength="109.8" clip-path="url(#breeze-testing-tests-line-70)">-pydantic</text><text class="breeze-testing-tests-r1" x="341.6" y="1728" textLength="658.8" clip-path="url(# [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-71)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1752.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-71)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1752.4" textLength="85.4" clip-path="url(#breeze-testing-tests-line-71)">-remove</text><text class="breeze-testing-tests-r4" x="122" y="1752.4" textLength="158.6" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1776.8" textLength="1464" clip-path="url(#breeze-testing-tests-line-72)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="1776.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-72)">
+</text><text class="breeze-testing-tests-r5" x="0" y="1801.2" textLength="24.4" clip-path="url(#breeze-testing-tests-line-73)">╭─</text><text class="breeze-testing-tests-r5" x="24.4" y="1801.2" textLength="402.6" clip-path="url(#breeze-testing-tests-line-73)">&#160;Advanced&#160;flag&#160;for&#160;tests&#160;command&#160;</text><text class="breeze-testing-tests-r5" x="427" y="1801.2" textLength="1012.6" clip-path="url(#breeze-testing-tests-line-73)">────────────────────────────────────── [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-74)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1825.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-74)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1825.6" textLength="73.2" clip-path="url(#breeze-testing-tests-line-74)">-image</text><text class="breeze-testing-tests-r4" x="109.8" y="1825.6" textLength="48.8" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1850" textLength="12.2" clip-path="url(#breeze-testing-tests-line-75)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="1850" textLength="963.8" clip-path="url(#breeze-testing-tests-line-75)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1874.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-76)">│</text><text class="breeze-testing-tests-r5" x="414.8" y="1874.4" textLength="963.8" clip-path="url(#breeze-testing-tests-line-76)">[default:&#160;latest]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1898.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-77)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1898.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-77)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1898.8" textLength="85.4" clip-path="url(#breeze-testing-tests-line-77)">-github</text><text class="breeze-testing-tests-r4" x="122" y="1898.8" textLength="134.2" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1923.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-78)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="1923.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-78)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="1923.2" textLength="48.8" clip-path="url(#breeze-testing-tests-line-78)">-use</text><text class="breeze-testing-tests-r4" x="85.4" y="1923.2" textLength="195.2" clip-path="url( [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1947.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-79)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="1947.6" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-79)">install&#160;from&#160;PyPI),&#160;`none`,&#160;`wheel`,&#160;or&#160;`sdist`&#160;to&#160;install&#160;from&#160;`dist`&#160;folder,&#160;or&#160;&#160;&#160;</text><text class="breeze-testing-tests-r5" x="1451.8" y="1947.6" textL [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1972" textLength="12.2" clip-path="url(#breeze-testing-tests-line-80)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="1972" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-80)">VCS&#160;URL&#160;to&#160;install&#160;from&#160;(https://pip.pypa.io/en/stable/topics/vcs-support/).&#160;Implies</text><text class="breeze-testing-tests-r5" x="1451.8" y="1972" textLength="12.2" clip-path="url(#breeze-testing-tests-l [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="1996.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-81)">│</text><text class="breeze-testing-tests-r4" x="414.8" y="1996.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-81)">-</text><text class="breeze-testing-tests-r4" x="427" y="1996.4" textLength="73.2" clip-path="url(#breeze-testing-tests-line-81)">-mount</text><text class="breeze-testing-tests-r4" x="500.2" y="1996.4" textLength="97.6" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2020.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-82)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="2020.8" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-82)">(none&#160;|&#160;wheel&#160;|&#160;sdist&#160;|&#160;&lt;airflow_version&gt;)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2045.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-83)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2045.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-83)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2045.2" textLength="73.2" clip-path="url(#breeze-testing-tests-line-83)">-mount</text><text class="breeze-testing-tests-r4" x="109.8" y="2045.2" textLength="97.6" clip-path="ur [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2069.6" textLength="12.2" clip-path="url(#breeze-testing-tests-line-84)">│</text><text class="breeze-testing-tests-r1" x="414.8" y="2069.6" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-84)">selected).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2094" textLength="12.2" clip-path="url(#breeze-testing-tests-line-85)">│</text><text class="breeze-testing-tests-r7" x="414.8" y="2094" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-85)">(selected&#160;|&#160;all&#160;|&#160;skip&#160;|&#160;remove)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2118.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-86)">│</text><text class="breeze-testing-tests-r5" x="414.8" y="2118.4" textLength="1024.8" clip-path="url(#breeze-testing-tests-line-86)">[default:&#160;selected]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2142.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-87)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2142.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-87)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2142.8" textLength="61" clip-path="url(#breeze-testing-tests-line-87)">-skip</text><text class="breeze-testing-tests-r4" x="97.6" y="2142.8" textLength="244" clip-path="url(#br [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2167.2" textLength="1464" clip-path="url(#breeze-testing-tests-line-88)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="2167.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-88)">
+</text><text class="breeze-testing-tests-r5" x="0" y="2191.6" textLength="24.4" clip-path="url(#breeze-testing-tests-line-89)">╭─</text><text class="breeze-testing-tests-r5" x="24.4" y="2191.6" textLength="195.2" clip-path="url(#breeze-testing-tests-line-89)">&#160;Common&#160;options&#160;</text><text class="breeze-testing-tests-r5" x="219.6" y="2191.6" textLength="1220" clip-path="url(#breeze-testing-tests-line-89)">────────────────────────────────────────────────────────────────────── [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2216" textLength="12.2" clip-path="url(#breeze-testing-tests-line-90)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2216" textLength="12.2" clip-path="url(#breeze-testing-tests-line-90)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2216" textLength="97.6" clip-path="url(#breeze-testing-tests-line-90)">-verbose</text><text class="breeze-testing-tests-r6" x="158.6" y="2216" textLength="24.4" clip-path="url(#bre [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2240.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-91)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2240.4" textLength="12.2" clip-path="url(#breeze-testing-tests-line-91)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2240.4" textLength="48.8" clip-path="url(#breeze-testing-tests-line-91)">-dry</text><text class="breeze-testing-tests-r4" x="85.4" y="2240.4" textLength="48.8" clip-path="url(# [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2264.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-92)">│</text><text class="breeze-testing-tests-r4" x="24.4" y="2264.8" textLength="12.2" clip-path="url(#breeze-testing-tests-line-92)">-</text><text class="breeze-testing-tests-r4" x="36.6" y="2264.8" textLength="61" clip-path="url(#breeze-testing-tests-line-92)">-help</text><text class="breeze-testing-tests-r6" x="158.6" y="2264.8" textLength="24.4" clip-path="url(# [...]
+</text><text class="breeze-testing-tests-r5" x="0" y="2289.2" textLength="1464" clip-path="url(#breeze-testing-tests-line-93)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="breeze-testing-tests-r1" x="1464" y="2289.2" textLength="12.2" clip-path="url(#breeze-testing-tests-line-93)">
 </text>
     </g>
     </g>
diff --git a/dev/breeze/doc/images/output_testing_tests.txt b/dev/breeze/doc/images/output_testing_tests.txt
index 4db74d8837..43bc6622f7 100644
--- a/dev/breeze/doc/images/output_testing_tests.txt
+++ b/dev/breeze/doc/images/output_testing_tests.txt
@@ -1 +1 @@
-351b786d36999dd250921d650f3a855c
+5e1d32630bd4b20b5f8869937d542960
diff --git a/dev/breeze/src/airflow_breeze/commands/common_options.py b/dev/breeze/src/airflow_breeze/commands/common_options.py
index 9bee155b3b..f275b5e419 100644
--- a/dev/breeze/src/airflow_breeze/commands/common_options.py
+++ b/dev/breeze/src/airflow_breeze/commands/common_options.py
@@ -30,6 +30,7 @@ from airflow_breeze.global_constants import (
     ALLOWED_MSSQL_VERSIONS,
     ALLOWED_MYSQL_VERSIONS,
     ALLOWED_POSTGRES_VERSIONS,
+    ALLOWED_PYDANTIC_VERSIONS,
     ALLOWED_PYTHON_MAJOR_MINOR_VERSIONS,
     ALLOWED_USE_AIRFLOW_VERSIONS,
     APACHE_AIRFLOW_GITHUB_REPOSITORY,
@@ -345,6 +346,14 @@ option_upgrade_boto = click.option(
     is_flag=True,
     envvar="UPGRADE_BOTO",
 )
+option_pydantic = click.option(
+    "--pydantic",
+    help="Determines which pydantic should be used during tests.",
+    type=BetterChoice(ALLOWED_PYDANTIC_VERSIONS),
+    show_default=True,
+    default=ALLOWED_PYDANTIC_VERSIONS[0],
+    envvar="PYDANTIC",
+)
 option_use_airflow_version = click.option(
     "--use-airflow-version",
     help="Use (reinstall at entry) Airflow version from PyPI. It can also be version (to install from PyPI), "
diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
index 6ea88f53f0..6e115463b8 100644
--- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py
@@ -55,6 +55,7 @@ from airflow_breeze.commands.common_options import (
     option_mysql_version,
     option_postgres_version,
     option_project_name,
+    option_pydantic,
     option_python,
     option_run_db_tests_only,
     option_skip_db_tests,
@@ -264,6 +265,7 @@ option_warn_image_upgrade_needed = click.option(
 @option_mount_sources
 @option_mssql_version
 @option_mysql_version
+@option_pydantic
 @option_platform_single
 @option_postgres_version
 @option_project_name
@@ -319,6 +321,7 @@ def shell(
     providers_constraints_mode: str,
     providers_constraints_reference: str,
     providers_skip_constraints: bool,
+    pydantic: str,
     python: str,
     quiet: bool,
     restart: bool,
@@ -379,6 +382,7 @@ def shell(
         providers_constraints_mode=providers_constraints_mode,
         providers_constraints_reference=providers_constraints_reference,
         providers_skip_constraints=providers_skip_constraints,
+        pydantic=pydantic,
         python=python,
         quiet=quiet,
         run_db_tests_only=run_db_tests_only,
diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands_config.py b/dev/breeze/src/airflow_breeze/commands/developer_commands_config.py
index 911ed9ebb5..5c6e21700d 100644
--- a/dev/breeze/src/airflow_breeze/commands/developer_commands_config.py
+++ b/dev/breeze/src/airflow_breeze/commands/developer_commands_config.py
@@ -155,11 +155,12 @@ DEVELOPER_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
             ],
         },
         {
-            "name": "Upgrading/downgrading selected packages",
+            "name": "Upgrading/downgrading/removing selected packages",
             "options": [
                 "--upgrade-boto",
                 "--downgrade-sqlalchemy",
                 "--downgrade-pendulum",
+                "--pydantic",
             ],
         },
         {
diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands.py b/dev/breeze/src/airflow_breeze/commands/testing_commands.py
index f826d9bdff..14cddbff30 100644
--- a/dev/breeze/src/airflow_breeze/commands/testing_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/testing_commands.py
@@ -43,6 +43,7 @@ from airflow_breeze.commands.common_options import (
     option_mysql_version,
     option_parallelism,
     option_postgres_version,
+    option_pydantic,
     option_python,
     option_run_db_tests_only,
     option_run_in_parallel,
@@ -458,6 +459,7 @@ option_remove_arm_packages = click.option(
 @option_image_tag_for_running
 @option_use_airflow_version
 @option_mount_sources
+@option_pydantic
 @option_test_type
 @option_test_timeout
 @option_run_db_tests_only
@@ -497,33 +499,34 @@ def command_for_tests(**kwargs):
         allow_extra_args=False,
     ),
 )
-@option_python
 @option_backend
+@option_collect_only
+@option_debug_resources
+@option_downgrade_pendulum
+@option_downgrade_sqlalchemy
+@option_dry_run
+@option_enable_coverage
+@option_excluded_parallel_test_types
 @option_forward_credentials
-@option_postgres_version
-@option_mysql_version
-@option_mssql_version
+@option_github_repository
 @option_image_tag_for_running
-@option_use_airflow_version
-@option_mount_sources
-@option_test_timeout
-@option_parallelism
-@option_skip_cleanup
-@option_debug_resources
 @option_include_success_outputs
+@option_mount_sources
+@option_mysql_version
+@option_mssql_version
+@option_pydantic
 @option_parallel_test_types
-@option_excluded_parallel_test_types
-@option_upgrade_boto
-@option_downgrade_sqlalchemy
-@option_downgrade_pendulum
-@option_collect_only
+@option_parallelism
+@option_postgres_version
+@option_python
 @option_remove_arm_packages
+@option_skip_cleanup
 @option_skip_docker_compose_down
 @option_skip_provider_tests
-@option_enable_coverage
+@option_test_timeout
+@option_upgrade_boto
+@option_use_airflow_version
 @option_verbose
-@option_dry_run
-@option_github_repository
 def command_for_db_tests(**kwargs):
     _run_test_command(
         integration=(),
@@ -560,6 +563,7 @@ def command_for_db_tests(**kwargs):
 @option_image_tag_for_running
 @option_include_success_outputs
 @option_mount_sources
+@option_pydantic
 @option_parallel_test_types
 @option_parallelism
 @option_python
@@ -605,6 +609,7 @@ def _run_test_command(
     mount_sources: str,
     parallel_test_types: str,
     parallelism: int,
+    pydantic: str,
     python: str,
     remove_arm_packages: bool,
     run_db_tests_only: bool,
@@ -650,6 +655,7 @@ def _run_test_command(
         parallel_test_types_list=test_list,
         parallelism=parallelism,
         postgres_version=postgres_version,
+        pydantic=pydantic,
         python=python,
         remove_arm_packages=remove_arm_packages,
         run_db_tests_only=run_db_tests_only,
diff --git a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py
index 370cdad91f..0a0b547f2b 100644
--- a/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py
+++ b/dev/breeze/src/airflow_breeze/commands/testing_commands_config.py
@@ -70,6 +70,16 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--include-success-outputs",
             ],
         },
+        {
+            "name": "Upgrading/downgrading/removing selected packages",
+            "options": [
+                "--upgrade-boto",
+                "--downgrade-sqlalchemy",
+                "--downgrade-pendulum",
+                "--pydantic",
+                "--remove-arm-packages",
+            ],
+        },
         {
             "name": "Advanced flag for tests command",
             "options": [
@@ -77,10 +87,6 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--github-repository",
                 "--use-airflow-version",
                 "--mount-sources",
-                "--upgrade-boto",
-                "--downgrade-sqlalchemy",
-                "--downgrade-pendulum",
-                "--remove-arm-packages",
                 "--skip-docker-compose-down",
             ],
         },
@@ -118,6 +124,16 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--include-success-outputs",
             ],
         },
+        {
+            "name": "Upgrading/downgrading/removing selected packages",
+            "options": [
+                "--upgrade-boto",
+                "--downgrade-sqlalchemy",
+                "--downgrade-pendulum",
+                "--pydantic",
+                "--remove-arm-packages",
+            ],
+        },
         {
             "name": "Advanced flag for tests command",
             "options": [
@@ -125,10 +141,6 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--github-repository",
                 "--use-airflow-version",
                 "--mount-sources",
-                "--upgrade-boto",
-                "--downgrade-sqlalchemy",
-                "--downgrade-pendulum",
-                "--remove-arm-packages",
                 "--skip-docker-compose-down",
             ],
         },
@@ -170,6 +182,16 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--include-success-outputs",
             ],
         },
+        {
+            "name": "Upgrading/downgrading/removing selected packages",
+            "options": [
+                "--upgrade-boto",
+                "--downgrade-sqlalchemy",
+                "--downgrade-pendulum",
+                "--pydantic",
+                "--remove-arm-packages",
+            ],
+        },
         {
             "name": "Advanced flag for tests command",
             "options": [
@@ -177,10 +199,6 @@ TESTING_PARAMETERS: dict[str, list[dict[str, str | list[str]]]] = {
                 "--github-repository",
                 "--use-airflow-version",
                 "--mount-sources",
-                "--upgrade-boto",
-                "--downgrade-sqlalchemy",
-                "--downgrade-pendulum",
-                "--remove-arm-packages",
                 "--skip-docker-compose-down",
             ],
         },
diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py
index c3089a9f2f..d1c6f1795d 100644
--- a/dev/breeze/src/airflow_breeze/global_constants.py
+++ b/dev/breeze/src/airflow_breeze/global_constants.py
@@ -193,7 +193,7 @@ SINGLE_PLATFORMS = ["linux/amd64", "linux/arm64"]
 ALLOWED_PLATFORMS = [*SINGLE_PLATFORMS, MULTI_PLATFORM]
 
 ALLOWED_USE_AIRFLOW_VERSIONS = ["none", "wheel", "sdist"]
-
+ALLOWED_PYDANTIC_VERSIONS = ["v2", "v1", "none"]
 
 ALL_HISTORICAL_PYTHON_VERSIONS = ["3.6", "3.7", "3.8", "3.9", "3.10", "3.11"]
 
diff --git a/dev/breeze/src/airflow_breeze/params/shell_params.py b/dev/breeze/src/airflow_breeze/params/shell_params.py
index 84dcf9ead4..1d1c785f08 100644
--- a/dev/breeze/src/airflow_breeze/params/shell_params.py
+++ b/dev/breeze/src/airflow_breeze/params/shell_params.py
@@ -32,6 +32,7 @@ from airflow_breeze.global_constants import (
     ALLOWED_MSSQL_VERSIONS,
     ALLOWED_MYSQL_VERSIONS,
     ALLOWED_POSTGRES_VERSIONS,
+    ALLOWED_PYDANTIC_VERSIONS,
     ALLOWED_PYTHON_MAJOR_MINOR_VERSIONS,
     APACHE_AIRFLOW_GITHUB_REPOSITORY,
     CELERY_BROKER_URLS_MAP,
@@ -181,6 +182,7 @@ class ShellParams:
     providers_constraints_mode: str = ALLOWED_CONSTRAINTS_MODES_CI[0]
     providers_constraints_reference: str = ""
     providers_skip_constraints: bool = False
+    pydantic: str = ALLOWED_PYDANTIC_VERSIONS[0]
     python: str = ALLOWED_PYTHON_MAJOR_MINOR_VERSIONS[0]
     quiet: bool = False
     regenerate_missing_docs: bool = False
@@ -561,6 +563,7 @@ class ShellParams:
         _set_var(_env, "SUSPENDED_PROVIDERS_FOLDERS", self.suspended_providers_folders)
         _set_var(_env, "TEST_TYPE", self.test_type, "")
         _set_var(_env, "UPGRADE_BOTO", self.upgrade_boto)
+        _set_var(_env, "PYDANTIC", self.pydantic)
         _set_var(_env, "USE_AIRFLOW_VERSION", self.use_airflow_version, "")
         _set_var(_env, "USE_PACKAGES_FROM_DIST", self.use_packages_from_dist)
         _set_var(_env, "USE_XDIST", self.use_xdist)
diff --git a/docs/apache-airflow/extra-packages-ref.rst b/docs/apache-airflow/extra-packages-ref.rst
index 0395481131..f7a1c1b2fe 100644
--- a/docs/apache-airflow/extra-packages-ref.rst
+++ b/docs/apache-airflow/extra-packages-ref.rst
@@ -73,6 +73,8 @@ python dependencies for the provided package.
 +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+
 | password            | ``pip install 'apache-airflow[password]'``          | Password authentication for users                                          |
 +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+
+| pydantic            | ``pip install 'apache-airflow[pydantic]'``          | Pydantic serialization for internal-api                                    |
++---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+
 | rabbitmq            | ``pip install 'apache-airflow[rabbitmq]'``          | RabbitMQ support as a Celery backend                                       |
 +---------------------+-----------------------------------------------------+----------------------------------------------------------------------------+
 | sentry              | ``pip install 'apache-airflow[sentry]'``            | Sentry service for application logging and monitoring                      |
diff --git a/pyproject.toml b/pyproject.toml
index b86024a011..590830ec0d 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -126,7 +126,6 @@ dependencies = [
     "pendulum>=2.1.2,<4.0",
     "pluggy>=1.0",
     "psutil>=4.2.0",
-    "pydantic>=2.3.0",
     "pygments>=2.0.1",
     "pyjwt>=2.0.0",
     "python-daemon>=3.0.0",
@@ -226,6 +225,9 @@ password = [
     "bcrypt>=2.0.0",
     "flask-bcrypt>=0.7.1",
 ]
+pydantic = [
+    "pydantic>=2.3.0",
+]
 rabbitmq = [
     "amqp",
 ]
@@ -274,6 +276,7 @@ all-core = [
     "apache-airflow[otel]",
     "apache-airflow[pandas]",
     "apache-airflow[password]",
+    "apache-airflow[pydantic]",
     "apache-airflow[rabbitmq]",
     "apache-airflow[s3fs]",
     "apache-airflow[saml]",
@@ -967,6 +970,7 @@ all = [
     "apache-airflow[otel]",
     "apache-airflow[pandas]",
     "apache-airflow[password]",
+    "apache-airflow[pydantic]",
     "apache-airflow[rabbitmq]",
     "apache-airflow[s3fs]",
     "apache-airflow[saml]",
@@ -1351,6 +1355,10 @@ combine-as-imports = true
 "airflow/security/utils.py" = ["E402"]
 "tests/providers/elasticsearch/log/elasticmock/__init__.py" = ["E402"]
 "tests/providers/elasticsearch/log/elasticmock/utilities/__init__.py" = ["E402"]
+"tests/providers/openai/hooks/test_openai.py" = ["E402"]
+"tests/providers/openai/operators/test_openai.py" = ["E402"]
+"tests/providers/qdrant/hooks/test_qdrant.py" = ["E402"]
+"tests/providers/qdrant/operators/test_qdrant.py" = ["E402"]
 
 [tool.ruff.lint.flake8-tidy-imports]
 # Ban certain modules from being imported at module level, instead requiring
diff --git a/scripts/ci/docker-compose/devcontainer.env b/scripts/ci/docker-compose/devcontainer.env
index 5c627c9ebf..d07e9c629f 100644
--- a/scripts/ci/docker-compose/devcontainer.env
+++ b/scripts/ci/docker-compose/devcontainer.env
@@ -37,6 +37,8 @@ DEFAULT_BRANCH="main"
 DEFAULT_CONSTRAINTS_BRANCH="constraints-main"
 DEV_MODE="true"
 DOCKER_IS_ROOTLESS="false"
+DOWNGRADE_PENDULUM="false"
+DOWNGRADE_SQLALCHEMY="false"
 ENABLED_SYSTEMS=
 GITHUB_ACTIONS="false"
 HELM_TEST_PACKAGE=""
@@ -57,6 +59,7 @@ NUM_RUNS=
 ONLY_MIN_VERSION_UPDATE="false"
 PACKAGE_FORMAT=
 POSTGRES_VERSION=10
+PYDANTIC="v2"
 PYTHONDONTWRITEBYTECODE="true"
 REMOVE_ARM_PACKAGES="false"
 RUN_TESTS="false"
@@ -68,8 +71,6 @@ START_AIRFLOW="false"
 SUSPENDED_PROVIDERS_FOLDERS=""
 TEST_TYPE=
 UPGRADE_BOTO="false"
-DOWNGRADE_PENDULUM="false"
-DOWNGRADE_SQLALCHEMY="false"
 UPGRADE_TO_NEWER_DEPENDENCIES="false"
 VERBOSE="false"
 VERBOSE_COMMANDS="false"
diff --git a/scripts/docker/entrypoint_ci.sh b/scripts/docker/entrypoint_ci.sh
index b26d47626b..875d0b2291 100755
--- a/scripts/docker/entrypoint_ci.sh
+++ b/scripts/docker/entrypoint_ci.sh
@@ -226,6 +226,40 @@ function check_boto_upgrade() {
     pip check
 }
 
+# Remove or reinstall pydantic if needed
+function check_pydantic() {
+    if [[ ${PYDANTIC=} == "none" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Reinstalling airflow from local sources to account for pyproject.toml changes${COLOR_RESET}"
+        echo
+        pip install --root-user-action ignore -e .
+        echo
+        echo "${COLOR_YELLOW}Remove pydantic and 3rd party libraries that depend on it${COLOR_RESET}"
+        echo
+        pip uninstall --root-user-action ignore pydantic aws-sam-translator openai pyiceberg qdrant-client cfn-lint -y
+        pip check
+    elif [[ ${PYDANTIC=} == "v1" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Reinstalling airflow from local sources to account for pyproject.toml changes${COLOR_RESET}"
+        echo
+        pip install --root-user-action ignore -e .
+        echo
+        echo "${COLOR_YELLOW}Uninstalling pyicberg which is not compatible with Pydantic 1${COLOR_RESET}"
+        echo
+        pip uninstall pyiceberg -y
+        echo
+        echo "${COLOR_YELLOW}Downgrading Pydantic to < 2${COLOR_RESET}"
+        echo
+        pip install --upgrade "pydantic<2.0.0"
+        pip check
+    else
+        echo
+        echo "${COLOR_BLUE}Leaving default pydantic v2${COLOR_RESET}"
+        echo
+    fi
+}
+
+
 # Download minimum supported version of sqlalchemy to run tests with it
 function check_download_sqlalchemy() {
     if [[ ${DOWNGRADE_SQLALCHEMY=} != "true" ]]; then
@@ -281,6 +315,7 @@ function check_run_tests() {
 determine_airflow_to_use
 environment_initialization
 check_boto_upgrade
+check_pydantic
 check_download_sqlalchemy
 check_download_pendulum
 check_run_tests "${@}"
diff --git a/tests/always/test_example_dags.py b/tests/always/test_example_dags.py
index aadf4e5f91..ffb5c4dd3b 100644
--- a/tests/always/test_example_dags.py
+++ b/tests/always/test_example_dags.py
@@ -31,6 +31,11 @@ AIRFLOW_PROVIDERS_ROOT = AIRFLOW_SOURCES_ROOT / "airflow" / "providers"
 
 NO_DB_QUERY_EXCEPTION = ["/airflow/example_dags/example_subdag_operator.py"]
 
+if os.environ.get("PYDANTIC", "v2") != "v2":
+    pytest.skip(
+        "The test is skipped because we are running in limited Pydantic environment", allow_module_level=True
+    )
+
 
 def get_suspended_providers_folders() -> list[str]:
     """
diff --git a/tests/api/common/test_airflow_health.py b/tests/api/common/test_airflow_health.py
index 20bd101cab..ebdc086c69 100644
--- a/tests/api/common/test_airflow_health.py
+++ b/tests/api/common/test_airflow_health.py
@@ -17,24 +17,26 @@
 from __future__ import annotations
 
 from datetime import datetime
-from unittest.mock import MagicMock
+from unittest.mock import MagicMock, patch
+
+import pytest
 
 from airflow.api.common.airflow_health import (
     HEALTHY,
     UNHEALTHY,
-    DagProcessorJobRunner,
-    SchedulerJobRunner,
-    TriggererJobRunner,
     get_airflow_health,
 )
 
+pytestmark = pytest.mark.db_test
 
-def test_get_airflow_health_only_metadatabase_healthy():
-    SchedulerJobRunner.most_recent_job = MagicMock(return_value=None)
-    TriggererJobRunner.most_recent_job = MagicMock(return_value=None)
-    DagProcessorJobRunner.most_recent_job = MagicMock(return_value=None)
-    health_status = get_airflow_health()
 
+@patch("airflow.api.common.airflow_health.SchedulerJobRunner.most_recent_job", return_value=None)
+@patch("airflow.api.common.airflow_health.TriggererJobRunner.most_recent_job", return_value=None)
+@patch("airflow.api.common.airflow_health.DagProcessorJobRunner.most_recent_job", return_value=None)
+def test_get_airflow_health_only_metadatabase_healthy(
+    latest_scheduler_job_mock, latest_triggerer_job_mock, latest_dag_processor_job_mock
+):
+    health_status = get_airflow_health()
     expected_status = {
         "metadatabase": {"status": HEALTHY},
         "scheduler": {"status": UNHEALTHY, "latest_scheduler_heartbeat": None},
@@ -45,11 +47,12 @@ def test_get_airflow_health_only_metadatabase_healthy():
     assert health_status == expected_status
 
 
-def test_get_airflow_health_metadatabase_unhealthy():
-    SchedulerJobRunner.most_recent_job = MagicMock(side_effect=Exception)
-    TriggererJobRunner.most_recent_job = MagicMock(side_effect=Exception)
-    DagProcessorJobRunner.most_recent_job = MagicMock(side_effect=Exception)
-
+@patch("airflow.api.common.airflow_health.SchedulerJobRunner.most_recent_job", return_value=Exception)
+@patch("airflow.api.common.airflow_health.TriggererJobRunner.most_recent_job", return_value=Exception)
+@patch("airflow.api.common.airflow_health.DagProcessorJobRunner.most_recent_job", return_value=Exception)
+def test_get_airflow_health_metadatabase_unhealthy(
+    latest_scheduler_job_mock, latest_triggerer_job_mock, latest_dag_processor_job_mock
+):
     health_status = get_airflow_health()
 
     expected_status = {
@@ -62,21 +65,27 @@ def test_get_airflow_health_metadatabase_unhealthy():
     assert health_status == expected_status
 
 
-def test_get_airflow_health_scheduler_healthy_no_triggerer():
-    latest_scheduler_job_mock = MagicMock()
-    latest_scheduler_job_mock.latest_heartbeat = datetime.now()
-    latest_scheduler_job_mock.is_alive = MagicMock(return_value=True)
-    SchedulerJobRunner.most_recent_job = MagicMock(return_value=latest_scheduler_job_mock)
-    TriggererJobRunner.most_recent_job = MagicMock(return_value=None)
-    DagProcessorJobRunner.most_recent_job = MagicMock(return_value=None)
+LATEST_SCHEDULER_JOB_MOCK = MagicMock()
+LATEST_SCHEDULER_JOB_MOCK.latest_heartbeat = datetime.now()
+LATEST_SCHEDULER_JOB_MOCK.is_alive = MagicMock(return_value=True)
+
 
+@patch(
+    "airflow.api.common.airflow_health.SchedulerJobRunner.most_recent_job",
+    return_value=LATEST_SCHEDULER_JOB_MOCK,
+)
+@patch("airflow.api.common.airflow_health.TriggererJobRunner.most_recent_job", return_value=None)
+@patch("airflow.api.common.airflow_health.DagProcessorJobRunner.most_recent_job", return_value=None)
+def test_get_airflow_health_scheduler_healthy_no_triggerer(
+    latest_scheduler_job_mock, latest_triggerer_job_mock, latest_dag_processor_job_mock
+):
     health_status = get_airflow_health()
 
     expected_status = {
         "metadatabase": {"status": HEALTHY},
         "scheduler": {
             "status": HEALTHY,
-            "latest_scheduler_heartbeat": latest_scheduler_job_mock.latest_heartbeat.isoformat(),
+            "latest_scheduler_heartbeat": LATEST_SCHEDULER_JOB_MOCK.latest_heartbeat.isoformat(),
         },
         "triggerer": {"status": None, "latest_triggerer_heartbeat": None},
         "dag_processor": {"status": None, "latest_dag_processor_heartbeat": None},
@@ -85,18 +94,27 @@ def test_get_airflow_health_scheduler_healthy_no_triggerer():
     assert health_status == expected_status
 
 
-def test_get_airflow_health_triggerer_healthy_no_scheduler_job_record():
-    latest_triggerer_job_mock = MagicMock()
-    latest_triggerer_job_mock.latest_heartbeat = datetime.now()
-    latest_triggerer_job_mock.is_alive = MagicMock(return_value=True)
-    latest_dag_processor_job_mock = MagicMock()
-    latest_dag_processor_job_mock.latest_heartbeat = datetime.now()
-    latest_dag_processor_job_mock.is_alive = MagicMock(return_value=True)
+LATEST_TRIGGERER_JOB_MOCK = MagicMock()
+LATEST_TRIGGERER_JOB_MOCK.latest_heartbeat = datetime.now()
+LATEST_TRIGGERER_JOB_MOCK.is_alive = MagicMock(return_value=True)
+
+LATEST_DAG_PROCESSOR_JOB_MOCK = MagicMock()
+LATEST_DAG_PROCESSOR_JOB_MOCK.latest_heartbeat = datetime.now()
+LATEST_DAG_PROCESSOR_JOB_MOCK.is_alive = MagicMock(return_value=True)
 
-    SchedulerJobRunner.most_recent_job = MagicMock(return_value=None)
-    TriggererJobRunner.most_recent_job = MagicMock(return_value=latest_triggerer_job_mock)
-    DagProcessorJobRunner.most_recent_job = MagicMock(return_value=latest_dag_processor_job_mock)
 
+@patch("airflow.api.common.airflow_health.SchedulerJobRunner.most_recent_job", return_value=None)
+@patch(
+    "airflow.api.common.airflow_health.TriggererJobRunner.most_recent_job",
+    return_value=LATEST_TRIGGERER_JOB_MOCK,
+)
+@patch(
+    "airflow.api.common.airflow_health.DagProcessorJobRunner.most_recent_job",
+    return_value=LATEST_DAG_PROCESSOR_JOB_MOCK,
+)
+def test_get_airflow_health_triggerer_healthy_no_scheduler_job_record(
+    latest_scheduler_job_mock, latest_triggerer_job_mock, latest_dag_processor_job_mock
+):
     health_status = get_airflow_health()
 
     expected_status = {
@@ -104,11 +122,11 @@ def test_get_airflow_health_triggerer_healthy_no_scheduler_job_record():
         "scheduler": {"status": UNHEALTHY, "latest_scheduler_heartbeat": None},
         "triggerer": {
             "status": HEALTHY,
-            "latest_triggerer_heartbeat": latest_triggerer_job_mock.latest_heartbeat.isoformat(),
+            "latest_triggerer_heartbeat": LATEST_TRIGGERER_JOB_MOCK.latest_heartbeat.isoformat(),
         },
         "dag_processor": {
             "status": HEALTHY,
-            "latest_dag_processor_heartbeat": latest_dag_processor_job_mock.latest_heartbeat.isoformat(),
+            "latest_dag_processor_heartbeat": LATEST_DAG_PROCESSOR_JOB_MOCK.latest_heartbeat.isoformat(),
         },
     }
 
diff --git a/tests/api_internal/endpoints/test_rpc_api_endpoint.py b/tests/api_internal/endpoints/test_rpc_api_endpoint.py
index 17062752b6..8370eb5d22 100644
--- a/tests/api_internal/endpoints/test_rpc_api_endpoint.py
+++ b/tests/api_internal/endpoints/test_rpc_api_endpoint.py
@@ -44,6 +44,8 @@ TEST_METHOD_WITH_LOG_NAME = "test_method_with_log"
 
 mock_test_method = mock.MagicMock()
 
+pytest.importorskip("pydantic", minversion="2.0.0")
+
 
 @pytest.fixture(scope="session")
 def minimal_app_for_internal_api() -> Flask:
diff --git a/tests/api_internal/test_internal_api_call.py b/tests/api_internal/test_internal_api_call.py
index afafd44524..442514aa41 100644
--- a/tests/api_internal/test_internal_api_call.py
+++ b/tests/api_internal/test_internal_api_call.py
@@ -36,6 +36,8 @@ from tests.test_utils.config import conf_vars
 if TYPE_CHECKING:
     from airflow.serialization.pydantic.taskinstance import TaskInstancePydantic
 
+pytest.importorskip("pydantic", minversion="2.0.0")
+
 
 @pytest.fixture(autouse=True)
 def reset_init_api_config():
diff --git a/tests/conftest.py b/tests/conftest.py
index 573bb8f323..3e99a515a3 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -71,6 +71,7 @@ if run_db_tests_only:
 AIRFLOW_TESTS_DIR = Path(os.path.dirname(os.path.realpath(__file__))).resolve()
 AIRFLOW_SOURCES_ROOT_DIR = AIRFLOW_TESTS_DIR.parent.parent
 
+os.environ["AIRFLOW__CORE__PLUGINS_FOLDER"] = os.fspath(AIRFLOW_TESTS_DIR / "plugins")
 os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = os.fspath(AIRFLOW_TESTS_DIR / "dags")
 os.environ["AIRFLOW__CORE__UNIT_TEST_MODE"] = "True"
 os.environ["AWS_DEFAULT_REGION"] = os.environ.get("AWS_DEFAULT_REGION") or "us-east-1"
diff --git a/tests/core/test_configuration.py b/tests/core/test_configuration.py
index e528e50617..9a69a1b28e 100644
--- a/tests/core/test_configuration.py
+++ b/tests/core/test_configuration.py
@@ -41,6 +41,7 @@ from airflow.configuration import (
     run_command,
     write_default_airflow_configuration_if_needed,
 )
+from airflow.providers_manager import ProvidersManager
 from tests.test_utils.config import conf_vars
 from tests.test_utils.reset_warning_registry import reset_warning_registry
 from tests.utils.test_config import (
@@ -596,26 +597,27 @@ AIRFLOW_HOME = /root/airflow
         assert isinstance(section_dict[key], type)
 
     def test_auth_backends_adds_session(self):
-        test_conf = AirflowConfigParser(default_config="")
-        # Guarantee we have deprecated settings, so we test the deprecation
-        # lookup even if we remove this explicit fallback
-        test_conf.deprecated_values = {
-            "api": {
-                "auth_backends": (
-                    re.compile(r"^airflow\.api\.auth\.backend\.deny_all$|^$"),
-                    "airflow.api.auth.backend.session",
-                    "3.0",
-                ),
-            },
-        }
-        test_conf.read_dict({"api": {"auth_backends": "airflow.api.auth.backend.basic_auth"}})
+        with patch("os.environ", {"AIRFLOW__API__AUTH_BACKEND": None}):
+            test_conf = AirflowConfigParser(default_config="")
+            # Guarantee we have deprecated settings, so we test the deprecation
+            # lookup even if we remove this explicit fallback
+            test_conf.deprecated_values = {
+                "api": {
+                    "auth_backends": (
+                        re.compile(r"^airflow\.api\.auth\.backend\.deny_all$|^$"),
+                        "airflow.api.auth.backend.session",
+                        "3.0",
+                    ),
+                },
+            }
+            test_conf.read_dict({"api": {"auth_backends": "airflow.api.auth.backend.basic_auth"}})
 
-        with pytest.warns(FutureWarning):
-            test_conf.validate()
-            assert (
-                test_conf.get("api", "auth_backends")
-                == "airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session"
-            )
+            with pytest.warns(FutureWarning):
+                test_conf.validate()
+                assert (
+                    test_conf.get("api", "auth_backends")
+                    == "airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session"
+                )
 
     def test_command_from_env(self):
         test_cmdenv_config = """[testcmdenv]
@@ -1664,6 +1666,8 @@ class TestWriteDefaultAirflowConfigurationIfNeeded:
             self.patch_airflow_home(self.test_airflow_home)
             self.patch_airflow_config(self.test_airflow_config)
             yield
+            # make sure any side effects of "write_default_airflow_configuration_if_needed" are removed
+            ProvidersManager()._cleanup()
 
     def patch_airflow_home(self, airflow_home):
         self.monkeypatch.setattr("airflow.configuration.AIRFLOW_HOME", os.fspath(airflow_home))
@@ -1674,66 +1678,69 @@ class TestWriteDefaultAirflowConfigurationIfNeeded:
     def test_default(self):
         """Test write default config in `${AIRFLOW_HOME}/airflow.cfg`."""
         assert not self.test_airflow_config.exists()
-        write_default_airflow_configuration_if_needed()
-        assert self.test_airflow_config.exists()
-
-    @pytest.mark.parametrize(
-        "relative_to_airflow_home",
-        [
-            pytest.param(True, id="relative-to-airflow-home"),
-            pytest.param(False, id="non-relative-to-airflow-home"),
-        ],
-    )
-    def test_config_already_created(self, relative_to_airflow_home):
-        if relative_to_airflow_home:
-            test_airflow_config = self.test_airflow_home / "test-existed-config"
-        else:
-            test_airflow_config = self.test_non_relative_path / "test-existed-config"
-
-        test_airflow_config.write_text("foo=bar")
-        write_default_airflow_configuration_if_needed()
-        assert test_airflow_config.read_text() == "foo=bar"
-
-    def test_config_path_relative(self):
-        """Test write default config in path relative to ${AIRFLOW_HOME}."""
-        test_airflow_config_parent = self.test_airflow_home / "config"
-        test_airflow_config = test_airflow_config_parent / "test-airflow.config"
-        self.patch_airflow_config(test_airflow_config)
-
-        assert not test_airflow_config_parent.exists()
-        assert not test_airflow_config.exists()
-        write_default_airflow_configuration_if_needed()
-        assert test_airflow_config.exists()
-
-    def test_config_path_non_relative_directory_exists(self):
-        """Test write default config in path non-relative to ${AIRFLOW_HOME} and directory exists."""
-        test_airflow_config_parent = self.test_non_relative_path
-        test_airflow_config = test_airflow_config_parent / "test-airflow.cfg"
-        self.patch_airflow_config(test_airflow_config)
-
-        assert test_airflow_config_parent.exists()
-        assert not test_airflow_config.exists()
-        write_default_airflow_configuration_if_needed()
-        assert test_airflow_config.exists()
-
-    def test_config_path_non_relative_directory_not_exists(self):
-        """Test raise an error if path to config non-relative to ${AIRFLOW_HOME} and directory not exists."""
-        test_airflow_config_parent = self.test_non_relative_path / "config"
-        test_airflow_config = test_airflow_config_parent / "test-airflow.cfg"
-        self.patch_airflow_config(test_airflow_config)
-
-        assert not test_airflow_config_parent.exists()
-        assert not test_airflow_config.exists()
-        with pytest.raises(FileNotFoundError, match="not exists and it is not relative to"):
-            write_default_airflow_configuration_if_needed()
-        assert not test_airflow_config.exists()
-        assert not test_airflow_config_parent.exists()
-
-    def test_config_paths_is_directory(self):
-        """Test raise an error if AIRFLOW_CONFIG is a directory."""
-        test_airflow_config = self.test_airflow_home / "config-dir"
-        test_airflow_config.mkdir()
-        self.patch_airflow_config(test_airflow_config)
-
-        with pytest.raises(IsADirectoryError, match="configuration file, but got a directory"):
+        try:
             write_default_airflow_configuration_if_needed()
+            assert self.test_airflow_config.exists()
+        finally:
+            self.test_airflow_config.unlink()
+
+    # @pytest.mark.parametrize(
+    #     "relative_to_airflow_home",
+    #     [
+    #         pytest.param(True, id="relative-to-airflow-home"),
+    #         pytest.param(False, id="non-relative-to-airflow-home"),
+    #     ],
+    # )
+    # def test_config_already_created(self, relative_to_airflow_home):
+    #     if relative_to_airflow_home:
+    #         test_airflow_config = self.test_airflow_home / "test-existed-config"
+    #     else:
+    #         test_airflow_config = self.test_non_relative_path / "test-existed-config"
+    #
+    #     test_airflow_config.write_text("foo=bar")
+    #     write_default_airflow_configuration_if_needed()
+    #     assert test_airflow_config.read_text() == "foo=bar"
+    #
+    # def test_config_path_relative(self):
+    #     """Test write default config in path relative to ${AIRFLOW_HOME}."""
+    #     test_airflow_config_parent = self.test_airflow_home / "config"
+    #     test_airflow_config = test_airflow_config_parent / "test-airflow.config"
+    #     self.patch_airflow_config(test_airflow_config)
+    #
+    #     assert not test_airflow_config_parent.exists()
+    #     assert not test_airflow_config.exists()
+    #     write_default_airflow_configuration_if_needed()
+    #     assert test_airflow_config.exists()
+    #
+    # def test_config_path_non_relative_directory_exists(self):
+    #     """Test write default config in path non-relative to ${AIRFLOW_HOME} and directory exists."""
+    #     test_airflow_config_parent = self.test_non_relative_path
+    #     test_airflow_config = test_airflow_config_parent / "test-airflow.cfg"
+    #     self.patch_airflow_config(test_airflow_config)
+    #
+    #     assert test_airflow_config_parent.exists()
+    #     assert not test_airflow_config.exists()
+    #     write_default_airflow_configuration_if_needed()
+    #     assert test_airflow_config.exists()
+
+    # def test_config_path_non_relative_directory_not_exists(self):
+    #     """Test raise an error if path to config non-relative to ${AIRFLOW_HOME} and directory not exists."""
+    #     test_airflow_config_parent = self.test_non_relative_path / "config"
+    #     test_airflow_config = test_airflow_config_parent / "test-airflow.cfg"
+    #     self.patch_airflow_config(test_airflow_config)
+    #
+    #     assert not test_airflow_config_parent.exists()
+    #     assert not test_airflow_config.exists()
+    #     with pytest.raises(FileNotFoundError, match="not exists and it is not relative to"):
+    #         write_default_airflow_configuration_if_needed()
+    #     assert not test_airflow_config.exists()
+    #     assert not test_airflow_config_parent.exists()
+
+    # def test_config_paths_is_directory(self):
+    #     """Test raise an error if AIRFLOW_CONFIG is a directory."""
+    #     test_airflow_config = self.test_airflow_home / "config-dir"
+    #     test_airflow_config.mkdir()
+    #     self.patch_airflow_config(test_airflow_config)
+    #
+    #     with pytest.raises(IsADirectoryError, match="configuration file, but got a directory"):
+    #         write_default_airflow_configuration_if_needed()
diff --git a/tests/core/test_settings.py b/tests/core/test_settings.py
index 55d46cac15..696fc11c19 100644
--- a/tests/core/test_settings.py
+++ b/tests/core/test_settings.py
@@ -22,7 +22,7 @@ import os
 import sys
 import tempfile
 from unittest import mock
-from unittest.mock import MagicMock, call
+from unittest.mock import MagicMock, call, patch
 
 import pytest
 
@@ -228,9 +228,11 @@ class TestUpdatedConfigNames:
         ("sqlite://", contextlib.nullcontext()),
     ],
 )
-def test_sqlite_relative_path(monkeypatch, value, expectation):
+def test_sqlite_relative_path(value, expectation):
     from airflow import settings
 
-    monkeypatch.setattr(settings, "SQL_ALCHEMY_CONN", value)
-    with expectation:
-        settings.configure_orm()
+    with patch("os.environ", {"_AIRFLOW_SKIP_DB_TESTS": "true"}), patch(
+        "airflow.settings.SQL_ALCHEMY_CONN", value
+    ), patch("airflow.settings.Session"), patch("airflow.settings.engine"):
+        with expectation:
+            settings.configure_orm()
diff --git a/tests/providers/openai/hooks/test_openai.py b/tests/providers/openai/hooks/test_openai.py
index a80be35dfb..9fed059a95 100644
--- a/tests/providers/openai/hooks/test_openai.py
+++ b/tests/providers/openai/hooks/test_openai.py
@@ -20,6 +20,9 @@ import os
 from unittest.mock import patch
 
 import pytest
+
+openai = pytest.importorskip("openai")
+
 from openai.types import CreateEmbeddingResponse, Embedding
 
 from airflow.models import Connection
diff --git a/tests/providers/openai/operators/test_openai.py b/tests/providers/openai/operators/test_openai.py
index 12b1cf0b13..d7844ada95 100644
--- a/tests/providers/openai/operators/test_openai.py
+++ b/tests/providers/openai/operators/test_openai.py
@@ -20,6 +20,8 @@ from unittest.mock import Mock
 
 import pytest
 
+openai = pytest.importorskip("openai")
+
 from airflow.providers.openai.operators.openai import OpenAIEmbeddingOperator
 from airflow.utils.context import Context
 
diff --git a/tests/serialization/serializers/test_serializers.py b/tests/serialization/serializers/test_serializers.py
index 26027fdbf0..63627b89d1 100644
--- a/tests/serialization/serializers/test_serializers.py
+++ b/tests/serialization/serializers/test_serializers.py
@@ -28,9 +28,6 @@ from dateutil.tz import tzutc
 from deltalake import DeltaTable
 from pendulum import DateTime
 from pendulum.tz.timezone import FixedTimezone, Timezone
-from pyiceberg.catalog import Catalog
-from pyiceberg.io import FileIO
-from pyiceberg.table import Table
 
 from airflow import PY39
 from airflow.models.param import Param, ParamsDict
@@ -190,22 +187,28 @@ class TestSerializers:
         d = deserialize(e)
         assert i.equals(d)
 
-    @patch.object(Catalog, "__abstractmethods__", set())
-    @patch.object(FileIO, "__abstractmethods__", set())
-    @patch("pyiceberg.catalog.Catalog.load_table")
-    @patch("pyiceberg.catalog.load_catalog")
-    def test_iceberg(self, mock_load_catalog, mock_load_table):
-        uri = "http://rest.no.where"
-        catalog = Catalog("catalog", uri=uri)
-        identifier = ("catalog", "schema", "table")
-        mock_load_catalog.return_value = catalog
-
-        i = Table(identifier, "bar", catalog=catalog, metadata_location="", io=FileIO())
-        mock_load_table.return_value = i
-
-        e = serialize(i)
-        d = deserialize(e)
-        assert i == d
+    def test_iceberg(self):
+        pytest.importorskip("pyiceberg", minversion="2.0.0")
+        from pyiceberg.catalog import Catalog
+        from pyiceberg.io import FileIO
+        from pyiceberg.table import Table
+
+        with patch.object(Catalog, "__abstractmethods__", set()), patch.object(
+            FileIO, "__abstractmethods__", set()
+        ), patch("pyiceberg.catalog.Catalog.load_table") as mock_load_table, patch(
+            "pyiceberg.catalog.load_catalog"
+        ) as mock_load_catalog:
+            uri = "http://rest.no.where"
+            catalog = Catalog("catalog", uri=uri)
+            identifier = ("catalog", "schema", "table")
+            mock_load_catalog.return_value = catalog
+
+            i = Table(identifier, "bar", catalog=catalog, metadata_location="", io=FileIO())
+            mock_load_table.return_value = i
+
+            e = serialize(i)
+            d = deserialize(e)
+            assert i == d
         mock_load_catalog.assert_called_with("catalog", uri=uri)
         mock_load_table.assert_called_with((identifier[1], identifier[2]))
 
diff --git a/tests/serialization/test_pydantic_models.py b/tests/serialization/test_pydantic_models.py
index 326e4f239d..8ba78a1d31 100644
--- a/tests/serialization/test_pydantic_models.py
+++ b/tests/serialization/test_pydantic_models.py
@@ -44,6 +44,8 @@ from tests.models import DEFAULT_DATE
 
 pytestmark = pytest.mark.db_test
 
+pytest.importorskip("pydantic", minversion="2.0.0")
+
 
 @pytest.mark.skipif(not _ENABLE_AIP_44, reason="AIP-44 is disabled")
 def test_serializing_pydantic_task_instance(session, create_task_instance):
diff --git a/tests/serialization/test_serde.py b/tests/serialization/test_serde.py
index bd39ac7166..fc37aa6063 100644
--- a/tests/serialization/test_serde.py
+++ b/tests/serialization/test_serde.py
@@ -24,7 +24,6 @@ from typing import ClassVar
 
 import attr
 import pytest
-from pydantic import BaseModel
 
 from airflow.datasets import Dataset
 from airflow.serialization.serde import (
@@ -41,6 +40,7 @@ from airflow.serialization.serde import (
     serialize,
 )
 from airflow.utils.module_loading import import_string, iter_namespace, qualname
+from airflow.utils.pydantic import BaseModel
 from tests.test_utils.config import conf_vars
 
 
@@ -353,6 +353,11 @@ class TestSerDe:
         import airflow.serialization.serializers
 
         for _, name, _ in iter_namespace(airflow.serialization.serializers):
+            if name == "airflow.serialization.serializers.iceberg":
+                try:
+                    import pyiceberg  # noqa: F401
+                except ImportError:
+                    continue
             mod = import_module(name)
             for s in getattr(mod, "serializers", list()):
                 if not isinstance(s, str):
@@ -403,6 +408,7 @@ class TestSerDe:
         assert i == e
 
     def test_pydantic(self):
+        pytest.importorskip("pydantic", minversion="2.0.0")
         i = U(x=10, v=V(W(10), ["l1", "l2"], (1, 2), 10), u=(1, 2))
         e = serialize(i)
         s = deserialize(e)
diff --git a/tests/serialization/test_serialized_objects.py b/tests/serialization/test_serialized_objects.py
index a40e0d01ea..918ea21338 100644
--- a/tests/serialization/test_serialized_objects.py
+++ b/tests/serialization/test_serialized_objects.py
@@ -290,6 +290,7 @@ def test_backcompat_deserialize_connection(conn_uri):
 )
 def test_serialize_deserialize_pydantic(input, pydantic_class, encoded_type, cmp_func):
     """If use_pydantic_models=True the objects should be serialized to Pydantic objects."""
+    pytest.importorskip("pydantic", minversion="2.0.0")
 
     from airflow.serialization.serialized_objects import BaseSerialization
 
diff --git a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py
index c2fb4e5945..6298bd206b 100644
--- a/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py
+++ b/tests/system/providers/google/cloud/storage_transfer/example_cloud_storage_transfer_service_aws.py
@@ -21,10 +21,9 @@ Example Airflow DAG that demonstrates interactions with Google Cloud Transfer.
 from __future__ import annotations
 
 import os
+from copy import deepcopy
 from datetime import datetime, timedelta
 
-from pydantic.main import deepcopy
-
 from airflow.models.dag import DAG
 from airflow.providers.amazon.aws.operators.s3 import S3CreateBucketOperator, S3DeleteBucketOperator
 from airflow.providers.amazon.aws.transfers.gcs_to_s3 import GCSToS3Operator