Skip to content

Commit

Permalink
Turn Pydantic into an optional depenedency
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
potiuk committed Feb 11, 2024
1 parent 70fd6ad commit ac41af9
Show file tree
Hide file tree
Showing 38 changed files with 390 additions and 161 deletions.
50 changes: 50 additions & 0 deletions .github/workflows/ci.yml
Expand Up @@ -1277,6 +1277,56 @@ jobs:
uses: ./.github/actions/post_tests_failure
if: failure()
tests-postgres-no-pydantic:
timeout-minutes: 130
name: >
DB:NoPydantic${{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}}"
NO_PYDANTIC: "true"
JOB_ID: >
postgres-nopydantic-${{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}}:Boto"
uses: ./.github/actions/post_tests_success
if: success()
- name: >
Post Tests failure: ${{needs.build-info.outputs.default-python-version}}:Boto"
uses: ./.github/actions/post_tests_failure
if: failure()
tests-postgres-pendulum-2:
timeout-minutes: 130
name: >
Expand Down
13 changes: 13 additions & 0 deletions Dockerfile.ci
Expand Up @@ -897,6 +897,18 @@ function check_boto_upgrade() {
pip check
}

function check_no_pydantic() {
if [[ ${NO_PYDANTIC=} != "true" ]]; then
return
fi
echo
echo "${COLOR_BLUE}Remove pydantic${COLOR_RESET}"
echo
pip uninstall --root-user-action ignore pydantic -y || true
pip check
}


function check_download_sqlalchemy() {
if [[ ${DOWNGRADE_SQLALCHEMY=} != "true" ]]; then
return
Expand Down Expand Up @@ -949,6 +961,7 @@ function check_run_tests() {
determine_airflow_to_use
environment_initialization
check_boto_upgrade
check_no_pydantic
check_download_sqlalchemy
check_download_pendulum
check_run_tests "${@}"
Expand Down
8 changes: 4 additions & 4 deletions INSTALL
Expand Up @@ -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

Expand Down
2 changes: 2 additions & 0 deletions airflow/providers/apache/hdfs/sensors/hdfs.py
Expand Up @@ -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.
Expand Down
5 changes: 1 addition & 4 deletions airflow/providers/papermill/hooks/kernel.py
Expand Up @@ -16,17 +16,14 @@
# under the License.
from __future__ import annotations

from typing import TYPE_CHECKING
import typing

from jupyter_client import AsyncKernelManager
from papermill.clientwrap import PapermillNotebookClient
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
Expand Down
10 changes: 5 additions & 5 deletions airflow/serialization/pydantic/dag.py
Expand Up @@ -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


Expand Down
11 changes: 8 additions & 3 deletions airflow/serialization/pydantic/dag_run.py
Expand Up @@ -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
from airflow.utils.session import NEW_SESSION, provide_session

if TYPE_CHECKING:
Expand Down Expand Up @@ -101,4 +100,10 @@ def get_task_instance(
)


DagRunPydantic.model_rebuild()
try:
import pydantic # noqa: F401

# Only run this is pydantic is installed
DagRunPydantic.model_rebuild()
except ImportError:
pass
2 changes: 1 addition & 1 deletion airflow/serialization/pydantic/dataset.py
Expand Up @@ -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):
Expand Down
3 changes: 1 addition & 2 deletions airflow/serialization/pydantic/job.py
Expand Up @@ -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:
Expand Down
12 changes: 9 additions & 3 deletions airflow/serialization/pydantic/taskinstance.py
Expand Up @@ -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
Expand All @@ -29,16 +28,17 @@
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
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


Expand Down Expand Up @@ -430,4 +430,10 @@ def command_as_list(
)


TaskInstancePydantic.model_rebuild()
try:
import pydantic # noqa: F401

# Only run this is pydantic is installed
TaskInstancePydantic.model_rebuild()
except ImportError:
pass
2 changes: 1 addition & 1 deletion airflow/serialization/pydantic/tasklog.py
Expand Up @@ -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):
Expand Down
3 changes: 1 addition & 2 deletions airflow/serialization/serialized_objects.py
Expand Up @@ -70,15 +70,14 @@
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
from airflow.models.taskmixin import DAGNode
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:
Expand Down
53 changes: 53 additions & 0 deletions airflow/utils/pydantic.py
@@ -0,0 +1,53 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

# 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

try:
from pydantic import BaseModel, ConfigDict, PlainSerializer, PlainValidator, ValidationInfo
except ImportError:

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
8 changes: 4 additions & 4 deletions contributing-docs/12_airflow_dependencies_and_extras.rst
Expand Up @@ -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
Expand Down

0 comments on commit ac41af9

Please sign in to comment.