Skip to content

Commit

Permalink
Consistent way of checking Airflow version in providers
Browse files Browse the repository at this point in the history
This PR introduces consistent way of checking version of Airflow
by Airflow providers. So far there were about 6 different ways on
how Providers checked for Airflow version - this PR aims to unify
this approach for now and in the future - at least until minimum
version of Airflow set to 2.11 where we are likely to introduce
a simpler check via #44607. Until then all providers are going
to have `version_references.py` module copied in their sources
that they will be importing the constants from.

This PR also adds pre-commit that checks if the
``version_compat.py`` module is imported from local package copy
or maybe from another provider or test code - both causing
unneeded dependencies from the provider - to another package or
to test code respectively.
  • Loading branch information
potiuk committed Dec 8, 2024
1 parent 088e796 commit 22b6bb2
Show file tree
Hide file tree
Showing 166 changed files with 847 additions and 347 deletions.
7 changes: 7 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,13 @@ repos:
language: python
additional_dependencies: ['rich>=12.4.4']
require_serial: true
- id: check-imports-in-providers
name: Check imports in providers
entry: ./scripts/ci/pre_commit/check_imports_in_providers.py
language: python
additional_dependencies: ['rich>=12.4.4', "ruff==0.8.1"]
files: ^providers/src/airflow/providers/.*\.py$
require_serial: true
- id: update-common-sql-api-stubs
name: Check and update common.sql API stubs
entry: ./scripts/ci/pre_commit/update_common_sql_api_stubs.py
Expand Down
2 changes: 2 additions & 0 deletions contributing-docs/08_static_code_checks.rst
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,8 @@ require Breeze Docker image to be built locally.
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| check-hooks-apply | Check if all hooks apply to the repository | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| check-imports-in-providers | Check imports in providers | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| check-incorrect-use-of-LoggingMixin | Make sure LoggingMixin is not used alone | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| check-init-decorator-arguments | Sync model __init__ and decorator arguments | |
Expand Down
2 changes: 1 addition & 1 deletion contributing-docs/testing/unit_tests.rst
Original file line number Diff line number Diff line change
Expand Up @@ -1108,7 +1108,7 @@ are not part of the public API. We deal with it in one of the following ways:

.. code-block:: python
from tests_common.test_utils.compat import AIRFLOW_V_2_9_PLUS
from tests_common.test_utils.version_compat import AIRFLOW_V_2_9_PLUS
@pytest.mark.skipif(not AIRFLOW_V_2_9_PLUS, reason="The tests should be skipped for Airflow < 2.9")
Expand Down
148 changes: 76 additions & 72 deletions dev/breeze/doc/images/output_static-checks.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
2 changes: 1 addition & 1 deletion dev/breeze/doc/images/output_static-checks.txt
Original file line number Diff line number Diff line change
@@ -1 +1 @@
f9eb3d902f6df6f3bcf01be469d958e3
1fd1ce1703cb27b2fd042f55f5aa6ddd
1 change: 1 addition & 0 deletions dev/breeze/src/airflow_breeze/pre_commit_ids.py
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
"check-google-re2-as-dependency",
"check-hatch-build-order",
"check-hooks-apply",
"check-imports-in-providers",
"check-incorrect-use-of-LoggingMixin",
"check-init-decorator-arguments",
"check-integrations-list-consistent",
Expand Down
25 changes: 25 additions & 0 deletions providers/src/airflow/providers/MANAGING_PROVIDERS_LIFECYCLE.rst
Original file line number Diff line number Diff line change
Expand Up @@ -433,6 +433,31 @@ the compatibility checks should be updated when min airflow version is updated.
Details on how this should be done are described in
`Provider policies <https://github.com/apache/airflow/blob/main/dev/README_RELEASE_PROVIDER_PACKAGES.md>`_

Conditional provider variants
=============================

Sometimes providers need to have different variants for different versions of Airflow. This is done by:

* copying ``version_compat.py`` from one of the providers that already have conditional variants to
the root package of the provider you are working on

* importing the ``AIRFLOW_V_X_Y_PLUS`` that you need from that imported ``version_compat.py`` file.

The main reasons we are doing it in this way:

* checking version >= in Python has a non-obvious problem that the pre-release version is always considered
lower than the final version. This is why we are using ``AIRFLOW_V_X_Y_PLUS`` to check for the version
that is greater or equal to the version we are checking against - because we want the RC candidates
to be considered as equal to the final version (because those RC candidates already contain the feature
that is added in the final version).
* We do not want to add dependencies to another provider (say ``common.compat``) without strong need
* Even if the code is duplicated, it is just one ``version_compat.py`` file that is wholly copied
and it is not a big deal to maintain it.
* There is a potential risk of one provider importing the same ``AIRFLOW_V_X_Y_PLUS`` from another provider
(and introduce accidental dependency) or from test code (which should not happen), but we are preventing it
via pre-commit check ``check-imports-in-providers`` that will fail if the
``version_compat`` module is imported from another provider or from test code.

Releasing pre-installed providers for the first time
====================================================

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
AwsSecurityManagerOverride,
)
from airflow.providers.amazon.aws.auth_manager.views.auth import AwsAuthManagerAuthenticationViews
from airflow.providers.amazon.version_compat import AIRFLOW_V_2_9_PLUS

try:
from airflow.auth.managers.base_auth_manager import BaseAuthManager, ResourceMethod
Expand Down Expand Up @@ -81,12 +82,7 @@ class AwsAuthManager(BaseAuthManager):
"""

def __init__(self, appbuilder: AirflowAppBuilder) -> None:
from packaging.version import Version

from airflow.version import version

# TODO: remove this if block when min_airflow_version is set to higher than 2.9.0
if Version(version) < Version("2.9"):
if not AIRFLOW_V_2_9_PLUS:
raise AirflowOptionalProviderFeatureException(
"``AwsAuthManager`` is compatible with Airflow versions >= 2.9."
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,19 +20,15 @@
from typing import TYPE_CHECKING

import redshift_connector
from packaging.version import Version
from redshift_connector import Connection as RedshiftConnection
from sqlalchemy import create_engine
from sqlalchemy.engine.url import URL

from airflow import __version__ as AIRFLOW_VERSION
from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook
from airflow.providers.amazon.version_compat import AIRFLOW_V_2_10_PLUS
from airflow.providers.common.sql.hooks.sql import DbApiHook

_IS_AIRFLOW_2_10_OR_HIGHER = Version(Version(AIRFLOW_VERSION).base_version) >= Version("2.10.0")


if TYPE_CHECKING:
from airflow.models.connection import Connection
from airflow.providers.openlineage.sqlparser import DatabaseInfo
Expand Down Expand Up @@ -265,6 +261,6 @@ def get_openlineage_database_dialect(self, connection: Connection) -> str:

def get_openlineage_default_schema(self) -> str | None:
"""Return current schema. This is usually changed with ``SEARCH_PATH`` parameter."""
if _IS_AIRFLOW_2_10_OR_HIGHER:
if AIRFLOW_V_2_10_PLUS:
return self.get_first("SELECT CURRENT_SCHEMA();")[0]
return super().get_openlineage_default_schema()
Original file line number Diff line number Diff line change
Expand Up @@ -124,9 +124,9 @@ def __init__(
self.s3_acl_policy = s3_acl_policy
self.keep_directory_structure = keep_directory_structure
try:
from airflow.providers.google import __version__
from airflow.providers.google import __version__ as _GOOGLE_PROVIDER_VERSION

if Version(__version__) >= Version("10.3.0"):
if Version(Version(_GOOGLE_PROVIDER_VERSION).base_version) >= Version("10.3.0"):
self.__is_match_glob_supported = True
else:
self.__is_match_glob_supported = False
Expand Down
16 changes: 1 addition & 15 deletions providers/src/airflow/providers/amazon/aws/utils/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,21 +31,7 @@


def trim_none_values(obj: dict):
from packaging.version import Version

from airflow.version import version

if Version(version) < Version("2.7"):
# before version 2.7, the behavior is not the same.
# Empty dict and lists are removed from the given dict.
return {key: val for key, val in obj.items() if val is not None}
else:
# once airflow 2.6 rolls out of compatibility support for provider packages,
# we can replace usages of this method with the core one in our code,
# and uncomment this warning for users who may use it.
# warnings.warn("use airflow.utils.helpers.prune_dict() instead",
# AirflowProviderDeprecationWarning, stacklevel=2)
return prune_dict(obj)
return prune_dict(obj)


def datetime_to_epoch(date_time: datetime) -> int:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,25 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#
# NOTE! THIS FILE IS COPIED MANUALLY IN OTHER PROVIDERS DELIBERATELY TO AVOID ADDING UNNECESSARY
# DEPENDENCIES BETWEEN PROVIDERS. IF YOU WANT TO ADD CONDITIONAL CODE IN YOUR PROVIDER THAT DEPENDS
# ON AIRFLOW VERSION, PLEASE COPY THIS FILE TO THE ROOT PACKAGE OF YOUR PROVIDER AND IMPORT
# THOSE CONSTANTS FROM IT RATHER THAN IMPORTING THEM FROM ANOTHER PROVIDER OR TEST CODE
#

from __future__ import annotations

from packaging.version import Version

from airflow import __version__ as AIRFLOW_VERSION

AIRFLOW_V_3_0_PLUS = Version(Version(AIRFLOW_VERSION).base_version) >= Version("3.0.0")
AIRFLOW_V_2_10_PLUS = Version(Version(AIRFLOW_VERSION).base_version) >= Version("2.10.0")
AIRFLOW_V_2_9_PLUS = Version(Version(AIRFLOW_VERSION).base_version) >= Version("2.9.0")
AIRFLOW_V_2_8_PLUS = Version(Version(AIRFLOW_VERSION).base_version) >= Version("2.8.0")
def _get_base_airflow_version() -> Version:
from airflow import __version__

return Version(Version(__version__).base_version)


AIRFLOW_V_2_8_PLUS = _get_base_airflow_version() >= Version("2.8.0")
AIRFLOW_V_2_9_PLUS = _get_base_airflow_version() >= Version("2.9.0")
AIRFLOW_V_2_10_PLUS = _get_base_airflow_version() >= Version("2.10.0")
AIRFLOW_V_3_0_PLUS = _get_base_airflow_version() >= Version("3.0.0")
Original file line number Diff line number Diff line change
Expand Up @@ -25,22 +25,18 @@
from kubernetes import client
from kubernetes.client.api_client import ApiClient
from kubernetes.client.rest import ApiException
from packaging.version import Version

from airflow import __version__ as airflow_version
from airflow.models import DagRun, TaskInstance
from airflow.providers.cncf.kubernetes import pod_generator
from airflow.providers.cncf.kubernetes.executors.kubernetes_executor import KubeConfig
from airflow.providers.cncf.kubernetes.kube_client import get_kube_client
from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import create_unique_id
from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator
from airflow.providers.cncf.kubernetes.version_compat import AIRFLOW_V_3_0_PLUS
from airflow.utils import cli as cli_utils, yaml
from airflow.utils.cli import get_dag
from airflow.utils.providers_configuration_loader import providers_configuration_loaded

AIRFLOW_VERSION = Version(airflow_version)
AIRFLOW_V_3_0_PLUS = Version(AIRFLOW_VERSION.base_version) >= Version("3.0.0")


@cli_utils.action_cli
@providers_configuration_loaded
Expand Down
38 changes: 38 additions & 0 deletions providers/src/airflow/providers/cncf/kubernetes/version_compat.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
# 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.
#
# NOTE! THIS FILE IS COPIED MANUALLY IN OTHER PROVIDERS DELIBERATELY TO AVOID ADDING UNNECESSARY
# DEPENDENCIES BETWEEN PROVIDERS. IF YOU WANT TO ADD CONDITIONAL CODE IN YOUR PROVIDER THAT DEPENDS
# ON AIRFLOW VERSION, PLEASE COPY THIS FILE TO THE ROOT PACKAGE OF YOUR PROVIDER AND IMPORT
# THOSE CONSTANTS FROM IT RATHER THAN IMPORTING THEM FROM ANOTHER PROVIDER OR TEST CODE
#

from __future__ import annotations

from packaging.version import Version


def _get_base_airflow_version() -> Version:
from airflow import __version__

return Version(Version(__version__).base_version)


AIRFLOW_V_2_8_PLUS = _get_base_airflow_version() >= Version("2.8.0")
AIRFLOW_V_2_9_PLUS = _get_base_airflow_version() >= Version("2.9.0")
AIRFLOW_V_2_10_PLUS = _get_base_airflow_version() >= Version("2.10.0")
AIRFLOW_V_3_0_PLUS = _get_base_airflow_version() >= Version("3.0.0")
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@

from typing import TYPE_CHECKING

from airflow.providers.common.compat.version_references import (
AIRFLOW_V_2_8_PLUS,
from airflow.providers.common.compat.version_compat import (
AIRFLOW_V_2_9_PLUS,
AIRFLOW_V_2_10_PLUS,
AIRFLOW_V_3_0_PLUS,
Expand All @@ -37,11 +36,9 @@
from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetAll, AssetAny
else:
# dataset is renamed to asset since Airflow 3.0
from airflow.auth.managers.models.resource_details import DatasetDetails as AssetDetails
from airflow.datasets import Dataset as Asset

if AIRFLOW_V_2_8_PLUS:
from airflow.auth.managers.models.resource_details import DatasetDetails as AssetDetails

if AIRFLOW_V_2_9_PLUS:
from airflow.datasets import (
DatasetAll as AssetAll,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
# under the License.
from __future__ import annotations

from airflow.providers.common.compat.version_references import AIRFLOW_V_2_10_PLUS, AIRFLOW_V_3_0_PLUS
from airflow.providers.common.compat.version_compat import AIRFLOW_V_2_10_PLUS, AIRFLOW_V_3_0_PLUS


def _get_asset_compat_hook_lineage_collector():
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

from typing import TYPE_CHECKING

from airflow import __version__ as AIRFLOW_VERSION
from airflow.providers.common.compat.version_compat import AIRFLOW_V_2_10_PLUS

if TYPE_CHECKING:
from airflow.providers.standard.operators.python import (
Expand All @@ -37,17 +37,13 @@
get_current_context,
)
except ModuleNotFoundError:
from packaging.version import Version

_IS_AIRFLOW_2_10_OR_HIGHER = Version(Version(AIRFLOW_VERSION).base_version) >= Version("2.10.0")

from airflow.operators.python import (
PythonOperator,
ShortCircuitOperator,
get_current_context,
)

if _IS_AIRFLOW_2_10_OR_HIGHER:
if AIRFLOW_V_2_10_PLUS:
from airflow.operators.python import _SERIALIZERS


Expand Down
38 changes: 38 additions & 0 deletions providers/src/airflow/providers/common/compat/version_compat.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
# 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.
#
# NOTE! THIS FILE IS COPIED MANUALLY IN OTHER PROVIDERS DELIBERATELY TO AVOID ADDING UNNECESSARY
# DEPENDENCIES BETWEEN PROVIDERS. IF YOU WANT TO ADD CONDITIONAL CODE IN YOUR PROVIDER THAT DEPENDS
# ON AIRFLOW VERSION, PLEASE COPY THIS FILE TO THE ROOT PACKAGE OF YOUR PROVIDER AND IMPORT
# THOSE CONSTANTS FROM IT RATHER THAN IMPORTING THEM FROM ANOTHER PROVIDER OR TEST CODE
#

from __future__ import annotations

from packaging.version import Version


def _get_base_airflow_version() -> Version:
from airflow import __version__

return Version(Version(__version__).base_version)


AIRFLOW_V_2_8_PLUS = _get_base_airflow_version() >= Version("2.8.0")
AIRFLOW_V_2_9_PLUS = _get_base_airflow_version() >= Version("2.9.0")
AIRFLOW_V_2_10_PLUS = _get_base_airflow_version() >= Version("2.10.0")
AIRFLOW_V_3_0_PLUS = _get_base_airflow_version() >= Version("3.0.0")
6 changes: 1 addition & 5 deletions providers/src/airflow/providers/common/io/assets/file.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,8 @@
import urllib.parse
from typing import TYPE_CHECKING

from packaging.version import Version
from airflow.providers.common.io.version_compat import AIRFLOW_V_3_0_PLUS

from airflow import __version__ as AIRFLOW_VERSION

# TODO: Remove version check block after bumping common provider to 1.3.0
AIRFLOW_V_3_0_PLUS = Version(Version(AIRFLOW_VERSION).base_version) >= Version("3.0.0")
if AIRFLOW_V_3_0_PLUS:
from airflow.sdk.definitions.asset import Asset
else:
Expand Down
Loading

0 comments on commit 22b6bb2

Please sign in to comment.