Skip to content

Commit

Permalink
Generate constants with airflow versions in providers
Browse files Browse the repository at this point in the history
Such constants can be imported in the code and used to implement
conditional logic for different versions of Airflow.

The main reasons we are doing it in this way:

* We do not want to add dependencies to another provider (say
  common.compat) without strong need
* We do not want to duplicate code (the code is generated automatically
  from ``PROVIDER__INIT__PY_TEMPLATE.py.jinja2``) -
  so it has one source of truth
* The PROVIDER_<PROVIDER> is added so that you do not accidentally
  import version from ``tests_common`` or another provider - which might
  happen if you use IDE and semi-automatically add missing importsUpdate

This PR also adds pre-commit that will automatically regenerate
providers `__init__.py` file when either provider.yaml changes or
when `__init__.py` files are modified by a commmit - this way pre-commit
will not let commit pass if the `__init__.py` file has been modified
manually (it will later be modified during release). This change will
prevent it from happening.

Fixes: #44024
  • Loading branch information
potiuk committed Dec 5, 2024
1 parent a242ff6 commit d077918
Show file tree
Hide file tree
Showing 59 changed files with 605 additions and 214 deletions.
8 changes: 8 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,14 @@ repos:
pass_filenames: true
files: ^providers/src/airflow/providers/.*/(operators|transfers|sensors)/.*\.py$
additional_dependencies: [ 'rich>=12.4.4' ]
- id: update-providers-init-py
name: Update providers __init__.py files
entry: ./scripts/ci/pre_commit/update_providers_init.py
language: python
pass_filenames: true
files: ^providers/.*/__init__.py$|^providers/.*/provider.yaml$
require_serial: true
additional_dependencies: ['rich>=12.4.4','requests']
- id: ruff
name: Run 'ruff' for extremely fast Python linting
description: "Run 'ruff' for extremely fast Python linting"
Expand Down
4 changes: 4 additions & 0 deletions airflow/provider.yaml.schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -489,6 +489,10 @@
"examples": [
1609459200
]
},
"need-version-variants": {
"type": "boolean",
"description": "Whether the provider implement variants multiple versions of airflow"
}
},
"additionalProperties": false,
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 @@ -384,6 +384,8 @@ require Breeze Docker image to be built locally.
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| update-providers-dependencies | Update dependencies for provider packages | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| update-providers-init-py | Update providers __init__.py files | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| update-reproducible-source-date-epoch | Update Source Date Epoch for reproducible builds | |
+-----------------------------------------------------------+--------------------------------------------------------+---------+
| update-spelling-wordlist-to-be-sorted | Sort spelling_wordlist.txt | |
Expand Down
78 changes: 41 additions & 37 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 @@
53b7f32a93cb7dec849138d404c47f6c
f9eb3d902f6df6f3bcf01be469d958e3
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 @@ -146,6 +146,7 @@
"update-migration-references",
"update-openapi-spec-tags-to-be-sorted",
"update-providers-dependencies",
"update-providers-init-py",
"update-reproducible-source-date-epoch",
"update-spelling-wordlist-to-be-sorted",
"update-supported-versions",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1068,6 +1068,7 @@ def update_changelog(
:param reapply_templates_only: only reapply templates, no changelog generation
:param with_breaking_changes: whether there are any breaking changes
:param maybe_with_new_features: whether there are any new features
:param only_min_version_update: whether to only update the min version
"""
provider_details = get_provider_details(package_id)
jinja_context = get_provider_documentation_jinja_context(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,9 @@
from __future__ import annotations

import packaging.version
{%- if NEED_VERSION_VARIANTS %}
from packaging.version import Version
{%- endif %}

from airflow import __version__ as airflow_version

Expand All @@ -56,3 +59,23 @@ if packaging.version.parse(packaging.version.parse(airflow_version).base_version
raise RuntimeError(
f"The package `{{ PACKAGE_PIP_NAME }}:{__version__}` needs Apache Airflow {{ MIN_AIRFLOW_VERSION }}+"
)

{% if NEED_VERSION_VARIANTS %}
def _get_airflow_version() -> Version:
"""
Return packaging Version object representing the base version.

This method and below versions are generated automatically via pre-commit when you set
"need-version-variants: true" in the provider.yaml file, so if you need them in another provider
you should set it there as well, add the modified provider.yaml to your commit
and run `pre-commit run update-providers-init-py` - or just commit it if you have
pre-commit installed and configured.
"""
return Version(Version(__version__).base_version)
_VERSION = _get_airflow_version()
PROVIDERS_{{ PROVIDER_CONSTANT_PREFIX }}_AIRFLOW_IS_2_8_PLUS = Version("2.8.0") <= _VERSION
PROVIDERS_{{ PROVIDER_CONSTANT_PREFIX }}_AIRFLOW_IS_2_9_PLUS = Version("2.9.0") <= _VERSION
PROVIDERS_{{ PROVIDER_CONSTANT_PREFIX }}_AIRFLOW_IS_2_10_PLUS = Version("2.10.0") <= _VERSION
PROVIDERS_{{ PROVIDER_CONSTANT_PREFIX }}_AIRFLOW_IS_2_11_PLUS = Version("2.11.0") <= _VERSION
PROVIDERS_{{ PROVIDER_CONSTANT_PREFIX }}_AIRFLOW_IS_3_0_PLUS = Version("3.0.0") <= _VERSION
{% endif -%}
6 changes: 5 additions & 1 deletion dev/breeze/src/airflow_breeze/utils/packages.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ class ProviderPackageDetails(NamedTuple):
excluded_python_versions: list[str]
plugins: list[PluginInfo]
removed: bool
need_version_variants: bool


class PackageSuspendedException(Exception):
Expand Down Expand Up @@ -554,9 +555,10 @@ def get_provider_details(provider_id: str) -> ProviderPackageDetails:
provider_description=provider_info["description"],
dependencies=provider_info["dependencies"],
versions=provider_info["versions"],
excluded_python_versions=provider_info.get("excluded-python-versions") or [],
excluded_python_versions=provider_info.get("excluded-python-versions", []),
plugins=plugins,
removed=provider_info["state"] == "removed",
need_version_variants=provider_info.get("need-version-variants", False),
)


Expand Down Expand Up @@ -652,9 +654,11 @@ def get_provider_jinja_context(
cross_providers_dependencies = get_cross_provider_dependent_packages(provider_package_id=provider_id)
context: dict[str, Any] = {
"PROVIDER_ID": provider_details.provider_id,
"PROVIDER_CONSTANT_PREFIX": provider_details.provider_id.replace(".", "_").upper(),
"PACKAGE_PIP_NAME": get_pip_package_name(provider_details.provider_id),
"PACKAGE_DIST_PREFIX": get_dist_package_name_prefix(provider_details.provider_id),
"FULL_PACKAGE_NAME": provider_details.full_package_name,
"NEED_VERSION_VARIANTS": provider_details.need_version_variants,
"RELEASE": current_release_version,
"RELEASE_NO_LEADING_ZEROS": release_version_no_leading_zeros,
"VERSION_SUFFIX": format_version_suffix(version_suffix),
Expand Down
36 changes: 32 additions & 4 deletions docs/apache-airflow-providers-edge/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -61,12 +61,12 @@


apache-airflow-providers-edge package
-------------------------------------
------------------------------------------------------

Handle Edge Workers via HTTP(s) connection and distribute work over remote sites
Handle edge workers on remote sites via HTTP(s) connection and orchestrates work over distributed sites


Release: 0.1.0pre0
Release: 0.9.0pre0

Provider package
----------------
Expand All @@ -90,5 +90,33 @@ The minimum Apache Airflow version supported by this provider package is ``2.10.
PIP package Version required
================== ==================
``apache-airflow`` ``>=2.10.0``
``pydantic``
``pydantic`` ``>=2.10.2``
================== ==================

Cross provider package dependencies
-----------------------------------

Those are dependencies that might be needed in order to use all the features of the package.
You need to install the specified provider packages in order to use them.

You can install such cross-provider dependencies when installing from PyPI. For example:

.. code-block:: bash
pip install apache-airflow-providers-edge[common.compat]
================================================================================================================== =================
Dependent package Extra
================================================================================================================== =================
`apache-airflow-providers-common-compat <https://airflow.apache.org/docs/apache-airflow-providers-common-compat>`_ ``common.compat``
================================================================================================================== =================

Downloading official packages
-----------------------------

You can download officially released packages and verify their checksums and signatures from the
`Official Apache Download site <https://downloads.apache.org/airflow/providers/>`_

* `The apache-airflow-providers-edge 0.9.0pre0 sdist package <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0.tar.gz>`_ (`asc <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0.tar.gz.asc>`__, `sha512 <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0.tar.gz.sha512>`__)
* `The apache-airflow-providers-edge 0.9.0pre0 wheel package <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0-py3-none-any.whl>`_ (`asc <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0-py3-none-any.whl.asc>`__, `sha512 <https://downloads.apache.org/airflow/providers/apache_airflow_providers_edge-0.9.0pre0-py3-none-any.whl.sha512>`__)
45 changes: 45 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,51 @@ 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
enabling the following lines to your ``provider.yaml`` file:

.. code-block:: yaml
need-version-variants: true
Adding it to the commit and running this pre-commit (or just committing the code you have pre-commit installed):

.. code-block:: bash
pre-commit run update-providers-init-py
This will generate version constants in the provider's ``__init__.py`` file that you will be able to use
in the code of the provider. Example for the Google provider:

.. code-block:: python
def _get_airflow_version() -> Version:
"""Return packaging Version object representing the base version."""
return Version(Version(__version__).base_version)
_VERSION = _get_airflow_version()
PROVIDERS_GOOGLE_AIRFLOW_IS_2_8_PLUS = Version("2.8.0") <= _VERSION
PROVIDERS_GOOGLE_AIRFLOW_IS_2_9_PLUS = Version("2.9.0") <= _VERSION
PROVIDERS_GOOGLE_AIRFLOW_IS_2_10_PLUS = Version("2.10.0") <= _VERSION
PROVIDERS_GOOGLE_AIRFLOW_IS_2_11_PLUS = Version("2.11.0") <= _VERSION
PROVIDERS_GOOGLE_AIRFLOW_IS_3_0_PLUS = Version("3.0.0") <= _VERSION
Such constants can be imported in the code and used to implement conditional logic for different versions
of Airflow.

The main reasons we are doing it in this way:

* We do not want to add dependencies to another provider (say common.compat) without strong need
* We do not want to duplicate code (the code is generated automatically from ``PROVIDER__INIT__PY_TEMPLATE.py.jinja2``) -
so it has one source of truth
* The PROVIDER_<PROVIDER> is added so that you do not accidentally import version from ``tests_common`` or
another provider - which might happen if you use IDE and semi-automatically add missing imports

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

Expand Down
22 changes: 22 additions & 0 deletions providers/src/airflow/providers/amazon/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
from __future__ import annotations

import packaging.version
from packaging.version import Version

from airflow import __version__ as airflow_version

Expand All @@ -37,3 +38,24 @@
raise RuntimeError(
f"The package `apache-airflow-providers-amazon:{__version__}` needs Apache Airflow 2.8.0+"
)


def _get_airflow_version() -> Version:
"""
Return packaging Version object representing the base version.
This method and below versions are generated automatically via pre-commit when you set
"need-version-variants: true" in the provider.yaml file, so if you need them in another provider
you should set it there as well, add the modified provider.yaml to your commit
and run `pre-commit run update-providers-init-py` - or just commit it if you have
pre-commit installed and configured.
"""
return Version(Version(__version__).base_version)


_VERSION = _get_airflow_version()
PROVIDERS_AMAZON_AIRFLOW_IS_2_8_PLUS = Version("2.8.0") <= _VERSION
PROVIDERS_AMAZON_AIRFLOW_IS_2_9_PLUS = Version("2.9.0") <= _VERSION
PROVIDERS_AMAZON_AIRFLOW_IS_2_10_PLUS = Version("2.10.0") <= _VERSION
PROVIDERS_AMAZON_AIRFLOW_IS_2_11_PLUS = Version("2.11.0") <= _VERSION
PROVIDERS_AMAZON_AIRFLOW_IS_3_0_PLUS = Version("3.0.0") <= _VERSION
10 changes: 2 additions & 8 deletions providers/src/airflow/providers/amazon/aws/assets/s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

from typing import TYPE_CHECKING

from airflow.providers.amazon import PROVIDERS_AMAZON_AIRFLOW_IS_3_0_PLUS
from airflow.providers.amazon.aws.hooks.s3 import S3Hook

if TYPE_CHECKING:
Expand All @@ -28,17 +29,10 @@
Dataset as OpenLineageDataset,
)
else:
# TODO: Remove this try-exception block after bumping common provider to 1.3.0
# This is due to common provider AssetDetails import error handling
try:
from airflow.providers.common.compat.assets import Asset
except ImportError:
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")
if AIRFLOW_V_3_0_PLUS:
if PROVIDERS_AMAZON_AIRFLOW_IS_3_0_PLUS:
from airflow.sdk.definitions.asset import Asset
else:
# dataset is renamed to asset since Airflow 3.0
Expand Down
1 change: 1 addition & 0 deletions providers/src/airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ description: |
state: ready
source-date-epoch: 1730011042
need-version-variants: true
# note that those versions are maintained by release manager - do not update them manually
versions:
- 9.1.0
Expand Down
22 changes: 22 additions & 0 deletions providers/src/airflow/providers/cncf/kubernetes/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
from __future__ import annotations

import packaging.version
from packaging.version import Version

from airflow import __version__ as airflow_version

Expand All @@ -37,3 +38,24 @@
raise RuntimeError(
f"The package `apache-airflow-providers-cncf-kubernetes:{__version__}` needs Apache Airflow 2.8.0+"
)


def _get_airflow_version() -> Version:
"""
Return packaging Version object representing the base version.
This method and below versions are generated automatically via pre-commit when you set
"need-version-variants: true" in the provider.yaml file, so if you need them in another provider
you should set it there as well, add the modified provider.yaml to your commit
and run `pre-commit run update-providers-init-py` - or just commit it if you have
pre-commit installed and configured.
"""
return Version(Version(__version__).base_version)


_VERSION = _get_airflow_version()
PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_2_8_PLUS = Version("2.8.0") <= _VERSION
PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_2_9_PLUS = Version("2.9.0") <= _VERSION
PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_2_10_PLUS = Version("2.10.0") <= _VERSION
PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_2_11_PLUS = Version("2.11.0") <= _VERSION
PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_3_0_PLUS = Version("3.0.0") <= _VERSION
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,9 @@
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 import PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_3_0_PLUS, 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
Expand All @@ -38,18 +36,15 @@
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
def generate_pod_yaml(args):
"""Generate yaml files for each task in the DAG. Used for testing output of KubernetesExecutor."""
logical_date = args.logical_date if AIRFLOW_V_3_0_PLUS else args.execution_date
logical_date = args.logical_date if PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_3_0_PLUS else args.execution_date
dag = get_dag(subdir=args.subdir, dag_id=args.dag_id)
yaml_output_path = args.output_path
if AIRFLOW_V_3_0_PLUS:
if PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_3_0_PLUS:
dr = DagRun(dag.dag_id, logical_date=logical_date)
else:
dr = DagRun(dag.dag_id, execution_date=logical_date)
Expand All @@ -63,7 +58,7 @@ def generate_pod_yaml(args):
pod_id=create_unique_id(args.dag_id, ti.task_id),
try_number=ti.try_number,
kube_image=kube_config.kube_image,
date=ti.logical_date if AIRFLOW_V_3_0_PLUS else ti.execution_date,
date=ti.logical_date if PROVIDERS_CNCF_KUBERNETES_AIRFLOW_IS_3_0_PLUS else ti.execution_date,
args=ti.command_as_list(),
pod_override_object=PodGenerator.from_obj(ti.executor_config),
scheduler_job_id="worker-config",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ description: |
state: ready
source-date-epoch: 1732429220
need-version-variants: true
# note that those versions are maintained by release manager - do not update them manually
versions:
- 10.0.1
Expand Down
Loading

0 comments on commit d077918

Please sign in to comment.