Skip to content

Commit

Permalink
Speed up Breeze experience on Mac OS (#23866)
Browse files Browse the repository at this point in the history
This change should significantly speed up Breeze experience (and
especially iterating over a change in Breeze for MacOS users -
independently if you are using x86 or arm architecture.

The problem with MacOS with docker is particularly slow filesystem
used to map sources from Host to Docker VM. It is particularly bad
when there are multiple small files involved.

The improvement come from two areas:
* removing duplicate pycache cleaning
* moving MyPy cache to docker volume

When entering breeze we are - just in case - cleaning .pyc and
__pychache__ files potentially generated outside of the docker
container - this is particularly useful if you use local IDE
and you do not have bytecode generation disabled (we have it
disabled in Breeze). Generating python bytecode might lead to
various problems when you are switching branches and Python
versions, so for Breeze development where the files change
often anyway, disabling them and removing when they are found
is important. This happens at entering breeze and it might take
a second or two depending if you have locally generated.

It could happen that __init script was called twice (depending which
script was called - therefore the time could be double the one
that was actually needed. Also if you ever generated provider
packages, the time could be much longer, because node_modules
generated in provider sources were not excluded from searching
(and on MacOS it takes a LOT of time).

This also led to duplicate time of exit as the initialization code
installed traps that were also run twice. The traps however were
rather fast so had no negative influence on performance.

The change adds a guard so that initialization is only ever executed
once.

Second part of the change is moving the cache of mypy to a docker
volume rather than being used from local source folder (default
when complete sources are mounted). We were already using selective
mount to make sure MacOS filesystem slowness affects us in minimal
way - but with this change, the cache will be stored in docker
volume that does not suffer from the same problems as mounting
volumes from host. The Docker volume is preserved until the
`docker stop` command is run - which means that iterating over
a change should be WAY faster now - observed speed-up were around
5x speedups for MyPy pre-commit.

(cherry picked from commit ac9b224)
  • Loading branch information
potiuk authored and ephraimbuddy committed May 27, 2022
1 parent c16feda commit e0c6fdd
Show file tree
Hide file tree
Showing 16 changed files with 86 additions and 35 deletions.
8 changes: 8 additions & 0 deletions BREEZE.rst
Original file line number Diff line number Diff line change
Expand Up @@ -258,6 +258,8 @@ If you have several checked out Airflow sources, Breeze will warn you if you are
source tree and will offer you to re-install from those sources - to make sure that you are using the right
version.

You can skip Breeze's upgrade check by setting ``SKIP_BREEZE_UPGRADE_CHECK`` variable to non empty value.

By default Breeze works on the version of Airflow that you run it in - in case you are outside of the
sources of Airflow and you installed Breeze from a directory - Breeze will be run on Airflow sources from
where it was installed.
Expand Down Expand Up @@ -1051,6 +1053,12 @@ command but it is very similar to current ``breeze`` command):
</a>
</div>

.. note::

When you run static checks, some of the artifacts (mypy_cache) is stored in docker-compose volume
so that it can speed up static checks execution significantly. However, sometimes, the cache might
get broken, in which case you should run ``breeze stop`` to clean up the cache.


Building the Documentation
--------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -523,7 +523,7 @@ def stop(verbose: bool, dry_run: bool, preserve_volumes: bool):
command_to_execute = ['docker-compose', 'down', "--remove-orphans"]
if not preserve_volumes:
command_to_execute.append("--volumes")
shell_params = ShellParams(verbose=verbose)
shell_params = ShellParams(verbose=verbose, backend="all")
env_variables = get_env_variables_for_docker_commands(shell_params)
run_command(command_to_execute, verbose=verbose, dry_run=dry_run, env=env_variables)

Expand Down
28 changes: 16 additions & 12 deletions dev/breeze/src/airflow_breeze/params/shell_params.py
Original file line number Diff line number Diff line change
Expand Up @@ -165,19 +165,26 @@ def print_badge_info(self):
get_console().print(f'[info]Backend: {self.backend} {self.backend_version}[/]')
get_console().print(f'[info]Airflow used at runtime: {self.use_airflow_version}[/]')

def get_backend_compose_files(self, backend: str):
if backend == "mssql":
backend_docker_compose_file = (
f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{backend}-{self.debian_version}.yml"
)
else:
backend_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{backend}.yml"
backend_port_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{backend}-port.yml"
return backend_docker_compose_file, backend_port_docker_compose_file

@property
def compose_files(self):
compose_ci_file = []
main_ci_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/base.yml"
if self.backend == "mssql":
backend_docker_compose_file = (
f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{self.backend}-{self.debian_version}.yml"
)
if self.backend != "all":
backend_files = self.get_backend_compose_files(self.backend)
else:
backend_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{self.backend}.yml"
backend_port_docker_compose_file = (
f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-{self.backend}-port.yml"
)
backend_files = []
for backend in ALLOWED_BACKENDS:
backend_files.extend(self.get_backend_compose_files(backend))
local_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/local.yml"
local_all_sources_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/local-all-sources.yml"
files_docker_compose_file = f"{str(SCRIPTS_CI_DIR)}/docker-compose/files.yml"
Expand All @@ -194,17 +201,14 @@ def compose_files(self):
compose_ci_file.append(
f"{str(SCRIPTS_CI_DIR)}/docker-compose/backend-mssql-docker-volume.yml"
)
compose_ci_file.extend(
[main_ci_docker_compose_file, backend_docker_compose_file, files_docker_compose_file]
)
compose_ci_file.extend([main_ci_docker_compose_file, *backend_files, files_docker_compose_file])

if self.mount_sources == MOUNT_SELECTED:
compose_ci_file.extend([local_docker_compose_file])
elif self.mount_sources == MOUNT_ALL:
compose_ci_file.extend([local_all_sources_docker_compose_file])
else: # none
compose_ci_file.extend([remove_sources_docker_compose_file])
compose_ci_file.extend([backend_port_docker_compose_file])
if self.forward_credentials:
compose_ci_file.append(forward_credentials_docker_compose_file)
if self.use_airflow_version is not None:
Expand Down
12 changes: 12 additions & 0 deletions dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
"""Various utils to prepare docker and docker compose commands."""
import os
import re
import subprocess
import sys
from copy import deepcopy
from random import randint
Expand Down Expand Up @@ -97,6 +98,16 @@
]


def create_volume_if_missing(volume_name: str):
res_inspect = run_command(cmd=["docker", "inspect", volume_name], stdout=subprocess.DEVNULL, check=False)
if res_inspect.returncode != 0:
run_command(cmd=["docker", "volume", "create", volume_name], check=True)


def create_static_check_volumes():
create_volume_if_missing("docker-compose_mypy-cache-volume")


def get_extra_docker_flags(mount_sources: str) -> List[str]:
"""
Returns extra docker flags based on the type of mounting we want to do for sources.
Expand All @@ -110,6 +121,7 @@ def get_extra_docker_flags(mount_sources: str) -> List[str]:
elif mount_sources == MOUNT_SELECTED:
for flag in NECESSARY_HOST_VOLUMES:
extra_docker_flags.extend(["-v", str(AIRFLOW_SOURCES_ROOT) + flag])
extra_docker_flags.extend(['-v', "docker-compose_mypy-cache-volume:/opt/airflow/.mypy_cache/"])
else: # none
extra_docker_flags.extend(["-v", f"{AIRFLOW_SOURCES_ROOT / 'empty'}:/opt/airflow/airflow"])
extra_docker_flags.extend(["-v", f"{AIRFLOW_SOURCES_ROOT}/files:/files"])
Expand Down
10 changes: 7 additions & 3 deletions dev/breeze/src/airflow_breeze/utils/path_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,13 @@ def in_help() -> bool:


def skip_upgrade_check():
return in_self_upgrade() or in_autocomplete() or in_help() or hasattr(sys, '_called_from_test')
return (
in_self_upgrade()
or in_autocomplete()
or in_help()
or hasattr(sys, '_called_from_test')
or os.environ.get('SKIP_BREEZE_UPGRADE_CHECK')
)


def get_package_setup_metadata_hash() -> str:
Expand Down Expand Up @@ -235,7 +241,6 @@ def find_airflow_sources_root_to_operate_on() -> Path:
BUILD_CACHE_DIR = AIRFLOW_SOURCES_ROOT / '.build'
FILES_DIR = AIRFLOW_SOURCES_ROOT / 'files'
MSSQL_DATA_VOLUME = AIRFLOW_SOURCES_ROOT / 'tmp_mssql_volume'
MYPY_CACHE_DIR = AIRFLOW_SOURCES_ROOT / '.mypy_cache'
LOGS_DIR = AIRFLOW_SOURCES_ROOT / 'logs'
DIST_DIR = AIRFLOW_SOURCES_ROOT / 'dist'
SCRIPTS_CI_DIR = AIRFLOW_SOURCES_ROOT / 'scripts' / 'ci'
Expand All @@ -253,7 +258,6 @@ def create_directories() -> None:
BUILD_CACHE_DIR.mkdir(parents=True, exist_ok=True)
FILES_DIR.mkdir(parents=True, exist_ok=True)
MSSQL_DATA_VOLUME.mkdir(parents=True, exist_ok=True)
MYPY_CACHE_DIR.mkdir(parents=True, exist_ok=True)
LOGS_DIR.mkdir(parents=True, exist_ok=True)
DIST_DIR.mkdir(parents=True, exist_ok=True)
OUTPUT_LOG.mkdir(parents=True, exist_ok=True)
2 changes: 2 additions & 0 deletions scripts/ci/docker-compose/backend-mssql-docker-volume.yml
Original file line number Diff line number Diff line change
Expand Up @@ -20,3 +20,5 @@ services:
mssql:
volumes:
- mssql-db-volume:/var/opt/mssql
volumes:
mssql-db-volume:
2 changes: 2 additions & 0 deletions scripts/ci/docker-compose/backend-mysql.yml
Original file line number Diff line number Diff line change
Expand Up @@ -44,3 +44,5 @@ services:
restart: always
command: ['mysqld', '--character-set-server=utf8mb4',
'--collation-server=utf8mb4_unicode_ci']
volumes:
mysql-db-volume:
2 changes: 2 additions & 0 deletions scripts/ci/docker-compose/backend-postgres.yml
Original file line number Diff line number Diff line change
Expand Up @@ -42,3 +42,5 @@ services:
timeout: 10s
retries: 5
restart: always
volumes:
postgres-db-volume:
2 changes: 2 additions & 0 deletions scripts/ci/docker-compose/backend-sqlite.yml
Original file line number Diff line number Diff line change
Expand Up @@ -25,3 +25,5 @@ services:
volumes:
- /dev/urandom:/dev/random # Required to get non-blocking entropy source
- sqlite-db-volume:/root/airflow
volumes:
sqlite-db-volume:
5 changes: 0 additions & 5 deletions scripts/ci/docker-compose/base.yml
Original file line number Diff line number Diff line change
Expand Up @@ -90,8 +90,3 @@ services:
- "${FLOWER_HOST_PORT}:5555"
cap_add:
- SYS_PTRACE
volumes:
sqlite-db-volume:
postgres-db-volume:
mysql-db-volume:
mssql-db-volume:
3 changes: 3 additions & 0 deletions scripts/ci/docker-compose/local.yml
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ services:
# or those that might be useful to see in the host as output of the
# tests (such as logs)
volumes:
- mypy-cache-volume:/opt/airflow/.mypy_cache/
# START automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh
- ../../../.bash_aliases:/root/.bash_aliases:cached
- ../../../.bash_history:/root/.bash_history:cached
Expand Down Expand Up @@ -58,3 +59,5 @@ services:
- ../../../chart:/opt/airflow/chart:cached
- ../../../metastore_browser:/opt/airflow/metastore_browser:cached
# END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh
volumes:
mypy-cache-volume:
1 change: 0 additions & 1 deletion scripts/ci/libraries/_initialization.sh
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@ function initialization::create_directories() {
export CI="${CI="false"}"

# Create useful directories if not yet created
mkdir -p "${AIRFLOW_SOURCES}/.mypy_cache"
mkdir -p "${AIRFLOW_SOURCES}/logs"
mkdir -p "${AIRFLOW_SOURCES}/dist"

Expand Down
7 changes: 5 additions & 2 deletions scripts/ci/pre_commit/pre_commit_flake8.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,22 +32,25 @@
GITHUB_REPOSITORY = os.environ.get('GITHUB_REPOSITORY', "apache/airflow")

if __name__ == '__main__':
os.environ['SKIP_BREEZE_UPGRADE_CHECK'] = "true"
sys.path.insert(0, str(Path(__file__).parents[3].resolve() / "dev" / "breeze" / "src"))
from airflow_breeze.branch_defaults import AIRFLOW_BRANCH
from airflow_breeze.global_constants import MOUNT_SELECTED
from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags

AIRFLOW_CI_IMAGE = f"ghcr.io/{GITHUB_REPOSITORY}/{AIRFLOW_BRANCH}/ci/python3.7"

if subprocess.call(args=["docker", "inspect", AIRFLOW_CI_IMAGE], stdout=subprocess.DEVNULL) != 0:
print(f'[red]The image {AIRFLOW_CI_IMAGE} is not available.[/]\n')
print("\n[yellow]Please run at the earliest convenience:[/]\n\nbreeze build-image --python 3.7\n\n")
sys.exit(1)
create_static_check_volumes()
return_code = subprocess.call(
args=[
"docker",
"run",
"-t",
"-v",
f"{AIRFLOW_SOURCES}:/opt/airflow/",
*get_extra_docker_flags(MOUNT_SELECTED),
"-e",
"SKIP_ENVIRONMENT_INITIALIZATION=true",
"-e",
Expand Down
7 changes: 5 additions & 2 deletions scripts/ci/pre_commit/pre_commit_mypy.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,22 +33,25 @@
GITHUB_REPOSITORY = os.environ.get('GITHUB_REPOSITORY', "apache/airflow")

if __name__ == '__main__':
os.environ['SKIP_BREEZE_UPGRADE_CHECK'] = "true"
sys.path.insert(0, str(Path(__file__).parents[3].resolve() / "dev" / "breeze" / "src"))
from airflow_breeze.branch_defaults import AIRFLOW_BRANCH
from airflow_breeze.global_constants import MOUNT_SELECTED
from airflow_breeze.utils.docker_command_utils import create_static_check_volumes, get_extra_docker_flags

AIRFLOW_CI_IMAGE = f"ghcr.io/{GITHUB_REPOSITORY}/{AIRFLOW_BRANCH}/ci/python3.7"

if subprocess.call(args=["docker", "inspect", AIRFLOW_CI_IMAGE], stdout=subprocess.DEVNULL) != 0:
print(f'[red]The image {AIRFLOW_CI_IMAGE} is not available.[/]\n')
print("\n[yellow]Please run at the earliest convenience:[/]\n\nbreeze build-image --python 3.7\n\n")
sys.exit(1)
create_static_check_volumes()
return_code = subprocess.call(
args=[
"docker",
"run",
"-t",
"-v",
f"{AIRFLOW_SOURCES}:/opt/airflow/",
*get_extra_docker_flags(MOUNT_SELECTED),
"-e",
"SKIP_ENVIRONMENT_INITIALIZATION=true",
"-e",
Expand Down
18 changes: 9 additions & 9 deletions scripts/in_container/_in_container_script_init.sh
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,13 @@ IN_CONTAINER_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"

# shellcheck source=scripts/in_container/_in_container_utils.sh
. "${IN_CONTAINER_DIR}/_in_container_utils.sh"
if [[ ${IN_CONTAINER_INITIALIZED=} != "true" ]]; then
in_container_set_colors
in_container_basic_sanity_check
in_container_script_start

in_container_set_colors

in_container_basic_sanity_check

in_container_script_start

add_trap "in_container_fix_ownership" EXIT HUP INT TERM
add_trap "in_container_clear_tmp" EXIT HUP INT TERM
add_trap "in_container_script_end" EXIT HUP INT TERM
add_trap "in_container_fix_ownership" EXIT HUP INT TERM
add_trap "in_container_clear_tmp" EXIT HUP INT TERM
add_trap "in_container_script_end" EXIT HUP INT TERM
export IN_CONTAINER_INITIALIZED="true"
fi
12 changes: 12 additions & 0 deletions scripts/in_container/_in_container_utils.sh
Original file line number Diff line number Diff line change
Expand Up @@ -80,29 +80,41 @@ function in_container_script_end() {
#
function in_container_cleanup_pyc() {
set +o pipefail
if [[ ${CLEANED_PYC=} == "true" ]]; then
return
fi
sudo find . \
-path "./airflow/www/node_modules" -prune -o \
-path "./airflow/ui/node_modules" -prune -o \
-path "./provider_packages/airflow/www/node_modules" -prune -o \
-path "./provider_packages/airflow/ui/node_modules" -prune -o \
-path "./.eggs" -prune -o \
-path "./docs/_build" -prune -o \
-path "./build" -prune -o \
-name "*.pyc" | grep ".pyc$" | sudo xargs rm -f
set -o pipefail
export CLEANED_PYC="true"
}

#
# Cleans up __pycache__ directories (in case they come in mounted folders)
#
function in_container_cleanup_pycache() {
set +o pipefail
if [[ ${CLEANED_PYCACHE=} == "true" ]]; then
return
fi
find . \
-path "./airflow/www/node_modules" -prune -o \
-path "./airflow/ui/node_modules" -prune -o \
-path "./provider_packages/airflow/www/node_modules" -prune -o \
-path "./provider_packages/airflow/ui/node_modules" -prune -o \
-path "./.eggs" -prune -o \
-path "./docs/_build" -prune -o \
-path "./build" -prune -o \
-name "__pycache__" | grep "__pycache__" | sudo xargs rm -rf
set -o pipefail
export CLEANED_PYCACHE="true"
}

#
Expand Down

0 comments on commit e0c6fdd

Please sign in to comment.