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/04/23 18:17:25 UTC

[airflow] branch main updated: Fix and improve consistency of checking command return code (#23189)

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

potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new be51aece54 Fix and improve consistency of checking command return code (#23189)
be51aece54 is described below

commit be51aece54ef98a8868845ad8033f08689dd7ad1
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Sat Apr 23 20:17:17 2022 +0200

    Fix and improve consistency of checking command return code (#23189)
    
    This is an aftermath of #23104 after switchig to docs building
    by breeze, failure of build documentation did not trigger failure
    of the docs build (but it did trigger main failure of pushing
    the documentation).
    
    This change improves and simplifies the return code processing and
    propagation in the commands executed by breeze - thanks to common
    returncode, stdout, stderr available in both CompletedProcess
    and CalledProcessError and returning fake CompletedProcess in dry_run
    mode, we can also satisfy MyPy type check by returning non-optional
    Union of those two types which simplifies returncode processing.
    
    This change fixes the error in the docs (lack of empty lines before
    auto-generated extras).
    
    All commands have been reviewed to see if the returncode is
    correctly handled where needed.
---
 CI.rst                                             |   2 -
 dev/breeze/src/airflow_breeze/breeze.py            |  14 ++-
 .../build_image/ci/build_ci_image.py               |  11 +-
 .../build_image/prod/build_prod_image.py           |  13 ++-
 dev/breeze/src/airflow_breeze/shell/enter_shell.py |  13 ++-
 .../src/airflow_breeze/shell/shell_params.py       |  15 +--
 dev/breeze/src/airflow_breeze/utils/constraints.py |   6 +-
 .../airflow_breeze/utils/docker_command_utils.py   | 127 ++++++++++++---------
 .../src/airflow_breeze/utils/host_info_utils.py    |   8 --
 dev/breeze/src/airflow_breeze/utils/pulll_image.py |  20 ++--
 dev/breeze/src/airflow_breeze/utils/registry.py    |   6 +-
 dev/breeze/src/airflow_breeze/utils/run_tests.py   |  30 ++---
 dev/breeze/src/airflow_breeze/utils/run_utils.py   |  52 ++++-----
 dev/breeze/src/airflow_ci/freespace.py             |   2 +-
 dev/breeze/tests/test_docker_command_utils.py      |  76 ++++++------
 docs/docker-stack/build-arg-ref.rst                |   2 +
 scripts/ci/docker-compose/_docker.env              |   1 -
 scripts/ci/docker-compose/base.yml                 |   1 -
 scripts/ci/libraries/_initialization.sh            |   5 -
 .../pre_commit_check_order_dockerfile_extras.py    |  28 ++++-
 scripts/in_container/run_docs_build.sh             |   1 +
 21 files changed, 225 insertions(+), 208 deletions(-)

diff --git a/CI.rst b/CI.rst
index 74623c60e1..f9c5777692 100644
--- a/CI.rst
+++ b/CI.rst
@@ -186,8 +186,6 @@ You can use those variables when you try to reproduce the build locally.
 +-----------------------------------------+-------------+--------------+------------+-------------------------------------------------+
 | ``HOST_OS``                             |             |    Linux     |    Linux   | OS of the Host (Darwin/Linux).                  |
 +-----------------------------------------+-------------+--------------+------------+-------------------------------------------------+
-| ``HOST_HOME``                           |             |              |            | Home directory on the host.                     |
-+-----------------------------------------+-------------+--------------+------------+-------------------------------------------------+
 |                                                            Git variables                                                            |
 +-----------------------------------------+-------------+--------------+------------+-------------------------------------------------+
 | ``COMMIT_SHA``                          |             | GITHUB_SHA   | GITHUB_SHA | SHA of the commit of the build is run           |
diff --git a/dev/breeze/src/airflow_breeze/breeze.py b/dev/breeze/src/airflow_breeze/breeze.py
index b743cd47ab..cb8d6c2023 100755
--- a/dev/breeze/src/airflow_breeze/breeze.py
+++ b/dev/breeze/src/airflow_breeze/breeze.py
@@ -1760,6 +1760,7 @@ def build_docs(
         spellcheck_only=spellcheck_only,
     )
     extra_docker_flags = get_extra_docker_flags(MOUNT_SELECTED)
+    env = construct_env_variables_docker_compose_command(params)
     cmd = [
         "docker",
         "run",
@@ -1767,13 +1768,16 @@ def build_docs(
         *extra_docker_flags,
         "-e",
         "GITHUB_ACTIONS=",
+        "-e",
+        "SKIP_ENVIRONMENT_INITIALIZATION=true",
         "--pull",
         "never",
         ci_image_name,
         "/opt/airflow/scripts/in_container/run_docs_build.sh",
         *doc_builder.args_doc_builder,
     ]
-    run_command(cmd, verbose=verbose, dry_run=dry_run, text=True)
+    process = run_command(cmd, verbose=verbose, dry_run=dry_run, text=True, env=env, check=False)
+    sys.exit(process.returncode)
 
 
 @main.command(
@@ -1830,7 +1834,7 @@ def static_checks(
             command_to_execute.extend(precommit_args)
         env = os.environ.copy()
         env['GITHUB_REPOSITORY'] = github_repository
-        run_command(
+        static_checks_result = run_command(
             command_to_execute,
             verbose=verbose,
             dry_run=dry_run,
@@ -1839,6 +1843,8 @@ def static_checks(
             text=True,
             env=env,
         )
+        if static_checks_result.returncode != 0:
+            console.print("[red]There were errors during pre-commit check. They should be fixed[/n]")
 
 
 @main.command(name="stop", help="Stop running breeze environment.")
@@ -2186,10 +2192,10 @@ def cleanup(verbose: bool, dry_run: bool, github_repository: str, all: bool, ans
             '--format',
             '{{.Repository}}:{{.Tag}}',
         ]
-        process = run_command(
+        command_result = run_command(
             docker_images_command_to_execute, verbose=verbose, text=True, capture_output=True
         )
-        images = process.stdout.splitlines() if process and process.stdout else []
+        images = command_result.stdout.splitlines() if command_result and command_result.stdout else []
         if images:
             console.print("[light_blue]Removing images:[/]")
             for image in images:
diff --git a/dev/breeze/src/airflow_breeze/build_image/ci/build_ci_image.py b/dev/breeze/src/airflow_breeze/build_image/ci/build_ci_image.py
index 124e0517b0..9740da54b2 100644
--- a/dev/breeze/src/airflow_breeze/build_image/ci/build_ci_image.py
+++ b/dev/breeze/src/airflow_breeze/build_image/ci/build_ci_image.py
@@ -42,7 +42,6 @@ from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, BUILD_CACHE_DI
 from airflow_breeze.utils.registry import login_to_docker_registry
 from airflow_breeze.utils.run_utils import (
     fix_group_permissions,
-    get_return_code,
     instruct_build_image,
     is_repo_rebased,
     run_command,
@@ -158,7 +157,7 @@ def build_ci_image(
         if ci_image_params.empty_image:
             console.print(f"\n[blue]Building empty CI Image for Python {ci_image_params.python}\n")
             cmd = construct_empty_docker_build_command(image_params=ci_image_params)
-            run_command(
+            build_result = run_command(
                 cmd,
                 input="FROM scratch\n",
                 verbose=verbose,
@@ -168,11 +167,11 @@ def build_ci_image(
             )
         else:
             console.print(f"\n[blue]Building CI Image for Python {ci_image_params.python}\n")
-            process = run_command(
+            build_result = run_command(
                 cmd, verbose=verbose, dry_run=dry_run, cwd=AIRFLOW_SOURCES_ROOT, text=True, check=False
             )
         if not dry_run:
-            if process and process.returncode == 0:
+            if build_result.returncode == 0:
                 ci_image_cache_dir = BUILD_CACHE_DIR / ci_image_params.airflow_branch
                 ci_image_cache_dir.mkdir(parents=True, exist_ok=True)
                 touch_cache_file(f"built_{ci_image_params.python}", root_dir=ci_image_cache_dir)
@@ -180,14 +179,14 @@ def build_ci_image(
             else:
                 console.print("[red]Error when building image![/]")
                 return (
-                    get_return_code(process=process, dry_run=dry_run),
+                    build_result.returncode,
                     f"Image build: {ci_image_params.python}",
                 )
         else:
             console.print("[blue]Not updating build cache because we are in `dry_run` mode.[/]")
         if ci_image_params.push_image:
             return tag_and_push_image(image_params=ci_image_params, dry_run=dry_run, verbose=verbose)
-        return get_return_code(process=process, dry_run=dry_run), f"Image build: {ci_image_params.python}"
+        return build_result.returncode, f"Image build: {ci_image_params.python}"
 
 
 def get_ci_image_build_params(parameters_passed: Dict) -> BuildCiParams:
diff --git a/dev/breeze/src/airflow_breeze/build_image/prod/build_prod_image.py b/dev/breeze/src/airflow_breeze/build_image/prod/build_prod_image.py
index 7883677eb5..87383c455b 100644
--- a/dev/breeze/src/airflow_breeze/build_image/prod/build_prod_image.py
+++ b/dev/breeze/src/airflow_breeze/build_image/prod/build_prod_image.py
@@ -30,7 +30,7 @@ from airflow_breeze.utils.docker_command_utils import (
 )
 from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, DOCKER_CONTEXT_DIR
 from airflow_breeze.utils.registry import login_to_docker_registry
-from airflow_breeze.utils.run_utils import fix_group_permissions, get_return_code, run_command
+from airflow_breeze.utils.run_utils import fix_group_permissions, run_command
 
 REQUIRED_PROD_IMAGE_ARGS = [
     "python_base_image",
@@ -187,12 +187,13 @@ def build_production_image(
         if prod_image_params.empty_image:
             console.print(f"\n[blue]Building empty PROD Image for Python {prod_image_params.python}\n")
             cmd = construct_empty_docker_build_command(image_params=prod_image_params)
-            process = run_command(
+            build_command_result = run_command(
                 cmd,
                 input="FROM scratch\n",
                 verbose=verbose,
                 dry_run=dry_run,
                 cwd=AIRFLOW_SOURCES_ROOT,
+                check=False,
                 text=True,
             )
         else:
@@ -203,8 +204,10 @@ def build_production_image(
                 optional_args=OPTIONAL_PROD_IMAGE_ARGS,
                 production_image=True,
             )
-            process = run_command(cmd, verbose=verbose, dry_run=dry_run, cwd=AIRFLOW_SOURCES_ROOT, text=True)
-        if process and process.returncode == 0:
+            build_command_result = run_command(
+                cmd, verbose=verbose, dry_run=dry_run, cwd=AIRFLOW_SOURCES_ROOT, check=False, text=True
+            )
+        if build_command_result.returncode == 0:
             if prod_image_params.push_image:
                 return tag_and_push_image(image_params=prod_image_params, dry_run=dry_run, verbose=verbose)
-        return get_return_code(process=process, dry_run=dry_run), f"Image build: {prod_image_params.python}"
+        return build_command_result.returncode, f"Image build: {prod_image_params.python}"
diff --git a/dev/breeze/src/airflow_breeze/shell/enter_shell.py b/dev/breeze/src/airflow_breeze/shell/enter_shell.py
index 0a426b3713..855a24f1d6 100644
--- a/dev/breeze/src/airflow_breeze/shell/enter_shell.py
+++ b/dev/breeze/src/airflow_breeze/shell/enter_shell.py
@@ -15,9 +15,10 @@
 # specific language governing permissions and limitations
 # under the License.
 """Command to enter container shell for Breeze."""
+import subprocess
 import sys
 from pathlib import Path
-from typing import Dict
+from typing import Dict, Union
 
 from airflow_breeze import global_constants
 from airflow_breeze.build_image.ci.build_ci_image import build_ci_image, get_ci_image_build_params
@@ -66,7 +67,7 @@ def synchronize_cached_params(parameters_passed_by_the_user: Dict[str, str]) ->
     return updated_params
 
 
-def enter_shell(**kwargs):
+def enter_shell(**kwargs) -> Union[subprocess.CompletedProcess, subprocess.CalledProcessError]:
     """
     Executes entering shell using the parameters passed as kwargs:
 
@@ -94,10 +95,12 @@ def enter_shell(**kwargs):
     if read_from_cache_file('suppress_cheatsheet') is None:
         console.print(CHEATSHEET, style=CHEATSHEET_STYLE)
     enter_shell_params = ShellParams(**filter_out_none(**updated_kwargs))
-    run_shell_with_build_image_checks(verbose, dry_run, enter_shell_params)
+    return run_shell_with_build_image_checks(verbose, dry_run, enter_shell_params)
 
 
-def run_shell_with_build_image_checks(verbose: bool, dry_run: bool, shell_params: ShellParams):
+def run_shell_with_build_image_checks(
+    verbose: bool, dry_run: bool, shell_params: ShellParams
+) -> Union[subprocess.CompletedProcess, subprocess.CalledProcessError]:
     """
     Executes a shell command built from params passed, checking if build is not needed.
     * checks if there are enough resources to run shell
@@ -135,4 +138,4 @@ def run_shell_with_build_image_checks(verbose: bool, dry_run: bool, shell_params
     env_variables = construct_env_variables_docker_compose_command(shell_params)
     if cmd_added is not None:
         cmd.extend(['-c', cmd_added])
-    run_command(cmd, verbose=verbose, dry_run=dry_run, env=env_variables, text=True)
+    return run_command(cmd, verbose=verbose, dry_run=dry_run, env=env_variables, text=True)
diff --git a/dev/breeze/src/airflow_breeze/shell/shell_params.py b/dev/breeze/src/airflow_breeze/shell/shell_params.py
index 45acd83085..eb4ae34a7a 100644
--- a/dev/breeze/src/airflow_breeze/shell/shell_params.py
+++ b/dev/breeze/src/airflow_breeze/shell/shell_params.py
@@ -32,7 +32,6 @@ from airflow_breeze.global_constants import (
     get_airflow_version,
 )
 from airflow_breeze.utils.console import console
-from airflow_breeze.utils.host_info_utils import get_host_group_id, get_host_user_id, get_stat_bin
 from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, BUILD_CACHE_DIR, SCRIPTS_CI_DIR
 from airflow_breeze.utils.run_utils import get_filesystem_type, run_command
 
@@ -80,15 +79,7 @@ class ShellParams:
         cmd = ['docker', 'run', '--entrypoint', '/bin/bash', f'{self.airflow_image_name}']
         cmd.extend(['-c', 'echo "${AIRFLOW_VERSION}"'])
         output = run_command(cmd, capture_output=True, text=True)
-        return output.stdout.strip()
-
-    @property
-    def host_user_id(self):
-        return get_host_user_id()
-
-    @property
-    def host_group_id(self):
-        return get_host_group_id()
+        return output.stdout.strip() if output.stdout else "UNKNOWN_VERSION"
 
     @property
     def airflow_base_image_name(self) -> str:
@@ -214,7 +205,3 @@ class ShellParams:
         if len(self.extra_args) > 0:
             cmd = str(self.extra_args[0])
         return cmd
-
-    @property
-    def get_stat_bin(self):
-        return get_stat_bin()
diff --git a/dev/breeze/src/airflow_breeze/utils/constraints.py b/dev/breeze/src/airflow_breeze/utils/constraints.py
index 77605678ae..e8b28ca7dd 100644
--- a/dev/breeze/src/airflow_breeze/utils/constraints.py
+++ b/dev/breeze/src/airflow_breeze/utils/constraints.py
@@ -25,7 +25,7 @@ from airflow_breeze.utils.docker_command_utils import (
     get_extra_docker_flags,
 )
 from airflow_breeze.utils.parallel import check_async_run_results
-from airflow_breeze.utils.run_utils import get_return_code, run_command
+from airflow_breeze.utils.run_utils import run_command
 
 
 def run_generate_constraints(
@@ -47,9 +47,9 @@ def run_generate_constraints(
         shell_params.airflow_image_name_with_tag,
         "/opt/airflow/scripts/in_container/run_generate_constraints.sh",
     ]
-    process = run_command(cmd_to_run, verbose=verbose, dry_run=dry_run, env=env_variables)
+    generate_constraints_result = run_command(cmd_to_run, verbose=verbose, dry_run=dry_run, env=env_variables)
     return (
-        get_return_code(process=process, dry_run=dry_run),
+        generate_constraints_result.returncode,
         f"Generate constraints Python {shell_params.python}:{generate_constraints_mode}",
     )
 
diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
index 05d02c746c..f6e59d1ea0 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -23,7 +23,7 @@ from typing import Dict, List, Tuple, Union
 from airflow_breeze.build_image.ci.build_ci_params import BuildCiParams
 from airflow_breeze.build_image.prod.build_prod_params import BuildProdParams
 from airflow_breeze.shell.shell_params import ShellParams
-from airflow_breeze.utils.host_info_utils import get_host_os
+from airflow_breeze.utils.host_info_utils import get_host_group_id, get_host_os, get_host_user_id
 from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT
 from airflow_breeze.utils.registry import login_to_docker_registry
 
@@ -40,7 +40,6 @@ from airflow_breeze.global_constants import (
     MIN_DOCKER_COMPOSE_VERSION,
     MIN_DOCKER_VERSION,
     MOUNT_ALL,
-    MOUNT_NONE,
     MOUNT_SELECTED,
     MSSQL_HOST_PORT,
     MSSQL_VERSION,
@@ -53,7 +52,7 @@ from airflow_breeze.global_constants import (
     WEBSERVER_HOST_PORT,
 )
 from airflow_breeze.utils.console import console
-from airflow_breeze.utils.run_utils import commit_sha, get_return_code, prepare_build_command, run_command
+from airflow_breeze.utils.run_utils import commit_sha, prepare_build_command, run_command
 
 NECESSARY_HOST_VOLUMES = [
     "/.bash_aliases:/root/.bash_aliases:cached",
@@ -110,22 +109,31 @@ def get_extra_docker_flags(mount_sources: str) -> List[str]:
     return extra_docker_flags
 
 
-def check_docker_resources(verbose: bool, airflow_image_name: str):
+def check_docker_resources(
+    verbose: bool, airflow_image_name: str
+) -> Union[subprocess.CompletedProcess, subprocess.CalledProcessError]:
     """
     Check if we have enough resources to run docker. This is done via running script embedded in our image.
     :param verbose: print commands when running
     :param airflow_image_name: name of the airflow image to use.
     """
-    extra_docker_flags = get_extra_docker_flags(MOUNT_NONE)
-    cmd = []
-    cmd.extend(["docker", "run", "-t"])
-    cmd.extend(extra_docker_flags)
-    cmd.extend(["--entrypoint", "/bin/bash", airflow_image_name])
-    cmd.extend(["-c", "python /opt/airflow/scripts/in_container/run_resource_check.py"])
-    run_command(cmd, verbose=verbose, text=True)
+    return run_command(
+        cmd=[
+            "docker",
+            "run",
+            "-t",
+            "--entrypoint",
+            "/bin/bash",
+            airflow_image_name,
+            "-c",
+            "python /opt/airflow/scripts/in_container/run_resource_check.py",
+        ],
+        verbose=verbose,
+        text=True,
+    )
 
 
-def check_docker_permission(verbose) -> bool:
+def check_docker_permission_denied(verbose) -> bool:
     """
     Checks if we have permission to write to docker socket. By default, on Linux you need to add your user
     to docker group and some new users do not realize that. We help those users if we have
@@ -136,18 +144,17 @@ def check_docker_permission(verbose) -> bool:
     """
     permission_denied = False
     docker_permission_command = ["docker", "info"]
-    try:
-        _ = run_command(
-            docker_permission_command,
-            verbose=verbose,
-            no_output_dump_on_exception=True,
-            capture_output=True,
-            text=True,
-            check=True,
-        )
-    except subprocess.CalledProcessError as ex:
+    command_result = run_command(
+        docker_permission_command,
+        verbose=verbose,
+        no_output_dump_on_exception=True,
+        capture_output=True,
+        text=True,
+        check=False,
+    )
+    if command_result.returncode != 0:
         permission_denied = True
-        if ex.stdout and 'Got permission denied while trying to connect' in ex.stdout:
+        if command_result.stdout and 'Got permission denied while trying to connect' in command_result.stdout:
             console.print('ERROR: You have `permission denied` error when trying to communicate with docker.')
             console.print(
                 'Most likely you need to add your user to `docker` group: \
@@ -171,11 +178,11 @@ def check_docker_is_running(verbose: bool) -> bool:
         verbose=verbose,
         no_output_dump_on_exception=True,
         text=False,
-        check=True,
         stdout=subprocess.DEVNULL,
         stderr=subprocess.STDOUT,
+        check=False,
     )
-    if not response:
+    if response.returncode != 0:
         return False
     return True
 
@@ -189,32 +196,36 @@ def check_docker_version(verbose: bool):
 
     :param verbose: print commands when running
     """
-    permission_denied = check_docker_permission(verbose)
+    permission_denied = check_docker_permission_denied(verbose)
     if not permission_denied:
         docker_version_command = ['docker', 'version', '--format', '{{.Client.Version}}']
         docker_version = ''
-        docker_version_output = run_command(
+        docker_version_result = run_command(
             docker_version_command,
             verbose=verbose,
             no_output_dump_on_exception=True,
             capture_output=True,
             text=True,
+            check=False,
         )
-        if docker_version_output and docker_version_output.returncode == 0:
-            docker_version = docker_version_output.stdout.strip()
+        if docker_version_result.returncode == 0:
+            docker_version = docker_version_result.stdout.strip()
         if docker_version == '':
             console.print(
-                f'Your version of docker is unknown. If the scripts fail, please make sure to \
-                    install docker at least: {MIN_DOCKER_VERSION} version.'
+                f"""
+[yellow]Your version of docker is unknown. If the scripts fail, please make sure to[/]
+[yellow]install docker at least: {MIN_DOCKER_VERSION} version.[/]
+"""
             )
         else:
             good_version = compare_version(docker_version, MIN_DOCKER_VERSION)
             if good_version:
-                console.print(f'Good version of Docker: {docker_version}.')
+                console.print(f'[green]Good version of Docker: {docker_version}.[/]')
             else:
                 console.print(
-                    f'Your version of docker is too old:{docker_version}. Please upgrade to \
-                    at least {MIN_DOCKER_VERSION}'
+                    f"""
+[yellow]Your version of docker is too old:{docker_version}. Please upgrade to at least {MIN_DOCKER_VERSION}[/]
+"""
                 )
 
 
@@ -228,34 +239,39 @@ def check_docker_compose_version(verbose: bool):
     """
     version_pattern = re.compile(r'(\d+)\.(\d+)\.(\d+)')
     docker_compose_version_command = ["docker-compose", "--version"]
-    docker_compose_version_output = run_command(
+    docker_compose_version_result = run_command(
         docker_compose_version_command,
         verbose=verbose,
         no_output_dump_on_exception=True,
         capture_output=True,
         text=True,
     )
-    if docker_compose_version_output and docker_compose_version_output.returncode == 0:
-        docker_compose_version = docker_compose_version_output.stdout
+    if docker_compose_version_result.returncode == 0:
+        docker_compose_version = docker_compose_version_result.stdout
         version_extracted = version_pattern.search(docker_compose_version)
         if version_extracted is not None:
             version = '.'.join(version_extracted.groups())
             good_version = compare_version(version, MIN_DOCKER_COMPOSE_VERSION)
             if good_version:
-                console.print(f'Good version of docker-compose: {version}')
+                console.print(f'[green]Good version of docker-compose: {version}[/]')
             else:
                 console.print(
-                    f'You have too old version of docker-compose: {version}! \
-                At least 1.29 is needed! Please upgrade!'
+                    f"""
+[yellow]You have too old version of docker-compose: {version}! At least 1.29 is needed! Please upgrade!
+"""
                 )
                 console.print(
-                    'See https://docs.docker.com/compose/install/ for instructions. \
-                Make sure docker-compose you install is first on the PATH variable of yours.'
+                    """
+See https://docs.docker.com/compose/install/ for instructions.
+Make sure docker-compose you install is first on the PATH variable of yours.
+"""
                 )
     else:
         console.print(
-            'Unknown docker-compose version. At least 1.29 is needed! \
-        If Breeze fails upgrade to latest available docker-compose version'
+            """
+[yellow]Unknown docker-compose version. At least 1.29 is needed![/]
+[yellow]If Breeze fails upgrade to latest available docker-compose version.[/]
+"""
         )
 
 
@@ -361,23 +377,20 @@ def tag_and_push_image(
         f"{image_params.airflow_image_name_with_tag}.[/]"
     )
     cmd = construct_docker_tag_command(image_params)
-    process = run_command(
+    command_result = run_command(
         cmd, verbose=verbose, dry_run=dry_run, cwd=AIRFLOW_SOURCES_ROOT, text=True, check=False
     )
-    if process and process.returncode == 0:
+    if command_result.returncode == 0:
         return_code, info = login_to_docker_registry(image_params, dry_run=dry_run)
         if return_code != 0:
             return return_code, f"Tag and pushing the image {image_params.python}: {info}"
         cmd = construct_docker_push_command(image_params)
-        process = run_command(
+        command_result = run_command(
             cmd, verbose=verbose, dry_run=dry_run, cwd=AIRFLOW_SOURCES_ROOT, text=True, check=False
         )
-        if process and process.returncode == 0:
+        if command_result.returncode == 0:
             return 0, f"Tag and pushing the image {image_params.python}"
-    return (
-        get_return_code(process=process, dry_run=dry_run),
-        f"Tag and pushing the image {image_params.python}",
-    )
+    return command_result.returncode, f"Tag and pushing the image {image_params.python}"
 
 
 def construct_empty_docker_build_command(
@@ -427,6 +440,8 @@ def update_expected_environment_variables(env: Dict[str, str]) -> None:
     set_value_to_default_if_not_set(env, 'GENERATE_CONSTRAINTS_MODE', "source-providers")
     set_value_to_default_if_not_set(env, 'GITHUB_REGISTRY_PULL_IMAGE_TAG', "latest")
     set_value_to_default_if_not_set(env, 'HOST_OS', get_host_os())
+    set_value_to_default_if_not_set(env, 'HOST_USER_ID', get_host_user_id())
+    set_value_to_default_if_not_set(env, 'HOST_GROUP_ID', get_host_group_id())
     set_value_to_default_if_not_set(env, 'INIT_SCRIPT_FILE', "init.sh")
     set_value_to_default_if_not_set(env, 'INSTALL_PROVIDERS_FROM_SOURCES', "true")
     set_value_to_default_if_not_set(env, 'INSTALL_FROM_DOCKER_CONTEXT_FILES', "false")
@@ -506,7 +521,9 @@ SOURCE_OF_DEFAULT_VALUES_FOR_VARIABLES = {
 }
 
 
-def construct_env_variables_docker_compose_command(shell_params: ShellParams) -> Dict[str, str]:
+def construct_env_variables_docker_compose_command(
+    params: Union[ShellParams, BuildCiParams]
+) -> Dict[str, str]:
     """
     Constructs environment variables needed by the docker-compose command, based on Shell parameters
     passed to it.
@@ -517,13 +534,13 @@ def construct_env_variables_docker_compose_command(shell_params: ShellParams) ->
       if the env variable that we run with does not have it.
     * Updates all other environment variables that docker-compose expects with default values if missing
 
-    :param shell_params: shell parameters passed
+    :param params: shell parameters passed
     :return: dictionary of env variables to set
     """
     env_variables: Dict[str, str] = os.environ.copy()
     for variable in MAP_ENV_VARIABLES_TO_PARAMS_FIELDS:
         param_name = MAP_ENV_VARIABLES_TO_PARAMS_FIELDS[variable]
-        param_value = getattr(shell_params, param_name)
+        param_value = getattr(params, param_name, None)
         env_variables[variable] = str(param_value) if param_value is not None else ""
     # Set constant defaults if not defined
     for variable in VARIABLES_FOR_DOCKER_COMPOSE_CONSTANTS:
diff --git a/dev/breeze/src/airflow_breeze/utils/host_info_utils.py b/dev/breeze/src/airflow_breeze/utils/host_info_utils.py
index 3f34153de3..0e3db45a3e 100644
--- a/dev/breeze/src/airflow_breeze/utils/host_info_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/host_info_utils.py
@@ -43,11 +43,3 @@ def get_host_os():
     # Mac: Darwin
     # Windows: Windows
     return platform.system()
-
-
-def get_stat_bin():
-    os = get_host_os()
-    stat_bin = 'stat'
-    if os == 'Darwin':
-        stat_bin = 'gstat'
-    return stat_bin
diff --git a/dev/breeze/src/airflow_breeze/utils/pulll_image.py b/dev/breeze/src/airflow_breeze/utils/pulll_image.py
index 4e470e032c..a60a3b15e2 100644
--- a/dev/breeze/src/airflow_breeze/utils/pulll_image.py
+++ b/dev/breeze/src/airflow_breeze/utils/pulll_image.py
@@ -24,7 +24,7 @@ from airflow_breeze.build_image.prod.build_prod_params import BuildProdParams
 from airflow_breeze.utils.console import console
 from airflow_breeze.utils.parallel import check_async_run_results
 from airflow_breeze.utils.run_tests import verify_an_image
-from airflow_breeze.utils.run_utils import get_return_code, run_command
+from airflow_breeze.utils.run_utils import run_command
 
 
 def run_pull_in_parallel(
@@ -96,14 +96,14 @@ def run_pull_image(
     )
     while True:
         command_to_run = ["docker", "pull", image_params.airflow_image_name_with_tag]
-        process = run_command(
+        command_result = run_command(
             command_to_run,
             verbose=verbose,
             dry_run=dry_run,
             check=False,
         )
-        if dry_run or process and process.returncode == 0:
-            process = run_command(
+        if command_result.returncode == 0:
+            command_result = run_command(
                 ["docker", "inspect", image_params.airflow_image_name_with_tag, "-f", "{{.Size}}"],
                 capture_output=True,
                 verbose=verbose,
@@ -112,19 +112,19 @@ def run_pull_image(
                 check=False,
             )
             if not dry_run:
-                if process and process.returncode == 0:
-                    image_size = int(process.stdout.strip())
+                if command_result.returncode == 0:
+                    image_size = int(command_result.stdout.strip())
                     if image_size == 0:
                         console.print("\n[red]The image size was 0 - image creation failed.[/]\n")
                         return 1, f"Image Python {image_params.python}"
                 else:
                     console.print("\n[red]There was an error pulling the size of the image. Failing.[/]\n")
                     return (
-                        get_return_code(process=process, dry_run=dry_run),
+                        command_result.returncode,
                         f"Image Python {image_params.python}",
                     )
             if tag_as_latest:
-                process = run_command(
+                command_result = run_command(
                     [
                         "docker",
                         "tag",
@@ -136,7 +136,7 @@ def run_pull_image(
                     dry_run=dry_run,
                     check=False,
                 )
-            return get_return_code(process=process, dry_run=dry_run), f"Image Python {image_params.python}"
+            return command_result.returncode, f"Image Python {image_params.python}"
         if wait_for_image:
             if verbose or dry_run:
                 console.print(f"\n[bright_blue]Waiting for {poll_time} seconds.[/]\n")
@@ -144,7 +144,7 @@ def run_pull_image(
             continue
         else:
             console.print(f"\n[red]There was an error pulling the image {image_params.python}. Failing.[/]\n")
-            return get_return_code(process=process, dry_run=dry_run), f"Image Python {image_params.python}"
+            return command_result.returncode, f"Image Python {image_params.python}"
 
 
 def run_pull_and_verify_image(
diff --git a/dev/breeze/src/airflow_breeze/utils/registry.py b/dev/breeze/src/airflow_breeze/utils/registry.py
index 8e1a980ed5..4eac7be01a 100644
--- a/dev/breeze/src/airflow_breeze/utils/registry.py
+++ b/dev/breeze/src/airflow_breeze/utils/registry.py
@@ -21,7 +21,7 @@ from typing import Tuple, Union
 from airflow_breeze.build_image.ci.build_ci_params import BuildCiParams
 from airflow_breeze.build_image.prod.build_prod_params import BuildProdParams
 from airflow_breeze.utils.console import console
-from airflow_breeze.utils.run_utils import get_return_code, run_command
+from airflow_breeze.utils.run_utils import run_command
 
 
 def login_to_docker_registry(
@@ -44,7 +44,7 @@ def login_to_docker_registry(
             )
         elif len(image_params.github_token) > 0:
             run_command(['docker', 'logout', 'ghcr.io'], verbose=True, text=False, check=False)
-            process = run_command(
+            command_result = run_command(
                 [
                     'docker',
                     'login',
@@ -58,7 +58,7 @@ def login_to_docker_registry(
                 input=image_params.github_token,
                 check=False,
             )
-            return get_return_code(process=process, dry_run=dry_run), "Docker login"
+            return command_result.returncode, "Docker login"
         else:
             console.print('\n[bright_blue]Skip Login to GitHub Container Registry as token is missing')
     return 0, "Docker login skipped"
diff --git a/dev/breeze/src/airflow_breeze/utils/run_tests.py b/dev/breeze/src/airflow_breeze/utils/run_tests.py
index b255d3cb2f..b8b0a2276f 100644
--- a/dev/breeze/src/airflow_breeze/utils/run_tests.py
+++ b/dev/breeze/src/airflow_breeze/utils/run_tests.py
@@ -28,12 +28,12 @@ from airflow_breeze.utils.run_utils import run_command
 def verify_an_image(
     image_name: str, image_type: str, dry_run: bool, verbose: bool, extra_pytest_args: Tuple
 ) -> Tuple[int, str]:
-    process = run_command(
+    command_result = run_command(
         ["docker", "inspect", image_name], dry_run=dry_run, verbose=verbose, check=False, stdout=DEVNULL
     )
-    if process and process.returncode != 0:
-        console.print(f"[red]Error when inspecting {image_type} image: {process.returncode}[/]")
-        return process.returncode, f"Testing {image_type} python {image_name}"
+    if command_result.returncode != 0:
+        console.print(f"[red]Error when inspecting {image_type} image: {command_result.returncode}[/]")
+        return command_result.returncode, f"Testing {image_type} python {image_name}"
     pytest_args = ("-n", "auto", "--color=yes")
     if image_type == 'PROD':
         test_path = AIRFLOW_SOURCES_ROOT / "docker_tests" / "test_prod_image.py"
@@ -41,40 +41,34 @@ def verify_an_image(
         test_path = AIRFLOW_SOURCES_ROOT / "docker_tests" / "test_ci_image.py"
     env = os.environ.copy()
     env['DOCKER_IMAGE'] = image_name
-    process = run_command(
+    command_result = run_command(
         [sys.executable, "-m", "pytest", str(test_path), *pytest_args, *extra_pytest_args],
         dry_run=dry_run,
         verbose=verbose,
         env=env,
         check=False,
     )
-    if process:
-        return process.returncode, f"Testing {image_type} python {image_name}"
-    else:
-        return 1, f"Testing {image_type} python {image_name}"
+    return command_result.returncode, f"Testing {image_type} python {image_name}"
 
 
 def run_docker_compose_tests(
     image_name: str, dry_run: bool, verbose: bool, extra_pytest_args: Tuple
 ) -> Tuple[int, str]:
-    process = run_command(
+    command_result = run_command(
         ["docker", "inspect", image_name], dry_run=dry_run, verbose=verbose, check=False, stdout=DEVNULL
     )
-    if process and process.returncode != 0:
-        console.print(f"[red]Error when inspecting PROD image: {process.returncode}[/]")
-        return process.returncode, f"Testing docker-compose python with {image_name}"
+    if command_result.returncode != 0:
+        console.print(f"[red]Error when inspecting PROD image: {command_result.returncode}[/]")
+        return command_result.returncode, f"Testing docker-compose python with {image_name}"
     pytest_args = ("-n", "auto", "--color=yes")
     test_path = AIRFLOW_SOURCES_ROOT / "docker_tests" / "test_docker_compose_quick_start.py"
     env = os.environ.copy()
     env['DOCKER_IMAGE'] = image_name
-    process = run_command(
+    command_result = run_command(
         [sys.executable, "-m", "pytest", str(test_path), *pytest_args, *extra_pytest_args],
         dry_run=dry_run,
         verbose=verbose,
         env=env,
         check=False,
     )
-    if process:
-        return process.returncode, f"Testing docker-compose python with {image_name}"
-    else:
-        return 1, f"Testing docker-compose python with {image_name}"
+    return command_result.returncode, f"Testing docker-compose python with {image_name}"
diff --git a/dev/breeze/src/airflow_breeze/utils/run_utils.py b/dev/breeze/src/airflow_breeze/utils/run_utils.py
index 76d1996308..d0310cf5a2 100644
--- a/dev/breeze/src/airflow_breeze/utils/run_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/run_utils.py
@@ -25,7 +25,7 @@ import sys
 from distutils.version import StrictVersion
 from functools import lru_cache
 from pathlib import Path
-from typing import Dict, List, Mapping, Optional
+from typing import List, Mapping, Optional, Union
 
 from airflow_breeze.utils.console import console
 from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT
@@ -42,7 +42,7 @@ def run_command(
     cwd: Optional[Path] = None,
     input: Optional[str] = None,
     **kwargs,
-) -> Optional[subprocess.CompletedProcess]:
+) -> Union[subprocess.CompletedProcess, subprocess.CalledProcessError]:
     """
     Runs command passed as list of strings with some extra functionality over POpen (kwargs from PoPen can
     be used in this command even if not explicitly specified).
@@ -75,7 +75,7 @@ def run_command(
         # Soft wrap allows to copy&paste and run resulting output as it has no hard EOL
         console.print(f"\n[bright_blue]{env_to_print}{command_to_print}[/]\n", soft_wrap=True)
         if dry_run:
-            return None
+            return subprocess.CompletedProcess(cmd, returncode=0)
     try:
         cmd_env = os.environ.copy()
         if env:
@@ -91,9 +91,9 @@ def run_command(
                 console.print("[red]========================= STDERR start ============================[/]")
                 console.print(ex.stderr)
                 console.print("[red]========================= STDERR end ==============================[/]")
-        if not check:
+        if check:
             raise
-    return None
+        return ex
 
 
 def check_pre_commit_installed(verbose: bool) -> bool:
@@ -111,11 +111,11 @@ def check_pre_commit_installed(verbose: bool) -> bool:
     pre_commit_name = "pre-commit"
     is_installed = False
     if shutil.which(pre_commit_name) is not None:
-        process = run_command(
-            [pre_commit_name, "--version"], verbose=verbose, check=True, capture_output=True, text=True
+        command_result = run_command(
+            [pre_commit_name, "--version"], verbose=verbose, capture_output=True, text=True
         )
-        if process and process.stdout:
-            pre_commit_version = process.stdout.split(" ")[-1].strip()
+        if command_result.stdout:
+            pre_commit_version = command_result.stdout.split(" ")[-1].strip()
             if StrictVersion(pre_commit_version) >= StrictVersion(min_pre_commit_version):
                 console.print(
                     f"\n[green]Package {pre_commit_name} is installed. "
@@ -228,8 +228,8 @@ def is_repo_rebased(repo: str, branch: str):
     headers_dict = {"Accept": "application/vnd.github.VERSION.sha"}
     latest_sha = requests.get(gh_url, headers=headers_dict).text.strip()
     rebased = False
-    process = run_command(['git', 'log', '--format=format:%H'], capture_output=True, text=True)
-    output = process.stdout.strip().splitlines() if process is not None else "missing"
+    command_result = run_command(['git', 'log', '--format=format:%H'], capture_output=True, text=True)
+    output = command_result.stdout.strip().splitlines() if command_result is not None else "missing"
     if latest_sha in output:
         rebased = True
     return rebased
@@ -243,7 +243,7 @@ def check_if_buildx_plugin_installed(verbose: bool) -> bool:
     """
     is_buildx_available = False
     check_buildx = ['docker', 'buildx', 'version']
-    docker_buildx_version_process = run_command(
+    docker_buildx_version_result = run_command(
         check_buildx,
         verbose=verbose,
         no_output_dump_on_exception=True,
@@ -251,9 +251,9 @@ def check_if_buildx_plugin_installed(verbose: bool) -> bool:
         text=True,
     )
     if (
-        docker_buildx_version_process
-        and docker_buildx_version_process.returncode == 0
-        and docker_buildx_version_process.stdout != ''
+        docker_buildx_version_result
+        and docker_buildx_version_result.returncode == 0
+        and docker_buildx_version_result.stdout != ''
     ):
         is_buildx_available = True
     return is_buildx_available
@@ -278,8 +278,8 @@ def prepare_build_command(prepare_buildx_cache: bool, verbose: bool) -> List[str
         if prepare_buildx_cache:
             build_command_param.extend(["buildx", "build", "--builder", "airflow_cache", "--progress=tty"])
             cmd = ['docker', 'buildx', 'inspect', 'airflow_cache']
-            process = run_command(cmd, verbose=True, text=True)
-            if process and process.returncode != 0:
+            buildx_command_result = run_command(cmd, verbose=True, text=True)
+            if buildx_command_result and buildx_command_result.returncode != 0:
                 next_cmd = ['docker', 'buildx', 'create', '--name', 'airflow_cache']
                 run_command(next_cmd, verbose=True, text=True, check=False)
         else:
@@ -300,22 +300,16 @@ def prepare_build_command(prepare_buildx_cache: bool, verbose: bool) -> List[str
 @lru_cache(maxsize=None)
 def commit_sha():
     """Returns commit SHA of current repo. Cached for various usages."""
-    return run_command(
-        ['git', 'rev-parse', 'HEAD'], capture_output=True, text=True, check=False
-    ).stdout.strip()
+    command_result = run_command(['git', 'rev-parse', 'HEAD'], capture_output=True, text=True, check=False)
+    if command_result.stdout:
+        return command_result.stdout.strip()
+    else:
+        return "COMMIT_SHA_NOT_FOUND"
 
 
-def filter_out_none(**kwargs) -> Dict[str, str]:
+def filter_out_none(**kwargs) -> dict:
     """Filters out all None values from parameters passed."""
     for key in list(kwargs):
         if kwargs[key] is None:
             kwargs.pop(key)
     return kwargs
-
-
-def get_return_code(process: Optional[subprocess.CompletedProcess], dry_run: bool):
-    if process:
-        return process.returncode
-    if dry_run:
-        return 0
-    return 1
diff --git a/dev/breeze/src/airflow_ci/freespace.py b/dev/breeze/src/airflow_ci/freespace.py
index cb24ac3a24..5c235a35be 100755
--- a/dev/breeze/src/airflow_ci/freespace.py
+++ b/dev/breeze/src/airflow_ci/freespace.py
@@ -52,7 +52,7 @@ def main(verbose, dry_run):
         ["docker", "system", "prune", "--all", "--force", "--volumes"], verbose=verbose, dry_run=dry_run
     )
     run_command(["df", "-h"], verbose=verbose, dry_run=dry_run)
-    run_command(["docker", "logout", "ghcr.io"], verbose=verbose, dry_run=dry_run)
+    run_command(["docker", "logout", "ghcr.io"], verbose=verbose, dry_run=dry_run, check=False)
 
 
 if __name__ == '__main__':
diff --git a/dev/breeze/tests/test_docker_command_utils.py b/dev/breeze/tests/test_docker_command_utils.py
index 598e760fd5..8d168844b9 100644
--- a/dev/breeze/tests/test_docker_command_utils.py
+++ b/dev/breeze/tests/test_docker_command_utils.py
@@ -21,92 +21,93 @@ from unittest.mock import call
 from airflow_breeze.utils.docker_command_utils import check_docker_compose_version, check_docker_version
 
 
+@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission_denied')
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
 @mock.patch('airflow_breeze.utils.docker_command_utils.console')
-def test_check_docker_version_unknown(mock_console, mock_run_command):
+def test_check_docker_version_unknown(mock_console, mock_run_command, mock_check_docker_permission_denied):
+    mock_check_docker_permission_denied.return_value = False
     check_docker_version(verbose=True)
     expected_run_command_calls = [
-        call(
-            ['docker', 'info'],
-            verbose=True,
-            check=True,
-            no_output_dump_on_exception=True,
-            capture_output=True,
-            text=True,
-        ),
         call(
             ['docker', 'version', '--format', '{{.Client.Version}}'],
             verbose=True,
             no_output_dump_on_exception=True,
             capture_output=True,
             text=True,
+            check=False,
         ),
     ]
-
     mock_run_command.assert_has_calls(expected_run_command_calls)
     mock_console.print.assert_called_with(
-        "Your version of docker is unknown. If the scripts fail, please make sure to"
-        "                     install docker at least: 20.10.0 version."
+        """
+[yellow]Your version of docker is unknown. If the scripts fail, please make sure to[/]
+[yellow]install docker at least: 20.10.0 version.[/]
+"""
     )
 
 
-@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission')
+@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission_denied')
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
 @mock.patch('airflow_breeze.utils.docker_command_utils.console')
-def test_check_docker_version_too_low(mock_console, mock_run_command, mock_check_docker_permission):
-    mock_check_docker_permission.return_value = False
+def test_check_docker_version_too_low(mock_console, mock_run_command, mock_check_docker_permission_denied):
+    mock_check_docker_permission_denied.return_value = False
     mock_run_command.return_value.returncode = 0
     mock_run_command.return_value.stdout = "0.9"
     check_docker_version(verbose=True)
-    mock_check_docker_permission.assert_called_with(True)
+    mock_check_docker_permission_denied.assert_called_with(True)
     mock_run_command.assert_called_with(
         ['docker', 'version', '--format', '{{.Client.Version}}'],
         verbose=True,
         no_output_dump_on_exception=True,
         capture_output=True,
         text=True,
+        check=False,
     )
     mock_console.print.assert_called_with(
-        "Your version of docker is too old:0.9. Please upgrade to                     at least 20.10.0"
+        """
+[yellow]Your version of docker is too old:0.9. Please upgrade to at least 20.10.0[/]
+"""
     )
 
 
-@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission')
+@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission_denied')
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
 @mock.patch('airflow_breeze.utils.docker_command_utils.console')
-def test_check_docker_version_ok(mock_console, mock_run_command, mock_check_docker_permission):
-    mock_check_docker_permission.return_value = False
+def test_check_docker_version_ok(mock_console, mock_run_command, mock_check_docker_permission_denied):
+    mock_check_docker_permission_denied.return_value = False
     mock_run_command.return_value.returncode = 0
     mock_run_command.return_value.stdout = "20.10.0"
     check_docker_version(verbose=True)
-    mock_check_docker_permission.assert_called_with(True)
+    mock_check_docker_permission_denied.assert_called_with(True)
     mock_run_command.assert_called_with(
         ['docker', 'version', '--format', '{{.Client.Version}}'],
         verbose=True,
         no_output_dump_on_exception=True,
         capture_output=True,
         text=True,
+        check=False,
     )
-    mock_console.print.assert_called_with("Good version of Docker: 20.10.0.")
+    mock_console.print.assert_called_with("[green]Good version of Docker: 20.10.0.[/]")
 
 
-@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission')
+@mock.patch('airflow_breeze.utils.docker_command_utils.check_docker_permission_denied')
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
 @mock.patch('airflow_breeze.utils.docker_command_utils.console')
-def test_check_docker_version_higher(mock_console, mock_run_command, mock_check_docker_permission):
-    mock_check_docker_permission.return_value = False
+def test_check_docker_version_higher(mock_console, mock_run_command, mock_check_docker_permission_denied):
+    mock_check_docker_permission_denied.return_value = False
     mock_run_command.return_value.returncode = 0
     mock_run_command.return_value.stdout = "21.10.0"
     check_docker_version(verbose=True)
-    mock_check_docker_permission.assert_called_with(True)
+    mock_check_docker_permission_denied.assert_called_with(True)
     mock_run_command.assert_called_with(
         ['docker', 'version', '--format', '{{.Client.Version}}'],
         verbose=True,
         no_output_dump_on_exception=True,
         capture_output=True,
         text=True,
+        check=False,
     )
-    mock_console.print.assert_called_with("Good version of Docker: 21.10.0.")
+    mock_console.print.assert_called_with("[green]Good version of Docker: 21.10.0.[/]")
 
 
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
@@ -124,8 +125,10 @@ def test_check_docker_compose_version_unknown(mock_console, mock_run_command):
     ]
     mock_run_command.assert_has_calls(expected_run_command_calls)
     mock_console.print.assert_called_with(
-        'Unknown docker-compose version. At least 1.29 is needed! \
-        If Breeze fails upgrade to latest available docker-compose version'
+        """
+[yellow]Unknown docker-compose version. At least 1.29 is needed![/]
+[yellow]If Breeze fails upgrade to latest available docker-compose version.[/]
+"""
     )
 
 
@@ -144,12 +147,15 @@ def test_check_docker_compose_version_low(mock_console, mock_run_command):
     )
     expected_print_calls = [
         call(
-            'You have too old version of docker-compose: 1.28.5! \
-                At least 1.29 is needed! Please upgrade!'
+            """
+[yellow]You have too old version of docker-compose: 1.28.5! At least 1.29 is needed! Please upgrade!
+"""
         ),
         call(
-            'See https://docs.docker.com/compose/install/ for instructions. \
-                Make sure docker-compose you install is first on the PATH variable of yours.'
+            """
+See https://docs.docker.com/compose/install/ for instructions.
+Make sure docker-compose you install is first on the PATH variable of yours.
+"""
         ),
     ]
     mock_console.print.assert_has_calls(expected_print_calls)
@@ -168,7 +174,7 @@ def test_check_docker_compose_version_ok(mock_console, mock_run_command):
         capture_output=True,
         text=True,
     )
-    mock_console.print.assert_called_with("Good version of docker-compose: 1.29.0")
+    mock_console.print.assert_called_with("[green]Good version of docker-compose: 1.29.0[/]")
 
 
 @mock.patch('airflow_breeze.utils.docker_command_utils.run_command')
@@ -184,4 +190,4 @@ def test_check_docker_compose_version_higher(mock_console, mock_run_command):
         capture_output=True,
         text=True,
     )
-    mock_console.print.assert_called_with("Good version of docker-compose: 1.29.2")
+    mock_console.print.assert_called_with("[green]Good version of docker-compose: 1.29.2[/]")
diff --git a/docs/docker-stack/build-arg-ref.rst b/docs/docker-stack/build-arg-ref.rst
index ea4c8a7d6c..a462179588 100644
--- a/docs/docker-stack/build-arg-ref.rst
+++ b/docs/docker-stack/build-arg-ref.rst
@@ -70,6 +70,7 @@ Those are the most common arguments that you use when you want to build a custom
 List of default extras in the production Dockerfile:
 
 .. BEGINNING OF EXTRAS LIST UPDATED BY PRE COMMIT
+
 * amazon
 * async
 * celery
@@ -96,6 +97,7 @@ List of default extras in the production Dockerfile:
 * ssh
 * statsd
 * virtualenv
+
 .. END OF EXTRAS LIST UPDATED BY PRE COMMIT
 
 Image optimization options
diff --git a/scripts/ci/docker-compose/_docker.env b/scripts/ci/docker-compose/_docker.env
index 56e371469a..325b72aee2 100644
--- a/scripts/ci/docker-compose/_docker.env
+++ b/scripts/ci/docker-compose/_docker.env
@@ -38,7 +38,6 @@ GITHUB_REGISTRY_PULL_IMAGE_TAG
 HOST_USER_ID
 HOST_GROUP_ID
 HOST_OS
-HOST_HOME
 INIT_SCRIPT_FILE
 INSTALL_AIRFLOW_VERSION
 GENERATE_CONSTRAINTS_MODE
diff --git a/scripts/ci/docker-compose/base.yml b/scripts/ci/docker-compose/base.yml
index 124ed9f6dc..04c173c865 100644
--- a/scripts/ci/docker-compose/base.yml
+++ b/scripts/ci/docker-compose/base.yml
@@ -25,7 +25,6 @@ services:
       - ADDITIONAL_PATH=~/.local/bin
       - CELERY_BROKER_URLS=amqp://guest:guest@rabbitmq:5672,redis://redis:6379/0
       - KUBECONFIG=/files/.kube/config
-      - HOST_HOME=${HOME}
       # We need all those env variables here because docker-compose-v2 does not really work well
       # With env files and there are many problems with it:
       - AIRFLOW_CI_IMAGE=${AIRFLOW_CI_IMAGE}
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index 5767779942..c36d3a50a8 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -342,9 +342,6 @@ function initialization::initialize_host_variables() {
     HOST_OS="$(uname -s)"
     export HOST_OS
 
-    # Home directory of the host user
-    export HOST_HOME="${HOME}"
-
     # In case of MacOS we need to use gstat - gnu version of the stats
     export STAT_BIN=stat
     if [[ "${OSTYPE}" == "darwin"* ]]; then
@@ -699,7 +696,6 @@ Host variables:
     HOST_USER_ID=${HOST_USER_ID}
     HOST_GROUP_ID=${HOST_GROUP_ID}
     HOST_OS=${HOST_OS}
-    HOST_HOME=${HOST_HOME}
 
 Git variables:
 
@@ -822,7 +818,6 @@ function initialization::make_constants_read_only() {
 
     readonly HOST_USER_ID
     readonly HOST_GROUP_ID
-    readonly HOST_HOME
     readonly HOST_OS
 
     readonly KUBERNETES_MODE
diff --git a/scripts/ci/pre_commit/pre_commit_check_order_dockerfile_extras.py b/scripts/ci/pre_commit/pre_commit_check_order_dockerfile_extras.py
index 2bdc39031c..2658b09ced 100755
--- a/scripts/ci/pre_commit/pre_commit_check_order_dockerfile_extras.py
+++ b/scripts/ci/pre_commit/pre_commit_check_order_dockerfile_extras.py
@@ -69,17 +69,27 @@ def _check_list_sorted(the_list: List[str], message: str) -> bool:
 
 
 def get_replaced_content(
-    content: List[str], extras_list: List[str], start_line: str, end_line: str, prefix: str, suffix: str
+    content: List[str],
+    extras_list: List[str],
+    start_line: str,
+    end_line: str,
+    prefix: str,
+    suffix: str,
+    add_empty_lines: bool,
 ) -> List[str]:
     result = []
     is_copying = True
     for line in content:
         if line.startswith(start_line):
             result.append(f"{line}")
+            if add_empty_lines:
+                result.append("\n")
             is_copying = False
             for extra in extras_list:
                 result.append(f'{prefix}{extra}{suffix}\n')
         elif line.startswith(end_line):
+            if add_empty_lines:
+                result.append("\n")
             result.append(f"{line}")
             is_copying = True
         elif is_copying:
@@ -96,12 +106,24 @@ def check_dockerfile():
             if _check_list_sorted(extras_list, "Dockerfile's AIRFLOW_EXTRAS"):
                 builds_args_content = BUILD_ARGS_REF_PATH.read_text().splitlines(keepends=True)
                 result = get_replaced_content(
-                    builds_args_content, extras_list, START_RST_LINE, END_RST_LINE, "* ", ""
+                    builds_args_content,
+                    extras_list,
+                    START_RST_LINE,
+                    END_RST_LINE,
+                    "* ",
+                    "",
+                    add_empty_lines=True,
                 )
                 BUILD_ARGS_REF_PATH.write_text("".join(result))
                 global_constants_path = GLOBAL_CONSTANTS_PATH.read_text().splitlines(keepends=True)
                 result = get_replaced_content(
-                    global_constants_path, extras_list, START_PYTHON_LINE, END_PYTHON_LINE, '    "', '",'
+                    global_constants_path,
+                    extras_list,
+                    START_PYTHON_LINE,
+                    END_PYTHON_LINE,
+                    '    "',
+                    '",',
+                    add_empty_lines=False,
                 )
                 GLOBAL_CONSTANTS_PATH.write_text("".join(result))
                 return
diff --git a/scripts/in_container/run_docs_build.sh b/scripts/in_container/run_docs_build.sh
index 0331557dc1..b96be02304 100755
--- a/scripts/in_container/run_docs_build.sh
+++ b/scripts/in_container/run_docs_build.sh
@@ -20,6 +20,7 @@
 
 sudo -E "${AIRFLOW_SOURCES}/docs/build_docs.py" "${@}"
 
+
 if [[ ( ${CI:="false"} == "true" || ${CI} == "True" ) && -d "${AIRFLOW_SOURCES}/docs/_build/docs/" ]]; then
     rm -rf "/files/documentation"
     cp -r "${AIRFLOW_SOURCES}/docs/_build" "/files/documentation"