Merge pull request #65186 from ClickHouse/optimize-tests

Stateless tests: Improve tests speed
This commit is contained in:
Nikita Fomichev 2024-07-09 21:13:42 +00:00 committed by GitHub
commit 249c80a26d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 265 additions and 52 deletions

View File

@ -284,6 +284,11 @@ function run_tests
NPROC=1
fi
export CLICKHOUSE_CONFIG_DIR=$FASTTEST_DATA
export CLICKHOUSE_CONFIG="$FASTTEST_DATA/config.xml"
export CLICKHOUSE_USER_FILES="$FASTTEST_DATA/user_files"
export CLICKHOUSE_SCHEMA_FILES="$FASTTEST_DATA/format_schemas"
local test_opts=(
--hung-check
--fast-tests-only

View File

@ -9,6 +9,16 @@ set -e -x -a
MAX_RUN_TIME=${MAX_RUN_TIME:-10800}
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME))
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
# disable for now
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
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
@ -89,10 +99,57 @@ 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|<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"
# distributed cache
replace "s|<tcp_port>10001</tcp_port>|<tcp_port>10003</tcp_port>|g"
replace "s|<tcp_port>10002</tcp_port>|<tcp_port>10004</tcp_port>|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/stderr3.log \
--logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.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
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
sudo sed -i "s|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches/</filesystem_caches_path>|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches_1/</filesystem_caches_path>|" /etc/clickhouse-server1/config.d/filesystem_caches_path.xml
sudo sed -i "s|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches/</filesystem_caches_path>|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches_2/</filesystem_caches_path>|" /etc/clickhouse-server2/config.d/filesystem_caches_path.xml
@ -129,7 +186,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \
| sed "s|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches/</filesystem_caches_path>|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches_1/</filesystem_caches_path>|" \
> /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp
@ -209,15 +266,15 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--no-random-merge-tree-settings')
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--shared-catalog')
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
# Too many tests fail for DatabaseReplicated in parallel.
ADDITIONAL_OPTIONS+=('--jobs')
ADDITIONAL_OPTIONS+=('2')
ADDITIONAL_OPTIONS+=('3')
elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%SANITIZE_COVERAGE%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then
# Coverage on a per-test basis could only be collected sequentially.
# Do not set the --jobs parameter.
@ -225,7 +282,11 @@ function run_tests()
else
# All other configurations are OK.
ADDITIONAL_OPTIONS+=('--jobs')
ADDITIONAL_OPTIONS+=('8')
ADDITIONAL_OPTIONS+=('5')
fi
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--run-sequential-tests-in-parallel')
fi
if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then
@ -289,7 +350,7 @@ do
err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes")
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
@ -298,7 +359,7 @@ do
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
@ -309,12 +370,17 @@ done
# Why do we read data with clickhouse-local?
# Because it's the simplest way to read it when server has crashed.
sudo clickhouse stop ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
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 ||:
sudo clickhouse stop --pid-path /var/run/clickhouse-server2 ||:
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||:
fi
@ -322,6 +388,12 @@ rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server.log ||:
rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst &
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
rg -Fa "<Fatal>" /var/log/clickhouse-server3/clickhouse-server.log ||:
rg -A50 -Fa "============" /var/log/clickhouse-server3/stderr.log ||:
zstd --threads=0 < /var/log/clickhouse-server3/clickhouse-server.log > /test_output/clickhouse-server3.log.zst &
fi
data_path_config="--path=/var/lib/clickhouse/"
if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then
# We need s3 storage configuration (but it's more likely that clickhouse-local will fail for some reason)
@ -341,12 +413,17 @@ if [ $failed_to_save_logs -ne 0 ]; then
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log
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 [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$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
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -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 ||:
fi
done
@ -382,7 +459,14 @@ 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 [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
rm -rf /var/lib/clickhouse3/data/system/*/
tar -chf /test_output/store.tar /var/lib/clickhouse3/store ||:
tar -chf /test_output/metadata.tar /var/lib/clickhouse3/metadata/*.sql ||:
fi
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-server2.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||:
@ -393,7 +477,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
tar -chf /test_output/coordination2.tar /var/lib/clickhouse2/coordination ||:
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server1.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||:
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:

View File

@ -311,42 +311,42 @@ class CI:
random_bucket="parrepl_with_sanitizer",
),
JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2
),
JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5
required_builds=[BuildNames.PACKAGE_TSAN], num_batches=2
),
JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6
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=5
required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2
),
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],
runner_type=Runners.FUNC_TESTER_ARM,
),
JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4
required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=3
),
JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6
required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2
),
JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, 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],
num_batches=5,
num_batches=3,
),
JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_DEBUG],

View File

@ -709,9 +709,9 @@ def get_localzone():
class SettingsRandomizer:
settings = {
"max_insert_threads": lambda: (
0 if random.random() < 0.5 else random.randint(1, 16)
),
"max_insert_threads": lambda: 32
if random.random() < 0.03
else random.randint(1, 3),
"group_by_two_level_threshold": threshold_generator(0.2, 0.2, 1, 1000000),
"group_by_two_level_threshold_bytes": threshold_generator(
0.2, 0.2, 1, 50000000
@ -727,7 +727,7 @@ class SettingsRandomizer:
"prefer_localhost_replica": lambda: random.randint(0, 1),
"max_block_size": lambda: random.randint(8000, 100000),
"max_joined_block_size_rows": lambda: random.randint(8000, 100000),
"max_threads": lambda: random.randint(1, 64),
"max_threads": lambda: 64 if random.random() < 0.03 else random.randint(1, 3),
"optimize_append_index": lambda: random.randint(0, 1),
"optimize_if_chain_to_multiif": lambda: random.randint(0, 1),
"optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1),
@ -1217,6 +1217,11 @@ class TestCase:
):
return FailureReason.OBJECT_STORAGE
elif "no-batch" in tags and (
args.run_by_hash_num is not None or args.run_by_hash_total is not None
):
return FailureReason.SKIP
elif tags:
for build_flag in args.build_flags:
if "no-" + build_flag in tags:
@ -1447,8 +1452,7 @@ class TestCase:
description_full = messages[result.status]
description_full += self.print_test_time(result.total_time)
if result.reason is not None:
description_full += " - "
description_full += result.reason.value
description_full += f"\nReason: {result.reason.value} "
description_full += result.description
@ -1575,10 +1579,11 @@ class TestCase:
# pylint:disable-next=consider-using-with; TODO: fix
proc = Popen(command, shell=True, env=os.environ, start_new_session=True)
while (
datetime.now() - start_time
).total_seconds() < args.timeout and proc.poll() is None:
sleep(0.01)
try:
proc.wait(args.timeout)
except subprocess.TimeoutExpired:
# Whether the test timed out will be decided later
pass
debug_log = ""
if os.path.exists(self.testcase_args.debug_log_file):
@ -1600,6 +1605,44 @@ class TestCase:
# Normalize hostname in stdout file.
replace_in_file(self.stdout_file, socket.gethostname(), "localhost")
if os.environ.get("CLICKHOUSE_PORT_TCP"):
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",
)
if os.environ.get("CLICKHOUSE_PORT_TCP_SECURE"):
replace_in_file(
self.stdout_file,
f"PORT {os.environ['CLICKHOUSE_PORT_TCP_SECURE']}",
"PORT 9440",
)
replace_in_file(
self.stdout_file,
f"localhost {os.environ['CLICKHOUSE_PORT_TCP_SECURE']}",
"localhost 9440",
)
if os.environ.get("CLICKHOUSE_PATH"):
replace_in_file(
self.stdout_file,
os.environ["CLICKHOUSE_PATH"],
"/var/lib/clickhouse",
)
if os.environ.get("CLICKHOUSE_PORT_HTTPS"):
replace_in_file(
self.stdout_file,
f"https://localhost:{os.environ['CLICKHOUSE_PORT_HTTPS']}/",
"https://localhost:8443/",
)
stdout = ""
if os.path.exists(self.stdout_file):
with open(self.stdout_file, "rb") as stdfd:
@ -2056,8 +2099,13 @@ class GlobalTimeout(Exception):
pass
def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]):
all_tests, num_tests, test_suite = all_tests_with_params
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
global stop_time
global exit_code
global server_died
@ -2100,14 +2148,12 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]):
failures_chain = 0
start_time = datetime.now()
is_concurrent = multiprocessing.current_process().name != "MainProcess"
client_options = get_additional_client_options(args)
if num_tests > 0:
about = "about " if is_concurrent else ""
proc_name = multiprocessing.current_process().name
print(f"\nRunning {about}{num_tests} {test_suite.suite} tests ({proc_name}).\n")
print(f"Running {about}{num_tests} {test_suite.suite} tests ({proc_name}).")
while True:
if all_tests:
@ -2128,16 +2174,17 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]):
try:
description = ""
test_cace_name = removesuffix(test_case.name, ".gen", ".sql") + ": "
if not is_concurrent:
test_case_name = removesuffix(test_case.name, ".gen", ".sql") + ": "
if is_concurrent or args.run_sequential_tests_in_parallel:
description = f"{test_case_name:72}"
else:
sys.stdout.flush()
sys.stdout.write(f"{test_cace_name:72}")
sys.stdout.write(f"{test_case_name:72}")
# This flush is needed so you can see the test name of the long
# running test before it will finish. But don't do it in parallel
# mode, so that the lines don't mix.
sys.stdout.flush()
else:
description = f"{test_cace_name:72}"
while True:
test_result = test_case.run(
@ -2372,6 +2419,35 @@ def extract_key(key: str) -> str:
)[1]
def override_envs(*args_, **kwargs):
global args
args.client += " --port 19000"
args.http_port = 18123
args.https_port = 18443
updated_env = {
"CLICKHOUSE_CONFIG": "/etc/clickhouse-server3/config.xml",
"CLICKHOUSE_CONFIG_DIR": "/etc/clickhouse-server3",
"CLICKHOUSE_CONFIG_GREP": "/etc/clickhouse-server3/preprocessed/config.xml",
"CLICKHOUSE_USER_FILES": "/var/lib/clickhouse3/user_files",
"CLICKHOUSE_SCHEMA_FILES": "/var/lib/clickhouse3/format_schemas",
"CLICKHOUSE_PATH": "/var/lib/clickhouse3",
"CLICKHOUSE_PORT_TCP": "19000",
"CLICKHOUSE_PORT_TCP_SECURE": "19440",
"CLICKHOUSE_PORT_TCP_WITH_PROXY": "19010",
"CLICKHOUSE_PORT_HTTP": "18123",
"CLICKHOUSE_PORT_HTTPS": "18443",
"CLICKHOUSE_PORT_INTERSERVER": "19009",
"CLICKHOUSE_PORT_KEEPER": "19181",
"CLICKHOUSE_PORT_PROMTHEUS_PORT": "19988",
"CLICKHOUSE_PORT_MYSQL": "19004",
"CLICKHOUSE_PORT_POSTGRESQL": "19005",
}
os.environ.update(updated_env)
run_tests_array(*args_, **kwargs)
def do_run_tests(jobs, test_suite: TestSuite):
if jobs > 1 and len(test_suite.parallel_tests) > 0:
print(
@ -2400,24 +2476,55 @@ def do_run_tests(jobs, test_suite: TestSuite):
for job in range(jobs):
range_ = job * batch_size, job * batch_size + batch_size
batch = test_suite.parallel_tests[range_[0] : range_[1]]
parallel_tests_array.append((batch, batch_size, test_suite))
parallel_tests_array.append((batch, batch_size, test_suite, True))
try:
with multiprocessing.Pool(processes=jobs) as pool:
with multiprocessing.Pool(processes=jobs + 1) as pool:
future = pool.map_async(run_tests_array, parallel_tests_array)
if args.run_sequential_tests_in_parallel:
# Run parallel tests and sequential tests at the same time
# Sequential tests will use different ClickHouse instance
# In this process we can safely override values in `args` and `os.environ`
future_seq = pool.map_async(
override_envs,
[
(
test_suite.sequential_tests,
len(test_suite.sequential_tests),
test_suite,
False,
)
],
)
future_seq.wait()
future.wait()
finally:
pool.terminate()
pool.close()
pool.join()
run_tests_array(
(test_suite.sequential_tests, len(test_suite.sequential_tests), test_suite)
)
if not args.run_sequential_tests_in_parallel:
run_tests_array(
(
test_suite.sequential_tests,
len(test_suite.sequential_tests),
test_suite,
False,
)
)
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))
run_tests_array(
(
test_suite.all_tests,
num_tests,
test_suite,
False,
)
)
return num_tests
@ -2722,6 +2829,7 @@ def main(args):
f"{get_db_engine(args, db_name)}",
settings=get_create_database_settings(args, None),
)
break
except HTTPError as e:
total_time = (datetime.now() - start_time).total_seconds()
if not need_retry(args, e.message, e.message, total_time):
@ -3234,6 +3342,15 @@ def parse_args():
help="Replace ordinary MergeTree engine with SharedMergeTree",
)
parser.add_argument(
"--run-sequential-tests-in-parallel",
action="store_true",
default=False,
help="If `true`, tests with the tag `no-parallel` will run on a "
"separate ClickHouse instance in parallel with other tests. "
"This is used in CI to make test jobs run faster.",
)
return parser.parse_args()

View File

@ -57,7 +57,6 @@ ln -sf $SRC_PATH/config.d/forbidden_headers.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/reverse_dns_query_function.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/disable_s3_env_credentials.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/enable_wait_for_shutdown_replicated_tables.xml $DEST_SERVER_PATH/config.d/

View File

@ -1,4 +1,4 @@
-- Tags: no-parallel, no-fasttest
-- Tags: no-parallel, no-fasttest, no-ubsan, no-batch
-- no-parallel because we want to run this test when most of the other tests already passed
-- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log

View File

@ -54,9 +54,17 @@ export CLICKHOUSE_OBFUSCATOR=${CLICKHOUSE_OBFUSCATOR:="${CLICKHOUSE_BINARY}-obfu
export CLICKHOUSE_COMPRESSOR=${CLICKHOUSE_COMPRESSOR:="${CLICKHOUSE_BINARY}-compressor"}
export CLICKHOUSE_GIT_IMPORT=${CLICKHOUSE_GIT_IMPORT="${CLICKHOUSE_BINARY}-git-import"}
export CLICKHOUSE_CONFIG_DIR=${CLICKHOUSE_CONFIG_DIR:="/etc/clickhouse-server"}
export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"}
export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:="/etc/clickhouse-client/config.xml"}
export CLICKHOUSE_USER_FILES=${CLICKHOUSE_USER_FILES:="/var/lib/clickhouse/user_files"}
export CLICKHOUSE_USER_FILES_UNIQUE=${CLICKHOUSE_USER_FILES_UNIQUE:="${CLICKHOUSE_USER_FILES}/${CLICKHOUSE_TEST_UNIQUE_NAME}"}
# synonym
export USER_FILES_PATH=$CLICKHOUSE_USER_FILES
export CLICKHOUSE_SCHEMA_FILES=${CLICKHOUSE_SCHEMA_FILES:="/var/lib/clickhouse/format_schemas"}
[ -x "${CLICKHOUSE_BINARY}-extract-from-config" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"}
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY extract-from-config --config=$CLICKHOUSE_CONFIG"}
export CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"}