Merge pull request #54010 from ClickHouse/modular-workflows

Use pathlib.Path in S3Helper, rewrite build reports, improve small things
This commit is contained in:
Mikhail f. Shiryaev 2023-09-06 14:29:37 +02:00 committed by GitHub
commit 587013d994
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 575 additions and 518 deletions

View File

@ -160,7 +160,7 @@ def main():
s3_helper = S3Helper()
for f in paths:
try:
paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f)
paths[f] = s3_helper.upload_test_report_to_s3(Path(paths[f]), s3_prefix + f)
except Exception as ex:
logging.info("Exception uploading file %s text %s", f, ex)
paths[f] = ""

View File

@ -1,10 +1,9 @@
#!/usr/bin/env python3
from pathlib import Path
from typing import List, Tuple
from typing import Tuple
import subprocess
import logging
import json
import os
import sys
import time
@ -22,6 +21,7 @@ from env_helper import (
)
from git_helper import Git, git_runner
from pr_info import PRInfo
from report import BuildResult, FAILURE, StatusType, SUCCESS
from s3_helper import S3Helper
from tee_popen import TeePopen
from version_helper import (
@ -98,7 +98,7 @@ def get_packager_cmd(
def build_clickhouse(
packager_cmd: str, logs_path: Path, build_output_path: Path
) -> Tuple[Path, bool]:
) -> Tuple[Path, StatusType]:
build_log_path = logs_path / BUILD_LOG_NAME
success = False
with TeePopen(packager_cmd, build_log_path) as process:
@ -118,15 +118,16 @@ def build_clickhouse(
)
else:
logging.info("Build failed")
return build_log_path, success
return build_log_path, SUCCESS if success else FAILURE
def check_for_success_run(
s3_helper: S3Helper,
s3_prefix: str,
build_name: str,
build_config: BuildConfig,
version: ClickHouseVersion,
) -> None:
# TODO: Remove after S3 artifacts
# the final empty argument is necessary for distinguish build and build_suffix
logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix, "")
logging.info("Checking for artifacts in %s", logged_prefix)
@ -155,15 +156,16 @@ def check_for_success_run(
return
success = len(build_urls) > 0
create_json_artifact(
TEMP_PATH,
build_result = BuildResult(
build_name,
log_url,
build_urls,
build_config,
version.describe,
SUCCESS if success else FAILURE,
0,
success,
GITHUB_JOB,
)
build_result.write_json(Path(TEMP_PATH))
# Fail build job if not successeded
if not success:
sys.exit(1)
@ -171,36 +173,6 @@ def check_for_success_run(
sys.exit(0)
def create_json_artifact(
temp_path: str,
build_name: str,
log_url: str,
build_urls: List[str],
build_config: BuildConfig,
elapsed: int,
success: bool,
) -> None:
subprocess.check_call(
f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True
)
result = {
"log_url": log_url,
"build_urls": build_urls,
"build_config": build_config.__dict__,
"elapsed_seconds": elapsed,
"status": success,
"job_name": GITHUB_JOB,
}
json_name = "build_urls_" + build_name + ".json"
print(f"Dump json report {result} to {json_name} with env build_urls_{build_name}")
with open(os.path.join(temp_path, json_name), "w", encoding="utf-8") as build_links:
json.dump(result, build_links)
def get_release_or_pr(pr_info: PRInfo, version: ClickHouseVersion) -> Tuple[str, str]:
"Return prefixes for S3 artifacts paths"
# FIXME performance
@ -269,7 +241,7 @@ def main():
# If this is rerun, then we try to find already created artifacts and just
# put them as github actions artifact (result)
check_for_success_run(s3_helper, s3_path_prefix, build_name, build_config)
check_for_success_run(s3_helper, s3_path_prefix, build_name, version)
docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME)
image_version = docker_image.version
@ -312,16 +284,17 @@ def main():
os.makedirs(logs_path, exist_ok=True)
start = time.time()
log_path, success = build_clickhouse(packager_cmd, logs_path, build_output_path)
log_path, build_status = build_clickhouse(
packager_cmd, logs_path, build_output_path
)
elapsed = int(time.time() - start)
subprocess.check_call(
f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True
)
logging.info("Build finished with %s, log path %s", success, log_path)
if success:
logging.info("Build finished as %s, log path %s", build_status, log_path)
if build_status == SUCCESS:
cargo_cache.upload()
if not success:
else:
# We check if docker works, because if it's down, it's infrastructure
try:
subprocess.check_call("docker info", shell=True)
@ -345,7 +318,7 @@ def main():
os.remove(performance_path)
build_urls = (
s3_helper.upload_build_folder_to_s3(
s3_helper.upload_build_directory_to_s3(
build_output_path,
s3_path_prefix,
keep_dirs_in_s3_path=False,
@ -367,8 +340,20 @@ def main():
print(f"::notice ::Log URL: {log_url}")
create_json_artifact(
TEMP_PATH, build_name, log_url, build_urls, build_config, elapsed, success
build_result = BuildResult(
build_name,
log_url,
build_urls,
version.describe,
build_status,
elapsed,
GITHUB_JOB,
)
result_json_path = build_result.write_json(temp_path)
logging.info(
"Build result file %s is written, content:\n %s",
result_json_path,
result_json_path.read_text(encoding="utf-8"),
)
upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path)
@ -449,7 +434,7 @@ FORMAT JSONCompactEachRow"""
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,
[],
"success" if success else "failure",
build_status,
stopwatch.duration_seconds,
stopwatch.start_time_str,
log_url,
@ -458,7 +443,7 @@ FORMAT JSONCompactEachRow"""
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
# Fail the build job if it didn't succeed
if not success:
if build_status != SUCCESS:
sys.exit(1)

View File

@ -6,7 +6,7 @@ import os
import sys
import time
from pathlib import Path
from typing import Any, Callable, List
from typing import Any, Callable, List, Union
import requests # type: ignore
@ -98,7 +98,7 @@ def get_build_name_for_check(check_name: str) -> str:
return CI_CONFIG.test_configs[check_name].required_build
def read_build_urls(build_name: str, reports_path: str) -> List[str]:
def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]:
for root, _, files in os.walk(reports_path):
for f in files:
if build_name in f:

View File

@ -5,19 +5,25 @@ import logging
import os
import sys
import atexit
from typing import Dict, List, Tuple
from pathlib import Path
from github import Github
from env_helper import (
GITHUB_JOB_URL,
GITHUB_REPOSITORY,
GITHUB_RUN_URL,
GITHUB_SERVER_URL,
REPORTS_PATH,
TEMP_PATH,
)
from report import create_build_html_report, BuildResult, BuildResults
from report import (
BuildResult,
ERROR,
PENDING,
SUCCESS,
create_build_html_report,
get_worst_status,
)
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from pr_info import NeedsDataType, PRInfo
@ -34,95 +40,17 @@ from ci_config import CI_CONFIG
NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "")
def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]:
groups = {
"apk": [],
"deb": [],
"binary": [],
"tgz": [],
"rpm": [],
"performance": [],
} # type: Dict[str, List[str]]
for url in build_urls:
if url.endswith("performance.tar.zst"):
groups["performance"].append(url)
elif (
url.endswith(".deb")
or url.endswith(".buildinfo")
or url.endswith(".changes")
or url.endswith(".tar.gz")
):
groups["deb"].append(url)
elif url.endswith(".apk"):
groups["apk"].append(url)
elif url.endswith(".rpm"):
groups["rpm"].append(url)
elif url.endswith(".tgz") or url.endswith(".tgz.sha512"):
groups["tgz"].append(url)
else:
groups["binary"].append(url)
return groups
def get_failed_report(
job_name: str,
) -> Tuple[BuildResults, List[List[str]], List[str]]:
message = f"{job_name} failed"
build_result = BuildResult(
compiler="unknown",
debug_build=False,
sanitizer="unknown",
status=message,
elapsed_seconds=0,
comment="",
)
return [build_result], [[""]], [GITHUB_RUN_URL]
def process_report(
build_report: dict,
) -> Tuple[BuildResults, List[List[str]], List[str]]:
build_config = build_report["build_config"]
build_result = BuildResult(
compiler=build_config["compiler"],
debug_build=build_config["debug_build"],
sanitizer=build_config["sanitizer"],
status="success" if build_report["status"] else "failure",
elapsed_seconds=build_report["elapsed_seconds"],
comment=build_config["comment"],
)
build_results = []
build_urls = []
build_logs_urls = []
urls_groups = group_by_artifacts(build_report["build_urls"])
found_group = False
for _, group_urls in urls_groups.items():
if group_urls:
build_results.append(build_result)
build_urls.append(group_urls)
build_logs_urls.append(build_report["log_url"])
found_group = True
# No one group of urls is found, a failed report
if not found_group:
build_results.append(build_result)
build_urls.append([""])
build_logs_urls.append(build_report["log_url"])
return build_results, build_urls, build_logs_urls
def get_build_name_from_file_name(file_name):
return file_name.replace("build_urls_", "").replace(".json", "")
def main():
logging.basicConfig(level=logging.INFO)
temp_path = TEMP_PATH
logging.info("Reports path %s", REPORTS_PATH)
temp_path = Path(TEMP_PATH)
temp_path.mkdir(parents=True, exist_ok=True)
if not os.path.exists(temp_path):
os.makedirs(temp_path)
logging.info("Reports path %s", REPORTS_PATH)
reports_path = Path(REPORTS_PATH)
logging.info(
"Reports found:\n %s",
"\n ".join(p.as_posix() for p in reports_path.rglob("*.json")),
)
build_check_name = sys.argv[1]
needs_data = {} # type: NeedsDataType
@ -132,11 +60,11 @@ def main():
needs_data = json.load(file_handler)
required_builds = len(needs_data)
if needs_data and all(i["result"] == "skipped" for i in needs_data.values()):
logging.info("All builds are skipped, exiting")
sys.exit(0)
logging.info("The next builds are required: %s", ", ".join(needs_data))
if needs_data:
logging.info("The next builds are required: %s", ", ".join(needs_data))
if all(i["result"] == "skipped" for i in needs_data.values()):
logging.info("All builds are skipped, exiting")
sys.exit(0)
gh = Github(get_best_robot_token(), per_page=100)
pr_info = PRInfo()
@ -153,73 +81,41 @@ def main():
required_builds = required_builds or len(builds_for_check)
# Collect reports from json artifacts
builds_report_map = {}
for root, _, files in os.walk(REPORTS_PATH):
for f in files:
if f.startswith("build_urls_") and f.endswith(".json"):
logging.info("Found build report json %s", f)
build_name = get_build_name_from_file_name(f)
if build_name in builds_for_check:
with open(os.path.join(root, f), "rb") as file_handler:
builds_report_map[build_name] = json.load(file_handler)
else:
logging.info(
"Skipping report %s for build %s, it's not in our reports list",
f,
build_name,
)
build_results = []
for build_name in builds_for_check:
report_name = BuildResult.get_report_name(build_name).stem
build_result = BuildResult.read_json(reports_path / report_name, build_name)
if build_result.is_missing:
logging.warning("Build results for %s are missing", build_name)
continue
build_results.append(build_result)
# Sort reports by config order
build_reports = [
builds_report_map[build_name]
for build_name in builds_for_check
if build_name in builds_report_map
# The code to collect missing reports for failed jobs
missing_job_names = [
name
for name in needs_data
if not any(1 for build_result in build_results if build_result.job_name == name)
]
some_builds_are_missing = len(build_reports) < required_builds
missing_build_names = []
if some_builds_are_missing:
logging.warning(
"Expected to get %s build results, got only %s",
required_builds,
len(build_reports),
)
missing_build_names = [
name
for name in needs_data
if not any(rep for rep in build_reports if rep["job_name"] == name)
]
else:
logging.info("Got exactly %s builds", len(builds_report_map))
# Group build artifacts by groups
build_results = [] # type: BuildResults
build_artifacts = [] # type: List[List[str]]
build_logs = [] # type: List[str]
for build_report in build_reports:
_build_results, build_artifacts_url, build_logs_url = process_report(
build_report
)
missing_builds = len(missing_job_names)
for job_name in reversed(missing_job_names):
build_result = BuildResult.missing_result("missing")
build_result.job_name = job_name
build_result.status = PENDING
logging.info(
"Got %s artifact groups for build report report", len(_build_results)
"There is missing report for %s, created a dummy result %s",
job_name,
build_result,
)
build_results.extend(_build_results)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)
build_results.insert(0, build_result)
for failed_job in missing_build_names:
_build_results, build_artifacts_url, build_logs_url = get_failed_report(
failed_job
)
build_results.extend(_build_results)
build_artifacts.extend(build_artifacts_url)
build_logs.extend(build_logs_url)
total_groups = len(build_results)
# Calculate artifact groups like packages and binaries
total_groups = sum(len(br.grouped_urls) for br in build_results)
ok_groups = sum(
len(br.grouped_urls) for br in build_results if br.status == SUCCESS
)
logging.info("Totally got %s artifact groups", total_groups)
if total_groups == 0:
logging.error("No success builds, failing check")
logging.error("No success builds, failing check without creating a status")
sys.exit(1)
s3_helper = S3Helper()
@ -234,17 +130,14 @@ def main():
report = create_build_html_report(
build_check_name,
build_results,
build_logs,
build_artifacts,
task_url,
branch_url,
branch_name,
commit_url,
)
report_path = os.path.join(temp_path, "report.html")
with open(report_path, "w", encoding="utf-8") as fd:
fd.write(report)
report_path = temp_path / "report.html"
report_path.write_text(report, encoding="utf-8")
logging.info("Going to upload prepared report")
context_name_for_path = build_check_name.lower().replace(" ", "_")
@ -259,27 +152,20 @@ def main():
print(f"::notice ::Report url: {url}")
# Prepare a commit status
ok_groups = 0
summary_status = "success"
for build_result in build_results:
if build_result.status == "failure" and summary_status != "error":
summary_status = "failure"
if build_result.status == "error" or not build_result.status:
summary_status = "error"
if build_result.status == "success":
ok_groups += 1
summary_status = get_worst_status(br.status for br in build_results)
# Check if there are no builds at all, do not override bad status
if summary_status == "success":
if some_builds_are_missing:
summary_status = "pending"
if summary_status == SUCCESS:
if missing_builds:
summary_status = PENDING
elif ok_groups == 0:
summary_status = "error"
summary_status = ERROR
addition = ""
if some_builds_are_missing:
addition = f" ({len(build_reports)} of {required_builds} builds are OK)"
if missing_builds:
addition = (
f" ({required_builds - missing_builds} of {required_builds} builds are OK)"
)
description = format_description(
f"{ok_groups}/{total_groups} artifact groups are OK{addition}"
@ -289,7 +175,7 @@ def main():
commit, summary_status, url, description, build_check_name, pr_info
)
if summary_status == "error":
if summary_status == ERROR:
sys.exit(1)

View File

@ -3,7 +3,7 @@
import csv
import os
import time
from typing import Dict, List, Literal, Optional, Union
from typing import Dict, List, Optional, Union
import logging
from github import Github
@ -16,7 +16,16 @@ from github.Repository import Repository
from ci_config import CI_CONFIG, REQUIRED_CHECKS, CHECK_DESCRIPTIONS, CheckDescription
from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL
from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL
from report import TestResult, TestResults
from report import (
ERROR,
FAILURE,
PENDING,
StatusType,
SUCCESS,
TestResult,
TestResults,
get_worst_status,
)
from s3_helper import S3Helper
from upload_result_helper import upload_results
@ -37,8 +46,8 @@ class RerunHelper:
# currently we agree even for failed statuses
for status in self.statuses:
if self.check_name in status.context and status.state in (
"success",
"failure",
SUCCESS,
FAILURE,
):
return True
return False
@ -53,12 +62,12 @@ class RerunHelper:
def override_status(status: str, check_name: str, invert: bool = False) -> str:
test_config = CI_CONFIG.test_configs.get(check_name)
if test_config and test_config.force_tests:
return "success"
return SUCCESS
if invert:
if status == "success":
return "error"
return "success"
if status == SUCCESS:
return ERROR
return SUCCESS
return status
@ -137,7 +146,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None:
# W/o pr_info to avoid recursion, and yes, one extra create_ci_report
post_commit_status(
commit,
"pending",
PENDING,
create_ci_report(pr_info, statuses),
"The report for running CI",
CI_STATUS_NAME,
@ -172,11 +181,11 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
"""The method generates the comment body, as well it updates the CI report"""
def beauty_state(state: str) -> str:
if state == "success":
if state == SUCCESS:
return f"🟢 {state}"
if state == "pending":
if state == PENDING:
return f"🟡 {state}"
if state in ["error", "failure"]:
if state in [ERROR, FAILURE]:
return f"🔴 {state}"
return state
@ -235,20 +244,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str:
def get_worst_state(statuses: CommitStatuses) -> str:
worst_status = None
states = {"error": 0, "failure": 1, "pending": 2, "success": 3}
for status in statuses:
if worst_status is None:
worst_status = status
continue
if states[status.state] < states[worst_status.state]:
worst_status = status
if worst_status.state == "error":
break
if worst_status is None:
return ""
return worst_status.state
return get_worst_status(status.state for status in statuses)
def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str:
@ -324,7 +320,7 @@ def format_description(description: str) -> str:
def set_mergeable_check(
commit: Commit,
description: str = "",
state: Literal["success", "failure"] = "success",
state: StatusType = "success",
) -> None:
commit.create_status(
context=MERGEABLE_NAME,
@ -363,7 +359,7 @@ def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None
success = []
fail = []
for status in required_checks:
if status.state == "success":
if status.state == SUCCESS:
success.append(status.context)
else:
fail.append(status.context)
@ -372,7 +368,7 @@ def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None
description = "failed: " + ", ".join(fail)
description = format_description(description)
if mergeable_status is None or mergeable_status.description != description:
set_mergeable_check(commit, description, "failure")
set_mergeable_check(commit, description, FAILURE)
return
description = ", ".join(success)

View File

@ -6,7 +6,8 @@ import time
import subprocess
import logging
from typing import List, Optional
from pathlib import Path
from typing import List, Optional, Union
class DockerImage:
@ -22,7 +23,7 @@ class DockerImage:
def get_images_with_versions(
reports_path: str,
reports_path: Union[Path, str],
required_images: List[str],
pull: bool = True,
version: Optional[str] = None,
@ -80,7 +81,10 @@ def get_images_with_versions(
def get_image_with_version(
reports_path: str, image: str, pull: bool = True, version: Optional[str] = None
reports_path: Union[Path, str],
image: str,
pull: bool = True,
version: Optional[str] = None,
) -> DockerImage:
logging.info("Looking for images file in %s", reports_path)
return get_images_with_versions(reports_path, [image], pull, version=version)[0]

View File

@ -1,6 +1,9 @@
#!/usr/bin/env python
import logging
import os
from os import path as p
from typing import Tuple
from build_download_helper import get_gh_api
@ -40,13 +43,27 @@ _GITHUB_JOB_URL = ""
def GITHUB_JOB_ID() -> str:
global _GITHUB_JOB_ID
global _GITHUB_JOB_URL
if GITHUB_RUN_ID == "0":
_GITHUB_JOB_ID = "0"
if _GITHUB_JOB_ID:
return _GITHUB_JOB_ID
_GITHUB_JOB_ID, _GITHUB_JOB_URL = get_job_id_url(GITHUB_JOB)
return _GITHUB_JOB_ID
def GITHUB_JOB_URL() -> str:
GITHUB_JOB_ID()
return _GITHUB_JOB_URL
def get_job_id_url(job_name: str) -> Tuple[str, str]:
job_id = ""
job_url = ""
if GITHUB_RUN_ID == "0":
job_id = "0"
if job_id:
return job_id, job_url
jobs = []
page = 1
while not _GITHUB_JOB_ID:
while not job_id:
response = get_gh_api(
f"https://api.github.com/repos/{GITHUB_REPOSITORY}/"
f"actions/runs/{GITHUB_RUN_ID}/jobs?per_page=100&page={page}"
@ -55,46 +72,41 @@ def GITHUB_JOB_ID() -> str:
data = response.json()
jobs.extend(data["jobs"])
for job in data["jobs"]:
if job["name"] != GITHUB_JOB:
if job["name"] != job_name:
continue
_GITHUB_JOB_ID = job["id"]
_GITHUB_JOB_URL = job["html_url"]
return _GITHUB_JOB_ID
job_id = job["id"]
job_url = job["html_url"]
return job_id, job_url
if (
len(jobs) >= data["total_count"] # just in case of inconsistency
or len(data["jobs"]) == 0 # if we excided pages
):
_GITHUB_JOB_ID = "0"
job_id = "0"
# FIXME: until it's here, we can't move to reusable workflows
if not _GITHUB_JOB_URL:
if not job_url:
# This is a terrible workaround for the case of another broken part of
# GitHub actions. For nested workflows it doesn't provide a proper GITHUB_JOB
# GitHub actions. For nested workflows it doesn't provide a proper job_name
# value, but only the final one. So, for `OriginalJob / NestedJob / FinalJob`
# full name, GITHUB_JOB contains only FinalJob
# full name, job_name contains only FinalJob
matched_jobs = []
for job in jobs:
nested_parts = job["name"].split(" / ")
if len(nested_parts) <= 1:
continue
if nested_parts[-1] == GITHUB_JOB:
if nested_parts[-1] == job_name:
matched_jobs.append(job)
if len(matched_jobs) == 1:
# The best case scenario
_GITHUB_JOB_ID = matched_jobs[0]["id"]
_GITHUB_JOB_URL = matched_jobs[0]["html_url"]
return _GITHUB_JOB_ID
job_id = matched_jobs[0]["id"]
job_url = matched_jobs[0]["html_url"]
return job_id, job_url
if matched_jobs:
logging.error(
"We could not get the ID and URL for the current job name %s, there "
"are more than one jobs match it for the nested workflows. Please, "
"refer to https://github.com/actions/runner/issues/2577",
GITHUB_JOB,
job_name,
)
return _GITHUB_JOB_ID
def GITHUB_JOB_URL() -> str:
GITHUB_JOB_ID()
return _GITHUB_JOB_URL
return job_id, job_url

View File

@ -54,23 +54,21 @@ def get_fasttest_cmd(workspace, output_path, repo_path, pr_number, commit_sha, i
)
def process_results(result_folder: str) -> Tuple[str, str, TestResults, List[str]]:
def process_results(result_folder: Path) -> Tuple[str, str, TestResults, List[str]]:
test_results = [] # type: TestResults
additional_files = []
# Just upload all files from result_folder.
# If task provides processed results, then it's responsible for content of
# result_folder
if os.path.exists(result_folder):
if result_folder.exists():
test_files = [
f
for f in os.listdir(result_folder)
if os.path.isfile(os.path.join(result_folder, f))
]
additional_files = [os.path.join(result_folder, f) for f in test_files]
f for f in result_folder.iterdir() if f.is_file()
] # type: List[Path]
additional_files = [f.absolute().as_posix() for f in test_files]
status = []
status_path = os.path.join(result_folder, "check_status.tsv")
if os.path.exists(status_path):
status_path = result_folder / "check_status.tsv"
if status_path.exists():
logging.info("Found test_results.tsv")
with open(status_path, "r", encoding="utf-8") as status_file:
status = list(csv.reader(status_file, delimiter="\t"))
@ -80,7 +78,7 @@ def process_results(result_folder: str) -> Tuple[str, str, TestResults, List[str
state, description = status[0][0], status[0][1]
try:
results_path = Path(result_folder) / "test_results.tsv"
results_path = result_folder / "test_results.tsv"
test_results = read_test_results(results_path)
if len(test_results) == 0:
return "error", "Empty test_results.tsv", test_results, additional_files
@ -100,10 +98,9 @@ def main():
stopwatch = Stopwatch()
temp_path = TEMP_PATH
temp_path = Path(TEMP_PATH)
if not os.path.exists(temp_path):
os.makedirs(temp_path)
temp_path.mkdir(parents=True, exist_ok=True)
pr_info = PRInfo()
@ -124,17 +121,14 @@ def main():
s3_helper = S3Helper()
workspace = os.path.join(temp_path, "fasttest-workspace")
if not os.path.exists(workspace):
os.makedirs(workspace)
workspace = temp_path / "fasttest-workspace"
workspace.mkdir(parents=True, exist_ok=True)
output_path = os.path.join(temp_path, "fasttest-output")
if not os.path.exists(output_path):
os.makedirs(output_path)
output_path = temp_path / "fasttest-output"
output_path.mkdir(parents=True, exist_ok=True)
repo_path = os.path.join(temp_path, "fasttest-repo")
if not os.path.exists(repo_path):
os.makedirs(repo_path)
repo_path = temp_path / "fasttest-repo"
repo_path.mkdir(parents=True, exist_ok=True)
run_cmd = get_fasttest_cmd(
workspace,
@ -146,11 +140,10 @@ def main():
)
logging.info("Going to run fasttest with cmd %s", run_cmd)
logs_path = os.path.join(temp_path, "fasttest-logs")
if not os.path.exists(logs_path):
os.makedirs(logs_path)
logs_path = temp_path / "fasttest-logs"
logs_path.mkdir(parents=True, exist_ok=True)
run_log_path = os.path.join(logs_path, "run.log")
run_log_path = logs_path / "run.log"
with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process:
retcode = process.wait()
if retcode == 0:
@ -161,9 +154,7 @@ def main():
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
test_output_files = os.listdir(output_path)
additional_logs = []
for f in test_output_files:
additional_logs.append(os.path.join(output_path, f))
additional_logs = [os.path.join(output_path, f) for f in test_output_files]
test_log_exists = (
"test_log.txt" in test_output_files or "test_result.txt" in test_output_files
@ -194,8 +185,8 @@ def main():
pr_info.sha,
"fast_tests",
)
build_urls = s3_helper.upload_build_folder_to_s3(
os.path.join(output_path, "binaries"),
build_urls = s3_helper.upload_build_directory_to_s3(
output_path / "binaries",
s3_path_prefix,
keep_dirs_in_s3_path=False,
upload_symlinks=False,
@ -206,7 +197,7 @@ def main():
pr_info.number,
pr_info.sha,
test_results,
[run_log_path] + additional_logs,
[run_log_path.as_posix()] + additional_logs,
NAME,
build_urls,
)

View File

@ -29,7 +29,7 @@ from docker_pull_helper import get_image_with_version, DockerImage
from env_helper import CI, TEMP_PATH as TEMP, REPORTS_PATH
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from report import TestResults, TestResult
from report import TestResults, TestResult, FAILURE, FAIL, OK, SUCCESS
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
@ -40,10 +40,6 @@ RPM_IMAGE = "clickhouse/install-rpm-test"
DEB_IMAGE = "clickhouse/install-deb-test"
TEMP_PATH = Path(TEMP)
LOGS_PATH = TEMP_PATH / "tests_logs"
SUCCESS = "success"
FAILURE = "failure"
OK = "OK"
FAIL = "FAIL"
def prepare_test_scripts():

View File

@ -7,6 +7,7 @@ import json
import subprocess
import traceback
import re
from pathlib import Path
from typing import Dict
from github import Github
@ -218,15 +219,17 @@ if __name__ == "__main__":
uploaded = {} # type: Dict[str, str]
for name, path in paths.items():
try:
uploaded[name] = s3_helper.upload_test_report_to_s3(path, s3_prefix + name)
uploaded[name] = s3_helper.upload_test_report_to_s3(
Path(path), s3_prefix + name
)
except Exception:
uploaded[name] = ""
traceback.print_exc()
# Upload all images and flamegraphs to S3
try:
s3_helper.upload_test_folder_to_s3(
os.path.join(result_path, "images"), s3_prefix + "images"
s3_helper.upload_test_directory_to_s3(
Path(result_path) / "images", s3_prefix + "images"
)
except Exception:
traceback.print_exc()

View File

@ -2,17 +2,58 @@
from ast import literal_eval
from dataclasses import dataclass
from pathlib import Path
from typing import List, Optional, Tuple
from typing import Dict, Final, Iterable, List, Literal, Optional, Tuple
from html import escape
import csv
import os
import datetime
import json
import logging
import os
from ci_config import BuildConfig, CI_CONFIG
from env_helper import get_job_id_url
logger = logging.getLogger(__name__)
ERROR: Final = "error"
FAILURE: Final = "failure"
PENDING: Final = "pending"
SUCCESS: Final = "success"
OK: Final = "OK"
FAIL: Final = "FAIL"
StatusType = Literal["error", "failure", "pending", "success"]
# The order of statuses from the worst to the best
_STATES = {ERROR: 0, FAILURE: 1, PENDING: 2, SUCCESS: 3}
def get_worst_status(statuses: Iterable[str]) -> str:
worst_status = None
for status in statuses:
if _STATES.get(status) is None:
continue
if worst_status is None:
worst_status = status
continue
if _STATES.get(status) < _STATES.get(worst_status):
worst_status = status
if worst_status == ERROR:
break
if worst_status is None:
return ""
return worst_status
### BEST FRONTEND PRACTICES BELOW
HTML_BASE_TEST_TEMPLATE = """
HEAD_HTML_TEMPLATE = """
<!DOCTYPE html>
<html>
<head>
<style>
:root {{
@ -98,15 +139,9 @@ tr:hover {{ filter: var(--tr-hover-filter); }}
<div class="main">
<span class="nowrap themes"><span id="toggle-dark">🌚</span><span id="toggle-light">🌞</span></span>
<h1><span class="gradient">{header}</span></h1>
<p class="links">
<a href="{raw_log_url}">{raw_log_name}</a>
<a href="{commit_url}">Commit</a>
{additional_urls}
<a href="{task_url}">Task (github actions)</a>
<a href="{job_url}">Job (github actions)</a>
</p>
{test_part}
<img id="fish" src="https://presentations.clickhouse.com/images/fish.png" />
"""
FOOTER_HTML_TEMPLATE = """<img id="fish" src="https://presentations.clickhouse.com/images/fish.png" />
<script type="text/javascript">
/// Straight from https://stackoverflow.com/questions/14267781/sorting-html-table-with-javascript
@ -161,6 +196,21 @@ tr:hover {{ filter: var(--tr-hover-filter); }}
</html>
"""
HTML_BASE_TEST_TEMPLATE = (
f"{HEAD_HTML_TEMPLATE}"
"""<p class="links">
<a href="{raw_log_url}">{raw_log_name}</a>
<a href="{commit_url}">Commit</a>
{additional_urls}
<a href="{task_url}">Task (github actions)</a>
<a href="{job_url}">Job (github actions)</a>
</p>
{test_part}
"""
f"{FOOTER_HTML_TEMPLATE}"
)
HTML_TEST_PART = """
<table>
<tr>
@ -238,12 +288,159 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes
@dataclass
class BuildResult:
compiler: str
debug_build: bool
sanitizer: str
status: str
build_name: str
log_url: str
build_urls: List[str]
version: str
status: StatusType
elapsed_seconds: int
comment: str
job_name: str
_job_link: Optional[str] = None
_grouped_urls: Optional[List[List[str]]] = None
@property
def build_config(self) -> Optional[BuildConfig]:
return CI_CONFIG.build_config.get(self.build_name, None)
@property
def comment(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.comment
@property
def compiler(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.compiler
@property
def debug_build(self) -> bool:
if self.build_config is None:
return False
return self.build_config.debug_build
@property
def sanitizer(self) -> str:
if self.build_config is None:
return self._wrong_config_message
return self.build_config.sanitizer
@property
def grouped_urls(self) -> List[List[str]]:
"Combine and preserve build_urls by artifact types"
if self._grouped_urls is not None:
return self._grouped_urls
if not self.build_urls:
self._grouped_urls = [[]]
return self._grouped_urls
artifacts_groups = {
"apk": [],
"deb": [],
"binary": [],
"tgz": [],
"rpm": [],
"performance": [],
} # type: Dict[str, List[str]]
for url in self.build_urls:
if url.endswith("performance.tar.zst"):
artifacts_groups["performance"].append(url)
elif (
url.endswith(".deb")
or url.endswith(".buildinfo")
or url.endswith(".changes")
or url.endswith(".tar.gz")
):
artifacts_groups["deb"].append(url)
elif url.endswith(".apk"):
artifacts_groups["apk"].append(url)
elif url.endswith(".rpm"):
artifacts_groups["rpm"].append(url)
elif url.endswith(".tgz") or url.endswith(".tgz.sha512"):
artifacts_groups["tgz"].append(url)
else:
artifacts_groups["binary"].append(url)
self._grouped_urls = [urls for urls in artifacts_groups.values() if urls]
return self._grouped_urls
@property
def _wrong_config_message(self) -> str:
return "missing"
@property
def file_name(self) -> Path:
return self.get_report_name(self.build_name)
@property
def is_missing(self) -> bool:
"The report is created for missing json file"
return not (
self.log_url
or self.build_urls
or self.version != "missing"
or self.status != ERROR
)
@property
def job_link(self) -> str:
if self._job_link is not None:
return self._job_link
_, job_url = get_job_id_url(self.job_name)
self._job_link = f'<a href="{job_url}">{self.job_name}</a>'
return self._job_link
@staticmethod
def get_report_name(name: str) -> Path:
return Path(f"build_report_{name}.json")
@staticmethod
def read_json(directory: Path, build_name: str) -> "BuildResult":
path = directory / BuildResult.get_report_name(build_name)
try:
with open(path, "r", encoding="utf-8") as pf:
data = json.load(pf) # type: dict
except FileNotFoundError:
logger.warning(
"File %s for build named '%s' is not found", path, build_name
)
return BuildResult.missing_result(build_name)
return BuildResult(
data.get("build_name", build_name),
data.get("log_url", ""),
data.get("build_urls", []),
data.get("version", ""),
data.get("status", ERROR),
data.get("elapsed_seconds", 0),
data.get("job_name", ""),
)
@staticmethod
def missing_result(build_name: str) -> "BuildResult":
return BuildResult(build_name, "", [], "missing", ERROR, 0, "missing")
def write_json(self, directory: Path) -> Path:
path = directory / self.file_name
path.write_text(
json.dumps(
{
"build_name": self.build_name,
"log_url": self.log_url,
"build_urls": self.build_urls,
"version": self.version,
"status": self.status,
"elapsed_seconds": self.elapsed_seconds,
"job_name": self.job_name,
}
),
encoding="utf-8",
)
# TODO: remove after the artifacts are in S3 completely
env_path = Path(os.getenv("GITHUB_ENV", "/dev/null"))
with env_path.open("a", encoding="utf-8") as ef:
ef.write(f"BUILD_URLS={path.stem}")
return path
BuildResults = List[BuildResult]
@ -281,8 +478,8 @@ def _format_header(
def _get_status_style(status: str, colortheme: Optional[ColorTheme] = None) -> str:
ok_statuses = ("OK", "success", "PASSED")
fail_statuses = ("FAIL", "failure", "error", "FAILED", "Timeout", "NOT_FAILED")
ok_statuses = (OK, SUCCESS, "PASSED")
fail_statuses = (FAIL, FAILURE, ERROR, "FAILED", "Timeout", "NOT_FAILED")
if colortheme is None:
colortheme = ReportColorTheme.default
@ -333,7 +530,7 @@ def create_test_html_report(
additional_urls = []
if test_results:
rows_part = ""
rows_part = []
num_fails = 0
has_test_time = False
has_log_urls = False
@ -348,11 +545,13 @@ def create_test_html_report(
if test_result.log_files is not None:
has_log_urls = True
row = "<tr>"
row = []
has_error = test_result.status in ("FAIL", "NOT_FAILED")
if has_error and test_result.raw_logs is not None:
row = '<tr class="failed">'
row += "<td>" + test_result.name + "</td>"
row.append('<tr class="failed">')
else:
row.append("<tr>")
row.append(f"<td>{test_result.name}</td>")
colspan += 1
style = _get_status_style(test_result.status, colortheme=statuscolors)
@ -362,12 +561,12 @@ def create_test_html_report(
num_fails = num_fails + 1
fail_id = f'id="fail{num_fails}" '
row += f'<td {fail_id}style="{style}">{test_result.status}</td>'
row.append(f'<td {fail_id}style="{style}">{test_result.status}</td>')
colspan += 1
if test_result.time is not None:
has_test_time = True
row += f"<td>{test_result.time}</td>"
row.append(f"<td>{test_result.time}</td>")
colspan += 1
if test_result.log_urls is not None:
@ -375,19 +574,19 @@ def create_test_html_report(
test_logs_html = "<br>".join(
[_get_html_url(url) for url in test_result.log_urls]
)
row += "<td>" + test_logs_html + "</td>"
row.append(f"<td>{test_logs_html}</td>")
colspan += 1
row += "</tr>"
rows_part += row
row.append("</tr>")
rows_part.append("".join(row))
if test_result.raw_logs is not None:
raw_logs = escape(test_result.raw_logs)
row = (
row_raw_logs = (
'<tr class="failed-content">'
f'<td colspan="{colspan}"><pre>{raw_logs}</pre></td>'
"</tr>"
)
rows_part += row
rows_part.append(row_raw_logs)
headers = BASE_HEADERS.copy()
if has_test_time:
@ -396,7 +595,7 @@ def create_test_html_report(
headers.append("Logs")
headers_html = "".join(["<th>" + h + "</th>" for h in headers])
test_part = HTML_TEST_PART.format(headers=headers_html, rows=rows_part)
test_part = HTML_TEST_PART.format(headers=headers_html, rows="".join(rows_part))
else:
test_part = ""
@ -423,31 +622,18 @@ def create_test_html_report(
return html
HTML_BASE_BUILD_TEMPLATE = """
<!DOCTYPE html>
<html>
<head>
<style>
body {{ font-family: "DejaVu Sans", "Noto Sans", Arial, sans-serif; background: #EEE; }}
h1 {{ margin-left: 10px; }}
th, td {{ border: 0; padding: 5px 10px 5px 10px; text-align: left; vertical-align: top; line-height: 1.5; background-color: #FFF;
border: 0; box-shadow: 0 0 0 1px rgba(0, 0, 0, 0.05), 0 8px 25px -5px rgba(0, 0, 0, 0.1); }}
a {{ color: #06F; text-decoration: none; }}
a:hover, a:active {{ color: #F40; text-decoration: underline; }}
table {{ border: 0; }}
.main {{ margin: auto; }}
p.links a {{ padding: 5px; margin: 3px; background: #FFF; line-height: 2; white-space: nowrap; box-shadow: 0 0 0 1px rgba(0, 0, 0, 0.05), 0 8px 25px -5px rgba(0, 0, 0, 0.1); }}
tr:hover td {{filter: brightness(95%);}}
</style>
<title>{title}</title>
</head>
<body>
<div class="main">
<h1>{header}</h1>
HTML_BASE_BUILD_TEMPLATE = (
f"{HEAD_HTML_TEMPLATE}"
"""<p class="links">
<a href="{commit_url}">Commit</a>
<a href="{task_url}">Task (github actions)</a>
</p>
<table>
<tr>
<th>Config/job name</th>
<th>Compiler</th>
<th>Build type</th>
<th>Version</th>
<th>Sanitizer</th>
<th>Status</th>
<th>Build log</th>
@ -457,13 +643,9 @@ tr:hover td {{filter: brightness(95%);}}
</tr>
{rows}
</table>
<p class="links">
<a href="{commit_url}">Commit</a>
<a href="{task_url}">Task (github actions)</a>
</p>
</body>
</html>
"""
f"{FOOTER_HTML_TEMPLATE}"
)
LINK_TEMPLATE = '<a href="{url}">{text}</a>'
@ -471,64 +653,63 @@ LINK_TEMPLATE = '<a href="{url}">{text}</a>'
def create_build_html_report(
header: str,
build_results: BuildResults,
build_logs_urls: List[str],
artifact_urls_list: List[List[str]],
task_url: str,
branch_url: str,
branch_name: str,
commit_url: str,
) -> str:
rows = ""
for build_result, build_log_url, artifact_urls in zip(
build_results, build_logs_urls, artifact_urls_list
):
row = "<tr>"
row += f"<td>{build_result.compiler}</td>"
if build_result.debug_build:
row += "<td>debug</td>"
else:
row += "<td>relwithdebuginfo</td>"
if build_result.sanitizer:
row += f"<td>{build_result.sanitizer}</td>"
else:
row += "<td>none</td>"
rows = []
for build_result in build_results:
for artifact_urls in build_result.grouped_urls:
row = ["<tr>"]
row.append(
f"<td>{build_result.build_name}<br/>{build_result.job_link}</td>"
)
row.append(f"<td>{build_result.compiler}</td>")
if build_result.debug_build:
row.append("<td>debug</td>")
else:
row.append("<td>relwithdebuginfo</td>")
row.append(f"<td>{build_result.version}</td>")
if build_result.sanitizer:
row.append(f"<td>{build_result.sanitizer}</td>")
else:
row.append("<td>none</td>")
if build_result.status:
style = _get_status_style(build_result.status)
row += f'<td style="{style}">{build_result.status}</td>'
else:
style = _get_status_style("error")
row += f'<td style="{style}">error</td>'
if build_result.status:
style = _get_status_style(build_result.status)
row.append(f'<td style="{style}">{build_result.status}</td>')
else:
style = _get_status_style(ERROR)
row.append(f'<td style="{style}">error</td>')
row += f'<td><a href="{build_log_url}">link</a></td>'
row.append(f'<td><a href="{build_result.log_url}">link</a></td>')
if build_result.elapsed_seconds:
delta = datetime.timedelta(seconds=build_result.elapsed_seconds)
else:
delta = "unknown" # type: ignore
delta = "unknown"
if build_result.elapsed_seconds:
delta = str(datetime.timedelta(seconds=build_result.elapsed_seconds))
row += f"<td>{delta}</td>"
row.append(f"<td>{delta}</td>")
links = ""
link_separator = "<br/>"
if artifact_urls:
for artifact_url in artifact_urls:
links += LINK_TEMPLATE.format(
text=_get_html_url_name(artifact_url), url=artifact_url
)
links += link_separator
if links:
links = links[: -len(link_separator)]
row += f"<td>{links}</td>"
links = []
link_separator = "<br/>"
if artifact_urls:
for artifact_url in artifact_urls:
links.append(
LINK_TEMPLATE.format(
text=_get_html_url_name(artifact_url), url=artifact_url
)
)
row.append(f"<td>{link_separator.join(links)}</td>")
row += f"<td>{build_result.comment}</td>"
row.append(f"<td>{build_result.comment}</td>")
row += "</tr>"
rows += row
row.append("</tr>")
rows.append("".join(row))
return HTML_BASE_BUILD_TEMPLATE.format(
title=_format_header(header, branch_name),
header=_format_header(header, branch_name, branch_url),
rows=rows,
rows="".join(rows),
task_url=task_url,
branch_name=branch_name,
commit_url=commit_url,

View File

@ -52,12 +52,14 @@ class S3Helper:
self.host = S3_URL
self.download_host = S3_DOWNLOAD
def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str:
def _upload_file_to_s3(
self, bucket_name: str, file_path: Path, s3_path: str
) -> str:
logging.debug(
"Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path
)
metadata = {}
if os.path.getsize(file_path) < 64 * 1024 * 1024:
if file_path.stat().st_size < 64 * 1024 * 1024:
if (
s3_path.endswith("txt")
or s3_path.endswith("log")
@ -97,17 +99,14 @@ class S3Helper:
if re.search(r"\.(txt|log|err|out)$", s3_path) or re.search(
r"\.log\..*(?<!\.zst)$", s3_path
):
compressed_path = file_path.with_suffix(file_path.suffix + ".zst")
logging.info(
"Going to compress file log file %s to %s",
file_path,
file_path + ".zst",
compressed_path,
)
# FIXME: rewrite S3 to Path
_file_path = Path(file_path)
compress_file_fast(
_file_path, _file_path.with_suffix(_file_path.suffix + ".zst")
)
file_path += ".zst"
compress_file_fast(file_path, compressed_path)
file_path = compressed_path
s3_path += ".zst"
else:
logging.info("Processing file without compression")
@ -121,22 +120,20 @@ class S3Helper:
logging.info("Upload %s to %s. Meta: %s", file_path, url, metadata)
return url
def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str:
def upload_test_report_to_s3(self, file_path: Path, s3_path: str) -> str:
if CI:
return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path)
else:
return S3Helper.copy_file_to_local(
S3_TEST_REPORTS_BUCKET, file_path, s3_path
)
def upload_build_file_to_s3(self, file_path, s3_path):
return S3Helper.copy_file_to_local(S3_TEST_REPORTS_BUCKET, file_path, s3_path)
def upload_build_file_to_s3(self, file_path: Path, s3_path: str) -> str:
if CI:
return self._upload_file_to_s3(S3_BUILDS_BUCKET, file_path, s3_path)
else:
return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path)
return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path)
def fast_parallel_upload_dir(
self, dir_path: Union[str, Path], s3_dir_path: str, bucket_name: str
self, dir_path: Path, s3_dir_path: str, bucket_name: str
) -> List[str]:
all_files = []
@ -196,37 +193,37 @@ class S3Helper:
logging.basicConfig(level=original_level)
return result
def _upload_folder_to_s3(
def _upload_directory_to_s3(
self,
folder_path,
s3_folder_path,
bucket_name,
keep_dirs_in_s3_path,
upload_symlinks,
):
directory_path: Path,
s3_directory_path: str,
bucket_name: str,
keep_dirs_in_s3_path: bool,
upload_symlinks: bool,
) -> List[str]:
logging.info(
"Upload folder '%s' to bucket=%s of s3 folder '%s'",
folder_path,
"Upload directory '%s' to bucket=%s of s3 directory '%s'",
directory_path,
bucket_name,
s3_folder_path,
s3_directory_path,
)
if not os.path.exists(folder_path):
if not directory_path.exists():
return []
files = os.listdir(folder_path)
files = list(directory_path.iterdir())
if not files:
return []
p = Pool(min(len(files), 5))
def task(file_name):
full_fs_path = os.path.join(folder_path, file_name)
def task(file_path: Path) -> Union[str, List[str]]:
full_fs_path = file_path.absolute()
if keep_dirs_in_s3_path:
full_s3_path = s3_folder_path + "/" + os.path.basename(folder_path)
full_s3_path = os.path.join(s3_directory_path, directory_path.name)
else:
full_s3_path = s3_folder_path
full_s3_path = s3_directory_path
if os.path.isdir(full_fs_path):
return self._upload_folder_to_s3(
return self._upload_directory_to_s3(
full_fs_path,
full_s3_path,
bucket_name,
@ -234,60 +231,63 @@ class S3Helper:
upload_symlinks,
)
if os.path.islink(full_fs_path):
if full_fs_path.is_symlink():
if upload_symlinks:
if CI:
return self._upload_file_to_s3(
bucket_name, full_fs_path, full_s3_path + "/" + file_name
)
else:
return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_name
bucket_name,
full_fs_path,
full_s3_path + "/" + file_path.name,
)
return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
)
return []
if CI:
return self._upload_file_to_s3(
bucket_name, full_fs_path, full_s3_path + "/" + file_name
)
else:
return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_name
bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
)
return S3Helper.copy_file_to_local(
bucket_name, full_fs_path, full_s3_path + "/" + file_path.name
)
return sorted(_flatten_list(list(p.map(task, files))))
def upload_build_folder_to_s3(
def upload_build_directory_to_s3(
self,
folder_path,
s3_folder_path,
keep_dirs_in_s3_path=True,
upload_symlinks=True,
):
return self._upload_folder_to_s3(
folder_path,
s3_folder_path,
directory_path: Path,
s3_directory_path: str,
keep_dirs_in_s3_path: bool = True,
upload_symlinks: bool = True,
) -> List[str]:
return self._upload_directory_to_s3(
directory_path,
s3_directory_path,
S3_BUILDS_BUCKET,
keep_dirs_in_s3_path,
upload_symlinks,
)
def upload_test_folder_to_s3(
def upload_test_directory_to_s3(
self,
folder_path,
s3_folder_path,
keep_dirs_in_s3_path=True,
upload_symlinks=True,
):
return self._upload_folder_to_s3(
folder_path,
s3_folder_path,
directory_path: Path,
s3_directory_path: str,
keep_dirs_in_s3_path: bool = True,
upload_symlinks: bool = True,
) -> List[str]:
return self._upload_directory_to_s3(
directory_path,
s3_directory_path,
S3_TEST_REPORTS_BUCKET,
keep_dirs_in_s3_path,
upload_symlinks,
)
def list_prefix(self, s3_prefix_path, bucket=S3_BUILDS_BUCKET):
def list_prefix(
self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET
) -> List[str]:
objects = self.client.list_objects_v2(Bucket=bucket, Prefix=s3_prefix_path)
result = []
if "Contents" in objects:
@ -296,7 +296,7 @@ class S3Helper:
return result
def exists(self, key, bucket=S3_BUILDS_BUCKET):
def exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> bool:
try:
self.client.head_object(Bucket=bucket, Key=key)
return True
@ -304,13 +304,12 @@ class S3Helper:
return False
@staticmethod
def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str:
local_path = os.path.abspath(
os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path)
)
local_dir = os.path.dirname(local_path)
if not os.path.exists(local_dir):
os.makedirs(local_dir)
def copy_file_to_local(bucket_name: str, file_path: Path, s3_path: str) -> str:
local_path = (
Path(RUNNER_TEMP) / "s3" / os.path.join(bucket_name, s3_path)
).absolute()
local_dir = local_path.parent
local_dir.mkdir(parents=True, exist_ok=True)
shutil.copy(file_path, local_path)
logging.info("Copied %s to %s", file_path, local_path)

View File

@ -4,6 +4,8 @@ import logging
import subprocess
import os
import sys
from pathlib import Path
from typing import Dict
from github import Github
@ -47,13 +49,12 @@ def main():
stopwatch = Stopwatch()
temp_path = TEMP_PATH
reports_path = REPORTS_PATH
temp_path = Path(TEMP_PATH)
reports_path = Path(REPORTS_PATH)
check_name = sys.argv[1]
if not os.path.exists(temp_path):
os.makedirs(temp_path)
temp_path.mkdir(parents=True, exist_ok=True)
pr_info = PRInfo()
@ -82,7 +83,7 @@ def main():
logging.info("Got build url %s", build_url)
workspace_path = os.path.join(temp_path, "workspace")
workspace_path = temp_path / "workspace"
if not os.path.exists(workspace_path):
os.makedirs(workspace_path)
@ -91,7 +92,7 @@ def main():
)
logging.info("Going to run %s", run_command)
run_log_path = os.path.join(temp_path, "run.log")
run_log_path = temp_path / "run.log"
with open(run_log_path, "w", encoding="utf-8") as log:
with subprocess.Popen(
run_command, shell=True, stderr=log, stdout=log
@ -110,23 +111,24 @@ def main():
s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/"
paths = {
"run.log": run_log_path,
"server.log.zst": os.path.join(workspace_path, "server.log.zst"),
"server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"),
"report.html": os.path.join(workspace_path, "report.html"),
"test.log": os.path.join(workspace_path, "test.log"),
"server.log.zst": workspace_path / "server.log.zst",
"server.err.log.zst": workspace_path / "server.err.log.zst",
"report.html": workspace_path / "report.html",
"test.log": workspace_path / "test.log",
}
path_urls = {} # type: Dict[str, str]
s3_helper = S3Helper()
for f in paths:
try:
paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f)
path_urls[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f)
except Exception as ex:
logging.info("Exception uploading file %s text %s", f, ex)
paths[f] = ""
path_urls[f] = ""
report_url = GITHUB_RUN_URL
if paths["report.html"]:
report_url = paths["report.html"]
if path_urls["report.html"]:
report_url = path_urls["report.html"]
status = "success"
description = "See the report"

View File

@ -34,7 +34,7 @@ def process_logs(
test_result.log_urls.append(processed_logs[path])
elif path:
url = s3_client.upload_test_report_to_s3(
path.as_posix(), s3_path_prefix + "/" + path.name
path, s3_path_prefix + "/" + path.name
)
test_result.log_urls.append(url)
processed_logs[path] = url
@ -44,7 +44,7 @@ def process_logs(
if log_path:
additional_urls.append(
s3_client.upload_test_report_to_s3(
log_path, s3_path_prefix + "/" + os.path.basename(log_path)
Path(log_path), s3_path_prefix + "/" + os.path.basename(log_path)
)
)
@ -100,9 +100,9 @@ def upload_results(
additional_urls,
statuscolors=statuscolors,
)
with open("report.html", "w", encoding="utf-8") as f:
f.write(html_report)
report_path = Path("report.html")
report_path.write_text(html_report, encoding="utf-8")
url = s3_client.upload_test_report_to_s3("report.html", s3_path_prefix + ".html")
url = s3_client.upload_test_report_to_s3(report_path, s3_path_prefix + ".html")
logging.info("Search result in url %s", url)
return url

View File

@ -245,8 +245,10 @@ def get_version_from_string(
def get_version_from_tag(tag: str) -> ClickHouseVersion:
Git.check_tag(tag)
tag = tag[1:].split("-")[0]
return get_version_from_string(tag)
tag, description = tag[1:].split("-", 1)
version = get_version_from_string(tag)
version.with_description(description)
return version
def version_arg(version: str) -> ClickHouseVersion: