mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #66520 from ClickHouse/fix-tests-oom
Stateless tests: improvements related to OOM of test runs
This commit is contained in:
commit
3d00badef7
@ -4,6 +4,9 @@
|
||||
source /setup_export_logs.sh
|
||||
set -e -x
|
||||
|
||||
MAX_RUN_TIME=${MAX_RUN_TIME:-3600}
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 3600 : MAX_RUN_TIME))
|
||||
|
||||
# Choose random timezone for this test run
|
||||
TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)"
|
||||
echo "Choosen random timezone $TZ"
|
||||
@ -242,7 +245,22 @@ function run_tests()
|
||||
}
|
||||
|
||||
export -f run_tests
|
||||
timeout "$MAX_RUN_TIME" bash -c run_tests ||:
|
||||
|
||||
function timeout_with_logging() {
|
||||
local exit_code=0
|
||||
|
||||
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
|
||||
|
||||
if [[ "${exit_code}" -eq "124" ]]
|
||||
then
|
||||
echo "The command 'timeout ${*}' has been killed by timeout"
|
||||
fi
|
||||
|
||||
return $exit_code
|
||||
}
|
||||
|
||||
TIMEOUT=$((MAX_RUN_TIME - 700))
|
||||
timeout_with_logging "$TIMEOUT" bash -c run_tests ||:
|
||||
|
||||
echo "Files in current directory"
|
||||
ls -la ./
|
||||
|
@ -12,12 +12,6 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME))
|
||||
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
|
||||
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
|
||||
|
||||
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
|
||||
@ -101,53 +95,6 @@ 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
|
||||
|
||||
@ -183,9 +130,6 @@ if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
--keeper_server.tcp_port 29181 --keeper_server.server_id 3 \
|
||||
--prometheus.port 29988 \
|
||||
--macros.shard s2 # It doesn't work :(
|
||||
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
|
||||
fi
|
||||
|
||||
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
|
||||
@ -210,9 +154,6 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
|
||||
--keeper_server.tcp_port 19181 --keeper_server.server_id 2 \
|
||||
--prometheus.port 19988 \
|
||||
--macros.replica r2 # It doesn't work :(
|
||||
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
|
||||
fi
|
||||
|
||||
# Wait for the server to start, but not for too long.
|
||||
@ -223,7 +164,6 @@ do
|
||||
done
|
||||
|
||||
setup_logs_replication
|
||||
|
||||
attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01
|
||||
|
||||
function fn_exists() {
|
||||
@ -284,11 +224,7 @@ function run_tests()
|
||||
else
|
||||
# All other configurations are OK.
|
||||
ADDITIONAL_OPTIONS+=('--jobs')
|
||||
ADDITIONAL_OPTIONS+=('5')
|
||||
fi
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
ADDITIONAL_OPTIONS+=('--run-sequential-tests-in-parallel')
|
||||
ADDITIONAL_OPTIONS+=('7')
|
||||
fi
|
||||
|
||||
if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then
|
||||
@ -373,9 +309,6 @@ done
|
||||
# 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 ||:
|
||||
@ -393,12 +326,6 @@ 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)
|
||||
@ -419,10 +346,6 @@ 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 [[ "$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 ||:
|
||||
@ -464,12 +387,6 @@ 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
|
||||
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 ||:
|
||||
|
@ -112,8 +112,8 @@ def get_run_command(
|
||||
]
|
||||
|
||||
if flaky_check:
|
||||
envs.append("-e NUM_TRIES=100")
|
||||
envs.append("-e MAX_RUN_TIME=1800")
|
||||
envs.append("-e NUM_TRIES=50")
|
||||
envs.append("-e MAX_RUN_TIME=2800")
|
||||
|
||||
envs += [f"-e {e}" for e in additional_envs]
|
||||
|
||||
|
@ -2187,7 +2187,7 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool
|
||||
description = ""
|
||||
test_case_name = removesuffix(test_case.name, ".gen", ".sql") + ": "
|
||||
|
||||
if is_concurrent or args.run_sequential_tests_in_parallel:
|
||||
if is_concurrent:
|
||||
description = f"{test_case_name:72}"
|
||||
else:
|
||||
sys.stdout.flush()
|
||||
@ -2447,35 +2447,6 @@ 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 run_tests_process(*args, **kwargs):
|
||||
return run_tests_array(*args, **kwargs)
|
||||
|
||||
@ -2519,24 +2490,6 @@ def do_run_tests(jobs, test_suite: TestSuite):
|
||||
processes.append(process)
|
||||
process.start()
|
||||
|
||||
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`
|
||||
process = multiprocessing.Process(
|
||||
target=override_envs,
|
||||
args=(
|
||||
(
|
||||
test_suite.sequential_tests,
|
||||
len(test_suite.sequential_tests),
|
||||
test_suite,
|
||||
False,
|
||||
),
|
||||
),
|
||||
)
|
||||
processes.append(process)
|
||||
process.start()
|
||||
|
||||
while processes:
|
||||
sys.stdout.flush()
|
||||
# Periodically check the server for hangs
|
||||
@ -2568,15 +2521,15 @@ def do_run_tests(jobs, test_suite: TestSuite):
|
||||
|
||||
sleep(5)
|
||||
|
||||
if not args.run_sequential_tests_in_parallel:
|
||||
run_tests_array(
|
||||
(
|
||||
test_suite.sequential_tests,
|
||||
len(test_suite.sequential_tests),
|
||||
test_suite,
|
||||
False,
|
||||
)
|
||||
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)
|
||||
@ -3419,15 +3372,6 @@ def parse_args():
|
||||
help="Path to file for fatal logs from client",
|
||||
)
|
||||
|
||||
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()
|
||||
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: deadlock, no-debug
|
||||
# Tags: deadlock, no-debug, no-parallel
|
||||
|
||||
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: shard
|
||||
-- Tags: shard, no-parallel
|
||||
|
||||
SET prefer_localhost_replica = 1;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: deadlock
|
||||
# Tags: deadlock, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: deadlock, shard
|
||||
# Tags: deadlock, shard, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: deadlock
|
||||
# Tags: deadlock, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-fasttest
|
||||
# Tags: zookeeper, no-fasttest, long, no-parallel
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel, no-fasttest
|
||||
# Tag no-fasttest: this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled
|
||||
# Tag no-fasttest: this test mistakenly requires access to /var/lib/clickhouse -- can't run this locally, disabled
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query="
|
||||
DROP DATABASE IF EXISTS 01053_db;
|
||||
|
||||
CREATE DATABASE 01053_db Engine = Ordinary;
|
||||
CREATE DATABASE 01053_db;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, zookeeper, no-fasttest
|
||||
# Tags: long, zookeeper, no-fasttest, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race, zookeeper
|
||||
# Tags: race, zookeeper, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -6,9 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
DROP DATABASE IF EXISTS 01280_db;
|
||||
CREATE DATABASE 01280_db Engine = Ordinary;
|
||||
CREATE DATABASE 01280_db;
|
||||
DROP TABLE IF EXISTS 01280_db.table_for_dict;
|
||||
CREATE TABLE 01280_db.table_for_dict
|
||||
(
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race
|
||||
# Tags: race, no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race
|
||||
# Tags: race, no-parallel
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-object-storage, no-random-settings
|
||||
-- Tags: no-object-storage, no-random-settings, no-parallel
|
||||
|
||||
SET use_uncompressed_cache = 0;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: deadlock
|
||||
# Tags: deadlock, no-parallel
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-object-storage, no-random-merge-tree-settings
|
||||
-- Tags: no-object-storage, no-random-merge-tree-settings, no-parallel
|
||||
-- no-s3 because read FileOpen metric
|
||||
DROP TABLE IF EXISTS nested;
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
-- Tags: no-parallel
|
||||
-- We check the existence of queries and metrics and don't check the results (a smoke test).
|
||||
|
||||
SYSTEM DROP MMAP CACHE;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, zookeeper
|
||||
# Tags: long, zookeeper, no-parallel
|
||||
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,4 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-random-settings, no-fasttest
|
||||
-- Tags: no-random-settings, no-fasttest, no-parallel
|
||||
|
||||
SET allow_prefetched_read_pool_for_remote_filesystem=0;
|
||||
SET allow_prefetched_read_pool_for_local_filesystem=0;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-random-settings, no-object-storage
|
||||
-- Tags: no-random-settings, no-object-storage, no-parallel
|
||||
-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages
|
||||
|
||||
SET allow_experimental_dynamic_type = 1;
|
||||
|
Loading…
Reference in New Issue
Block a user