You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by as...@apache.org on 2021/04/16 07:47:20 UTC

[airflow] branch v2-0-test updated (20d4c38 -> a46e809)

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

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


 discard 20d4c38  Test!
     new 0e0dc73  Bugfix: ``TypeError`` when Serializing & sorting iterables (#15395)
     new 3e3e450  Don't try to push the python build image when building on release branches (#15394)
     new 3369e2f  Do not remove 'full-tests-needed' when approval missing (#15175)
     new a46e809  Fixes pushing constraints (#15243)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (20d4c38)
            \
             N -- N -- N   refs/heads/v2-0-test (a46e809)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 4 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/actions/label-when-approved-action         |  2 +-
 .github/workflows/ci.yml                           |  1 +
 .../workflows/label_when_reviewed_workflow_run.yml |  1 +
 airflow/serialization/serialized_objects.py        |  9 ++++--
 scripts/ci/constraints/ci_commit_constraints.sh    |  2 +-
 tests/serialization/test_dag_serialization.py      | 36 +++++++++++++++++-----
 6 files changed, 39 insertions(+), 12 deletions(-)

[airflow] 01/04: Bugfix: ``TypeError`` when Serializing & sorting iterables (#15395)

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

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

commit 0e0dc73c0b511598414f8d89e595efd09e5087ad
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Fri Apr 16 03:39:26 2021 +0100

    Bugfix: ``TypeError`` when Serializing & sorting iterables (#15395)
    
    This bug got introduced in #14909. Removed sorting from list and tuple as list & tuples preserve order unlike set.
    
    The following DAG errors with: `TypeError: '<' not supported between instances of 'dict' and 'dict'`
    
    ```python
    from airflow import models
    from airflow.operators.dummy import DummyOperator
    from datetime import datetime, timedelta
    params = {
        "staging_schema": [{"key:":"foo","value":"bar"},
                           {"key:":"this","value":"that"}]
    }
    
    with models.DAG(dag_id='test-dag',
                    start_date=datetime(2019, 2, 14),
                    schedule_interval='30 13 * * *',
                    catchup=False,
                    max_active_runs=1,
                    params=params
                    ) as dag:
        my_task = DummyOperator(
            task_id='task1'
        )
    ```
    
    Full Error:
    
    ```
      File "/usr/local/lib/python3.7/site-packages/airflow/serialization/serialized_objects.py", line 210, in <dictcomp>
        return cls._encode({str(k): cls._serialize(v) for k, v in var.items()}, type_=DAT.DICT)
      File "/usr/local/lib/python3.7/site-packages/airflow/serialization/serialized_objects.py", line 212, in _serialize
        return sorted(cls._serialize(v) for v in var)
    TypeError: '<' not supported between instances of 'dict' and 'dict'
    During handling of the above exception, another exception occurred:
    ...
    ```
    
    This is because `sorted()` does not work with dict as it can't compare. Removed sorting from list & tuples which fixes it.
    It also fails when we have set with multiple types.
    
    (cherry picked from commit d1150403a35c497a774a4ffbb1ca4546c532dc81)
---
 airflow/serialization/serialized_objects.py   |  9 ++++---
 tests/serialization/test_dag_serialization.py | 36 +++++++++++++++++++++------
 2 files changed, 35 insertions(+), 10 deletions(-)

diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py
index a890cd1..b6cfdf2 100644
--- a/airflow/serialization/serialized_objects.py
+++ b/airflow/serialization/serialized_objects.py
@@ -214,7 +214,7 @@ class BaseSerialization:
         elif isinstance(var, dict):
             return cls._encode({str(k): cls._serialize(v) for k, v in var.items()}, type_=DAT.DICT)
         elif isinstance(var, list):
-            return sorted(cls._serialize(v) for v in var)
+            return [cls._serialize(v) for v in var]
         elif HAS_KUBERNETES and isinstance(var, k8s.V1Pod):
             json_pod = PodGenerator.serialize_pod(var)
             return cls._encode(json_pod, type_=DAT.POD)
@@ -240,10 +240,13 @@ class BaseSerialization:
             return str(get_python_source(var))
         elif isinstance(var, set):
             # FIXME: casts set to list in customized serialization in future.
-            return cls._encode(sorted(cls._serialize(v) for v in var), type_=DAT.SET)
+            try:
+                return cls._encode(sorted(cls._serialize(v) for v in var), type_=DAT.SET)
+            except TypeError:
+                return cls._encode([cls._serialize(v) for v in var], type_=DAT.SET)
         elif isinstance(var, tuple):
             # FIXME: casts tuple to list in customized serialization in future.
-            return cls._encode(sorted(cls._serialize(v) for v in var), type_=DAT.TUPLE)
+            return cls._encode([cls._serialize(v) for v in var], type_=DAT.TUPLE)
         elif isinstance(var, TaskGroup):
             return SerializedTaskGroup.serialize_task_group(var)
         else:
diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py
index e447751..895f2cf 100644
--- a/tests/serialization/test_dag_serialization.py
+++ b/tests/serialization/test_dag_serialization.py
@@ -357,10 +357,9 @@ class TestStringifiedDAGs(unittest.TestCase):
             "_task_group",
         }
         for field in fields_to_check:
-            dag_field = getattr(dag, field)
-            if isinstance(dag_field, list):
-                dag_field = sorted(dag_field)
-            assert getattr(serialized_dag, field) == dag_field, f'{dag.dag_id}.{field} does not match'
+            assert getattr(serialized_dag, field) == getattr(
+                dag, field
+            ), f'{dag.dag_id}.{field} does not match'
 
         if dag.default_args:
             for k, v in dag.default_args.items():
@@ -1041,7 +1040,7 @@ class TestStringifiedDAGs(unittest.TestCase):
         [
             (
                 ['task_1', 'task_5', 'task_2', 'task_4'],
-                ['task_1', 'task_2', 'task_4', 'task_5'],
+                ['task_1', 'task_5', 'task_2', 'task_4'],
             ),
             (
                 {'task_1', 'task_5', 'task_2', 'task_4'},
@@ -1049,16 +1048,39 @@ class TestStringifiedDAGs(unittest.TestCase):
             ),
             (
                 ('task_1', 'task_5', 'task_2', 'task_4'),
-                ['task_1', 'task_2', 'task_4', 'task_5'],
+                ['task_1', 'task_5', 'task_2', 'task_4'],
+            ),
+            (
+                {
+                    "staging_schema": [
+                        {"key:": "foo", "value": "bar"},
+                        {"key:": "this", "value": "that"},
+                        "test_conf",
+                    ]
+                },
+                {
+                    "staging_schema": [
+                        {"__type": "dict", "__var": {"key:": "foo", "value": "bar"}},
+                        {
+                            "__type": "dict",
+                            "__var": {"key:": "this", "value": "that"},
+                        },
+                        "test_conf",
+                    ]
+                },
             ),
             (
                 {"task3": "test3", "task2": "test2", "task1": "test1"},
                 {"task1": "test1", "task2": "test2", "task3": "test3"},
             ),
+            (
+                ('task_1', 'task_5', 'task_2', 3, ["x", "y"]),
+                ['task_1', 'task_5', 'task_2', 3, ["x", "y"]],
+            ),
         ]
     )
     def test_serialized_objects_are_sorted(self, object_to_serialized, expected_output):
-        """Test Serialized Lists, Sets and Tuples are sorted"""
+        """Test Serialized Sets are sorted while list and tuple preserve order"""
         serialized_obj = SerializedDAG._serialize(object_to_serialized)
         if isinstance(serialized_obj, dict) and "__type" in serialized_obj:
             serialized_obj = serialized_obj["__var"]

[airflow] 02/04: Don't try to push the python build image when building on release branches (#15394)

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

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

commit 3e3e4507e4b3aed9a1d3a95035ce6d2937279a09
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Fri Apr 16 07:57:16 2021 +0100

    Don't try to push the python build image when building on release branches (#15394)
    
    They use the same python image as master (as already mentioned in the
    comments in ci_prepare_prod_image_on_ci.sh) so we don't want to try
    and push the python image when we aren't building the main branch.
    
    (cherry picked from commit f94effeab1a943a7fad19845c10a2b3ea2731fce)
---
 .github/workflows/ci.yml                         | 13 +++++++++++++
 scripts/ci/images/ci_prepare_prod_image_on_ci.sh | 10 ++++++----
 scripts/ci/libraries/_build_images.sh            |  1 -
 scripts/ci/libraries/_push_pull_remove_images.sh |  5 +++--
 4 files changed, 22 insertions(+), 7 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index edd499c..11cf9c1 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -1053,6 +1053,16 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           python-version: ${{ env.PYTHON_MAJOR_MINOR_VERSION }}
       - name: "Free space"
         run: ./scripts/ci/tools/ci_free_space_on_ci.sh
+      - name: Set push-python-image
+        id: push-python-image
+        run: |
+          if [[ "${REF}" == 'refs/head/master' || "${REF}" == 'refs/head/main' ]]; then
+              echo "::set-output name=wanted::true"
+          else
+              echo "::set-output name=wanted::false"
+          fi
+        env:
+          REF: ${{ github.ref }}
       - name:
           "Prepare PROD image ${{env.PYTHON_MAJOR_MINOR_VERSION}}:${{ env.GITHUB_REGISTRY_PULL_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_prepare_prod_image_on_ci.sh
@@ -1060,8 +1070,11 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           # Since we are going to push both final image and build image segment, we need to pull the
           # build image, in case we are pulling from registry rather than building.
           WAIT_FOR_PROD_BUILD_IMAGE: "true"
+          WAIT_FOR_PYTHON_BASE_IMAGE: ${{ steps.push-python-image.outputs.wanted}}
       - name: "Push PROD images ${{ matrix.python-version }}:${{ env.GITHUB_REGISTRY_PUSH_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_push_production_images.sh
+        env:
+          PUSH_PYTHON_BASE_IMAGE: ${{ steps.push-python-image.outputs.wanted}}
 
   push-ci-images-to-github-registry:
     timeout-minutes: 10
diff --git a/scripts/ci/images/ci_prepare_prod_image_on_ci.sh b/scripts/ci/images/ci_prepare_prod_image_on_ci.sh
index 52ec2c6..9b088e1 100755
--- a/scripts/ci/images/ci_prepare_prod_image_on_ci.sh
+++ b/scripts/ci/images/ci_prepare_prod_image_on_ci.sh
@@ -41,10 +41,12 @@ function build_prod_images_on_ci() {
             python_tag_suffix="-${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
         fi
 
-        # first we pull base python image. We will need it to re-push it after master build
-        # Becoming the new "latest" image for other builds
-        build_images::wait_for_image_tag "${GITHUB_REGISTRY_PYTHON_BASE_IMAGE}" \
-            "${python_tag_suffix}" "${AIRFLOW_PYTHON_BASE_IMAGE}"
+        if [[ "${WAIT_FOR_PYTHON_BASE_IMAGE=}" == "true" ]]; then
+            # first we pull base python image. We will need it to re-push it after master build
+            # Becoming the new "latest" image for other builds
+            build_images::wait_for_image_tag "${GITHUB_REGISTRY_PYTHON_BASE_IMAGE}" \
+                "${python_tag_suffix}" "${AIRFLOW_PYTHON_BASE_IMAGE}"
+        fi
 
         # And then the actual image
         build_images::wait_for_image_tag "${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}" \
diff --git a/scripts/ci/libraries/_build_images.sh b/scripts/ci/libraries/_build_images.sh
index edf9e29..da015b9 100644
--- a/scripts/ci/libraries/_build_images.sh
+++ b/scripts/ci/libraries/_build_images.sh
@@ -453,7 +453,6 @@ function build_images::get_docker_image_names() {
     export GITHUB_REGISTRY_PYTHON_BASE_IMAGE="${image_name}${image_separator}python${GITHUB_REGISTRY_IMAGE_SUFFIX}:${PYTHON_BASE_IMAGE_VERSION}-slim-buster"
 
     export GITHUB_REGISTRY_AIRFLOW_CI_IMAGE="${image_name}${image_separator}${AIRFLOW_CI_BASE_TAG}${GITHUB_REGISTRY_IMAGE_SUFFIX}"
-    export GITHUB_REGISTRY_PYTHON_BASE_IMAGE="${image_name}${image_separator}python${GITHUB_REGISTRY_IMAGE_SUFFIX}:${PYTHON_BASE_IMAGE_VERSION}-slim-buster"
 }
 
 # If GitHub Registry is used, login to the registry using GITHUB_USERNAME and
diff --git a/scripts/ci/libraries/_push_pull_remove_images.sh b/scripts/ci/libraries/_push_pull_remove_images.sh
index b0723e4..5b16240 100644
--- a/scripts/ci/libraries/_push_pull_remove_images.sh
+++ b/scripts/ci/libraries/_push_pull_remove_images.sh
@@ -222,7 +222,9 @@ function push_pull_remove_images::push_python_image_to_github() {
 
 # Pushes Ci images and their tags to registry in GitHub
 function push_pull_remove_images::push_ci_images_to_github() {
-    push_pull_remove_images::push_python_image_to_github
+    if [[ "${PUSH_PYTHON_BASE_IMAGE=}" != "false" ]]; then
+        push_pull_remove_images::push_python_image_to_github
+    fi
     AIRFLOW_CI_TAGGED_IMAGE="${GITHUB_REGISTRY_AIRFLOW_CI_IMAGE}:${GITHUB_REGISTRY_PUSH_IMAGE_TAG}"
     docker_v tag "${AIRFLOW_CI_IMAGE}" "${AIRFLOW_CI_TAGGED_IMAGE}"
     push_pull_remove_images::push_image_with_retries "${AIRFLOW_CI_TAGGED_IMAGE}"
@@ -263,7 +265,6 @@ function push_pull_remove_images::push_prod_images_to_dockerhub () {
 #     "${GITHUB_RUN_ID}" - in case of pull-request triggered 'workflow_run' builds
 #     "latest"           - in case of push builds
 function push_pull_remove_images::push_prod_images_to_github () {
-    push_pull_remove_images::push_python_image_to_github
     AIRFLOW_PROD_TAGGED_IMAGE="${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}:${GITHUB_REGISTRY_PUSH_IMAGE_TAG}"
     docker_v tag "${AIRFLOW_PROD_IMAGE}" "${AIRFLOW_PROD_TAGGED_IMAGE}"
     push_pull_remove_images::push_image_with_retries "${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}:${GITHUB_REGISTRY_PUSH_IMAGE_TAG}"

[airflow] 04/04: Fixes pushing constraints (#15243)

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

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

commit a46e809731241caa7dba5e983e8677ca7e539e79
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Wed Apr 7 12:22:21 2021 +0200

    Fixes pushing constraints (#15243)
    
    Afer merging the constraints, the 'recursive' mode was not added
    to checkout resulting with non-checked out github push action.
    
    This commit fixes it and adds color to diff output in commit
    to better show differences when pushing.
    
    (cherry picked from commit 6b78394617c7e699dda1acf42e36161d2fc29925)
---
 .github/workflows/ci.yml                        | 1 +
 scripts/ci/constraints/ci_commit_constraints.sh | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 11cf9c1..6311b40 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -1146,6 +1146,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         uses: actions/checkout@v2
         with:
           persist-credentials: false
+          submodules: recursive
       - name: "Setup python"
         uses: actions/setup-python@v2
         with:
diff --git a/scripts/ci/constraints/ci_commit_constraints.sh b/scripts/ci/constraints/ci_commit_constraints.sh
index c3a7521..7eda70f 100755
--- a/scripts/ci/constraints/ci_commit_constraints.sh
+++ b/scripts/ci/constraints/ci_commit_constraints.sh
@@ -22,7 +22,7 @@ cp -v ./files/constraints-*/constraints*.txt repo/
 cd repo || exit 1
 git config --local user.email "dev@airflow.apache.org"
 git config --local user.name "Automated GitHub Actions commit"
-git diff --exit-code || git commit --all --message "Updating constraints. Build id:${CI_BUILD_ID}
+git diff --color --exit-code || git commit --all --message "Updating constraints. Build id:${CI_BUILD_ID}
 
 This update in constraints is automatically committed by the CI 'constraints-push' step based on
 HEAD of '${CI_REF}' in '${CI_TARGET_REPO}'

[airflow] 03/04: Do not remove 'full-tests-needed' when approval missing (#15175)

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

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

commit 3369e2f6d275083f5b1b01df9d3299104f9340b8
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Wed Apr 14 13:00:22 2021 +0200

    Do not remove 'full-tests-needed' when approval missing (#15175)
    
    (cherry picked from commit a0b217ae3de0a180e746e1e2238ede795b47fb23)
---
 .github/actions/label-when-approved-action             | 2 +-
 .github/workflows/label_when_reviewed_workflow_run.yml | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/.github/actions/label-when-approved-action b/.github/actions/label-when-approved-action
index 4c5190f..0058d00 160000
--- a/.github/actions/label-when-approved-action
+++ b/.github/actions/label-when-approved-action
@@ -1 +1 @@
-Subproject commit 4c5190fec5661e98d83f50bbd4ef9ebb48bd1194
+Subproject commit 0058d0094da27e116fad6e0da516ebe1107f26de
diff --git a/.github/workflows/label_when_reviewed_workflow_run.yml b/.github/workflows/label_when_reviewed_workflow_run.yml
index 1ed50dd..59bde48 100644
--- a/.github/workflows/label_when_reviewed_workflow_run.yml
+++ b/.github/workflows/label_when_reviewed_workflow_run.yml
@@ -102,6 +102,7 @@ jobs:
           token: ${{ secrets.GITHUB_TOKEN }}
           label: 'full tests needed'
           require_committers_approval: 'true'
+          remove_label_when_approval_missing: 'false'
           pullRequestNumber: ${{ steps.source-run-info.outputs.pullRequestNumber }}
           comment: >
             The PR most likely needs to run full matrix of tests because it modifies parts of the core