This commit is contained in:
Nikita Fomichev 2024-09-18 23:24:06 +03:00 committed by GitHub
commit 969ecd27af
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 287 additions and 119 deletions

View File

@ -110,6 +110,8 @@ function setup_logs_replication
# The function is launched in a separate shell instance to not expose the # The function is launched in a separate shell instance to not expose the
# exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS # exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS
set +x set +x
PORT=${1:-"9000"}
# disable output # disable output
if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then
echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup" echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup"
@ -127,16 +129,16 @@ function setup_logs_replication
echo "My hostname is ${HOSTNAME}" echo "My hostname is ${HOSTNAME}"
echo 'Create all configured system logs' echo 'Create all configured system logs'
clickhouse-client --query "SYSTEM FLUSH LOGS" clickhouse-client --port "$PORT" --query "SYSTEM FLUSH LOGS"
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%'") debug_or_sanitizer_build=$(clickhouse-client --port "$PORT" -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" echo "Build is debug or sanitizer: $debug_or_sanitizer_build"
# We will pre-create a table system.coverage_log. # We will pre-create a table system.coverage_log.
# It is normally created by clickhouse-test rather than the server, # 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: # so we will create it in advance to make it be picked up by the next commands:
clickhouse-client --query " clickhouse-client --port "$PORT" --query "
CREATE TABLE IF NOT EXISTS system.coverage_log CREATE TABLE IF NOT EXISTS system.coverage_log
( (
time DateTime COMMENT 'The time of test run', time DateTime COMMENT 'The time of test run',
@ -147,7 +149,7 @@ function setup_logs_replication
# For each system log table: # For each system log table:
echo 'Create %_log tables' echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table clickhouse-client --port "$PORT" --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
do do
if [[ "$table" = "trace_log" ]] if [[ "$table" = "trace_log" ]]
then then
@ -171,7 +173,7 @@ function setup_logs_replication
fi fi
# Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed:
hash=$(clickhouse-client --query " hash=$(clickhouse-client --port "$PORT" --query "
SELECT sipHash64(9, groupArray((name, type))) SELECT sipHash64(9, groupArray((name, type)))
FROM (SELECT name, type FROM system.columns FROM (SELECT name, type FROM system.columns
WHERE database = 'system' AND table = '$table' WHERE database = 'system' AND table = '$table'
@ -179,7 +181,7 @@ function setup_logs_replication
") ")
# Create the destination table with adapted name and structure: # Create the destination table with adapted name and structure:
statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' statement=$(clickhouse-client --port "$PORT" --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e '
s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/;
s/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/; s/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/;
s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/;
@ -205,7 +207,7 @@ function setup_logs_replication
echo "Creating table system.${table}_sender" >&2 echo "Creating table system.${table}_sender" >&2
# Create Distributed table and materialized view to watch on the original table: # Create Distributed table and materialized view to watch on the original table:
clickhouse-client --query " clickhouse-client --port "$PORT" --query "
CREATE TABLE system.${table}_sender CREATE TABLE system.${table}_sender
ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash})
SETTINGS flush_on_detach=0 SETTINGS flush_on_detach=0
@ -214,10 +216,20 @@ function setup_logs_replication
FROM system.${table} FROM system.${table}
" || continue " || continue
echo "Creating materialized view system.${table}_watcher" >&2 echo "Creating buffer table system.${table}_buffer" >&2
clickhouse-client --port "$PORT" --query "
CREATE TABLE system.${table}_buffer AS system.${table}_sender
ENGINE=Buffer(system, ${table}_sender,
/*num_layers*/ 1,
/*min_time*/ 60, /*max_time*/ 300,
/*min_rows*/ 10000, /*max_rows*/ 20000,
/*min_bytes*/ 500000, /*max_bytes*/ 2000000
)
" || continue
clickhouse-client --query " echo "Creating materialized view system.${table}_watcher" >&2
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS clickhouse-client --port "$PORT" --query "
CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_buffer AS
SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, *
FROM system.${table} FROM system.${table}
" || continue " || continue
@ -227,9 +239,10 @@ function setup_logs_replication
function stop_logs_replication function stop_logs_replication
{ {
echo "Detach all 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')" | { PORT=${1:-"9000"}
clickhouse-client --port "$PORT" --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | {
tee /dev/stderr tee /dev/stderr
} | { } | {
timeout --preserve-status --signal TERM --kill-after 5m 15m xargs -n1 -r -i clickhouse-client --query "drop table {}" timeout --preserve-status --signal TERM --kill-after 1m 10m xargs -n1 -P8 -r -i clickhouse-client --port "$PORT" --query "drop table {}"
} }
} }

View File

@ -323,22 +323,22 @@ class CI:
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2
), ),
JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_TSAN], num_batches=4 required_builds=[BuildNames.PACKAGE_TSAN], num_batches=3
), ),
JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_MSAN], num_batches=4 required_builds=[BuildNames.PACKAGE_MSAN], num_batches=3
), ),
JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=1
), ),
JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1
), ),
JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE], required_builds=[BuildNames.PACKAGE_RELEASE],
), ),
JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=6 required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=5
), ),
JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64], required_builds=[BuildNames.PACKAGE_AARCH64],
@ -351,7 +351,7 @@ class CI:
required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1 required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1
), ),
JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=3, release_only=True required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2, release_only=True
), ),
JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_TSAN], required_builds=[BuildNames.PACKAGE_TSAN],

View File

@ -1584,7 +1584,8 @@ class TestCase:
print("Cannot insert coverage data: ", str(e)) print("Cannot insert coverage data: ", str(e))
# Check for dumped coverage files # Check for dumped coverage files
file_pattern = "coverage.*" coverage_prefix = os.getenv("CLICKHOUSE_WRITE_COVERAGE", "coverage")
file_pattern = coverage_prefix + ".*"
matching_files = glob.glob(file_pattern) matching_files = glob.glob(file_pattern)
for file_path in matching_files: for file_path in matching_files:
try: try:
@ -1740,11 +1741,27 @@ class TestCase:
f"PORT {os.environ['CLICKHOUSE_PORT_TCP']}", f"PORT {os.environ['CLICKHOUSE_PORT_TCP']}",
"PORT 9000", "PORT 9000",
) )
replace_in_file(
self.stdout_file,
f"port {os.environ['CLICKHOUSE_PORT_TCP']}",
"port 9000",
)
replace_in_file( replace_in_file(
self.stdout_file, self.stdout_file,
f"localhost {os.environ['CLICKHOUSE_PORT_TCP']}", f"localhost {os.environ['CLICKHOUSE_PORT_TCP']}",
"localhost 9000", "localhost 9000",
) )
replace_in_file(
self.stdout_file,
f"127.0.0.1:{os.environ['CLICKHOUSE_PORT_TCP']}",
"127.0.0.1:9000",
)
if os.environ.get("CLICKHOUSE_PORT_MYSQL"):
replace_in_file(
self.stdout_file,
f"127.0.0.1:{os.environ['CLICKHOUSE_PORT_MYSQL']}",
"127.0.0.1:9004",
)
if os.environ.get("CLICKHOUSE_PORT_TCP_SECURE"): if os.environ.get("CLICKHOUSE_PORT_TCP_SECURE"):
replace_in_file( replace_in_file(
@ -2161,6 +2178,11 @@ class TestSuite:
else: else:
self.parallel_tests.append(test_name) self.parallel_tests.append(test_name)
if args.run_parallel_only:
self.sequential_tests = []
if args.run_no_parallel_only:
self.parallel_tests = []
def is_sequential_test(self, test_name): def is_sequential_test(self, test_name):
if args.sequential: if args.sequential:
if any(s in test_name for s in args.sequential): if any(s in test_name for s in args.sequential):
@ -2269,13 +2291,9 @@ class GlobalTimeout(Exception):
pass pass
def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool]): def run_tests_array(
( all_tests: List[str], num_tests: int, test_suite: TestSuite, is_concurrent: bool
all_tests, ):
num_tests,
test_suite,
is_concurrent,
) = all_tests_with_params
global stop_time global stop_time
global exit_code global exit_code
global server_died global server_died
@ -2621,94 +2639,81 @@ def run_tests_process(*args, **kwargs):
def do_run_tests(jobs, test_suite: TestSuite): def do_run_tests(jobs, test_suite: TestSuite):
if jobs > 1 and len(test_suite.parallel_tests) > 0: print(
print( "Found",
"Found", len(test_suite.parallel_tests),
len(test_suite.parallel_tests), "parallel tests and",
"parallel tests and", len(test_suite.sequential_tests),
len(test_suite.sequential_tests), "sequential tests",
"sequential tests", )
tests_n = max(len(test_suite.parallel_tests), 1)
jobs = min(jobs, tests_n)
# If we don't do random shuffling then there will be always
# nearly the same groups of test suites running concurrently.
# Thus, if there is a test within group which appears to be broken
# then it will affect all other tests in a non-random form.
# So each time a bad test fails - other tests from the group will also fail
# and this process will be more or less stable.
# It makes it more difficult to detect real flaky tests,
# because the distribution and the amount
# of failures will be nearly the same for all tests from the group.
random.shuffle(test_suite.parallel_tests)
batch_size = len(test_suite.parallel_tests) // jobs
manager = multiprocessing.Manager()
parallel_tests = manager.list()
parallel_tests.extend(test_suite.parallel_tests)
is_concurrent = (jobs > 1) or (args.run_parallel_only or args.run_no_parallel_only)
processes = []
for _ in range(jobs):
process = multiprocessing.Process(
target=run_tests_process,
args=(parallel_tests, batch_size, test_suite, is_concurrent),
) )
tests_n = len(test_suite.parallel_tests) processes.append(process)
jobs = min(jobs, tests_n) process.start()
# If we don't do random shuffling then there will be always while processes:
# nearly the same groups of test suites running concurrently. sys.stdout.flush()
# Thus, if there is a test within group which appears to be broken # Periodically check the server for hangs
# then it will affect all other tests in a non-random form. # and stop all processes in this case
# So each time a bad test fails - other tests from the group will also fail try:
# and this process will be more or less stable. clickhouse_execute(
# It makes it more difficult to detect real flaky tests, args,
# because the distribution and the amount query="SELECT 1 /*hung check*/",
# of failures will be nearly the same for all tests from the group. max_http_retries=20,
random.shuffle(test_suite.parallel_tests) timeout=10,
batch_size = len(test_suite.parallel_tests) // jobs
manager = multiprocessing.Manager()
parallel_tests = manager.list()
parallel_tests.extend(test_suite.parallel_tests)
processes = []
for _ in range(jobs):
process = multiprocessing.Process(
target=run_tests_process,
args=((parallel_tests, batch_size, test_suite, True),),
) )
processes.append(process)
process.start()
while processes:
sys.stdout.flush()
# Periodically check the server for hangs
# and stop all processes in this case
try:
clickhouse_execute(
args,
query="SELECT 1 /*hung check*/",
max_http_retries=20,
timeout=10,
)
except Exception:
print("Hung check failed")
server_died.set()
if server_died.is_set():
print("Server died, terminating all processes...")
kill_gdb_if_any()
# Wait for test results
sleep(args.timeout)
for p in processes:
if p.is_alive():
p.terminate()
break
for p in processes[:]:
if not p.is_alive():
processes.remove(p)
sleep(5) sleep(5)
run_tests_array( except Exception:
( print("Hung check failed")
test_suite.sequential_tests, server_died.set()
len(test_suite.sequential_tests),
test_suite,
False,
)
)
return len(test_suite.sequential_tests) + len(test_suite.parallel_tests) if server_died.is_set():
else: print("Server died, terminating all processes...")
num_tests = len(test_suite.all_tests) kill_gdb_if_any()
run_tests_array( # Wait for test results
( sleep(args.timeout)
test_suite.all_tests, for p in processes:
num_tests, if p.is_alive():
test_suite, p.terminate()
False, break
)
) for p in processes[:]:
return num_tests if not p.is_alive():
processes.remove(p)
run_tests_array(
test_suite.sequential_tests,
len(test_suite.sequential_tests),
test_suite,
is_concurrent,
)
return len(test_suite.sequential_tests) + len(test_suite.parallel_tests)
def is_test_from_dir(suite_dir, case): def is_test_from_dir(suite_dir, case):
@ -3552,6 +3557,19 @@ def parse_args():
help="Capture stacktraces from clickhouse-client/local on errors", help="Capture stacktraces from clickhouse-client/local on errors",
) )
parser.add_argument(
"--run-parallel-only",
action="store_true",
default=False,
help="",
)
parser.add_argument(
"--run-no-parallel-only",
action="store_true",
default=False,
help="",
)
return parser.parse_args() return parser.parse_args()

View File

@ -0,0 +1,7 @@
<clickhouse>
<user_directories>
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
</clickhouse>

View File

@ -70,6 +70,7 @@ ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/user_directories.xml $DEST_SERVER_PATH/config.d/
# Not supported with fasttest. # Not supported with fasttest.
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ]

View File

@ -17,7 +17,11 @@ set -e -x -a
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
fi
# Choose random timezone for this test run. # Choose random timezone for this test run.
# #
# NOTE: that clickhouse-test will randomize session_timezone by itself as well # NOTE: that clickhouse-test will randomize session_timezone by itself as well
@ -108,6 +112,59 @@ if [ "$NUM_TRIES" -gt "1" ]; then
mkdir -p /var/run/clickhouse-server mkdir -p /var/run/clickhouse-server
fi fi
# Run a CH instance to execute sequential tests on it in parallel with all other tests.
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
mkdir -p /var/run/clickhouse-server3 /etc/clickhouse-server3 /var/lib/clickhouse3
cp -r -L /etc/clickhouse-server/* /etc/clickhouse-server3/
sudo chown clickhouse:clickhouse /var/run/clickhouse-server3 /var/lib/clickhouse3 /etc/clickhouse-server3/
sudo chown -R clickhouse:clickhouse /etc/clickhouse-server3/*
function replace(){
sudo find /etc/clickhouse-server3/ -type f -name '*.xml' -exec sed -i "$1" {} \;
}
replace "s|<port>9000</port>|<port>19000</port>|g"
replace "s|<port>9440</port>|<port>19440</port>|g"
replace "s|<port>9988</port>|<port>19988</port>|g"
replace "s|<port>9234</port>|<port>19234</port>|g"
replace "s|<port>9181</port>|<port>19181</port>|g"
replace "s|<https_port>8443</https_port>|<https_port>18443</https_port>|g"
replace "s|<tcp_port>9000</tcp_port>|<tcp_port>19000</tcp_port>|g"
replace "s|<tcp_port>9181</tcp_port>|<tcp_port>19181</tcp_port>|g"
replace "s|<tcp_port_secure>9440</tcp_port_secure>|<tcp_port_secure>19440</tcp_port_secure>|g"
replace "s|<tcp_with_proxy_port>9010</tcp_with_proxy_port>|<tcp_with_proxy_port>19010</tcp_with_proxy_port>|g"
replace "s|<mysql_port>9004</mysql_port>|<mysql_port>19004</mysql_port>|g"
replace "s|<postgresql_port>9005</postgresql_port>|<postgresql_port>19005</postgresql_port>|g"
replace "s|<interserver_http_port>9009</interserver_http_port>|<interserver_http_port>19009</interserver_http_port>|g"
replace "s|8123|18123|g"
replace "s|/var/lib/clickhouse/|/var/lib/clickhouse3/|g"
replace "s|/etc/clickhouse-server/|/etc/clickhouse-server3/|g"
replace "s|<path>/var/lib/clickhouse/access/</path>|<path>/var/lib/clickhouse3/access/</path>|g"
# distributed cache
replace "s|<tcp_port>10001</tcp_port>|<tcp_port>10004</tcp_port>|g"
replace "s|<tcp_port>10002</tcp_port>|<tcp_port>10005</tcp_port>|g"
replace "s|<tcp_port>10003</tcp_port>|<tcp_port>10006</tcp_port>|g"
# use half of available memory for each server
sudo find /etc/clickhouse-server/ -type f -name '*.xml' -exec sed -i "s|<max_server_memory_usage_to_ram_ratio>0.9</max_server_memory_usage_to_ram_ratio>|<max_server_memory_usage_to_ram_ratio>0.67</max_server_memory_usage_to_ram_ratio>|g" {} \;
replace "s|<max_server_memory_usage_to_ram_ratio>0.9</max_server_memory_usage_to_ram_ratio>|<max_server_memory_usage_to_ram_ratio>0.3</max_server_memory_usage_to_ram_ratio>|g"
sudo -E -u clickhouse /usr/bin/clickhouse server --daemon --config /etc/clickhouse-server3/config.xml \
--pid-file /var/run/clickhouse-server3/clickhouse-server.pid \
-- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr-no-parallel.log \
--logger.log /var/log/clickhouse-server/clickhouse-server-no-parallel.log \
--logger.errorlog /var/log/clickhouse-server/clickhouse-server-no-parallel.err.log \
--tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \
--prometheus.port 19988 --keeper_server.raft_configuration.server.port 19234 --keeper_server.tcp_port 19181 \
--mysql_port 19004 --postgresql_port 19005
for _ in {1..100}
do
clickhouse-client --port 19000 --query "SELECT 1" && break
sleep 1
done
fi
# simplest way to forward env variables to server # simplest way to forward env variables to server
sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid
@ -177,6 +234,10 @@ do
done done
setup_logs_replication setup_logs_replication
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
setup_logs_replication 19000
fi
attach_gdb_to_clickhouse attach_gdb_to_clickhouse
# create tables for minio log webhooks # create tables for minio log webhooks
@ -290,7 +351,7 @@ function run_tests()
else else
# All other configurations are OK. # All other configurations are OK.
ADDITIONAL_OPTIONS+=('--jobs') ADDITIONAL_OPTIONS+=('--jobs')
ADDITIONAL_OPTIONS+=('8') ADDITIONAL_OPTIONS+=('5')
fi fi
if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then
@ -343,7 +404,44 @@ if [ "$NUM_TRIES" -gt "1" ]; then
| sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||: | sed 's/All tests have finished/Redacted: a message about tests finish is deleted/' | sed 's/No tests were run/Redacted: a message about no tests run is deleted/' ||:
fi fi
run_tests ||: function run_no_parallel_test()
{
export CLICKHOUSE_CONFIG="/etc/clickhouse-server3/config.xml"
export CLICKHOUSE_CONFIG_DIR="/etc/clickhouse-server3"
export CLICKHOUSE_CONFIG_GREP="/etc/clickhouse-server3/preprocessed/config.xml"
export CLICKHOUSE_USER_FILES="/var/lib/clickhouse3/user_files"
export CLICKHOUSE_SCHEMA_FILES="/var/lib/clickhouse3/format_schemas"
export CLICKHOUSE_PATH="/var/lib/clickhouse3"
export CLICKHOUSE_PORT_TCP="19000"
export CLICKHOUSE_PORT_TCP_SECURE="19440"
export CLICKHOUSE_PORT_TCP_WITH_PROXY="19010"
export CLICKHOUSE_PORT_HTTP="18123"
export CLICKHOUSE_PORT_HTTPS="18443"
export CLICKHOUSE_PORT_INTERSERVER="19009"
export CLICKHOUSE_PORT_KEEPER="19181"
export CLICKHOUSE_PORT_PROMTHEUS_PORT="19988"
export CLICKHOUSE_PORT_MYSQL="19004"
export CLICKHOUSE_PORT_POSTGRESQL="19005"
export CLICKHOUSE_WRITE_COVERAGE="coverage_no_parallel"
export ADDITIONAL_OPTIONS+=" --run-no-parallel-only"
run_tests
}
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
run_no_parallel_test &
PID1=$!
export ADDITIONAL_OPTIONS+=" --run-parallel-only"
run_tests &
PID2=$!
wait $PID1 $PID2 ||:
else
run_tests ||:
fi
echo "Files in current directory" echo "Files in current directory"
ls -la ./ ls -la ./
@ -356,6 +454,9 @@ clickhouse-client -q "system flush logs" ||:
# stop logs replication to make it possible to dump logs tables via clickhouse-local # stop logs replication to make it possible to dump logs tables via clickhouse-local
stop_logs_replication stop_logs_replication
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
stop_logs_replication 19000
fi
# Try to get logs while server is running # Try to get logs while server is running
failed_to_save_logs=0 failed_to_save_logs=0
@ -364,6 +465,12 @@ do
if ! clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.zst' format TSVWithNamesAndTypes"; then if ! clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.zst' format TSVWithNamesAndTypes"; then
failed_to_save_logs=1 failed_to_save_logs=1
fi fi
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
if ! clickhouse-client --port 19000 -q "select * from system.$table into outfile '/test_output/$table.no-parallel.tsv.zst' format TSVWithNamesAndTypes"; then
failed_to_save_logs=1
fi
fi
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if ! clickhouse-client --port 19000 -q "select * from system.$table into outfile '/test_output/$table.1.tsv.zst' format TSVWithNamesAndTypes"; then if ! clickhouse-client --port 19000 -q "select * from system.$table into outfile '/test_output/$table.1.tsv.zst' format TSVWithNamesAndTypes"; then
failed_to_save_logs=1 failed_to_save_logs=1
@ -393,6 +500,9 @@ clickhouse-client --max_block_size 8192 --max_memory_usage 10G --max_threads 1 -
# Because it's the simplest way to read it when server has crashed. # Because it's the simplest way to read it when server has crashed.
sudo clickhouse stop ||: sudo clickhouse stop ||:
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
sudo clickhouse stop --pid-path /var/run/clickhouse-server3 ||:
fi
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||:
@ -430,7 +540,12 @@ if [ $failed_to_save_logs -ne 0 ]; then
do do
clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse3/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.3.tsv.zst ||:
fi
if
[[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||: clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi fi
@ -473,6 +588,12 @@ rm -rf /var/lib/clickhouse/data/system/*/
tar -chf /test_output/store.tar /var/lib/clickhouse/store ||: tar -chf /test_output/store.tar /var/lib/clickhouse/store ||:
tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||:
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server-no-parallel.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server-no-parallel.log > /test_output/clickhouse-server-no-parallel.log.zst ||:
mv /var/log/clickhouse-server/stderr-no-parallel.log /test_output/ ||:
tar -chf /test_output/coordination-no-parallel.tar /var/lib/clickhouse3/coordination ||:
fi
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server1.log ||: rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server1.log ||:

View File

@ -18,7 +18,9 @@ function run_selects()
{ {
thread_num=$1 thread_num=$1
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables
WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name != 'zookeeper' and name != 'models' WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA')
and name != 'zookeeper' and name != 'models'
and name not like '%_sender' and name not like '%_buffer'
AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num AND name NOT LIKE '%\\_sender' AND name NOT LIKE '%\\_watcher' AND name != 'coverage_log'") AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num AND name NOT LIKE '%\\_sender' AND name NOT LIKE '%\\_watcher' AND name != 'coverage_log'")
for t in "${tables_arr[@]}" for t in "${tables_arr[@]}"

View File

@ -7,15 +7,20 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh . "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP DNS CACHE"
${CLICKHOUSE_CLIENT} --query " ${CLICKHOUSE_CLIENT} --query "
DROP USER IF EXISTS dns_fail_1, dns_fail_2; DROP USER IF EXISTS dns_fail_1, dns_fail_2, dns_fail_3;
CREATE USER dns_fail_1 HOST NAME 'non.existing.host.name', '${MYHOSTNAME}'; CREATE USER dns_fail_1 HOST NAME 'non.existing.host.name', '${MYHOSTNAME}';
CREATE USER dns_fail_2 HOST NAME '${MYHOSTNAME}', 'non.existing.host.name';" CREATE USER dns_fail_2 HOST NAME '${MYHOSTNAME}', 'non.existing.host.name';
CREATE USER dns_fail_3 HOST NAME 'localhost', 'non.existing.host.name';"
${CLICKHOUSE_CLIENT} --query "SELECT 1" --user dns_fail_1 --host ${MYHOSTNAME} ${CLICKHOUSE_CLIENT} --query "SELECT 1" --user dns_fail_1 --host ${MYHOSTNAME}
${CLICKHOUSE_CLIENT} --query "SELECT 2" --user dns_fail_2 --host ${MYHOSTNAME} ${CLICKHOUSE_CLIENT} --query "SELECT 2" --user dns_fail_2 --host ${MYHOSTNAME}
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS dns_fail_1, dns_fail_2" ${CLICKHOUSE_CLIENT} --query "SELECT 3" --user dns_fail_3 --host ${MYHOSTNAME}
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS dns_fail_1, dns_fail_2, dns_fail_3"
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP DNS CACHE" ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP DNS CACHE"

View File

@ -17,7 +17,7 @@ SELECT 'prefer_localhost_replica=1, remote query with a lot of union all' AS tes
-- query with lot of dummy union all will create a lot of streams -- query with lot of dummy union all will create a lot of streams
-- let's check how many threads clickhouse will start for that -- let's check how many threads clickhouse will start for that
select count() from remote('127.0.0.1:9000', view( select count() from remote('127.0.0.1', view(
{% for n in range(77) -%} {% for n in range(77) -%}
SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }}
{% endfor -%} {% endfor -%}
@ -49,7 +49,7 @@ WHERE
-------------------- --------------------
SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname; SELECT 'prefer_localhost_replica=0, remote query with a lot of union all' AS testname;
select count() from remote('127.0.0.1:9000', view( select count() from remote('127.0.0.1', view(
{% for n in range(77) -%} {% for n in range(77) -%}
SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }}
{% endfor -%} {% endfor -%}
@ -85,7 +85,7 @@ SELECT 'prefer_localhost_replica=1, async_socket_for_remote=0, remote query with
-- now is happens only for async_socket_for_remote=0 (while it is 1 by default) -- now is happens only for async_socket_for_remote=0 (while it is 1 by default)
-- see https://github.com/ClickHouse/ClickHouse/issues/53287 -- see https://github.com/ClickHouse/ClickHouse/issues/53287
select count() from remote('127.0.0.1:9000', view( select count() from remote('127.0.0.1', view(
{% for n in range(77) -%} {% for n in range(77) -%}
SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }}
{% endfor -%} {% endfor -%}

View File

@ -3,7 +3,7 @@
# If ClickHouse was built with coverage - dump the coverage information at exit # If ClickHouse was built with coverage - dump the coverage information at exit
# (in other cases this environment variable has no effect) # (in other cases this environment variable has no effect)
export CLICKHOUSE_WRITE_COVERAGE="coverage" export CLICKHOUSE_WRITE_COVERAGE=${CLICKHOUSE_WRITE_COVERAGE:="coverage"}
export CLICKHOUSE_DATABASE=${CLICKHOUSE_DATABASE:="test"} export CLICKHOUSE_DATABASE=${CLICKHOUSE_DATABASE:="test"}
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"}