diff --git a/Dockerfile.ci b/Dockerfile.ci index b569c47e41930..845a1c8f44e0d 100644 --- a/Dockerfile.ci +++ b/Dockerfile.ci @@ -710,7 +710,7 @@ PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION:=3.8} export AIRFLOW_HOME=${AIRFLOW_HOME:=${HOME}} -: "${AIRFLOW_SOURCES:?"ERROR: AIRFLOW_SOURCES not set !!!!"}" +mkdir "${AIRFLOW_HOME}/sqlite" -p || true ASSET_COMPILATION_WAIT_MULTIPLIER=${ASSET_COMPILATION_WAIT_MULTIPLIER:=1} @@ -755,8 +755,10 @@ If it does not complete soon, you might want to stop it and remove file lock: fi } -if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then - +function environment_initialization() { + if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} == "true" ]]; then + return + fi if [[ $(uname -m) == "arm64" || $(uname -m) == "aarch64" ]]; then if [[ ${BACKEND:=} == "mssql" ]]; then echo "${COLOR_RED}ARM platform is not supported for ${BACKEND} backend. Exiting.${COLOR_RESET}" @@ -778,6 +780,7 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then echo export AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR=True fi + if [[ ${DATABASE_ISOLATION=} == "true" ]]; then echo "${COLOR_BLUE}Force database isolation configuration:${COLOR_RESET}" export AIRFLOW__CORE__DATABASE_ACCESS_ISOLATION=True @@ -787,33 +790,10 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then RUN_TESTS=${RUN_TESTS:="false"} CI=${CI:="false"} - USE_AIRFLOW_VERSION="${USE_AIRFLOW_VERSION:=""}" - - if [[ ${USE_AIRFLOW_VERSION} == "" && ${USE_PACKAGES_FROM_DIST=} != "true" ]]; then - export PYTHONPATH=${AIRFLOW_SOURCES} - echo - echo "${COLOR_BLUE}Using airflow version from current sources${COLOR_RESET}" - echo - # Cleanup the logs, tmp when entering the environment - sudo rm -rf "${AIRFLOW_SOURCES}"/logs/* - sudo rm -rf "${AIRFLOW_SOURCES}"/tmp/* - mkdir -p "${AIRFLOW_SOURCES}"/logs/ - mkdir -p "${AIRFLOW_SOURCES}"/tmp/ - else - python "${IN_CONTAINER_DIR}/install_airflow_and_providers.py" - fi - - if [[ "${USE_AIRFLOW_VERSION}" =~ ^2\.2\..*|^2\.1\..*|^2\.0\..* && "${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN=}" != "" ]]; then - # make sure old variable is used for older airflow versions - export AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN}" - fi # Added to have run-tests on path export PATH=${PATH}:${AIRFLOW_SOURCES} - # This is now set in conftest.py - only for pytest tests - unset AIRFLOW__CORE__UNIT_TEST_MODE - mkdir -pv "${AIRFLOW_HOME}/logs/" # Change the default worker_concurrency for tests @@ -869,54 +849,86 @@ if [[ ${SKIP_ENVIRONMENT_INITIALIZATION=} != "true" ]]; then # shellcheck source=scripts/in_container/bin/run_tmux exec run_tmux fi -fi +} + +function determine_airflow_to_use() { + USE_AIRFLOW_VERSION="${USE_AIRFLOW_VERSION:=""}" + if [[ ${USE_AIRFLOW_VERSION} == "" && ${USE_PACKAGES_FROM_DIST=} != "true" ]]; then + export PYTHONPATH=${AIRFLOW_SOURCES} + echo + echo "${COLOR_BLUE}Using airflow version from current sources${COLOR_RESET}" + echo + # Cleanup the logs, tmp when entering the environment + sudo rm -rf "${AIRFLOW_SOURCES}"/logs/* + sudo rm -rf "${AIRFLOW_SOURCES}"/tmp/* + mkdir -p "${AIRFLOW_SOURCES}"/logs/ + mkdir -p "${AIRFLOW_SOURCES}"/tmp/ + else + python "${IN_CONTAINER_DIR}/install_airflow_and_providers.py" + fi -rm -f "${AIRFLOW_SOURCES}/pytest.ini" -rm -f "${AIRFLOW_SOURCES}/.coveragerc" + if [[ "${USE_AIRFLOW_VERSION}" =~ ^2\.2\..*|^2\.1\..*|^2\.0\..* && "${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN=}" != "" ]]; then + # make sure old variable is used for older airflow versions + export AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__DATABASE__SQL_ALCHEMY_CONN}" + fi +} -if [[ ${UPGRADE_BOTO=} == "true" ]]; then +function check_boto_upgrade() { + if [[ ${UPGRADE_BOTO=} != "true" ]]; then + return + fi echo echo "${COLOR_BLUE}Upgrading boto3, botocore to latest version to run Amazon tests with them${COLOR_RESET}" echo pip uninstall --root-user-action ignore aiobotocore s3fs -y || true pip install --root-user-action ignore --upgrade boto3 botocore pip check -fi -if [[ ${DOWNGRADE_SQLALCHEMY=} == "true" ]]; then +} + +function check_download_sqlalchemy() { + if [[ ${DOWNGRADE_SQLALCHEMY=} != "true" ]]; then + return + fi min_sqlalchemy_version=$(grep "sqlalchemy>=" setup.cfg | sed "s/.*>=\([0-9\.]*\).*/\1/") echo echo "${COLOR_BLUE}Downgrading sqlalchemy to minimum supported version: ${min_sqlalchemy_version}${COLOR_RESET}" echo pip install --root-user-action ignore "sqlalchemy==${min_sqlalchemy_version}" pip check -fi +} -pip uninstall --root-user-action ignore "pytest-capture-warnings" -y >/dev/null 2>&1 || true +function check_run_tests() { + if [[ ${RUN_TESTS=} != "true" ]]; then + return + fi -set +u -if [[ "${RUN_TESTS}" != "true" ]]; then - exec /bin/bash "${@}" -fi -set -u + if [[ ${REMOVE_ARM_PACKAGES:="false"} == "true" ]]; then + # Test what happens if we do not have ARM packages installed. + # This is useful to see if pytest collection works without ARM packages which is important + # for the MacOS M1 users running tests in their ARM machines with `breeze testing tests` command + python "${IN_CONTAINER_DIR}/remove_arm_packages.py" + fi -if [[ ${REMOVE_ARM_PACKAGES:="false"} == "true" ]]; then - # Test what happens if we do not have ARM packages installed. - # This is useful to see if pytest collection works without ARM packages which is important - # for the MacOS M1 users running tests in their ARM machines with `breeze testing tests` command - python "${IN_CONTAINER_DIR}/remove_arm_packages.py" -fi + if [[ ${TEST_TYPE} == "PlainAsserts" ]]; then + # Plain asserts should be converted to env variable to make sure they are taken into account + # otherwise they will not be effective during test collection when plain assert is breaking collection + export PYTEST_PLAIN_ASSERTS="true" + fi -if [[ ${TEST_TYPE} == "PlainAsserts" ]]; then - # Plain asserts should be converted to env variable to make sure they are taken into account - # otherwise they will not be effective during test collection when plain assert is breaking collection - export PYTEST_PLAIN_ASSERTS="true" -fi + if [[ ${RUN_SYSTEM_TESTS:="false"} == "true" ]]; then + exec "${IN_CONTAINER_DIR}/run_system_tests.sh" "${@}" + else + exec "${IN_CONTAINER_DIR}/run_ci_tests.sh" "${@}" + fi +} -if [[ ${RUN_SYSTEM_TESTS:="false"} == "true" ]]; then - "${IN_CONTAINER_DIR}/run_system_tests.sh" "${@}" -else - "${IN_CONTAINER_DIR}/run_ci_tests.sh" "${@}" -fi +determine_airflow_to_use +environment_initialization +check_boto_upgrade +check_download_sqlalchemy +check_run_tests "${@}" + +exec /bin/bash "${@}" EOF # The content below is automatically copied from scripts/docker/entrypoint_exec.sh diff --git a/dev/breeze/src/airflow_breeze/commands/developer_commands.py b/dev/breeze/src/airflow_breeze/commands/developer_commands.py index 9e1d50a33e735..050bc02271625 100644 --- a/dev/breeze/src/airflow_breeze/commands/developer_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/developer_commands.py @@ -18,6 +18,7 @@ import os import re +import shlex import shutil import sys import threading @@ -33,14 +34,11 @@ from airflow_breeze.global_constants import ( ALLOWED_TTY, DEFAULT_PYTHON_MAJOR_MINOR_VERSION, - DOCKER_DEFAULT_PLATFORM, - MOUNT_SELECTED, ) from airflow_breeze.params.build_ci_params import BuildCiParams from airflow_breeze.params.doc_build_params import DocBuildParams from airflow_breeze.params.shell_params import ShellParams from airflow_breeze.pre_commit_ids import PRE_COMMIT_LIST -from airflow_breeze.utils.cache import read_from_cache_file from airflow_breeze.utils.coertions import one_or_none_set from airflow_breeze.utils.common_options import ( argument_doc_packages, @@ -98,26 +96,24 @@ from airflow_breeze.utils.console import get_console from airflow_breeze.utils.custom_param_types import BetterChoice from airflow_breeze.utils.docker_command_utils import ( + bring_compose_project_down, check_docker_resources, + enter_shell, + execute_command_in_shell, fix_ownership_using_docker, - get_extra_docker_flags, perform_environment_checks, ) from airflow_breeze.utils.packages import expand_all_provider_packages from airflow_breeze.utils.path_utils import ( AIRFLOW_SOURCES_ROOT, cleanup_python_generated_files, - create_mypy_volume_if_needed, ) from airflow_breeze.utils.run_utils import ( - RunCommandResult, assert_pre_commit_installed, - filter_out_none, run_command, run_compile_www_assets, ) from airflow_breeze.utils.shared_options import get_dry_run, get_verbose, set_forced_answer -from airflow_breeze.utils.visuals import ASCIIART, ASCIIART_STYLE, CHEATSHEET, CHEATSHEET_STYLE def _determine_constraint_branch_used(airflow_constraints_reference: str, use_airflow_version: str | None): @@ -288,11 +284,12 @@ def shell( airflow_constraints_reference = _determine_constraint_branch_used( airflow_constraints_reference, use_airflow_version ) - result = enter_shell( + shell_params = ShellParams( airflow_constraints_location=airflow_constraints_location, airflow_constraints_mode=airflow_constraints_mode, airflow_constraints_reference=airflow_constraints_reference, airflow_extras=airflow_extras, + airflow_skip_constraints=airflow_skip_constraints, backend=backend, builder=builder, celery_broker=celery_broker, @@ -320,6 +317,7 @@ def shell( providers_constraints_location=providers_constraints_location, providers_constraints_mode=providers_constraints_mode, providers_constraints_reference=providers_constraints_reference, + providers_skip_constraints=providers_skip_constraints, python=python, quiet=quiet, run_db_tests_only=run_db_tests_only, @@ -335,6 +333,8 @@ def shell( restart=restart, warn_image_upgrade_needed=warn_image_upgrade_needed, ) + rebuild_or_pull_ci_image_if_needed(command_params=shell_params) + result = enter_shell(shell_params=shell_params) fix_ownership_using_docker() sys.exit(result.returncode) @@ -450,12 +450,12 @@ def start_airflow( airflow_constraints_reference, use_airflow_version ) - result = enter_shell( + shell_params = ShellParams( airflow_constraints_location=airflow_constraints_location, airflow_constraints_mode=airflow_constraints_mode, airflow_constraints_reference=airflow_constraints_reference, - airflow_skip_constraints=airflow_skip_constraints, airflow_extras=airflow_extras, + airflow_skip_constraints=airflow_skip_constraints, backend=backend, builder=builder, celery_broker=celery_broker, @@ -492,6 +492,8 @@ def start_airflow( use_airflow_version=use_airflow_version, use_packages_from_dist=use_packages_from_dist, ) + rebuild_or_pull_ci_image_if_needed(command_params=shell_params) + result = enter_shell(shell_params=shell_params) fix_ownership_using_docker() sys.exit(result.returncode) @@ -549,7 +551,6 @@ def build_docs( for directory in docs_dir.rglob(dir_name): get_console().print(f"[info]Removing {directory}") shutil.rmtree(directory, ignore_errors=True) - ci_image_name = build_params.airflow_image_name doc_builder = DocBuildParams( package_filter=package_filter, docs_only=docs_only, @@ -557,32 +558,20 @@ def build_docs( one_pass_only=one_pass_only, short_doc_packages=expand_all_provider_packages(doc_packages), ) + cmd = "/opt/airflow/scripts/in_container/run_docs_build.sh " + " ".join( + [shlex.quote(arg) for arg in doc_builder.args_doc_builder] + ) shell_params = ShellParams( github_repository=github_repository, python=DEFAULT_PYTHON_MAJOR_MINOR_VERSION, - quiet=True, - skip_environment_initialization=True, - skip_image_upgrade_check=True, ) - extra_docker_flags = get_extra_docker_flags(mount_sources=MOUNT_SELECTED) - cmd = [ - "docker", - "run", - "-t", - *extra_docker_flags, - "--pull", - "never", - ci_image_name, - "/opt/airflow/scripts/in_container/run_docs_build.sh", - *doc_builder.args_doc_builder, - ] - process = run_command(cmd, text=True, check=False, env=shell_params.env_variables_for_docker_commands) + result = execute_command_in_shell(shell_params, project_name="docs", command=cmd) fix_ownership_using_docker() - if process.returncode == 0: + if result.returncode == 0: get_console().print( "[info]Start the webserver in breeze and view the built docs at http://localhost:28080/docs/[/]" ) - sys.exit(process.returncode) + sys.exit(result.returncode) @main.command( @@ -861,125 +850,6 @@ def exec(exec_args: tuple): sys.exit(process.returncode) -def enter_shell(**kwargs) -> RunCommandResult: - """ - Executes entering shell using the parameters passed as kwargs: - - * checks if docker version is good - * checks if docker-compose version is good - * updates kwargs with cached parameters - * displays ASCIIART and CHEATSHEET unless disabled - * build ShellParams from the updated kwargs - * shuts down existing project - * executes the command to drop the user to Breeze shell - - """ - quiet: bool = kwargs.get("quiet") or False - perform_environment_checks(quiet=quiet) - fix_ownership_using_docker(quiet=quiet) - cleanup_python_generated_files() - if read_from_cache_file("suppress_asciiart") is None and not quiet: - get_console().print(ASCIIART, style=ASCIIART_STYLE) - if read_from_cache_file("suppress_cheatsheet") is None and not quiet: - get_console().print(CHEATSHEET, style=CHEATSHEET_STYLE) - shell_params = ShellParams(**filter_out_none(**kwargs)) - rebuild_or_pull_ci_image_if_needed(command_params=shell_params) - if shell_params.use_airflow_version: - # in case you use specific version of Airflow, you want to bring airflow down automatically before - # using it. This prevents the problem that if you have newer DB, airflow will not know how - # to migrate to it and fail with "Can't locate revision identified by 'xxxx'". - get_console().print( - f"[warning]Bringing the project down as {shell_params.use_airflow_version} " - f"airflow version is used[/]" - ) - bring_compose_project_down(preserve_volumes=False, shell_params=shell_params) - if shell_params.backend == "sqlite": - get_console().print( - f"\n[warning]backend: sqlite is not " - f"compatible with executor: {shell_params.executor}. " - f"Changing the executor to SequentialExecutor.\n" - ) - shell_params.executor = "SequentialExecutor" - - if shell_params.executor == "CeleryExecutor" and shell_params.use_airflow_version: - if shell_params.airflow_extras and "celery" not in shell_params.airflow_extras.split(): - get_console().print( - f"\n[warning]CeleryExecutor requires airflow_extras: celery. " - f"Adding celery to extras: '{shell_params.airflow_extras}'.\n" - ) - shell_params.airflow_extras += ",celery" - elif not shell_params.airflow_extras: - get_console().print( - "\n[warning]CeleryExecutor requires airflow_extras: celery. " - "Setting airflow extras to 'celery'.\n" - ) - shell_params.airflow_extras = "celery" - if shell_params.restart: - bring_compose_project_down(preserve_volumes=False, shell_params=shell_params) - if shell_params.include_mypy_volume: - create_mypy_volume_if_needed() - shell_params.print_badge_info() - cmd = ["docker", "compose"] - if shell_params.quiet: - cmd.extend(["--progress", "quiet"]) - if shell_params.project_name: - cmd.extend(["--project-name", shell_params.project_name]) - cmd.extend(["run", "--service-ports", "--rm"]) - if shell_params.tty == "disabled": - cmd.append("--no-TTY") - elif shell_params.tty == "enabled": - cmd.append("--tty") - cmd.append("airflow") - cmd_added = shell_params.command_passed - if cmd_added is not None: - cmd.extend(["-c", cmd_added]) - if "arm64" in DOCKER_DEFAULT_PLATFORM: - if shell_params.backend == "mysql": - get_console().print("\n[warn]MySQL use MariaDB client binaries on ARM architecture.[/]\n") - elif shell_params.backend == "mssql": - get_console().print("\n[error]MSSQL is not supported on ARM architecture[/]\n") - sys.exit(1) - - if "openlineage" in shell_params.integration or "all" in shell_params.integration: - if shell_params.backend != "postgres" or shell_params.postgres_version not in ["12", "13", "14"]: - get_console().print( - "\n[error]Only PostgreSQL 12, 13, and 14 are supported " - "as a backend with OpenLineage integration via Breeze[/]\n" - ) - sys.exit(1) - - command_result = run_command( - cmd, - text=True, - check=False, - env=shell_params.env_variables_for_docker_commands, - output_outside_the_group=True, - ) - if command_result.returncode == 0: - return command_result - else: - get_console().print(f"[red]Error {command_result.returncode} returned[/]") - if get_verbose(): - get_console().print(command_result.stderr) - return command_result - - -def bring_compose_project_down(preserve_volumes: bool, shell_params: ShellParams): - down_command_to_execute = ["docker", "compose"] - if shell_params.project_name: - down_command_to_execute.extend(["--project-name", shell_params.project_name]) - down_command_to_execute.extend(["down", "--remove-orphans"]) - if not preserve_volumes: - down_command_to_execute.append("--volumes") - run_command( - down_command_to_execute, - text=True, - check=False, - capture_output=shell_params.quiet, - env=shell_params.env_variables_for_docker_commands, - ) - - def stop_exec_on_error(returncode: int): get_console().print("\n[error]ERROR in finding the airflow docker-compose process id[/]\n") sys.exit(returncode) diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py index ad3df865f5447..ff3f9bf9c7f82 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands.py @@ -20,7 +20,6 @@ import operator import os import re -import shlex import shutil import sys import textwrap @@ -107,8 +106,8 @@ from airflow_breeze.utils.custom_param_types import BetterChoice from airflow_breeze.utils.docker_command_utils import ( check_remote_ghcr_io_commands, + execute_command_in_shell, fix_ownership_using_docker, - get_extra_docker_flags, perform_environment_checks, ) from airflow_breeze.utils.github import download_constraints_file, get_active_airflow_versions @@ -144,7 +143,6 @@ from airflow_breeze.utils.publish_docs_builder import PublishDocsBuilder from airflow_breeze.utils.python_versions import get_python_version_list from airflow_breeze.utils.run_utils import ( - RunCommandResult, clean_www_assets, run_command, run_compile_www_assets, @@ -152,13 +150,6 @@ from airflow_breeze.utils.shared_options import get_dry_run, get_verbose from airflow_breeze.utils.versions import is_pre_release -option_debug_release_management = click.option( - "--debug", - is_flag=True, - help="Drop user in shell instead of running the command. Useful for debugging.", - envvar="DEBUG", -) - if TYPE_CHECKING: from packaging.version import Version @@ -171,60 +162,6 @@ class VersionedFile(NamedTuple): comparable_version: Version -def run_docker_command_with_debug( - shell_params: ShellParams, - command: list[str], - debug: bool, - enable_input: bool = False, - output_outside_the_group: bool = False, - **kwargs, -) -> RunCommandResult: - env = shell_params.env_variables_for_docker_commands - extra_docker_flags = get_extra_docker_flags(mount_sources=shell_params.mount_sources) - if enable_input or debug: - term_flag = "-it" - else: - term_flag = "-t" - base_command = [ - "docker", - "run", - term_flag, - *extra_docker_flags, - "--pull", - "never", - shell_params.airflow_image_name_with_tag, - ] - if debug: - cmd_string = " ".join([shlex.quote(s) for s in command if s != "-c"]) - base_command.extend( - [ - "-c", - f""" -echo -e '\\e[34mRun this command to debug: - - {cmd_string} - -\\e[0m\n'; exec bash -""", - ] - ) - return run_command( - base_command, - output_outside_the_group=output_outside_the_group, - env=env, - **kwargs, - ) - else: - base_command.extend(command) - return run_command( - base_command, - check=False, - env=env, - output_outside_the_group=output_outside_the_group, - **kwargs, - ) - - AIRFLOW_PIP_VERSION = "23.3.1" WHEEL_VERSION = "0.36.2" GITPYTHON_VERSION = "3.1.40" @@ -627,22 +564,16 @@ def prepare_provider_packages( def run_generate_constraints( shell_params: ShellParams, - debug: bool, output: Output | None, ) -> tuple[int, str]: - cmd_to_run = [ - "/opt/airflow/scripts/in_container/run_generate_constraints.sh", - ] - generate_constraints_result = run_docker_command_with_debug( - shell_params=shell_params, - command=cmd_to_run, - debug=debug, - output=output, - output_outside_the_group=True, + result = execute_command_in_shell( + shell_params, + project_name="constraints", + command="/opt/airflow/scripts/in_container/run_generate_constraints.sh", ) fix_ownership_using_docker() return ( - generate_constraints_result.returncode, + result.returncode, f"Constraints {shell_params.airflow_constraints_mode}:{shell_params.python}", ) @@ -679,7 +610,6 @@ def run_generate_constraints_in_parallel( run_generate_constraints, kwds={ "shell_params": shell_params, - "debug": False, "output": outputs[index], }, ) @@ -707,7 +637,6 @@ def run_generate_constraints_in_parallel( @option_debug_resources @option_python_versions @option_image_tag_for_running -@option_debug_release_management @option_airflow_constraints_mode_ci @option_chicken_egg_providers @option_github_repository @@ -716,7 +645,6 @@ def run_generate_constraints_in_parallel( @option_answer def generate_constraints( airflow_constraints_mode: str, - debug: bool, debug_resources: bool, github_repository: str, image_tag: str | None, @@ -731,9 +659,6 @@ def generate_constraints( check_remote_ghcr_io_commands() fix_ownership_using_docker() cleanup_python_generated_files() - if debug and run_in_parallel: - get_console().print("\n[error]Cannot run --debug and --run-in-parallel at the same time[/]\n") - sys.exit(1) if run_in_parallel: given_answer = user_confirm( f"Did you build all CI images {python_versions} with --upgrade-to-newer-dependencies flag set?", @@ -765,37 +690,33 @@ def generate_constraints( python_version_list = get_python_version_list(python_versions) shell_params_list = [ ShellParams( - image_tag=image_tag, - python=python, - github_repository=github_repository, airflow_constraints_mode=airflow_constraints_mode, chicken_egg_providers=chicken_egg_providers, + github_repository=github_repository, + image_tag=image_tag, + python=python, ) for python in python_version_list ] run_generate_constraints_in_parallel( - shell_params_list=shell_params_list, - parallelism=parallelism, - skip_cleanup=skip_cleanup, debug_resources=debug_resources, include_success_outputs=True, + parallelism=parallelism, python_version_list=python_version_list, + shell_params_list=shell_params_list, + skip_cleanup=skip_cleanup, ) else: shell_params = ShellParams( - image_tag=image_tag, - python=python, - github_repository=github_repository, - skip_environment_initialization=True, - skip_image_upgrade_check=True, - quiet=True, airflow_constraints_mode=airflow_constraints_mode, chicken_egg_providers=chicken_egg_providers, + github_repository=github_repository, + image_tag=image_tag, + python=python, ) return_code, info = run_generate_constraints( shell_params=shell_params, output=None, - debug=debug, ) if return_code != 0: get_console().print(f"[error]There was an error when generating constraints: {info}[/]") @@ -849,17 +770,11 @@ def get_all_providers_in_dist(package_format: str, install_selected_providers: s def _run_command_for_providers( shell_params: ShellParams, - cmd_to_run: list[str], list_of_providers: list[str], output: Output | None, ) -> tuple[int, str]: shell_params.install_selected_providers = " ".join(list_of_providers) - result_command = run_docker_command_with_debug( - shell_params=shell_params, - command=cmd_to_run, - debug=False, - output=output, - ) + result_command = execute_command_in_shell(shell_params, project_name="providers") return result_command.returncode, f"{list_of_providers}" @@ -875,7 +790,6 @@ def _run_command_for_providers( @option_airflow_constraints_reference @option_airflow_extras @option_airflow_skip_constraints -@option_debug_release_management @option_debug_resources @option_dry_run @option_github_repository @@ -900,7 +814,6 @@ def install_provider_packages( airflow_constraints_reference: str, airflow_skip_constraints: bool, airflow_extras: str, - debug: bool, debug_resources: bool, github_repository: str, include_success_outputs: bool, @@ -927,12 +840,15 @@ def install_provider_packages( airflow_constraints_reference=airflow_constraints_reference, airflow_extras=airflow_extras, airflow_skip_constraints=airflow_skip_constraints, + # We just want to install the providers by entrypoint + # we do not need to run any command in the container + extra_args=("exit 0",), github_repository=github_repository, install_selected_providers=install_selected_providers, mount_sources=mount_sources, package_format=package_format, - providers_constraints_mode=providers_constraints_mode, providers_constraints_location=providers_constraints_location, + providers_constraints_mode=providers_constraints_mode, providers_constraints_reference=providers_constraints_reference, providers_skip_constraints=providers_skip_constraints, python=python, @@ -940,11 +856,6 @@ def install_provider_packages( use_packages_from_dist=use_packages_from_dist, ) rebuild_or_pull_ci_image_if_needed(command_params=shell_params) - # We just want to install the providers by entrypoint, we do not need to run any command in the container - cmd_to_run = [ - "-c", - "exit 0", - ] if run_in_parallel: list_of_all_providers = get_all_providers_in_dist( package_format=package_format, install_selected_providers=install_selected_providers @@ -995,7 +906,6 @@ def install_provider_packages( _run_command_for_providers, kwds={ "shell_params": shell_params, - "cmd_to_run": cmd_to_run, "list_of_providers": list_of_providers, "output": outputs[index], }, @@ -1010,12 +920,7 @@ def install_provider_packages( skip_cleanup=skip_cleanup, ) else: - result_command = run_docker_command_with_debug( - shell_params=shell_params, - command=cmd_to_run, - debug=debug, - output_outside_the_group=True, - ) + result_command = execute_command_in_shell(shell_params, project_name="providers") fix_ownership_using_docker() sys.exit(result_command.returncode) @@ -1029,7 +934,6 @@ def install_provider_packages( @option_airflow_constraints_reference @option_airflow_extras @option_airflow_skip_constraints -@option_debug_release_management @option_dry_run @option_github_repository @option_install_selected_providers @@ -1048,7 +952,6 @@ def verify_provider_packages( airflow_constraints_mode: str, airflow_constraints_reference: str, airflow_extras: str, - debug: bool, github_repository: str, install_selected_providers: str, mount_sources: str, @@ -1074,13 +977,11 @@ def verify_provider_packages( airflow_constraints_reference=airflow_constraints_reference, airflow_extras=airflow_extras, airflow_skip_constraints=airflow_skip_constraints, - backend="sqlite", - executor="SequentialExecutor", github_repository=github_repository, mount_sources=mount_sources, package_format=package_format, - providers_constraints_mode=providers_constraints_mode, providers_constraints_location=providers_constraints_location, + providers_constraints_mode=providers_constraints_mode, providers_constraints_reference=providers_constraints_reference, providers_skip_constraints=providers_skip_constraints, python=python, @@ -1088,15 +989,10 @@ def verify_provider_packages( use_packages_from_dist=use_packages_from_dist, ) rebuild_or_pull_ci_image_if_needed(command_params=shell_params) - cmd_to_run = [ - "-c", - "python /opt/airflow/scripts/in_container/verify_providers.py", - ] - result_command = run_docker_command_with_debug( - shell_params=shell_params, - command=cmd_to_run, - debug=debug, - output_outside_the_group=True, + result_command = execute_command_in_shell( + shell_params, + project_name="providers", + command="python /opt/airflow/scripts/in_container/verify_providers.py", ) fix_ownership_using_docker() sys.exit(result_command.returncode) diff --git a/dev/breeze/src/airflow_breeze/commands/release_management_commands_config.py b/dev/breeze/src/airflow_breeze/commands/release_management_commands_config.py index d51aa8722ed1e..bf03085e992f8 100644 --- a/dev/breeze/src/airflow_breeze/commands/release_management_commands_config.py +++ b/dev/breeze/src/airflow_breeze/commands/release_management_commands_config.py @@ -67,7 +67,6 @@ "--python", "--mount-sources", "--github-repository", - "--debug", ], }, { @@ -96,7 +95,6 @@ "--python", "--mount-sources", "--github-repository", - "--debug", ], }, { @@ -163,7 +161,6 @@ "--python", "--airflow-constraints-mode", "--chicken-egg-providers", - "--debug", "--github-repository", ], }, diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 6d8306058d86b..f336900d047fb 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -86,6 +86,9 @@ DEFAULT_ALLOWED_EXECUTOR = ALLOWED_EXECUTORS[0] START_AIRFLOW_ALLOWED_EXECUTORS = ["LocalExecutor", "CeleryExecutor", "SequentialExecutor"] START_AIRFLOW_DEFAULT_ALLOWED_EXECUTOR = START_AIRFLOW_ALLOWED_EXECUTORS[0] + +SEQUENTIAL_EXECUTOR = "SequentialExecutor" + ALLOWED_KIND_OPERATIONS = ["start", "stop", "restart", "status", "deploy", "test", "shell", "k9s"] ALLOWED_CONSTRAINTS_MODES_CI = ["constraints-source-providers", "constraints", "constraints-no-providers"] ALLOWED_CONSTRAINTS_MODES_PROD = ["constraints", "constraints-no-providers", "constraints-source-providers"] @@ -196,7 +199,7 @@ def all_helm_test_packages() -> list[str]: def get_default_platform_machine() -> str: machine = platform.uname().machine # Some additional conversion for various platforms... - machine = {"AMD64": "x86_64"}.get(machine, machine) + machine = {"x86_64": "amd64"}.get(machine, machine) return machine diff --git a/dev/breeze/src/airflow_breeze/params/shell_params.py b/dev/breeze/src/airflow_breeze/params/shell_params.py index 311e2497fa322..58c49ecf9b26e 100644 --- a/dev/breeze/src/airflow_breeze/params/shell_params.py +++ b/dev/breeze/src/airflow_breeze/params/shell_params.py @@ -52,8 +52,8 @@ get_airflow_version, ) from airflow_breeze.utils.console import get_console +from airflow_breeze.utils.docker_command_utils import is_docker_rootless from airflow_breeze.utils.host_info_utils import get_host_group_id, get_host_os, get_host_user_id -from airflow_breeze.utils.packages import get_suspended_provider_folders from airflow_breeze.utils.path_utils import ( AIRFLOW_SOURCES_ROOT, BUILD_CACHE_DIR, @@ -153,7 +153,7 @@ class ShellParams: executor: str = START_AIRFLOW_DEFAULT_ALLOWED_EXECUTOR extra_args: tuple = () force_build: bool = False - forward_credentials: str = "false" + forward_credentials: bool = False forward_ports: bool = True github_actions: str = os.environ.get("GITHUB_ACTIONS", "false") github_repository: str = APACHE_AIRFLOW_GITHUB_REPOSITORY @@ -198,7 +198,7 @@ class ShellParams: skip_provider_tests: bool = False skip_ssh_setup: bool = os.environ.get("SKIP_SSH_SETUP", "false") == "true" standalone_dag_processor: bool = False - start_airflow: str = "false" + start_airflow: bool = False test_type: str | None = None tty: str = "auto" upgrade_boto: bool = False @@ -298,7 +298,13 @@ def print_badge_info(self): get_console().print(f"[info]Airflow used at runtime: {self.use_airflow_version}[/]") def get_backend_compose_files(self, backend: str) -> list[Path]: - backend_docker_compose_file = DOCKER_COMPOSE_DIR / f"backend-{backend}.yml" + if backend == "sqlite" and self.project_name != "breeze": + # When running scripts, we do not want to mount the volume to make sure that the + # sqlite database is not persisted between runs of the script and that the + # breeze database is not cleaned accidentally + backend_docker_compose_file = DOCKER_COMPOSE_DIR / f"backend-{backend}-no-volume.yml" + else: + backend_docker_compose_file = DOCKER_COMPOSE_DIR / f"backend-{backend}.yml" if backend in ("sqlite", "none") or not self.forward_ports: return [backend_docker_compose_file] if self.project_name == "pre-commit": @@ -344,8 +350,6 @@ def compose_file(self) -> str: compose_file_list.append(DOCKER_COMPOSE_DIR / "remove-sources.yml") if self.forward_credentials: compose_file_list.append(DOCKER_COMPOSE_DIR / "forward-credentials.yml") - if self.use_airflow_version is not None: - compose_file_list.append(DOCKER_COMPOSE_DIR / "remove-sources.yml") if self.include_mypy_volume: compose_file_list.append(DOCKER_COMPOSE_DIR / "mypy.yml") if "all-testable" in self.integration: @@ -383,6 +387,8 @@ def airflow_celery_broker_url(self) -> str: @cached_property def suspended_providers_folders(self): + from airflow_breeze.utils.packages import get_suspended_provider_folders + return " ".join(get_suspended_provider_folders()).strip() @cached_property @@ -453,20 +459,7 @@ def add_docker_in_docker(self, compose_file_list: list[Path]): @cached_property def rootless_docker(self) -> bool: - try: - response = run_command( - ["docker", "info", "-f", "{{println .SecurityOptions}}"], - capture_output=True, - check=False, - text=True, - ) - if response.returncode == 0 and "rootless" in response.stdout.strip(): - get_console().print("[info]Docker is running in rootless mode.[/]\n") - return True - except FileNotFoundError: - # we ignore if docker is missing - pass - return False + return is_docker_rootless() @cached_property def env_variables_for_docker_commands(self) -> dict[str, str]: diff --git a/dev/breeze/src/airflow_breeze/utils/common_options.py b/dev/breeze/src/airflow_breeze/utils/common_options.py index 2be5db94c4b13..bf6288658eae8 100644 --- a/dev/breeze/src/airflow_breeze/utils/common_options.py +++ b/dev/breeze/src/airflow_breeze/utils/common_options.py @@ -48,6 +48,7 @@ AUTOCOMPLETE_INTEGRATIONS, DEFAULT_ALLOWED_EXECUTOR, DEFAULT_CELERY_BROKER, + DOCKER_DEFAULT_PLATFORM, SINGLE_PLATFORMS, START_AIRFLOW_ALLOWED_EXECUTORS, START_AIRFLOW_DEFAULT_ALLOWED_EXECUTOR, @@ -276,12 +277,14 @@ def _set_default_from_parent(ctx: click.core.Context, option: click.core.Option, "--platform", help="Platform for Airflow image.", envvar="PLATFORM", + default=DOCKER_DEFAULT_PLATFORM if not generating_command_images() else "linux/amd64", type=BetterChoice(ALLOWED_PLATFORMS), ) option_platform_single = click.option( "--platform", help="Platform for Airflow image.", envvar="PLATFORM", + default=DOCKER_DEFAULT_PLATFORM if not generating_command_images() else "linux/amd64", type=BetterChoice(SINGLE_PLATFORMS), ) option_upgrade_to_newer_dependencies = click.option( 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 10e628ba5d6fc..c3c5b3ef8147d 100644 --- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py @@ -26,9 +26,15 @@ from typing import TYPE_CHECKING from airflow_breeze.params.build_prod_params import BuildProdParams -from airflow_breeze.utils.host_info_utils import get_host_os -from airflow_breeze.utils.image import find_available_ci_image -from airflow_breeze.utils.path_utils import AIRFLOW_SOURCES_ROOT, GENERATED_DOCKER_ENV_FILE +from airflow_breeze.utils.cache import read_from_cache_file +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, + cleanup_python_generated_files, + create_mypy_volume_if_needed, +) +from airflow_breeze.utils.shared_options import get_verbose +from airflow_breeze.utils.visuals import ASCIIART, ASCIIART_STYLE, CHEATSHEET, CHEATSHEET_STYLE try: from packaging import version @@ -39,13 +45,11 @@ from airflow_breeze.global_constants import ( ALLOWED_CELERY_BROKERS, ALLOWED_DEBIAN_VERSIONS, - APACHE_AIRFLOW_GITHUB_REPOSITORY, DEFAULT_PYTHON_MAJOR_MINOR_VERSION, + DOCKER_DEFAULT_PLATFORM, MIN_DOCKER_COMPOSE_VERSION, MIN_DOCKER_VERSION, - MOUNT_ALL, - MOUNT_REMOVE, - MOUNT_SELECTED, + SEQUENTIAL_EXECUTOR, ) from airflow_breeze.utils.console import Output, get_console from airflow_breeze.utils.run_utils import ( @@ -56,6 +60,7 @@ if TYPE_CHECKING: from airflow_breeze.params.common_build_params import CommonBuildParams + from airflow_breeze.params.shell_params import ShellParams # Those are volumes that are mounted when MOUNT_SELECTED is chosen (which is the default when # entering Breeze. MOUNT_SELECTED prevents to mount the files that you can have accidentally added @@ -99,39 +104,6 @@ ] -def get_extra_docker_flags(mount_sources: str, include_mypy_volume: bool = False) -> list[str]: - """ - Returns extra docker flags based on the type of mounting we want to do for sources. - - :param mount_sources: type of mounting we want to have - :param env: environment variables to pass to docker - :param include_mypy_volume: includes mypy_volume - :return: extra flag as list of strings - """ - extra_docker_flags = [] - if mount_sources == MOUNT_ALL: - extra_docker_flags.extend(["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT},dst=/opt/airflow/"]) - elif mount_sources == MOUNT_SELECTED: - for src, dst in VOLUMES_FOR_SELECTED_MOUNTS: - if (AIRFLOW_SOURCES_ROOT / src).exists(): - extra_docker_flags.extend( - ["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT / src},dst={dst}"] - ) - if include_mypy_volume: - extra_docker_flags.extend( - ["--mount", "type=volume,src=mypy-cache-volume,dst=/opt/airflow/.mypy_cache"] - ) - elif mount_sources == MOUNT_REMOVE: - extra_docker_flags.extend( - ["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT / 'empty'},dst=/opt/airflow/airflow"] - ) - extra_docker_flags.extend(["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT / 'files'},dst=/files"]) - extra_docker_flags.extend(["--mount", f"type=bind,src={AIRFLOW_SOURCES_ROOT / 'dist'},dst=/dist"]) - extra_docker_flags.extend(["--rm"]) - extra_docker_flags.extend(["--env-file", GENERATED_DOCKER_ENV_FILE.as_posix()]) - return extra_docker_flags - - def check_docker_resources(airflow_image_name: str) -> RunCommandResult: """ Check if we have enough resources to run docker. This is done via running script embedded in our image. @@ -563,21 +535,26 @@ def fix_ownership_using_docker(quiet: bool = False): if get_host_os() != "linux": # no need to even attempt fixing ownership on MacOS/Windows return - shell_params = find_available_ci_image( - github_repository=APACHE_AIRFLOW_GITHUB_REPOSITORY, - ) - extra_docker_flags = get_extra_docker_flags(mount_sources=MOUNT_ALL) cmd = [ "docker", "run", + "-v", + "./scripts/in_container:/opt/airflow/scripts/in_container", + "-e", + f"HOST_OS={get_host_os()}", + "-e", + f"HOST_USER_ID={get_host_user_id()}", + "-e", + f"HOST_GROUP_ID={get_host_group_id()}", + "-e", + f"DOCKER_IS_ROOTLESS={is_docker_rootless()}", + "-e", + f"VERBOSE_COMMANDS={str(not quiet).lower()}", "-t", - *extra_docker_flags, OWNERSHIP_CLEANUP_DOCKER_TAG, "/opt/airflow/scripts/in_container/run_fix_ownership.py", ] - run_command( - cmd, text=True, check=False, env=shell_params.env_variables_for_docker_commands, capture_output=quiet - ) + run_command(cmd, text=True, check=False, capture_output=quiet) def remove_docker_networks(networks: list[str] | None = None) -> None: @@ -671,3 +648,185 @@ def get_docker_build_env(image_params: CommonBuildParams) -> dict[str, str]: if image_params.docker_host: env["DOCKER_HOST"] = image_params.docker_host return env + + +def bring_compose_project_down(preserve_volumes: bool, shell_params: ShellParams): + down_command_to_execute = ["docker", "compose"] + if shell_params.project_name: + down_command_to_execute.extend(["--project-name", shell_params.project_name]) + down_command_to_execute.extend(["down", "--remove-orphans"]) + if not preserve_volumes: + down_command_to_execute.append("--volumes") + run_command( + down_command_to_execute, + text=True, + check=False, + capture_output=shell_params.quiet, + env=shell_params.env_variables_for_docker_commands, + ) + + +def execute_command_in_shell( + shell_params: ShellParams, project_name: str, command: str | None = None +) -> RunCommandResult: + """Executes command in shell. + + When you want to execute a script/bash command inside the CI container and want to use `enter_shell` + for this purpose, the helper methods sets the following parameters of shell_params: + + * backend - to force sqlite backend + * clean_sql_db=True - to clean the sqlite DB + * executor - to force SequentialExecutor + * forward_ports=False - to avoid forwarding ports from the container to the host - again that will + allow to avoid clashes with other commands and opened breeze shell + * project_name - to avoid name clashes with default "breeze" project name used + * quiet=True - avoid displaying all "interactive" parts of Breeze: ASCIIART, CHEATSHEET, some diagnostics + * skip_environment_initialization - to avoid initializing interactive environment + * skip_image_upgrade_check - to avoid checking if the image is up to date + + if command is passed as parameter, extra_args - to pass the command to execute in the shell + + :param shell_params: shell parameters to use + :param project_name: Name of the project to use. This avoids name clashes with default 'breeze" + project name used - this way you will be able to run the command in parallel to regular + "breeze" shell opened in parallel + :param command: + """ + shell_params.backend = "sqlite" + shell_params.executor = SEQUENTIAL_EXECUTOR + shell_params.forward_ports = False + shell_params.project_name = project_name + shell_params.quiet = True + shell_params.skip_environment_initialization = True + shell_params.skip_image_upgrade_check = True + if get_verbose(): + get_console().print(f"[warning]Backend forced to: sqlite and {SEQUENTIAL_EXECUTOR}[/]") + get_console().print("[warning]Sqlite DB is cleaned[/]") + get_console().print(f"[warning]Executor forced to {SEQUENTIAL_EXECUTOR}[/]") + get_console().print("[warning]Disabled port forwarding[/]") + get_console().print(f"[warning]Project name set to: {project_name}[/]") + get_console().print("[warning]Forced quiet mode[/]") + get_console().print("[warning]Forced skipping environment initialization[/]") + get_console().print("[warning]Forced skipping upgrade check[/]") + if command: + shell_params.extra_args = (command,) + if get_verbose(): + get_console().print(f"[info]Command to execute: '{command}'[/]") + return enter_shell(shell_params) + + +def enter_shell(shell_params: ShellParams) -> RunCommandResult: + """ + Executes entering shell using the parameters passed as kwargs: + + * checks if docker version is good + * checks if docker-compose version is good + * updates kwargs with cached parameters + * displays ASCIIART and CHEATSHEET unless disabled + * build ShellParams from the updated kwargs + * shuts down existing project + * executes the command to drop the user to Breeze shell + """ + perform_environment_checks(quiet=shell_params.quiet) + fix_ownership_using_docker(quiet=shell_params.quiet) + cleanup_python_generated_files() + if read_from_cache_file("suppress_asciiart") is None and not shell_params.quiet: + get_console().print(ASCIIART, style=ASCIIART_STYLE) + if read_from_cache_file("suppress_cheatsheet") is None and not shell_params.quiet: + get_console().print(CHEATSHEET, style=CHEATSHEET_STYLE) + if shell_params.use_airflow_version: + # in case you use specific version of Airflow, you want to bring airflow down automatically before + # using it. This prevents the problem that if you have newer DB, airflow will not know how + # to migrate to it and fail with "Can't locate revision identified by 'xxxx'". + get_console().print( + f"[warning]Bringing the project down as {shell_params.use_airflow_version} " + f"airflow version is used[/]" + ) + bring_compose_project_down(preserve_volumes=False, shell_params=shell_params) + + if shell_params.backend == "sqlite" and shell_params.executor != SEQUENTIAL_EXECUTOR: + get_console().print( + f"\n[warning]backend: sqlite is not " + f"compatible with executor: {shell_params.executor}. " + f"Changing the executor to {SEQUENTIAL_EXECUTOR}.\n" + ) + shell_params.executor = SEQUENTIAL_EXECUTOR + + if shell_params.executor == "CeleryExecutor" and shell_params.use_airflow_version: + if shell_params.airflow_extras and "celery" not in shell_params.airflow_extras.split(): + get_console().print( + f"\n[warning]CeleryExecutor requires airflow_extras: celery. " + f"Adding celery to extras: '{shell_params.airflow_extras}'.\n" + ) + shell_params.airflow_extras += ",celery" + elif not shell_params.airflow_extras: + get_console().print( + "\n[warning]CeleryExecutor requires airflow_extras: celery. " + "Setting airflow extras to 'celery'.\n" + ) + shell_params.airflow_extras = "celery" + if shell_params.restart: + bring_compose_project_down(preserve_volumes=False, shell_params=shell_params) + if shell_params.include_mypy_volume: + create_mypy_volume_if_needed() + shell_params.print_badge_info() + cmd = ["docker", "compose"] + if shell_params.quiet: + cmd.extend(["--progress", "quiet"]) + if shell_params.project_name: + cmd.extend(["--project-name", shell_params.project_name]) + cmd.extend(["run", "--service-ports", "--rm"]) + if shell_params.tty == "disabled": + cmd.append("--no-TTY") + elif shell_params.tty == "enabled": + cmd.append("--tty") + cmd.append("airflow") + cmd_added = shell_params.command_passed + if cmd_added is not None: + cmd.extend(["-c", cmd_added]) + if "arm64" in DOCKER_DEFAULT_PLATFORM: + if shell_params.backend == "mysql": + get_console().print("\n[warn]MySQL use MariaDB client binaries on ARM architecture.[/]\n") + elif shell_params.backend == "mssql": + get_console().print("\n[error]MSSQL is not supported on ARM architecture[/]\n") + sys.exit(1) + + if "openlineage" in shell_params.integration or "all" in shell_params.integration: + if shell_params.backend != "postgres" or shell_params.postgres_version not in ["12", "13", "14"]: + get_console().print( + "\n[error]Only PostgreSQL 12, 13, and 14 are supported " + "as a backend with OpenLineage integration via Breeze[/]\n" + ) + sys.exit(1) + + command_result = run_command( + cmd, + text=True, + check=False, + env=shell_params.env_variables_for_docker_commands, + output_outside_the_group=True, + ) + if command_result.returncode == 0: + return command_result + else: + get_console().print(f"[red]Error {command_result.returncode} returned[/]") + if get_verbose(): + get_console().print(command_result.stderr) + return command_result + + +def is_docker_rootless() -> bool: + try: + response = run_command( + ["docker", "info", "-f", "{{println .SecurityOptions}}"], + capture_output=True, + check=False, + text=True, + ) + if response.returncode == 0 and "rootless" in response.stdout.strip(): + get_console().print("[info]Docker is running in rootless mode.[/]\n") + return True + except FileNotFoundError: + # we ignore if docker is missing + pass + return False diff --git a/dev/breeze/src/airflow_breeze/utils/path_utils.py b/dev/breeze/src/airflow_breeze/utils/path_utils.py index cc2e302e4d856..6e701e0c243fa 100644 --- a/dev/breeze/src/airflow_breeze/utils/path_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/path_utils.py @@ -356,11 +356,17 @@ def cleanup_python_generated_files(): for path in AIRFLOW_SOURCES_ROOT.rglob("*.pyc"): try: path.unlink() + except FileNotFoundError: + # File has been removed in the meantime. + pass except PermissionError: permission_errors.append(path) for path in AIRFLOW_SOURCES_ROOT.rglob("__pycache__"): try: shutil.rmtree(path) + except FileNotFoundError: + # File has been removed in the meantime. + pass except PermissionError: permission_errors.append(path) if permission_errors: diff --git a/dev/breeze/src/airflow_breeze/utils/run_utils.py b/dev/breeze/src/airflow_breeze/utils/run_utils.py index 874ae954d0ca3..c4389a9962d11 100644 --- a/dev/breeze/src/airflow_breeze/utils/run_utils.py +++ b/dev/breeze/src/airflow_breeze/utils/run_utils.py @@ -367,11 +367,6 @@ def commit_sha(): return "COMMIT_SHA_NOT_FOUND" -def filter_out_none(**kwargs) -> dict: - """Filters out all None values from parameters passed.""" - return {key: val for key, val in kwargs.items() if val is not None} - - def check_if_image_exists(image: str) -> bool: cmd_result = run_command( ["docker", "inspect", image], diff --git a/dev/breeze/tests/test_commands.py b/dev/breeze/tests/test_commands.py deleted file mode 100644 index 7420aa4940ec4..0000000000000 --- a/dev/breeze/tests/test_commands.py +++ /dev/null @@ -1,55 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from airflow_breeze.global_constants import MOUNT_ALL, MOUNT_REMOVE, MOUNT_SELECTED, MOUNT_SKIP -from airflow_breeze.utils.docker_command_utils import get_extra_docker_flags -from airflow_breeze.utils.visuals import ASCIIART - - -def test_visuals(): - assert 2051 == len(ASCIIART) - - -def test_get_extra_docker_flags_all(): - flags = get_extra_docker_flags(MOUNT_ALL) - assert "/empty," not in "".join(flags) - assert len(flags) < 10 - - -def test_get_extra_docker_flags_selected(): - flags = get_extra_docker_flags(MOUNT_SELECTED) - assert "/empty," not in "".join(flags) - assert len(flags) > 40 - - -def test_get_extra_docker_flags_remove(): - flags = get_extra_docker_flags(MOUNT_REMOVE) - assert "/empty," in "".join(flags) - assert len(flags) < 10 - - -def test_get_extra_docker_flags_skip(): - flags = get_extra_docker_flags(MOUNT_SKIP) - assert "/empty," not in "".join(flags) - assert len(flags) < 10 - - -def test_get_no_unknown_common_options(): - flags = get_extra_docker_flags(MOUNT_SKIP) - assert "/empty," not in "".join(flags) - assert len(flags) < 10 diff --git a/dev/breeze/tests/test_run_utils.py b/dev/breeze/tests/test_run_utils.py index 46e98c731d164..5b243e8f0e47b 100644 --- a/dev/breeze/tests/test_run_utils.py +++ b/dev/breeze/tests/test_run_utils.py @@ -21,7 +21,6 @@ from airflow_breeze.utils.run_utils import ( change_directory_permission, change_file_permission, - filter_out_none, ) @@ -44,10 +43,3 @@ def test_change_directory_permission(tmp_path): and (mode & stat.S_IXGRP) and (mode & stat.S_IXOTH) ) - - -def test_filter_out_none(): - dict_input_with_none = {"sample": None, "sample1": "One", "sample2": "Two", "samplen": None} - expected_dict_output = {"sample1": "One", "sample2": "Two"} - output_dict = filter_out_none(**dict_input_with_none) - assert output_dict == expected_dict_output diff --git a/images/breeze/output_ci-image_build.txt b/images/breeze/output_ci-image_build.txt index 5191c862d1290..e3df365cb4669 100644 --- a/images/breeze/output_ci-image_build.txt +++ b/images/breeze/output_ci-image_build.txt @@ -1 +1 @@ -6c0ed7011cf8358021d0ad3efaf0bb60 +fd70dd26b3009e8c5d2afa843084ecc1 diff --git a/images/breeze/output_prod-image_build.txt b/images/breeze/output_prod-image_build.txt index 7e0ba5119d44d..726e3487a87e4 100644 --- a/images/breeze/output_prod-image_build.txt +++ b/images/breeze/output_prod-image_build.txt @@ -1 +1 @@ -15f90171c145343a70af69092bbc4465 +43490a03007cb452ee5780d39149bbfb diff --git a/images/breeze/output_release-management_generate-constraints.svg b/images/breeze/output_release-management_generate-constraints.svg index 33caf14d07dcd..b8bc2f4b31b88 100644 --- a/images/breeze/output_release-management_generate-constraints.svg +++ b/images/breeze/output_release-management_generate-constraints.svg @@ -1,4 +1,4 @@ - +