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/06/21 10:30:36 UTC

[airflow] branch switch-to-image-building-in-parallel created (now d95093cfee)

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

potiuk pushed a change to branch switch-to-image-building-in-parallel
in repository https://gitbox.apache.org/repos/asf/airflow.git


      at d95093cfee Switch to building images in parallell

This branch includes the following new commits:

     new d95093cfee Switch to building images in parallell

The 1 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.



[airflow] 01/01: Switch to building images in parallell

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

potiuk pushed a commit to branch switch-to-image-building-in-parallel
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit d95093cfeed4d27cdcca2879c1aa93965d3490af
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Tue Jun 21 12:20:02 2022 +0200

    Switch to building images in parallell
    
    In the new Breeze, switching to using parallelism is a ... breeze.
    
    This PR adds the capability of building the images in parallel in Breeze
    locally - for breeze command, but also uses this capability to build the
    images in parallel in our CI. Our builds are always executed on
    powerful, big machines with lots of CPU and docker run in memory
    filesystem with 32GB RAM, so it should be possible to run all builds in
    parallel on a single machine rather then spin off parallel machines to
    run the builds using the matrix strategy of Github Actions.
    
    Generally speaking - this will either speed up or get 4x cost saving for
    the build steps for all the "full test needed" PRs as well as all the
    main builds.
    
    There are a number of savings and improvements we can achieve this way:
    
    1) less overhead for starting and runnning the machines
    2) seems that with the new buildkit, the parallel builds are not
       suffering from some sequential locks (as it used to be, so
       we are basically do the same job using 25% resources for building
       the images.
    3) we will stop having random "one image failed to build" cases - they
       will all either fail or succeed.
    4) Less checks in the output
    5) Production builds will additionally gain from single CI image
       pulled in order to perform the preparation of the packages
       and single package preparation step - it will save 4-5 minutes
       per image.
    
    The disadvantage is a less clear output of such parallel build where
    outputs from multiple builds will be interleaved in one CI output.
---
 .github/workflows/build-images.yml                 |  38 +--
 .github/workflows/ci.yml                           |  32 +-
 dev/REFRESHING_CI_CACHE.md                         |   2 +-
 .../airflow_breeze/commands/ci_image_commands.py   |  65 ++--
 .../commands/production_image_commands.py          |  67 +++-
 .../src/airflow_breeze/utils/common_options.py     |   6 -
 dev/refresh_images.sh                              |   4 +-
 images/breeze/output-build-image.svg               | 276 +++++++++--------
 images/breeze/output-build-prod-image.svg          | 344 +++++++++++----------
 images/breeze/output-commands-hash.txt             |   2 +-
 scripts/ci/selective_ci_checks.sh                  |   4 +
 11 files changed, 452 insertions(+), 388 deletions(-)

diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml
index 9970a82e6c..a3edf708c6 100644
--- a/.github/workflows/build-images.yml
+++ b/.github/workflows/build-images.yml
@@ -57,6 +57,7 @@ jobs:
       pythonVersions: "${{ steps.selective-checks.python-versions }}"
       upgradeToNewerDependencies: ${{ steps.selective-checks.outputs.upgrade-to-newer-dependencies }}
       allPythonVersions: ${{ steps.selective-checks.outputs.all-python-versions }}
+      allPythonVersionsListAsString: ${{ steps.selective-checks.outputs.all-python-versions-list-as-string }}
       defaultPythonVersion: ${{ steps.selective-checks.outputs.default-python-version }}
       run-tests: ${{ steps.selective-checks.outputs.run-tests }}
       run-kubernetes-tests: ${{ steps.selective-checks.outputs.run-kubernetes-tests }}
@@ -155,19 +156,14 @@ jobs:
     permissions:
       packages: write
     timeout-minutes: 80
-    name: "Build CI image ${{matrix.python-version}}"
+    name: "Build CI images"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info]
-    strategy:
-      matrix:
-        python-version: ${{ fromJson(needs.build-info.outputs.allPythonVersions) }}
-      fail-fast: true
     if: |
       needs.build-info.outputs.image-build == 'true' &&
       github.event.pull_request.head.repo.full_name != 'apache/airflow'
     env:
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn)[0] }}
-      PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       BACKEND: sqlite
       outputs: ${{toJSON(needs.build-info.outputs) }}
     steps:
@@ -226,20 +222,23 @@ jobs:
       - run: ./scripts/ci/install_breeze.sh
       - name: "Free space"
         run: breeze free-space
-      - name: Build & Push CI image ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
-        run: breeze build-image --push-image --tag-as-latest
+      - name: Build & Push CI images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+        run: breeze build-image --push-image --tag-as-latest --run-in-parallel
         env:
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cacheDirective }}
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+          PYTHON_VERSIONS: $${ needs.build-info.outputs.allPythonVersionsAsString }}
       - name: Push empty CI image ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
         if: failure() || cancelled()
-        run: breeze build-image --push-image --empty-image
+        run: breeze build-image --push-image --empty-image --run-in-parallel
         env:
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
-      - name: "Candidates for pip resolver backtrack triggers: ${{ matrix.python-version }}"
+      - name: "Candidates for pip resolver backtrack triggers"
         if: failure() || cancelled()
-        run: breeze find-newer-dependencies --max-age 1 --python "${{ matrix.python-version }}"
+        run: >
+          breeze find-newer-dependencies --max-age 1
+          --python "${{ needs.build-info.outputs.defaultPythonVersion }}"
       - name: "Fix ownership"
         run: breeze fix-ownership
         if: always()
@@ -248,19 +247,14 @@ jobs:
     permissions:
       packages: write
     timeout-minutes: 80
-    name: "Build PROD image ${{matrix.python-version}}"
+    name: "Build PROD images"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, build-ci-images]
-    strategy:
-      matrix:
-        python-version: ${{ fromJson(needs.build-info.outputs.allPythonVersions) }}
-      fail-fast: true
     if: |
       needs.build-info.outputs.image-build == 'true' &&
       github.event.pull_request.head.repo.full_name != 'apache/airflow'
     env:
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn)[0] }}
-      PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       BACKEND: sqlite
     steps:
       - name: Cleanup repo
@@ -320,7 +314,7 @@ jobs:
         run: breeze free-space
       - name: >
           Pull CI image for PROD build:
-          ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+          ${{ needs.build-info.outputs.defaultPythonVersion }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
         run: breeze pull-image --tag-as-latest
         env:
           # Always use default Python version of CI image for preparing packages
@@ -338,9 +332,10 @@ jobs:
         run: breeze prepare-airflow-package --package-format wheel --version-suffix-for-pypi dev0
       - name: "Move dist packages to docker-context files"
         run: mv -v ./dist/*.whl ./docker-context-files
-      - name: Build & Push PROD image ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+      - name: Build & Push PROD images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
         run: >
           breeze build-prod-image
+          --run-in-parallel
           --tag-as-latest
           --push-image
           --install-packages-from-context
@@ -350,9 +345,10 @@ jobs:
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cacheDirective }}
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
-      - name: Push empty PROD image ${{ env.PYTHON_MAJOR_MINOR_VERSION }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+          PYTHON_VERSIONS: $${ needs.build-info.outputs.allPythonVersionsAsString }}
+      - name: Push empty PROD images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
         if: failure() || cancelled()
-        run: breeze build-prod-image --cleanup-context --push-image --empty-image
+        run: breeze build-prod-image --cleanup-context --push-image --empty-image --run-in-parallel
         env:
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
       - name: "Fix ownership"
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index c112a6da72..82dc7d44f4 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -132,6 +132,7 @@ jobs:
       upgradeToNewerDependencies: ${{ steps.selective-checks.outputs.upgrade-to-newer-dependencies }}
       pythonVersions: ${{ steps.selective-checks.outputs.python-versions }}
       pythonVersionsListAsString: ${{ steps.selective-checks.outputs.python-versions-list-as-string }}
+      allPythonVersionsListAsString: ${{ steps.selective-checks.outputs.all-python-versions-list-as-string }}
       defaultPythonVersion: ${{ steps.selective-checks.outputs.default-python-version }}
       kubernetesVersions: ${{ steps.selective-checks.outputs.kubernetes-versions }}
       kubernetesVersionsListAsString: ${{ steps.selective-checks.outputs.kubernetes-versions-list-as-string }}
@@ -285,13 +286,9 @@ jobs:
     permissions:
       packages: write
     timeout-minutes: 80
-    name: "${{needs.build-info.outputs.buildJobDescription}} CI image ${{matrix.python-version}}"
+    name: "${{needs.build-info.outputs.buildJobDescription}} CI images"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info]
-    strategy:
-      matrix:
-        python-version: ${{ fromJson(needs.build-info.outputs.allPythonVersions) }}
-      fail-fast: true
     env:
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn)[0] }}
     steps:
@@ -311,7 +308,7 @@ jobs:
         if: needs.build-info.outputs.inWorkflowBuild == 'true'
       - name: "Retrieve DEFAULTS from the _initialization.sh"
         # We cannot "source" the script here because that would be a security problem (we cannot run
-        # any code that comes from the sources coming from the PR. Therefore we extract the
+        # any code that comes from the sources coming from the PR. Therefore, we extract the
         # DEFAULT_BRANCH and DEFAULT_CONSTRAINTS_BRANCH and DEBIAN_VERSION via custom grep/awk/sed commands
         id: defaults
         run: |
@@ -331,17 +328,19 @@ jobs:
       - name: "Free space"
         run: breeze free-space
         if: needs.build-info.outputs.inWorkflowBuild == 'true'
-      - name: Build & Push CI image ${{ matrix.python-version }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
-        run: breeze build-image --push-image --tag-as-latest
+      - name: Build & Push CI images ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+        run: breeze build-image --push-image --tag-as-latest --run-in-parallel
         env:
-          PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cacheDirective }}
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+          PYTHON_VERSIONS: $${ needs.build-info.outputs.allPythonVersionsAsString }}
         if: needs.build-info.outputs.inWorkflowBuild == 'true'
-      - name: "Candidates for pip resolver backtrack triggers: ${{ matrix.python-version }}"
+      - name: "Candidates for pip resolver backtrack triggers"
         if: failure() || cancelled()
-        run: breeze find-newer-dependencies --max-age 1 --python "${{ matrix.python-version }}"
+        run: >
+          breeze find-newer-dependencies --max-age 1
+          --python "${{ needs.build-info.outputs.defaultPythonVersion }}"
       - name: "Fix ownership"
         run: breeze fix-ownership
         if: always() && needs.build-info.outputs.inWorkflowBuild == 'true'
@@ -350,17 +349,12 @@ jobs:
     permissions:
       packages: write
     timeout-minutes: 80
-    name: "${{needs.build-info.outputs.buildJobDescription}} PROD image ${{matrix.python-version}}"
+    name: "${{needs.build-info.outputs.buildJobDescription}} PROD images"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, build-ci-images]
-    strategy:
-      matrix:
-        python-version: ${{ fromJson(needs.build-info.outputs.allPythonVersions) }}
-      fail-fast: true
     env:
       RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn)[0] }}
       BACKEND: sqlite
-      PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       DOCKER_CACHE: ${{ needs.build-info.outputs.cacheDirective }}
       VERSION_SUFFIX_FOR_PYPI: "dev0"
     steps:
@@ -428,6 +422,7 @@ jobs:
         run: >
           breeze build-prod-image
           --tag-as-latest
+          --run-in-parallel
           --push-image
           --install-packages-from-context
           --disable-airflow-repo-cache
@@ -436,6 +431,7 @@ jobs:
           UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
           DOCKER_CACHE: ${{ needs.build-info.outputs.cacheDirective }}
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
+          PYTHON_VERSIONS: $${ needs.build-info.outputs.allPythonVersionsAsString }}
         if: needs.build-info.outputs.inWorkflowBuild == 'true'
       - name: "Fix ownership"
         run: breeze fix-ownership
@@ -1385,7 +1381,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           key: ${{ runner.os }}-docker-venv-${{ hashFiles('scripts/ci/images/ci_run_docker_tests.py') }}
       - name: Wait for PROD images ${{ env.PYTHON_VERSIONS }}:${{ env.IMAGE_TAG_FOR_THE_BUILD }}
         # We wait for the images to be available either from "build-images.yml' run as pull_request_target
-        # or from build-prod-image above.
+        # or from build-prod-images above.
         # We are utilising single job to wait for all images because this job merely waits
         # For the images to be available and test them.
         run: breeze pull-prod-image --verify-image --wait-for-image --run-in-parallel
diff --git a/dev/REFRESHING_CI_CACHE.md b/dev/REFRESHING_CI_CACHE.md
index 85194845dc..5fd458bf34 100644
--- a/dev/REFRESHING_CI_CACHE.md
+++ b/dev/REFRESHING_CI_CACHE.md
@@ -51,7 +51,7 @@ manual refresh might be needed.
 # Manually generating constraint files
 
 ```bash
-breeze build-image --build-multiple-images --upgrade-to-newer-dependencies --answer yes
+breeze build-image --run-in-parallel --upgrade-to-newer-dependencies --answer yes
 breeze generate-constraints --airflow-constraints-mode constraints --run-in-parallel --answer yes
 breeze generate-constraints --airflow-constraints-mode constraints-source-providers --run-in-parallel --answer yes
 breeze generate-constraints --airflow-constraints-mode constraints-no-providers --run-in-parallel --answer yes
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 9deac3a276..6c42f0846e 100644
--- a/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
@@ -37,7 +37,6 @@ from airflow_breeze.utils.common_options import (
     option_airflow_constraints_mode_ci,
     option_airflow_constraints_reference_build,
     option_answer,
-    option_build_multiple_images,
     option_debian_version,
     option_dev_apt_command,
     option_dev_apt_deps,
@@ -115,9 +114,10 @@ CI_IMAGE_TOOLS_PARAMETERS = {
             ],
         },
         {
-            "name": "Building multiple images",
+            "name": "Building images in parallel",
             "options": [
-                "--build-multiple-images",
+                "--run-in-parallel",
+                "--parallelism",
                 "--python-versions",
             ],
         },
@@ -189,13 +189,36 @@ CI_IMAGE_TOOLS_PARAMETERS = {
 }
 
 
+def run_build_in_parallel(
+    image_params_list: List[BuildCiParams],
+    python_version_list: List[str],
+    parallelism: int,
+    dry_run: bool,
+    verbose: bool,
+) -> None:
+    get_console().print(
+        f"\n[info]Building with parallelism = {parallelism} for the images: {python_version_list}:"
+    )
+    pool = mp.Pool(parallelism)
+    results = [
+        pool.apply_async(
+            run_build_ci_image,
+            args=(verbose, dry_run, image_param, True),
+        )
+        for image_param in image_params_list
+    ]
+    check_async_run_results(results)
+    pool.close()
+
+
 @main.command(name='build-image')
 @option_github_repository
 @option_verbose
 @option_dry_run
 @option_answer
 @option_python
-@option_build_multiple_images
+@option_run_in_parallel
+@option_parallelism
 @option_python_versions
 @option_upgrade_to_newer_dependencies
 @option_platform
@@ -228,7 +251,8 @@ CI_IMAGE_TOOLS_PARAMETERS = {
 def build_image(
     verbose: bool,
     dry_run: bool,
-    build_multiple_images: bool,
+    run_in_parallel: bool,
+    parallelism: int,
     python_versions: str,
     answer: str,
     **kwargs,
@@ -236,7 +260,7 @@ def build_image(
     """Build CI image. Include building multiple images for all python versions (sequentially)."""
 
     def run_build(ci_image_params: BuildCiParams) -> None:
-        return_code, info = build_ci_image(
+        return_code, info = run_build_ci_image(
             verbose=verbose, dry_run=dry_run, ci_image_params=ci_image_params, parallel=False
         )
         if return_code != 0:
@@ -246,13 +270,21 @@ def build_image(
     perform_environment_checks(verbose=verbose)
     parameters_passed = filter_out_none(**kwargs)
     parameters_passed['force_build'] = True
-    if build_multiple_images:
+    if run_in_parallel:
         python_version_list = get_python_version_list(python_versions)
+        params_list: List[BuildCiParams] = []
         for python in python_version_list:
             params = BuildCiParams(**parameters_passed)
             params.python = python
             params.answer = answer
-            run_build(ci_image_params=params)
+            params_list.append(params)
+        run_build_in_parallel(
+            image_params_list=params_list,
+            python_version_list=python_version_list,
+            parallelism=parallelism,
+            dry_run=dry_run,
+            verbose=verbose,
+        )
     else:
         params = BuildCiParams(**parameters_passed)
         run_build(ci_image_params=params)
@@ -434,7 +466,7 @@ def should_we_run_the_build(build_ci_params: BuildCiParams) -> bool:
         sys.exit(1)
 
 
-def build_ci_image(
+def run_build_ci_image(
     verbose: bool, dry_run: bool, ci_image_params: BuildCiParams, parallel: bool
 ) -> Tuple[int, str]:
     """
@@ -522,19 +554,6 @@ def build_ci_image(
     return build_command_result.returncode, f"Image build: {ci_image_params.python}"
 
 
-def build_ci_image_in_parallel(
-    verbose: bool, dry_run: bool, parallelism: int, python_version_list: List[str], **kwargs
-):
-    """Run CI image builds in parallel."""
-    get_console().print(
-        f"\n[info]Running with parallelism = {parallelism} for the images: {python_version_list}:"
-    )
-    pool = mp.Pool(parallelism)
-    results = [pool.apply_async(build_ci_image, args=(verbose, dry_run, False), kwds=kwargs)]
-    check_async_run_results(results)
-    pool.close()
-
-
 def rebuild_or_pull_ci_image_if_needed(
     command_params: Union[ShellParams, BuildCiParams], dry_run: bool, verbose: bool
 ) -> None:
@@ -573,4 +592,4 @@ def rebuild_or_pull_ci_image_if_needed(
             'Forcing build.[/]'
         )
         ci_image_params.force_build = True
-    build_ci_image(verbose, dry_run=dry_run, ci_image_params=ci_image_params, parallel=False)
+    run_build_ci_image(verbose, dry_run=dry_run, ci_image_params=ci_image_params, parallel=False)
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 0b5ed2e68b..c98446ffcd 100644
--- a/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/production_image_commands.py
@@ -15,9 +15,10 @@
 # specific language governing permissions and limitations
 # under the License.
 import contextlib
+import multiprocessing as mp
 import os
 import sys
-from typing import Optional, Tuple
+from typing import List, Optional, Tuple
 
 import click
 
@@ -36,7 +37,6 @@ from airflow_breeze.utils.common_options import (
     option_airflow_constraints_mode_prod,
     option_airflow_constraints_reference_build,
     option_answer,
-    option_build_multiple_images,
     option_debian_version,
     option_dev_apt_command,
     option_dev_apt_deps,
@@ -74,6 +74,7 @@ from airflow_breeze.utils.docker_command_utils import (
     prepare_docker_build_from_input,
 )
 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
 from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, DOCKER_CONTEXT_DIR
 from airflow_breeze.utils.python_versions import get_python_version_list
 from airflow_breeze.utils.registry import login_to_github_docker_registry
@@ -103,9 +104,10 @@ PRODUCTION_IMAGE_TOOLS_PARAMETERS = {
             ],
         },
         {
-            "name": "Building multiple images",
+            "name": "Building images in parallel",
             "options": [
-                "--build-multiple-images",
+                "--run-in-parallel",
+                "--parallelism",
                 "--python-versions",
             ],
         },
@@ -192,12 +194,40 @@ PRODUCTION_IMAGE_TOOLS_PARAMETERS = {
 }
 
 
+def run_build_in_parallel(
+    image_params_list: List[BuildProdParams],
+    python_version_list: List[str],
+    parallelism: int,
+    dry_run: bool,
+    verbose: bool,
+) -> None:
+    get_console().print(
+        f"\n[info]Building with parallelism = {parallelism} for the images: {python_version_list}:"
+    )
+    pool = mp.Pool(parallelism)
+    results = [
+        pool.apply_async(
+            run_build_production_image,
+            args=(
+                verbose,
+                dry_run,
+                image_param,
+                True,
+            ),
+        )
+        for image_param in image_params_list
+    ]
+    check_async_run_results(results)
+    pool.close()
+
+
 @option_verbose
 @option_dry_run
 @option_answer
 @main.command(name='build-prod-image')
 @option_python
-@option_build_multiple_images
+@option_run_in_parallel
+@option_parallelism
 @option_python_versions
 @option_upgrade_to_newer_dependencies
 @option_platform
@@ -269,7 +299,8 @@ PRODUCTION_IMAGE_TOOLS_PARAMETERS = {
 def build_prod_image(
     verbose: bool,
     dry_run: bool,
-    build_multiple_images: bool,
+    run_in_parallel: bool,
+    parallelism: int,
     python_versions: str,
     answer: Optional[str],
     **kwargs,
@@ -279,8 +310,8 @@ def build_prod_image(
     """
 
     def run_build(prod_image_params: BuildProdParams) -> None:
-        return_code, info = build_production_image(
-            verbose=verbose, dry_run=dry_run, prod_image_params=prod_image_params
+        return_code, info = run_build_production_image(
+            verbose=verbose, dry_run=dry_run, prod_image_params=prod_image_params, parallel=False
         )
         if return_code != 0:
             get_console().print(f"[error]Error when building image! {info}")
@@ -288,13 +319,21 @@ def build_prod_image(
 
     perform_environment_checks(verbose=verbose)
     parameters_passed = filter_out_none(**kwargs)
-    if build_multiple_images:
+    if run_in_parallel:
         python_version_list = get_python_version_list(python_versions)
+        params_list: List[BuildProdParams] = []
         for python in python_version_list:
             params = BuildProdParams(**parameters_passed)
             params.python = python
             params.answer = answer
-            run_build(prod_image_params=params)
+            params_list.append(params)
+        run_build_in_parallel(
+            image_params_list=params_list,
+            python_version_list=python_version_list,
+            parallelism=parallelism,
+            dry_run=dry_run,
+            verbose=verbose,
+        )
     else:
         params = BuildProdParams(**parameters_passed)
         run_build(prod_image_params=params)
@@ -473,8 +512,8 @@ def check_docker_context_files(install_packages_from_context: bool):
             sys.exit(1)
 
 
-def build_production_image(
-    verbose: bool, dry_run: bool, prod_image_params: BuildProdParams
+def run_build_production_image(
+    verbose: bool, dry_run: bool, prod_image_params: BuildProdParams, parallel: bool
 ) -> Tuple[int, str]:
     """
     Builds PROD image:
@@ -519,7 +558,7 @@ def build_production_image(
     get_console().print(f"\n[info]Building PROD Image for Python {prod_image_params.python}\n")
     if prod_image_params.prepare_buildx_cache:
         build_command_result = build_cache(
-            image_params=prod_image_params, dry_run=dry_run, verbose=verbose, parallel=False
+            image_params=prod_image_params, dry_run=dry_run, verbose=verbose, parallel=parallel
         )
     else:
         if prod_image_params.empty_image:
@@ -547,7 +586,7 @@ def build_production_image(
                 cwd=AIRFLOW_SOURCES_ROOT,
                 check=False,
                 text=True,
-                enabled_output_group=True,
+                enabled_output_group=not parallel,
             )
             if build_command_result.returncode == 0:
                 if prod_image_params.tag_as_latest:
diff --git a/dev/breeze/src/airflow_breeze/utils/common_options.py b/dev/breeze/src/airflow_breeze/utils/common_options.py
index 0bc4d5af9f..292e74891d 100644
--- a/dev/breeze/src/airflow_breeze/utils/common_options.py
+++ b/dev/breeze/src/airflow_breeze/utils/common_options.py
@@ -377,12 +377,6 @@ option_parallelism = click.option(
     envvar='PARALLELISM',
     show_default=True,
 )
-option_build_multiple_images = click.option(
-    '--build-multiple-images',
-    help="Run the operation sequentially on all or selected subset of Python versions.",
-    is_flag=True,
-    envvar='BUILD_MULTIPLE_IMAGES',
-)
 argument_packages = click.argument(
     "packages",
     nargs=-1,
diff --git a/dev/refresh_images.sh b/dev/refresh_images.sh
index ed5591585a..4a11212b2c 100755
--- a/dev/refresh_images.sh
+++ b/dev/refresh_images.sh
@@ -26,7 +26,7 @@ export GITHUB_TOKEN=""
 breeze self-upgrade --force
 
 breeze build-image \
-     --build-multiple-images \
+     --run-in-parallel \
      --prepare-buildx-cache \
      --force-build \
      --platform linux/amd64,linux/arm64 \
@@ -44,7 +44,7 @@ breeze prepare-airflow-package --package-format wheel --version-suffix-for-pypi
 mv -v ./dist/*.whl ./docker-context-files
 
 breeze build-prod-image \
-     --build-multiple-images \
+     --run-in-parallel \
      --airflow-is-in-context \
      --install-packages-from-context \
      --prepare-buildx-cache \
diff --git a/images/breeze/output-build-image.svg b/images/breeze/output-build-image.svg
index 81703735d0..389c973d0e 100644
--- a/images/breeze/output-build-image.svg
+++ b/images/breeze/output-build-image.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 1514.0" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 1562.8" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -19,281 +19,289 @@
         font-weight: 700;
     }
 
-    .terminal-1489702336-matrix {
+    .terminal-4238821259-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-1489702336-title {
+    .terminal-4238821259-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-1489702336-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1489702336-r2 { fill: #c5c8c6 }
-.terminal-1489702336-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1489702336-r4 { fill: #868887 }
-.terminal-1489702336-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1489702336-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1489702336-r7 { fill: #8d7b39 }
+    .terminal-4238821259-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-4238821259-r2 { fill: #c5c8c6 }
+.terminal-4238821259-r3 { fill: #d0b344;font-weight: bold }
+.terminal-4238821259-r4 { fill: #868887 }
+.terminal-4238821259-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-4238821259-r6 { fill: #98a84b;font-weight: bold }
+.terminal-4238821259-r7 { fill: #8d7b39 }
     </style>
 
     <defs>
-    <clipPath id="terminal-1489702336-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="1463.0" />
+    <clipPath id="terminal-4238821259-clip-terminal">
+      <rect x="0" y="0" width="1463.0" height="1511.8" />
     </clipPath>
-    <clipPath id="terminal-1489702336-line-0">
+    <clipPath id="terminal-4238821259-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-1">
+<clipPath id="terminal-4238821259-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-2">
+<clipPath id="terminal-4238821259-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-3">
+<clipPath id="terminal-4238821259-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-4">
+<clipPath id="terminal-4238821259-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-5">
+<clipPath id="terminal-4238821259-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-6">
+<clipPath id="terminal-4238821259-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-7">
+<clipPath id="terminal-4238821259-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-8">
+<clipPath id="terminal-4238821259-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-9">
+<clipPath id="terminal-4238821259-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-10">
+<clipPath id="terminal-4238821259-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-11">
+<clipPath id="terminal-4238821259-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-12">
+<clipPath id="terminal-4238821259-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-13">
+<clipPath id="terminal-4238821259-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-14">
+<clipPath id="terminal-4238821259-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-15">
+<clipPath id="terminal-4238821259-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-16">
+<clipPath id="terminal-4238821259-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-17">
+<clipPath id="terminal-4238821259-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-18">
+<clipPath id="terminal-4238821259-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-19">
+<clipPath id="terminal-4238821259-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-20">
+<clipPath id="terminal-4238821259-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-21">
+<clipPath id="terminal-4238821259-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-22">
+<clipPath id="terminal-4238821259-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-23">
+<clipPath id="terminal-4238821259-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-24">
+<clipPath id="terminal-4238821259-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-25">
+<clipPath id="terminal-4238821259-line-25">
     <rect x="0" y="611.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-26">
+<clipPath id="terminal-4238821259-line-26">
     <rect x="0" y="635.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-27">
+<clipPath id="terminal-4238821259-line-27">
     <rect x="0" y="660.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-28">
+<clipPath id="terminal-4238821259-line-28">
     <rect x="0" y="684.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-29">
+<clipPath id="terminal-4238821259-line-29">
     <rect x="0" y="709.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-30">
+<clipPath id="terminal-4238821259-line-30">
     <rect x="0" y="733.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-31">
+<clipPath id="terminal-4238821259-line-31">
     <rect x="0" y="757.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-32">
+<clipPath id="terminal-4238821259-line-32">
     <rect x="0" y="782.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-33">
+<clipPath id="terminal-4238821259-line-33">
     <rect x="0" y="806.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-34">
+<clipPath id="terminal-4238821259-line-34">
     <rect x="0" y="831.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-35">
+<clipPath id="terminal-4238821259-line-35">
     <rect x="0" y="855.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-36">
+<clipPath id="terminal-4238821259-line-36">
     <rect x="0" y="879.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-37">
+<clipPath id="terminal-4238821259-line-37">
     <rect x="0" y="904.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-38">
+<clipPath id="terminal-4238821259-line-38">
     <rect x="0" y="928.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-39">
+<clipPath id="terminal-4238821259-line-39">
     <rect x="0" y="953.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-40">
+<clipPath id="terminal-4238821259-line-40">
     <rect x="0" y="977.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-41">
+<clipPath id="terminal-4238821259-line-41">
     <rect x="0" y="1001.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-42">
+<clipPath id="terminal-4238821259-line-42">
     <rect x="0" y="1026.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-43">
+<clipPath id="terminal-4238821259-line-43">
     <rect x="0" y="1050.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-44">
+<clipPath id="terminal-4238821259-line-44">
     <rect x="0" y="1075.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-45">
+<clipPath id="terminal-4238821259-line-45">
     <rect x="0" y="1099.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-46">
+<clipPath id="terminal-4238821259-line-46">
     <rect x="0" y="1123.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-47">
+<clipPath id="terminal-4238821259-line-47">
     <rect x="0" y="1148.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-48">
+<clipPath id="terminal-4238821259-line-48">
     <rect x="0" y="1172.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-49">
+<clipPath id="terminal-4238821259-line-49">
     <rect x="0" y="1197.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-50">
+<clipPath id="terminal-4238821259-line-50">
     <rect x="0" y="1221.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-51">
+<clipPath id="terminal-4238821259-line-51">
     <rect x="0" y="1245.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-52">
+<clipPath id="terminal-4238821259-line-52">
     <rect x="0" y="1270.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-53">
+<clipPath id="terminal-4238821259-line-53">
     <rect x="0" y="1294.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-54">
+<clipPath id="terminal-4238821259-line-54">
     <rect x="0" y="1319.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-55">
+<clipPath id="terminal-4238821259-line-55">
     <rect x="0" y="1343.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-56">
+<clipPath id="terminal-4238821259-line-56">
     <rect x="0" y="1367.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-57">
+<clipPath id="terminal-4238821259-line-57">
     <rect x="0" y="1392.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1489702336-line-58">
+<clipPath id="terminal-4238821259-line-58">
     <rect x="0" y="1416.7" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="terminal-4238821259-line-59">
+    <rect x="0" y="1441.1" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="terminal-4238821259-line-60">
+    <rect x="0" y="1465.5" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1512" rx="8"/><text class="terminal-1489702336-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-image</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1560.8" rx="8"/><text class="terminal-4238821259-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-image</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
             <circle cx="44" cy="0" r="7" fill="#28c840"/>
             </g>
         
-    <g transform="translate(9, 41)" clip-path="url(#terminal-1489702336-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-4238821259-clip-terminal)">
     
-    <g class="terminal-1489702336-matrix">
-    <text class="terminal-1489702336-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1489702336-line-0)">
-</text><text class="terminal-1489702336-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1489702336-line-1)">Usage:&#160;</text><text class="terminal-1489702336-r1" x="97.6" y="44.4" textLength="341.6" clip-path="url(#terminal-1489702336-line-1)">breeze&#160;build-image&#160;[OPTIONS]</text><text class="terminal-1489702336-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-1)">
-</text><text class="terminal-1489702336-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-2)">
-</text><text class="terminal-1489702336-r2" x="12.2" y="93.2" textLength="1073.6" clip-path="url(#terminal-1489702336-line-3)">Build&#160;CI&#160;image.&#160;Include&#160;building&#160;multiple&#160;images&#160;for&#160;all&#160;python&#160;versions&#160;(sequentially).</text><text class="terminal-1489702336-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-3)">
-</text><text class="terminal-1489702336-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-4)">
-</text><text class="terminal-1489702336-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-1489702336-line-5)">╭─</text><text class="terminal-1489702336-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-1489702336-line-5)">&#160;Basic&#160;usage&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1489702336-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-148970 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-6)">│</text><text class="terminal-1489702336-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-6)">-</text><text class="terminal-1489702336-r5" x="36.6" y="166.4" textLength="85.4" clip-path="url(#terminal-1489702336-line-6)">-python</text><text class="terminal-1489702336-r6" x="427" y="166.4" textLength="24.4" clip-path="url(#terminal-148 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-7)">│</text><text class="terminal-1489702336-r7" x="475.8" y="190.8" textLength="732" clip-path="url(#terminal-1489702336-line-7)">(&gt;3.7&lt;&#160;|&#160;3.8&#160;|&#160;3.9&#160;|&#160;3.10)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="terminal-1489702336-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-8)">│</text><text class="terminal-1489702336-r4" x="475.8" y="215.2" textLength="732" clip-path="url(#terminal-1489702336-line-8)">[default:&#160;3.7]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="terminal-1489702336-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-9)">│</text><text class="terminal-1489702336-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-9)">-</text><text class="terminal-1489702336-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-1489702336-line-9)">-upgrade</text><text class="terminal-1489702336-r5" x="134.2" y="239.6" textLength="268.4" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-1489702336-line-10)">│</text><text class="terminal-1489702336-r5" x="24.4" y="264" textLength="12.2" clip-path="url(#terminal-1489702336-line-10)">-</text><text class="terminal-1489702336-r5" x="36.6" y="264" textLength="85.4" clip-path="url(#terminal-1489702336-line-10)">-debian</text><text class="terminal-1489702336-r5" x="122" y="264" textLength="97.6" clip-path="url(#terminal-14897023 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-11)">│</text><text class="terminal-1489702336-r5" x="24.4" y="288.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-11)">-</text><text class="terminal-1489702336-r5" x="36.6" y="288.4" textLength="73.2" clip-path="url(#terminal-1489702336-line-11)">-image</text><text class="terminal-1489702336-r5" x="109.8" y="288.4" textLength="48.8" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-12)">│</text><text class="terminal-1489702336-r5" x="475.8" y="312.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-12)">-</text><text class="terminal-1489702336-r5" x="488" y="312.8" textLength="73.2" clip-path="url(#terminal-1489702336-line-12)">-mount</text><text class="terminal-1489702336-r5" x="561.2" y="312.8" textLength="97.6" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-13)">│</text><text class="terminal-1489702336-r7" x="475.8" y="337.2" textLength="963.8" clip-path="url(#terminal-1489702336-line-13)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-14)">│</text><text class="terminal-1489702336-r5" x="24.4" y="361.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-14)">-</text><text class="terminal-1489702336-r5" x="36.6" y="361.6" textLength="48.8" clip-path="url(#terminal-1489702336-line-14)">-tag</text><text class="terminal-1489702336-r5" x="85.4" y="361.6" textLength="122" clip-path="url(#terminal-148 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-1489702336-line-15)">│</text><text class="terminal-1489702336-r2" x="475.8" y="386" textLength="414.8" clip-path="url(#terminal-1489702336-line-15)">when&#160;you&#160;build&#160;or&#160;pull&#160;image&#160;with&#160;</text><text class="terminal-1489702336-r5" x="890.6" y="386" textLength="12.2" clip-path="url(#terminal-1489702336-line-15)">-</text><text class="terminal-1489702336-r5" x= [...]
-</text><text class="terminal-1489702336-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-16)">│</text><text class="terminal-1489702336-r5" x="24.4" y="410.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-16)">-</text><text class="terminal-1489702336-r5" x="36.6" y="410.4" textLength="85.4" clip-path="url(#terminal-1489702336-line-16)">-docker</text><text class="terminal-1489702336-r5" x="122" y="410.4" textLength="73.2" clip-path="url(#terminal- [...]
-</text><text class="terminal-1489702336-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-17)">│</text><text class="terminal-1489702336-r4" x="475.8" y="434.8" textLength="549" clip-path="url(#terminal-1489702336-line-17)">[default:&#160;registry]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1489702336-r4" x="1451.8" y="43 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-18)">│</text><text class="terminal-1489702336-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-18)">-</text><text class="terminal-1489702336-r5" x="36.6" y="459.2" textLength="73.2" clip-path="url(#terminal-1489702336-line-18)">-force</text><text class="terminal-1489702336-r5" x="109.8" y="459.2" textLength="73.2" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-1489702336-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1489702336-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-19)">
-</text><text class="terminal-1489702336-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-1489702336-line-20)">╭─</text><text class="terminal-1489702336-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-1489702336-line-20)">&#160;Building&#160;multiple&#160;images&#160;──────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1489702336-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-21)">│</text><text class="terminal-1489702336-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-21)">-</text><text class="terminal-1489702336-r5" x="36.6" y="532.4" textLength="73.2" clip-path="url(#terminal-1489702336-line-21)">-build</text><text class="terminal-1489702336-r5" x="109.8" y="532.4" textLength="195.2" clip-path="url(#termina [...]
-</text><text class="terminal-1489702336-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-22)">│</text><text class="terminal-1489702336-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-22)">-</text><text class="terminal-1489702336-r5" x="36.6" y="556.8" textLength="85.4" clip-path="url(#terminal-1489702336-line-22)">-python</text><text class="terminal-1489702336-r5" x="122" y="556.8" textLength="109.8" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-23)">│</text><text class="terminal-1489702336-r4" x="353.8" y="581.2" textLength="951.6" clip-path="url(#terminal-1489702336-line-23)">[default:&#160;3.7&#160;3.8&#160;3.9&#160;3.10]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="605.6" textLength="1464" clip-path="url(#terminal-1489702336-line-24)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1489702336-r2" x="1464" y="605.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-24)">
-</text><text class="terminal-1489702336-r4" x="0" y="630" textLength="24.4" clip-path="url(#terminal-1489702336-line-25)">╭─</text><text class="terminal-1489702336-r4" x="24.4" y="630" textLength="1415.2" clip-path="url(#terminal-1489702336-line-25)">&#160;Advanced&#160;options&#160;(for&#160;power&#160;users)&#160;────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1489702336-r4" x="1439.6" y="630" textLength="24.4" clip-path="url [...]
-</text><text class="terminal-1489702336-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-26)">│</text><text class="terminal-1489702336-r5" x="24.4" y="654.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-26)">-</text><text class="terminal-1489702336-r5" x="36.6" y="654.4" textLength="97.6" clip-path="url(#terminal-1489702336-line-26)">-install</text><text class="terminal-1489702336-r5" x="134.2" y="654.4" textLength="280.6" clip-path="url(#termi [...]
-</text><text class="terminal-1489702336-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-27)">│</text><text class="terminal-1489702336-r5" x="24.4" y="678.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-27)">-</text><text class="terminal-1489702336-r5" x="36.6" y="678.8" textLength="97.6" clip-path="url(#terminal-1489702336-line-27)">-airflow</text><text class="terminal-1489702336-r5" x="134.2" y="678.8" textLength="207.4" clip-path="url(#termi [...]
-</text><text class="terminal-1489702336-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-28)">│</text><text class="terminal-1489702336-r7" x="463.6" y="703.2" textLength="866.2" clip-path="url(#terminal-1489702336-line-28)">(constraints-source-providers&#160;|&#160;constraints&#160;|&#160;constraints-no-providers)</text><text class="terminal-1489702336-r4" x="1451.8" y="703.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-28)">│</text><text clas [...]
-</text><text class="terminal-1489702336-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-29)">│</text><text class="terminal-1489702336-r4" x="463.6" y="727.6" textLength="866.2" clip-path="url(#terminal-1489702336-line-29)">[default:&#160;constraints-source-providers]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</ [...]
-</text><text class="terminal-1489702336-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-1489702336-line-30)">│</text><text class="terminal-1489702336-r5" x="24.4" y="752" textLength="12.2" clip-path="url(#terminal-1489702336-line-30)">-</text><text class="terminal-1489702336-r5" x="36.6" y="752" textLength="97.6" clip-path="url(#terminal-1489702336-line-30)">-airflow</text><text class="terminal-1489702336-r5" x="134.2" y="752" textLength="268.4" clip-path="url(#terminal-1489 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-31)">│</text><text class="terminal-1489702336-r5" x="24.4" y="776.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-31)">-</text><text class="terminal-1489702336-r5" x="36.6" y="776.4" textLength="134.2" clip-path="url(#terminal-1489702336-line-31)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="776.4" textLength="146.4" clip-path="url(#t [...]
-</text><text class="terminal-1489702336-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-32)">│</text><text class="terminal-1489702336-r5" x="24.4" y="800.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-32)">-</text><text class="terminal-1489702336-r5" x="36.6" y="800.8" textLength="97.6" clip-path="url(#terminal-1489702336-line-32)">-runtime</text><text class="terminal-1489702336-r5" x="134.2" y="800.8" textLength="109.8" clip-path="url(#termi [...]
-</text><text class="terminal-1489702336-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-33)">│</text><text class="terminal-1489702336-r5" x="24.4" y="825.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-33)">-</text><text class="terminal-1489702336-r5" x="36.6" y="825.2" textLength="97.6" clip-path="url(#terminal-1489702336-line-33)">-runtime</text><text class="terminal-1489702336-r5" x="134.2" y="825.2" textLength="146.4" clip-path="url(#termi [...]
-</text><text class="terminal-1489702336-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-34)">│</text><text class="terminal-1489702336-r5" x="24.4" y="849.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-34)">-</text><text class="terminal-1489702336-r5" x="36.6" y="849.6" textLength="134.2" clip-path="url(#terminal-1489702336-line-34)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="849.6" textLength="85.4" clip-path="url(#te [...]
-</text><text class="terminal-1489702336-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-1489702336-line-35)">│</text><text class="terminal-1489702336-r5" x="24.4" y="874" textLength="12.2" clip-path="url(#terminal-1489702336-line-35)">-</text><text class="terminal-1489702336-r5" x="36.6" y="874" textLength="134.2" clip-path="url(#terminal-1489702336-line-35)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="874" textLength="207.4" clip-path="url(#terminal- [...]
-</text><text class="terminal-1489702336-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-36)">│</text><text class="terminal-1489702336-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-36)">-</text><text class="terminal-1489702336-r5" x="36.6" y="898.4" textLength="134.2" clip-path="url(#terminal-1489702336-line-36)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="898.4" textLength="195.2" clip-path="url(#t [...]
-</text><text class="terminal-1489702336-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-37)">│</text><text class="terminal-1489702336-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-37)">-</text><text class="terminal-1489702336-r5" x="36.6" y="922.8" textLength="134.2" clip-path="url(#terminal-1489702336-line-37)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="922.8" textLength="244" clip-path="url(#ter [...]
-</text><text class="terminal-1489702336-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-38)">│</text><text class="terminal-1489702336-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-38)">-</text><text class="terminal-1489702336-r5" x="36.6" y="947.2" textLength="134.2" clip-path="url(#terminal-1489702336-line-38)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="947.2" textLength="158.6" clip-path="url(#t [...]
-</text><text class="terminal-1489702336-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-39)">│</text><text class="terminal-1489702336-r5" x="24.4" y="971.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-39)">-</text><text class="terminal-1489702336-r5" x="36.6" y="971.6" textLength="134.2" clip-path="url(#terminal-1489702336-line-39)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="971.6" textLength="146.4" clip-path="url(#t [...]
-</text><text class="terminal-1489702336-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-1489702336-line-40)">│</text><text class="terminal-1489702336-r5" x="24.4" y="996" textLength="12.2" clip-path="url(#terminal-1489702336-line-40)">-</text><text class="terminal-1489702336-r5" x="36.6" y="996" textLength="134.2" clip-path="url(#terminal-1489702336-line-40)">-additional</text><text class="terminal-1489702336-r5" x="170.8" y="996" textLength="195.2" clip-path="url(#terminal- [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-41)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1020.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-41)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1020.4" textLength="48.8" clip-path="url(#terminal-1489702336-line-41)">-dev</text><text class="terminal-1489702336-r5" x="85.4" y="1020.4" textLength="109.8" clip-path="url(#termin [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-42)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-42)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1044.8" textLength="48.8" clip-path="url(#terminal-1489702336-line-42)">-dev</text><text class="terminal-1489702336-r5" x="85.4" y="1044.8" textLength="146.4" clip-path="url(#termin [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1069.2" textLength="1464" clip-path="url(#terminal-1489702336-line-43)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1489702336-r2" x="1464" y="1069.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-43)">
-</text><text class="terminal-1489702336-r4" x="0" y="1093.6" textLength="24.4" clip-path="url(#terminal-1489702336-line-44)">╭─</text><text class="terminal-1489702336-r4" x="24.4" y="1093.6" textLength="1415.2" clip-path="url(#terminal-1489702336-line-44)">&#160;Preparing&#160;cache&#160;and&#160;push&#160;(for&#160;maintainers&#160;and&#160;CI)&#160;─────────────────────────────────────────────────────────────────</text><text class="terminal-1489702336-r4" x="1439.6" y="1093.6" textLeng [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-1489702336-line-45)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1118" textLength="12.2" clip-path="url(#terminal-1489702336-line-45)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1118" textLength="85.4" clip-path="url(#terminal-1489702336-line-45)">-github</text><text class="terminal-1489702336-r5" x="122" y="1118" textLength="73.2" clip-path="url(#terminal-1489 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-46)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-46)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1142.4" textLength="85.4" clip-path="url(#terminal-1489702336-line-46)">-github</text><text class="terminal-1489702336-r5" x="122" y="1142.4" textLength="109.8" clip-path="url(#term [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-47)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-47)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1166.8" textLength="109.8" clip-path="url(#terminal-1489702336-line-47)">-platform</text><text class="terminal-1489702336-r2" x="341.6" y="1166.8" textLength="329.4" clip-path="url( [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-48)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1191.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-48)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1191.2" textLength="61" clip-path="url(#terminal-1489702336-line-48)">-push</text><text class="terminal-1489702336-r5" x="97.6" y="1191.2" textLength="73.2" clip-path="url(#terminal [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1215.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-49)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1215.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-49)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1215.6" textLength="73.2" clip-path="url(#terminal-1489702336-line-49)">-empty</text><text class="terminal-1489702336-r5" x="109.8" y="1215.6" textLength="73.2" clip-path="url(#term [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1240" textLength="12.2" clip-path="url(#terminal-1489702336-line-50)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1240" textLength="12.2" clip-path="url(#terminal-1489702336-line-50)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1240" textLength="97.6" clip-path="url(#terminal-1489702336-line-50)">-prepare</text><text class="terminal-1489702336-r5" x="134.2" y="1240" textLength="158.6" clip-path="url(#terminal- [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-51)">│</text><text class="terminal-1489702336-r2" x="341.6" y="1264.4" textLength="1098" clip-path="url(#terminal-1489702336-line-51)">image).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1288.8" textLength="1464" clip-path="url(#terminal-1489702336-line-52)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1489702336-r2" x="1464" y="1288.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-52)">
-</text><text class="terminal-1489702336-r4" x="0" y="1313.2" textLength="24.4" clip-path="url(#terminal-1489702336-line-53)">╭─</text><text class="terminal-1489702336-r4" x="24.4" y="1313.2" textLength="1415.2" clip-path="url(#terminal-1489702336-line-53)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1489702336-r4" x="1439.6" y="1313.2" textLength="24.4" clip-path="url(#terminal- [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1337.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-54)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-54)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1337.6" textLength="85.4" clip-path="url(#terminal-1489702336-line-54)">-github</text><text class="terminal-1489702336-r5" x="122" y="1337.6" textLength="134.2" clip-path="url(#term [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1362" textLength="12.2" clip-path="url(#terminal-1489702336-line-55)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1362" textLength="12.2" clip-path="url(#terminal-1489702336-line-55)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1362" textLength="97.6" clip-path="url(#terminal-1489702336-line-55)">-verbose</text><text class="terminal-1489702336-r6" x="280.6" y="1362" textLength="24.4" clip-path="url(#terminal-1 [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1386.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-56)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1386.4" textLength="12.2" clip-path="url(#terminal-1489702336-line-56)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1386.4" textLength="48.8" clip-path="url(#terminal-1489702336-line-56)">-dry</text><text class="terminal-1489702336-r5" x="85.4" y="1386.4" textLength="48.8" clip-path="url(#termina [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1410.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-57)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#terminal-1489702336-line-57)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1410.8" textLength="85.4" clip-path="url(#terminal-1489702336-line-57)">-answer</text><text class="terminal-1489702336-r6" x="280.6" y="1410.8" textLength="24.4" clip-path="url(#ter [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1435.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-58)">│</text><text class="terminal-1489702336-r5" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#terminal-1489702336-line-58)">-</text><text class="terminal-1489702336-r5" x="36.6" y="1435.2" textLength="61" clip-path="url(#terminal-1489702336-line-58)">-help</text><text class="terminal-1489702336-r6" x="280.6" y="1435.2" textLength="24.4" clip-path="url(#termina [...]
-</text><text class="terminal-1489702336-r4" x="0" y="1459.6" textLength="1464" clip-path="url(#terminal-1489702336-line-59)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1489702336-r2" x="1464" y="1459.6" textLength="12.2" clip-path="url(#terminal-1489702336-line-59)">
+    <g class="terminal-4238821259-matrix">
+    <text class="terminal-4238821259-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-4238821259-line-0)">
+</text><text class="terminal-4238821259-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-4238821259-line-1)">Usage:&#160;</text><text class="terminal-4238821259-r1" x="97.6" y="44.4" textLength="341.6" clip-path="url(#terminal-4238821259-line-1)">breeze&#160;build-image&#160;[OPTIONS]</text><text class="terminal-4238821259-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-1)">
+</text><text class="terminal-4238821259-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-2)">
+</text><text class="terminal-4238821259-r2" x="12.2" y="93.2" textLength="1073.6" clip-path="url(#terminal-4238821259-line-3)">Build&#160;CI&#160;image.&#160;Include&#160;building&#160;multiple&#160;images&#160;for&#160;all&#160;python&#160;versions&#160;(sequentially).</text><text class="terminal-4238821259-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-3)">
+</text><text class="terminal-4238821259-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-4)">
+</text><text class="terminal-4238821259-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-4238821259-line-5)">╭─</text><text class="terminal-4238821259-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-4238821259-line-5)">&#160;Basic&#160;usage&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-4238821259-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-423882 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-6)">│</text><text class="terminal-4238821259-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-6)">-</text><text class="terminal-4238821259-r5" x="36.6" y="166.4" textLength="85.4" clip-path="url(#terminal-4238821259-line-6)">-python</text><text class="terminal-4238821259-r6" x="427" y="166.4" textLength="24.4" clip-path="url(#terminal-423 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-7)">│</text><text class="terminal-4238821259-r7" x="475.8" y="190.8" textLength="732" clip-path="url(#terminal-4238821259-line-7)">(&gt;3.7&lt;&#160;|&#160;3.8&#160;|&#160;3.9&#160;|&#160;3.10)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="terminal-4238821259-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-8)">│</text><text class="terminal-4238821259-r4" x="475.8" y="215.2" textLength="732" clip-path="url(#terminal-4238821259-line-8)">[default:&#160;3.7]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="terminal-4238821259-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-9)">│</text><text class="terminal-4238821259-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-9)">-</text><text class="terminal-4238821259-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-4238821259-line-9)">-upgrade</text><text class="terminal-4238821259-r5" x="134.2" y="239.6" textLength="268.4" clip-path="url(#terminal [...]
+</text><text class="terminal-4238821259-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-4238821259-line-10)">│</text><text class="terminal-4238821259-r5" x="24.4" y="264" textLength="12.2" clip-path="url(#terminal-4238821259-line-10)">-</text><text class="terminal-4238821259-r5" x="36.6" y="264" textLength="85.4" clip-path="url(#terminal-4238821259-line-10)">-debian</text><text class="terminal-4238821259-r5" x="122" y="264" textLength="97.6" clip-path="url(#terminal-42388212 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-11)">│</text><text class="terminal-4238821259-r5" x="24.4" y="288.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-11)">-</text><text class="terminal-4238821259-r5" x="36.6" y="288.4" textLength="73.2" clip-path="url(#terminal-4238821259-line-11)">-image</text><text class="terminal-4238821259-r5" x="109.8" y="288.4" textLength="48.8" clip-path="url(#terminal [...]
+</text><text class="terminal-4238821259-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-12)">│</text><text class="terminal-4238821259-r5" x="475.8" y="312.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-12)">-</text><text class="terminal-4238821259-r5" x="488" y="312.8" textLength="73.2" clip-path="url(#terminal-4238821259-line-12)">-mount</text><text class="terminal-4238821259-r5" x="561.2" y="312.8" textLength="97.6" clip-path="url(#terminal [...]
+</text><text class="terminal-4238821259-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-13)">│</text><text class="terminal-4238821259-r7" x="475.8" y="337.2" textLength="963.8" clip-path="url(#terminal-4238821259-line-13)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-14)">│</text><text class="terminal-4238821259-r5" x="24.4" y="361.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-14)">-</text><text class="terminal-4238821259-r5" x="36.6" y="361.6" textLength="48.8" clip-path="url(#terminal-4238821259-line-14)">-tag</text><text class="terminal-4238821259-r5" x="85.4" y="361.6" textLength="122" clip-path="url(#terminal-423 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-4238821259-line-15)">│</text><text class="terminal-4238821259-r2" x="475.8" y="386" textLength="414.8" clip-path="url(#terminal-4238821259-line-15)">when&#160;you&#160;build&#160;or&#160;pull&#160;image&#160;with&#160;</text><text class="terminal-4238821259-r5" x="890.6" y="386" textLength="12.2" clip-path="url(#terminal-4238821259-line-15)">-</text><text class="terminal-4238821259-r5" x= [...]
+</text><text class="terminal-4238821259-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-16)">│</text><text class="terminal-4238821259-r5" x="24.4" y="410.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-16)">-</text><text class="terminal-4238821259-r5" x="36.6" y="410.4" textLength="85.4" clip-path="url(#terminal-4238821259-line-16)">-docker</text><text class="terminal-4238821259-r5" x="122" y="410.4" textLength="73.2" clip-path="url(#terminal- [...]
+</text><text class="terminal-4238821259-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-17)">│</text><text class="terminal-4238821259-r4" x="475.8" y="434.8" textLength="549" clip-path="url(#terminal-4238821259-line-17)">[default:&#160;registry]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-4238821259-r4" x="1451.8" y="43 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-18)">│</text><text class="terminal-4238821259-r5" x="24.4" y="459.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-18)">-</text><text class="terminal-4238821259-r5" x="36.6" y="459.2" textLength="73.2" clip-path="url(#terminal-4238821259-line-18)">-force</text><text class="terminal-4238821259-r5" x="109.8" y="459.2" textLength="73.2" clip-path="url(#terminal [...]
+</text><text class="terminal-4238821259-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-4238821259-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-4238821259-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-19)">
+</text><text class="terminal-4238821259-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-4238821259-line-20)">╭─</text><text class="terminal-4238821259-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-4238821259-line-20)">&#160;Building&#160;images&#160;in&#160;parallel&#160;───────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-4238821259-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#ter [...]
+</text><text class="terminal-4238821259-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-21)">│</text><text class="terminal-4238821259-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-21)">-</text><text class="terminal-4238821259-r5" x="36.6" y="532.4" textLength="48.8" clip-path="url(#terminal-4238821259-line-21)">-run</text><text class="terminal-4238821259-r5" x="85.4" y="532.4" textLength="146.4" clip-path="url(#terminal-4 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-22)">│</text><text class="terminal-4238821259-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-22)">-</text><text class="terminal-4238821259-r5" x="36.6" y="556.8" textLength="146.4" clip-path="url(#terminal-4238821259-line-22)">-parallelism</text><text class="terminal-4238821259-r2" x="280.6" y="556.8" textLength="915" clip-path="url(#te [...]
+</text><text class="terminal-4238821259-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-23)">│</text><text class="terminal-4238821259-r4" x="280.6" y="581.2" textLength="915" clip-path="url(#terminal-4238821259-line-23)">[default:&#160;4;&#160;1&lt;=x&lt;=8]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-24)">│</text><text class="terminal-4238821259-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-24)">-</text><text class="terminal-4238821259-r5" x="36.6" y="605.6" textLength="85.4" clip-path="url(#terminal-4238821259-line-24)">-python</text><text class="terminal-4238821259-r5" x="122" y="605.6" textLength="109.8" clip-path="url(#terminal [...]
+</text><text class="terminal-4238821259-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-4238821259-line-25)">│</text><text class="terminal-4238821259-r4" x="280.6" y="630" textLength="951.6" clip-path="url(#terminal-4238821259-line-25)">[default:&#160;3.7&#160;3.8&#160;3.9&#160;3.10]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="654.4" textLength="1464" clip-path="url(#terminal-4238821259-line-26)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-4238821259-r2" x="1464" y="654.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-26)">
+</text><text class="terminal-4238821259-r4" x="0" y="678.8" textLength="24.4" clip-path="url(#terminal-4238821259-line-27)">╭─</text><text class="terminal-4238821259-r4" x="24.4" y="678.8" textLength="1415.2" clip-path="url(#terminal-4238821259-line-27)">&#160;Advanced&#160;options&#160;(for&#160;power&#160;users)&#160;────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-4238821259-r4" x="1439.6" y="678.8" textLength="24.4" clip-pat [...]
+</text><text class="terminal-4238821259-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-28)">│</text><text class="terminal-4238821259-r5" x="24.4" y="703.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-28)">-</text><text class="terminal-4238821259-r5" x="36.6" y="703.2" textLength="97.6" clip-path="url(#terminal-4238821259-line-28)">-install</text><text class="terminal-4238821259-r5" x="134.2" y="703.2" textLength="280.6" clip-path="url(#termi [...]
+</text><text class="terminal-4238821259-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-29)">│</text><text class="terminal-4238821259-r5" x="24.4" y="727.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-29)">-</text><text class="terminal-4238821259-r5" x="36.6" y="727.6" textLength="97.6" clip-path="url(#terminal-4238821259-line-29)">-airflow</text><text class="terminal-4238821259-r5" x="134.2" y="727.6" textLength="207.4" clip-path="url(#termi [...]
+</text><text class="terminal-4238821259-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-4238821259-line-30)">│</text><text class="terminal-4238821259-r7" x="463.6" y="752" textLength="866.2" clip-path="url(#terminal-4238821259-line-30)">(constraints-source-providers&#160;|&#160;constraints&#160;|&#160;constraints-no-providers)</text><text class="terminal-4238821259-r4" x="1451.8" y="752" textLength="12.2" clip-path="url(#terminal-4238821259-line-30)">│</text><text class="ter [...]
+</text><text class="terminal-4238821259-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-31)">│</text><text class="terminal-4238821259-r4" x="463.6" y="776.4" textLength="866.2" clip-path="url(#terminal-4238821259-line-31)">[default:&#160;constraints-source-providers]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</ [...]
+</text><text class="terminal-4238821259-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-32)">│</text><text class="terminal-4238821259-r5" x="24.4" y="800.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-32)">-</text><text class="terminal-4238821259-r5" x="36.6" y="800.8" textLength="97.6" clip-path="url(#terminal-4238821259-line-32)">-airflow</text><text class="terminal-4238821259-r5" x="134.2" y="800.8" textLength="268.4" clip-path="url(#termi [...]
+</text><text class="terminal-4238821259-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-33)">│</text><text class="terminal-4238821259-r5" x="24.4" y="825.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-33)">-</text><text class="terminal-4238821259-r5" x="36.6" y="825.2" textLength="134.2" clip-path="url(#terminal-4238821259-line-33)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="825.2" textLength="146.4" clip-path="url(#t [...]
+</text><text class="terminal-4238821259-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-34)">│</text><text class="terminal-4238821259-r5" x="24.4" y="849.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-34)">-</text><text class="terminal-4238821259-r5" x="36.6" y="849.6" textLength="97.6" clip-path="url(#terminal-4238821259-line-34)">-runtime</text><text class="terminal-4238821259-r5" x="134.2" y="849.6" textLength="109.8" clip-path="url(#termi [...]
+</text><text class="terminal-4238821259-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-4238821259-line-35)">│</text><text class="terminal-4238821259-r5" x="24.4" y="874" textLength="12.2" clip-path="url(#terminal-4238821259-line-35)">-</text><text class="terminal-4238821259-r5" x="36.6" y="874" textLength="97.6" clip-path="url(#terminal-4238821259-line-35)">-runtime</text><text class="terminal-4238821259-r5" x="134.2" y="874" textLength="146.4" clip-path="url(#terminal-4238 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-36)">│</text><text class="terminal-4238821259-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-36)">-</text><text class="terminal-4238821259-r5" x="36.6" y="898.4" textLength="134.2" clip-path="url(#terminal-4238821259-line-36)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="898.4" textLength="85.4" clip-path="url(#te [...]
+</text><text class="terminal-4238821259-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-37)">│</text><text class="terminal-4238821259-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-37)">-</text><text class="terminal-4238821259-r5" x="36.6" y="922.8" textLength="134.2" clip-path="url(#terminal-4238821259-line-37)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="922.8" textLength="207.4" clip-path="url(#t [...]
+</text><text class="terminal-4238821259-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-38)">│</text><text class="terminal-4238821259-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-38)">-</text><text class="terminal-4238821259-r5" x="36.6" y="947.2" textLength="134.2" clip-path="url(#terminal-4238821259-line-38)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="947.2" textLength="195.2" clip-path="url(#t [...]
+</text><text class="terminal-4238821259-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-39)">│</text><text class="terminal-4238821259-r5" x="24.4" y="971.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-39)">-</text><text class="terminal-4238821259-r5" x="36.6" y="971.6" textLength="134.2" clip-path="url(#terminal-4238821259-line-39)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="971.6" textLength="244" clip-path="url(#ter [...]
+</text><text class="terminal-4238821259-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-4238821259-line-40)">│</text><text class="terminal-4238821259-r5" x="24.4" y="996" textLength="12.2" clip-path="url(#terminal-4238821259-line-40)">-</text><text class="terminal-4238821259-r5" x="36.6" y="996" textLength="134.2" clip-path="url(#terminal-4238821259-line-40)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="996" textLength="158.6" clip-path="url(#terminal- [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-41)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1020.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-41)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1020.4" textLength="134.2" clip-path="url(#terminal-4238821259-line-41)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="1020.4" textLength="146.4" clip-path="ur [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-42)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-42)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1044.8" textLength="134.2" clip-path="url(#terminal-4238821259-line-42)">-additional</text><text class="terminal-4238821259-r5" x="170.8" y="1044.8" textLength="195.2" clip-path="ur [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-43)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1069.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-43)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1069.2" textLength="48.8" clip-path="url(#terminal-4238821259-line-43)">-dev</text><text class="terminal-4238821259-r5" x="85.4" y="1069.2" textLength="109.8" clip-path="url(#termin [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-44)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1093.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-44)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1093.6" textLength="48.8" clip-path="url(#terminal-4238821259-line-44)">-dev</text><text class="terminal-4238821259-r5" x="85.4" y="1093.6" textLength="146.4" clip-path="url(#termin [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1118" textLength="1464" clip-path="url(#terminal-4238821259-line-45)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-4238821259-r2" x="1464" y="1118" textLength="12.2" clip-path="url(#terminal-4238821259-line-45)">
+</text><text class="terminal-4238821259-r4" x="0" y="1142.4" textLength="24.4" clip-path="url(#terminal-4238821259-line-46)">╭─</text><text class="terminal-4238821259-r4" x="24.4" y="1142.4" textLength="1415.2" clip-path="url(#terminal-4238821259-line-46)">&#160;Preparing&#160;cache&#160;and&#160;push&#160;(for&#160;maintainers&#160;and&#160;CI)&#160;─────────────────────────────────────────────────────────────────</text><text class="terminal-4238821259-r4" x="1439.6" y="1142.4" textLeng [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-47)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-47)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1166.8" textLength="85.4" clip-path="url(#terminal-4238821259-line-47)">-github</text><text class="terminal-4238821259-r5" x="122" y="1166.8" textLength="73.2" clip-path="url(#termi [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-48)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1191.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-48)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1191.2" textLength="85.4" clip-path="url(#terminal-4238821259-line-48)">-github</text><text class="terminal-4238821259-r5" x="122" y="1191.2" textLength="109.8" clip-path="url(#term [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1215.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-49)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1215.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-49)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1215.6" textLength="109.8" clip-path="url(#terminal-4238821259-line-49)">-platform</text><text class="terminal-4238821259-r2" x="341.6" y="1215.6" textLength="329.4" clip-path="url( [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1240" textLength="12.2" clip-path="url(#terminal-4238821259-line-50)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1240" textLength="12.2" clip-path="url(#terminal-4238821259-line-50)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1240" textLength="61" clip-path="url(#terminal-4238821259-line-50)">-push</text><text class="terminal-4238821259-r5" x="97.6" y="1240" textLength="73.2" clip-path="url(#terminal-4238821 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-51)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-51)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1264.4" textLength="73.2" clip-path="url(#terminal-4238821259-line-51)">-empty</text><text class="terminal-4238821259-r5" x="109.8" y="1264.4" textLength="73.2" clip-path="url(#term [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1288.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-52)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-52)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1288.8" textLength="97.6" clip-path="url(#terminal-4238821259-line-52)">-prepare</text><text class="terminal-4238821259-r5" x="134.2" y="1288.8" textLength="158.6" clip-path="url(#t [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1313.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-53)">│</text><text class="terminal-4238821259-r2" x="341.6" y="1313.2" textLength="1098" clip-path="url(#terminal-4238821259-line-53)">image).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1337.6" textLength="1464" clip-path="url(#terminal-4238821259-line-54)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-4238821259-r2" x="1464" y="1337.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-54)">
+</text><text class="terminal-4238821259-r4" x="0" y="1362" textLength="24.4" clip-path="url(#terminal-4238821259-line-55)">╭─</text><text class="terminal-4238821259-r4" x="24.4" y="1362" textLength="1415.2" clip-path="url(#terminal-4238821259-line-55)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-4238821259-r4" x="1439.6" y="1362" textLength="24.4" clip-path="url(#terminal-423882 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1386.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-56)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1386.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-56)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1386.4" textLength="85.4" clip-path="url(#terminal-4238821259-line-56)">-github</text><text class="terminal-4238821259-r5" x="122" y="1386.4" textLength="134.2" clip-path="url(#term [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1410.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-57)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#terminal-4238821259-line-57)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1410.8" textLength="97.6" clip-path="url(#terminal-4238821259-line-57)">-verbose</text><text class="terminal-4238821259-r6" x="280.6" y="1410.8" textLength="24.4" clip-path="url(#te [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1435.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-58)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#terminal-4238821259-line-58)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1435.2" textLength="48.8" clip-path="url(#terminal-4238821259-line-58)">-dry</text><text class="terminal-4238821259-r5" x="85.4" y="1435.2" textLength="48.8" clip-path="url(#termina [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1459.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-59)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1459.6" textLength="12.2" clip-path="url(#terminal-4238821259-line-59)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1459.6" textLength="85.4" clip-path="url(#terminal-4238821259-line-59)">-answer</text><text class="terminal-4238821259-r6" x="280.6" y="1459.6" textLength="24.4" clip-path="url(#ter [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1484" textLength="12.2" clip-path="url(#terminal-4238821259-line-60)">│</text><text class="terminal-4238821259-r5" x="24.4" y="1484" textLength="12.2" clip-path="url(#terminal-4238821259-line-60)">-</text><text class="terminal-4238821259-r5" x="36.6" y="1484" textLength="61" clip-path="url(#terminal-4238821259-line-60)">-help</text><text class="terminal-4238821259-r6" x="280.6" y="1484" textLength="24.4" clip-path="url(#terminal-423882 [...]
+</text><text class="terminal-4238821259-r4" x="0" y="1508.4" textLength="1464" clip-path="url(#terminal-4238821259-line-61)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-4238821259-r2" x="1464" y="1508.4" textLength="12.2" clip-path="url(#terminal-4238821259-line-61)">
 </text>
     </g>
     </g>
diff --git a/images/breeze/output-build-prod-image.svg b/images/breeze/output-build-prod-image.svg
index dcbd32a966..331cd3808b 100644
--- a/images/breeze/output-build-prod-image.svg
+++ b/images/breeze/output-build-prod-image.svg
@@ -1,4 +1,4 @@
-<svg class="rich-terminal" viewBox="0 0 1482 1928.8" xmlns="http://www.w3.org/2000/svg">
+<svg class="rich-terminal" viewBox="0 0 1482 1977.6" xmlns="http://www.w3.org/2000/svg">
     <!-- Generated with Rich https://www.textualize.io -->
     <style>
 
@@ -19,349 +19,357 @@
         font-weight: 700;
     }
 
-    .terminal-1314099363-matrix {
+    .terminal-3171273326-matrix {
         font-family: Fira Code, monospace;
         font-size: 20px;
         line-height: 24.4px;
         font-variant-east-asian: full-width;
     }
 
-    .terminal-1314099363-title {
+    .terminal-3171273326-title {
         font-size: 18px;
         font-weight: bold;
         font-family: arial;
     }
 
-    .terminal-1314099363-r1 { fill: #c5c8c6;font-weight: bold }
-.terminal-1314099363-r2 { fill: #c5c8c6 }
-.terminal-1314099363-r3 { fill: #d0b344;font-weight: bold }
-.terminal-1314099363-r4 { fill: #868887 }
-.terminal-1314099363-r5 { fill: #68a0b3;font-weight: bold }
-.terminal-1314099363-r6 { fill: #98a84b;font-weight: bold }
-.terminal-1314099363-r7 { fill: #8d7b39 }
+    .terminal-3171273326-r1 { fill: #c5c8c6;font-weight: bold }
+.terminal-3171273326-r2 { fill: #c5c8c6 }
+.terminal-3171273326-r3 { fill: #d0b344;font-weight: bold }
+.terminal-3171273326-r4 { fill: #868887 }
+.terminal-3171273326-r5 { fill: #68a0b3;font-weight: bold }
+.terminal-3171273326-r6 { fill: #98a84b;font-weight: bold }
+.terminal-3171273326-r7 { fill: #8d7b39 }
     </style>
 
     <defs>
-    <clipPath id="terminal-1314099363-clip-terminal">
-      <rect x="0" y="0" width="1463.0" height="1877.8" />
+    <clipPath id="terminal-3171273326-clip-terminal">
+      <rect x="0" y="0" width="1463.0" height="1926.6" />
     </clipPath>
-    <clipPath id="terminal-1314099363-line-0">
+    <clipPath id="terminal-3171273326-line-0">
     <rect x="0" y="1.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-1">
+<clipPath id="terminal-3171273326-line-1">
     <rect x="0" y="25.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-2">
+<clipPath id="terminal-3171273326-line-2">
     <rect x="0" y="50.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-3">
+<clipPath id="terminal-3171273326-line-3">
     <rect x="0" y="74.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-4">
+<clipPath id="terminal-3171273326-line-4">
     <rect x="0" y="99.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-5">
+<clipPath id="terminal-3171273326-line-5">
     <rect x="0" y="123.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-6">
+<clipPath id="terminal-3171273326-line-6">
     <rect x="0" y="147.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-7">
+<clipPath id="terminal-3171273326-line-7">
     <rect x="0" y="172.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-8">
+<clipPath id="terminal-3171273326-line-8">
     <rect x="0" y="196.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-9">
+<clipPath id="terminal-3171273326-line-9">
     <rect x="0" y="221.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-10">
+<clipPath id="terminal-3171273326-line-10">
     <rect x="0" y="245.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-11">
+<clipPath id="terminal-3171273326-line-11">
     <rect x="0" y="269.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-12">
+<clipPath id="terminal-3171273326-line-12">
     <rect x="0" y="294.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-13">
+<clipPath id="terminal-3171273326-line-13">
     <rect x="0" y="318.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-14">
+<clipPath id="terminal-3171273326-line-14">
     <rect x="0" y="343.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-15">
+<clipPath id="terminal-3171273326-line-15">
     <rect x="0" y="367.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-16">
+<clipPath id="terminal-3171273326-line-16">
     <rect x="0" y="391.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-17">
+<clipPath id="terminal-3171273326-line-17">
     <rect x="0" y="416.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-18">
+<clipPath id="terminal-3171273326-line-18">
     <rect x="0" y="440.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-19">
+<clipPath id="terminal-3171273326-line-19">
     <rect x="0" y="465.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-20">
+<clipPath id="terminal-3171273326-line-20">
     <rect x="0" y="489.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-21">
+<clipPath id="terminal-3171273326-line-21">
     <rect x="0" y="513.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-22">
+<clipPath id="terminal-3171273326-line-22">
     <rect x="0" y="538.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-23">
+<clipPath id="terminal-3171273326-line-23">
     <rect x="0" y="562.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-24">
+<clipPath id="terminal-3171273326-line-24">
     <rect x="0" y="587.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-25">
+<clipPath id="terminal-3171273326-line-25">
     <rect x="0" y="611.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-26">
+<clipPath id="terminal-3171273326-line-26">
     <rect x="0" y="635.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-27">
+<clipPath id="terminal-3171273326-line-27">
     <rect x="0" y="660.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-28">
+<clipPath id="terminal-3171273326-line-28">
     <rect x="0" y="684.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-29">
+<clipPath id="terminal-3171273326-line-29">
     <rect x="0" y="709.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-30">
+<clipPath id="terminal-3171273326-line-30">
     <rect x="0" y="733.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-31">
+<clipPath id="terminal-3171273326-line-31">
     <rect x="0" y="757.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-32">
+<clipPath id="terminal-3171273326-line-32">
     <rect x="0" y="782.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-33">
+<clipPath id="terminal-3171273326-line-33">
     <rect x="0" y="806.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-34">
+<clipPath id="terminal-3171273326-line-34">
     <rect x="0" y="831.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-35">
+<clipPath id="terminal-3171273326-line-35">
     <rect x="0" y="855.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-36">
+<clipPath id="terminal-3171273326-line-36">
     <rect x="0" y="879.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-37">
+<clipPath id="terminal-3171273326-line-37">
     <rect x="0" y="904.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-38">
+<clipPath id="terminal-3171273326-line-38">
     <rect x="0" y="928.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-39">
+<clipPath id="terminal-3171273326-line-39">
     <rect x="0" y="953.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-40">
+<clipPath id="terminal-3171273326-line-40">
     <rect x="0" y="977.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-41">
+<clipPath id="terminal-3171273326-line-41">
     <rect x="0" y="1001.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-42">
+<clipPath id="terminal-3171273326-line-42">
     <rect x="0" y="1026.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-43">
+<clipPath id="terminal-3171273326-line-43">
     <rect x="0" y="1050.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-44">
+<clipPath id="terminal-3171273326-line-44">
     <rect x="0" y="1075.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-45">
+<clipPath id="terminal-3171273326-line-45">
     <rect x="0" y="1099.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-46">
+<clipPath id="terminal-3171273326-line-46">
     <rect x="0" y="1123.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-47">
+<clipPath id="terminal-3171273326-line-47">
     <rect x="0" y="1148.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-48">
+<clipPath id="terminal-3171273326-line-48">
     <rect x="0" y="1172.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-49">
+<clipPath id="terminal-3171273326-line-49">
     <rect x="0" y="1197.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-50">
+<clipPath id="terminal-3171273326-line-50">
     <rect x="0" y="1221.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-51">
+<clipPath id="terminal-3171273326-line-51">
     <rect x="0" y="1245.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-52">
+<clipPath id="terminal-3171273326-line-52">
     <rect x="0" y="1270.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-53">
+<clipPath id="terminal-3171273326-line-53">
     <rect x="0" y="1294.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-54">
+<clipPath id="terminal-3171273326-line-54">
     <rect x="0" y="1319.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-55">
+<clipPath id="terminal-3171273326-line-55">
     <rect x="0" y="1343.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-56">
+<clipPath id="terminal-3171273326-line-56">
     <rect x="0" y="1367.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-57">
+<clipPath id="terminal-3171273326-line-57">
     <rect x="0" y="1392.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-58">
+<clipPath id="terminal-3171273326-line-58">
     <rect x="0" y="1416.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-59">
+<clipPath id="terminal-3171273326-line-59">
     <rect x="0" y="1441.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-60">
+<clipPath id="terminal-3171273326-line-60">
     <rect x="0" y="1465.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-61">
+<clipPath id="terminal-3171273326-line-61">
     <rect x="0" y="1489.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-62">
+<clipPath id="terminal-3171273326-line-62">
     <rect x="0" y="1514.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-63">
+<clipPath id="terminal-3171273326-line-63">
     <rect x="0" y="1538.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-64">
+<clipPath id="terminal-3171273326-line-64">
     <rect x="0" y="1563.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-65">
+<clipPath id="terminal-3171273326-line-65">
     <rect x="0" y="1587.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-66">
+<clipPath id="terminal-3171273326-line-66">
     <rect x="0" y="1611.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-67">
+<clipPath id="terminal-3171273326-line-67">
     <rect x="0" y="1636.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-68">
+<clipPath id="terminal-3171273326-line-68">
     <rect x="0" y="1660.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-69">
+<clipPath id="terminal-3171273326-line-69">
     <rect x="0" y="1685.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-70">
+<clipPath id="terminal-3171273326-line-70">
     <rect x="0" y="1709.5" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-71">
+<clipPath id="terminal-3171273326-line-71">
     <rect x="0" y="1733.9" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-72">
+<clipPath id="terminal-3171273326-line-72">
     <rect x="0" y="1758.3" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-73">
+<clipPath id="terminal-3171273326-line-73">
     <rect x="0" y="1782.7" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-74">
+<clipPath id="terminal-3171273326-line-74">
     <rect x="0" y="1807.1" width="1464" height="24.65"/>
             </clipPath>
-<clipPath id="terminal-1314099363-line-75">
+<clipPath id="terminal-3171273326-line-75">
     <rect x="0" y="1831.5" width="1464" height="24.65"/>
             </clipPath>
+<clipPath id="terminal-3171273326-line-76">
+    <rect x="0" y="1855.9" width="1464" height="24.65"/>
+            </clipPath>
+<clipPath id="terminal-3171273326-line-77">
+    <rect x="0" y="1880.3" width="1464" height="24.65"/>
+            </clipPath>
     </defs>
 
-    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1926.8" rx="8"/><text class="terminal-1314099363-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-prod-image</text>
+    <rect fill="#292929" stroke="rgba(255,255,255,0.35)" stroke-width="1" x="1" y="1" width="1480" height="1975.6" rx="8"/><text class="terminal-3171273326-title" fill="#c5c8c6" text-anchor="middle" x="740" y="27">Command:&#160;build-prod-image</text>
             <g transform="translate(26,22)">
             <circle cx="0" cy="0" r="7" fill="#ff5f57"/>
             <circle cx="22" cy="0" r="7" fill="#febc2e"/>
             <circle cx="44" cy="0" r="7" fill="#28c840"/>
             </g>
         
-    <g transform="translate(9, 41)" clip-path="url(#terminal-1314099363-clip-terminal)">
+    <g transform="translate(9, 41)" clip-path="url(#terminal-3171273326-clip-terminal)">
     
-    <g class="terminal-1314099363-matrix">
-    <text class="terminal-1314099363-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-1314099363-line-0)">
-</text><text class="terminal-1314099363-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-1314099363-line-1)">Usage:&#160;</text><text class="terminal-1314099363-r1" x="97.6" y="44.4" textLength="402.6" clip-path="url(#terminal-1314099363-line-1)">breeze&#160;build-prod-image&#160;[OPTIONS]</text><text class="terminal-1314099363-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-1)">
-</text><text class="terminal-1314099363-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-2)">
-</text><text class="terminal-1314099363-r2" x="12.2" y="93.2" textLength="1293.2" clip-path="url(#terminal-1314099363-line-3)">Build&#160;Production&#160;image.&#160;Include&#160;building&#160;multiple&#160;images&#160;for&#160;all&#160;or&#160;selected&#160;Python&#160;versions&#160;sequentially.</text><text class="terminal-1314099363-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-3)">
-</text><text class="terminal-1314099363-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-4)">
-</text><text class="terminal-1314099363-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-1314099363-line-5)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-1314099363-line-5)">&#160;Basic&#160;usage&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-131409 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-6)">│</text><text class="terminal-1314099363-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-6)">-</text><text class="terminal-1314099363-r5" x="36.6" y="166.4" textLength="85.4" clip-path="url(#terminal-1314099363-line-6)">-python</text><text class="terminal-1314099363-r6" x="427" y="166.4" textLength="24.4" clip-path="url(#terminal-131 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-7)">│</text><text class="terminal-1314099363-r7" x="475.8" y="190.8" textLength="732" clip-path="url(#terminal-1314099363-line-7)">(&gt;3.7&lt;&#160;|&#160;3.8&#160;|&#160;3.9&#160;|&#160;3.10)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="terminal-1314099363-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-8)">│</text><text class="terminal-1314099363-r4" x="475.8" y="215.2" textLength="732" clip-path="url(#terminal-1314099363-line-8)">[default:&#160;3.7]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="terminal-1314099363-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-9)">│</text><text class="terminal-1314099363-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-9)">-</text><text class="terminal-1314099363-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-9)">-install</text><text class="terminal-1314099363-r5" x="134.2" y="239.6" textLength="195.2" clip-path="url(#terminal [...]
-</text><text class="terminal-1314099363-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-1314099363-line-10)">│</text><text class="terminal-1314099363-r5" x="24.4" y="264" textLength="12.2" clip-path="url(#terminal-1314099363-line-10)">-</text><text class="terminal-1314099363-r5" x="36.6" y="264" textLength="97.6" clip-path="url(#terminal-1314099363-line-10)">-upgrade</text><text class="terminal-1314099363-r5" x="134.2" y="264" textLength="268.4" clip-path="url(#terminal-1314 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-11)">│</text><text class="terminal-1314099363-r5" x="24.4" y="288.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-11)">-</text><text class="terminal-1314099363-r5" x="36.6" y="288.4" textLength="85.4" clip-path="url(#terminal-1314099363-line-11)">-debian</text><text class="terminal-1314099363-r5" x="122" y="288.4" textLength="97.6" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-12)">│</text><text class="terminal-1314099363-r5" x="24.4" y="312.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-12)">-</text><text class="terminal-1314099363-r5" x="36.6" y="312.8" textLength="73.2" clip-path="url(#terminal-1314099363-line-12)">-image</text><text class="terminal-1314099363-r5" x="109.8" y="312.8" textLength="48.8" clip-path="url(#terminal [...]
-</text><text class="terminal-1314099363-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-13)">│</text><text class="terminal-1314099363-r5" x="475.8" y="337.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-13)">-</text><text class="terminal-1314099363-r5" x="488" y="337.2" textLength="73.2" clip-path="url(#terminal-1314099363-line-13)">-mount</text><text class="terminal-1314099363-r5" x="561.2" y="337.2" textLength="97.6" clip-path="url(#terminal [...]
-</text><text class="terminal-1314099363-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-14)">│</text><text class="terminal-1314099363-r7" x="475.8" y="361.6" textLength="963.8" clip-path="url(#terminal-1314099363-line-14)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-1314099363-line-15)">│</text><text class="terminal-1314099363-r5" x="24.4" y="386" textLength="12.2" clip-path="url(#terminal-1314099363-line-15)">-</text><text class="terminal-1314099363-r5" x="36.6" y="386" textLength="48.8" clip-path="url(#terminal-1314099363-line-15)">-tag</text><text class="terminal-1314099363-r5" x="85.4" y="386" textLength="122" clip-path="url(#terminal-1314099363- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-16)">│</text><text class="terminal-1314099363-r2" x="475.8" y="410.4" textLength="414.8" clip-path="url(#terminal-1314099363-line-16)">when&#160;you&#160;build&#160;or&#160;pull&#160;image&#160;with&#160;</text><text class="terminal-1314099363-r5" x="890.6" y="410.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-16)">-</text><text class="terminal-1314099363- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-17)">│</text><text class="terminal-1314099363-r5" x="24.4" y="434.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-17)">-</text><text class="terminal-1314099363-r5" x="36.6" y="434.8" textLength="85.4" clip-path="url(#terminal-1314099363-line-17)">-docker</text><text class="terminal-1314099363-r5" x="122" y="434.8" textLength="73.2" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-18)">│</text><text class="terminal-1314099363-r4" x="475.8" y="459.2" textLength="549" clip-path="url(#terminal-1314099363-line-18)">[default:&#160;registry]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-1314099363-r4" x="1451.8" y="45 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-1314099363-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-19)">
-</text><text class="terminal-1314099363-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-1314099363-line-20)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-1314099363-line-20)">&#160;Building&#160;multiple&#160;images&#160;──────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#terminal [...]
-</text><text class="terminal-1314099363-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-21)">│</text><text class="terminal-1314099363-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-21)">-</text><text class="terminal-1314099363-r5" x="36.6" y="532.4" textLength="73.2" clip-path="url(#terminal-1314099363-line-21)">-build</text><text class="terminal-1314099363-r5" x="109.8" y="532.4" textLength="195.2" clip-path="url(#termina [...]
-</text><text class="terminal-1314099363-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-22)">│</text><text class="terminal-1314099363-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-22)">-</text><text class="terminal-1314099363-r5" x="36.6" y="556.8" textLength="85.4" clip-path="url(#terminal-1314099363-line-22)">-python</text><text class="terminal-1314099363-r5" x="122" y="556.8" textLength="109.8" clip-path="url(#terminal [...]
-</text><text class="terminal-1314099363-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-23)">│</text><text class="terminal-1314099363-r4" x="353.8" y="581.2" textLength="951.6" clip-path="url(#terminal-1314099363-line-23)">[default:&#160;3.7&#160;3.8&#160;3.9&#160;3.10]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="605.6" textLength="1464" clip-path="url(#terminal-1314099363-line-24)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="605.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-24)">
-</text><text class="terminal-1314099363-r4" x="0" y="630" textLength="24.4" clip-path="url(#terminal-1314099363-line-25)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="630" textLength="1415.2" clip-path="url(#terminal-1314099363-line-25)">&#160;Options&#160;for&#160;customizing&#160;images&#160;────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="630" textLength="24.4" clip-path="url(#ter [...]
-</text><text class="terminal-1314099363-r4" x="0" y="654.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-26)">│</text><text class="terminal-1314099363-r5" x="24.4" y="654.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-26)">-</text><text class="terminal-1314099363-r5" x="36.6" y="654.4" textLength="97.6" clip-path="url(#terminal-1314099363-line-26)">-install</text><text class="terminal-1314099363-r5" x="134.2" y="654.4" textLength="280.6" clip-path="url(#termi [...]
-</text><text class="terminal-1314099363-r4" x="0" y="678.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-27)">│</text><text class="terminal-1314099363-r5" x="24.4" y="678.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-27)">-</text><text class="terminal-1314099363-r5" x="36.6" y="678.8" textLength="97.6" clip-path="url(#terminal-1314099363-line-27)">-airflow</text><text class="terminal-1314099363-r5" x="134.2" y="678.8" textLength="85.4" clip-path="url(#termin [...]
-</text><text class="terminal-1314099363-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-28)">│</text><text class="terminal-1314099363-r7" x="463.6" y="703.2" textLength="976" clip-path="url(#terminal-1314099363-line-28)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
-</text><text class="terminal-1314099363-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-29)">│</text><text class="terminal-1314099363-r4" x="463.6" y="727.6" textLength="976" clip-path="url(#terminal-1314099363-line-29)">[default:&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-1314099363-line-30)">│</text><text class="terminal-1314099363-r4" x="463.6" y="752" textLength="976" clip-path="url(#terminal-1314099363-line-30)">amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,google…</text><text class="terminal-1314099363-r4" x="1451.8" y="752" textLength="12.2" clip-path="url(#terminal-1314099363-line-30)">│</text><text class="terminal-1314099 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-31)">│</text><text class="terminal-1314099363-r5" x="24.4" y="776.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-31)">-</text><text class="terminal-1314099363-r5" x="36.6" y="776.4" textLength="97.6" clip-path="url(#terminal-1314099363-line-31)">-airflow</text><text class="terminal-1314099363-r5" x="134.2" y="776.4" textLength="207.4" clip-path="url(#termi [...]
-</text><text class="terminal-1314099363-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-32)">│</text><text class="terminal-1314099363-r7" x="463.6" y="800.8" textLength="866.2" clip-path="url(#terminal-1314099363-line-32)">(constraints&#160;|&#160;constraints-no-providers&#160;|&#160;constraints-source-providers)</text><text class="terminal-1314099363-r4" x="1451.8" y="800.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-32)">│</text><text clas [...]
-</text><text class="terminal-1314099363-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-33)">│</text><text class="terminal-1314099363-r4" x="463.6" y="825.2" textLength="866.2" clip-path="url(#terminal-1314099363-line-33)">[default:&#160;constraints]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
-</text><text class="terminal-1314099363-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-34)">│</text><text class="terminal-1314099363-r5" x="24.4" y="849.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-34)">-</text><text class="terminal-1314099363-r5" x="36.6" y="849.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-34)">-airflow</text><text class="terminal-1314099363-r5" x="134.2" y="849.6" textLength="268.4" clip-path="url(#termi [...]
-</text><text class="terminal-1314099363-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-1314099363-line-35)">│</text><text class="terminal-1314099363-r5" x="24.4" y="874" textLength="12.2" clip-path="url(#terminal-1314099363-line-35)">-</text><text class="terminal-1314099363-r5" x="36.6" y="874" textLength="134.2" clip-path="url(#terminal-1314099363-line-35)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="874" textLength="146.4" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-36)">│</text><text class="terminal-1314099363-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-36)">-</text><text class="terminal-1314099363-r5" x="36.6" y="898.4" textLength="134.2" clip-path="url(#terminal-1314099363-line-36)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="898.4" textLength="85.4" clip-path="url(#te [...]
-</text><text class="terminal-1314099363-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-37)">│</text><text class="terminal-1314099363-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-37)">-</text><text class="terminal-1314099363-r5" x="36.6" y="922.8" textLength="134.2" clip-path="url(#terminal-1314099363-line-37)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="922.8" textLength="207.4" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-38)">│</text><text class="terminal-1314099363-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-38)">-</text><text class="terminal-1314099363-r5" x="36.6" y="947.2" textLength="134.2" clip-path="url(#terminal-1314099363-line-38)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="947.2" textLength="195.2" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-39)">│</text><text class="terminal-1314099363-r5" x="24.4" y="971.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-39)">-</text><text class="terminal-1314099363-r5" x="36.6" y="971.6" textLength="134.2" clip-path="url(#terminal-1314099363-line-39)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="971.6" textLength="244" clip-path="url(#ter [...]
-</text><text class="terminal-1314099363-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-1314099363-line-40)">│</text><text class="terminal-1314099363-r5" x="24.4" y="996" textLength="12.2" clip-path="url(#terminal-1314099363-line-40)">-</text><text class="terminal-1314099363-r5" x="36.6" y="996" textLength="134.2" clip-path="url(#terminal-1314099363-line-40)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="996" textLength="158.6" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-41)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1020.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-41)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1020.4" textLength="134.2" clip-path="url(#terminal-1314099363-line-41)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="1020.4" textLength="146.4" clip-path="ur [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-42)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-42)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1044.8" textLength="134.2" clip-path="url(#terminal-1314099363-line-42)">-additional</text><text class="terminal-1314099363-r5" x="170.8" y="1044.8" textLength="195.2" clip-path="ur [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-43)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1069.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-43)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1069.2" textLength="97.6" clip-path="url(#terminal-1314099363-line-43)">-runtime</text><text class="terminal-1314099363-r5" x="134.2" y="1069.2" textLength="109.8" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-44)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1093.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-44)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1093.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-44)">-runtime</text><text class="terminal-1314099363-r5" x="134.2" y="1093.6" textLength="146.4" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-1314099363-line-45)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1118" textLength="12.2" clip-path="url(#terminal-1314099363-line-45)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1118" textLength="48.8" clip-path="url(#terminal-1314099363-line-45)">-dev</text><text class="terminal-1314099363-r5" x="85.4" y="1118" textLength="109.8" clip-path="url(#terminal-13140 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-46)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-46)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1142.4" textLength="48.8" clip-path="url(#terminal-1314099363-line-46)">-dev</text><text class="terminal-1314099363-r5" x="85.4" y="1142.4" textLength="146.4" clip-path="url(#termin [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1166.8" textLength="1464" clip-path="url(#terminal-1314099363-line-47)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="1166.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-47)">
-</text><text class="terminal-1314099363-r4" x="0" y="1191.2" textLength="24.4" clip-path="url(#terminal-1314099363-line-48)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="1191.2" textLength="1415.2" clip-path="url(#terminal-1314099363-line-48)">&#160;Customization&#160;options&#160;(for&#160;specific&#160;customization&#160;needs)&#160;──────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="1191.2" textLength="24.4"  [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1215.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-49)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1215.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-49)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1215.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-49)">-install</text><text class="terminal-1314099363-r5" x="134.2" y="1215.6" textLength="268.4" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1240" textLength="12.2" clip-path="url(#terminal-1314099363-line-50)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1240" textLength="12.2" clip-path="url(#terminal-1314099363-line-50)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1240" textLength="97.6" clip-path="url(#terminal-1314099363-line-50)">-airflow</text><text class="terminal-1314099363-r5" x="134.2" y="1240" textLength="170.8" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-51)">│</text><text class="terminal-1314099363-r2" x="536.8" y="1264.4" textLength="902.8" clip-path="url(#terminal-1314099363-line-51)">from&#160;PyPI&#160;or&#160;sources.&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1288.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-52)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-52)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1288.8" textLength="97.6" clip-path="url(#terminal-1314099363-line-52)">-cleanup</text><text class="terminal-1314099363-r5" x="134.2" y="1288.8" textLength="97.6" clip-path="url(#te [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1313.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-53)">│</text><text class="terminal-1314099363-r2" x="536.8" y="1313.2" textLength="170.8" clip-path="url(#terminal-1314099363-line-53)">together&#160;with&#160;</text><text class="terminal-1314099363-r5" x="707.6" y="1313.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-53)">-</text><text class="terminal-1314099363-r5" x="719.8" y="1313.2" textLength="97.6" [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1337.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-54)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-54)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1337.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-54)">-disable</text><text class="terminal-1314099363-r5" x="134.2" y="1337.6" textLength="317.2" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1362" textLength="12.2" clip-path="url(#terminal-1314099363-line-55)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1362" textLength="12.2" clip-path="url(#terminal-1314099363-line-55)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1362" textLength="97.6" clip-path="url(#terminal-1314099363-line-55)">-disable</text><text class="terminal-1314099363-r5" x="134.2" y="1362" textLength="317.2" clip-path="url(#terminal- [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1386.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-56)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1386.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-56)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1386.4" textLength="97.6" clip-path="url(#terminal-1314099363-line-56)">-disable</text><text class="terminal-1314099363-r5" x="134.2" y="1386.4" textLength="353.8" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1410.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-57)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-57)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1410.8" textLength="97.6" clip-path="url(#terminal-1314099363-line-57)">-disable</text><text class="terminal-1314099363-r5" x="134.2" y="1410.8" textLength="231.8" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1435.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-58)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-58)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1435.2" textLength="97.6" clip-path="url(#terminal-1314099363-line-58)">-install</text><text class="terminal-1314099363-r5" x="134.2" y="1435.2" textLength="219.6" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1459.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-59)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1459.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-59)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1459.6" textLength="158.6" clip-path="url(#terminal-1314099363-line-59)">-installation</text><text class="terminal-1314099363-r5" x="195.2" y="1459.6" textLength="85.4" clip-path="u [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1484" textLength="1464" clip-path="url(#terminal-1314099363-line-60)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="1484" textLength="12.2" clip-path="url(#terminal-1314099363-line-60)">
-</text><text class="terminal-1314099363-r4" x="0" y="1508.4" textLength="24.4" clip-path="url(#terminal-1314099363-line-61)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="1508.4" textLength="1415.2" clip-path="url(#terminal-1314099363-line-61)">&#160;Preparing&#160;cache&#160;and&#160;push&#160;(for&#160;maintainers&#160;and&#160;CI)&#160;─────────────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="1508.4" textLeng [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1532.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-62)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1532.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-62)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1532.8" textLength="85.4" clip-path="url(#terminal-1314099363-line-62)">-github</text><text class="terminal-1314099363-r5" x="122" y="1532.8" textLength="73.2" clip-path="url(#termi [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1557.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-63)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1557.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-63)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1557.2" textLength="85.4" clip-path="url(#terminal-1314099363-line-63)">-github</text><text class="terminal-1314099363-r5" x="122" y="1557.2" textLength="109.8" clip-path="url(#term [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1581.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-64)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1581.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-64)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1581.6" textLength="109.8" clip-path="url(#terminal-1314099363-line-64)">-platform</text><text class="terminal-1314099363-r2" x="341.6" y="1581.6" textLength="329.4" clip-path="url( [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1606" textLength="12.2" clip-path="url(#terminal-1314099363-line-65)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1606" textLength="12.2" clip-path="url(#terminal-1314099363-line-65)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1606" textLength="61" clip-path="url(#terminal-1314099363-line-65)">-push</text><text class="terminal-1314099363-r5" x="97.6" y="1606" textLength="73.2" clip-path="url(#terminal-1314099 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1630.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-66)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1630.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-66)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1630.4" textLength="73.2" clip-path="url(#terminal-1314099363-line-66)">-empty</text><text class="terminal-1314099363-r5" x="109.8" y="1630.4" textLength="73.2" clip-path="url(#term [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1654.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-67)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-67)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1654.8" textLength="97.6" clip-path="url(#terminal-1314099363-line-67)">-prepare</text><text class="terminal-1314099363-r5" x="134.2" y="1654.8" textLength="158.6" clip-path="url(#t [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1679.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-68)">│</text><text class="terminal-1314099363-r2" x="341.6" y="1679.2" textLength="1098" clip-path="url(#terminal-1314099363-line-68)">image).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1703.6" textLength="1464" clip-path="url(#terminal-1314099363-line-69)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="1703.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-69)">
-</text><text class="terminal-1314099363-r4" x="0" y="1728" textLength="24.4" clip-path="url(#terminal-1314099363-line-70)">╭─</text><text class="terminal-1314099363-r4" x="24.4" y="1728" textLength="1415.2" clip-path="url(#terminal-1314099363-line-70)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-1314099363-r4" x="1439.6" y="1728" textLength="24.4" clip-path="url(#terminal-131409 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1752.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-71)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1752.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-71)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1752.4" textLength="85.4" clip-path="url(#terminal-1314099363-line-71)">-github</text><text class="terminal-1314099363-r5" x="122" y="1752.4" textLength="134.2" clip-path="url(#term [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1776.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-72)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1776.8" textLength="12.2" clip-path="url(#terminal-1314099363-line-72)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1776.8" textLength="85.4" clip-path="url(#terminal-1314099363-line-72)">-answer</text><text class="terminal-1314099363-r6" x="280.6" y="1776.8" textLength="24.4" clip-path="url(#ter [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1801.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-73)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1801.2" textLength="12.2" clip-path="url(#terminal-1314099363-line-73)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1801.2" textLength="48.8" clip-path="url(#terminal-1314099363-line-73)">-dry</text><text class="terminal-1314099363-r5" x="85.4" y="1801.2" textLength="48.8" clip-path="url(#termina [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1825.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-74)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1825.6" textLength="12.2" clip-path="url(#terminal-1314099363-line-74)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1825.6" textLength="97.6" clip-path="url(#terminal-1314099363-line-74)">-verbose</text><text class="terminal-1314099363-r6" x="280.6" y="1825.6" textLength="24.4" clip-path="url(#te [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1850" textLength="12.2" clip-path="url(#terminal-1314099363-line-75)">│</text><text class="terminal-1314099363-r5" x="24.4" y="1850" textLength="12.2" clip-path="url(#terminal-1314099363-line-75)">-</text><text class="terminal-1314099363-r5" x="36.6" y="1850" textLength="61" clip-path="url(#terminal-1314099363-line-75)">-help</text><text class="terminal-1314099363-r6" x="280.6" y="1850" textLength="24.4" clip-path="url(#terminal-131409 [...]
-</text><text class="terminal-1314099363-r4" x="0" y="1874.4" textLength="1464" clip-path="url(#terminal-1314099363-line-76)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-1314099363-r2" x="1464" y="1874.4" textLength="12.2" clip-path="url(#terminal-1314099363-line-76)">
+    <g class="terminal-3171273326-matrix">
+    <text class="terminal-3171273326-r2" x="1464" y="20" textLength="12.2" clip-path="url(#terminal-3171273326-line-0)">
+</text><text class="terminal-3171273326-r3" x="12.2" y="44.4" textLength="85.4" clip-path="url(#terminal-3171273326-line-1)">Usage:&#160;</text><text class="terminal-3171273326-r1" x="97.6" y="44.4" textLength="402.6" clip-path="url(#terminal-3171273326-line-1)">breeze&#160;build-prod-image&#160;[OPTIONS]</text><text class="terminal-3171273326-r2" x="1464" y="44.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-1)">
+</text><text class="terminal-3171273326-r2" x="1464" y="68.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-2)">
+</text><text class="terminal-3171273326-r2" x="12.2" y="93.2" textLength="1293.2" clip-path="url(#terminal-3171273326-line-3)">Build&#160;Production&#160;image.&#160;Include&#160;building&#160;multiple&#160;images&#160;for&#160;all&#160;or&#160;selected&#160;Python&#160;versions&#160;sequentially.</text><text class="terminal-3171273326-r2" x="1464" y="93.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-3)">
+</text><text class="terminal-3171273326-r2" x="1464" y="117.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-4)">
+</text><text class="terminal-3171273326-r4" x="0" y="142" textLength="24.4" clip-path="url(#terminal-3171273326-line-5)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="142" textLength="1415.2" clip-path="url(#terminal-3171273326-line-5)">&#160;Basic&#160;usage&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="142" textLength="24.4" clip-path="url(#terminal-317127 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="166.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-6)">│</text><text class="terminal-3171273326-r5" x="24.4" y="166.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-6)">-</text><text class="terminal-3171273326-r5" x="36.6" y="166.4" textLength="85.4" clip-path="url(#terminal-3171273326-line-6)">-python</text><text class="terminal-3171273326-r6" x="427" y="166.4" textLength="24.4" clip-path="url(#terminal-317 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="190.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-7)">│</text><text class="terminal-3171273326-r7" x="475.8" y="190.8" textLength="732" clip-path="url(#terminal-3171273326-line-7)">(&gt;3.7&lt;&#160;|&#160;3.8&#160;|&#160;3.9&#160;|&#160;3.10)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="terminal-3171273326-r4" x="0" y="215.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-8)">│</text><text class="terminal-3171273326-r4" x="475.8" y="215.2" textLength="732" clip-path="url(#terminal-3171273326-line-8)">[default:&#160;3.7]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;& [...]
+</text><text class="terminal-3171273326-r4" x="0" y="239.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-9)">│</text><text class="terminal-3171273326-r5" x="24.4" y="239.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-9)">-</text><text class="terminal-3171273326-r5" x="36.6" y="239.6" textLength="97.6" clip-path="url(#terminal-3171273326-line-9)">-install</text><text class="terminal-3171273326-r5" x="134.2" y="239.6" textLength="195.2" clip-path="url(#terminal [...]
+</text><text class="terminal-3171273326-r4" x="0" y="264" textLength="12.2" clip-path="url(#terminal-3171273326-line-10)">│</text><text class="terminal-3171273326-r5" x="24.4" y="264" textLength="12.2" clip-path="url(#terminal-3171273326-line-10)">-</text><text class="terminal-3171273326-r5" x="36.6" y="264" textLength="97.6" clip-path="url(#terminal-3171273326-line-10)">-upgrade</text><text class="terminal-3171273326-r5" x="134.2" y="264" textLength="268.4" clip-path="url(#terminal-3171 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="288.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-11)">│</text><text class="terminal-3171273326-r5" x="24.4" y="288.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-11)">-</text><text class="terminal-3171273326-r5" x="36.6" y="288.4" textLength="85.4" clip-path="url(#terminal-3171273326-line-11)">-debian</text><text class="terminal-3171273326-r5" x="122" y="288.4" textLength="97.6" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="312.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-12)">│</text><text class="terminal-3171273326-r5" x="24.4" y="312.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-12)">-</text><text class="terminal-3171273326-r5" x="36.6" y="312.8" textLength="73.2" clip-path="url(#terminal-3171273326-line-12)">-image</text><text class="terminal-3171273326-r5" x="109.8" y="312.8" textLength="48.8" clip-path="url(#terminal [...]
+</text><text class="terminal-3171273326-r4" x="0" y="337.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-13)">│</text><text class="terminal-3171273326-r5" x="475.8" y="337.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-13)">-</text><text class="terminal-3171273326-r5" x="488" y="337.2" textLength="73.2" clip-path="url(#terminal-3171273326-line-13)">-mount</text><text class="terminal-3171273326-r5" x="561.2" y="337.2" textLength="97.6" clip-path="url(#terminal [...]
+</text><text class="terminal-3171273326-r4" x="0" y="361.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-14)">│</text><text class="terminal-3171273326-r7" x="475.8" y="361.6" textLength="963.8" clip-path="url(#terminal-3171273326-line-14)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="386" textLength="12.2" clip-path="url(#terminal-3171273326-line-15)">│</text><text class="terminal-3171273326-r5" x="24.4" y="386" textLength="12.2" clip-path="url(#terminal-3171273326-line-15)">-</text><text class="terminal-3171273326-r5" x="36.6" y="386" textLength="48.8" clip-path="url(#terminal-3171273326-line-15)">-tag</text><text class="terminal-3171273326-r5" x="85.4" y="386" textLength="122" clip-path="url(#terminal-3171273326- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="410.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-16)">│</text><text class="terminal-3171273326-r2" x="475.8" y="410.4" textLength="414.8" clip-path="url(#terminal-3171273326-line-16)">when&#160;you&#160;build&#160;or&#160;pull&#160;image&#160;with&#160;</text><text class="terminal-3171273326-r5" x="890.6" y="410.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-16)">-</text><text class="terminal-3171273326- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="434.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-17)">│</text><text class="terminal-3171273326-r5" x="24.4" y="434.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-17)">-</text><text class="terminal-3171273326-r5" x="36.6" y="434.8" textLength="85.4" clip-path="url(#terminal-3171273326-line-17)">-docker</text><text class="terminal-3171273326-r5" x="122" y="434.8" textLength="73.2" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="459.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-18)">│</text><text class="terminal-3171273326-r4" x="475.8" y="459.2" textLength="549" clip-path="url(#terminal-3171273326-line-18)">[default:&#160;registry]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;</text><text class="terminal-3171273326-r4" x="1451.8" y="45 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="483.6" textLength="1464" clip-path="url(#terminal-3171273326-line-19)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="483.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-19)">
+</text><text class="terminal-3171273326-r4" x="0" y="508" textLength="24.4" clip-path="url(#terminal-3171273326-line-20)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="508" textLength="1415.2" clip-path="url(#terminal-3171273326-line-20)">&#160;Building&#160;images&#160;in&#160;parallel&#160;───────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="508" textLength="24.4" clip-path="url(#ter [...]
+</text><text class="terminal-3171273326-r4" x="0" y="532.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-21)">│</text><text class="terminal-3171273326-r5" x="24.4" y="532.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-21)">-</text><text class="terminal-3171273326-r5" x="36.6" y="532.4" textLength="48.8" clip-path="url(#terminal-3171273326-line-21)">-run</text><text class="terminal-3171273326-r5" x="85.4" y="532.4" textLength="146.4" clip-path="url(#terminal-3 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="556.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-22)">│</text><text class="terminal-3171273326-r5" x="24.4" y="556.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-22)">-</text><text class="terminal-3171273326-r5" x="36.6" y="556.8" textLength="146.4" clip-path="url(#terminal-3171273326-line-22)">-parallelism</text><text class="terminal-3171273326-r2" x="280.6" y="556.8" textLength="915" clip-path="url(#te [...]
+</text><text class="terminal-3171273326-r4" x="0" y="581.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-23)">│</text><text class="terminal-3171273326-r4" x="280.6" y="581.2" textLength="915" clip-path="url(#terminal-3171273326-line-23)">[default:&#160;4;&#160;1&lt;=x&lt;=8]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="605.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-24)">│</text><text class="terminal-3171273326-r5" x="24.4" y="605.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-24)">-</text><text class="terminal-3171273326-r5" x="36.6" y="605.6" textLength="85.4" clip-path="url(#terminal-3171273326-line-24)">-python</text><text class="terminal-3171273326-r5" x="122" y="605.6" textLength="109.8" clip-path="url(#terminal [...]
+</text><text class="terminal-3171273326-r4" x="0" y="630" textLength="12.2" clip-path="url(#terminal-3171273326-line-25)">│</text><text class="terminal-3171273326-r4" x="280.6" y="630" textLength="951.6" clip-path="url(#terminal-3171273326-line-25)">[default:&#160;3.7&#160;3.8&#160;3.9&#160;3.10]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="654.4" textLength="1464" clip-path="url(#terminal-3171273326-line-26)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="654.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-26)">
+</text><text class="terminal-3171273326-r4" x="0" y="678.8" textLength="24.4" clip-path="url(#terminal-3171273326-line-27)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="678.8" textLength="1415.2" clip-path="url(#terminal-3171273326-line-27)">&#160;Options&#160;for&#160;customizing&#160;images&#160;────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="678.8" textLength="24.4" clip-path="ur [...]
+</text><text class="terminal-3171273326-r4" x="0" y="703.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-28)">│</text><text class="terminal-3171273326-r5" x="24.4" y="703.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-28)">-</text><text class="terminal-3171273326-r5" x="36.6" y="703.2" textLength="97.6" clip-path="url(#terminal-3171273326-line-28)">-install</text><text class="terminal-3171273326-r5" x="134.2" y="703.2" textLength="280.6" clip-path="url(#termi [...]
+</text><text class="terminal-3171273326-r4" x="0" y="727.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-29)">│</text><text class="terminal-3171273326-r5" x="24.4" y="727.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-29)">-</text><text class="terminal-3171273326-r5" x="36.6" y="727.6" textLength="97.6" clip-path="url(#terminal-3171273326-line-29)">-airflow</text><text class="terminal-3171273326-r5" x="134.2" y="727.6" textLength="85.4" clip-path="url(#termin [...]
+</text><text class="terminal-3171273326-r4" x="0" y="752" textLength="12.2" clip-path="url(#terminal-3171273326-line-30)">│</text><text class="terminal-3171273326-r7" x="463.6" y="752" textLength="976" clip-path="url(#terminal-3171273326-line-30)">(TEXT)&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#16 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="776.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-31)">│</text><text class="terminal-3171273326-r4" x="463.6" y="776.4" textLength="976" clip-path="url(#terminal-3171273326-line-31)">[default:&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#1 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="800.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-32)">│</text><text class="terminal-3171273326-r4" x="463.6" y="800.8" textLength="976" clip-path="url(#terminal-3171273326-line-32)">amazon,async,celery,cncf.kubernetes,dask,docker,elasticsearch,ftp,google,google…</text><text class="terminal-3171273326-r4" x="1451.8" y="800.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-32)">│</text><text class="terminal-3 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="825.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-33)">│</text><text class="terminal-3171273326-r5" x="24.4" y="825.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-33)">-</text><text class="terminal-3171273326-r5" x="36.6" y="825.2" textLength="97.6" clip-path="url(#terminal-3171273326-line-33)">-airflow</text><text class="terminal-3171273326-r5" x="134.2" y="825.2" textLength="207.4" clip-path="url(#termi [...]
+</text><text class="terminal-3171273326-r4" x="0" y="849.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-34)">│</text><text class="terminal-3171273326-r7" x="463.6" y="849.6" textLength="866.2" clip-path="url(#terminal-3171273326-line-34)">(constraints&#160;|&#160;constraints-no-providers&#160;|&#160;constraints-source-providers)</text><text class="terminal-3171273326-r4" x="1451.8" y="849.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-34)">│</text><text clas [...]
+</text><text class="terminal-3171273326-r4" x="0" y="874" textLength="12.2" clip-path="url(#terminal-3171273326-line-35)">│</text><text class="terminal-3171273326-r4" x="463.6" y="874" textLength="866.2" clip-path="url(#terminal-3171273326-line-35)">[default:&#160;constraints]&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="898.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-36)">│</text><text class="terminal-3171273326-r5" x="24.4" y="898.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-36)">-</text><text class="terminal-3171273326-r5" x="36.6" y="898.4" textLength="97.6" clip-path="url(#terminal-3171273326-line-36)">-airflow</text><text class="terminal-3171273326-r5" x="134.2" y="898.4" textLength="268.4" clip-path="url(#termi [...]
+</text><text class="terminal-3171273326-r4" x="0" y="922.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-37)">│</text><text class="terminal-3171273326-r5" x="24.4" y="922.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-37)">-</text><text class="terminal-3171273326-r5" x="36.6" y="922.8" textLength="134.2" clip-path="url(#terminal-3171273326-line-37)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="922.8" textLength="146.4" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="947.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-38)">│</text><text class="terminal-3171273326-r5" x="24.4" y="947.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-38)">-</text><text class="terminal-3171273326-r5" x="36.6" y="947.2" textLength="134.2" clip-path="url(#terminal-3171273326-line-38)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="947.2" textLength="85.4" clip-path="url(#te [...]
+</text><text class="terminal-3171273326-r4" x="0" y="971.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-39)">│</text><text class="terminal-3171273326-r5" x="24.4" y="971.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-39)">-</text><text class="terminal-3171273326-r5" x="36.6" y="971.6" textLength="134.2" clip-path="url(#terminal-3171273326-line-39)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="971.6" textLength="207.4" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="996" textLength="12.2" clip-path="url(#terminal-3171273326-line-40)">│</text><text class="terminal-3171273326-r5" x="24.4" y="996" textLength="12.2" clip-path="url(#terminal-3171273326-line-40)">-</text><text class="terminal-3171273326-r5" x="36.6" y="996" textLength="134.2" clip-path="url(#terminal-3171273326-line-40)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="996" textLength="195.2" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1020.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-41)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1020.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-41)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1020.4" textLength="134.2" clip-path="url(#terminal-3171273326-line-41)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="1020.4" textLength="244" clip-path="url( [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1044.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-42)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1044.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-42)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1044.8" textLength="134.2" clip-path="url(#terminal-3171273326-line-42)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="1044.8" textLength="158.6" clip-path="ur [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1069.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-43)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1069.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-43)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1069.2" textLength="134.2" clip-path="url(#terminal-3171273326-line-43)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="1069.2" textLength="146.4" clip-path="ur [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1093.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-44)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1093.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-44)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1093.6" textLength="134.2" clip-path="url(#terminal-3171273326-line-44)">-additional</text><text class="terminal-3171273326-r5" x="170.8" y="1093.6" textLength="195.2" clip-path="ur [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1118" textLength="12.2" clip-path="url(#terminal-3171273326-line-45)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1118" textLength="12.2" clip-path="url(#terminal-3171273326-line-45)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1118" textLength="97.6" clip-path="url(#terminal-3171273326-line-45)">-runtime</text><text class="terminal-3171273326-r5" x="134.2" y="1118" textLength="109.8" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1142.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-46)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1142.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-46)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1142.4" textLength="97.6" clip-path="url(#terminal-3171273326-line-46)">-runtime</text><text class="terminal-3171273326-r5" x="134.2" y="1142.4" textLength="146.4" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1166.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-47)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1166.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-47)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1166.8" textLength="48.8" clip-path="url(#terminal-3171273326-line-47)">-dev</text><text class="terminal-3171273326-r5" x="85.4" y="1166.8" textLength="109.8" clip-path="url(#termin [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1191.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-48)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1191.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-48)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1191.2" textLength="48.8" clip-path="url(#terminal-3171273326-line-48)">-dev</text><text class="terminal-3171273326-r5" x="85.4" y="1191.2" textLength="146.4" clip-path="url(#termin [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1215.6" textLength="1464" clip-path="url(#terminal-3171273326-line-49)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="1215.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-49)">
+</text><text class="terminal-3171273326-r4" x="0" y="1240" textLength="24.4" clip-path="url(#terminal-3171273326-line-50)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="1240" textLength="1415.2" clip-path="url(#terminal-3171273326-line-50)">&#160;Customization&#160;options&#160;(for&#160;specific&#160;customization&#160;needs)&#160;──────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="1240" textLength="24.4" clip-p [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1264.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-51)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1264.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-51)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1264.4" textLength="97.6" clip-path="url(#terminal-3171273326-line-51)">-install</text><text class="terminal-3171273326-r5" x="134.2" y="1264.4" textLength="268.4" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1288.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-52)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1288.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-52)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1288.8" textLength="97.6" clip-path="url(#terminal-3171273326-line-52)">-airflow</text><text class="terminal-3171273326-r5" x="134.2" y="1288.8" textLength="170.8" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1313.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-53)">│</text><text class="terminal-3171273326-r2" x="536.8" y="1313.2" textLength="902.8" clip-path="url(#terminal-3171273326-line-53)">from&#160;PyPI&#160;or&#160;sources.&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&# [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1337.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-54)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1337.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-54)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1337.6" textLength="97.6" clip-path="url(#terminal-3171273326-line-54)">-cleanup</text><text class="terminal-3171273326-r5" x="134.2" y="1337.6" textLength="97.6" clip-path="url(#te [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1362" textLength="12.2" clip-path="url(#terminal-3171273326-line-55)">│</text><text class="terminal-3171273326-r2" x="536.8" y="1362" textLength="170.8" clip-path="url(#terminal-3171273326-line-55)">together&#160;with&#160;</text><text class="terminal-3171273326-r5" x="707.6" y="1362" textLength="12.2" clip-path="url(#terminal-3171273326-line-55)">-</text><text class="terminal-3171273326-r5" x="719.8" y="1362" textLength="97.6" clip-pa [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1386.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-56)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1386.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-56)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1386.4" textLength="97.6" clip-path="url(#terminal-3171273326-line-56)">-disable</text><text class="terminal-3171273326-r5" x="134.2" y="1386.4" textLength="317.2" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1410.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-57)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1410.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-57)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1410.8" textLength="97.6" clip-path="url(#terminal-3171273326-line-57)">-disable</text><text class="terminal-3171273326-r5" x="134.2" y="1410.8" textLength="317.2" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1435.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-58)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1435.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-58)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1435.2" textLength="97.6" clip-path="url(#terminal-3171273326-line-58)">-disable</text><text class="terminal-3171273326-r5" x="134.2" y="1435.2" textLength="353.8" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1459.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-59)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1459.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-59)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1459.6" textLength="97.6" clip-path="url(#terminal-3171273326-line-59)">-disable</text><text class="terminal-3171273326-r5" x="134.2" y="1459.6" textLength="231.8" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1484" textLength="12.2" clip-path="url(#terminal-3171273326-line-60)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1484" textLength="12.2" clip-path="url(#terminal-3171273326-line-60)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1484" textLength="97.6" clip-path="url(#terminal-3171273326-line-60)">-install</text><text class="terminal-3171273326-r5" x="134.2" y="1484" textLength="219.6" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1508.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-61)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1508.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-61)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1508.4" textLength="158.6" clip-path="url(#terminal-3171273326-line-61)">-installation</text><text class="terminal-3171273326-r5" x="195.2" y="1508.4" textLength="85.4" clip-path="u [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1532.8" textLength="1464" clip-path="url(#terminal-3171273326-line-62)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="1532.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-62)">
+</text><text class="terminal-3171273326-r4" x="0" y="1557.2" textLength="24.4" clip-path="url(#terminal-3171273326-line-63)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="1557.2" textLength="1415.2" clip-path="url(#terminal-3171273326-line-63)">&#160;Preparing&#160;cache&#160;and&#160;push&#160;(for&#160;maintainers&#160;and&#160;CI)&#160;─────────────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="1557.2" textLeng [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1581.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-64)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1581.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-64)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1581.6" textLength="85.4" clip-path="url(#terminal-3171273326-line-64)">-github</text><text class="terminal-3171273326-r5" x="122" y="1581.6" textLength="73.2" clip-path="url(#termi [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1606" textLength="12.2" clip-path="url(#terminal-3171273326-line-65)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1606" textLength="12.2" clip-path="url(#terminal-3171273326-line-65)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1606" textLength="85.4" clip-path="url(#terminal-3171273326-line-65)">-github</text><text class="terminal-3171273326-r5" x="122" y="1606" textLength="109.8" clip-path="url(#terminal-317 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1630.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-66)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1630.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-66)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1630.4" textLength="109.8" clip-path="url(#terminal-3171273326-line-66)">-platform</text><text class="terminal-3171273326-r2" x="341.6" y="1630.4" textLength="329.4" clip-path="url( [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1654.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-67)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1654.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-67)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1654.8" textLength="61" clip-path="url(#terminal-3171273326-line-67)">-push</text><text class="terminal-3171273326-r5" x="97.6" y="1654.8" textLength="73.2" clip-path="url(#terminal [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1679.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-68)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1679.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-68)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1679.2" textLength="73.2" clip-path="url(#terminal-3171273326-line-68)">-empty</text><text class="terminal-3171273326-r5" x="109.8" y="1679.2" textLength="73.2" clip-path="url(#term [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1703.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-69)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1703.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-69)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1703.6" textLength="97.6" clip-path="url(#terminal-3171273326-line-69)">-prepare</text><text class="terminal-3171273326-r5" x="134.2" y="1703.6" textLength="158.6" clip-path="url(#t [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1728" textLength="12.2" clip-path="url(#terminal-3171273326-line-70)">│</text><text class="terminal-3171273326-r2" x="341.6" y="1728" textLength="1098" clip-path="url(#terminal-3171273326-line-70)">image).&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160;&#160; [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1752.4" textLength="1464" clip-path="url(#terminal-3171273326-line-71)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="1752.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-71)">
+</text><text class="terminal-3171273326-r4" x="0" y="1776.8" textLength="24.4" clip-path="url(#terminal-3171273326-line-72)">╭─</text><text class="terminal-3171273326-r4" x="24.4" y="1776.8" textLength="1415.2" clip-path="url(#terminal-3171273326-line-72)">&#160;Options&#160;───────────────────────────────────────────────────────────────────────────────────────────────────────────</text><text class="terminal-3171273326-r4" x="1439.6" y="1776.8" textLength="24.4" clip-path="url(#terminal- [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1801.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-73)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1801.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-73)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1801.2" textLength="85.4" clip-path="url(#terminal-3171273326-line-73)">-github</text><text class="terminal-3171273326-r5" x="122" y="1801.2" textLength="134.2" clip-path="url(#term [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1825.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-74)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1825.6" textLength="12.2" clip-path="url(#terminal-3171273326-line-74)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1825.6" textLength="85.4" clip-path="url(#terminal-3171273326-line-74)">-answer</text><text class="terminal-3171273326-r6" x="280.6" y="1825.6" textLength="24.4" clip-path="url(#ter [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1850" textLength="12.2" clip-path="url(#terminal-3171273326-line-75)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1850" textLength="12.2" clip-path="url(#terminal-3171273326-line-75)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1850" textLength="48.8" clip-path="url(#terminal-3171273326-line-75)">-dry</text><text class="terminal-3171273326-r5" x="85.4" y="1850" textLength="48.8" clip-path="url(#terminal-317127 [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1874.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-76)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1874.4" textLength="12.2" clip-path="url(#terminal-3171273326-line-76)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1874.4" textLength="97.6" clip-path="url(#terminal-3171273326-line-76)">-verbose</text><text class="terminal-3171273326-r6" x="280.6" y="1874.4" textLength="24.4" clip-path="url(#te [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1898.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-77)">│</text><text class="terminal-3171273326-r5" x="24.4" y="1898.8" textLength="12.2" clip-path="url(#terminal-3171273326-line-77)">-</text><text class="terminal-3171273326-r5" x="36.6" y="1898.8" textLength="61" clip-path="url(#terminal-3171273326-line-77)">-help</text><text class="terminal-3171273326-r6" x="280.6" y="1898.8" textLength="24.4" clip-path="url(#termina [...]
+</text><text class="terminal-3171273326-r4" x="0" y="1923.2" textLength="1464" clip-path="url(#terminal-3171273326-line-78)">╰──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────╯</text><text class="terminal-3171273326-r2" x="1464" y="1923.2" textLength="12.2" clip-path="url(#terminal-3171273326-line-78)">
 </text>
     </g>
     </g>
diff --git a/images/breeze/output-commands-hash.txt b/images/breeze/output-commands-hash.txt
index 70ac9b305d..fffdd1e618 100644
--- a/images/breeze/output-commands-hash.txt
+++ b/images/breeze/output-commands-hash.txt
@@ -1 +1 @@
-8b4116c1808c84d491961283a4ddbec2
+c6435adeff6ad4b283110d516173920f
diff --git a/scripts/ci/selective_ci_checks.sh b/scripts/ci/selective_ci_checks.sh
index c9f215c53c..69572d6024 100755
--- a/scripts/ci/selective_ci_checks.sh
+++ b/scripts/ci/selective_ci_checks.sh
@@ -55,6 +55,8 @@ function output_all_basic_variables() {
             "$(initialization::parameters_to_json "${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS[@]}")"
         initialization::ga_output all-python-versions \
             "$(initialization::parameters_to_json "${ALL_PYTHON_MAJOR_MINOR_VERSIONS[@]}")"
+        initialization::ga_output all-python-versions-list-as-string \
+            "$(initialization::parameters_to_json "${ALL_PYTHON_MAJOR_MINOR_VERSIONS[@]}")"
         initialization::ga_output python-versions-list-as-string "${CURRENT_PYTHON_MAJOR_MINOR_VERSIONS[*]}"
         initialization::ga_output kubernetes-versions-list-as-string "${CURRENT_KUBERNETES_VERSIONS[*]}"
     else
@@ -64,6 +66,8 @@ function output_all_basic_variables() {
         # all-python-versions are used in BuildImage Workflow
         initialization::ga_output all-python-versions \
             "$(initialization::parameters_to_json "${DEFAULT_PYTHON_MAJOR_MINOR_VERSION}")"
+        initialization::ga_output all-python-versions-list-as-string \
+            "$(initialization::parameters_to_json "${DEFAULT_PYTHON_MAJOR_MINOR_VERSION}")"
         initialization::ga_output python-versions-list-as-string "${DEFAULT_PYTHON_MAJOR_MINOR_VERSION}"
         initialization::ga_output kubernetes-versions-list-as-string "${DEFAULT_KUBERNETES_VERSION}"
     fi