diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index a39f96867be..2eb5d7cf1b5 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -110,6 +110,8 @@ 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 + + PORT=${1:-"9000"} # disable output if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then 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 '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" # 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 " + clickhouse-client --port "$PORT" --query " CREATE TABLE IF NOT EXISTS system.coverage_log ( time DateTime COMMENT 'The time of test run', @@ -147,7 +149,7 @@ function setup_logs_replication # For each system log table: 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 if [[ "$table" = "trace_log" ]] then @@ -171,7 +173,7 @@ function setup_logs_replication fi # 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))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' @@ -179,7 +181,7 @@ function setup_logs_replication ") # 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/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/; 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 # 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 ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) SETTINGS flush_on_detach=0 @@ -214,10 +216,20 @@ function setup_logs_replication FROM system.${table} " || 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 " - CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS + echo "Creating materialized view system.${table}_watcher" >&2 + clickhouse-client --port "$PORT" --query " + CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_buffer AS SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * FROM system.${table} " || continue @@ -227,9 +239,10 @@ function setup_logs_replication 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')" | { + 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 } | { - 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 {}" } } diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 3587405d95f..fa6744a5ec0 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -323,22 +323,22 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 ), 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( - 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( - 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( - 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( required_builds=[BuildNames.PACKAGE_RELEASE], ), 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( required_builds=[BuildNames.PACKAGE_AARCH64], @@ -351,7 +351,7 @@ class CI: required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=1 ), 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( required_builds=[BuildNames.PACKAGE_TSAN], diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 810bae86cb0..ab47a32f77b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1584,7 +1584,8 @@ class TestCase: print("Cannot insert coverage data: ", str(e)) # 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) for file_path in matching_files: try: @@ -1740,11 +1741,27 @@ class TestCase: f"PORT {os.environ['CLICKHOUSE_PORT_TCP']}", "PORT 9000", ) + replace_in_file( + self.stdout_file, + f"port {os.environ['CLICKHOUSE_PORT_TCP']}", + "port 9000", + ) replace_in_file( self.stdout_file, f"localhost {os.environ['CLICKHOUSE_PORT_TCP']}", "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"): replace_in_file( @@ -2161,6 +2178,11 @@ class TestSuite: else: 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): if args.sequential: if any(s in test_name for s in args.sequential): @@ -2269,13 +2291,9 @@ class GlobalTimeout(Exception): pass -def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool]): - ( - all_tests, - num_tests, - test_suite, - is_concurrent, - ) = all_tests_with_params +def run_tests_array( + all_tests: List[str], num_tests: int, test_suite: TestSuite, is_concurrent: bool +): global stop_time global exit_code global server_died @@ -2621,94 +2639,81 @@ def run_tests_process(*args, **kwargs): def do_run_tests(jobs, test_suite: TestSuite): - if jobs > 1 and len(test_suite.parallel_tests) > 0: - print( - "Found", - len(test_suite.parallel_tests), - "parallel tests and", - len(test_suite.sequential_tests), - "sequential tests", + print( + "Found", + len(test_suite.parallel_tests), + "parallel tests and", + len(test_suite.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) - jobs = min(jobs, tests_n) + processes.append(process) + process.start() - # 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) - - processes = [] - for _ in range(jobs): - process = multiprocessing.Process( - target=run_tests_process, - args=((parallel_tests, batch_size, test_suite, True),), + 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, ) - 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) - run_tests_array( - ( - test_suite.sequential_tests, - len(test_suite.sequential_tests), - test_suite, - False, - ) - ) + except Exception: + print("Hung check failed") + server_died.set() - return len(test_suite.sequential_tests) + len(test_suite.parallel_tests) - else: - num_tests = len(test_suite.all_tests) - run_tests_array( - ( - test_suite.all_tests, - num_tests, - test_suite, - False, - ) - ) - return num_tests + 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) + + 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): @@ -3552,6 +3557,19 @@ def parse_args(): 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() diff --git a/tests/config/config.d/user_directories.xml b/tests/config/config.d/user_directories.xml new file mode 100644 index 00000000000..fe0617f41df --- /dev/null +++ b/tests/config/config.d/user_directories.xml @@ -0,0 +1,7 @@ + + + + /var/lib/clickhouse/access/ + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index fda74bd7a8d..14d787b2f29 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -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/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/user_directories.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] diff --git a/tests/docker_scripts/stateless_runner.sh b/tests/docker_scripts/stateless_runner.sh index d8921a04458..57c827b1dd0 100755 --- a/tests/docker_scripts/stateless_runner.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -17,7 +17,11 @@ set -e -x -a USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=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. # # 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 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|9000|19000|g" + replace "s|9440|19440|g" + replace "s|9988|19988|g" + replace "s|9234|19234|g" + replace "s|9181|19181|g" + replace "s|8443|18443|g" + replace "s|9000|19000|g" + replace "s|9181|19181|g" + replace "s|9440|19440|g" + replace "s|9010|19010|g" + replace "s|9004|19004|g" + replace "s|9005|19005|g" + replace "s|9009|19009|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|/var/lib/clickhouse/access/|/var/lib/clickhouse3/access/|g" + # distributed cache + replace "s|10001|10004|g" + replace "s|10002|10005|g" + replace "s|10003|10006|g" + # use half of available memory for each server + sudo find /etc/clickhouse-server/ -type f -name '*.xml' -exec sed -i "s|0.9|0.67|g" {} \; + replace "s|0.9|0.3|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 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 setup_logs_replication +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + setup_logs_replication 19000 +fi + attach_gdb_to_clickhouse # create tables for minio log webhooks @@ -290,7 +351,7 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('8') + ADDITIONAL_OPTIONS+=('5') fi 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/' ||: 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" 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 +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + stop_logs_replication 19000 +fi # Try to get logs while server is running 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 failed_to_save_logs=1 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 ! 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 @@ -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. 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 sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: @@ -430,7 +540,12 @@ if [ $failed_to_save_logs -ne 0 ]; then 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 ||: - 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/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||: 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/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + rg -Fa "" /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 rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index d4a80d074dc..f94ccfb044a 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -18,7 +18,9 @@ function run_selects() { thread_num=$1 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'") for t in "${tables_arr[@]}" diff --git a/tests/queries/0_stateless/01946_test_wrong_host_name_access.reference b/tests/queries/0_stateless/01946_test_wrong_host_name_access.reference index 1191247b6d9..01e79c32a8c 100644 --- a/tests/queries/0_stateless/01946_test_wrong_host_name_access.reference +++ b/tests/queries/0_stateless/01946_test_wrong_host_name_access.reference @@ -1,2 +1,3 @@ 1 2 +3 diff --git a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh index ed2828c3f54..f37675b2fb1 100755 --- a/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh +++ b/tests/queries/0_stateless/01946_test_wrong_host_name_access.sh @@ -7,15 +7,20 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP DNS CACHE" + ${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_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 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" diff --git a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 index 26b858d8c00..a0e01546f84 100644 --- a/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 +++ b/tests/queries/0_stateless/02845_threads_count_in_distributed_queries.sql.j2 @@ -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 -- 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) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% endfor -%} @@ -49,7 +49,7 @@ WHERE -------------------- 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) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% 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) -- 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) -%} SELECT * FROM system.one {{ "UNION ALL" if not loop.last }} {% endfor -%} diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 9e5bf75d335..23b671399ff 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -3,7 +3,7 @@ # If ClickHouse was built with coverage - dump the coverage information at exit # (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_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"}