Merge pull request #58881 from ClickHouse/jepsen_to_reusable_builds

Jepsen job to reuse builds
This commit is contained in:
Max K 2024-01-18 13:47:14 +01:00 committed by GitHub
commit a4cab68a06
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 64 additions and 36 deletions

View File

@ -8,7 +8,6 @@ on: # yamllint disable-line rule:truthy
schedule:
- cron: '0 */6 * * *'
workflow_dispatch:
workflow_call:
jobs:
KeeperJepsenRelease:
uses: ./.github/workflows/reusable_simple_job.yml

View File

@ -966,13 +966,20 @@ jobs:
#############################################################################################
###################################### JEPSEN TESTS #########################################
#############################################################################################
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
Jepsen:
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
if: ${{ !failure() && !cancelled() && contains(github.event.pull_request.labels.*.name, 'jepsen-test') }}
# we need concurrency as the job uses dedicated instances in the cloud
concurrency:
group: jepsen
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuilderBinRelease]
uses: ./.github/workflows/jepsen.yml
uses: ./.github/workflows/reusable_test.yml
with:
test_name: ClickHouse Keeper Jepsen
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
#############################################################################################
####################################### libFuzzer ###########################################
#############################################################################################

View File

@ -825,10 +825,18 @@ CI_CONFIG = CiConfig(
"package_asan",
job_config=JobConfig(**{**statless_test_common_params, "timeout": 3600}), # type: ignore
),
# FIXME: add digest and params
"ClickHouse Keeper Jepsen": TestConfig("binary_release"),
# FIXME: add digest and params
"ClickHouse Server Jepsen": TestConfig("binary_release"),
"ClickHouse Keeper Jepsen": TestConfig(
"binary_release",
job_config=JobConfig(
run_by_label="jepsen-test", run_command="jepsen_check.py keeper"
),
),
"ClickHouse Server Jepsen": TestConfig(
"binary_release",
job_config=JobConfig(
run_by_label="jepsen-test", run_command="jepsen_check.py server"
),
),
"Performance Comparison": TestConfig(
"package_release",
job_config=JobConfig(num_batches=4, **perf_test_common_params), # type: ignore

View File

@ -10,14 +10,18 @@ from pathlib import Path
from typing import Any, List
import boto3 # type: ignore
import requests # type: ignore
from github import Github
from build_download_helper import get_build_name_for_check
import requests # type: ignore
from git_helper import git_runner
from build_download_helper import (
download_build_with_progress,
get_build_name_for_check,
read_build_urls,
)
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from commit_status_helper import RerunHelper, get_commit, post_commit_status
from compress_files import compress_fast
from env_helper import REPO_COPY, TEMP_PATH, S3_BUILDS_BUCKET, S3_DOWNLOAD
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, S3_URL, TEMP_PATH
from get_robot_token import get_best_robot_token, get_parameter_from_ssm
from pr_info import PRInfo
from report import TestResults, TestResult
@ -26,8 +30,6 @@ from ssh import SSHKey
from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
from version_helper import get_version_from_repo
from build_check import get_release_or_pr
JEPSEN_GROUP_NAME = "jepsen_group"
@ -212,27 +214,36 @@ def main():
# always use latest
docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME
build_name = get_build_name_for_check(check_name)
release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo())
# This check run separately from other checks because it requires exclusive
# run (see .github/workflows/jepsen.yml) So we cannot add explicit
# dependency on a build job and using busy loop on it's results. For the
# same reason we are using latest docker image.
build_url = (
f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/{release_or_pr}/{pr_info.sha}/"
f"{build_name}/clickhouse"
)
head = requests.head(build_url)
counter = 0
while head.status_code != 200:
time.sleep(10)
if pr_info.schedule:
# get latest clcikhouse by the static link for latest master buit - get its version and provide permanent url for this version to the jepsen
build_url = f"{S3_URL}/{S3_BUILDS_BUCKET}/master/amd64/clickhouse"
download_build_with_progress(build_url, Path(TEMP_PATH) / "clickhouse")
git_runner.run(f"chmod +x {TEMP_PATH}/clickhouse")
sha = git_runner.run(
f"{TEMP_PATH}/clickhouse local -q \"select value from system.build_options where name='GIT_HASH'\""
)
version_full = git_runner.run(
f'{TEMP_PATH}/clickhouse local -q "select version()"'
)
version = ".".join(version_full.split(".")[0:2])
assert len(sha) == 40, f"failed to fetch sha from the binary. result: {sha}"
assert (
version
), f"failed to fetch version from the binary. result: {version_full}"
build_url = (
f"{S3_URL}/{S3_BUILDS_BUCKET}/{version}/{sha}/binary_release/clickhouse"
)
print(f"Clickhouse version: [{version_full}], sha: [{sha}], url: [{build_url}]")
head = requests.head(build_url)
counter += 1
if counter >= 180:
logging.warning("Cannot fetch build in 30 minutes, exiting")
sys.exit(0)
assert head.status_code == 200, f"Clickhouse binary not found: {build_url}"
else:
build_name = get_build_name_for_check(check_name)
urls = read_build_urls(build_name, REPORT_PATH)
build_url = None
for url in urls:
if url.endswith("clickhouse"):
build_url = url
assert build_url, "No build url found in the report"
extra_args = ""
if args.program == "server":

View File

@ -99,6 +99,7 @@ class PRInfo:
# release_pr and merged_pr are used for docker images additional cache
self.release_pr = 0
self.merged_pr = 0
self.schedule = False
ref = github_event.get("ref", "refs/heads/master")
if ref and ref.startswith("refs/heads/"):
ref = ref[11:]
@ -243,6 +244,8 @@ class PRInfo:
)
)
else:
if "schedule" in github_event:
self.schedule = True
print("event.json does not match pull_request or push:")
print(json.dumps(github_event, sort_keys=True, indent=4))
self.sha = os.getenv(