Merge pull request #41088 from ClickHouse/darwin-releases

Add macos binaries to release assets
This commit is contained in:
Mikhail f. Shiryaev 2022-09-09 17:17:15 +02:00 committed by GitHub
commit b815fdbc8c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 377 additions and 31 deletions

View File

@ -349,6 +349,100 @@ jobs:
# shellcheck disable=SC2046 # shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||: docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinDarwin:
needs: [DockerHubPush]
runs-on: [self-hosted, builder]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/build_check
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_darwin
EOF
- name: Download changed images
uses: actions/download-artifact@v2
with:
name: changed_images
path: ${{ env.IMAGES_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
with:
fetch-depth: 0 # otherwise we will have no info about contributors
- name: Build
run: |
git -C "$GITHUB_WORKSPACE" submodule sync --recursive
git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME"
- name: Upload build URLs to artifacts
if: ${{ success() || failure() }}
uses: actions/upload-artifact@v2
with:
name: ${{ env.BUILD_URLS }}
path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinDarwinAarch64:
needs: [DockerHubPush]
runs-on: [self-hosted, builder]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/build_check
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_darwin_aarch64
EOF
- name: Download changed images
uses: actions/download-artifact@v2
with:
name: changed_images
path: ${{ env.IMAGES_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
with:
fetch-depth: 0 # otherwise we will have no info about contributors
- name: Build
run: |
git -C "$GITHUB_WORKSPACE" submodule sync --recursive
git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME"
- name: Upload build URLs to artifacts
if: ${{ success() || failure() }}
uses: actions/upload-artifact@v2
with:
name: ${{ env.BUILD_URLS }}
path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
############################################################################################ ############################################################################################
##################################### Docker images ####################################### ##################################### Docker images #######################################
############################################################################################ ############################################################################################
@ -425,6 +519,46 @@ jobs:
# shellcheck disable=SC2046 # shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||: docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" sudo rm -fr "$TEMP_PATH"
BuilderSpecialReport:
needs:
- BuilderBinDarwin
- BuilderBinDarwinAarch64
runs-on: [self-hosted, style-checker]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse special build check
NEEDS_DATA_PATH=${{runner.temp}}/needs.json
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Report Builder
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cat > "$NEEDS_DATA_PATH" << 'EOF'
${{ toJSON(needs) }}
EOF
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH"
############################################################################################## ##############################################################################################
########################### FUNCTIONAl STATELESS TESTS ####################################### ########################### FUNCTIONAl STATELESS TESTS #######################################
############################################################################################## ##############################################################################################
@ -592,6 +726,7 @@ jobs:
- DockerHubPush - DockerHubPush
- DockerServerImages - DockerServerImages
- BuilderReport - BuilderReport
- BuilderSpecialReport
- FunctionalStatelessTestAsan - FunctionalStatelessTestAsan
- FunctionalStatefulTestDebug - FunctionalStatefulTestDebug
- StressTestTsan - StressTestTsan

View File

@ -29,8 +29,12 @@ jobs:
rm -rf "$TEMP_PATH" && mkdir -p "$TEMP_PATH" rm -rf "$TEMP_PATH" && mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY" cd "$REPO_COPY"
# Download and push packages to artifactory
python3 ./tests/ci/push_to_artifactory.py --release "${{ github.ref }}" \ python3 ./tests/ci/push_to_artifactory.py --release "${{ github.ref }}" \
--commit '${{ github.sha }}' --artifactory-url "${{ secrets.JFROG_ARTIFACTORY_URL }}" --all --commit '${{ github.sha }}' --artifactory-url "${{ secrets.JFROG_ARTIFACTORY_URL }}" --all
# Download macos binaries to ${{runner.temp}}/download_binary
python3 ./tests/ci/download_binary.py binary_darwin binary_darwin_aarch64
mv '${{runner.temp}}/download_binary/'clickhouse-* '${{runner.temp}}/push_to_artifactory'
- name: Upload packages to release assets - name: Upload packages to release assets
uses: svenstaro/upload-release-action@v2 uses: svenstaro/upload-release-action@v2
with: with:

View File

@ -426,6 +426,100 @@ jobs:
# shellcheck disable=SC2046 # shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||: docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinDarwin:
needs: [DockerHubPush]
runs-on: [self-hosted, builder]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/build_check
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_darwin
EOF
- name: Download changed images
uses: actions/download-artifact@v2
with:
name: changed_images
path: ${{ env.IMAGES_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
with:
fetch-depth: 0 # otherwise we will have no info about contributors
- name: Build
run: |
git -C "$GITHUB_WORKSPACE" submodule sync --recursive
git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME"
- name: Upload build URLs to artifacts
if: ${{ success() || failure() }}
uses: actions/upload-artifact@v2
with:
name: ${{ env.BUILD_URLS }}
path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
BuilderBinDarwinAarch64:
needs: [DockerHubPush]
runs-on: [self-hosted, builder]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/build_check
IMAGES_PATH=${{runner.temp}}/images_path
REPO_COPY=${{runner.temp}}/build_check/ClickHouse
CACHES_PATH=${{runner.temp}}/../ccaches
BUILD_NAME=binary_darwin_aarch64
EOF
- name: Download changed images
uses: actions/download-artifact@v2
with:
name: changed_images
path: ${{ env.IMAGES_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
with:
fetch-depth: 0 # otherwise we will have no info about contributors
- name: Build
run: |
git -C "$GITHUB_WORKSPACE" submodule sync --recursive
git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME"
- name: Upload build URLs to artifacts
if: ${{ success() || failure() }}
uses: actions/upload-artifact@v2
with:
name: ${{ env.BUILD_URLS }}
path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
############################################################################################ ############################################################################################
##################################### Docker images ####################################### ##################################### Docker images #######################################
############################################################################################ ############################################################################################
@ -505,6 +599,46 @@ jobs:
# shellcheck disable=SC2046 # shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||: docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH" sudo rm -fr "$TEMP_PATH"
BuilderSpecialReport:
needs:
- BuilderBinDarwin
- BuilderBinDarwinAarch64
runs-on: [self-hosted, style-checker]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/report_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=ClickHouse special build check
NEEDS_DATA_PATH=${{runner.temp}}/needs.json
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Report Builder
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cat > "$NEEDS_DATA_PATH" << 'EOF'
${{ toJSON(needs) }}
EOF
cd "$GITHUB_WORKSPACE/tests/ci"
python3 build_report_check.py "$CHECK_NAME"
- name: Cleanup
if: always()
run: |
# shellcheck disable=SC2046
docker kill $(docker ps -q) ||:
# shellcheck disable=SC2046
docker rm -f $(docker ps -a -q) ||:
sudo rm -fr "$TEMP_PATH"
############################################################################################## ##############################################################################################
########################### FUNCTIONAl STATELESS TESTS ####################################### ########################### FUNCTIONAl STATELESS TESTS #######################################
############################################################################################## ##############################################################################################
@ -1847,6 +1981,7 @@ jobs:
- DockerHubPush - DockerHubPush
- DockerServerImages - DockerServerImages
- BuilderReport - BuilderReport
- BuilderSpecialReport
- FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug0
- FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug1
- FunctionalStatelessTestDebug2 - FunctionalStatelessTestDebug2

View File

@ -17,7 +17,7 @@ from env_helper import (
from s3_helper import S3Helper from s3_helper import S3Helper
from get_robot_token import get_best_robot_token from get_robot_token import get_best_robot_token
from pr_info import PRInfo from pr_info import PRInfo
from build_download_helper import get_build_name_for_check, get_build_urls from build_download_helper import get_build_name_for_check, read_build_urls
from docker_pull_helper import get_image_with_version from docker_pull_helper import get_image_with_version
from commit_status_helper import post_commit_status from commit_status_helper import post_commit_status
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
@ -69,7 +69,7 @@ if __name__ == "__main__":
build_name = get_build_name_for_check(check_name) build_name = get_build_name_for_check(check_name)
print(build_name) print(build_name)
urls = get_build_urls(build_name, reports_path) urls = read_build_urls(build_name, reports_path)
if not urls: if not urls:
raise Exception("No build URLs found") raise Exception("No build URLs found")

View File

@ -1,11 +1,11 @@
#!/usr/bin/env python3 #!/usr/bin/env python3
import os
import json import json
import logging import logging
import os
import sys import sys
import time import time
from typing import Optional from typing import List, Optional
import requests # type: ignore import requests # type: ignore
@ -41,11 +41,11 @@ def get_with_retries(
return response return response
def get_build_name_for_check(check_name): def get_build_name_for_check(check_name) -> str:
return CI_CONFIG["tests_config"][check_name]["required_build"] return CI_CONFIG["tests_config"][check_name]["required_build"]
def get_build_urls(build_name, reports_path): def read_build_urls(build_name, reports_path) -> List[str]:
for root, _, files in os.walk(reports_path): for root, _, files in os.walk(reports_path):
for f in files: for f in files:
if build_name in f: if build_name in f:
@ -56,7 +56,7 @@ def get_build_urls(build_name, reports_path):
return [] return []
def dowload_build_with_progress(url, path): def download_build_with_progress(url, path):
logging.info("Downloading from %s to temp path %s", url, path) logging.info("Downloading from %s to temp path %s", url, path)
for i in range(DOWNLOAD_RETRIES_COUNT): for i in range(DOWNLOAD_RETRIES_COUNT):
try: try:
@ -104,14 +104,14 @@ def download_builds(result_path, build_urls, filter_fn):
if filter_fn(url): if filter_fn(url):
fname = os.path.basename(url.replace("%2B", "+").replace("%20", " ")) fname = os.path.basename(url.replace("%2B", "+").replace("%20", " "))
logging.info("Will download %s to %s", fname, result_path) logging.info("Will download %s to %s", fname, result_path)
dowload_build_with_progress(url, os.path.join(result_path, fname)) download_build_with_progress(url, os.path.join(result_path, fname))
def download_builds_filter( def download_builds_filter(
check_name, reports_path, result_path, filter_fn=lambda _: True check_name, reports_path, result_path, filter_fn=lambda _: True
): ):
build_name = get_build_name_for_check(check_name) build_name = get_build_name_for_check(check_name)
urls = get_build_urls(build_name, reports_path) urls = read_build_urls(build_name, reports_path)
print(urls) print(urls)
if not urls: if not urls:

79
tests/ci/download_binary.py Executable file
View File

@ -0,0 +1,79 @@
#!/usr/bin/env python
"""
This file is needed to avoid cicle import build_download_helper.py <=> env_helper.py
"""
import argparse
import logging
import os
from build_download_helper import download_build_with_progress
from ci_config import CI_CONFIG, BuildConfig
from env_helper import RUNNER_TEMP, S3_ARTIFACT_DOWNLOAD_TEMPLATE
from git_helper import Git, commit
from version_helper import get_version_from_repo, version_arg
TEMP_PATH = os.path.join(RUNNER_TEMP, "download_binary")
def parse_args() -> argparse.Namespace:
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Script to download binary artifacts from S3. Downloaded artifacts "
"are renamed to clickhouse-{static_binary_name}",
)
parser.add_argument(
"--version",
type=version_arg,
default=get_version_from_repo().string,
help="a version to generate a download url, get from the repo by default",
)
parser.add_argument(
"--commit",
type=commit,
default=Git(True).sha,
help="a version to generate a download url, get from the repo by default",
)
parser.add_argument("--rename", default=True, help=argparse.SUPPRESS)
parser.add_argument(
"--no-rename",
dest="rename",
action="store_false",
default=argparse.SUPPRESS,
help="if set, the downloaded binary won't be renamed to "
"clickhouse-{static_binary_name}, makes sense only for a single build name",
)
parser.add_argument(
"build_names",
nargs="+",
help="the build names to download",
)
args = parser.parse_args()
if not args.rename and len(args.build_names) > 1:
parser.error("`--no-rename` shouldn't be used with more than one build name")
return args
def main():
logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
args = parse_args()
os.makedirs(TEMP_PATH, exist_ok=True)
for build in args.build_names:
# check if it's in CI_CONFIG
config = CI_CONFIG["build_config"][build] # type: BuildConfig
if args.rename:
path = os.path.join(TEMP_PATH, f"clickhouse-{config['static_binary_name']}")
else:
path = os.path.join(TEMP_PATH, "clickhouse")
url = S3_ARTIFACT_DOWNLOAD_TEMPLATE.format(
pr_or_release=f"{args.version.major}.{args.version.minor}",
commit=args.commit,
build_name=build,
artifact="clickhouse",
)
download_build_with_progress(url, path)
if __name__ == "__main__":
main()

View File

@ -22,10 +22,14 @@ IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH)
REPORTS_PATH = os.getenv("REPORTS_PATH", p.abspath(p.join(module_dir, "./reports"))) REPORTS_PATH = os.getenv("REPORTS_PATH", p.abspath(p.join(module_dir, "./reports")))
REPO_COPY = os.getenv("REPO_COPY", git_root) REPO_COPY = os.getenv("REPO_COPY", git_root)
RUNNER_TEMP = os.getenv("RUNNER_TEMP", p.abspath(p.join(module_dir, "./tmp"))) RUNNER_TEMP = os.getenv("RUNNER_TEMP", p.abspath(p.join(module_dir, "./tmp")))
S3_URL = os.getenv("S3_URL", "https://s3.amazonaws.com")
S3_DOWNLOAD = os.getenv("S3_DOWNLOAD", S3_URL)
S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds")
S3_TEST_REPORTS_BUCKET = os.getenv("S3_TEST_REPORTS_BUCKET", "clickhouse-test-reports") S3_TEST_REPORTS_BUCKET = os.getenv("S3_TEST_REPORTS_BUCKET", "clickhouse-test-reports")
S3_URL = os.getenv("S3_URL", "https://s3.amazonaws.com")
S3_DOWNLOAD = os.getenv("S3_DOWNLOAD", S3_URL)
S3_ARTIFACT_DOWNLOAD_TEMPLATE = (
f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/"
"{pr_or_release}/{commit}/{build_name}/{artifact}"
)
# These parameters are set only on demand, and only once # These parameters are set only on demand, and only once
_GITHUB_JOB_ID = "" _GITHUB_JOB_ID = ""

View File

@ -8,8 +8,8 @@ from collections import namedtuple
from typing import Dict, List, Tuple from typing import Dict, List, Tuple
from artifactory import ArtifactorySaaSPath # type: ignore from artifactory import ArtifactorySaaSPath # type: ignore
from build_download_helper import dowload_build_with_progress from build_download_helper import download_build_with_progress
from env_helper import RUNNER_TEMP, S3_BUILDS_BUCKET, S3_DOWNLOAD from env_helper import S3_ARTIFACT_DOWNLOAD_TEMPLATE, RUNNER_TEMP
from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix
@ -97,18 +97,6 @@ class Packages:
class S3: class S3:
template = (
f"{S3_DOWNLOAD}/"
# "clickhouse-builds/"
f"{S3_BUILDS_BUCKET}/"
# "33333/" or "21.11/" from --release, if pull request is omitted
"{pr}/"
# "2bef313f75e4cacc6ea2ef2133e8849ecf0385ec/"
"{commit}/"
# "package_release/clickhouse-common-static_21.11.5.0_amd64.deb"
"{s3_path_suffix}"
)
def __init__( def __init__(
self, self,
pr: int, pr: int,
@ -117,7 +105,7 @@ class S3:
force_download: bool, force_download: bool,
): ):
self._common = dict( self._common = dict(
pr=pr, pr_or_release=pr,
commit=commit, commit=commit,
) )
self.force_download = force_download self.force_download = force_download
@ -133,18 +121,19 @@ class S3:
self.packages.replace_with_fallback(package_file) self.packages.replace_with_fallback(package_file)
return return
url = self.template.format_map( build_name, artifact = s3_path_suffix.split("/")
{**self._common, "s3_path_suffix": s3_path_suffix} url = S3_ARTIFACT_DOWNLOAD_TEMPLATE.format_map(
{**self._common, "build_name": build_name, "artifact": artifact}
) )
try: try:
dowload_build_with_progress(url, path) download_build_with_progress(url, path)
except Exception as e: except Exception as e:
if "Cannot download dataset from" in e.args[0]: if "Cannot download dataset from" in e.args[0]:
new_url = Packages.fallback_to_all(url) new_url = Packages.fallback_to_all(url)
logging.warning( logging.warning(
"Fallback downloading %s for old release", fallback_path "Fallback downloading %s for old release", fallback_path
) )
dowload_build_with_progress(new_url, fallback_path) download_build_with_progress(new_url, fallback_path)
self.packages.replace_with_fallback(package_file) self.packages.replace_with_fallback(package_file)
def download_deb(self): def download_deb(self):

View File

@ -20,7 +20,7 @@ const char * auto_contributors[] {{
VERSIONS = Dict[str, Union[int, str]] VERSIONS = Dict[str, Union[int, str]]
VERSIONS_TEMPLATE = """# This variables autochanged by release_lib.sh: VERSIONS_TEMPLATE = """# This variables autochanged by tests/ci/version_helper.py:
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.