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:06 UTC

[airflow] branch v2-3-test updated (0ad211d9d2 -> 069a84c4ad)

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

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


    from 0ad211d9d2 Unified "dash-name" convention for outputs in ci workflows.
     new 951eb0029f Prepare ARM images much faster and cheaper (#24813)
     new cd6987f7cb Add "generated" folder to volumes mounted when "MOUNT_SELECTED" used (#24818)
     new ec02981daa Adding Docker context check for breeze (#24751)
     new 069a84c4ad Remove upper-binding for SQLAlchemy (#24819)

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


Summary of changes:
 .github/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   | 89 +++++++++++++++++++++-
 dev/breeze/src/airflow_breeze/utils/run_utils.py   |  6 +-
 dev/breeze/tests/test_docker_command_utils.py      | 64 +++++++++++++++-
 scripts/ci/docker-compose/local.yml                |  4 +-
 .../ci_start_arm_instance_and_connect_to_docker.sh |  8 +-
 .../ci/images/{self_terminate.sh => initialize.sh} | 15 +++-
 .../ci/pre_commit/pre_commit_local_yml_mounts.py   |  9 ++-
 setup.cfg                                          |  6 +-
 tests/utils/test_db.py                             |  3 +
 13 files changed, 202 insertions(+), 40 deletions(-)
 rename scripts/ci/images/{self_terminate.sh => initialize.sh} (67%)


[airflow] 03/04: Adding Docker context check for breeze (#24751)

Posted by po...@apache.org.
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 ec02981daa0ef8f05c8b899982371c1074abe6ac
Author: Alex Kruchkov <36...@users.noreply.github.com>
AuthorDate: Mon Jul 4 13:46:38 2022 +0300

    Adding Docker context check for breeze (#24751)
    
    (cherry picked from commit e520ef5160bcc2d98e5ccb042ef9a6506553e7fe)
---
 .../airflow_breeze/utils/docker_command_utils.py   | 36 ++++++++++++
 dev/breeze/tests/test_docker_command_utils.py      | 64 +++++++++++++++++++++-
 2 files changed, 99 insertions(+), 1 deletion(-)

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 c49c0f183f..d646ba0fab 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -307,6 +307,41 @@ Make sure docker-compose you install is first on the PATH variable of yours.
         )
 
 
+def check_docker_context(verbose: bool):
+    """
+    Checks whether Docker is using the expected context
+    :param verbose: print commands when running
+    """
+    expected_docker_context = "default"
+    response = run_command(
+        ["docker", "info", "--format", "{{json .ClientInfo.Context}}"],
+        verbose=verbose,
+        no_output_dump_on_exception=False,
+        text=True,
+        capture_output=True,
+    )
+    if response.returncode != 0:
+        get_console().print(
+            '[warning]Could not check for Docker context.[/]\n'
+            '[warning]Please make sure that Docker is using the right context by running "docker info" and '
+            'checking the active Context.[/]'
+        )
+        return
+
+    used_docker_context = response.stdout.strip().replace('"', '')
+
+    if used_docker_context == expected_docker_context:
+        get_console().print(f'[success]Good Docker context used: {used_docker_context}.[/]')
+    else:
+        get_console().print(
+            f'[error]Docker is not using the default context, used context is: {used_docker_context}[/]\n'
+            f'[warning]Please make sure Docker is using the {expected_docker_context} context.[/]\n'
+            f'[warning]You can try switching contexts by running: "docker context use '
+            f'{expected_docker_context}"[/]'
+        )
+        sys.exit(1)
+
+
 def get_env_variable_value(arg_name: str, params: Union[CommonBuildParams, ShellParams]):
     raw_value = getattr(params, arg_name, None)
     value = str(raw_value) if raw_value is not None else ''
@@ -630,6 +665,7 @@ def perform_environment_checks(verbose: bool):
     check_docker_is_running(verbose=verbose)
     check_docker_version(verbose=verbose)
     check_docker_compose_version(verbose=verbose)
+    check_docker_context(verbose=verbose)
 
 
 def get_docker_syntax_version() -> str:
diff --git a/dev/breeze/tests/test_docker_command_utils.py b/dev/breeze/tests/test_docker_command_utils.py
index 55c7c3e4d4..80fe192764 100644
--- a/dev/breeze/tests/test_docker_command_utils.py
+++ b/dev/breeze/tests/test_docker_command_utils.py
@@ -18,7 +18,13 @@
 from unittest import mock
 from unittest.mock import call
 
-from airflow_breeze.utils.docker_command_utils import check_docker_compose_version, check_docker_version
+import pytest
+
+from airflow_breeze.utils.docker_command_utils import (
+    check_docker_compose_version,
+    check_docker_context,
+    check_docker_version,
+)
 
 
 @mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission_denied')
@@ -199,3 +205,59 @@ def test_check_docker_compose_version_higher(mock_get_console, mock_run_command)
     mock_get_console.return_value.print.assert_called_with(
         "[success]Good version of docker-compose: 1.29.2[/]"
     )
+
+
+@mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
+@mock.patch('airflow_breeze.utils.docker_command_utils.get_console')
+def test_check_docker_context_default(mock_get_console, mock_run_command):
+    mock_run_command.return_value.returncode = 0
+    mock_run_command.return_value.stdout = "default"
+    check_docker_context(verbose=True)
+    mock_run_command.assert_called_with(
+        ["docker", "info", "--format", "{{json .ClientInfo.Context}}"],
+        verbose=True,
+        no_output_dump_on_exception=False,
+        text=True,
+        capture_output=True,
+    )
+    mock_get_console.return_value.print.assert_called_with('[success]Good Docker context used: default.[/]')
+
+
+@mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
+@mock.patch('airflow_breeze.utils.docker_command_utils.get_console')
+def test_check_docker_context_other(mock_get_console, mock_run_command):
+    mock_run_command.return_value.returncode = 0
+    mock_run_command.return_value.stdout = "other"
+    with pytest.raises(SystemExit):
+        check_docker_context(verbose=True)
+    mock_run_command.assert_called_with(
+        ["docker", "info", "--format", "{{json .ClientInfo.Context}}"],
+        verbose=True,
+        no_output_dump_on_exception=False,
+        text=True,
+        capture_output=True,
+    )
+    mock_get_console.return_value.print.assert_called_with(
+        '[error]Docker is not using the default context, used context is: other[/]\n'
+        '[warning]Please make sure Docker is using the default context.[/]\n'
+        '[warning]You can try switching contexts by running: "docker context use default"[/]'
+    )
+
+
+@mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
+@mock.patch('airflow_breeze.utils.docker_command_utils.get_console')
+def test_check_docker_context_command_failed(mock_get_console, mock_run_command):
+    mock_run_command.return_value.returncode = 1
+    check_docker_context(verbose=True)
+    mock_run_command.assert_called_with(
+        ["docker", "info", "--format", "{{json .ClientInfo.Context}}"],
+        verbose=True,
+        no_output_dump_on_exception=False,
+        text=True,
+        capture_output=True,
+    )
+    mock_get_console.return_value.print.assert_called_with(
+        '[warning]Could not check for Docker context.[/]\n'
+        '[warning]Please make sure that Docker is using the right context by running "docker info" and '
+        'checking the active Context.[/]'
+    )


[airflow] 04/04: Remove upper-binding for SQLAlchemy (#24819)

Posted by po...@apache.org.
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 069a84c4adffbe4c32eb8096a183fd0c53f2bafe
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Tue Jul 5 11:13:50 2022 +0200

    Remove upper-binding for SQLAlchemy (#24819)
    
    There was a problem with custom classes for SQLAlchemy that
    prevented it to work on MySQL. This PR removes the SQLAlchemy
    upper binding.
    
    This has been added as an issue in:
    
    https://github.com/sqlalchemy/sqlalchemy/issues/7660
    
    But apparently it's been fixed in one of the more recent SQLAlchemy
    releases.
    
    (cherry picked from commit 0853d60f16b45472ebac83bcb4fb6de112c90d6b)
---
 setup.cfg              | 6 +-----
 tests/utils/test_db.py | 3 +++
 2 files changed, 4 insertions(+), 5 deletions(-)

diff --git a/setup.cfg b/setup.cfg
index 323e21b234..0e1e9f7b84 100644
--- a/setup.cfg
+++ b/setup.cfg
@@ -141,11 +141,7 @@ install_requires =
     python-slugify>=5.0
     rich>=12.4.4
     setproctitle>=1.1.8
-    # SQL Alchemy 1.4.10 introduces a bug where for PyODBC driver UTCDateTime fields get wrongly converted
-    # as string and fail to be converted back to datetime. It was supposed to be fixed in
-    # https://github.com/sqlalchemy/sqlalchemy/issues/6366 (released in 1.4.12) but apparently our case
-    # is different. Opened https://github.com/sqlalchemy/sqlalchemy/issues/7660 to track it
-    sqlalchemy>=1.4,<1.4.10
+    sqlalchemy>=1.4
     sqlalchemy_jsonfield>=1.0
     tabulate>=0.7.5
     tenacity>=6.2.0
diff --git a/tests/utils/test_db.py b/tests/utils/test_db.py
index 752f9873aa..7340bc40cb 100644
--- a/tests/utils/test_db.py
+++ b/tests/utils/test_db.py
@@ -62,6 +62,9 @@ class TestDb:
             lambda t: (t[0] == 'remove_table' and t[1].name == 'spt_fallback_usg'),
             lambda t: (t[0] == 'remove_table' and t[1].name == 'MSreplication_options'),
             lambda t: (t[0] == 'remove_table' and t[1].name == 'spt_fallback_dev'),
+            # MSSQL foreign keys where CASCADE has been removed
+            lambda t: (t[0] == 'remove_fk' and t[1].name == 'task_reschedule_dr_fkey'),
+            lambda t: (t[0] == 'add_fk' and t[1].name == 'task_reschedule_dr_fkey'),
             # Ignore flask-session table/index
             lambda t: (t[0] == 'remove_table' and t[1].name == 'session'),
             lambda t: (t[0] == 'remove_index' and t[1].name == 'session_id'),


[airflow] 02/04: Add "generated" folder to volumes mounted when "MOUNT_SELECTED" used (#24818)

Posted by po...@apache.org.
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 cd6987f7cb4d076660a867a3789815cf854486ff
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Mon Jul 4 12:36:08 2022 +0200

    Add "generated" folder to volumes mounted when "MOUNT_SELECTED" used (#24818)
    
    (cherry picked from commit 31578583124c5badcd8d3b7411315b1b02d4df68)
---
 dev/breeze/src/airflow_breeze/utils/docker_command_utils.py | 12 ++++++++++--
 scripts/ci/docker-compose/local.yml                         |  4 ++--
 scripts/ci/pre_commit/pre_commit_local_yml_mounts.py        |  9 +++++----
 3 files changed, 17 insertions(+), 8 deletions(-)

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 7b5e37f3b0..c49c0f183f 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -60,7 +60,14 @@ from airflow_breeze.utils.run_utils import (
     run_command,
 )
 
-NECESSARY_HOST_VOLUMES = [
+# Those are volumes that are mounted when MOUNT_SELECTED is chosen (which is the default when
+# entering Breeze. MOUNT_SELECTED prevents to mount the files that you can have accidentally added
+# in your sources (or they were added automatically by setup.py etc.) to be mounted to container.
+# This is important to get a "clean" environment for different python versions and to avoid
+# unnecessary slow-downs when you are mounting files on MacOS (which has very slow filesystem)
+# Any time you add a top-level folder in airflow that should also be added to container you should
+# add it here.
+VOLUMES_FOR_SELECTED_MOUNTS = [
     (".bash_aliases", "/root/.bash_aliases"),
     (".bash_history", "/root/.bash_history"),
     (".coveragerc", "/opt/airflow/.coveragerc"),
@@ -78,6 +85,7 @@ NECESSARY_HOST_VOLUMES = [
     ("dags", "/opt/airflow/dags"),
     ("dev", "/opt/airflow/dev"),
     ("docs", "/opt/airflow/docs"),
+    ("generated", "/opt/airflow/generated"),
     ("hooks", "/opt/airflow/hooks"),
     ("logs", "/root/airflow/logs"),
     ("pyproject.toml", "/opt/airflow/pyproject.toml"),
@@ -105,7 +113,7 @@ def get_extra_docker_flags(mount_sources: str) -> List[str]:
     if mount_sources == MOUNT_ALL:
         extra_docker_flags.extend(["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT},dst=/opt/airflow/"])
     elif mount_sources == MOUNT_SELECTED:
-        for (src, dst) in NECESSARY_HOST_VOLUMES:
+        for (src, dst) in VOLUMES_FOR_SELECTED_MOUNTS:
             if (AIRFLOW_SOURCES_ROOT / src).exists():
                 extra_docker_flags.extend(
                     ["--mount", f'type=bind,src={AIRFLOW_SOURCES_ROOT / src},dst={dst}']
diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml
index 9e63ffa467..7df00f7462 100644
--- a/scripts/ci/docker-compose/local.yml
+++ b/scripts/ci/docker-compose/local.yml
@@ -26,7 +26,7 @@ services:
     # or those that might be useful to see in the host as output of the
     # tests (such as logs)
     volumes:
-      # START automatically generated volumes from NECESSARY_HOST_VOLUMES in docker_command_utils.py
+      # START automatically generated volumes from VOLUMES_FOR_SELECTED_MOUNTS in docker_command_utils.py
       - type: bind
         source: ../../../.bash_aliases
         target: /root/.bash_aliases
@@ -117,4 +117,4 @@ services:
       - type: bind
         source: ../../../metastore_browser
         target: /opt/airflow/metastore_browser
-        # END automatically generated volumes from NECESSARY_HOST_VOLUMES in docker_command_utils.py
+        # END automatically generated volumes from VOLUMES_FOR_SELECTED_MOUNTS in docker_command_utils.py
diff --git a/scripts/ci/pre_commit/pre_commit_local_yml_mounts.py b/scripts/ci/pre_commit/pre_commit_local_yml_mounts.py
index 391214fa29..ceaf5a4db8 100755
--- a/scripts/ci/pre_commit/pre_commit_local_yml_mounts.py
+++ b/scripts/ci/pre_commit/pre_commit_local_yml_mounts.py
@@ -33,19 +33,20 @@ sys.path.append(str(AIRFLOW_SOURCES_DIR))
 
 MOUNTS_HEADER = (
     '        # START automatically generated volumes from '
-    'NECESSARY_HOST_VOLUMES in docker_command_utils.py'
+    'VOLUMES_FOR_SELECTED_MOUNTS in docker_command_utils.py'
 )
 MOUNTS_FOOTER = (
-    '        # END automatically generated volumes from ' 'NECESSARY_HOST_VOLUMES in docker_command_utils.py'
+    '        # END automatically generated volumes from '
+    'VOLUMES_FOR_SELECTED_MOUNTS in docker_command_utils.py'
 )
 
 if __name__ == '__main__':
-    from airflow_breeze.utils.docker_command_utils import NECESSARY_HOST_VOLUMES
+    from airflow_breeze.utils.docker_command_utils import VOLUMES_FOR_SELECTED_MOUNTS
 
     local_mount_file_path = AIRFLOW_SOURCES_DIR / 'scripts' / 'ci' / 'docker-compose' / 'local.yml'
     PREFIX = '      '
     volumes = []
-    for (src, dest) in NECESSARY_HOST_VOLUMES:
+    for (src, dest) in VOLUMES_FOR_SELECTED_MOUNTS:
         volumes.extend(
             [
                 PREFIX + "- type: bind\n",


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

Posted by po...@apache.org.
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