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 2022/03/29 18:27:47 UTC

[airflow] branch main updated: Update our approach for executor-bound dependencies (#22573)

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

potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new 6db30f3  Update our approach for executor-bound dependencies (#22573)
6db30f3 is described below

commit 6db30f32074e4ef50993628e810781cd704d4ddd
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Mar 29 20:26:52 2022 +0200

    Update our approach for executor-bound dependencies (#22573)
    
    Kubernetes and Celery are both providers and part of the core.
    The dependencies for both are added via "extras" which makes them
    "soft" limits and in case of serious dependency bumps this might
    end up with a mess (as we experienced with bumping min K8S
    library version from 11.0.0 to 22.* (resulting in yanking 4
    versions of `cncf.kubernetes` provider.
    
    After this learning, we approach K8S and Celery dependencies a bit
    differently than any other dependencies.
    
    * for Celery and K8S (and Dask but this is rather an afterhought)
      we do not strip-off the dependencies from the extra (so for
      example [cncf.kubernetes] extra will have dependencies on
      both 'apache-airflow-providers-cncf-kubernetes' as well as
      directly on kubernetes library
    
    * We add upper-bound limits for both Celery and Kubernetes to prevent
      from accidental upgrades. Both Celery and Kubernetes Python library
      follow SemVer, and they are crucial components of Airlfow so they
      both squarely fit our "do not upper-bound" exceptions.
    
    * We also add a rule that whenever dependency upper-bound limit is
      raised, we should also make sure that additional testing is done
      and appropriate `apache-airflow` lower-bound limit is added for
      the `apache-airflow-providers-cncf-kubernetes` and
      `apache-airflow-providers-celery` providers.
    
    As part of this change we also had to fix two issues:
    * the image was needlesly rebuilt during constraint generation as
      we already have the image and we even warn that it should
      be built before we run constraint generation
    
    * after this change, the currently released, unyanked cncf.kubernetes
      provider cannot be installed with airflow, because it has
      conflicting requirements for kubernetes library (provider has
      <11 and airflow has > 22.7). Therefore during constraint
      generation with PyPI providers we install providers from PyPI, we
      explicitly install the yanked 3.1.2 version. This should be
      removed after we release the next K8S provider version.
    
    That should protect our users in all scenarios where they might
    unknowingly attempt to upgrade Kubernetes or Celery to incompatible
    version.
    
    Related to: #22560, #21727
---
 .github/workflows/ci.yml                           |  2 +
 README.md                                          |  8 ++++
 airflow/providers/celery/provider.yaml             |  3 +-
 airflow/providers/cncf/kubernetes/provider.yaml    |  2 +-
 scripts/ci/constraints/ci_generate_constraints.sh  |  2 -
 .../ci_make_providers_2_1_compliant.sh}            | 18 +++------
 scripts/in_container/_in_container_utils.sh        |  4 +-
 setup.py                                           | 43 ++++++++++++++++++----
 8 files changed, 57 insertions(+), 25 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index f59f137..3ec3dab 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -781,6 +781,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         env:
           USE_AIRFLOW_VERSION: "wheel"
           PACKAGE_FORMAT: "wheel"
+      - name: "Replace non-compliant providers with their 2.1-compliant versions"
+        run: ./scripts/ci/provider_packages/ci_make_providers_2_1_compliant.sh
       - name: "Install and test provider packages and airflow on Airflow 2.1 files"
         run: ./scripts/ci/provider_packages/ci_install_and_test_provider_packages.sh
         env:
diff --git a/README.md b/README.md
index cbb326d..b8cb4c4 100644
--- a/README.md
+++ b/README.md
@@ -379,6 +379,14 @@ The important dependencies are:
    are very likely to introduce breaking changes across those so limiting it to MAJOR version makes sense
 * `werkzeug`: the library is known to cause problems in new versions. It is tightly coupled with Flask
    libraries, and we should update them together
+* `celery`: Celery is crucial component of Airflow as it used for CeleryExecutor (and similar). Celery
+   [follows SemVer](https://docs.celeryq.dev/en/stable/contributing.html?highlight=semver#versions), so
+   we should upper-bound it to the next MAJOR version. Also when we bump the upper version of the library,
+   we should make sure Celery Provider minimum Airflow version is updated).
+* `kubernetes`: Kubernetes is a crucial component of Airflow as it is used for the KubernetesExecutor
+   (and similar). Kubernetes Python library [follows SemVer](https://github.com/kubernetes-client/python#compatibility),
+   so we should upper-bound it to the next MAJOR version. Also when we bump the upper version of the library,
+   we should make sure Kubernetes Provider minimum Airflow version is updated.
 
 ### Approach for dependencies in Airflow Providers and extras
 
diff --git a/airflow/providers/celery/provider.yaml b/airflow/providers/celery/provider.yaml
index b9b448c..eb0c2b4 100644
--- a/airflow/providers/celery/provider.yaml
+++ b/airflow/providers/celery/provider.yaml
@@ -31,8 +31,7 @@ versions:
   - 1.0.0
 
 additional-dependencies:
-  - apache-airflow>=2.1.0
-  - celery~=5.1,>=5.1.2
+  - apache-airflow>=2.2.0
 
 integrations:
   - integration-name: Celery
diff --git a/airflow/providers/cncf/kubernetes/provider.yaml b/airflow/providers/cncf/kubernetes/provider.yaml
index b5b5054..404ceff 100644
--- a/airflow/providers/cncf/kubernetes/provider.yaml
+++ b/airflow/providers/cncf/kubernetes/provider.yaml
@@ -41,7 +41,7 @@ versions:
   - 1.0.0
 
 additional-dependencies:
-  - apache-airflow>=2.1.0
+  - apache-airflow>=2.3.0
 
 integrations:
   - integration-name: Kubernetes
diff --git a/scripts/ci/constraints/ci_generate_constraints.sh b/scripts/ci/constraints/ci_generate_constraints.sh
index 7e1cefa..fd9b8dd 100755
--- a/scripts/ci/constraints/ci_generate_constraints.sh
+++ b/scripts/ci/constraints/ci_generate_constraints.sh
@@ -28,6 +28,4 @@ shift
 
 build_images::prepare_ci_build
 
-build_images::rebuild_ci_image_if_needed_with_group
-
 runs::run_generate_constraints
diff --git a/scripts/ci/constraints/ci_generate_constraints.sh b/scripts/ci/provider_packages/ci_make_providers_2_1_compliant.sh
similarity index 69%
copy from scripts/ci/constraints/ci_generate_constraints.sh
copy to scripts/ci/provider_packages/ci_make_providers_2_1_compliant.sh
index 7e1cefa..f890c2a 100755
--- a/scripts/ci/constraints/ci_generate_constraints.sh
+++ b/scripts/ci/provider_packages/ci_make_providers_2_1_compliant.sh
@@ -15,19 +15,13 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-if [[ $1 == "" ]]; then
-  >&2 echo "Requires python MAJOR/MINOR version as first parameter"
-  exit 1
-fi
-
-export PYTHON_MAJOR_MINOR_VERSION=$1
-shift
-
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-build_images::prepare_ci_build
-
-build_images::rebuild_ci_image_if_needed_with_group
+# Some of our provider sources are not Airflow 2.1 compliant any more
+# We replace them with 2.1 compliant versions from PyPI to run the checks
 
-runs::run_generate_constraints
+cd "${AIRFLOW_SOURCES}" || exit 1
+rm -rvf dist/apache_airflow_providers_cncf_kubernetes* dist/apache_airflow_providers_celery*
+pip download --no-deps --dest dist apache-airflow-providers-cncf-kubernetes==3.0.0 \
+    apache-airflow-providers-celery==2.1.3
diff --git a/scripts/in_container/_in_container_utils.sh b/scripts/in_container/_in_container_utils.sh
index b09d58a..82b2609 100644
--- a/scripts/in_container/_in_container_utils.sh
+++ b/scripts/in_container/_in_container_utils.sh
@@ -280,8 +280,10 @@ function install_all_providers_from_pypi_with_eager_upgrade() {
     # Installing it with Airflow makes sure that the version of package that matches current
     # Airflow requirements will be used.
     # shellcheck disable=SC2086
+    # NOTE! Until we unyank the cncf.kubernetes provider, we explicitly install yanked 3.1.2 version
+    # TODO:(potiuk) REMOVE IT WHEN provider is released
     pip install -e ".[${NO_PROVIDERS_EXTRAS}]" "${packages_to_install[@]}" ${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} \
-        --upgrade --upgrade-strategy eager
+        --upgrade --upgrade-strategy eager apache-airflow-providers-cncf-kubernetes==3.1.2
 
 }
 
diff --git a/setup.py b/setup.py
index 9804e91..b72021d 100644
--- a/setup.py
+++ b/setup.py
@@ -235,7 +235,16 @@ cassandra = [
     'cassandra-driver>=3.13.0',
 ]
 celery = [
-    'celery>=5.2.3',
+    # The Celery is known to introduce problems when upgraded to a MAJOR version. Airflow Core
+    # Uses Celery for CeleryExecutor, and we also know that Kubernetes Python client follows SemVer
+    # (https://docs.celeryq.dev/en/stable/contributing.html?highlight=semver#versions).
+    # This is a crucial component of Airflow, so we should limit it to the next MAJOR version and only
+    # deliberately bump the version when we tested it, and we know it can be bumped.
+    # Bumping this version should also be connected with
+    # limiting minimum airflow version supported in cncf.kubernetes provider, due to the
+    # potential breaking changes in Airflow Core as well (celery is added as extra, so Airflow
+    # core is not hard-limited via install-requirements, only by extra).
+    'celery>=5.2.3,<6',
     'flower>=1.0.0',
 ]
 cgroups = [  # type:ignore
@@ -419,7 +428,15 @@ kerberos = [
 ]
 kubernetes = [
     'cryptography>=2.0.0',
-    'kubernetes>=21.7.0',
+    # The Kubernetes API is known to introduce problems when upgraded to a MAJOR version. Airflow Core
+    # Uses Kubernetes for Kubernetes executor, and we also know that Kubernetes Python client follows SemVer
+    # (https://github.com/kubernetes-client/python#compatibility). This is a crucial component of Airflow
+    # So we should limit it to the next MAJOR version and only deliberately bump the version when we
+    # tested it, and we know it can be bumped. Bumping this version should also be connected with
+    # limiting minimum airflow version supported in cncf.kubernetes provider, due to the
+    # potential breaking changes in Airflow Core as well (kubernetes is added as extra, so Airflow
+    # core is not hard-limited via install-requirements, only by extra).
+    'kubernetes>=21.7.0,<24',
 ]
 kylin = ['kylinpy>=2.6']
 ldap = [
@@ -745,7 +762,7 @@ ADDITIONAL_EXTRAS_REQUIREMENTS: Dict[str, List[str]] = {
 # To airflow core. They do not have separate providers because they do not have any operators/hooks etc.
 CORE_EXTRAS_REQUIREMENTS: Dict[str, List[str]] = {
     'async': async_packages,
-    'celery': celery,  # also has provider, but it extends the core with the Celery executor
+    'celery': celery,  # also has provider, but it extends the core with the CeleryExecutor
     'cgroups': cgroups,
     'cncf.kubernetes': kubernetes,  # also has provider, but it extends the core with the KubernetesExecutor
     'dask': dask,
@@ -1033,17 +1050,29 @@ def replace_extra_requirement_with_provider_packages(extra: str, providers: List
             ['simple-salesforce>=1.0.0', 'tableauserverclient']
 
     So transitively 'salesforce' extra has all the requirements it needs and in case the provider
-    changes it's dependencies, they will transitively change as well.
+    changes its dependencies, they will transitively change as well.
 
     In the constraint mechanism we save both - provider versions and it's dependencies
     version, which means that installation using constraints is repeatable.
 
+    For K8s, Celery which are both "Core executors" and "Providers" we have to
+    add the base dependencies to the core as well - in order to mitigate problems where
+    newer version of provider will have less strict limits. This should be done for both
+    extras and their deprecated aliases. This is not a full protection however, the way
+    extras work, this will not add "hard" limits for Airflow and the user who does not use
+    constraints
+
     :param extra: Name of the extra to add providers to
     :param providers: list of provider ids
     """
-    EXTRAS_REQUIREMENTS[extra] = [
-        get_provider_package_from_package_id(package_name) for package_name in providers
-    ]
+    if extra in ['cncf.kubernetes', 'kubernetes', 'celery']:
+        EXTRAS_REQUIREMENTS[extra].extend(
+            [get_provider_package_from_package_id(package_name) for package_name in providers]
+        )
+    else:
+        EXTRAS_REQUIREMENTS[extra] = [
+            get_provider_package_from_package_id(package_name) for package_name in providers
+        ]
 
 
 def add_provider_packages_to_extra_requirements(extra: str, providers: List[str]) -> None: