You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2022/07/05 12:24:07 UTC

[airflow] 01/04: Prepare ARM images much faster and cheaper (#24813)

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

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

commit 951eb0029fcff5af1faf5965fde472ef86578cf9
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Mon Jul 4 09:08:38 2022 +0200

    Prepare ARM images much faster and cheaper (#24813)
    
    Since we are now building ARM images in parallel, We need more powerful
    machines and we implemented in-memory docker, similarly as in our AMD
    instances. The m6g.2xlarge  are quite a bit better than c6g.xlarge for
    our case:
    
    1) They have 8 vCPUs
    2) They have 32 GB memory (should be enough to build 4 ARM CI images
    3) Thye are Memory-optimised, and since docker is build in memory
       the memory speed is the most important factor
    
    This also allows to switch building all images (including cache)
    in parallel - so that we can have 1 job instead of 4 - similarly
    as we have in case of regular AMD builds.
    
    Another advantage of it is that we loose far less time of the
    AMD instance which "triggers" the docker build, because this instance
    will control 4 parallel builds at a time effectively, which will
    decrease a lot of overhead connected with running the instance mostly
    idle during the build (and since the builds will be generally
    faster, the overhead will be even smaller).
    
    (cherry picked from commit 6621f685ce4af4c6e429fbbba8f8fa418d287c24)
---
 .github/workflows/build-images.yml                 |  5 ++-
 .github/workflows/ci.yml                           | 29 +++++++--------
 .../airflow_breeze/commands/ci_image_commands.py   |  2 ++
 .../commands/production_image_commands.py          |  2 ++
 .../airflow_breeze/utils/docker_command_utils.py   | 41 ++++++++++++++++++++++
 dev/breeze/src/airflow_breeze/utils/run_utils.py   |  6 +++-
 .../ci_start_arm_instance_and_connect_to_docker.sh |  8 ++---
 .../ci/images/{self_terminate.sh => initialize.sh} | 15 ++++++--
 8 files changed, 82 insertions(+), 26 deletions(-)

diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml
index 03e156dacf..638118727b 100644
--- a/.github/workflows/build-images.yml
+++ b/.github/workflows/build-images.yml
@@ -370,7 +370,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
 
 
   build-ci-images-arm:
-    timeout-minutes: 120
+    timeout-minutes: 50
     name: "Build ARM CI images ${{ needs.build-info.outputs.all-python-versions-list-as-string }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runs-on) }}
     needs: [build-info, build-prod-images]
@@ -426,8 +426,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           Build ARM CI images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
           ${{ needs.build-info.outputs.all-python-versions-list-as-string }}
         run: >
-          breeze build-image --run-in-parallel --parallelism 1
-          --builder airflow_cache --platform "linux/arm64"
+          breeze build-image --run-in-parallel --builder airflow_cache --platform "linux/arm64"
         env:
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cache-directive }}
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 5cfe017ff7..ac1fb07688 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -1677,7 +1677,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
   push-buildx-cache-to-github-registry:
     permissions:
       packages: write
-    timeout-minutes: 120
+    timeout-minutes: 50
     name: "Push Image Cache"
     runs-on: ${{ fromJson(needs.build-info.outputs.runs-on) }}
     needs:
@@ -1688,7 +1688,6 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     strategy:
       fail-fast: false
       matrix:
-        python-version: ${{ fromJson(needs.build-info.outputs.python-versions) }}
         platform: ["linux/amd64", "linux/arm64"]
     env:
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runs-on) }}
@@ -1738,6 +1737,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           breeze build-image
           --builder airflow_cache
           --prepare-buildx-cache
+          --run-in-parallel
           --force-build
           --platform ${{ matrix.platform }}
         env:
@@ -1763,9 +1763,9 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         if: always()
 
   build-ci-arm-images:
-    timeout-minutes: 120
+    timeout-minutes: 50
     name: >
-      ${{needs.build-info.outputs.build-job-description}} CI ARM images
+      Build CI ARM images
       ${{ needs.build-info.outputs.all-python-versions-list-as-string }}
     runs-on: ${{ fromJson(needs.build-info.outputs.runs-on) }}
     needs:
@@ -1782,45 +1782,42 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       DEFAULT_CONSTRAINTS_BRANCH: ${{ needs.build-info.outputs.default-constraints-branch }}
       DEBIAN_VERSION: ${{ needs.build-info.outputs.debian-version }}
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runs-on)[0] }}
-    if: needs.build-info.outputs.upgrade-to-newer-dependencies != 'false'
+    if: >
+      needs.build-info.outputs.upgrade-to-newer-dependencies != 'false' &&
+      needs.build-info.outputs.in-workflow-build == 'true'
     steps:
       - name: Cleanup repo
         run: docker run -v "${GITHUB_WORKSPACE}:/workspace" -u 0:0 bash -c "rm -rf /workspace/*"
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - uses: actions/checkout@v2
         with:
           ref: ${{ needs.build-info.outputs.targetCommitSha }}
           persist-credentials: false
           submodules: recursive
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - name: "Setup python"
         uses: actions/setup-python@v2
         with:
           python-version: ${{ needs.build-info.outputs.default-python-version }}
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - run: ./scripts/ci/install_breeze.sh
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - name: "Free space"
         run: breeze free-space
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - name: "Start ARM instance"
         run: ./scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - name: >
           Build CI ARM images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
           ${{ needs.build-info.outputs.all-python-versions-list-as-string }}
         run: >
-          breeze build-image --run-in-parallel --parallelism 1
-          --builder airflow_cache --platform "linux/arm64"
+          breeze build-image
+          --run-in-parallel
+          --builder airflow_cache
+          --platform "linux/arm64"
         env:
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgrade-to-newer-dependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cache-directive }}
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
           PYTHON_VERSIONS: ${{ needs.build-info.outputs.all-python-versions-list-as-string }}
-        if: needs.build-info.outputs.in-workflow-build == 'true'
       - name: "Stop ARM instance"
         run: ./scripts/ci/images/ci_stop_arm_instance.sh
-        if: always() && needs.build-info.outputs.in-workflow-build == 'true'
+        if: always()
       - name: "Fix ownership"
         run: breeze fix-ownership
-        if: always() && needs.build-info.outputs.in-workflow-build == 'true'
+        if: always()
diff --git a/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py b/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
index 298d264b0b..dbc6acda2f 100644
--- a/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
@@ -77,6 +77,7 @@ from airflow_breeze.utils.docker_command_utils import (
     perform_environment_checks,
     prepare_docker_build_command,
     prepare_docker_build_from_input,
+    warm_up_docker_builder,
 )
 from airflow_breeze.utils.image import run_pull_image, run_pull_in_parallel, tag_image_as_latest
 from airflow_breeze.utils.mark_image_as_refreshed import mark_image_as_refreshed
@@ -211,6 +212,7 @@ def run_build_in_parallel(
     dry_run: bool,
     verbose: bool,
 ) -> None:
+    warm_up_docker_builder(image_params_list[0], verbose=verbose, dry_run=dry_run)
     get_console().print(
         f"\n[info]Building with parallelism = {parallelism} for the images: {python_version_list}:"
     )
diff --git a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
index d08e1d89b5..a9442ba639 100644
--- a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
@@ -76,6 +76,7 @@ from airflow_breeze.utils.docker_command_utils import (
     perform_environment_checks,
     prepare_docker_build_command,
     prepare_docker_build_from_input,
+    warm_up_docker_builder,
 )
 from airflow_breeze.utils.image import run_pull_image, run_pull_in_parallel, tag_image_as_latest
 from airflow_breeze.utils.parallel import check_async_run_results
@@ -214,6 +215,7 @@ def run_build_in_parallel(
     dry_run: bool,
     verbose: bool,
 ) -> None:
+    warm_up_docker_builder(image_params_list[0], verbose=verbose, dry_run=dry_run)
     get_console().print(
         f"\n[info]Building with parallelism = {parallelism} for the images: {python_version_list}:"
     )
diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
index cdb1ceb258..7b5e37f3b0 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -622,3 +622,44 @@ def perform_environment_checks(verbose: bool):
     check_docker_is_running(verbose=verbose)
     check_docker_version(verbose=verbose)
     check_docker_compose_version(verbose=verbose)
+
+
+def get_docker_syntax_version() -> str:
+    from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT
+
+    return (AIRFLOW_SOURCES_ROOT / "Dockerfile").read_text().splitlines()[0]
+
+
+def warm_up_docker_builder(image_params: CommonBuildParams, verbose: bool, dry_run: bool):
+    from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT
+
+    if image_params.builder == "default":
+        return
+    docker_syntax = get_docker_syntax_version()
+    get_console().print(f"[info]Warming up the {image_params.builder} builder for syntax: {docker_syntax}")
+    warm_up_image_param = deepcopy(image_params)
+    warm_up_image_param.image_tag = "warmup"
+    warm_up_image_param.push_image = False
+    build_command = prepare_base_build_command(image_params=warm_up_image_param, verbose=verbose)
+    warm_up_command = []
+    warm_up_command.extend(["docker"])
+    warm_up_command.extend(build_command)
+    warm_up_command.extend(["--platform", image_params.platform, "-"])
+    warm_up_command_result = run_command(
+        warm_up_command,
+        input=f"""{docker_syntax}
+FROM scratch
+LABEL description="test warmup image"
+""",
+        verbose=verbose,
+        dry_run=dry_run,
+        cwd=AIRFLOW_SOURCES_ROOT,
+        text=True,
+        enabled_output_group=True,
+    )
+    if warm_up_command_result.returncode != 0:
+        get_console().print(
+            f"[error]Error {warm_up_command_result.returncode} when warming up builder:"
+            f" {warm_up_command_result.stdout} {warm_up_command_result.stderr}"
+        )
+        sys.exit(warm_up_command_result.returncode)
diff --git a/dev/breeze/src/airflow_breeze/utils/run_utils.py b/dev/breeze/src/airflow_breeze/utils/run_utils.py
index 9754605f49..29268be8b3 100644
--- a/dev/breeze/src/airflow_breeze/utils/run_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/run_utils.py
@@ -89,7 +89,11 @@ def run_command(
         command_to_print = ' '.join(shlex.quote(c) for c in cmd)
         env_to_print = get_environments_to_print(env)
         with ci_group(title=f"Running {title}"):
-            get_console().print(f"\n[info]Working directory {workdir} [/]\n")
+            get_console().print(f"\n[info]Working directory {workdir}\n")
+            if input:
+                get_console().print("[info]Input:")
+                get_console().print(input)
+                get_console().print()
             # Soft wrap allows to copy&paste and run resulting output as it has no hard EOL
             get_console().print(f"\n[info]{env_to_print}{command_to_print}[/]\n", soft_wrap=True)
         if dry_run:
diff --git a/scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh b/scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh
index a3fbf6b5ba..5cf486e611 100755
--- a/scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh
+++ b/scripts/ci/images/ci_start_arm_instance_and_connect_to_docker.sh
@@ -22,12 +22,12 @@ SCRIPTS_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd)"
 # This is an AMI that is based on Basic Amazon Linux AMI with installed and configured docker service
 WORKING_DIR="/tmp/armdocker"
 INSTANCE_INFO="${WORKING_DIR}/instance_info.json"
-ARM_AMI="ami-06b8158ea372d3259"
-INSTANCE_TYPE="c6g.xlarge"
-MARKET_OPTIONS="MarketType=spot,SpotOptions={MaxPrice=0.1,SpotInstanceType=one-time}"
+ARM_AMI="ami-0e43196369d299715"  # AMI ID of latest arm-docker-ami-v*
+INSTANCE_TYPE="m6g.2xlarge"  # m6g.2xlarge -> 8 vCPUS 32 GB RAM
+MARKET_OPTIONS="MarketType=spot,SpotOptions={MaxPrice=0.2,SpotInstanceType=one-time}"
 REGION="us-east-2"
 EC2_USER="ec2-user"
-USER_DATA_FILE="${SCRIPTS_DIR}/self_terminate.sh"
+USER_DATA_FILE="${SCRIPTS_DIR}/initialize.sh"
 METADATA_ADDRESS="http://169.254.169.254/latest/meta-data"
 MAC_ADDRESS=$(curl -s "${METADATA_ADDRESS}/network/interfaces/macs/" | head -n1 | tr -d '/')
 CIDR=$(curl -s "${METADATA_ADDRESS}/network/interfaces/macs/${MAC_ADDRESS}/vpc-ipv4-cidr-block/")
diff --git a/scripts/ci/images/self_terminate.sh b/scripts/ci/images/initialize.sh
similarity index 67%
rename from scripts/ci/images/self_terminate.sh
rename to scripts/ci/images/initialize.sh
index ca3af5e7a1..f76fdb801b 100755
--- a/scripts/ci/images/self_terminate.sh
+++ b/scripts/ci/images/initialize.sh
@@ -16,7 +16,18 @@
 # specific language governing permissions and limitations
 # under the License.
 
-# This instance will run for maximum 50 minutes and
+# We are mounting /var/lib/docker and /tmp as tmpfs in order
+# to gain speed when building the images The docker storage
+# is ephemeral anyway and will be removed when instance stops
+
+sudo service docker stop || true
+
+sudo mount -t tmpfs -o size=3% tmpfs /tmp
+sudo mount -t tmpfs -o size=85% tmpfs /var/lib/docker
+
+sudo service docker start
+
+# This instance will run for maximum 40 minutes and
 # It will terminate itself after that (it can also
 # be terminated immediately when the job finishes)
-echo "sudo shutdown -h now" | at now +50 min
+echo "sudo shutdown -h now" | at now +40 min