mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Rework setup of CI logs export
This commit is contained in:
parent
a683d5e2f3
commit
f0c18d4bd7
@ -5,24 +5,112 @@
|
||||
# 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
|
||||
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}")
|
||||
|
||||
echo 'Create all configured system logs'
|
||||
clickhouse-client --query "SYSTEM FLUSH LOGS"
|
||||
|
||||
# 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
|
||||
|
||||
# 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)))
|
||||
@ -39,21 +127,21 @@ do
|
||||
/^TTL /d
|
||||
')
|
||||
|
||||
echo "Creating destination table ${table}_${hash}" >&2
|
||||
echo -e "Creating remote destination table ${table}_${hash} with statement:\n${statement}" >&2
|
||||
|
||||
echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 --receive_timeout=10 --send_timeout=10 $CONNECTION_PARAMETERS || continue
|
||||
echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 --receive_timeout=10 --send_timeout=10 "${CONNECTION_ARGS[@]}" || continue
|
||||
|
||||
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(${CLUSTER}, default, ${table}_${hash})
|
||||
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
|
||||
|
||||
@ -61,5 +149,6 @@ do
|
||||
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS
|
||||
SELECT ${EXTRA_COLUMNS_EXPRESSION}, *
|
||||
FROM system.${table}
|
||||
"
|
||||
done
|
||||
" || continue
|
||||
done
|
||||
)
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
||||
|
@ -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
|
||||
|
@ -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,10 +124,7 @@ 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:
|
||||
with TeePopen(run_command, run_log_path) as process:
|
||||
retcode = process.wait()
|
||||
if retcode == 0:
|
||||
logging.info("Run successfully")
|
||||
@ -133,24 +132,7 @@ def main():
|
||||
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(" ", "")
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
Loading…
Reference in New Issue
Block a user