ci unit test

This commit is contained in:
Max K 2024-07-12 12:29:34 +02:00
parent a4591a4dc4
commit f9eb0f9efd
6 changed files with 135 additions and 22 deletions

View File

@ -995,11 +995,12 @@ def main() -> int:
ci_settings,
args.skip_jobs,
)
ci_cache.print_status()
if IS_CI and pr_info.is_pr:
ci_cache.filter_out_not_affected_jobs()
ci_cache.print_status()
if IS_CI and not pr_info.is_merge_queue:
# wait for pending jobs to be finished, await_jobs is a long blocking call
ci_cache.await_pending_jobs(pr_info.is_release)

View File

@ -676,37 +676,68 @@ class CiCache:
def filter_out_not_affected_jobs(self):
"""
removes the following jobs from to_do and to_wait lists:
test jobs - as not affected by the change
build jobs which are not required by left test jobs
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)
2. 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):
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]
required_builds = list()
# 2.
remove_from_to_do = []
for job_name, job_config in self.jobs_to_do.items():
if CI.is_test_job(job_name):
batches_to_remove = []
if 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)"
)
remove_from_to_do.append(job_name)
for job in remove_from_to_do:
del self.jobs_to_do[job]
# 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 = []
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 += job_name
remove_builds.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")
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]

View File

@ -1,4 +1,5 @@
"""Module to get the token for GitHub"""
from dataclasses import dataclass
import json
import time

View File

@ -37,9 +37,9 @@ class SSHAgent:
ssh_options = (
"," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else ""
)
os.environ[
"SSH_OPTIONS"
] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no"
os.environ["SSH_OPTIONS"] = (
f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no"
)
def add(self, key):
key_pub = self._key_pub(key)

View File

@ -417,7 +417,7 @@ class TestCIConfig(unittest.TestCase):
assert not ci_cache.jobs_to_skip
assert not ci_cache.jobs_to_wait
# pretend there are pending jobs that we neet to wait
# pretend there are pending jobs that we need to wait
ci_cache.jobs_to_wait = dict(ci_cache.jobs_to_do)
for job, config in ci_cache.jobs_to_wait.items():
assert not config.pending_batches
@ -489,3 +489,87 @@ class TestCIConfig(unittest.TestCase):
self.assertCountEqual(
list(ci_cache.jobs_to_do) + ci_cache.jobs_to_skip, all_jobs_in_wf
)
def test_ci_py_filters_not_affected_jobs_in_prs(self):
"""
checks ci.py filters not affected jobs in PRs
"""
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
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
# 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
# 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
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,
]
self.assertCountEqual(
list(ci_cache.jobs_to_wait),
[
CI.BuildNames.PACKAGE_ASAN,
CI.BuildNames.PACKAGE_TSAN,
CI.JobNames.BUILD_CHECK,
],
)
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)

View File

@ -172,14 +172,10 @@ class TestCIOptions(unittest.TestCase):
job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER)
for job in _TEST_JOB_LIST
}
jobs_configs[
"fuzzers"
].run_by_label = (
jobs_configs["fuzzers"].run_by_label = (
"TEST_LABEL" # check "fuzzers" appears in the result due to the label
)
jobs_configs[
"Integration tests (asan)"
].release_only = (
jobs_configs["Integration tests (asan)"].release_only = (
True # still must be included as it's set with include keywords
)
filtered_jobs = list(
@ -311,9 +307,9 @@ class TestCIOptions(unittest.TestCase):
job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER)
for job in _TEST_JOB_LIST
}
jobs_configs[
"fuzzers"
].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result
jobs_configs["fuzzers"].run_by_label = (
"TEST_LABEL" # check "fuzzers" does not appears in the result
)
jobs_configs["Integration tests (asan)"].release_only = True
filtered_jobs = list(
ci_options.apply(