2023-08-06 00:53:11 +00:00
#!/bin/bash
# This script sets up export of system log tables to a remote server.
# Remote tables are created if not exist, and augmented with extra columns,
# and their names will contain a hash of the table structure,
# which allows exporting tables from servers of different versions.
2023-08-16 20:53:51 +00:00
# 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 }
2023-08-06 00:53:11 +00:00
# Pre-configured destination cluster, where to export the data
2023-08-16 20:53:51 +00:00
CLICKHOUSE_CI_LOGS_CLUSTER = ${ CLICKHOUSE_CI_LOGS_CLUSTER :- system_logs_export }
2023-08-06 00:53:11 +00:00
2023-11-11 06:37:39 +00:00
EXTRA_COLUMNS = ${ EXTRA_COLUMNS :- "pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, " }
2023-11-11 06:46:35 +00:00
EXTRA_COLUMNS_EXPRESSION = ${ EXTRA_COLUMNS_EXPRESSION :- "CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id" }
2024-01-20 07:59:29 +00:00
EXTRA_ORDER_BY_COLUMNS = ${ EXTRA_ORDER_BY_COLUMNS :- "check_name" }
2023-08-06 00:53:11 +00:00
2023-11-11 04:52:44 +00:00
# trace_log needs more columns for symbolization
2023-11-11 07:23:58 +00:00
EXTRA_COLUMNS_TRACE_LOG = " ${ EXTRA_COLUMNS } symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), "
2023-12-12 20:21:58 +00:00
EXTRA_COLUMNS_EXPRESSION_TRACE_LOG = " ${ EXTRA_COLUMNS_EXPRESSION } , arrayMap(x -> demangle(addressToSymbol(x)), trace)::Array(LowCardinality(String)) AS symbols, arrayMap(x -> addressToLine(x), trace)::Array(LowCardinality(String)) AS lines "
2023-11-11 04:52:44 +00:00
2024-01-15 04:07:23 +00:00
# coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate)
EXTRA_COLUMNS_COVERAGE_LOG = " ${ EXTRA_COLUMNS } symbols Array(LowCardinality(String)), "
EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG = " ${ EXTRA_COLUMNS_EXPRESSION } , arrayMap(x -> demangle(addressToSymbol(x)), coverage)::Array(LowCardinality(String)) AS symbols "
2023-11-11 04:52:44 +00:00
2023-08-29 20:05:04 +00:00
function __set_connection_args
2023-08-16 20:53:51 +00:00
{
2024-01-15 22:36:48 +00:00
# It's impossible to use a generic $CONNECTION_ARGS string, it's unsafe from word splitting perspective.
2023-08-29 20:05:04 +00:00
# That's why we must stick to the generated option
CONNECTION_ARGS = (
2023-09-01 09:25:42 +00:00
--receive_timeout= 45 --send_timeout= 45 --secure
2023-08-29 20:05:04 +00:00
--user " ${ CLICKHOUSE_CI_LOGS_USER } " --host " ${ CLICKHOUSE_CI_LOGS_HOST } "
--password " ${ CLICKHOUSE_CI_LOGS_PASSWORD } "
)
}
2023-08-16 20:53:51 +00:00
2023-08-29 20:05:04 +00:00
function __shadow_credentials
{
# The function completely screws the output, it shouldn't be used in normal functions, only in ()
2023-08-16 20:53:51 +00:00
# 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;
' )
2023-08-29 20:05:04 +00:00
}
function check_logs_credentials
(
# The function connects with given credentials, and if it's unable to execute the simplest query, returns exit code
# First check, if all necessary parameters are set
set +x
for parameter in CLICKHOUSE_CI_LOGS_HOST CLICKHOUSE_CI_LOGS_USER CLICKHOUSE_CI_LOGS_PASSWORD; do
export -p | grep -q " $parameter " || {
echo " Credentials parameter $parameter is unset "
return 1
}
done
__shadow_credentials
__set_connection_args
2023-08-16 20:53:51 +00:00
local code
# Catch both success and error to not fail on `set -e`
2023-08-29 20:05:04 +00:00
clickhouse-client " ${ CONNECTION_ARGS [@] } " -q 'SELECT 1 FORMAT Null' && return 0 || code = $?
2023-08-16 20:53:51 +00:00
if [ " $code " != 0 ] ; then
echo 'Failed to connect to CI Logs cluster'
return $code
fi
2023-08-29 20:05:04 +00:00
)
2023-08-16 20:53:51 +00:00
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
2023-08-29 20:05:04 +00:00
__shadow_credentials
2023-08-16 20:53:51 +00:00
echo "Checking if the credentials work"
2023-08-29 20:05:04 +00:00
check_logs_credentials || return 0
2023-08-16 20:53:51 +00:00
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 } "
)
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
2023-08-29 20:05:04 +00:00
__shadow_credentials
2023-08-16 20:53:51 +00:00
echo "Checking if the credentials work"
2023-08-29 20:05:04 +00:00
check_logs_credentials || return 0
__set_connection_args
2023-08-16 20:53:51 +00:00
echo 'Create all configured system logs'
clickhouse-client --query "SYSTEM FLUSH LOGS"
2023-11-22 20:09:07 +00:00
debug_or_sanitizer_build = $( clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'" )
echo " Build is debug or sanitizer: $debug_or_sanitizer_build "
2024-01-14 23:41:14 +00:00
# We will pre-create a table system.coverage_log.
# It is normally created by clickhouse-test rather than the server,
# so we will create it in advance to make it be picked up by the next commands:
clickhouse-client --query "
CREATE TABLE IF NOT EXISTS system.coverage_log
(
2024-02-08 21:11:37 +00:00
time DateTime COMMENT 'The time of test run' ,
test_name String COMMENT 'The name of the test' ,
coverage Array( UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run'
2024-05-07 07:33:47 +00:00
) ENGINE = MergeTree ORDER BY test_name COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster'
2024-01-14 23:41:14 +00:00
"
2023-08-16 20:53:51 +00:00
# For each system log table:
echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do
2023-11-11 04:52:44 +00:00
if [ [ " $table " = "trace_log" ] ]
then
EXTRA_COLUMNS_FOR_TABLE = " ${ EXTRA_COLUMNS_TRACE_LOG } "
2023-11-22 20:09:07 +00:00
# Do not try to resolve stack traces in case of debug/sanitizers
# build, since it is too slow (flushing of trace_log can take ~1min
# with such MV attached)
2024-01-17 22:17:53 +00:00
if [ [ " $debug_or_sanitizer_build " = 1 ] ]
then
2023-11-22 20:09:07 +00:00
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE = " ${ EXTRA_COLUMNS_EXPRESSION } "
else
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE = " ${ EXTRA_COLUMNS_EXPRESSION_TRACE_LOG } "
fi
2024-01-15 04:07:23 +00:00
elif [ [ " $table " = "coverage_log" ] ]
2024-02-07 19:28:37 +00:00
then
2024-01-15 04:07:23 +00:00
EXTRA_COLUMNS_FOR_TABLE = " ${ EXTRA_COLUMNS_COVERAGE_LOG } "
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE = " ${ EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG } "
2024-02-07 19:28:37 +00:00
else
2023-11-11 04:52:44 +00:00
EXTRA_COLUMNS_FOR_TABLE = " ${ EXTRA_COLUMNS } "
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE = " ${ EXTRA_COLUMNS_EXPRESSION } "
fi
2023-10-13 12:52:25 +00:00
# Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed:
2023-08-16 20:53:51 +00:00
hash = $( clickhouse-client --query "
2023-11-11 08:38:02 +00:00
SELECT sipHash64( 9, groupArray( ( name, type ) ) )
2023-08-16 20:53:51 +00:00
FROM ( SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table'
ORDER BY position)
" )
# Create the destination table with adapted name and structure:
statement = $( clickhouse-client --format TSVRaw --query " SHOW CREATE TABLE system. ${ table } " | sed -r -e '
2023-11-11 04:52:44 +00:00
s/^\( $/( '"$EXTRA_COLUMNS_FOR_TABLE"' /;
2024-01-20 07:59:29 +00:00
s/^ORDER BY ( ( [ ^\( ] .+?) | \( ( .+?) \) ) $/ORDER BY ( '"$EXTRA_ORDER_BY_COLUMNS"' , \2 \3 ) /;
2023-08-16 20:53:51 +00:00
s/^CREATE TABLE system\. \w +_log$/CREATE TABLE IF NOT EXISTS '"$table"' _'"$hash"' /;
/^TTL /d
' )
echo -e " Creating remote destination table ${ table } _ ${ hash } with statement:\n ${ statement } " >& 2
2023-08-29 20:05:04 +00:00
echo " $statement " | clickhouse-client --database_replicated_initial_query_timeout_sec= 10 \
2024-02-18 08:29:34 +00:00
--distributed_ddl_task_timeout= 30 --distributed_ddl_output_mode= throw_only_active \
2023-08-29 20:05:04 +00:00
" ${ CONNECTION_ARGS [@] } " || continue
2023-08-16 20:53:51 +00:00
echo " Creating table system. ${ table } _sender " >& 2
# Create Distributed table and materialized view to watch on the original table:
2024-01-15 05:26:54 +00:00
clickhouse-client --query "
2023-08-16 20:53:51 +00:00
CREATE TABLE system.${ table } _sender
ENGINE = Distributed( ${ CLICKHOUSE_CI_LOGS_CLUSTER } , default, ${ table } _${ hash } )
SETTINGS flush_on_detach = 0
EMPTY AS
2023-11-11 04:52:44 +00:00
SELECT ${ EXTRA_COLUMNS_EXPRESSION_FOR_TABLE } , *
2023-08-16 20:53:51 +00:00
FROM system.${ table }
" || continue
echo " Creating materialized view system. ${ table } _watcher " >& 2
2024-01-15 05:26:54 +00:00
clickhouse-client --query "
2023-08-16 20:53:51 +00:00
CREATE MATERIALIZED VIEW system.${ table } _watcher TO system.${ table } _sender AS
2023-11-11 04:52:44 +00:00
SELECT ${ EXTRA_COLUMNS_EXPRESSION_FOR_TABLE } , *
2023-08-16 20:53:51 +00:00
FROM system.${ table }
" || continue
done
)
Fix system.*_log in artifacts on CI
For now it is broken due to #53086:
Code: 701. DB::Exception: Requested cluster 'system_logs_export' not found: Cannot attach table `system`.`query_views_log_sender` from metadata file /var/lib/clickhouse/store/c05/c05d0eb0-ac48-4b02-aea4-f05cf4f875a5/query_views_log_sender.sql from query ATTACH TABLE system.query_views_log_sender UUID 'ffd9aed6-344c-4b1b-8444-287d82c5a712' (`pull_request_number` UInt32, `commit_sha` String, `check_start_time` DateTime('UTC'), `check_name` LowCardinality(String), `instance_type` LowCardinality(String), `instance_id` String, `event_date` Date, `event_time` DateTime, `event_time_microseconds` DateTime64(6), `view_duration_ms` UInt64, `initial_query_id` String, `view_name` String, `view_uuid` UUID, `view_type` Enum8('Default' = 1, 'Materialized' = 2, 'Live' = 3, 'Window' = 4), `view_query` String, `view_target` String, `read_rows` UInt64, `read_bytes` UInt64, `written_rows` UInt64, `written_bytes` UInt64, `peak_memory_usage` Int64, `ProfileEvents` Map(String, UInt64), `status` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4), `exception_code` Int32, `exception` String, `stack_trace` String) ENGINE = Distributed('system_logs_export', 'default', 'query_views_log_17512133378547479980') SETTINGS flush_on_detach = 0. (CLUSTER_DOESNT_EXIST)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-22 19:58:02 +00:00
function stop_logs_replication
{
echo "Detach all logs replication"
clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | {
tee /dev/stderr
} | {
xargs -n1 -r -i clickhouse-client --query "drop table {}"
}
}