CI: Jepsen Workflow with CI buddy

This commit is contained in:
Max K 2024-07-24 19:51:34 +02:00
parent 8d3568ce71
commit 9de43325e4
8 changed files with 159 additions and 69 deletions

View File

@ -9,19 +9,64 @@ on: # yamllint disable-line rule:truthy
- cron: '0 */6 * * *'
workflow_dispatch:
jobs:
RunConfig:
runs-on: [self-hosted, style-checker-aarch64]
outputs:
data: ${{ steps.runconfig.outputs.CI_DATA }}
steps:
- name: DebugInfo
uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: PrepareRunConfig
id: runconfig
run: |
echo "::group::configure CI run"
python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --configure --workflow "$GITHUB_WORKFLOW" --outfile ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
echo "::group::CI run configure results"
python3 -m json.tool ${{ runner.temp }}/ci_run_data.json
echo "::endgroup::"
{
echo 'CI_DATA<<EOF'
cat ${{ runner.temp }}/ci_run_data.json
echo 'EOF'
} >> "$GITHUB_OUTPUT"
KeeperJepsenRelease:
uses: ./.github/workflows/reusable_simple_job.yml
needs: [RunConfig]
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Jepsen keeper check
runner_type: style-checker
report_required: true
test_name: ClickHouse Keeper Jepsen
runner_type: style-checker-aarch64
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 jepsen_check.py keeper
# ServerJepsenRelease:
# uses: ./.github/workflows/reusable_simple_job.yml
# with:
# test_name: Jepsen server check
# runner_type: style-checker
# run_command: |
# cd "$REPO_COPY/tests/ci"
# python3 jepsen_check.py server
ServerJepsenRelease:
if: false # skip for server
needs: [RunConfig]
uses: ./.github/workflows/reusable_test.yml
with:
test_name: ClickHouse Server Jepsen
runner_type: style-checker-aarch64
data: ${{ needs.RunConfig.outputs.data }}
run_command: |
python3 jepsen_check.py server
CheckWorkflow:
if: ${{ !cancelled() }}
needs: [RunConfig, ServerJepsenRelease, KeeperJepsenRelease]
runs-on: [self-hosted, style-checker-aarch64]
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
- name: Check Workflow results
run: |
export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json"
cat >> "$WORKFLOW_RESULT_FILE" << 'EOF'
${{ toJson(needs) }}
EOF
python3 ./tests/ci/ci_buddy.py --check-wf-status

View File

@ -95,6 +95,12 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace:
action="store_true",
help="Action that configures ci run. Calculates digests, checks job to be executed, generates json output",
)
parser.add_argument(
"--workflow",
default="",
type=str,
help="Workflow Name, to be provided with --configure for workflow-specific CI runs",
)
parser.add_argument(
"--update-gh-statuses",
action="store_true",
@ -287,7 +293,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info):
# for release/master branches reports must be from the same branch
report_prefix = ""
if pr_info.is_master or pr_info.is_release:
report_prefix = normalize_string(pr_info.head_ref)
# do not set report prefix for scheduled or dispatched wf (in case it started from feature branch while
# testing), otherwise reports won't be found
if not (pr_info.is_scheduled or pr_info.is_dispatched):
report_prefix = normalize_string(pr_info.head_ref)
print(
f"Use report prefix [{report_prefix}], pr_num [{pr_info.number}], head_ref [{pr_info.head_ref}]"
)
@ -520,6 +529,7 @@ def _configure_jobs(
pr_info: PRInfo,
ci_settings: CiSettings,
skip_jobs: bool,
workflow_name: str = "",
dry_run: bool = False,
) -> CiCache:
"""
@ -537,18 +547,27 @@ def _configure_jobs(
is_docs_only=pr_info.has_changes_in_documentation_only(),
is_master=pr_info.is_master,
is_pr=pr_info.is_pr,
workflow_name=workflow_name,
)
else:
job_configs = {}
# filter jobs in accordance with ci settings
job_configs = ci_settings.apply(
job_configs,
pr_info.is_release,
is_pr=pr_info.is_pr,
is_mq=pr_info.is_merge_queue,
labels=pr_info.labels,
)
if not workflow_name:
# filter jobs in accordance with ci settings
job_configs = ci_settings.apply(
job_configs,
pr_info.is_release,
is_pr=pr_info.is_pr,
is_mq=pr_info.is_merge_queue,
labels=pr_info.labels,
)
# add all job batches to job's to_do batches
for _job, job_config in job_configs.items():
batches = []
for batch in range(job_config.num_batches):
batches.append(batch)
job_config.batches = batches
# check jobs in ci cache
ci_cache = CiCache.calc_digests_and_create(
@ -1102,6 +1121,7 @@ def main() -> int:
pr_info,
ci_settings,
args.skip_jobs,
args.workflow,
)
ci_cache.print_status()
@ -1111,7 +1131,7 @@ def main() -> int:
if IS_CI and not pr_info.is_merge_queue:
if pr_info.is_release:
if pr_info.is_release and pr_info.is_push_event:
print("Release/master: CI Cache add pending records for all todo jobs")
ci_cache.push_pending_all(pr_info.is_release)

View File

@ -37,11 +37,22 @@ class CI:
from ci_utils import GHActions as GHActions
from ci_definitions import Labels as Labels
from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS
from ci_definitions import WorkFlowNames as WorkFlowNames
from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL
# Jobs that run for doc related updates
_DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK]
WORKFLOW_CONFIGS = {
WorkFlowNames.JEPSEN: LabelConfig(
run_jobs=[
BuildNames.BINARY_RELEASE,
JobNames.JEPSEN_KEEPER,
JobNames.JEPSEN_SERVER,
]
)
} # type: Dict[str, LabelConfig]
TAG_CONFIGS = {
Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]),
Tags.CI_SET_ARM: LabelConfig(
@ -68,7 +79,7 @@ class CI:
JobNames.STATEFUL_TEST_ASAN,
]
),
}
} # type: Dict[str, LabelConfig]
JOB_CONFIGS: Dict[str, JobConfig] = {
BuildNames.PACKAGE_RELEASE: CommonJobConfigs.BUILD.with_properties(
@ -599,16 +610,25 @@ class CI:
@classmethod
def get_workflow_jobs_with_configs(
cls, is_mq: bool, is_docs_only: bool, is_master: bool, is_pr: bool
cls,
is_mq: bool,
is_docs_only: bool,
is_master: bool,
is_pr: bool,
workflow_name: str,
) -> Dict[str, JobConfig]:
"""
get a list of all jobs for a workflow with configs
"""
jobs = []
if is_mq:
jobs = MQ_JOBS
elif is_docs_only:
jobs = cls._DOCS_CHECK_JOBS
elif workflow_name:
assert (
workflow_name in cls.WORKFLOW_CONFIGS
), "Workflow name if provided must be configured in WORKFLOW_CONFIGS"
jobs = list(cls.WORKFLOW_CONFIGS[workflow_name].run_jobs)
else:
# add all jobs
jobs = list(cls.JOB_CONFIGS)

View File

@ -106,6 +106,14 @@ class Tags(metaclass=WithIter):
libFuzzer = "libFuzzer"
class WorkFlowNames(metaclass=WithIter):
"""
CI WorkFlow Names for custom CI runs
"""
JEPSEN = "JepsenWorkflow"
class BuildNames(metaclass=WithIter):
"""
Build' job names

View File

@ -234,11 +234,4 @@ class CiSettings:
for job in add_parents:
res[job] = job_configs[job]
for job, job_config in res.items():
batches = []
for batch in range(job_config.num_batches):
if not self.job_batches or batch in self.job_batches:
batches.append(batch)
job_config.batches = batches
return res

View File

@ -9,16 +9,13 @@ from pathlib import Path
from typing import Any, List
import boto3 # type: ignore
import requests
from build_download_helper import (
download_build_with_progress,
read_build_urls,
)
from compress_files import compress_fast
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, S3_URL, TEMP_PATH
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from get_robot_token import get_parameter_from_ssm
from git_helper import git_runner
from pr_info import PRInfo
from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults
from ssh import SSHKey
@ -32,11 +29,10 @@ KEEPER_DESIRED_INSTANCE_COUNT = 3
SERVER_DESIRED_INSTANCE_COUNT = 4
KEEPER_IMAGE_NAME = "clickhouse/keeper-jepsen-test"
KEEPER_CHECK_NAME = "ClickHouse Keeper Jepsen"
KEEPER_CHECK_NAME = CI.JobNames.JEPSEN_KEEPER
SERVER_IMAGE_NAME = "clickhouse/server-jepsen-test"
SERVER_CHECK_NAME = "ClickHouse Server Jepsen"
SERVER_CHECK_NAME = CI.JobNames.JEPSEN_SERVER
SUCCESSFUL_TESTS_ANCHOR = "# Successful tests"
INTERMINATE_TESTS_ANCHOR = "# Indeterminate tests"
@ -201,36 +197,14 @@ def main():
# always use latest
docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME
if pr_info.is_scheduled or pr_info.is_dispatched:
# get latest clickhouse 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, timeout=60)
assert head.status_code == 200, f"Clickhouse binary not found: {build_url}"
else:
build_name = CI.get_required_build_name(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"
# binary_release assumed to be always ready on the master as it's part of the merge queue workflow
build_name = CI.get_required_build_name(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

@ -337,6 +337,10 @@ class PRInfo:
return True
return False
@property
def is_push_event(self) -> bool:
return self.event_type == EventType.PUSH
@property
def is_scheduled(self) -> bool:
return self.event_type == EventType.SCHEDULE

View File

@ -419,6 +419,32 @@ class TestCIConfig(unittest.TestCase):
]
self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do)
def test_ci_py_for_specific_workflow(self):
"""
checks ci.py job configuration
"""
settings = CiSettings()
settings.no_ci_cache = True
pr_info = PRInfo(github_event=_TEST_EVENT_JSON)
# make it merge_queue
pr_info.event_type = EventType.SCHEDULE
assert pr_info.number == 0 and not pr_info.is_merge_queue and not pr_info.is_pr
ci_cache = CIPY._configure_jobs(
S3Helper(),
pr_info,
settings,
skip_jobs=False,
dry_run=True,
workflow_name=CI.WorkFlowNames.JEPSEN,
)
actual_jobs_to_do = list(ci_cache.jobs_to_do)
expected_jobs_to_do = [
CI.BuildNames.BINARY_RELEASE,
CI.JobNames.JEPSEN_KEEPER,
CI.JobNames.JEPSEN_SERVER,
]
self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do)
def test_ci_py_await(self):
"""
checks ci.py job configuration