You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/08/17 23:47:58 UTC

[GitHub] [airflow] potiuk opened a new pull request #10368: CI Images are now pre-build and stored in registry

potiuk opened a new pull request #10368:
URL: https://github.com/apache/airflow/pull/10368


   With this change we utilise the latest pull_request_target
   event type from Github Actions and we are building the
   CI image only once (per version) for the entire run.
   
   This safes from 2 to 10 minutes per job (!) depending on
   how much of the Docker image needs to be rebuilt.
   
   It works in the way that the image is built only in the
   build-or-wait step. In case of direct push run or
   scheduled runs, the build-or-wait step builds and pushes
   to the GitHub registry the CI image. In case of the
   pull_request runs, the build-and-wait step waits until
   separate build-ci-image.yml workflow builds and pushes
   the image and it will only move forward once the image
   is ready.
   
   This has numerous advantages:
   
   1) Each job that requires CI image is much faster because
      instead of pulling + rebuilding the image it only pulls
      the image that was build once. This saves around 2 minutes
      per job in regular builds but in case of python patch level
      updates, or adding new requirements it can save up to 10
      minutes per job (!)
   
   2) While the images are buing rebuilt we only block one job waiting
      for all the images. The tests will start running in parallell
      only when all images are ready, so we are not blocking
      other runs from running.
   
   3) Whole run uses THE SAME image. Previously we could have some
      variations because the images were built at different times
      and potentially releases of dependencies in-between several
      jobs could make different jobs in the same run use slightly
      different image. This is not happening any more.
   
   4) Also when we push image to github or dockerhub we push the
      very same image that was built and tested. Previously it could
      happen that the image pushed was slightly different than the
      one that was used for testing (for the same reason)
   
   5) Similar case is with the production images. We are now building
      and pushing consistently the same images accross the board.
   
   6) Documentation building is split into two parallel jobs docs
      building and spell checking - decreases elapsed time for
      the docs build.
   
   7) Last but not least - we keep the history of al the images
      - those images contain SHA of the commit. This means
      that we can simply download and run the image locally to reproduce
      any problem that anyone had in their PR (!). This is super useful
      to be able to help others to test their problems.
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ad-m commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
ad-m commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r471851549



##########
File path: BREEZE.rst
##########
@@ -1135,6 +1167,12 @@ This is the current syntax for  `./breeze <./breeze>`_:
 
         You can also pass '--production-image' flag to build production image rather than CI image.
 
+        For DockerHub pull --dockerhub-user and --dockerhub-repo flags can be used to specify

Review comment:
       ```suggestion
           For DockerHub pull '--dockerhub-user' and '--dockerhub-repo' flags can be used to specify
   ```
   
   See line above. Keep formatting consistent. 

##########
File path: CI.rst
##########
@@ -188,32 +568,104 @@ The issues that gets updated are different for different branches:
 
 * master: `Quarantine tests master <https://github.com/apache/airflow/issues/10118>`_
 * v1-10-stable: `Quarantine tests v1-10-stable <https://github.com/apache/airflow/issues/10127>`_
-* v1-10-test: `Quarantine tests v1-10-test <hhttps://github.com/apache/airflow/issues/10128>`_
-
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Job                       | Description                                                                                                    | Pull Request Run                   | Direct Push/Merge Run           | Scheduled Run                                                        |
-+===========================+================================================================================================================+====================================+=================================+======================================================================+
-| Cancel previous workflow  | Cancels the previously running workflow run if there is one running                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Trigger tests             | Checks if tests should be triggered                                                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Quarantined tests         | Those are tests that are flaky and we need to fix them                                                         | Yes (if tests-triggered)           | Yes (Updates quarantine issue)  | Yes * (updates quarantine issue)                                     |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-
-Cancel other workflow runs workflow
------------------------------------
-
-This workflow is run only on schedule (every 5 minutes) it's only purpose is to cancel other running
-``CI Build`` workflows if important jobs failed in those runs. This is to save runners for other runs
-in case we know that the build will not succeed anyway without some basic fixes to static checks or
-documentation - effectively implementing missing "fail-fast" (on a job level) in Github Actions
-similar to fail-fast in matrix strategy.
-
-The jobs that are considered as "fail-fast" are:
-
-* Static checks
-* Docs
-* Prepare Backport packages
-* Helm tests
-* Build Prod Image
-* TTest OpenAPI client gen
+* v1-10-test: `Quarantine tests v1-10-test <https://github.com/apache/airflow/issues/10128>`_
+
+Force sync master from apache/airflow
+-------------------------------------
+
+This is manually triggered workflow (via GitHub UI manual run) that should only be run in GitHub forks.
+When triggered, it will force-push the "apache/airflow" master to the fork's master. It's the easiest
+way to sync your fork master to the Apache Airflow's one.
+
+Naming conventions for stored images
+====================================
+
+The images produced during the CI builds are stored in the
+`GitHub Registry <https://github.com/apache/airflow/packages>`_
+
+The images are stored with both "latest" tag (for last master push image that passes all the tests as well
+with the tags indicating the origin of the image.
+
+The image names follow the patterns:
+
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| Image        | Name pattern               | Tag for format                 | Comment                                                                                    |
++==============+============================+================================+============================================================================================+
+| Python image | python                     | <X.Y>-slim-buster-<RUN_ID>     | Base python image used by both production and CI image.                                    |
+|              |                            | <X.Y>-slim-buster-<COMMIT_SHA> | Python maintainer release new versions of those image with security fixes every few weeks. |
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| CI image     | <BRANCH>-python<X.Y>-ci    | <RUN_ID>                       | CI image - this is the image used for most of the tests.                                   |

Review comment:
       To be a good citizen, are we planning some form of garbage collector and retention period for these images? We can potentially produce a lot of images.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-677439723


   Wohooooo! 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473519252



##########
File path: scripts/ci/images/ci_prepare_prod_image_on_ci.sh
##########
@@ -18,4 +18,77 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-build_prod_image_on_ci
+# In case GITHUB_REGISTRY_PULL_IMAGE_TAG is different than latest, tries to pull the image indefinitely
+# skips further image checks - assuming that this is the right image
+function wait_for_prod_images_tag {
+    PROD_IMAGE_TO_WAIT_FOR="${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+    PROD_BUILD_IMAGE_TO_WAIT_FOR="${GITHUB_REGISTRY_AIRFLOW_PROD_BUILD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+    echo
+    echo "Waiting for image ${PROD_IMAGE_TO_WAIT_FOR}"
+    echo
+    while true; do
+        docker pull "${PROD_IMAGE_TO_WAIT_FOR}" || true
+        if [[ "$(docker images -q "${PROD_IMAGE_TO_WAIT_FOR}" 2> /dev/null)" == "" ]]; then
+            echo
+            echo "The image ${PROD_IMAGE_TO_WAIT_FOR} is not yet available. Waiting"
+            echo
+            sleep 10
+        else
+            echo
+            echo "The image ${PROD_IMAGE_TO_WAIT_FOR} downloaded."
+            echo "Tagging ${PROD_IMAGE_TO_WAIT_FOR} as ${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}."
+            docker tag  "${PROD_IMAGE_TO_WAIT_FOR}" "${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}"
+            echo "Tagging ${PROD_IMAGE_TO_WAIT_FOR} as ${AIRFLOW_PROD_IMAGE}."
+            docker tag  "${PROD_IMAGE_TO_WAIT_FOR}" "${AIRFLOW_PROD_IMAGE}"
+            echo
+            break
+        fi
+    done
+    echo
+    echo "Waiting for image ${PROD_BUILD_IMAGE_TO_WAIT_FOR}"
+    echo
+    while true; do

Review comment:
       This and the above loop are very similar. Can this code be generalized?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675177393


   Hello everyone. This is a major overhaul of the way how we are utilizing GitHub Actions - something that was enabled by recent features released by GitHub Actions (namely "workflow_run" feature). 
   
   I've been working on it last week and heavily tested it on my fork https://github.com/potiuk/airflow/actions. I hope there will be rather little number of teething problems, but I am already very familiar with how GA work and I will be able to fix any problems quickly. I am also going to watch it once we merge it to make sure it works as expected. 
   
   See commit description for what is achieved by this change. I have just one thing to say - this is my "dream" architecture of the CI builds that I had in mind at the very beginning of my work on Airflow, one that could only be achieved by the most recent changes by GitHub. I really hope this is one of the last fundamental changes in the scripting for CI because I literally run out of ideas what can be improved (just kidding - there are always small things ;).
   
   It has many nice properties but the most important ones:
   
   *  5-12 minutes saved for each Job (Builds of images are done only once not for each job). Not per whole run - but per Job (!). This will help both - increase number of parallell PRs that can be run and decrease the feedback time for each build. There were sometimes much slower builds when python base image was upgraded or Dockerfile changed - this problem will be gone.
   
   * the jobs/runs are fully consistent - all jobs in the same build use exactly the same image prepared only once. 
   
   * full trackability and reproducibility of each run - we keep the images in GitHub registry and you can recreate the exact failed run by running `./breeze --github-image-id <RUN_ID>` or `./breeze --github-image-id <COMMIT_ID>` for merged runs.
   
   * I cleaned up outputs of the job so that they only show relevant information
   
   * I cleaned up initialization code for bash scripts - removed some duplicates and organized it better and I fully documented it - describing the purpose of all options (that was the lat script refactoring I planned)
   
   It's quite a huge change, and I can try to split it into smaller ones (but conceptually it is one big overhaul of the way our CI works) 
   
   When you can start from the workflows at the end of the documentation: https://github.com/PolideaInternal/airflow/blob/prebuild-ci-images-in-github-actions/CI.rst  - I prepared some sequence diagrams of the CI architecture (using mermaid - which is an absolutely cool tool for converting markdownish descriptions of diagrams into really nice diagrams). It explains all the "whys" and also "hows". 
   
   NOTE! For Review/Merge I needed to disable waiting for images, so the speedups are not visible yet - I have to merge it to master in order to  enable the "Build Image" workflows. I also use "master" version of my own Github Cancel Action which I developed for that purpose - I will release it's v2 version and switch to it once we get a few days of the builds working in Airflow.
   
   I really look forward to review comments and merging it eventually. This will help all of contributors and committers to move faster. This is literally completion of 2 years of the "dream" architecture for our CI :).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675196719


   This change is huge. What can we reduce its size? I can see a few changes that we can extract and do as a separate change:
   * merging to ``--github-organisation`` and ``--github-repo`` flags into ``--github-repository`` 
   * building documentation and spell checking separately,.
   * renaming `ci_*..sh` scripts 
   * pylint fixes in ``docs/build_docs.py``
   * renaming variable in ``scripts/ci/in_container/run_prepare_backport_readme.sh``, ``scripts/ci/in_container/run_test_package_import_all_classes.sh``
   * less verbose output from helm,kubectl - scripts/ci/libraries/_verbosity.sh
   * pyllint fixes in ``scripts/tools/list-integrations.py``.
   
   This may seem like a lot of extra work, but I think it's the only way we can be sure about this change and not get a lot of teething problems. Now it is very painful for me to review this change because I have to remember every change and think about the consequences of many changes at once. 
   
   If you like, I will be happy to help you break down this change to make it easier to review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676481211


   > @potiuk I am in favor of the documentation being merged as the next PR because that requires adding integration with memraid.
   
   Fine for me. I will remove it from this PR then.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473518461



##########
File path: scripts/ci/images/ci_prepare_ci_image_on_ci.sh
##########
@@ -18,4 +18,59 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
+# In case GITHUB_REGISTRY_PULL_IMAGE_TAG is different than latest, tries to pull the image indefinitely
+# skips further image checks - assuming that this is the right image
+function wait_for_ci_image_tag {

Review comment:
       This function looks very similar to ``wait_for_prod_images_tag``? Is it on purpose? Is it possible to get rid of this repetitive code?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675177393


   Hello everyone. This is a major overhaul of the way how we are utilizing GitHub Actions - something that was enabled by recent features released by GitHub Actions (namely "workflow_run" feature). 
   
   I've been working on it last week and heavily tested it on my fork https://github.com/potiuk/airflow/actions. I hope there will be rather little number of teething problems, but I am already very familiar with how GA work and I will be able to fix any problems quickly. I am also going to watch it once we merge it to make sure it works as expected. 
   
   See commit description for what is achieved by this change. I have just one thing to say - this is my "dream" architecture of the CI builds that I had in mind at the very beginning of my work on Airflow, one that could only be achieved by the most recent changes by GitHub. I really hope this is one of the last fundamental changes in the scripting for CI because I literally run out of ideas what can be improved (just kidding - there are always small things ;).
   
   It has many nice properties but the most important ones:
   
   *  5-12 minutes saved for each Job (Builds of images are done only once not for each job). Not per whole run - but per Job (!). This will help both - increase number of parallell PRs that can be run and decrease the feedback time for each build. There were sometimes much slower builds when python base image was upgraded or Dockerfile changed - this problem will be gone.
   
   * the jobs/runs are fully consistent - all jobs in the same build use exactly the same image prepared only once. 
   
   * full trackability and reproducibility of each run - we keep the images in GitHub registry and you can recreate the exact failed run by running `./breeze --github-image-id <RUN_ID>` or `./breeze --github-image-id <COMMIT_ID>` for merged runs.
   
   * I cleaned up outputs of the job so that they only show relevant information
   
   * I cleaned up initialization code for bash scripts - removed some duplicates and organized it better and I fully documented it - describing the purpose of all options (that was the lat script refactoring I planned)
   
   It's quite a huge change, and I can try to split it into smaller ones (but conceptually it is one big overhaul of the way our CI works) 
   
   When you can start from the workflows at the end of the documentation: https://github.com/PolideaInternal/airflow/blob/prebuild-ci-images-in-github-actions/CI.rst  - I prepared some sequence diagrams of the CI architecture (using mermaid - which is an absolutely cool tool for converting markdownish descriptions of diagrams into really nice diagrams). It explains all the "whys" and also "hows". 
   
   NOTE! For Review/Merge I needed to disable waiting for images, so the speedups are not visible yet - I have to merge it to master in order to  enable the "Build Image" workflows. I also use "master" version of my own Github Cancel Action which I developed for that purpose - I will release it's v2 version and switch to it once we get a few days of the builds working in Airflow.
   
   I developed https://github.com/potiuk/cancel-workflow-runs new Github Action for "Cancel Workflow Run" that is a swiss-army-knife of Run cancelling and I plan to share it with Apache Beam and other Apache projects that might need it as well.
   
   I really look forward to review comments and merging it eventually. This will help all of contributors and committers to move faster. This is literally completion of 2 years of the "dream" architecture for our CI :).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675196719


   This change is huge. What can we reduce its size? I can see a few changes that we can extract and do as a separate change:
   * merging to ``--github-organisation`` and ``--github-repo`` flags into ``--github-repository`` 
   * building documentation and spell checking separately,.
   * renaming `ci_*..sh` scripts 
   * pylint fixes in ``docs/build_docs.py``
   * renaming variable in ``scripts/ci/in_container/run_prepare_backport_readme.sh``, ``scripts/ci/in_container/run_test_package_import_all_classes.sh``
   * less verbose output from helm,kubectl - scripts/ci/libraries/_verbosity.sh
   * pyllint fixes in ``scripts/tools/list-integrations.py``.
   
   This may seem like a lot of extra work, but I think it's the only way we can be sure about this change and not get a lot of teething problems. Now it is very painful for me to review this change because I have to remember every change and think about the consequences of many changes at once. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-677403199


   Tested that it works with "WAIT_FOR_IMAGES" enabled as well. https://github.com/potiuk/airflow/runs/1006462184?check_suite_focus=true


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473474631



##########
File path: scripts/ci/images/ci_wait_for_all_prod_images.sh
##########
@@ -0,0 +1,43 @@
+#!/usr/bin/env 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.
+
+# This is hook build used by DockerHub. We are also using it
+# on CI to potentially rebuild (and refresh layers that
+# are not cached) Docker images that are used to run CI jobs
+# shellcheck source=scripts/ci/libraries/_script_init.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+
+
+echo
+echo "Waiting for all images to appear: ${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS[*]}"
+echo
+
+echo
+echo "Check if jq is installed"
+echo
+command -v jq
+
+jq --version

Review comment:
       This will just fail if "jq" is not installed, isn't it?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473617761



##########
File path: scripts/ci/images/ci_prepare_ci_image_on_ci.sh
##########
@@ -18,4 +18,59 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
+# In case GITHUB_REGISTRY_PULL_IMAGE_TAG is different than latest, tries to pull the image indefinitely
+# skips further image checks - assuming that this is the right image
+function wait_for_ci_image_tag {

Review comment:
       Will take a look.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676484548


   Heh. I already  removed it  :).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472975291



##########
File path: scripts/ci/libraries/_verbosity.sh
##########
@@ -28,58 +27,87 @@ function check_verbose_setup {
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
 # printed before execution
 function verbose_docker {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
+    if [[ ${VERBOSE:="false"} == "true" && \
+        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
+        ${VERBOSE_COMMANDS:=} != "true" && \
+        # And when generally printing info is disabled
+        ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
+        >&2 echo "docker" "${@}"
+    fi
+    if [[ ${PRINT_INFO_FROM_SCRIPTS} == "false" ]]; then
+        docker "${@}" >>"${OUTPUT_LOG}" 2>&1
+    else
+        docker "${@}" 2>&1 | tee -a "${OUTPUT_LOG}"
+    fi
+    EXIT_CODE="$?"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        # This way in the output log we only see the most recent failed command and what was echoed before
+        rm -f "${OUTPUT_LOG}"
     fi
-    docker "${@}"
+    return "${EXIT_CODE}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all helm commands run will be
 # printed before execution
 function verbose_helm {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "helm" "${@}"
+        >&2 echo "helm" "${@}"
+    fi
+    helm "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    helm "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kubectl commands run will be
 # printed before execution
 function verbose_kubectl {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kubectl" "${@}"
+        >&2 echo "kubectl" "${@}"
+    fi
+    kubectl "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    kubectl "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kind commands run will be
 # printed before execution
 function verbose_kind {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kind" "${@}"
+        >&2 echo "kind" "${@}"
     fi
+    # kind outputs nice output on terminal.
     kind "${@}"
 }
 
-
-# In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
-# printed before execution
-function verbose_docker_hide_output_on_success {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
-    fi
-    docker "${@}" >>"${OUTPUT_LOG}" 2>&1
-}
-
-
 # Prints verbose information in case VERBOSE variable is set
 function print_info() {
-    if [[ ${VERBOSE:="false"} == "true" ]]; then
+    if [[ ${VERBOSE:="false"} == "true" && ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
         echo "$@"
     fi
 }
+
+function set_verbosity() {
+    # whether verbose output should be produced
+    export VERBOSE=${VERBOSE:="false"}
+
+    # whether every bash statement should be printed as they are executed
+    export VERBOSE_COMMANDS=${VERBOSE_COMMANDS:="false"}
+
+    # whether the output from script should be printed at all
+    export PRINT_INFO_FROM_SCRIPTS=${PRINT_INFO_FROM_SCRIPTS:="true"}
+}
+
+set_verbosity
+
+alias docker=verbose_docker
+alias kubectl=verbose_kubectl
+alias helm=verbose_helm
+alias kind=verbose_kind

Review comment:
       Handled it better in #10402 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-690033427


   > That is what was confusing me -- I didn't realise it was possible to trigger actions on "master" but targeting a PR; and my workflow had been to filtering by the branch I was working on, so had filtered out the very build I was looking for.
   
   Yep. this is a very new feature of GA and this is something they will have to add Filter in action on the "triggering PR" or an option "show related workflows"  would be a nice thing to have.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473634888



##########
File path: scripts/ci/mermaid-config.json
##########
@@ -0,0 +1,4 @@
+{

Review comment:
       Indeed :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676399967


   @potiuk I am in favor of the documentation being merged as the next PR because that requires adding integration with memraid.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675936557


   I have  3 more PRs extracted now from this one:
   - [ ] Kubernetes image no embedded #10399 
   - [x] When precommits are run, output is silenced #10390 
   - [ ] Added Documentation for future CI with mermaid sequence diagrams  #10380 
   - [ ] Replaced aliases for common tools with functions #10402 
   
   Once we merge those, it will be much easier for me to see what's left from the original PR and maybe split it even farther into few smaller ones. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-689823453


   Thanks.
   
   > though there is no direct link in the original PR
   
   That is what was confusing me -- I didn't realise it was possible to trigger actions on "master" but targeting a PR; and my workflow had been to filtering by the branch I was working on, so had filtered out the very build I was looking for.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675388464


   > This change is huge. What can we reduce its size? I can see a few changes that we can extract and do as a separate cha
   
   Yep. I am happy to do it. I will try to separate out some of the changes and keep on rebasing this one until the "gist" of the change remains as last commit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675936557


   I have  3 more PRs extracted now from this one:
   - [x] Kubernetes image no embedded #10399 
   - [x] When precommits are run, output is silenced #10390 
   - [ ] Added Documentation for future CI with mermaid sequence diagrams  #10380 
   - [x] Replaced aliases for common tools with functions #10402 
   
   Once we merge those, it will be much easier for me to see what's left from the original PR and maybe split it even farther into few smaller ones. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675183645


   CC: @ad-m 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473633360



##########
File path: scripts/ci/images/ci_prepare_ci_image_on_ci.sh
##########
@@ -18,4 +18,59 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
+# In case GITHUB_REGISTRY_PULL_IMAGE_TAG is different than latest, tries to pull the image indefinitely
+# skips further image checks - assuming that this is the right image
+function wait_for_ci_image_tag {

Review comment:
       Indeed! It's much simpler now!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473524858



##########
File path: scripts/ci/libraries/_initialization.sh
##########
@@ -223,94 +275,190 @@ function initialize_common_environment {
     export CI_BUILD_ID="0"
 }
 
+function initialize_git_variables() {
+    # SHA of the commit for the current sources
+    COMMIT_SHA="$(git rev-parse HEAD || echo "Unknown")"
+    export COMMIT_SHA
+}
+
+function initialize_github_variables() {
+    # Defaults for interacting with GitHub
+    export GITHUB_REPOSITORY=${GITHUB_REPOSITORY:="apache/airflow"}
+    export GITHUB_REGISTRY=${GITHUB_REGISTRY:="docker.pkg.github.com"}
+    export USE_GITHUB_REGISTRY=${USE_GITHUB_REGISTRY:="false"}
+    export GITHUB_REGISTRY_PULL_IMAGE_TAG=${GITHUB_REGISTRY_PULL_IMAGE_TAG:="latest"}
+    export GITHUB_REGISTRY_PUSH_IMAGE_TAG=${GITHUB_REGISTRY_PUSH_IMAGE_TAG:="latest"}
+
+}
+
+# Common environment that is initialized by both Breeze and CI scripts
+function initialize_common_environment() {

Review comment:
       Thanks for breaking it down into smaller functions. It is now much clearer.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-689725405


   Hi @potiuk this change is awesome (and does wonders for the test time I'm sure) -- just one question that is confusing me;
   
   I can't quite work out where the images are actually built.
   
   For example this build https://github.com/apache/airflow/runs/1092589566 (one of my PRs, I don't think which one specifically is relevant?) I see this in the logs
   
   
   ```
   Waiting for master-python3.6-ci:246742927 image
   ........................ OK.
   Found master-python3.6-ci:246742927 image
   Digest: '"sha256:e064d441d5c4dc2aff4db69ad907de016476541f4fe55175860716b295599e64"'
   Waiting for master-python3.7-ci:246742927 image
   ... OK.
   Found master-python3.7-ci:246742927 image
   Digest: '"sha256:ae7ec96311f778f8e1ffc5364e9ce7759480fd14925c552183f6fb0062b6dc59"'
   Waiting for master-python3.8-ci:246742927 image
   ```
   
   But what I don't see in the logs of that entire action anywhere is the docker build logs... Did I just miss them?
   
   I'm just trying to understand how this change works -- it's definitely a good feature.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-677188163


   > Is it expected?
   
   Nope. But I thought I saw and fixed it. I will take a look.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473529858



##########
File path: scripts/ci/mermaid-config.json
##########
@@ -0,0 +1,4 @@
+{

Review comment:
       Shouldn't that be in PR with documentation?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675936557


   I have  3 more PRs extracted now from this one:
   - [ ] Kubernetes image no embedded #10399 
   - [ ] When precommits are run, output is silenced #10390 
   - [ ] Added Documentation for future CI with mermaid sequence diagrams  #10380 
   
   Once we merge those, it will be much easier for me to see what's left from the original PR and maybe split it even farther into few smaller ones. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472064881



##########
File path: CI.rst
##########
@@ -188,32 +568,104 @@ The issues that gets updated are different for different branches:
 
 * master: `Quarantine tests master <https://github.com/apache/airflow/issues/10118>`_
 * v1-10-stable: `Quarantine tests v1-10-stable <https://github.com/apache/airflow/issues/10127>`_
-* v1-10-test: `Quarantine tests v1-10-test <hhttps://github.com/apache/airflow/issues/10128>`_
-
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Job                       | Description                                                                                                    | Pull Request Run                   | Direct Push/Merge Run           | Scheduled Run                                                        |
-+===========================+================================================================================================================+====================================+=================================+======================================================================+
-| Cancel previous workflow  | Cancels the previously running workflow run if there is one running                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Trigger tests             | Checks if tests should be triggered                                                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Quarantined tests         | Those are tests that are flaky and we need to fix them                                                         | Yes (if tests-triggered)           | Yes (Updates quarantine issue)  | Yes * (updates quarantine issue)                                     |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-
-Cancel other workflow runs workflow
------------------------------------
-
-This workflow is run only on schedule (every 5 minutes) it's only purpose is to cancel other running
-``CI Build`` workflows if important jobs failed in those runs. This is to save runners for other runs
-in case we know that the build will not succeed anyway without some basic fixes to static checks or
-documentation - effectively implementing missing "fail-fast" (on a job level) in Github Actions
-similar to fail-fast in matrix strategy.
-
-The jobs that are considered as "fail-fast" are:
-
-* Static checks
-* Docs
-* Prepare Backport packages
-* Helm tests
-* Build Prod Image
-* TTest OpenAPI client gen
+* v1-10-test: `Quarantine tests v1-10-test <https://github.com/apache/airflow/issues/10128>`_
+
+Force sync master from apache/airflow
+-------------------------------------
+
+This is manually triggered workflow (via GitHub UI manual run) that should only be run in GitHub forks.
+When triggered, it will force-push the "apache/airflow" master to the fork's master. It's the easiest
+way to sync your fork master to the Apache Airflow's one.
+
+Naming conventions for stored images
+====================================
+
+The images produced during the CI builds are stored in the
+`GitHub Registry <https://github.com/apache/airflow/packages>`_
+
+The images are stored with both "latest" tag (for last master push image that passes all the tests as well
+with the tags indicating the origin of the image.
+
+The image names follow the patterns:
+
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| Image        | Name pattern               | Tag for format                 | Comment                                                                                    |
++==============+============================+================================+============================================================================================+
+| Python image | python                     | <X.Y>-slim-buster-<RUN_ID>     | Base python image used by both production and CI image.                                    |
+|              |                            | <X.Y>-slim-buster-<COMMIT_SHA> | Python maintainer release new versions of those image with security fixes every few weeks. |
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| CI image     | <BRANCH>-python<X.Y>-ci    | <RUN_ID>                       | CI image - this is the image used for most of the tests.                                   |

Review comment:
       yeah. I wanted to introduce some retention after I got it up and running. Just did not want to make the change any bigger than it is. This is not as bad as it seems because most of these images will be just adding few last layers (mostly changed android sources) because we are always building incrementally using the previous images with --cache-from.
   
   Until python releases a new version of base python image, 9X% of each image is the same - we simply add new layers on top. You can actually see it in my fork where most of the images we push are "Layer already exists".
   
   But indeed the scheduled build always produces a new image built from scratch and every time we change Dockerfile at the beginning or python releases a new base image we have new layers updated. So yeah. I plan to delete the old tags (> week I think should be enough) and run garbage collection.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #10368:
URL: https://github.com/apache/airflow/pull/10368


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675388464


   > This change is huge. What can we reduce its size? I can see a few changes that we can extract and do as a separate cha
   
   Yep. I am happy to do it. I will try to separate out some of the changes and keep on rebasing this own until the "gist of the change remains.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-689746306


   Also - the "Build Image" worklow performs all the cleanup operations (since it has the Write Token), It looks for duplicated runs or run where important jobs failed, and cancels them. This way we save a lot of unnecessary worker runs.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676390751


   @mik-laj @feluelle (others) I think this PR now is much better to review. All the rest is already merged except the separate #10380 - documentation - but I am on the fence here whether i should merge it separately (mermaid sequence diagrams generation) or whether to merge it one big PR (documentation + code together). I can do either - happy to hear about the opinions of others. 
   
   No matter what - I think it s much easier to review now  so looking forward to speeding up our CI. 
   
   BTW. Yesterday we also had a significant slow-down of the builds (new version of pythion got released so some of the builds blocked the queue for far longer than they should) - after we merge this one, this problem will be gone :).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473633577



##########
File path: scripts/ci/images/ci_prepare_prod_image_on_ci.sh
##########
@@ -18,4 +18,77 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-build_prod_image_on_ci
+# In case GITHUB_REGISTRY_PULL_IMAGE_TAG is different than latest, tries to pull the image indefinitely
+# skips further image checks - assuming that this is the right image
+function wait_for_prod_images_tag {
+    PROD_IMAGE_TO_WAIT_FOR="${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+    PROD_BUILD_IMAGE_TO_WAIT_FOR="${GITHUB_REGISTRY_AIRFLOW_PROD_BUILD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+    echo
+    echo "Waiting for image ${PROD_IMAGE_TO_WAIT_FOR}"
+    echo
+    while true; do
+        docker pull "${PROD_IMAGE_TO_WAIT_FOR}" || true
+        if [[ "$(docker images -q "${PROD_IMAGE_TO_WAIT_FOR}" 2> /dev/null)" == "" ]]; then
+            echo
+            echo "The image ${PROD_IMAGE_TO_WAIT_FOR} is not yet available. Waiting"
+            echo
+            sleep 10
+        else
+            echo
+            echo "The image ${PROD_IMAGE_TO_WAIT_FOR} downloaded."
+            echo "Tagging ${PROD_IMAGE_TO_WAIT_FOR} as ${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}."
+            docker tag  "${PROD_IMAGE_TO_WAIT_FOR}" "${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}"
+            echo "Tagging ${PROD_IMAGE_TO_WAIT_FOR} as ${AIRFLOW_PROD_IMAGE}."
+            docker tag  "${PROD_IMAGE_TO_WAIT_FOR}" "${AIRFLOW_PROD_IMAGE}"
+            echo
+            break
+        fi
+    done
+    echo
+    echo "Waiting for image ${PROD_BUILD_IMAGE_TO_WAIT_FOR}"
+    echo
+    while true; do

Review comment:
       Yep




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676841026


   <img width="1098" alt="Screenshot 2020-08-20 at 03 19 52" src="https://user-images.githubusercontent.com/12058428/90705798-0f6c1480-e294-11ea-8bb8-af6b0dae6129.png">
   Is it expected?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472953178



##########
File path: scripts/ci/libraries/_verbosity.sh
##########
@@ -28,58 +27,87 @@ function check_verbose_setup {
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
 # printed before execution
 function verbose_docker {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
+    if [[ ${VERBOSE:="false"} == "true" && \
+        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
+        ${VERBOSE_COMMANDS:=} != "true" && \
+        # And when generally printing info is disabled
+        ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
+        >&2 echo "docker" "${@}"
+    fi
+    if [[ ${PRINT_INFO_FROM_SCRIPTS} == "false" ]]; then
+        docker "${@}" >>"${OUTPUT_LOG}" 2>&1
+    else
+        docker "${@}" 2>&1 | tee -a "${OUTPUT_LOG}"
+    fi
+    EXIT_CODE="$?"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        # This way in the output log we only see the most recent failed command and what was echoed before
+        rm -f "${OUTPUT_LOG}"
     fi
-    docker "${@}"
+    return "${EXIT_CODE}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all helm commands run will be
 # printed before execution
 function verbose_helm {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "helm" "${@}"
+        >&2 echo "helm" "${@}"
+    fi
+    helm "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    helm "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kubectl commands run will be
 # printed before execution
 function verbose_kubectl {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kubectl" "${@}"
+        >&2 echo "kubectl" "${@}"
+    fi
+    kubectl "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    kubectl "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kind commands run will be
 # printed before execution
 function verbose_kind {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kind" "${@}"
+        >&2 echo "kind" "${@}"
     fi
+    # kind outputs nice output on terminal.
     kind "${@}"
 }
 
-
-# In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
-# printed before execution
-function verbose_docker_hide_output_on_success {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
-    fi
-    docker "${@}" >>"${OUTPUT_LOG}" 2>&1
-}
-
-
 # Prints verbose information in case VERBOSE variable is set
 function print_info() {
-    if [[ ${VERBOSE:="false"} == "true" ]]; then
+    if [[ ${VERBOSE:="false"} == "true" && ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
         echo "$@"
     fi
 }
+
+function set_verbosity() {
+    # whether verbose output should be produced
+    export VERBOSE=${VERBOSE:="false"}
+
+    # whether every bash statement should be printed as they are executed
+    export VERBOSE_COMMANDS=${VERBOSE_COMMANDS:="false"}
+
+    # whether the output from script should be printed at all
+    export PRINT_INFO_FROM_SCRIPTS=${PRINT_INFO_FROM_SCRIPTS:="true"}
+}
+
+set_verbosity
+
+alias docker=verbose_docker
+alias kubectl=verbose_kubectl
+alias helm=verbose_helm
+alias kind=verbose_kind

Review comment:
       Alias exists only until the script ends. Hoever I read a bit more about aliases and I think I will get rid of those and replace with functions. They provide much the same functionality and there are no special exceptions to them when it comes to "pre-commit" non-interactive bash scripts.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-689744391


   > I'm just trying to understand how this change works -- it's definitely a good feature.
   
   Sure, no problem it's all quite well described including the sequence diagrams and what happens in the [CI.rst](https://github.com/apache/airflow/blob/master/CI.rst#ci-sequence-diagrams). I think a look at the sequence diagrams can help to understand it.
   
   However indeed it's not straightforward how it works, and it's a long document with lots of details so here, the gist of it:
   
   For every Push/Pull Request/Schedule Workflow you have a corresponding "Build Image" run (which is of `workflow_run' type). The reason why they are separate, is that the "workflow_run" type of event has the "write" access token and can push the images to the registry. It uses the "master" version of workflow to run, so there is no risk that someone will inject their own code from PR into the build process, so "write access" is quite OK. In our case I make sure that we use "workflow" + "scripts"  from the master branch even if we use other sources (airflow sources that end up in the image) from the incoming PR commit. The "Build image" pushes the build to the registry and "CI Image" waits until the image is available in that registry.
   
   You can very easily find those builds (though there is no direct link in the original PR.  When you click on [Actions tab](https://github.com/apache/airflow/actions) - you will always see two builds next to each other "Build Images" one and the actual "PR build". They are always next to each other (because they are started at the same time). Also I made sure that you have an easy way to make sure which build matches which. Each "CI build" has a unique "run id" - and in both "Build Image" and "CI Build" workflows there is "Build Info" step that (when you click on it) show all the information necessary to match those (it's in job name to make it easier), The run id, commit hash of the commit, branch and repository from which the build was triggered is all there and you can be quite sure which build is which.
   
   Also, the "run_id" is used to identify the imge itself. If you look at the "Wait for image" it will RUN ID it waits for and the images in github registry are tagged with this Run id. For example you can see this in the logs of "waiting for image"
   
   ```
   Waiting for master-python3.6-build:246742927 image
   ```
   
   This also means that you can immediately locally reproduce any test failure - because the images are in the registry. It's enough to run this (if you see a failure in python 3.6 postgres core test for example):
   ```
   ./breeze --github-image-id 246742927 --python 3.6 --backend postgres 
   ```
   
   And after pulling the image you are dropped into the very same image that was used to run on CI and you can 100% reproduce the test that failed (barring heisentests and side effects of other tests). 
   
   I hope it's helpful. Let me know if you have any more questions on that.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472064881



##########
File path: CI.rst
##########
@@ -188,32 +568,104 @@ The issues that gets updated are different for different branches:
 
 * master: `Quarantine tests master <https://github.com/apache/airflow/issues/10118>`_
 * v1-10-stable: `Quarantine tests v1-10-stable <https://github.com/apache/airflow/issues/10127>`_
-* v1-10-test: `Quarantine tests v1-10-test <hhttps://github.com/apache/airflow/issues/10128>`_
-
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Job                       | Description                                                                                                    | Pull Request Run                   | Direct Push/Merge Run           | Scheduled Run                                                        |
-+===========================+================================================================================================================+====================================+=================================+======================================================================+
-| Cancel previous workflow  | Cancels the previously running workflow run if there is one running                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Trigger tests             | Checks if tests should be triggered                                                                            | Yes                                | Yes                             | Yes *                                                                |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-| Quarantined tests         | Those are tests that are flaky and we need to fix them                                                         | Yes (if tests-triggered)           | Yes (Updates quarantine issue)  | Yes * (updates quarantine issue)                                     |
-+---------------------------+----------------------------------------------------------------------------------------------------------------+------------------------------------+---------------------------------+----------------------------------------------------------------------+
-
-Cancel other workflow runs workflow
------------------------------------
-
-This workflow is run only on schedule (every 5 minutes) it's only purpose is to cancel other running
-``CI Build`` workflows if important jobs failed in those runs. This is to save runners for other runs
-in case we know that the build will not succeed anyway without some basic fixes to static checks or
-documentation - effectively implementing missing "fail-fast" (on a job level) in Github Actions
-similar to fail-fast in matrix strategy.
-
-The jobs that are considered as "fail-fast" are:
-
-* Static checks
-* Docs
-* Prepare Backport packages
-* Helm tests
-* Build Prod Image
-* TTest OpenAPI client gen
+* v1-10-test: `Quarantine tests v1-10-test <https://github.com/apache/airflow/issues/10128>`_
+
+Force sync master from apache/airflow
+-------------------------------------
+
+This is manually triggered workflow (via GitHub UI manual run) that should only be run in GitHub forks.
+When triggered, it will force-push the "apache/airflow" master to the fork's master. It's the easiest
+way to sync your fork master to the Apache Airflow's one.
+
+Naming conventions for stored images
+====================================
+
+The images produced during the CI builds are stored in the
+`GitHub Registry <https://github.com/apache/airflow/packages>`_
+
+The images are stored with both "latest" tag (for last master push image that passes all the tests as well
+with the tags indicating the origin of the image.
+
+The image names follow the patterns:
+
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| Image        | Name pattern               | Tag for format                 | Comment                                                                                    |
++==============+============================+================================+============================================================================================+
+| Python image | python                     | <X.Y>-slim-buster-<RUN_ID>     | Base python image used by both production and CI image.                                    |
+|              |                            | <X.Y>-slim-buster-<COMMIT_SHA> | Python maintainer release new versions of those image with security fixes every few weeks. |
++--------------+----------------------------+--------------------------------+--------------------------------------------------------------------------------------------+
+| CI image     | <BRANCH>-python<X.Y>-ci    | <RUN_ID>                       | CI image - this is the image used for most of the tests.                                   |

Review comment:
       yeah. I wanted to introduce some retention after I got it up and running. Just did not want to make the change any bigger than it is. This is not as bad as it seems because most of these images will be just adding few last layers (mostly changed android sources) because we are always building incrementally using the previous images with --cache-from.
   
   Until python releases a new version of base python image, 9X% of each image is the same - we simply add new layers on top. You can actually see it in my fork where most of the images we push are "Layer already exists".
   
   But indeed the scheduled build always produces a new image built from scratch and every time we change Dockerfile at the beginning or python releases a new base image we have new layers updated. So yeah. I plan to delete the old tags (> week I think should be enough).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472952239



##########
File path: scripts/ci/in_container/_in_container_utils.sh
##########
@@ -39,17 +39,19 @@ function in_container_script_end() {
     #shellcheck disable=2181
     EXIT_CODE=$?
     if [[ ${EXIT_CODE} != 0 ]]; then
-        if [[ -n ${OUT_FILE:=} ]]; then
-            echo "  ERROR ENCOUNTERED!"
-            echo
-            echo "  Output:"
-            echo
-            cat "${OUT_FILE}"
+        if [[ "${PRINT_INFO_FROM_SCRIPTS=="ture"}" == "true" ]] ;then

Review comment:
       Nice :+1: 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-675936557


   I have  3 more PRs extracted now from this one:
   - [x] Kubernetes image no embedded #10399 
   - [x] When precommits are run, output is silenced #10390 
   - [ ] Added Documentation for future CI with mermaid sequence diagrams  #10380 
   - [ ] Replaced aliases for common tools with functions #10402 
   
   Once we merge those, it will be much easier for me to see what's left from the original PR and maybe split it even farther into few smaller ones. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-676496752


   @potiuk I will look at it in the evening, because it requires concentration.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#issuecomment-689744391


   > I'm just trying to understand how this change works -- it's definitely a good feature.
   
   Sure, no problem it's all quite well described including the sequence diagrams and what happens in the [CI.rst](https://github.com/apache/airflow/blob/master/CI.rst#ci-sequence-diagrams). I think a look at the sequence diagrams can help to understand it.
   
   However indeed it's not straightforward how it works, and it's a long document with lots of details so here, the gist of it:
   
   For every Push/Pull Request/Schedule Workflow you have a corresponding "Build Image" run (which is of `workflow_run' type). The reason why they are separate, is that the "workflow_run" type of event has the "write" access token and can push the images to the registry. It uses the "master" version of workflow to run, so there is no risk that someone will inject their own code from PR into the build process, so "write access" is quite OK. In our case I make sure that we use "workflow" + "scripts"  from the master branch even if we use other sources (airflow sources that end up in the image) from the incoming PR commit. The "Build image" pushes the build to the registry and "CI Image" waits until the image is available in that registry.
   
   You can very easily find those builds (though there is no direct link in the original PR.  When you click on [Actions tab](https://github.com/apache/airflow/actions) - you will always see two builds next to each other "Build Images" one and the actual "PR build". They are always next to each other (because they are started at the same time). Also I made sure that you have an easy way to make sure which build matches which. Each "CI build" has a unique "run id" - and in both "Build Image" and "CI Build" workflows there is "Build Info" step that (when you click on it) show all the information necessary to match those (it's in job name to make it easier), The run id, commit hash of the commit, branch and repository from which the build was triggered is all there and you can be quite sure which build is which.
   
   Also, the "run_id" is used to identify the imge itself. If you look at the "Wait for image" it will show the "run_id" it waits for and the images in github registry are tagged with this Run id. For example you can see this in the logs of "waiting for image"
   
   ```
   Waiting for master-python3.6-build:246742927 image
   ```
   
   This also means that you can immediately locally reproduce any test failure - because the images are in the registry. It's enough to run this (if you see a failure in python 3.6 postgres core test for example):
   ```
   ./breeze --github-image-id 246742927 --python 3.6 --backend postgres 
   ```
   
   And after pulling the image you are dropped into the very same image that was used to run on CI and you can 100% reproduce the test that failed (barring heisentests and side effects of other tests). 
   
   I hope it's helpful. Let me know if you have any more questions on that.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] feluelle commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r472848834



##########
File path: scripts/ci/libraries/_push_pull_remove_images.sh
##########
@@ -33,41 +33,43 @@ function pull_image_if_needed() {
         echo
         echo "Pulling the image ${IMAGE_TO_PULL}"
         echo
-        verbose_docker pull "${IMAGE_TO_PULL}" | tee -a "${OUTPUT_LOG}"
+        # need eto be explicitly verbose in order to have pre-commit spinner working

Review comment:
       ```suggestion
           # need to be explicitly verbose in order to have pre-commit spinner working
   ```

##########
File path: scripts/ci/in_container/_in_container_utils.sh
##########
@@ -39,17 +39,19 @@ function in_container_script_end() {
     #shellcheck disable=2181
     EXIT_CODE=$?
     if [[ ${EXIT_CODE} != 0 ]]; then
-        if [[ -n ${OUT_FILE:=} ]]; then
-            echo "  ERROR ENCOUNTERED!"
-            echo
-            echo "  Output:"
-            echo
-            cat "${OUT_FILE}"
+        if [[ "${PRINT_INFO_FROM_SCRIPTS=="ture"}" == "true" ]] ;then

Review comment:
       ```suggestion
           if [[ "${PRINT_INFO_FROM_SCRIPTS=="true"}" == "true" ]] ;then
   ```

##########
File path: scripts/ci/libraries/_push_pull_remove_images.sh
##########
@@ -103,75 +111,122 @@ function pull_prod_images_if_needed() {
             echo
             echo "Force pull base image ${PYTHON_BASE_IMAGE}"
             echo
-            if [[ ${PULL_PYTHON_BASE_IMAGES_FROM_CACHE:="true"} == "true" ]]; then
-                pull_image_possibly_from_cache "${PYTHON_BASE_IMAGE}" "${CACHED_PYTHON_BASE_IMAGE}"
+            if [[ ${USE_GITHUB_REGISTRY:="false"} == "true" ]]; then
+                PYTHON_TAG_SUFFIX=""
+                if [[ ${GITHUB_REGISTRY_PULL_IMAGE_TAG} != "latest" ]]; then
+                    PYTHON_TAG_SUFFIX="-${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+                fi
+                pull_image_github_dockerhub "${PYTHON_BASE_IMAGE}" "${GITHUB_REGISTRY_PYTHON_BASE_IMAGE}${PYTHON_TAG_SUFFIX}"
             else
-                verbose_docker pull "${PYTHON_BASE_IMAGE}" | tee -a "${OUTPUT_LOG}"
+                # need eto be explicitly verbose in order to have pre-commit spinner working
+                # No aliases in pre-commit non-interactive shells :(
+                verbose_docker pull "${PYTHON_BASE_IMAGE}"
             fi
             echo
         fi
         # "Build" segment of production image
-        pull_image_possibly_from_cache "${AIRFLOW_PROD_BUILD_IMAGE}" "${CACHED_AIRFLOW_PROD_BUILD_IMAGE}"
-        # we never pull the main segment of production image - we always build it locally = this is
-        # usually very fast this way and it is much nicer for rebuilds and development
+        pull_image_github_dockerhub "${AIRFLOW_PROD_BUILD_IMAGE}" "${GITHUB_REGISTRY_AIRFLOW_PROD_BUILD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
+        # "Main" segment of production image
+        pull_image_github_dockerhub "${AIRFLOW_PROD_IMAGE}" "${GITHUB_REGISTRY_AIRFLOW_PROD_IMAGE}:${GITHUB_REGISTRY_PULL_IMAGE_TAG}"
     fi
 }
 
-# Pushes Ci image and it's manifest to the registry. In case the image was taken from cache registry
-# it is pushed to the cache, not to the main registry. Manifest is only pushed to the main registry
-function push_ci_image() {
-    if [[ ${CACHED_AIRFLOW_CI_IMAGE:=} != "" ]]; then
-        verbose_docker tag "${AIRFLOW_CI_IMAGE}" "${CACHED_AIRFLOW_CI_IMAGE}"
-        IMAGE_TO_PUSH="${CACHED_AIRFLOW_CI_IMAGE}"
-    else
-        IMAGE_TO_PUSH="${AIRFLOW_CI_IMAGE}"
+# Pushes Ci images and the manifest to the registry in DockerHub.
+function push_ci_images_to_dockerhub() {
+    docker push "${AIRFLOW_CI_IMAGE}"
+    docker tag "${AIRFLOW_CI_LOCAL_MANIFEST_IMAGE}" "${AIRFLOW_CI_REMOTE_MANIFEST_IMAGE}"
+    docker push "${AIRFLOW_CI_REMOTE_MANIFEST_IMAGE}"
+    if [[ -n ${DEFAULT_CI_IMAGE:=""} ]]; then
+        # Only push default image to DockerHub registry if it is defined and

Review comment:
       ```suggestion
           # Only push default image to DockerHub registry if it is defined
   ```

##########
File path: scripts/ci/libraries/_verbosity.sh
##########
@@ -28,58 +27,87 @@ function check_verbose_setup {
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
 # printed before execution
 function verbose_docker {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
+    if [[ ${VERBOSE:="false"} == "true" && \
+        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
+        ${VERBOSE_COMMANDS:=} != "true" && \
+        # And when generally printing info is disabled
+        ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
+        >&2 echo "docker" "${@}"
+    fi
+    if [[ ${PRINT_INFO_FROM_SCRIPTS} == "false" ]]; then
+        docker "${@}" >>"${OUTPUT_LOG}" 2>&1
+    else
+        docker "${@}" 2>&1 | tee -a "${OUTPUT_LOG}"
+    fi
+    EXIT_CODE="$?"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        # This way in the output log we only see the most recent failed command and what was echoed before
+        rm -f "${OUTPUT_LOG}"
     fi
-    docker "${@}"
+    return "${EXIT_CODE}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all helm commands run will be
 # printed before execution
 function verbose_helm {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "helm" "${@}"
+        >&2 echo "helm" "${@}"
+    fi
+    helm "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    helm "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kubectl commands run will be
 # printed before execution
 function verbose_kubectl {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kubectl" "${@}"
+        >&2 echo "kubectl" "${@}"
+    fi
+    kubectl "${@}" | tee -a "${OUTPUT_LOG}"
+    if [[ ${EXIT_CODE} == "0" ]]; then
+        # No matter if "set -e" is used the log will be removed on success.
+        rm -f "${OUTPUT_LOG}"
     fi
-    kubectl "${@}"
 }
 
 # In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all kind commands run will be
 # printed before execution
 function verbose_kind {
     if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
        # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "kind" "${@}"
+        >&2 echo "kind" "${@}"
     fi
+    # kind outputs nice output on terminal.
     kind "${@}"
 }
 
-
-# In case "VERBOSE" is set to "true" (--verbose flag in Breeze) all docker commands run will be
-# printed before execution
-function verbose_docker_hide_output_on_success {
-    if [[ ${VERBOSE:="false"} == "true" && ${VERBOSE_COMMANDS:=} != "true" ]]; then
-       # do not print echo if VERBOSE_COMMAND is set (set -x does it already)
-        echo "docker" "${@}"
-    fi
-    docker "${@}" >>"${OUTPUT_LOG}" 2>&1
-}
-
-
 # Prints verbose information in case VERBOSE variable is set
 function print_info() {
-    if [[ ${VERBOSE:="false"} == "true" ]]; then
+    if [[ ${VERBOSE:="false"} == "true" && ${PRINT_INFO_FROM_SCRIPTS} == "true" ]]; then
         echo "$@"
     fi
 }
+
+function set_verbosity() {
+    # whether verbose output should be produced
+    export VERBOSE=${VERBOSE:="false"}
+
+    # whether every bash statement should be printed as they are executed
+    export VERBOSE_COMMANDS=${VERBOSE_COMMANDS:="false"}
+
+    # whether the output from script should be printed at all
+    export PRINT_INFO_FROM_SCRIPTS=${PRINT_INFO_FROM_SCRIPTS:="true"}
+}
+
+set_verbosity
+
+alias docker=verbose_docker
+alias kubectl=verbose_kubectl
+alias helm=verbose_helm
+alias kind=verbose_kind

Review comment:
       For how long do these alias' exist? I mean if you simply run docker commands after a breeze session. Does it still use the verbose version?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473633829



##########
File path: scripts/ci/libraries/_initialization.sh
##########
@@ -223,94 +275,190 @@ function initialize_common_environment {
     export CI_BUILD_ID="0"
 }
 
+function initialize_git_variables() {
+    # SHA of the commit for the current sources
+    COMMIT_SHA="$(git rev-parse HEAD || echo "Unknown")"
+    export COMMIT_SHA
+}
+
+function initialize_github_variables() {
+    # Defaults for interacting with GitHub
+    export GITHUB_REPOSITORY=${GITHUB_REPOSITORY:="apache/airflow"}
+    export GITHUB_REGISTRY=${GITHUB_REGISTRY:="docker.pkg.github.com"}
+    export USE_GITHUB_REGISTRY=${USE_GITHUB_REGISTRY:="false"}
+    export GITHUB_REGISTRY_PULL_IMAGE_TAG=${GITHUB_REGISTRY_PULL_IMAGE_TAG:="latest"}
+    export GITHUB_REGISTRY_PUSH_IMAGE_TAG=${GITHUB_REGISTRY_PUSH_IMAGE_TAG:="latest"}
+
+}
+
+# Common environment that is initialized by both Breeze and CI scripts
+function initialize_common_environment() {

Review comment:
       Yeah. And unit testable. Soon.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #10368: CI Images are now pre-build and stored in registry

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #10368:
URL: https://github.com/apache/airflow/pull/10368#discussion_r473617171



##########
File path: scripts/ci/images/ci_wait_for_all_prod_images.sh
##########
@@ -0,0 +1,43 @@
+#!/usr/bin/env 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.
+
+# This is hook build used by DockerHub. We are also using it
+# on CI to potentially rebuild (and refresh layers that
+# are not cached) Docker images that are used to run CI jobs
+# shellcheck source=scripts/ci/libraries/_script_init.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+
+
+echo
+echo "Waiting for all images to appear: ${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS[*]}"
+echo
+
+echo
+echo "Check if jq is installed"
+echo
+command -v jq
+
+jq --version

Review comment:
       Yes. But tha's OK. IT's mainly to diagnose in case we have some version change. JQ is installed by default on Githb Actions (and pretty much and CI). I will add some more meaningful message though.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org