Merge pull request #35124 from vdimir/ci-bugfix-validation

This commit is contained in:
Vladimir C 2022-03-22 17:34:15 +01:00 committed by GitHub
commit ac6fef5c70
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 403 additions and 61 deletions

View File

@ -1733,6 +1733,51 @@ jobs:
docker kill "$(docker ps -q)" ||:
docker rm -f "$(docker ps -a -q)" ||:
sudo rm -fr "$TEMP_PATH"
TestsBugfixCheck:
runs-on: [self-hosted, stress-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/tests_bugfix_check
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Tests bugfix validate check (actions)
KILL_TIMEOUT=3600
REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse
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: Bugfix test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
TEMP_PATH="${TEMP_PATH}/integration" \
REPORTS_PATH="${REPORTS_PATH}/integration" \
python3 integration_test_check.py "Integration tests bugfix validate check" \
--validate-bugfix --post-commit-status=file || echo 'ignore exit code'
TEMP_PATH="${TEMP_PATH}/stateless" \
REPORTS_PATH="${REPORTS_PATH}/stateless" \
python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \
--validate-bugfix --post-commit-status=file || echo 'ignore exit code'
python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv"
- name: Cleanup
if: always()
run: |
docker kill "$(docker ps -q)" ||:
docker rm -f "$(docker ps -a -q)" ||:
sudo rm -fr "$TEMP_PATH"
##############################################################################################
############################ FUNCTIONAl STATEFUL TESTS #######################################
##############################################################################################

View File

@ -0,0 +1,46 @@
#!/usr/bin/env python3
import argparse
import csv
import itertools
import os
import sys
NO_CHANGES_MSG = 'Nothing to run'
def parse_args():
parser = argparse.ArgumentParser()
parser.add_argument('report1')
parser.add_argument('report2')
return parser.parse_args()
def post_commit_status_from_file(file_path):
res = []
with open(file_path, 'r', encoding='utf-8') as f:
fin = csv.reader(f, delimiter='\t')
res = list(itertools.islice(fin, 1))
if len(res) < 1:
raise Exception(f'Can\'t read from "{file_path}"')
if len(res[0]) != 3:
raise Exception(f'Can\'t read from "{file_path}"')
return res[0]
def process_results(file_path):
state, report_url, description = post_commit_status_from_file(file_path)
prefix = os.path.basename(os.path.dirname(file_path))
print(f'::notice:: bugfix check: {prefix} - {state}: {description} Report url: {report_url}')
return state == 'success'
def main(args):
is_ok = False
is_ok = process_results(args.report1) or is_ok
is_ok = process_results(args.report2) or is_ok
sys.exit(0 if is_ok else 1)
if __name__ == '__main__':
main(parse_args())

View File

@ -349,6 +349,9 @@ CI_CONFIG = {
"Stateless tests flaky check (address, actions)": {
"required_build": "package_asan",
},
"Stateless tests bugfix validate check (address, actions)": {
"required_build": "package_asan",
},
"ClickHouse Keeper Jepsen (actions)": {
"required_build": "binary_release",
},

View File

@ -1,15 +1,23 @@
#!/usr/bin/env python3
import time
import os
import csv
from env_helper import GITHUB_REPOSITORY
from ci_config import CI_CONFIG
RETRY = 5
def override_status(status, check_name):
if CI_CONFIG["tests_config"][check_name].get("force_tests", False):
def override_status(status, check_name, invert=False):
if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False):
return "success"
if invert:
if status == 'success':
return 'error'
return 'success'
return status
@ -43,3 +51,11 @@ def post_commit_status(gh, sha, check_name, description, state, report_url):
if i == RETRY - 1:
raise ex
time.sleep(i)
def post_commit_status_to_file(file_path, description, state, report_url):
if os.path.exists(file_path):
raise Exception(f'File "{file_path}" already exists!')
with open(file_path, 'w', encoding='utf-8') as f:
out = csv.writer(f, delimiter='\t')
out.writerow([state, report_url, description])

View File

@ -6,8 +6,10 @@ import time
import subprocess
import logging
from typing import Optional
class DockerImage:
def __init__(self, name, version=None):
def __init__(self, name, version : Optional[str] = None):
self.name = name
if version is None:
self.version = 'latest'
@ -17,7 +19,7 @@ class DockerImage:
def __str__(self):
return f"{self.name}:{self.version}"
def get_images_with_versions(reports_path, required_image, pull=True):
def get_images_with_versions(reports_path, required_image, pull=True, version : Optional[str] = None):
images_path = None
for root, _, files in os.walk(reports_path):
for f in files:
@ -40,7 +42,7 @@ def get_images_with_versions(reports_path, required_image, pull=True):
docker_images = []
for image_name in required_image:
docker_image = DockerImage(image_name)
docker_image = DockerImage(image_name, version)
if image_name in images:
docker_image.version = images[image_name]
docker_images.append(docker_image)
@ -60,5 +62,5 @@ def get_images_with_versions(reports_path, required_image, pull=True):
return docker_images
def get_image_with_version(reports_path, image, pull=True):
return get_images_with_versions(reports_path, [image], pull)[0]
def get_image_with_version(reports_path, image, pull=True, version=None):
return get_images_with_versions(reports_path, [image], pull, version=version)[0]

View File

@ -0,0 +1,130 @@
#!/usr/bin/env python3
###########################################################################
# #
# TODO (@vdimir, @Avogar) #
# Merge with one from https://github.com/ClickHouse/ClickHouse/pull/27928 #
# #
###########################################################################
import re
import os
import logging
import requests
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags"
CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-client_{version}_all.deb"
CLICKHOUSE_COMMON_STATIC_PACKET_NAME = "clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME = "clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_PACKET_NAME = "clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb"
PACKETS_DIR = "previous_release_package_folder/"
VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)"
class Version:
def __init__(self, version):
self.version = version
def __lt__(self, other):
return list(map(int, self.version.split('.'))) < list(map(int, other.version.split('.')))
def __str__(self):
return self.version
class ReleaseInfo:
def __init__(self, version, release_type):
self.version = version
self.type = release_type
def __repr__(self):
return f"ReleaseInfo: {self.version}-{self.type}"
def find_previous_release(server_version, releases):
releases.sort(key=lambda x: x.version, reverse=True)
if server_version is None:
return True, releases[0]
for release in releases:
if release.version < server_version:
return True, release
return False, None
def get_previous_release(server_version=None):
page = 1
found = False
while not found:
response = requests.get(CLICKHOUSE_TAGS_URL, {'page': page, 'per_page': 100})
if not response.ok:
raise Exception('Cannot load the list of tags from github: ' + response.reason)
releases_str = set(re.findall(VERSION_PATTERN, response.text))
if len(releases_str) == 0:
raise Exception('Cannot find previous release for ' + str(server_version) + ' server version')
releases = list(map(lambda x: ReleaseInfo(Version(x.split('-')[0]), x.split('-')[1]), releases_str))
found, previous_release = find_previous_release(server_version, releases)
page += 1
return previous_release
def download_packet(url, out_path):
"""
TODO: use dowload_build_with_progress from build_download_helper.py
"""
response = requests.get(url)
logging.info('Downloading %s', url)
if response.ok:
open(out_path, 'wb').write(response.content)
def download_packets(release, dest_path=PACKETS_DIR):
if not os.path.exists(dest_path):
os.makedirs(dest_path)
logging.info('Will download %s', release)
download_packet(
CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL.format(version=release.version, type=release.type),
out_path=os.path.join(dest_path, CLICKHOUSE_COMMON_STATIC_PACKET_NAME.format(version=release.version)),
)
download_packet(
CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL.format(version=release.version, type=release.type),
out_path=os.path.join(dest_path, CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME.format(version=release.version)),
)
download_packet(
CLICKHOUSE_SERVER_DOWNLOAD_URL.format(version=release.version, type=release.type),
out_path=os.path.join(dest_path, CLICKHOUSE_SERVER_PACKET_NAME.format(version=release.version)),
)
download_packet(
CLICKHOUSE_CLIENT_DOWNLOAD_URL.format(version=release.version, type=release.type),
out_path=os.path.join(dest_path, CLICKHOUSE_CLIENT_PACKET_NAME.format(version=release.version)),
)
def download_previous_release(dest_path):
current_release = get_previous_release(None)
download_packets(current_release, dest_path=dest_path)
if __name__ == '__main__':
logging.basicConfig(level=logging.INFO)
server_version = Version(input())
previous_release = get_previous_release(server_version)
download_packets(previous_release)

View File

@ -1,9 +1,10 @@
#!/usr/bin/env python3
import argparse
import csv
import logging
import subprocess
import os
import subprocess
import sys
from github import Github
@ -13,14 +14,18 @@ from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from build_download_helper import download_all_deb_packages
from download_previous_release import download_previous_release
from upload_result_helper import upload_results
from docker_pull_helper import get_image_with_version
from commit_status_helper import post_commit_status, get_commit, override_status
from commit_status_helper import post_commit_status, get_commit, override_status, post_commit_status_to_file
from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse
from stopwatch import Stopwatch
from rerun_helper import RerunHelper
from tee_popen import TeePopen
NO_CHANGES_MSG = 'Nothing to run'
def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total):
result = []
if 'DatabaseReplicated' in check_name:
@ -48,6 +53,7 @@ def get_image_name(check_name):
else:
raise Exception(f"Cannot deduce image name based on check name {check_name}")
def get_run_command(builds_path, repo_tests_path, result_path, server_log_path, kill_timeout, additional_envs, image, flaky_check, tests_to_run):
additional_options = ['--hung-check']
additional_options.append('--print-time')
@ -127,6 +133,15 @@ def process_results(result_folder, server_log_path):
return state, description, test_results, additional_files
def parse_args():
parser = argparse.ArgumentParser()
parser.add_argument("check_name")
parser.add_argument("kill_timeout", type=int)
parser.add_argument("--validate-bugfix", action='store_true', help="Check that added tests failed on latest stable")
parser.add_argument("--post-commit-status", default='commit_status', choices=['commit_status', 'file'], help="Where to public post commit status")
return parser.parse_args()
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
@ -136,13 +151,26 @@ if __name__ == "__main__":
repo_path = REPO_COPY
reports_path = REPORTS_PATH
check_name = sys.argv[1]
kill_timeout = int(sys.argv[2])
args = parse_args()
check_name = args.check_name
kill_timeout = args.kill_timeout
validate_bugix_check = args.validate_bugfix
flaky_check = 'flaky' in check_name.lower()
run_changed_tests = flaky_check or validate_bugix_check
gh = Github(get_best_robot_token())
pr_info = PRInfo(need_changed_files=flaky_check)
pr_info = PRInfo(need_changed_files=run_changed_tests)
if not os.path.exists(temp_path):
os.makedirs(temp_path)
if validate_bugix_check and 'pr-bugfix' not in pr_info.labels:
if args.post_commit_status == 'file':
post_commit_status_to_file(os.path.join(temp_path, "post_commit_status.tsv"), 'Skipped (no pr-bugfix)', 'success', 'null')
logging.info("Skipping '%s' (no pr-bugfix)", check_name)
sys.exit(0)
if 'RUN_BY_HASH_NUM' in os.environ:
run_by_hash_num = int(os.getenv('RUN_BY_HASH_NUM'))
@ -158,15 +186,17 @@ if __name__ == "__main__":
logging.info("Check is already finished according to github status, exiting")
sys.exit(0)
if not os.path.exists(temp_path):
os.makedirs(temp_path)
tests_to_run = []
if flaky_check:
if run_changed_tests:
tests_to_run = get_tests_to_run(pr_info)
if not tests_to_run:
commit = get_commit(gh, pr_info.sha)
commit.create_status(context=check_name_with_group, description='Not found changed stateless tests', state='success')
state = override_status('success', check_name, validate_bugix_check)
if args.post_commit_status == 'commit_status':
commit.create_status(context=check_name_with_group, description=NO_CHANGES_MSG, state=state)
elif args.post_commit_status == 'file':
fpath = os.path.join(temp_path, "post_commit_status.tsv")
post_commit_status_to_file(fpath, description=NO_CHANGES_MSG, state=state, report_url='null')
sys.exit(0)
image_name = get_image_name(check_name)
@ -178,7 +208,10 @@ if __name__ == "__main__":
if not os.path.exists(packages_path):
os.makedirs(packages_path)
download_all_deb_packages(check_name, reports_path, packages_path)
if validate_bugix_check:
download_previous_release(packages_path)
else:
download_all_deb_packages(check_name, reports_path, packages_path)
server_log_path = os.path.join(temp_path, "server_log")
if not os.path.exists(server_log_path):
@ -191,6 +224,9 @@ if __name__ == "__main__":
run_log_path = os.path.join(result_path, "runlog.log")
additional_envs = get_additional_envs(check_name, run_by_hash_num, run_by_hash_total)
if validate_bugix_check:
additional_envs.append('GLOBAL_TAGS=no-random-settings')
run_command = get_run_command(packages_path, repo_tests_path, result_path, server_log_path, kill_timeout, additional_envs, docker_image, flaky_check, tests_to_run)
logging.info("Going to run func tests: %s", run_command)
@ -206,15 +242,20 @@ if __name__ == "__main__":
s3_helper = S3Helper('https://s3.amazonaws.com')
state, description, test_results, additional_logs = process_results(result_path, server_log_path)
state = override_status(state, check_name)
state = override_status(state, check_name, validate_bugix_check)
ch_helper = ClickHouseHelper()
mark_flaky_tests(ch_helper, check_name, test_results)
report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [run_log_path] + additional_logs, check_name_with_group)
print(f"::notice ::Report url: {report_url}")
post_commit_status(gh, pr_info.sha, check_name_with_group, description, state, report_url)
print(f"::notice:: {check_name} Report url: {report_url}")
if args.post_commit_status == 'commit_status':
post_commit_status(gh, pr_info.sha, check_name_with_group, description, state, report_url)
elif args.post_commit_status == 'file':
post_commit_status_to_file(os.path.join(temp_path, "post_commit_status.tsv"), description, state, report_url)
else:
raise Exception(f'Unknown post_commit_status option "{args.post_commit_status}"')
prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name_with_group)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)

View File

@ -1,11 +1,12 @@
#!/usr/bin/env python3
import os
import logging
import sys
import json
import subprocess
import argparse
import csv
import json
import logging
import os
import subprocess
import sys
from github import Github
@ -14,9 +15,10 @@ from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from build_download_helper import download_all_deb_packages
from download_previous_release import download_previous_release
from upload_result_helper import upload_results
from docker_pull_helper import get_images_with_versions
from commit_status_helper import post_commit_status
from commit_status_helper import post_commit_status, override_status, post_commit_status_to_file
from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse
from stopwatch import Stopwatch
from rerun_helper import RerunHelper
@ -102,6 +104,15 @@ def process_results(result_folder):
return state, description, test_results, additional_files
def parse_args():
parser = argparse.ArgumentParser()
parser.add_argument("check_name")
parser.add_argument("--validate-bugfix", action='store_true', help="Check that added tests failed on latest stable")
parser.add_argument("--post-commit-status", default='commit_status', choices=['commit_status', 'file'], help="Where to public post commit status")
return parser.parse_args()
if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
@ -111,7 +122,9 @@ if __name__ == "__main__":
repo_path = REPO_COPY
reports_path = REPORTS_PATH
check_name = sys.argv[1]
args = parse_args()
check_name = args.check_name
validate_bugix_check = args.validate_bugfix
if 'RUN_BY_HASH_NUM' in os.environ:
run_by_hash_num = int(os.getenv('RUN_BY_HASH_NUM'))
@ -126,7 +139,13 @@ if __name__ == "__main__":
os.makedirs(temp_path)
is_flaky_check = 'flaky' in check_name
pr_info = PRInfo(need_changed_files=is_flaky_check)
pr_info = PRInfo(need_changed_files=is_flaky_check or validate_bugix_check)
if validate_bugix_check and 'pr-bugfix' not in pr_info.labels:
if args.post_commit_status == 'file':
post_commit_status_to_file(os.path.join(temp_path, "post_commit_status.tsv"), 'Skipped (no pr-bugfix)', 'success', 'null')
logging.info("Skipping '%s' (no pr-bugfix)", check_name)
sys.exit(0)
gh = Github(get_best_robot_token())
@ -149,7 +168,10 @@ if __name__ == "__main__":
if not os.path.exists(build_path):
os.makedirs(build_path)
download_all_deb_packages(check_name, reports_path, build_path)
if validate_bugix_check:
download_previous_release(build_path)
else:
download_all_deb_packages(check_name, reports_path, build_path)
my_env = get_env_for_runner(build_path, repo_path, result_path, work_path)
@ -172,14 +194,21 @@ if __name__ == "__main__":
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
state, description, test_results, additional_logs = process_results(result_path)
state = override_status(state, check_name, validate_bugix_check)
ch_helper = ClickHouseHelper()
mark_flaky_tests(ch_helper, check_name, test_results)
s3_helper = S3Helper('https://s3.amazonaws.com')
report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [output_path_log] + additional_logs, check_name_with_group, False)
print(f"::notice ::Report url: {report_url}")
post_commit_status(gh, pr_info.sha, check_name_with_group, description, state, report_url)
print(f"::notice:: {check_name} Report url: {report_url}")
if args.post_commit_status == 'commit_status':
post_commit_status(gh, pr_info.sha, check_name_with_group, description, state, report_url)
elif args.post_commit_status == 'file':
post_commit_status_to_file(os.path.join(temp_path, "post_commit_status.tsv"), description, state, report_url)
else:
raise Exception(f'Unknown post_commit_status option "{args.post_commit_status}"')
prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name_with_group)
ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events)

View File

@ -78,7 +78,7 @@ class PRInfo:
else:
github_event = PRInfo.default_event.copy()
self.event = github_event
self.changed_files = set([])
self.changed_files = set()
self.body = ""
ref = github_event.get("ref", "refs/head/master")
if ref and ref.startswith("refs/heads/"):
@ -209,6 +209,7 @@ class PRInfo:
else:
diff_object = PatchSet(response.text)
self.changed_files = {f.path for f in diff_object}
print("Fetched info about %d changed files", len(self.changed_files))
def get_dict(self):
return {

View File

@ -95,6 +95,17 @@ HTML_TEST_PART = """
BASE_HEADERS = ['Test name', 'Test status']
class ReportColorTheme:
class ReportColor:
yellow = '#FFB400'
red = '#F00'
green = '#0A0'
blue = '#00B4FF'
default = (ReportColor.green, ReportColor.red, ReportColor.yellow)
bugfixcheck = (ReportColor.yellow, ReportColor.blue, ReportColor.blue)
def _format_header(header, branch_name, branch_url=None):
result = ' '.join([w.capitalize() for w in header.split(' ')])
result = result.replace("Clickhouse", "ClickHouse")
@ -109,14 +120,20 @@ def _format_header(header, branch_name, branch_url=None):
return result
def _get_status_style(status):
def _get_status_style(status, colortheme=None):
ok_statuses = ('OK', 'success', 'PASSED')
fail_statuses = ('FAIL', 'failure', 'error', 'FAILED', 'Timeout')
if colortheme is None:
colortheme = ReportColorTheme.default
style = "font-weight: bold;"
if status in ('OK', 'success', 'PASSED'):
style += 'color: #0A0;'
elif status in ('FAIL', 'failure', 'error', 'FAILED', 'Timeout'):
style += 'color: #F00;'
if status in ok_statuses:
style += f'color: {colortheme[0]};'
elif status in fail_statuses:
style += f'color: {colortheme[1]};'
else:
style += 'color: #FFB400;'
style += f'color: {colortheme[2]};'
return style
@ -140,7 +157,7 @@ def _get_html_url(url):
return ''
def create_test_html_report(header, test_result, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls=None, with_raw_logs=False):
def create_test_html_report(header, test_result, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls=None, with_raw_logs=False, statuscolors=None):
if additional_urls is None:
additional_urls = []
@ -168,7 +185,7 @@ def create_test_html_report(header, test_result, raw_log_url, task_url, branch_u
if is_fail and with_raw_logs and test_logs is not None:
row = "<tr class=\"failed\">"
row += "<td>" + test_name + "</td>"
style = _get_status_style(test_status)
style = _get_status_style(test_status, colortheme=statuscolors)
# Allow to quickly scroll to the first failure.
is_fail_id = ""

View File

@ -3,7 +3,7 @@ import logging
import ast
from env_helper import GITHUB_SERVER_URL, GITHUB_REPOSITORY, GITHUB_RUN_ID
from report import create_test_html_report
from report import ReportColorTheme, create_test_html_report
def process_logs(s3_client, additional_logs, s3_path_prefix, test_results, with_raw_logs):
@ -39,7 +39,7 @@ def process_logs(s3_client, additional_logs, s3_path_prefix, test_results, with_
return additional_urls
def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files, check_name, with_raw_logs=True):
def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files, check_name, with_raw_logs=True, statuscolors=None):
s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace(' ', '_').replace('(', '_').replace(')', '_').replace(',', '_')
additional_urls = process_logs(s3_client, additional_files, s3_path_prefix, test_results, with_raw_logs)
@ -58,7 +58,11 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi
else:
raw_log_url = task_url
html_report = create_test_html_report(check_name, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls, with_raw_logs)
statuscolors = ReportColorTheme.bugfixcheck if 'bugfix validate check' in check_name else None
html_report = create_test_html_report(check_name, test_results, raw_log_url,
task_url, branch_url, branch_name, commit_url,
additional_urls, with_raw_logs, statuscolors=statuscolors)
with open('report.html', 'w', encoding='utf-8') as f:
f.write(html_report)

View File

@ -490,6 +490,9 @@ class TestCase:
self.case: str = case # case file name
self.tags: Set[str] = suite.all_tags[case] if case in suite.all_tags else set()
for tag in os.getenv("GLOBAL_TAGS", "").split(","):
self.tags.add(tag.strip())
self.case_file: str = os.path.join(suite.suite_path, case)
(self.name, self.ext) = os.path.splitext(case)

View File

@ -21,19 +21,21 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse"
CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge"
CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge"
TRIES_COUNT = 10
FLAKY_TRIES_COUNT = 10
MAX_TIME_SECONDS = 3600
MAX_TIME_IN_SANDBOX = 20 * 60 # 20 minutes
TASK_TIMEOUT = 8 * 60 * 60 # 8 hours
NO_CHANGES_MSG = 'Nothing to run'
def stringhash(s):
return zlib.crc32(s.encode("utf-8"))
def get_tests_to_run(pr_info):
result = set([])
def get_changed_tests_to_run(pr_info, repo_path):
result = set()
changed_files = pr_info["changed_files"]
if changed_files is None:
@ -43,7 +45,7 @@ def get_tests_to_run(pr_info):
if "tests/integration/test_" in fpath:
logging.info("File %s changed and seems like integration test", fpath)
result.add(fpath.split("/")[2])
return list(result)
return filter_existing_tests(result, repo_path)
def filter_existing_tests(tests_to_run, repo_path):
@ -207,6 +209,7 @@ class ClickhouseIntegrationTestsRunner:
self.image_versions = self.params["docker_images_with_versions"]
self.shuffle_groups = self.params["shuffle_test_groups"]
self.flaky_check = "flaky check" in self.params["context_name"]
self.bugfix_validate_check = "bugfix validate check" in self.params["context_name"]
# if use_tmpfs is not set we assume it to be true, otherwise check
self.use_tmpfs = "use_tmpfs" not in self.params or self.params["use_tmpfs"]
self.disable_net_host = (
@ -703,14 +706,13 @@ class ClickhouseIntegrationTestsRunner:
return counters, tests_times, log_paths
def run_flaky_check(self, repo_path, build_path):
def run_flaky_check(self, repo_path, build_path, should_fail=False):
pr_info = self.params["pr_info"]
# pytest swears, if we require to run some tests which was renamed or deleted
tests_to_run = filter_existing_tests(get_tests_to_run(pr_info), repo_path)
tests_to_run = get_changed_tests_to_run(pr_info, repo_path)
if not tests_to_run:
logging.info("No tests to run found")
return "success", "Nothing to run", [("Nothing to run", "OK")], ""
return "success", NO_CHANGES_MSG, [(NO_CHANGES_MSG, "OK")], ""
self._install_clickhouse(build_path)
logging.info("Found '%s' tests to run", " ".join(tests_to_run))
@ -720,26 +722,29 @@ class ClickhouseIntegrationTestsRunner:
logging.info("Starting check with retries")
final_retry = 0
logs = []
for i in range(TRIES_COUNT):
tires_num = 1 if should_fail else FLAKY_TRIES_COUNT
for i in range(tires_num):
final_retry += 1
logging.info("Running tests for the %s time", i)
counters, tests_times, log_paths = self.try_run_test_group(
repo_path, "flaky", tests_to_run, 1, 1
repo_path, "bugfix" if should_fail else "flaky", tests_to_run, 1, 1
)
logs += log_paths
if counters["FAILED"]:
logging.info("Found failed tests: %s", " ".join(counters["FAILED"]))
description_prefix = "Flaky tests found: "
description_prefix = "Failed tests found: "
result_state = "failure"
break
if not should_fail:
break
if counters["ERROR"]:
description_prefix = "Flaky tests found: "
description_prefix = "Failed tests found: "
logging.info("Found error tests: %s", " ".join(counters["ERROR"]))
# NOTE "error" result state will restart the whole test task,
# so we use "failure" here
result_state = "failure"
break
assert len(counters["FLAKY"]) == 0
if not should_fail:
break
assert len(counters["FLAKY"]) == 0 or should_fail
logging.info("Try is OK, all tests passed, going to clear env")
clear_ip_tables_and_restart_daemons()
logging.info("And going to sleep for some time")
@ -774,8 +779,8 @@ class ClickhouseIntegrationTestsRunner:
return result_state, status_text, test_result, logs
def run_impl(self, repo_path, build_path):
if self.flaky_check:
return self.run_flaky_check(repo_path, build_path)
if self.flaky_check or self.bugfix_validate_check:
return self.run_flaky_check(repo_path, build_path, should_fail=self.bugfix_validate_check)
self._install_clickhouse(build_path)
logging.info(