Skip to content

Commit

Permalink
Remove "eager upgrade" from PROD image completely (#33784)
Browse files Browse the repository at this point in the history
There were still some left-overs of EAGER_UPGRADE in PROD image
building. Howwever "eager upgrade" only makes sense for CI images.
PROD images when being built should use eager upgrades as they
are produced in the CI image step.

This PR does the following:

* removes eager upgrade parameters from PROD image
* instead, prod image build has a new flag for installing
  the images: --use-constraints-for-context-packages which will
  automatically use constraints from "docker-context-files" if
  they are present there.
* modifies the CI workflows to upload constraints as artifacts
  and download them for PROD image build when "eager upgrade"
  has been used and directs it to use "source" constraints
* adds back support to "upgrade to newer dependencies" label
  that makes it easy to test "eager upgrade"

As the result, when PROD image is build in CI:

* when regular PR is run, it will use latest github "source" constraints
* whwn "eager upgrade" PR is run, it will use the eager-upgrade
  constraints that were generated during CI build
  • Loading branch information
potiuk authored Aug 26, 2023
1 parent a746def commit 2b1a194
Show file tree
Hide file tree
Showing 21 changed files with 428 additions and 322 deletions.
6 changes: 6 additions & 0 deletions .github/actions/build-ci-images/action.yml
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,12 @@ runs:
cat "files/constraints-${PYTHON_VERSION}/*.md" >> $GITHUB_STEP_SUMMARY || true
done
if: env.UPGRADE_TO_NEWER_DEPENDENCIES != 'false'
- name: "Upload constraint artifacts"
uses: actions/upload-artifact@v3
with:
name: constraints
path: ./files/constraints-*/constraints-*.txt
retention-days: 7
- name: "Fix ownership"
shell: bash
run: breeze ci fix-ownership
Expand Down
9 changes: 8 additions & 1 deletion .github/actions/build-prod-images/action.yml
Original file line number Diff line number Diff line change
Expand Up @@ -56,11 +56,18 @@ runs:
- name: "Move dist packages to docker-context files"
shell: bash
run: mv -v ./dist/*.whl ./docker-context-files
- name: "Download constraints from the CI build"
uses: actions/download-artifact@v3
with:
name: constraints
path: ./docker-context-files
if: env.UPGRADE_TO_NEWER_DEPENDENCIES != 'false'
- name: "Build & Push PROD images ${{ env.IMAGE_TAG }}:${{ env.PYTHON_VERSIONS }}"
shell: bash
run: >
breeze prod-image build --tag-as-latest --run-in-parallel --push
--install-packages-from-context --upgrade-on-failure
--install-packages-from-context --airflow-constraints-mode constraints-source-providers
--use-constraints-for-context-packages
env:
COMMIT_SHA: ${{ github.sha }}
- name: "Fix ownership"
Expand Down
52 changes: 42 additions & 10 deletions Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -593,17 +593,42 @@ function install_airflow_and_providers_from_docker_context_files(){
return
fi

echo
echo "${COLOR_BLUE}Force re-installing airflow and providers from local files with eager upgrade${COLOR_RESET}"
echo
# force reinstall all airflow + provider package local files with eager upgrade
set -x
pip install "${pip_flags[@]}" --root-user-action ignore --upgrade --upgrade-strategy eager \
${ADDITIONAL_PIP_INSTALL_FLAGS} \
${reinstalling_apache_airflow_package} ${reinstalling_apache_airflow_providers_packages} \
${EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS=}
set +x
if [[ ${USE_CONSTRAINTS_FOR_CONTEXT_PACKAGES=} == "true" ]]; then
local python_version
python_version=$(python -c 'import sys; print(f"{sys.version_info.major}.{sys.version_info.minor}")')
local local_constraints_file=/docker-context-files/constraints-"${python_version}"/${AIRFLOW_CONSTRAINTS_MODE}-"${python_version}".txt

if [[ -f "${local_constraints_file}" ]]; then
echo
echo "${COLOR_BLUE}Installing docker-context-files packages with constraints found in ${local_constraints_file}${COLOR_RESET}"
echo
# force reinstall all airflow + provider packages with constraints found in
set -x
pip install "${pip_flags[@]}" --root-user-action ignore --upgrade \
${ADDITIONAL_PIP_INSTALL_FLAGS} --constraint "${local_constraints_file}" \
${reinstalling_apache_airflow_package} ${reinstalling_apache_airflow_providers_packages}
set +x
else
echo
echo "${COLOR_BLUE}Installing docker-context-files packages with constraints from GitHub${COLOR_RESET}"
echo
set -x
pip install "${pip_flags[@]}" --root-user-action ignore \
${ADDITIONAL_PIP_INSTALL_FLAGS} \
--constraint "${AIRFLOW_CONSTRAINTS_LOCATION}" \
${reinstalling_apache_airflow_package} ${reinstalling_apache_airflow_providers_packages}
set +x
fi
else
echo
echo "${COLOR_BLUE}Installing docker-context-files packages without constraints${COLOR_RESET}"
echo
set -x
pip install "${pip_flags[@]}" --root-user-action ignore \
${ADDITIONAL_PIP_INSTALL_FLAGS} \
${reinstalling_apache_airflow_package} ${reinstalling_apache_airflow_providers_packages}
set +x
fi
common::install_pip_version
pip check
}
Expand Down Expand Up @@ -1280,6 +1305,12 @@ COPY --from=scripts common.sh install_pip_version.sh \
# is installed from docker-context files rather than from PyPI)
ARG INSTALL_PACKAGES_FROM_CONTEXT="false"

# Normally constraints are not used when context packages are build - because we might have packages
# that are conflicting with Airflow constraints, however there are cases when we want to use constraints
# for example in CI builds when we already have source-package constraints - either from github branch or
# from eager-upgraded constraints by the CI builds
ARG USE_CONSTRAINTS_FOR_CONTEXT_PACKAGES="false"

# In case of Production build image segment we want to pre-install main version of airflow
# dependencies from GitHub so that we do not have to always reinstall it from the scratch.
# The Airflow (and providers in case INSTALL_PROVIDERS_FROM_SOURCES is "false")
Expand All @@ -1304,6 +1335,7 @@ ARG VERSION_SUFFIX_FOR_PYPI=""

ENV ADDITIONAL_PYTHON_DEPS=${ADDITIONAL_PYTHON_DEPS} \
INSTALL_PACKAGES_FROM_CONTEXT=${INSTALL_PACKAGES_FROM_CONTEXT} \
USE_CONSTRAINTS_FOR_CONTEXT_PACKAGES=${USE_CONSTRAINTS_FOR_CONTEXT_PACKAGES} \
VERSION_SUFFIX_FOR_PYPI=${VERSION_SUFFIX_FOR_PYPI}

WORKDIR ${AIRFLOW_HOME}
Expand Down
2 changes: 2 additions & 0 deletions dev/breeze/src/airflow_breeze/commands/ci_image_commands.py
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
option_airflow_constraints_mode_ci,
option_airflow_constraints_reference_build,
option_answer,
option_build_progress,
option_build_timeout_minutes,
option_builder,
option_commit_sha,
Expand Down Expand Up @@ -226,6 +227,7 @@ def kill_process_group(build_process_group_id: int):
@option_additional_dev_apt_command
@option_additional_dev_apt_env
@option_builder
@option_build_progress
@option_build_timeout_minutes
@option_commit_sha
@option_dev_apt_command
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
"--tag-as-latest",
"--docker-cache",
"--force-build",
"--build-progress",
],
},
{
Expand Down
41 changes: 16 additions & 25 deletions dev/breeze/src/airflow_breeze/commands/production_image_commands.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
option_airflow_constraints_location,
option_airflow_constraints_mode_prod,
option_airflow_constraints_reference_build,
option_build_progress,
option_builder,
option_commit_sha,
option_debug_resources,
Expand Down Expand Up @@ -67,8 +68,6 @@
option_runtime_apt_deps,
option_skip_cleanup,
option_tag_as_latest,
option_upgrade_on_failure,
option_upgrade_to_newer_dependencies,
option_verbose,
option_verify,
option_version_suffix_for_pypi,
Expand Down Expand Up @@ -156,8 +155,6 @@ def prod_image():
@option_debug_resources
@option_include_success_outputs
@option_python_versions
@option_upgrade_to_newer_dependencies
@option_upgrade_on_failure
@option_platform_multiple
@option_github_token
@option_docker_cache
Expand All @@ -178,6 +175,12 @@ def prod_image():
"Implies --disable-airflow-repo-cache.",
is_flag=True,
)
@click.option(
"--use-constraints-for-context-packages",
help="Uses constraints for context packages installation - "
"either from constraints store in docker-context-files or from github.",
is_flag=True,
)
@click.option(
"--cleanup-context",
help="Clean up docker context files before running build (cannot be used together"
Expand Down Expand Up @@ -213,6 +216,7 @@ def prod_image():
@option_additional_runtime_apt_env
@option_additional_runtime_apt_command
@option_builder
@option_build_progress
@option_dev_apt_command
@option_dev_apt_deps
@option_python_image
Expand Down Expand Up @@ -434,7 +438,12 @@ def check_docker_context_files(install_packages_from_context: bool):
:param install_packages_from_context: whether we want to install from docker-context-files
"""
context_file = DOCKER_CONTEXT_DIR.rglob("*")
any_context_files = any(context.is_file() and context.name != ".README.md" for context in context_file)
any_context_files = any(
context.is_file()
and context.name not in (".README.md", ".DS_Store")
and not context.parent.name.startswith("constraints")
for context in context_file
)
if not any_context_files and install_packages_from_context:
get_console().print("[warning]\nERROR! You want to install packages from docker-context-files")
get_console().print("[warning]\n but there are no packages to install in this folder.")
Expand Down Expand Up @@ -501,24 +510,6 @@ def run_build_production_image(
text=True,
output=output,
)
if (
build_command_result.returncode != 0
and prod_image_params.upgrade_on_failure
and not prod_image_params.upgrade_to_newer_dependencies
):
prod_image_params.upgrade_to_newer_dependencies = True
get_console().print("[warning]Attempting to build with upgrade_to_newer_dependencies on failure")
build_command_result = run_command(
prepare_docker_build_command(
image_params=prod_image_params,
),
cwd=AIRFLOW_SOURCES_ROOT,
check=False,
text=True,
env=env,
output=output,
)
if build_command_result.returncode == 0:
if prod_image_params.tag_as_latest:
build_command_result = tag_image_as_latest(image_params=prod_image_params, output=output)
if build_command_result.returncode == 0 and prod_image_params.tag_as_latest:
build_command_result = tag_image_as_latest(image_params=prod_image_params, output=output)
return build_command_result.returncode, f"Image build: {prod_image_params.python}"
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,10 @@
"options": [
"--python",
"--install-airflow-version",
"--upgrade-to-newer-dependencies",
"--upgrade-on-failure",
"--image-tag",
"--tag-as-latest",
"--docker-cache",
"--build-progress",
],
},
{
Expand Down Expand Up @@ -79,6 +78,7 @@
"name": "Customization options (for specific customization needs)",
"options": [
"--install-packages-from-context",
"--use-constraints-for-context-packages",
"--cleanup-context",
"--disable-mysql-client-installation",
"--disable-mssql-client-installation",
Expand Down
1 change: 1 addition & 0 deletions dev/breeze/src/airflow_breeze/global_constants.py
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ def all_helm_test_packages() -> list[str]:
ALLOWED_INSTALLATION_PACKAGE_FORMATS = ["wheel", "sdist"]
ALLOWED_INSTALLATION_METHODS = [".", "apache-airflow"]
ALLOWED_BUILD_CACHE = ["registry", "local", "disabled"]
ALLOWED_BUILD_PROGRESS = ["auto", "plain", "tty"]
MULTI_PLATFORM = "linux/amd64,linux/arm64"
SINGLE_PLATFORMS = ["linux/amd64", "linux/arm64"]
ALLOWED_PLATFORMS = [*SINGLE_PLATFORMS, MULTI_PLATFORM]
Expand Down
5 changes: 4 additions & 1 deletion dev/breeze/src/airflow_breeze/params/build_ci_params.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@ class BuildCiParams(CommonBuildParams):
airflow_extras: str = "devel_ci"
airflow_pre_cached_pip_packages: bool = True
force_build: bool = False
upgrade_to_newer_dependencies: bool = False
upgrade_on_failure: bool = False
eager_upgrade_additional_requirements: str = ""
skip_provider_dependencies_check: bool = False

Expand Down Expand Up @@ -66,7 +68,7 @@ def extra_docker_build_flags(self) -> list[str]:
f"EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS={eager_upgrade_arg}",
]
)
return extra_ci_flags
return super().extra_docker_build_flags + extra_ci_flags

@property
def md5sum_cache_dir(self) -> Path:
Expand Down Expand Up @@ -111,6 +113,7 @@ def optional_image_args(self) -> list[str]:
"additional_python_deps",
"version_suffix_for_pypi",
"commit_sha",
"build_progress",
]

def __post_init__(self):
Expand Down
7 changes: 4 additions & 3 deletions dev/breeze/src/airflow_breeze/params/build_prod_params.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ class BuildProdParams(CommonBuildParams):
install_airflow_reference: str = ""
install_airflow_version: str = ""
install_packages_from_context: bool = False
use_constraints_for_context_packages: bool = False
installation_method: str = "."
runtime_apt_command: str = ""
runtime_apt_deps: str = ""
Expand Down Expand Up @@ -159,7 +160,6 @@ def extra_docker_build_flags(self) -> list[str]:
f"AIRFLOW_CONSTRAINTS_REFERENCE={self.airflow_constraints_reference}",
]
)

maintainers = json.dumps([{"name": "Apache Airflow PMC", "email": "[email protected]"}])
logo_url = "https://github.com/apache/airflow/raw/main/docs/apache-airflow/img/logos/wordmark_1.png"
readme_url = "https://raw.githubusercontent.com/apache/airflow/main/docs/docker-stack/README.md"
Expand All @@ -175,7 +175,7 @@ def extra_docker_build_flags(self) -> list[str]:
f"io.artifacthub.package.logo-url={logo_url}",
]
)
return extra_build_flags
return super().extra_docker_build_flags + extra_build_flags

@property
def airflow_pre_cached_pip_packages(self) -> str:
Expand Down Expand Up @@ -221,7 +221,6 @@ def required_image_args(self) -> list[str]:
"install_postgres_client",
"install_providers_from_sources",
"python_base_image",
"upgrade_to_newer_dependencies",
]

@property
Expand All @@ -242,4 +241,6 @@ def optional_image_args(self) -> list[str]:
"runtime_apt_deps",
"version_suffix_for_pypi",
"commit_sha",
"build_progress",
"use_constraints_for_context_packages",
]
14 changes: 10 additions & 4 deletions dev/breeze/src/airflow_breeze/params/common_build_params.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,11 @@
from datetime import datetime

from airflow_breeze.branch_defaults import AIRFLOW_BRANCH, DEFAULT_AIRFLOW_CONSTRAINTS_BRANCH
from airflow_breeze.global_constants import APACHE_AIRFLOW_GITHUB_REPOSITORY, DOCKER_DEFAULT_PLATFORM
from airflow_breeze.global_constants import (
ALLOWED_BUILD_PROGRESS,
APACHE_AIRFLOW_GITHUB_REPOSITORY,
DOCKER_DEFAULT_PLATFORM,
)
from airflow_breeze.utils.console import get_console
from airflow_breeze.utils.platforms import get_real_platform

Expand All @@ -46,6 +50,7 @@ class CommonBuildParams:
airflow_constraints_location: str = ""
build_id: int = 0
builder: str = "autodetect"
build_progress: str = ALLOWED_BUILD_PROGRESS[0]
constraints_github_repository: str = APACHE_AIRFLOW_GITHUB_REPOSITORY
commit_sha: str = ""
dev_apt_command: str = ""
Expand All @@ -62,8 +67,6 @@ class CommonBuildParams:
push: bool = False
python: str = "3.8"
tag_as_latest: bool = False
upgrade_to_newer_dependencies: bool = False
upgrade_on_failure: bool = False
dry_run: bool = False
version_suffix_for_pypi: str = ""
verbose: bool = False
Expand Down Expand Up @@ -96,7 +99,10 @@ def airflow_image_name(self):

@property
def extra_docker_build_flags(self) -> list[str]:
raise NotImplementedError()
extra_flass = []
if self.build_progress:
extra_flass.append(f"--progress={self.build_progress}")
return extra_flass

@property
def docker_cache_directive(self) -> list[str]:
Expand Down
9 changes: 9 additions & 0 deletions dev/breeze/src/airflow_breeze/utils/common_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
ALL_HISTORICAL_PYTHON_VERSIONS,
ALLOWED_BACKENDS,
ALLOWED_BUILD_CACHE,
ALLOWED_BUILD_PROGRESS,
ALLOWED_CELERY_BROKERS,
ALLOWED_CONSTRAINTS_MODES_CI,
ALLOWED_CONSTRAINTS_MODES_PROD,
Expand Down Expand Up @@ -511,6 +512,14 @@ def _set_default_from_parent(ctx: click.core.Context, option: click.core.Option,
show_default=True,
default="autodetect",
)
option_build_progress = click.option(
"--build-progress",
help="Build progress.",
type=BetterChoice(ALLOWED_BUILD_PROGRESS),
envvar="BUILD_PROGRESS",
show_default=True,
default=ALLOWED_BUILD_PROGRESS[0],
)
option_include_success_outputs = click.option(
"--include-success-outputs",
help="Whether to include outputs of successful parallel runs (skipped by default).",
Expand Down
Loading

0 comments on commit 2b1a194

Please sign in to comment.