Merge pull request #29992 from ClickHouse/more_timeouts_in_scripts

More timeouts in test scripts
This commit is contained in:
alesapin 2021-10-12 10:14:21 +03:00 committed by GitHub
commit 2b3a0c892d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 66 additions and 41 deletions

View File

@ -71,42 +71,42 @@ def prepare_for_hung_check(drop_databases):
# FIXME this function should not exist, but...
# ThreadFuzzer significantly slows down server and causes false-positive hung check failures
call("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'", shell=True, stderr=STDOUT, timeout=30)
# We attach gdb to clickhouse-server before running tests
# to print stacktraces of all crashes even if clickhouse cannot print it for some reason.
# However, it obstruct checking for hung queries.
logging.info("Will terminate gdb (if any)")
call("kill -TERM $(pidof gdb)", shell=True, stderr=STDOUT)
call("kill -TERM $(pidof gdb)", shell=True, stderr=STDOUT, timeout=30)
# Some tests set too low memory limit for default user and forget to reset in back.
# It may cause SYSTEM queries to fail, let's disable memory limit.
call("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'", shell=True, stderr=STDOUT)
call("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'", shell=True, stderr=STDOUT, timeout=30)
# Some tests execute SYSTEM STOP MERGES or similar queries.
# It may cause some ALTERs to hang.
# Possibly we should fix tests and forbid to use such queries without specifying table.
call("clickhouse client -q 'SYSTEM START MERGES'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START TTL MERGES'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START MOVES'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START FETCHES'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START REPLICATED SENDS'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'", shell=True, stderr=STDOUT)
call("clickhouse client -q 'SYSTEM START MERGES'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START TTL MERGES'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START MOVES'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START FETCHES'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START REPLICATED SENDS'", shell=True, stderr=STDOUT, timeout=30)
call("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'", shell=True, stderr=STDOUT, timeout=30)
# Issue #21004, live views are experimental, so let's just suppress it
call("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """, shell=True, stderr=STDOUT)
call("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """, shell=True, stderr=STDOUT, timeout=30)
# Kill other queries which known to be slow
# It's query from 01232_preparing_sets_race_condition_long, it may take up to 1000 seconds in slow builds
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """, shell=True, stderr=STDOUT)
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """, shell=True, stderr=STDOUT, timeout=30)
# Long query from 00084_external_agregation
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT)
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT, timeout=30)
if drop_databases:
# Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too.
# Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds).
databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True).decode('utf-8').strip().split()
databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True, timeout=30).decode('utf-8').strip().split()
for db in databases:
if db == "system":
continue
@ -117,13 +117,13 @@ def prepare_for_hung_check(drop_databases):
# Wait for last queries to finish if any, not longer than 300 seconds
call("""clickhouse client -q "select sleepEachRow((
select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300
) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT)
) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=330)
# Even if all clickhouse-test processes are finished, there are probably some sh scripts,
# which still run some new queries. Let's ignore them.
try:
query = """clickhouse client -q "SELECT count() FROM system.processes where where elapsed > 300" """
output = check_output(query, shell=True, stderr=STDOUT).decode('utf-8').strip()
output = check_output(query, shell=True, stderr=STDOUT, timeout=30).decode('utf-8').strip()
if int(output) == 0:
return False
except:
@ -176,6 +176,7 @@ if __name__ == "__main__":
if res != 0 and have_long_running_queries:
logging.info("Hung check failed with exit code {}".format(res))
hung_check_status = "Hung check failed\tFAIL\n"
open(os.path.join(args.output_folder, "test_results.tsv"), 'w+').write(hung_check_status)
with open(os.path.join(args.output_folder, "test_results.tsv"), 'w+') as results:
results.write(hung_check_status)
logging.info("Stress test finished")

View File

@ -89,10 +89,13 @@ def make_clickhouse_client(base_args):
# hence we should use 'system'.
database='system',
settings=get_additional_client_options_dict(base_args))
def clickhouse_execute_one(base_args, *args, **kwargs):
return make_clickhouse_client(base_args).execute_one(*args, **kwargs)
def clickhouse_execute(base_args, *args, **kwargs):
return make_clickhouse_client(base_args).execute(*args, **kwargs)
def clickhouse_execute_pandas(base_args, *args, **kwargs):
return make_clickhouse_client(base_args).execute_pandas(*args, **kwargs)
@ -109,6 +112,7 @@ def stop_tests():
global restarted_tests
with stop_tests_triggered_lock:
print("Stopping tests")
if not stop_tests_triggered.is_set():
stop_tests_triggered.set()
@ -875,7 +879,7 @@ def run_tests_array(all_tests_with_params):
while True:
if is_concurrent:
case = queue.get()
case = queue.get(timeout=args.timeout * 1.1)
if not case:
break
else:
@ -1076,10 +1080,10 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel):
pool.map_async(run_tests_array, parallel_tests_array)
for suit in test_suite.parallel_tests:
queue.put(suit)
queue.put(suit, timeout=args.timeout * 1.1)
for _ in range(jobs):
queue.put(None)
queue.put(None, timeout=args.timeout * 1.1)
queue.close()

View File

@ -7,9 +7,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CURL} --max-time 1 -sS "${CLICKHOUSE_URL}&query_id=cancel_http_readonly_queries_on_client_close&cancel_http_readonly_queries_on_client_close=1&query=SELECT+count()+FROM+system.numbers" 2>&1 | grep -cF 'curl: (28)'
while true
do
i=0 retries=300
while [[ $i -lt $retries ]]; do
${CLICKHOUSE_CURL} -sS --data "SELECT count() FROM system.processes WHERE query_id = 'cancel_http_readonly_queries_on_client_close'" "${CLICKHOUSE_URL}" | grep '0' && break
((++i))
sleep 0.2
done

View File

@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
i=0 retries=300
# Sometimes five seconds are not enough due to system overload.
# But if it can run in less than five seconds at least sometimes - it is enough for the test.
while true
do
while [[ $i -lt $retries ]]; do
opts=(
--max_distributed_connections 20
--max_threads 1
@ -19,4 +19,5 @@ do
# "$@" left to pass manual options (like --experimental_use_processors 0) during manual testing
timeout 10s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break
((++i))
done

View File

@ -8,9 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# Sometimes 1.8 seconds are not enough due to system overload.
# But if it can run in less than five seconds at least sometimes - it is enough for the test.
while true
do
i=0 retries=100
while [[ $i -lt $retries ]]; do
query="SELECT sleepEachRow(1) FROM remote('127.{2,3}', system.one) FORMAT Null"
# 1.8 less then 2 seconds, but long enough to cover possible load peaks
timeout 1.8s ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_distributed_connections=2&max_threads=1" -d "$query" && break
((++i))
done

View File

@ -18,9 +18,10 @@ for i in $(seq 1 $NUM_REPLICAS); do
done
function thread {
while true
do
i=0 retries=300
while [[ $i -lt $retries ]]; do # server can be dead
$CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break
((++i))
sleep 0.1
done
}

View File

@ -21,7 +21,9 @@ CREATE QUOTA quota_by_forwarded_ip_${CLICKHOUSE_DATABASE} KEYED BY forwarded_ip_
echo '--- Test with quota by immediate IP ---'
while true; do
i=0 retries=300
while [[ $i -lt $retries ]]; do
((++i))
${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break
done | uniq
@ -33,14 +35,18 @@ ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quo
echo '--- Test with quota by forwarded IP ---'
while true; do
i=0 retries=300
while [[ $i -lt $retries ]]; do
((++i))
${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break
done | uniq
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded'
i=0 retries=300
# X-Forwarded-For is respected for quota by forwarded IP address
while true; do
while [[ $i -lt $retries ]]; do
((++i))
${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oP '^10$' || break
done | uniq

View File

@ -13,14 +13,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# If concurrency is 10 (good), the query may take less than 10 second with non-zero probability
# and the following loops will finish with probability 1 assuming independent random variables.
while true; do
i=0 retries=30
while [[ $i -lt $retries ]]; do
timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query "
SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break
((++i))
done
while true; do
i=0 retries=30
while [[ $i -lt $retries ]]; do
timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query "
SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break
((++i))
done
# If max_distributed_connections is low and async_socket_for_remote is disabled,

View File

@ -4,12 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
counter=0 retries=60
I=0
while true
do
while [[ $counter -lt $retries ]]; do
I=$((I + 1))
TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I")
${CLICKHOUSE_CLIENT} --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break;
((++counter))
done
#echo "I = ${I}"

View File

@ -13,13 +13,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
M=1000000
while true
do
i=0 retries=300
while [[ $i -lt $retries ]]; do
$CLICKHOUSE_CLIENT --allow_hyperscan 1 --max_memory_usage $M --format Null --query "
SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', 'рикан', 'а', 'f[a${RANDOM}e]b[ei]rl', 'ф[иа${RANDOM}эе]б[еэи][рпл]', 'афиукд', 'a[f${RANDOM}t],th', '^ф[аие${RANDOM}э]?б?[еэи]?$', 'бе${RANDOM}рлик', 'fa${RANDOM}b', 'фа[беьв]+е?[рл${RANDOM}ко]']))
" 2>&1 | grep -q 'Memory limit' || break;
M=$((M + 100000))
((++i))
done
echo 'Ok'

View File

@ -16,9 +16,10 @@ insert into aliases_lazyness(x) select * from numbers(40);
# The exact time is not guaranteed, so we check in a loop that at least once
# the query will process in less than one second, that proves that the behaviour is not like it was long time ago.
while true
do
i=0 retries=300
while [[ $i -lt $retries ]]; do
timeout 1 ${CLICKHOUSE_CLIENT} --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" && break
((++i))
done
${CLICKHOUSE_CLIENT} --multiquery --query "

View File

@ -5,9 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
i=0 retries=60
# Sometimes five seconds are not enough due to system overload.
# But if it can run in less than five seconds at least sometimes - it is enough for the test.
while true
do
while [[ $i -lt $retries ]]; do
timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --query "SELECT * FROM url('http://127.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null && break
((++i))
done