From 16aeb19fe1a1f4c06d753c7c8b2484dc646ca8bc Mon Sep 17 00:00:00 2001 From: Jarek Potiuk Date: Sat, 10 Feb 2024 23:46:38 +0100 Subject: [PATCH] Turn Pydantic into an optional dependency 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. --- .github/workflows/ci.yml | 97 +++++++--- Dockerfile.ci | 34 ++++ INSTALL | 8 +- 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 +- .../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 | 8 +- contributing-docs/testing/unit_tests.rst | 2 - dev/breeze/doc/images/output_shell.svg | 46 ++--- dev/breeze/doc/images/output_shell.txt | 2 +- .../doc/images/output_testing_db-tests.svg | 68 ++++--- .../doc/images/output_testing_db-tests.txt | 2 +- .../images/output_testing_non-db-tests.svg | 68 ++++--- .../images/output_testing_non-db-tests.txt | 2 +- .../doc/images/output_testing_tests.svg | 68 ++++--- .../doc/images/output_testing_tests.txt | 2 +- .../airflow_breeze/commands/common_options.py | 9 + .../commands/developer_commands.py | 4 + .../commands/developer_commands_config.py | 3 +- .../commands/testing_commands.py | 40 +++-- .../commands/testing_commands_config.py | 42 +++-- .../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 + .../providers/openai/operators/test_openai.py | 2 + tests/providers/qdrant/hooks/test_qdrant.py | 4 + .../providers/qdrant/operators/test_qdrant.py | 3 + .../serializers/test_serializers.py | 41 +++-- tests/serialization/test_pydantic_models.py | 2 + tests/serialization/test_serde.py | 8 +- .../serialization/test_serialized_objects.py | 1 + ...mple_cloud_storage_transfer_service_aws.py | 3 +- 51 files changed, 704 insertions(+), 340 deletions(-) create mode 100644 airflow/utils/pydantic.py diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 98f610a4fba26..8d501c9d5fcfd 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -1178,14 +1178,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}}" @@ -1200,7 +1249,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: @@ -1217,24 +1266,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}}" @@ -1248,9 +1299,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: @@ -1267,22 +1318,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] @@ -1300,7 +1350,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: @@ -1317,7 +1367,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" @@ -1328,11 +1378,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] @@ -1350,7 +1399,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: @@ -1367,7 +1416,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 6d75ac2d5113b..8946f2d9037c4 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -900,6 +900,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 @@ -952,6 +985,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 082858264c58a..71c520b752082 100644 --- a/INSTALL +++ b/INSTALL @@ -253,10 +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, qdrant, 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, qdrant, 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 49b12d740b7f3..9e2551c4e88e7 100644 --- a/airflow/providers/apache/hdfs/sensors/hdfs.py +++ b/airflow/providers/apache/hdfs/sensors/hdfs.py @@ -18,6 +18,8 @@ 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 0bac65dc9867c..9e6bb12a80e93 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.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 1f1fe397b9785..6ecca347fa690 100644 --- a/airflow/providers_manager.py +++ b/airflow/providers_manager.py @@ -1268,3 +1268,26 @@ def provider_configs(self) -> list[tuple[str, dict[str, Any]]]: @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 04b2472355a93..03a49f5663dbe 100644 --- a/airflow/serialization/pydantic/dag.py +++ b/airflow/serialization/pydantic/dag.py @@ -21,17 +21,17 @@ 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 8faabc5ee47e9..3de072cf90fa3 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -19,10 +19,9 @@ 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 @@ def get_task_instance( ) -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 0c233a3fd67c6..44822c546d957 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 2db30ab2c8558..fd805284253eb 100644 --- a/airflow/serialization/pydantic/job.py +++ b/airflow/serialization/pydantic/job.py @@ -18,10 +18,9 @@ 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 13760f030d8a3..9e874c569b027 100644 --- a/airflow/serialization/pydantic/taskinstance.py +++ b/airflow/serialization/pydantic/taskinstance.py @@ -19,7 +19,6 @@ 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_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 @@ def command_as_list( ) -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 a23204400c1f9..3fbbf872b8f8a 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 5ec07e4f6c1a9..7adddbab10b00 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -70,8 +70,6 @@ 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 @@ 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 0000000000000..0ec184672c2c3 --- /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 270f567afd065..c460d003a005f 100644 --- a/contributing-docs/12_airflow_dependencies_and_extras.rst +++ b/contributing-docs/12_airflow_dependencies_and_extras.rst @@ -209,10 +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, qdrant, 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, qdrant, 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 4b84cf92c783b..53bf3a1210a61 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 22111e4033d15..4f9e7ec86e712 100644 --- a/dev/breeze/doc/images/output_shell.svg +++ b/dev/breeze/doc/images/output_shell.svg @@ -1,4 +1,4 @@ - +