Rework setup of CI logs export

This commit is contained in:
Mikhail f. Shiryaev 2023-08-16 22:53:51 +02:00
parent a683d5e2f3
commit f0c18d4bd7
No known key found for this signature in database
GPG Key ID: 4B02ED204C7D93F4
9 changed files with 311 additions and 305 deletions

View File

@ -5,61 +5,150 @@
# and their names will contain a hash of the table structure,
# which allows exporting tables from servers of different versions.
# Config file contains KEY=VALUE pairs with any necessary parameters like:
# CLICKHOUSE_CI_LOGS_HOST - remote host
# CLICKHOUSE_CI_LOGS_USER - password for user
# CLICKHOUSE_CI_LOGS_PASSWORD - password for user
CLICKHOUSE_CI_LOGS_CREDENTIALS=${CLICKHOUSE_CI_LOGS_CREDENTIALS:-/tmp/export-logs-config.sh}
CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci}
# Pre-configured destination cluster, where to export the data
CLUSTER=${CLUSTER:-system_logs_export}
CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type"}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "}
CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:-""}
function _check_logs_credentials
{
# The function connects with given credentials, and if it's unable to execute the simplest query, returns exit code
# Create all configured system logs:
clickhouse-client --query "SYSTEM FLUSH LOGS"
# The only way to substitute the env as a plain text is using perl 's/\Qsomething\E/another/
exec &> >(perl -pe '
s(\Q$ENV{CLICKHOUSE_CI_LOGS_HOST}\E)[CLICKHOUSE_CI_LOGS_HOST]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_USER}\E)[CLICKHOUSE_CI_LOGS_USER]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_PASSWORD}\E)[CLICKHOUSE_CI_LOGS_PASSWORD]g;
')
CONNECTION_ARGS=(--secure --user "${CLICKHOUSE_CI_LOGS_USER}" --host "${CLICKHOUSE_CI_LOGS_HOST}" --password "${CLICKHOUSE_CI_LOGS_PASSWORD}")
local code
# Catch both success and error to not fail on `set -e`
clickhouse-client "${CONNECTION_ARGS[@]}" -q 'SELECT 1' && return 0 || code=$?
if [ "$code" != 0 ]; then
echo 'Failed to connect to CI Logs cluster'
return $code
fi
}
# It's doesn't make sense to try creating tables if SYNC fails
echo "SYSTEM SYNC DATABASE REPLICA default" | clickhouse-client --receive_timeout 180 $CONNECTION_PARAMETERS || exit 0
function config_logs_export_cluster
(
# The function is launched in a separate shell instance to not expose the
# exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS
set +x
if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then
echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup"
return
fi
set -a
# shellcheck disable=SC1090
source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}"
set +a
# The only way to substitute the env as a plain text is using perl 's/\Qsomething\E/another/
exec &> >(perl -pe '
s(\Q$ENV{CLICKHOUSE_CI_LOGS_HOST}\E)[CLICKHOUSE_CI_LOGS_HOST]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_USER}\E)[CLICKHOUSE_CI_LOGS_USER]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_PASSWORD}\E)[CLICKHOUSE_CI_LOGS_PASSWORD]g;
')
echo "Checking if the credentials work"
_check_logs_credentials || return 0
cluster_config="${1:-/etc/clickhouse-server/config.d/system_logs_export.yaml}"
mkdir -p "$(dirname "$cluster_config")"
echo "remote_servers:
${CLICKHOUSE_CI_LOGS_CLUSTER}:
shard:
replica:
secure: 1
user: '${CLICKHOUSE_CI_LOGS_USER}'
host: '${CLICKHOUSE_CI_LOGS_HOST}'
port: 9440
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
" > "$cluster_config"
echo "Cluster ${CLICKHOUSE_CI_LOGS_CLUSTER} is confugured in ${cluster_config}"
)
# For each system log table:
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do
# Calculate hash of its structure:
hash=$(clickhouse-client --query "
SELECT sipHash64(groupArray((name, type)))
FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table'
ORDER BY position)
")
function setup_logs_replication
(
# The function is launched in a separate shell instance to not expose the
# exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS
set +x
# disable output
if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then
echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup"
return 0
fi
set -a
# shellcheck disable=SC1090
source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}"
set +a
# The only way to substitute the env as a plain text is using perl 's/\Qsomething\E/another/
exec &> >(perl -pe '
s(\Q$ENV{CLICKHOUSE_CI_LOGS_HOST}\E)[CLICKHOUSE_CI_LOGS_HOST]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_USER}\E)[CLICKHOUSE_CI_LOGS_USER]g;
s(\Q$ENV{CLICKHOUSE_CI_LOGS_PASSWORD}\E)[CLICKHOUSE_CI_LOGS_PASSWORD]g;
')
echo "Checking if the credentials work"
_check_logs_credentials || return 0
# It's impossible to use generous $CONNECTION_ARGS string, it's unsafe from word splitting perspective.
# That's why we must stick to the generated option
CONNECTION_ARGS=(--secure --user "${CLICKHOUSE_CI_LOGS_USER}" --host "${CLICKHOUSE_CI_LOGS_HOST}" --password "${CLICKHOUSE_CI_LOGS_PASSWORD}")
# Create the destination table with adapted name and structure:
statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e '
s/^\($/('"$EXTRA_COLUMNS"'/;
s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/;
s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/;
/^TTL /d
')
echo 'Create all configured system logs'
clickhouse-client --query "SYSTEM FLUSH LOGS"
echo "Creating destination table ${table}_${hash}" >&2
# It's doesn't make sense to try creating tables if SYNC fails
echo "SYSTEM SYNC DATABASE REPLICA default" | clickhouse-client --receive_timeout 180 "${CONNECTION_ARGS[@]}" || return 0
echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 --receive_timeout=10 --send_timeout=10 $CONNECTION_PARAMETERS || continue
# For each system log table:
echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do
# Calculate hash of its structure:
hash=$(clickhouse-client --query "
SELECT sipHash64(groupArray((name, type)))
FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table'
ORDER BY position)
")
echo "Creating table system.${table}_sender" >&2
# Create the destination table with adapted name and structure:
statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e '
s/^\($/('"$EXTRA_COLUMNS"'/;
s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/;
s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/;
/^TTL /d
')
# Create Distributed table and materialized view to watch on the original table:
clickhouse-client --query "
CREATE TABLE system.${table}_sender
ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash})
SETTINGS flush_on_detach=0
EMPTY AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, *
FROM system.${table}
"
echo -e "Creating remote destination table ${table}_${hash} with statement:\n${statement}" >&2
echo "Creating materialized view system.${table}_watcher" >&2
echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 --receive_timeout=10 --send_timeout=10 "${CONNECTION_ARGS[@]}" || continue
clickhouse-client --query "
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, *
FROM system.${table}
"
done
echo "Creating table system.${table}_sender" >&2
# Create Distributed table and materialized view to watch on the original table:
clickhouse-client --query "
CREATE TABLE system.${table}_sender
ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash})
SETTINGS flush_on_detach=0
EMPTY AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, *
FROM system.${table}
" || continue
echo "Creating materialized view system.${table}_watcher" >&2
clickhouse-client --query "
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS
SELECT ${EXTRA_COLUMNS_EXPRESSION}, *
FROM system.${table}
" || continue
done
)

View File

@ -1,6 +1,8 @@
#!/bin/bash
# shellcheck disable=SC2086,SC2001,SC2046,SC2030,SC2031,SC2010,SC2015
# shellcheck disable=SC1091
source /setup_export_logs.sh
set -x
# core.COMM.PID-TID
@ -123,22 +125,7 @@ EOL
</clickhouse>
EOL
# Setup a cluster for logs export to ClickHouse Cloud
# Note: these variables are provided to the Docker run command by the Python script in tests/ci
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
echo "
remote_servers:
system_logs_export:
shard:
replica:
secure: 1
user: ci
host: '${CLICKHOUSE_CI_LOGS_HOST}'
port: 9440
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
" > db/config.d/system_logs_export.yaml
fi
config_logs_export_cluster db/config.d/system_logs_export.yaml
}
function filter_exists_and_template
@ -242,20 +229,7 @@ quit
kill -0 $server_pid # This checks that it is our server that is started and not some other one
echo 'Server started and responded'
# Initialize export of system logs to ClickHouse Cloud
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
export EXTRA_COLUMNS_EXPRESSION="$PR_TO_TEST AS pull_request_number, '$SHA_TO_TEST' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type"
# TODO: Check if the password will appear in the logs.
export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}"
/setup_export_logs.sh
# Unset variables after use
export CONNECTION_PARAMETERS=''
export CLICKHOUSE_CI_LOGS_HOST=''
export CLICKHOUSE_CI_LOGS_PASSWORD=''
fi
setup_logs_replication
# SC2012: Use find instead of ls to better handle non-alphanumeric filenames. They are all alphanumeric.
# SC2046: Quote this to prevent word splitting. Actually I need word splitting.

View File

@ -1,5 +1,7 @@
#!/bin/bash
# shellcheck disable=SC1091
source /setup_export_logs.sh
set -e -x
# Choose random timezone for this test run
@ -20,21 +22,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
./setup_minio.sh stateful
# Setup a cluster for logs export to ClickHouse Cloud
# Note: these variables are provided to the Docker run command by the Python script in tests/ci
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
echo "
remote_servers:
system_logs_export:
shard:
replica:
secure: 1
user: ci
host: '${CLICKHOUSE_CI_LOGS_HOST}'
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
" > /etc/clickhouse-server/config.d/system_logs_export.yaml
fi
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
function start()
{
@ -82,20 +70,7 @@ function start()
start
# Initialize export of system logs to ClickHouse Cloud
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type"
# TODO: Check if the password will appear in the logs.
export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}"
./setup_export_logs.sh
# Unset variables after use
export CONNECTION_PARAMETERS=''
export CLICKHOUSE_CI_LOGS_HOST=''
export CLICKHOUSE_CI_LOGS_PASSWORD=''
fi
setup_logs_replication
# shellcheck disable=SC2086 # No quotes because I want to split it into words.
/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS

View File

@ -1,5 +1,8 @@
#!/bin/bash
# shellcheck disable=SC1091
source /setup_export_logs.sh
# fail on errors, verbose and export all env variables
set -e -x -a
@ -36,21 +39,7 @@ fi
./setup_minio.sh stateless
./setup_hdfs_minicluster.sh
# Setup a cluster for logs export to ClickHouse Cloud
# Note: these variables are provided to the Docker run command by the Python script in tests/ci
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
echo "
remote_servers:
system_logs_export:
shard:
replica:
secure: 1
user: ci
host: '${CLICKHOUSE_CI_LOGS_HOST}'
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
" > /etc/clickhouse-server/config.d/system_logs_export.yaml
fi
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
# For flaky check we also enable thread fuzzer
if [ "$NUM_TRIES" -gt "1" ]; then
@ -116,20 +105,7 @@ do
sleep 1
done
# Initialize export of system logs to ClickHouse Cloud
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type"
# TODO: Check if the password will appear in the logs.
export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}"
./setup_export_logs.sh
# Unset variables after use
export CONNECTION_PARAMETERS=''
export CLICKHOUSE_CI_LOGS_HOST=''
export CLICKHOUSE_CI_LOGS_PASSWORD=''
fi
setup_logs_replication
attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01

View File

@ -5,6 +5,8 @@
# Avoid overlaps with previous runs
dmesg --clear
# shellcheck disable=SC1091
source /setup_export_logs.sh
set -x
@ -51,38 +53,11 @@ configure
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
./setup_minio.sh stateless # to have a proper environment
# Setup a cluster for logs export to ClickHouse Cloud
# Note: these variables are provided to the Docker run command by the Python script in tests/ci
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
echo "
remote_servers:
system_logs_export:
shard:
replica:
secure: 1
user: ci
host: '${CLICKHOUSE_CI_LOGS_HOST}'
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
" > /etc/clickhouse-server/config.d/system_logs_export.yaml
fi
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
start
# Initialize export of system logs to ClickHouse Cloud
if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ]
then
export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type"
# TODO: Check if the password will appear in the logs.
export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}"
./setup_export_logs.sh
# Unset variables after use
export CONNECTION_PARAMETERS=''
export CLICKHOUSE_CI_LOGS_HOST=''
export CLICKHOUSE_CI_LOGS_PASSWORD=''
fi
setup_logs_replication
# shellcheck disable=SC2086 # No quotes because I want to split it into words.
/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS

View File

@ -4,14 +4,15 @@ import logging
import subprocess
import os
import sys
from pathlib import Path
from github import Github
from build_download_helper import get_build_name_for_check, read_build_urls
from clickhouse_helper import (
CiLogsCredentials,
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
get_instance_type,
)
from commit_status_helper import (
RerunHelper,
@ -19,7 +20,7 @@ from commit_status_helper import (
get_commit,
post_commit_status,
)
from docker_pull_helper import get_image_with_version
from docker_pull_helper import DockerImage, get_image_with_version
from env_helper import (
REPORTS_PATH,
TEMP_PATH,
@ -29,25 +30,23 @@ from pr_info import PRInfo
from report import TestResult
from s3_helper import S3Helper
from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
IMAGE_NAME = "clickhouse/fuzzer"
def get_run_command(
check_start_time, check_name, pr_number, sha, download_url, workspace_path, image
):
instance_type = get_instance_type()
pr_info: PRInfo,
build_url: str,
workspace_path: str,
ci_logs_args: str,
image: DockerImage,
) -> str:
envs = [
"-e CLICKHOUSE_CI_LOGS_HOST",
"-e CLICKHOUSE_CI_LOGS_PASSWORD",
f"-e CHECK_START_TIME='{check_start_time}'",
f"-e CHECK_NAME='{check_name}'",
f"-e INSTANCE_TYPE='{instance_type}'",
f"-e PR_TO_TEST={pr_number}",
f"-e SHA_TO_TEST={sha}",
f"-e BINARY_URL_TO_DOWNLOAD='{download_url}'",
f"-e PR_TO_TEST={pr_info.number}",
f"-e SHA_TO_TEST={pr_info.sha}",
f"-e BINARY_URL_TO_DOWNLOAD='{build_url}'",
]
env_str = " ".join(envs)
@ -57,6 +56,7 @@ def get_run_command(
# For sysctl
"--privileged "
"--network=host "
f"{ci_logs_args}"
f"--volume={workspace_path}:/workspace "
f"{env_str} "
"--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE "
@ -107,14 +107,16 @@ def main():
workspace_path = os.path.join(temp_path, "workspace")
if not os.path.exists(workspace_path):
os.makedirs(workspace_path)
ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh")
ci_logs_args = ci_logs_credentials.get_docker_arguments(
pr_info, stopwatch.start_time_str, check_name
)
run_command = get_run_command(
stopwatch.start_time_str,
check_name,
pr_info.number,
pr_info.sha,
pr_info,
build_url,
workspace_path,
ci_logs_args,
docker_image,
)
logging.info("Going to run %s", run_command)
@ -122,35 +124,15 @@ def main():
run_log_path = os.path.join(temp_path, "run.log")
main_log_path = os.path.join(workspace_path, "main.log")
with open(run_log_path, "w", encoding="utf-8") as log:
with subprocess.Popen(
run_command, shell=True, stderr=log, stdout=log
) as process:
retcode = process.wait()
if retcode == 0:
logging.info("Run successfully")
else:
logging.info("Run failed")
with TeePopen(run_command, run_log_path) as process:
retcode = process.wait()
if retcode == 0:
logging.info("Run successfully")
else:
logging.info("Run failed")
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
# Cleanup run log from the credentials of CI logs database.
# Note: a malicious user can still print them by splitting the value into parts.
# But we will be warned when a malicious user modifies CI script.
# Although they can also print them from inside tests.
# Nevertheless, the credentials of the CI logs have limited scope
# and does not provide access to sensitive info.
ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST")
ci_logs_password = os.getenv(
"CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD"
)
if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""):
subprocess.check_call(
f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'",
shell=True,
)
ci_logs_credentials.clean_ci_logs_from_credentials(Path(run_log_path))
check_name_lower = (
check_name.lower().replace("(", "").replace(")", "").replace(" ", "")

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
from pathlib import Path
from typing import Dict, List, Optional
import fileinput
import json
import logging
import time
@ -235,3 +236,89 @@ def prepare_tests_results_for_clickhouse(
result.append(current_row)
return result
class CiLogsCredentials:
def __init__(self, config_path: Path):
self.config_path = config_path
try:
self._host = get_parameter_from_ssm("clickhouse_ci_logs_host") # type: str
self._password = get_parameter_from_ssm(
"clickhouse_ci_logs_password"
) # type: str
except:
logging.warning(
"Unable to retreive host and/or password from smm, all other "
"methods will noop"
)
self._host = ""
self._password = ""
def create_ci_logs_credentials(self) -> None:
if not (self.host and self.password):
logging.info(
"Hostname or password for CI logs instance are unknown, "
"skipping creating of credentials file, removing existing"
)
self.config_path.unlink(missing_ok=True)
return
self.config_path.parent.mkdir(parents=True, exist_ok=True)
self.config_path.write_text(
f"CLICKHOUSE_CI_LOGS_HOST={self.host}\n"
"CLICKHOUSE_CI_LOGS_USER=ci\n"
f"CLICKHOUSE_CI_LOGS_PASSWORD={self.password}\n",
encoding="utf-8",
)
def get_docker_arguments(
self, pr_info: PRInfo, check_start_time: str, check_name: str
) -> str:
self.create_ci_logs_credentials()
if not self.config_path.exists():
logging.info("Do not use external logs pushing")
return ""
extra_columns = (
f"{pr_info.number} AS pull_request_number, '{pr_info.sha}' AS commit_sha, "
f"'{check_start_time}' AS check_start_time, '{check_name}' AS check_name, "
f"'{get_instance_type()}' AS instance_type"
)
return (
f'-e EXTRA_COLUMNS_EXPRESSION="{extra_columns}" '
f"-e CLICKHOUSE_CI_LOGS_CREDENTIALS=/tmp/export-logs-config.sh "
f"--volume={self.config_path.absolute()}:/tmp/export-logs-config.sh:ro "
)
def clean_ci_logs_from_credentials(self, log_path: Path) -> None:
if not (self.host or self.password):
logging.info(
"Hostname and password for CI logs instance are unknown, "
"skipping cleaning %s",
log_path,
)
return
def process_line(line: str) -> str:
if self.host and self.password:
return line.replace(self.host, "CLICKHOUSE_CI_LOGS_HOST").replace(
self.password, "CLICKHOUSE_CI_LOGS_PASSWORD"
)
if self.host:
return line.replace(self.host, "CLICKHOUSE_CI_LOGS_HOST")
# the remaining is self.password
return line.replace(self.password, "CLICKHOUSE_CI_LOGS_PASSWORD")
# errors="surrogateescape" require python 3.10.
# With ubuntu 22.04 we are safe
with fileinput.input(
log_path, inplace=True, errors="surrogateescape"
) as log_fd:
for line in log_fd:
print(process_line(line), end="")
@property
def host(self) -> str:
return self._host
@property
def password(self) -> str:
return self._password

View File

@ -15,9 +15,9 @@ from github import Github
from build_download_helper import download_all_deb_packages
from clickhouse_helper import (
CiLogsCredentials,
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
get_instance_type,
)
from commit_status_helper import (
NotSet,
@ -28,7 +28,7 @@ from commit_status_helper import (
post_commit_status_to_file,
update_mergeable_check,
)
from docker_pull_helper import get_image_with_version
from docker_pull_helper import DockerImage, get_image_with_version
from download_release_packages import download_last_release
from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH
from get_robot_token import get_best_robot_token
@ -74,19 +74,18 @@ def get_image_name(check_name):
def get_run_command(
pr_info,
check_start_time,
check_name,
builds_path,
repo_path,
result_path,
server_log_path,
kill_timeout,
additional_envs,
image,
flaky_check,
tests_to_run,
):
check_name: str,
builds_path: str,
repo_path: str,
result_path: str,
server_log_path: str,
kill_timeout: int,
additional_envs: List[str],
ci_logs_args: str,
image: DockerImage,
flaky_check: bool,
tests_to_run: List[str],
) -> str:
additional_options = ["--hung-check"]
additional_options.append("--print-time")
@ -104,39 +103,30 @@ def get_run_command(
]
if flaky_check:
envs += ["-e NUM_TRIES=100", "-e MAX_RUN_TIME=1800"]
envs.append("-e NUM_TRIES=100")
envs.append("-e MAX_RUN_TIME=1800")
envs += [f"-e {e}" for e in additional_envs]
instance_type = get_instance_type()
envs += [
"-e CLICKHOUSE_CI_LOGS_HOST",
"-e CLICKHOUSE_CI_LOGS_PASSWORD",
f"-e PULL_REQUEST_NUMBER='{pr_info.number}'",
f"-e COMMIT_SHA='{pr_info.sha}'",
f"-e CHECK_START_TIME='{check_start_time}'",
f"-e CHECK_NAME='{check_name}'",
f"-e INSTANCE_TYPE='{instance_type}'",
]
env_str = " ".join(envs)
volume_with_broken_test = (
f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt"
f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt "
if "analyzer" in check_name
else ""
)
return (
f"docker run --volume={builds_path}:/package_folder "
f"{ci_logs_args}"
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
f"{volume_with_broken_test} "
f"--volume={result_path}:/test_output --volume={server_log_path}:/var/log/clickhouse-server "
f"{volume_with_broken_test}"
f"--volume={result_path}:/test_output "
f"--volume={server_log_path}:/var/log/clickhouse-server "
f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}"
)
def get_tests_to_run(pr_info):
def get_tests_to_run(pr_info: PRInfo) -> List[str]:
result = set()
if pr_info.changed_files is None:
@ -346,9 +336,12 @@ def main():
if validate_bugfix_check:
additional_envs.append("GLOBAL_TAGS=no-random-settings")
ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh")
ci_logs_args = ci_logs_credentials.get_docker_arguments(
pr_info, stopwatch.start_time_str, check_name
)
run_command = get_run_command(
pr_info,
stopwatch.start_time_str,
check_name,
packages_path,
repo_path,
@ -356,6 +349,7 @@ def main():
server_log_path,
kill_timeout,
additional_envs,
ci_logs_args,
docker_image,
flaky_check,
tests_to_run,
@ -374,6 +368,7 @@ def main():
except subprocess.CalledProcessError:
logging.warning("Failed to change files owner in %s, ignoring it", temp_path)
ci_logs_credentials.clean_ci_logs_from_credentials(Path(run_log_path))
s3_helper = S3Helper()
state, description, test_results, additional_logs = process_results(
@ -383,23 +378,6 @@ def main():
ch_helper = ClickHouseHelper()
# Cleanup run log from the credentials of CI logs database.
# Note: a malicious user can still print them by splitting the value into parts.
# But we will be warned when a malicious user modifies CI script.
# Although they can also print them from inside tests.
# Nevertheless, the credentials of the CI logs have limited scope
# and does not provide access to sensitive info.
ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST")
ci_logs_password = os.getenv(
"CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD"
)
if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""):
subprocess.check_call(
f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'",
shell=True,
)
report_url = upload_results(
s3_helper,
pr_info.number,

View File

@ -12,12 +12,12 @@ from github import Github
from build_download_helper import download_all_deb_packages
from clickhouse_helper import (
CiLogsCredentials,
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
get_instance_type,
)
from commit_status_helper import RerunHelper, get_commit, post_commit_status
from docker_pull_helper import get_image_with_version
from docker_pull_helper import DockerImage, get_image_with_version
from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
@ -29,40 +29,24 @@ from upload_result_helper import upload_results
def get_run_command(
pr_info,
check_start_time,
check_name,
build_path,
result_folder,
repo_tests_path,
server_log_folder,
image,
):
instance_type = get_instance_type()
envs = [
# a static link, don't use S3_URL or S3_DOWNLOAD
"-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets'",
"-e CLICKHOUSE_CI_LOGS_HOST",
"-e CLICKHOUSE_CI_LOGS_PASSWORD",
f"-e PULL_REQUEST_NUMBER='{pr_info.number}'",
f"-e COMMIT_SHA='{pr_info.sha}'",
f"-e CHECK_START_TIME='{check_start_time}'",
f"-e CHECK_NAME='{check_name}'",
f"-e INSTANCE_TYPE='{instance_type}'",
]
env_str = " ".join(envs)
build_path: str,
result_path: str,
repo_tests_path: str,
server_log_path: str,
ci_logs_args: str,
image: DockerImage,
) -> str:
cmd = (
"docker run --cap-add=SYS_PTRACE "
f"{env_str} "
# For dmesg and sysctl
"--privileged "
# a static link, don't use S3_URL or S3_DOWNLOAD
"-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets' "
f"{ci_logs_args}"
f"--volume={build_path}:/package_folder "
f"--volume={result_folder}:/test_output "
f"--volume={result_path}:/test_output "
f"--volume={repo_tests_path}:/usr/share/clickhouse-test "
f"--volume={server_log_folder}:/var/log/clickhouse-server {image} "
f"--volume={server_log_path}:/var/log/clickhouse-server {image} "
)
return cmd
@ -170,15 +154,17 @@ def run_stress_test(docker_image_name):
os.makedirs(result_path)
run_log_path = os.path.join(temp_path, "run.log")
ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh")
ci_logs_args = ci_logs_credentials.get_docker_arguments(
pr_info, stopwatch.start_time_str, check_name
)
run_command = get_run_command(
pr_info,
stopwatch.start_time_str,
check_name,
packages_path,
result_path,
repo_tests_path,
server_log_path,
ci_logs_args,
docker_image,
)
logging.info("Going to run stress test: %s", run_command)
@ -191,6 +177,7 @@ def run_stress_test(docker_image_name):
logging.info("Run failed")
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
ci_logs_credentials.clean_ci_logs_from_credentials(Path(run_log_path))
s3_helper = S3Helper()
state, description, test_results, additional_logs = process_results(
@ -198,23 +185,6 @@ def run_stress_test(docker_image_name):
)
ch_helper = ClickHouseHelper()
# Cleanup run log from the credentials of CI logs database.
# Note: a malicious user can still print them by splitting the value into parts.
# But we will be warned when a malicious user modifies CI script.
# Although they can also print them from inside tests.
# Nevertheless, the credentials of the CI logs have limited scope
# and does not provide access to sensitive info.
ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST")
ci_logs_password = os.getenv(
"CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD"
)
if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""):
subprocess.check_call(
f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'",
shell=True,
)
report_url = upload_results(
s3_helper,
pr_info.number,