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 2021/09/09 19:01:02 UTC

[airflow] branch v2-1-test updated (2690c79 -> ae06868)

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

potiuk pushed a change to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git.


    from 2690c79  Improve the description of how to handle dynamic task generation (#17963)
     new bc5e75f  Reapply "Build CI images for the merge result of a PR, not the tip of the PR (#18060)" (#18086)
     new ecae2ae  Fix building documentation broken by upgrade of dnspython (#18046)
     new 92899ca  Add Python2 to installed packages (#18037)
     new 6c2ae7d  Fix constraints generation scripts. (#18094)
     new fada02f  Eager upgrade for Airflow 2.1. should now include celery 4 limit
     new ae06868  Move instriuctions of constraint/image refreshing to dev

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .github/workflows/build-images.yml                 | 41 +++++-----
 CONTRIBUTING.rst                                   | 39 ---------
 Dockerfile                                         |  6 +-
 Dockerfile.ci                                      |  7 +-
 dev/REFRESHING_CI_CACHE.md                         | 94 ++++++++++++++++++++++
 .../refresh_images.sh                              | 27 ++++---
 docs/exts/exampleinclude.py                        |  6 ++
 scripts/ci/docker-compose/_docker.env              |  1 +
 scripts/ci/libraries/_build_images.sh              | 68 +++++++++-------
 scripts/ci/libraries/_initialization.sh            | 11 ---
 scripts/in_container/run_generate_constraints.sh   |  7 +-
 setup.py                                           |  6 ++
 12 files changed, 197 insertions(+), 116 deletions(-)
 create mode 100644 dev/REFRESHING_CI_CACHE.md
 copy scripts/ci/constraints/ci_generate_constraints.sh => dev/refresh_images.sh (58%)

[airflow] 01/06: Reapply "Build CI images for the merge result of a PR, not the tip of the PR (#18060)" (#18086)

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit bc5e75ffcd722709632264d2b3d5b64cc086bf34
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Wed Sep 8 14:27:50 2021 +0100

    Reapply "Build CI images for the merge result of a PR, not the tip of the PR (#18060)" (#18086)
    
    This reverts commit 0dba2e0d644ab0bd2512144231b56463218a3b74.
    
    Revert "Revert "Build CI images for the merge result of a PR, not the tip of the PR (#18060)" (#18063)" (#18086)
    
    (cherry picked from commit 9496235959f6ffaacb69ed391a6ddb82214e9132)
---
 .github/workflows/build-images.yml | 41 +++++++++++++++++++-------------------
 1 file changed, 21 insertions(+), 20 deletions(-)

diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml
index 729d13f..062f34c 100644
--- a/.github/workflows/build-images.yml
+++ b/.github/workflows/build-images.yml
@@ -46,8 +46,8 @@ env:
   GITHUB_REGISTRY_PULL_IMAGE_TAG: "latest"
   GITHUB_REGISTRY_WAIT_FOR_IMAGE: "false"
   INSTALL_PROVIDERS_FROM_SOURCES: "true"
-  TARGET_COMMIT_SHA: ${{ github.event.pull_request.head.sha || github.sha }}
   AIRFLOW_LOGIN_TO_GITHUB_REGISTRY: "true"
+  GITHUB_REGISTRY_PUSH_IMAGE_TAG: ${{ github.event.pull_request.head.sha || github.sha }}
 
 concurrency:
   group: build-${{ github.event.pull_request.number || github.ref }}
@@ -60,7 +60,6 @@ jobs:
     name: "Build Info"
     runs-on: ${{ github.repository == 'apache/airflow' && 'self-hosted' || 'ubuntu-20.04' }}
     env:
-      GITHUB_CONTEXT: ${{ toJson(github) }}
       targetBranch: ${{ github.event.pull_request.base.ref }}
       pullRequestLabels: "${{ toJSON(github.event.pull_request.labels.*.name) }}"
     outputs:
@@ -75,7 +74,18 @@ jobs:
       cacheDirective: ${{ steps.dynamic-outputs.outputs.cacheDirective }}
       targetBranch: ${{ steps.dynamic-outputs.outputs.targetBranch }}
       defaultBranch: ${{ steps.selective-checks.outputs.default-branch }}
+      targetCommitSha: "${{steps.discover-pr-merge-commit.outputs.targetCommitSha ||
+          github.event.pull_request.head.sha ||
+          github.sha
+        }}"
     steps:
+      - name: Discover PR merge commit
+        id: discover-pr-merge-commit
+        run: |
+          TARGET_COMMIT_SHA="$(gh api '${{ github.event.pull_request.url }}' --jq .merge_commit_sha)"
+          echo "TARGET_COMMIT_SHA=$TARGET_COMMIT_SHA" >> $GITHUB_ENV
+          echo "::set-output name=targetCommitSha::${TARGET_COMMIT_SHA}"
+        if: github.event_name == 'pull_request_target'
       # Retrieve it to be able to determine which files has changed in the incoming commit of the PR
       # we checkout the target commit and it's parent to be able to compare them
       - uses: actions/checkout@v2
@@ -92,6 +102,7 @@ jobs:
         run: printenv
         env:
           dynamicOutputs: ${{ toJSON(steps.dynamic-outputs.outputs) }}
+          GITHUB_CONTEXT: ${{ toJson(github) }}
       - name: Selective checks
         id: selective-checks
         env:
@@ -154,13 +165,9 @@ jobs:
         ${{ github.event_name == 'pull_request_target' && 'false' || 'true' }}
       outputs: ${{toJSON(needs.build-info.outputs) }}
     steps:
-      - name: Set envs
-        # Can't refer to "global" env to set a var in a job's env context
-        run: |
-          echo "GITHUB_REGISTRY_PUSH_IMAGE_TAG=${TARGET_COMMIT_SHA}" >> "$GITHUB_ENV"
       - uses: actions/checkout@v2
         with:
-          ref: ${{ env.TARGET_COMMIT_SHA }}
+          ref: ${{ needs.build-info.outputs.targetCommitSha }}
           persist-credentials: false
           submodules: recursive
       - name: "Retrieve DEFAULTS from the _initialization.sh"
@@ -200,9 +207,9 @@ jobs:
           mv "main-airflow/scripts/ci" "scripts"
       - name: "Free space"
         run: ./scripts/ci/tools/free_space.sh
-      - name: "Build CI images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
+      - name: "Build CI images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_prepare_ci_image_on_ci.sh
-      - name: "Push CI images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
+      - name: "Push CI images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_push_ci_images.sh
       # Remove me on 15th of August 2021 after all users had chance to rebase
       - name: "Push Legacy CI images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
@@ -235,15 +242,9 @@ jobs:
       INSTALL_PROVIDERS_FROM_SOURCES: >
         ${{ needs.build-info.outputs.defaultBranch == 'main' && 'true' || 'false' }}
     steps:
-      - name: Set envs
-        # Set pull image tag for CI image build, in order to pull the image pushed
-        # Just a moment ago by build-ci-images job
-        run: |
-          echo "GITHUB_REGISTRY_PUSH_IMAGE_TAG=${TARGET_COMMIT_SHA}" >> "$GITHUB_ENV"
-          echo "GITHUB_REGISTRY_PULL_IMAGE_TAG=${TARGET_COMMIT_SHA}" >> "$GITHUB_ENV"
       - uses: actions/checkout@v2
         with:
-          ref: ${{ env.TARGET_COMMIT_SHA }}
+          ref: ${{ needs.build-info.outputs.targetCommitSha }}
           persist-credentials: false
           submodules: recursive
       - name: "Retrieve DEFAULTS from the _initialization.sh"
@@ -283,18 +284,18 @@ jobs:
           mv "main-airflow/scripts/ci" "scripts"
       - name: "Free space"
         run: ./scripts/ci/tools/free_space.sh
-      - name: "Build CI images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
+      - name: "Build CI images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_prepare_ci_image_on_ci.sh
         # Pull images built in the previous step
         env:
           GITHUB_REGISTRY_WAIT_FOR_IMAGE: "true"
           # Here we are using PULL_IMAGE_TAG set in the environment variables above
-      - name: "Build PROD images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
+      - name: "Build PROD images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_prepare_prod_image_on_ci.sh
         env:
           # GITHUB_REGISTRY_PULL_IMAGE_TAG is overriden to latest in order to build PROD image using "latest"
           GITHUB_REGISTRY_PULL_IMAGE_TAG: "latest"
-      - name: "Push PROD images ${{ matrix.python-version }}:${{ env.TARGET_COMMIT_SHA }}"
+      - name: "Push PROD images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_push_production_images.sh
         env:
           # GITHUB_REGISTRY_PULL_IMAGE_TAG is overriden to latest in order to build PROD image using "latest"
@@ -328,7 +329,7 @@ jobs:
               gh api "/repos/$GITHUB_REPOSITORY/actions/runs?${event_filter}branch=${branch}" \
                   jq -r '
                     .workflow_runs[] |
-                    select(.head_sha == $ENV.TARGET_COMMIT_SHA and .status != "completed") |
+                    select(.head_sha == $ENV.GITHUB_REGISTRY_PUSH_IMAGE_TAG and .status != "completed") |
                     .cancel_url
                   ' \
           ); do

[airflow] 04/06: Fix constraints generation scripts. (#18094)

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 6c2ae7da580cb4d0dc4c868924e6d1b71d9ce9ea
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Thu Sep 9 18:32:08 2021 +0200

    Fix constraints generation scripts. (#18094)
    
    We are now generatnung constraints with better description, and
    we include information about DEFAULT_BRANCH (main/v2-1-test etc.)
    
    The scripts to generate the constraints need to get teh variable
    passed to docker.
    
    Also names of generated files were wrong. The constraints did
    not update the right constraint files.
    
    (cherry picked from commit afd4ba666149b27a4aab7e15c8d76ed1fd4f134a)
---
 scripts/ci/docker-compose/_docker.env            | 1 +
 scripts/in_container/run_generate_constraints.sh | 7 ++++---
 2 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/scripts/ci/docker-compose/_docker.env b/scripts/ci/docker-compose/_docker.env
index 6e8a5c5..a4e0178 100644
--- a/scripts/ci/docker-compose/_docker.env
+++ b/scripts/ci/docker-compose/_docker.env
@@ -26,6 +26,7 @@ CI_TARGET_REPO
 CI_TARGET_BRANCH
 COMMIT_SHA
 DB_RESET
+DEFAULT_BRANCH
 DEFAULT_CONSTRAINTS_BRANCH
 ENABLED_INTEGRATIONS
 ENABLED_SYSTEMS
diff --git a/scripts/in_container/run_generate_constraints.sh b/scripts/in_container/run_generate_constraints.sh
index 402ac85..dbb349c 100755
--- a/scripts/in_container/run_generate_constraints.sh
+++ b/scripts/in_container/run_generate_constraints.sh
@@ -21,14 +21,13 @@
 CONSTRAINTS_DIR="/files/constraints-${PYTHON_MAJOR_MINOR_VERSION}"
 
 LATEST_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/original-${AIRFLOW_CONSTRAINTS}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
-CURRENT_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/${AIRFLOW_CONSTRAINTS}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
-
 mkdir -pv "${CONSTRAINTS_DIR}"
 
 
 if [[ ${GENERATE_CONSTRAINTS_MODE} == "no-providers" ]]; then
     AIRFLOW_CONSTRAINTS="constraints-no-providers"
     NO_PROVIDERS_EXTRAS=$(python -c 'import setup; print(",".join(setup.CORE_EXTRAS_REQUIREMENTS.keys()))')
+    CURRENT_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/${AIRFLOW_CONSTRAINTS}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
     echo
     echo "UnInstall All PIP packages."
     echo
@@ -50,6 +49,7 @@ if [[ ${GENERATE_CONSTRAINTS_MODE} == "no-providers" ]]; then
 EOF
 elif [[ ${GENERATE_CONSTRAINTS_MODE} == "source-providers" ]]; then
     AIRFLOW_CONSTRAINTS="constraints-source-providers"
+    CURRENT_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/${AIRFLOW_CONSTRAINTS}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
     echo
     echo "Providers are already installed from sources."
     echo
@@ -67,11 +67,11 @@ elif [[ ${GENERATE_CONSTRAINTS_MODE} == "source-providers" ]]; then
 EOF
 elif [[ ${GENERATE_CONSTRAINTS_MODE} == "pypi-providers" ]]; then
     AIRFLOW_CONSTRAINTS="constraints"
+    CURRENT_CONSTRAINT_FILE="${CONSTRAINTS_DIR}/${AIRFLOW_CONSTRAINTS}-${PYTHON_MAJOR_MINOR_VERSION}.txt"
     echo
     echo "Install all providers from PyPI so that they are included in the constraints."
     echo
     install_all_providers_from_pypi_with_eager_upgrade
-else
     cat <<EOF >"${CURRENT_CONSTRAINT_FILE}"
 #
 # This constraints file was automatically generated on $(date -u +'%Y-%m-%dT%H:%M:%SZ')
@@ -84,6 +84,7 @@ else
 # "constraints-X.Y.Z" tag to build the production image for that version.
 #
 EOF
+else
     echo
     echo "${COLOR_RED}Error! GENERATE_CONSTRAINTS_MODE has wrong value: '${GENERATE_CONSTRAINTS_MODE}' ${COLOR_RESET}"
     echo

[airflow] 03/06: Add Python2 to installed packages (#18037)

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 92899ca01fa33b1969683fd5cd540f5f90845954
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Sun Sep 5 16:35:27 2021 +0200

    Add Python2 to installed packages (#18037)
    
    As of August 2021, the buster-slim python images, no longer
    contain python2 packages. We still support running Python2 via
    PythonVirtualenvOperator and our tests started to fail when
    we run the tests in `main` - those tests always pull and build
    the images using latest-available buster-slim images.
    
    Our system to prevent PR failures in this case has proven to be
    useful - the main tests failed to succeed so the base images
    we have are still using previous buster-slim images which still
    contain Python 2.
    
    This PR adds python2 to installed packages - on both CI images
    and PROD images. For CI images it is needed to pass tests, for
    PROD images, it is needed for backwards-compatibility.
    
    (cherry picked from commit 6898a2f8a587adacd97beec739ad63c35bfe5537)
---
 Dockerfile    | 4 ++++
 Dockerfile.ci | 5 +++++
 2 files changed, 9 insertions(+)

diff --git a/Dockerfile b/Dockerfile
index 00ae238..554e461 100644
--- a/Dockerfile
+++ b/Dockerfile
@@ -339,6 +339,9 @@ RUN apt-get update \
     && apt-get clean \
     && rm -rf /var/lib/apt/lists/*
 
+# As of August 2021, Debian buster-slim does not include Python2 by default and we need it
+# as we still support running Python2 via PythonVirtualenvOperator
+# TODO: Remove python2 when we stop supporting it
 ARG RUNTIME_APT_DEPS="\
        apt-transport-https \
        apt-utils \
@@ -360,6 +363,7 @@ ARG RUNTIME_APT_DEPS="\
        netcat \
        openssh-client \
        postgresql-client \
+       python2 \
        rsync \
        sasl2-bin \
        sqlite3 \
diff --git a/Dockerfile.ci b/Dockerfile.ci
index 6267770..5be31be 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -57,6 +57,10 @@ ENV DEV_APT_COMMAND=${DEV_APT_COMMAND} \
     ADDITIONAL_DEV_APT_DEPS=${ADDITIONAL_DEV_APT_DEPS} \
     ADDITIONAL_DEV_APT_COMMAND=${ADDITIONAL_DEV_APT_COMMAND}
 
+# As of August 2021, Debian buster-slim does not include Python2 by default and we need it
+# as we still support running Python2 via PythonVirtualenvOperator
+# TODO: Remove python2 when we stop supporting it
+
 # Install basic and additional apt dependencies
 RUN mkdir -pv /usr/share/man/man1 \
     && mkdir -pv /usr/share/man/man7 \
@@ -86,6 +90,7 @@ RUN mkdir -pv /usr/share/man/man1 \
            locales  \
            netcat \
            nodejs \
+           python2 \
            rsync \
            sasl2-bin \
            sudo \

[airflow] 05/06: Eager upgrade for Airflow 2.1. should now include celery 4 limit

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit fada02f564f80a28bb548ec94ac0a0ec9061ea54
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Thu Sep 9 19:43:55 2021 +0200

    Eager upgrade for Airflow 2.1. should now include celery 4 limit
    
    Since we released Celery provider with celery 5, we should
    limit celery to < 5 for Airlfow 2.1  EAGER_UPGRADE limits.
    
    EAGER_UPGRADE limits are only used during constraint generation.
---
 Dockerfile    | 2 +-
 Dockerfile.ci | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/Dockerfile b/Dockerfile
index 554e461..e08a050 100644
--- a/Dockerfile
+++ b/Dockerfile
@@ -237,7 +237,7 @@ ARG INSTALL_FROM_PYPI="true"
 # * certifi<2021.0.0 required to keep snowflake happy
 # * pyjwt<2.0.0: flask-jwt-extended requires it
 # * dill<0.3.3 required by apache-beam
-ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS="pyjwt<2.0.0 dill<0.3.3 certifi<2021.0.0"
+ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS="pyjwt<2.0.0 dill<0.3.3 certifi<2021.0.0 celery<5"
 
 ENV ADDITIONAL_PYTHON_DEPS=${ADDITIONAL_PYTHON_DEPS} \
     INSTALL_FROM_DOCKER_CONTEXT_FILES=${INSTALL_FROM_DOCKER_CONTEXT_FILES} \
diff --git a/Dockerfile.ci b/Dockerfile.ci
index 5be31be..13dd89b 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -271,7 +271,7 @@ ENV AIRFLOW_REPO=${AIRFLOW_REPO}\
 # * lazy-object-proxy<1.5.0: required by astroid
 # * pyjwt<2.0.0: flask-jwt-extended requires it
 # * dill<0.3.3 required by apache-beam
-ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS="lazy-object-proxy<1.5.0 pyjwt<2.0.0 dill<0.3.3 certifi<2021.0.0"
+ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS="lazy-object-proxy<1.5.0 pyjwt<2.0.0 dill<0.3.3 certifi<2021.0.0 celery<5"
 ARG UPGRADE_TO_NEWER_DEPENDENCIES="false"
 ENV EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS=${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS} \
     UPGRADE_TO_NEWER_DEPENDENCIES=${UPGRADE_TO_NEWER_DEPENDENCIES}

[airflow] 02/06: Fix building documentation broken by upgrade of dnspython (#18046)

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit ecae2ae9b21ea890b2d6e4044fb4ad759d4cef22
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Mon Sep 6 20:44:11 2021 +0200

    Fix building documentation broken by upgrade of dnspython (#18046)
    
    The automated upgrade of dependencies in main broken building of
    Airflow documentation in main build.
    
    After a lot of experimentation, It has been narrowed down
    to upgrade of dnspython from 1.16.0 to 2.+ which was brought
    by upgrading eventlet to 0.32.0.
    
    This PR limits the dnspython library to < 2.0.0. An issue
    has been opened:
    https://github.com/rthalley/dnspython/issues/681
    
    (cherry picked from commit 022b4e0bccb91c2ed829adf9e5e6b83dbf352673)
---
 docs/exts/exampleinclude.py | 6 ++++++
 setup.py                    | 6 ++++++
 2 files changed, 12 insertions(+)

diff --git a/docs/exts/exampleinclude.py b/docs/exts/exampleinclude.py
index 8bf395d..097ec7c 100644
--- a/docs/exts/exampleinclude.py
+++ b/docs/exts/exampleinclude.py
@@ -20,6 +20,7 @@
 
 
 """Nice formatted include for examples"""
+import traceback
 from os import path
 
 from docutils import nodes
@@ -150,6 +151,11 @@ def register_source(app, env, modname):
             logger.info(
                 "Module \"%s\" could not be loaded. Full source will not be available. \"%s\"", modname, ex
             )
+            # We cannot use regular warnings or exception methods because those warnings are interpreted
+            # by running python process and converted into "real" warnings, so we need to print the
+            # traceback here at info level
+            tb = traceback.format_exc()
+            logger.info("%s", tb)
             env._viewcode_modules[modname] = False
             return False
 
diff --git a/setup.py b/setup.py
index f35ff83..33cb4f9 100644
--- a/setup.py
+++ b/setup.py
@@ -188,6 +188,12 @@ apache_beam = [
 ]
 asana = ['asana>=0.10', 'cached-property>=1.5.2']
 async_packages = [
+    # DNS Python 2.0.0 and above breaks building documentation on Sphinx. When dnspython 2.0.0 is installed
+    # building documentation fails with trying to import google packages with
+    # TypeError("unsupported operand type(s) for +: 'SSL_VERIFY_PEER' and
+    # 'SSL_VERIFY_FAIL_IF_NO_PEER_CERT'")
+    # The issue is opened for it https://github.com/rthalley/dnspython/issues/681
+    'dnspython<2.0.0',
     'eventlet>= 0.9.7',
     'gevent>=0.13',
     'greenlet>=0.4.9',

[airflow] 06/06: Move instriuctions of constraint/image refreshing to dev

Posted by po...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch v2-1-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit ae068686b6a07d7214b53c5ac6367caf4c1980bc
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Mon Aug 23 11:35:44 2021 +0200

    Move instriuctions of constraint/image refreshing to dev
    
    When we have a prolonged issue with flaky tests or Github runners
    instabilities, our automated constraint and image refresh might
    not work, so we might need to manually refresh the constraints
    and images. Documentation about that was in CONTRIBUTING.rst
    but it is more appriate to keep it in ``dev`` as it only applies
    to committers.
    
    Also during testing the parallell refresh without delays an error
    was discovered  which prevented parallell check of random image
    hash during the build. This has been fixed and parallell
    image cache building should work flawlessly now.
    
    (cherry picked from commit 36c5fd3df9b271702e1dd2d73c579de3f3bd5fc0)
---
 CONTRIBUTING.rst                        | 39 --------------
 dev/REFRESHING_CI_CACHE.md              | 94 +++++++++++++++++++++++++++++++++
 dev/refresh_images.sh                   | 38 +++++++++++++
 scripts/ci/libraries/_build_images.sh   | 68 +++++++++++++-----------
 scripts/ci/libraries/_initialization.sh | 11 ----
 5 files changed, 170 insertions(+), 80 deletions(-)

diff --git a/CONTRIBUTING.rst b/CONTRIBUTING.rst
index be807f4..577f925 100644
--- a/CONTRIBUTING.rst
+++ b/CONTRIBUTING.rst
@@ -860,45 +860,6 @@ The ``constraints-<PYTHON_MAJOR_MINOR_VERSION>.txt`` and ``constraints-no-provid
 will be automatically regenerated by CI job every time after the ``setup.py`` is updated and pushed
 if the tests are successful.
 
-Manually generating constraint files
-------------------------------------
-
-The constraint files are generated automatically by the CI job. Sometimes however it is needed to regenerate
-them manually (committers only). For example when main build did not succeed for quite some time).
-This can be done by running this (it utilizes parallel preparation of the constraints):
-
-.. code-block:: bash
-
-    export CURRENT_PYTHON_MAJOR_MINOR_VERSIONS_AS_STRING="3.6 3.7 3.8 3.9"
-    for python_version in $(echo "${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS_AS_STRING}")
-    do
-      ./breeze build-image --upgrade-to-newer-dependencies --python ${python_version} --build-cache-local
-      ./breeze build-image --upgrade-to-newer-dependencies --python ${python_version} --build-cache-local
-      ./breeze build-image --upgrade-to-newer-dependencies --python ${python_version} --build-cache-local
-    done
-
-    GENERATE_CONSTRAINTS_MODE="pypi-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
-    GENERATE_CONSTRAINTS_MODE="source-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
-    GENERATE_CONSTRAINTS_MODE="no-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
-
-    AIRFLOW_SOURCES=$(pwd)
-
-
-The constraints will be generated in "files/constraints-PYTHON_VERSION/constraints-*.txt files. You need to
-checkout the right 'constraints-' branch in a separate repository and then you can copy, commit and push the
-generated files:
-
-.. code-block:: bash
-
-    cd <AIRFLOW_WITH_CONSTRAINT_main_DIRECTORY>
-    git pull
-    cp ${AIRFLOW_SOURCES}/files/constraints-*/constraints*.txt .
-    git diff
-    git add .
-    git commit -m "Your commit message here" --no-verify
-    git push
-
-
 Documentation
 =============
 
diff --git a/dev/REFRESHING_CI_CACHE.md b/dev/REFRESHING_CI_CACHE.md
new file mode 100644
index 0000000..c5a27ee
--- /dev/null
+++ b/dev/REFRESHING_CI_CACHE.md
@@ -0,0 +1,94 @@
+<!--
+ 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.
+-->
+
+<!-- START doctoc generated TOC please keep comment here to allow auto update -->
+<!-- DON'T EDIT THIS SECTION, INSTEAD RE-RUN doctoc TO UPDATE -->
+**Table of Contents**  *generated with [DocToc](https://github.com/thlorenz/doctoc)*
+
+- [Automated cache refreshing in CI](#automated-cache-refreshing-in-ci)
+- [Manually generating constraint files](#manually-generating-constraint-files)
+- [Manually refreshing the images](#manually-refreshing-the-images)
+
+<!-- END doctoc generated TOC please keep comment here to allow auto update -->
+
+# Automated cache refreshing in CI
+
+Our [CI system](../CI.rst) is build in the way that it self-maintains. Regular scheduled builds and
+merges to `main` branch have separate maintenance step that take care about refreshing the cache that is
+used to speed up our builds and to speed up rebuilding of [Breeze](../BREEZE.rst) images for development
+purpose. This is all happening automatically, usually:
+
+* The latest [constraints](../COMMITTERS.rst#pinned-constraint-files) are pushed to appropriate branch
+  after all tests succeeded in `main` merge or in `scheduled` build
+
+* The [images](../IMAGES.rst) in `ghcr.io` registry are refreshed after every successful merge to `main`
+  or `scheduled` build and after pushing the constraints, this means that the latest image cache uses
+  also the latest tested constraints
+
+Sometimes however, when we have prolonged period of fighting with flakiness of GitHub Actions runners or our
+tests, the refresh might not be triggered - because tests will not succeed for some time. In this case
+manual refresh might be needed.
+
+# Manually generating constraint files
+
+```bash
+export CURRENT_PYTHON_MAJOR_MINOR_VERSIONS_AS_STRING="3.6 3.7 3.8 3.9"
+for python_version in $(echo "${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS_AS_STRING}")
+do
+  ./breeze build-image --upgrade-to-newer-dependencies --python ${python_version} --build-cache-local
+done
+
+GENERATE_CONSTRAINTS_MODE="pypi-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
+GENERATE_CONSTRAINTS_MODE="source-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
+GENERATE_CONSTRAINTS_MODE="no-providers" ./scripts/ci/constraints/ci_generate_all_constraints.sh
+
+AIRFLOW_SOURCES=$(pwd)
+```
+
+The constraints will be generated in `files/constraints-PYTHON_VERSION/constraints-*.txt` files. You need to
+check out the right 'constraints-' branch in a separate repository, and then you can copy, commit and push the
+generated files:
+
+```bash
+cd <AIRFLOW_WITH_CONSTRAINTS-MAIN_DIRECTORY>
+git pull
+cp ${AIRFLOW_SOURCES}/files/constraints-*/constraints*.txt .
+git diff
+git add .
+git commit -m "Your commit message here" --no-verify
+git push
+```
+
+# Manually refreshing the images
+
+The images can be rebuilt and refreshed after the constraints are pushed. Refreshing image for particular
+python version is a simple as running the [refresh_images.sh](refresh_images.sh) script with pyhon version
+as parameter:
+
+```bash
+./dev/refresh_images.sh 3.9
+```
+
+If you have fast network and powerful computer, you can refresh the images in parallel running the
+[refresh_images.sh](refresh_images.sh) with all python versions. You might do it with `tmux` manually
+or with gnu parallel:
+
+```bash
+parallel -j 4 --linebuffer --tagstring '{}' ./dev/refresh_images.sh ::: 3.6 3.7 3.8 3.9
+```
diff --git a/dev/refresh_images.sh b/dev/refresh_images.sh
new file mode 100755
index 0000000..38e283a
--- /dev/null
+++ b/dev/refresh_images.sh
@@ -0,0 +1,38 @@
+#!/bin/bash
+# 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.
+
+set -euo pipefail
+rm -rf docker-context-files/*.whl
+rm -rf docker-context-files/*.tgz
+export FORCE_ANSWER_TO_QUESTIONS="true"
+export CI="true"
+
+if [[ $1 == "" ]]; then
+  echo
+  echo ERROR! Please specify python version as parameter
+  echo
+  exit 1
+fi
+
+python_version=$1
+
+./breeze build-image --python "${python_version}" --build-cache-pulled  --check-if-base-python-image-updated --verbose
+./breeze build-image --python "${python_version}" --build-cache-pulled  --production-image --verbose
+
+./breeze push-image --python "${python_version}"
+./breeze push-image --production-image --python "${python_version}"
diff --git a/scripts/ci/libraries/_build_images.sh b/scripts/ci/libraries/_build_images.sh
index 3ce020b..88169a0 100644
--- a/scripts/ci/libraries/_build_images.sh
+++ b/scripts/ci/libraries/_build_images.sh
@@ -232,16 +232,13 @@ function build_images::check_for_docker_context_files() {
     fi
 }
 
-# Builds local image manifest
-# It contains only one .json file - result of docker inspect - describing the image
-# We cannot use docker registry APIs as they are available only with authorisation
-# But this image can be pulled without authentication
+# Builds local image manifest. It contains only one random file generated during Docker.ci build
 function build_images::build_ci_image_manifest() {
     docker_v build \
         --tag="${AIRFLOW_CI_LOCAL_MANIFEST_IMAGE}" \
         -f- . <<EOF
 FROM scratch
-COPY "manifests/local-build-cache-hash" /build-cache-hash
+COPY "manifests/local-build-cache-hash-${PYTHON_MAJOR_MINOR_VERSION}" /build-cache-hash
 LABEL org.opencontainers.image.source="https://github.com/${GITHUB_REPOSITORY}"
 CMD ""
 EOF
@@ -249,9 +246,13 @@ EOF
 
 #
 # Retrieves information about build cache hash random file from the local image
+# The random file is generated during the build and is best indicator whether your local CI image
+# has been built using the same pulled image as the remote one
 #
 function build_images::get_local_build_cache_hash() {
     set +e
+    local local_image_build_cache_file
+    local_image_build_cache_file="${AIRFLOW_SOURCES}/manifests/local-build-cache-hash-${PYTHON_MAJOR_MINOR_VERSION}"
     # Remove the container just in case
     docker_v rm --force "local-airflow-ci-container" 2>/dev/null >/dev/null
     if ! docker_v inspect "${AIRFLOW_CI_IMAGE}" 2>/dev/null >/dev/null; then
@@ -260,34 +261,37 @@ function build_images::get_local_build_cache_hash() {
         verbosity::print_info
         LOCAL_MANIFEST_IMAGE_UNAVAILABLE="true"
         export LOCAL_MANIFEST_IMAGE_UNAVAILABLE
-        touch "${LOCAL_IMAGE_BUILD_CACHE_HASH_FILE}"
+        touch "${local_image_build_cache_file}"
         set -e
         return
 
     fi
     docker_v create --name "local-airflow-ci-container" "${AIRFLOW_CI_IMAGE}" 2>/dev/null
     docker_v cp "local-airflow-ci-container:/build-cache-hash" \
-        "${LOCAL_IMAGE_BUILD_CACHE_HASH_FILE}" 2>/dev/null ||
-        touch "${LOCAL_IMAGE_BUILD_CACHE_HASH_FILE}"
+        "${local_image_build_cache_file}" 2>/dev/null ||
+        touch "${local_image_build_cache_file}"
     set -e
     verbosity::print_info
-    verbosity::print_info "Local build cache hash: '$(cat "${LOCAL_IMAGE_BUILD_CACHE_HASH_FILE}")'"
+    verbosity::print_info "Local build cache hash: '$(cat "${local_image_build_cache_file}")'"
     verbosity::print_info
 }
 
 # Retrieves information about the build cache hash random file from the remote image.
-# We actually use manifest image for that, which is a really, really small image to pull!
-# The problem is that inspecting information about remote image cannot be done easily with existing APIs
-# of Dockerhub because they require additional authentication even for public images.
-# Therefore instead we are downloading a specially prepared manifest image
-# which is built together with the main image and pushed with it. This special manifest image is prepared
-# during building of the main image and contains single file which is randomly built during the docker
-# build in the right place in the image (right after installing all dependencies of Apache Airflow
-# for the first time). When this random file gets regenerated it means that either base image has
-# changed or some of the earlier layers was modified - which means that it is usually faster to pull
-# that image first and then rebuild it - because this will likely be faster
+# We use manifest image for that, which is a really, really small image to pull!
+# The image is a specially prepared manifest image which is built together with the main image and
+# pushed with it. This special manifest image is prepared during building of the CI image and contains
+# single file which is generated with random content during the docker
+# build in the right step of the image build (right after installing all dependencies of Apache Airflow
+# for the first time).
+# When this random file gets regenerated it means that either base image has changed before that step
+# or some of the earlier layers was modified - which means that it is usually faster to pull
+# that image first and then rebuild it.
 function build_images::get_remote_image_build_cache_hash() {
     set +e
+    local remote_image_container_id_file
+    remote_image_container_id_file="${AIRFLOW_SOURCES}/manifests/remote-airflow-manifest-image-${PYTHON_MAJOR_MINOR_VERSION}"
+    local remote_image_build_cache_file
+    remote_image_build_cache_file="${AIRFLOW_SOURCES}/manifests/remote-build-cache-hash-${PYTHON_MAJOR_MINOR_VERSION}"
     # Pull remote manifest image
     if ! docker_v pull "${AIRFLOW_CI_REMOTE_MANIFEST_IMAGE}" 2>/dev/null >/dev/null; then
         verbosity::print_info
@@ -295,32 +299,36 @@ function build_images::get_remote_image_build_cache_hash() {
         verbosity::print_info
         REMOTE_DOCKER_REGISTRY_UNREACHABLE="true"
         export REMOTE_DOCKER_REGISTRY_UNREACHABLE
-        touch "${REMOTE_IMAGE_BUILD_CACHE_HASH_FILE}"
+        touch "${remote_image_build_cache_file}"
         set -e
         return
     fi
     set -e
-    rm -f "${REMOTE_IMAGE_CONTAINER_ID_FILE}"
+    rm -f "${remote_image_container_id_file}"
     # Create container dump out of the manifest image without actually running it
-    docker_v create --cidfile "${REMOTE_IMAGE_CONTAINER_ID_FILE}" "${AIRFLOW_CI_REMOTE_MANIFEST_IMAGE}"
+    docker_v create --cidfile "${remote_image_container_id_file}" "${AIRFLOW_CI_REMOTE_MANIFEST_IMAGE}"
     # Extract manifest and store it in local file
-    docker_v cp "$(cat "${REMOTE_IMAGE_CONTAINER_ID_FILE}"):/build-cache-hash" \
-        "${REMOTE_IMAGE_BUILD_CACHE_HASH_FILE}"
-    docker_v rm --force "$(cat "${REMOTE_IMAGE_CONTAINER_ID_FILE}")"
-    rm -f "${REMOTE_IMAGE_CONTAINER_ID_FILE}"
+    docker_v cp "$(cat "${remote_image_container_id_file}"):/build-cache-hash" \
+        "${remote_image_build_cache_file}"
+    docker_v rm --force "$(cat "${remote_image_container_id_file}")"
+    rm -f "${remote_image_container_id_file}"
     verbosity::print_info
-    verbosity::print_info "Remote build cache hash: '$(cat "${REMOTE_IMAGE_BUILD_CACHE_HASH_FILE}")'"
+    verbosity::print_info "Remote build cache hash: '$(cat "${remote_image_build_cache_file}")'"
     verbosity::print_info
 }
 
 # Compares layers from both remote and local image and set FORCE_PULL_IMAGES to true in case
-# More than the last NN layers are different.
+# The random has in remote image is different than that in the local image
+# indicating that it is likely faster to pull the image from cache rather than let the
+# image rebuild fully locally
 function build_images::compare_local_and_remote_build_cache_hash() {
     set +e
+    local local_image_build_cache_file
+    local_image_build_cache_file="${AIRFLOW_SOURCES}/manifests/local-build-cache-hash-${PYTHON_MAJOR_MINOR_VERSION}"
     local remote_hash
-    remote_hash=$(cat "${REMOTE_IMAGE_BUILD_CACHE_HASH_FILE}")
+    remote_hash=$(cat "${remote_image_build_cache_file}")
     local local_hash
-    local_hash=$(cat "${LOCAL_IMAGE_BUILD_CACHE_HASH_FILE}")
+    local_hash=$(cat "${local_image_build_cache_file}")
 
     if [[ ${remote_hash} != "${local_hash}" || -z ${local_hash} ]] \
         ; then
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index d392d98..6068880 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -558,12 +558,6 @@ function initialization::initialize_package_variables() {
 }
 
 
-function initialization::initialize_build_image_variables() {
-    REMOTE_IMAGE_CONTAINER_ID_FILE="${AIRFLOW_SOURCES}/manifests/remote-airflow-manifest-image"
-    LOCAL_IMAGE_BUILD_CACHE_HASH_FILE="${AIRFLOW_SOURCES}/manifests/local-build-cache-hash"
-    REMOTE_IMAGE_BUILD_CACHE_HASH_FILE="${AIRFLOW_SOURCES}/manifests/remote-build-cache-hash"
-}
-
 function initialization::set_output_color_variables() {
     COLOR_BLUE=$'\e[34m'
     COLOR_GREEN=$'\e[32m'
@@ -597,7 +591,6 @@ function initialization::initialize_common_environment() {
     initialization::initialize_github_variables
     initialization::initialize_test_variables
     initialization::initialize_package_variables
-    initialization::initialize_build_image_variables
 }
 
 function initialization::set_default_python_version_if_empty() {
@@ -845,10 +838,6 @@ function initialization::make_constants_read_only() {
     readonly BUILT_CI_IMAGE_FLAG_FILE
     readonly INIT_SCRIPT_FILE
 
-    readonly REMOTE_IMAGE_CONTAINER_ID_FILE
-    readonly LOCAL_IMAGE_BUILD_CACHE_HASH_FILE
-    readonly REMOTE_IMAGE_BUILD_CACHE_HASH_FILE
-
     readonly INSTALLED_EXTRAS
     readonly INSTALLED_PROVIDERS