Merge pull request #66471 from ClickHouse/ci_skip_all_not_affected_jobs_in_ci

CI: Autoskip all non-affected jobs in PRs
This commit is contained in:
Max K 2024-07-13 17:30:35 +00:00 committed by GitHub
commit bce601f22e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 151 additions and 121 deletions

View File

@ -172,7 +172,7 @@ jobs:
################################# Stage Final #################################
#
FinishCheck:
if: ${{ !failure() }}
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
runs-on: [self-hosted, style-checker-aarch64]
steps:

View File

@ -996,7 +996,7 @@ def main() -> int:
args.skip_jobs,
)
if IS_CI and pr_info.is_pr:
if IS_CI and pr_info.is_pr and not ci_settings.no_ci_cache:
ci_cache.filter_out_not_affected_jobs()
ci_cache.print_status()
@ -1086,6 +1086,16 @@ def main() -> int:
print(status)
print("::endgroup::")
previous_status = status.state
print("Create dummy job report with job_skipped flag")
JobReport(
status=status.state,
description="",
test_results=[],
start_time="",
duration=0.0,
additional_files=[],
job_skipped=True,
).dump()
# ci cache check
if not previous_status and not ci_settings.no_ci_cache:
@ -1136,7 +1146,7 @@ def main() -> int:
has_oom_error = True
job_report = JobReport.load() if JobReport.exist() else None
if job_report:
if job_report and not job_report.job_skipped:
ch_helper = ClickHouseHelper()
check_url = ""
@ -1236,12 +1246,17 @@ def main() -> int:
indata["build"],
ch_helper,
)
elif job_report.job_skipped:
print(f"Skipped after rerun check {[args.job_name]} - do nothing")
else:
if CI.is_test_job(args.job_name):
if has_oom_error:
description = "ERROR: Out Of Memory"
else:
description = "ERROR: Unknown job status"
print(
f"No job report for {[args.job_name]} - post status [{description}]"
)
gh = GitHub(get_best_robot_token(), per_page=100)
commit = get_commit(gh, pr_info.sha)
post_commit_status(
@ -1249,7 +1264,7 @@ def main() -> int:
ERROR,
"",
description,
job_report.check_name or _get_ext_check_name(args.job_name),
_get_ext_check_name(args.job_name),
pr_info,
dump_to_file=True,
)

View File

@ -520,6 +520,35 @@ class CiCache:
self.RecordType.SUCCESSFUL, job, batch, num_batches, release_branch
)
def has_evidence(self, job: str, job_config: CI.JobConfig) -> bool:
"""
checks if the job has been seen in master/release CI
function is to be used to check if change did not affect the job
:param job_config:
:param job:
:return:
"""
return (
self.is_successful(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
or self.is_pending(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
or self.is_failed(
job=job,
batch=0,
num_batches=job_config.num_batches,
release_branch=not job_config.pr_only,
)
)
def is_failed(
self, job: str, batch: int, num_batches: int, release_branch: bool
) -> bool:
@ -677,74 +706,46 @@ class CiCache:
def filter_out_not_affected_jobs(self):
"""
Filter is to be applied in PRs to remove jobs that are not affected by the change
It removes jobs from @jobs_to_do if it is a:
1. test job and it is in @jobs_to_wait (no need to wait not affected jobs in PRs)
2. test job and it has finished on release branch (even if failed)
3. build job which is not required by any test job that is left in @jobs_to_do
:return:
"""
# 1.
remove_from_await_list = []
for job_name, job_config in self.jobs_to_wait.items():
if CI.is_test_job(job_name) and job_name != CI.JobNames.BUILD_CHECK:
remove_from_await_list.append(job_name)
for job in remove_from_await_list:
print(f"Filter job [{job}] - test job and not affected by the change")
del self.jobs_to_wait[job]
del self.jobs_to_do[job]
# 2.
remove_from_to_do = []
required_builds = []
for job_name, job_config in self.jobs_to_do.items():
if CI.is_test_job(job_name) and job_name != CI.JobNames.BUILD_CHECK:
batches_to_remove = []
assert job_config.batches is not None
for batch in job_config.batches:
if self.is_failed(
job_name, batch, job_config.num_batches, release_branch=True
):
print(
f"Filter [{job_name}/{batch}] - not affected by the change (failed on release branch)"
)
batches_to_remove.append(batch)
for batch in batches_to_remove:
job_config.batches.remove(batch)
if not job_config.batches:
print(
f"Filter [{job_name}] - not affected by the change (failed on release branch)"
)
if job_config.reference_job_name:
reference_name = job_config.reference_job_name
reference_config = self.jobs_to_do[reference_name]
else:
reference_name = job_name
reference_config = job_config
if self.has_evidence(
job=reference_name,
job_config=reference_config,
):
remove_from_to_do.append(job_name)
for job in remove_from_to_do:
del self.jobs_to_do[job]
else:
required_builds += (
job_config.required_builds if job_config.required_builds else []
)
# 3.
required_builds = [] # type: List[str]
for job_name, job_config in self.jobs_to_do.items():
if CI.is_test_job(job_name) and job_config.required_builds:
required_builds += job_config.required_builds
required_builds = list(set(required_builds))
remove_builds = [] # type: List[str]
has_builds_to_do = False
for job_name, job_config in self.jobs_to_do.items():
if CI.is_build_job(job_name):
if job_name not in required_builds:
remove_builds.append(job_name)
remove_from_to_do.append(job_name)
else:
has_builds_to_do = True
for build_job in remove_builds:
print(
f"Filter build job [{build_job}] - not affected and not required by test jobs"
)
del self.jobs_to_do[build_job]
if build_job in self.jobs_to_wait:
del self.jobs_to_wait[build_job]
if not has_builds_to_do:
remove_from_to_do.append(CI.JobNames.BUILD_CHECK)
if not has_builds_to_do and CI.JobNames.BUILD_CHECK in self.jobs_to_do:
print(f"Filter job [{CI.JobNames.BUILD_CHECK}] - no builds to do")
del self.jobs_to_do[CI.JobNames.BUILD_CHECK]
for job in remove_from_to_do:
print(f"Filter job [{job}] - not affected by the change")
if job in self.jobs_to_do:
del self.jobs_to_do[job]
if job in self.jobs_to_wait:
del self.jobs_to_wait[job]
self.jobs_to_skip.append(job)
def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None:
"""

View File

@ -413,7 +413,9 @@ class CI:
release_only=True,
),
JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
reference_job_name=JobNames.INTEGRATION_TEST_TSAN,
),
JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
@ -455,7 +457,10 @@ class CI:
required_builds=[BuildNames.PACKAGE_UBSAN],
),
JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
timeout=3600,
reference_job_name=JobNames.STATELESS_TEST_RELEASE,
),
JobNames.JEPSEN_KEEPER: JobConfig(
required_builds=[BuildNames.BINARY_RELEASE],
@ -640,7 +645,7 @@ class CI:
@classmethod
def is_test_job(cls, job: str) -> bool:
return not cls.is_build_job(job) and job != cls.JobNames.STYLE_CHECK
return not cls.is_build_job(job)
@classmethod
def is_docs_job(cls, job: str) -> bool:

View File

@ -284,8 +284,12 @@ class JobConfig:
# GH Runner type (tag from @Runners)
runner_type: str
# used for config validation in ci unittests
# used in ci unittests for config validation
job_name_keyword: str = ""
# name of another job that (if provided) should be used to check if job was affected by the change or not (in CiCache.has_evidence(job=@reference_job_name) call)
# for example: "Stateless flaky check" can use reference_job_name="Stateless tests (release)". "Stateless flaky check" does not run on master
# and there cannot be an evidence for it, so instead "Stateless tests (release)" job name can be used to check the evidence
reference_job_name: str = ""
# builds required for the job (applicable for test jobs)
required_builds: Optional[List[str]] = None
# build config for the build job (applicable for builds)

View File

@ -23,7 +23,7 @@ from typing import (
from build_download_helper import get_gh_api
from ci_config import CI
from ci_utils import normalize_string
from env_helper import REPORT_PATH, TEMP_PATH
from env_helper import REPORT_PATH, GITHUB_WORKSPACE
logger = logging.getLogger(__name__)
@ -244,7 +244,8 @@ HTML_TEST_PART = """
"""
BASE_HEADERS = ["Test name", "Test status"]
JOB_REPORT_FILE = Path(TEMP_PATH) / "job_report.json"
# should not be in TEMP directory or any directory that may be cleaned during the job execution
JOB_REPORT_FILE = Path(GITHUB_WORKSPACE) / "job_report.json"
@dataclass
@ -296,6 +297,7 @@ class JobReport:
build_dir_for_upload: Union[Path, str] = ""
# if False no GH commit status will be created by CI
need_commit_status: bool = True
job_skipped: bool = False
def __post_init__(self):
assert self.status in (SUCCESS, ERROR, FAILURE, PENDING)

View File

@ -1,6 +1,8 @@
#!/usr/bin/env python3
import unittest
import random
from ci_config import CI
import ci as CIPY
from ci_settings import CiSettings
@ -57,6 +59,18 @@ class TestCIConfig(unittest.TestCase):
f"Job [{job}] apparently uses wrong common config with job keyword [{CI.JOB_CONFIGS[job].job_name_keyword}]",
)
def test_job_config_has_proper_values(self):
for job in CI.JobNames:
if CI.JOB_CONFIGS[job].reference_job_name:
reference_job_config = CI.JOB_CONFIGS[
CI.JOB_CONFIGS[job].reference_job_name
]
# reference job must run in all workflows and has digest
self.assertTrue(reference_job_config.pr_only == False)
self.assertTrue(reference_job_config.release_only == False)
self.assertTrue(reference_job_config.run_always == False)
self.assertTrue(reference_job_config.digest != CI.DigestConfig())
def test_required_checks(self):
for job in CI.REQUIRED_CHECKS:
if job in (CI.StatusNames.PR_CHECK, CI.StatusNames.SYNC):
@ -497,79 +511,68 @@ class TestCIConfig(unittest.TestCase):
settings = CiSettings()
settings.no_ci_cache = True
pr_info = PRInfo(github_event=_TEST_EVENT_JSON)
pr_info.event_type = EventType.PUSH
pr_info.number = 0
assert pr_info.is_release and not pr_info.is_merge_queue
pr_info.event_type = EventType.PULL_REQUEST
pr_info.number = 123
assert pr_info.is_pr
ci_cache = CIPY._configure_jobs(
S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True
)
self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list")
all_jobs_in_wf = list(ci_cache.jobs_to_do)
assert not ci_cache.jobs_to_wait
assert not ci_cache.jobs_to_skip
MOCK_AFFECTED_JOBS = [
CI.JobNames.STATELESS_TEST_S3_DEBUG,
CI.JobNames.STRESS_TEST_TSAN,
]
MOCK_REQUIRED_BUILDS = []
# pretend there are pending jobs that we need to wait
for job, job_config in ci_cache.jobs_to_do.items():
ci_cache.jobs_to_wait[job] = job_config
if job in MOCK_AFFECTED_JOBS:
MOCK_REQUIRED_BUILDS += job_config.required_builds
elif job not in MOCK_AFFECTED_JOBS:
ci_cache.jobs_to_wait[job] = job_config
# remove couple tests from to_wait and
# expect they are preserved in @jobs_to_to along with required package_asan
del ci_cache.jobs_to_wait[CI.JobNames.STATELESS_TEST_ASAN]
del ci_cache.jobs_to_wait[CI.JobNames.INTEGRATION_TEST_TSAN]
del ci_cache.jobs_to_wait[CI.JobNames.STATELESS_TEST_MSAN]
# pretend we have some batches failed for one of the job from the to_do list
failed_job = CI.JobNames.INTEGRATION_TEST_TSAN
failed_job_config = ci_cache.jobs_to_do[failed_job]
FAILED_BATCHES = [0, 3]
for batch in FAILED_BATCHES:
assert batch < failed_job_config.num_batches
record = CiCache.Record(
record_type=CiCache.RecordType.FAILED,
job_name=failed_job,
job_digest=ci_cache.job_digests[failed_job],
batch=batch,
num_batches=failed_job_config.num_batches,
release_branch=True,
)
for record_t_, records_ in ci_cache.records.items():
if record_t_.value == CiCache.RecordType.FAILED.value:
records_[record.to_str_key()] = record
# pretend we have all batches failed for one of the job from the to_do list
failed_job = CI.JobNames.STATELESS_TEST_MSAN
failed_job_config = ci_cache.jobs_to_do[failed_job]
assert failed_job_config.num_batches > 1
for batch in range(failed_job_config.num_batches):
record = CiCache.Record(
record_type=CiCache.RecordType.FAILED,
job_name=failed_job,
job_digest=ci_cache.job_digests[failed_job],
batch=batch,
num_batches=failed_job_config.num_batches,
release_branch=True,
)
for record_t_, records_ in ci_cache.records.items():
if record_t_.value == CiCache.RecordType.FAILED.value:
records_[record.to_str_key()] = record
for job, job_config in ci_cache.jobs_to_do.items():
if job_config.reference_job_name:
# jobs with reference_job_name in config are not supposed to have records in the cache - continue
continue
if job in MOCK_AFFECTED_JOBS:
continue
for batch in range(job_config.num_batches):
# add any record into cache
record = CiCache.Record(
record_type=random.choice(
[
CiCache.RecordType.FAILED,
CiCache.RecordType.PENDING,
CiCache.RecordType.SUCCESSFUL,
]
),
job_name=job,
job_digest=ci_cache.job_digests[job],
batch=batch,
num_batches=job_config.num_batches,
release_branch=True,
)
for record_t_, records_ in ci_cache.records.items():
if record_t_.value == CiCache.RecordType.FAILED.value:
records_[record.to_str_key()] = record
ci_cache.filter_out_not_affected_jobs()
expected_to_do = [
CI.JobNames.STATELESS_TEST_ASAN,
CI.BuildNames.PACKAGE_ASAN,
CI.JobNames.INTEGRATION_TEST_TSAN,
CI.BuildNames.PACKAGE_TSAN,
CI.JobNames.BUILD_CHECK,
]
expected_to_do = (
[
CI.JobNames.BUILD_CHECK,
]
+ MOCK_AFFECTED_JOBS
+ MOCK_REQUIRED_BUILDS
)
self.assertCountEqual(
list(ci_cache.jobs_to_wait),
[
CI.BuildNames.PACKAGE_ASAN,
CI.BuildNames.PACKAGE_TSAN,
CI.JobNames.BUILD_CHECK,
],
]
+ MOCK_REQUIRED_BUILDS,
)
self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do)
self.assertTrue(ci_cache.jobs_to_do[CI.JobNames.INTEGRATION_TEST_TSAN].batches)
for batch in ci_cache.jobs_to_do[CI.JobNames.INTEGRATION_TEST_TSAN].batches:
self.assertTrue(batch not in FAILED_BATCHES)