Merge pull request #23517 from azat/flaky-tests-v21.6

Fix function tests flaps
This commit is contained in:
alexey-milovidov 2021-04-25 04:08:49 +03:00 committed by GitHub
commit 1c472b014d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 188 additions and 116 deletions

View File

@ -0,0 +1,59 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
function random_str()
{
local n=$1 && shift
tr -cd '[:lower:]' < /dev/urandom | head -c"$n"
}
function test_query_duration_ms()
{
local query_id
query_id="01548_query_log_query_execution_ms-$SECONDS-$(random_str 6)"
local query_opts=(
"--log_query_threads=1"
"--log_queries_min_type=QUERY_FINISH"
"--log_queries=1"
"--query_id=$query_id"
"--format=Null"
)
$CLICKHOUSE_CLIENT "${query_opts[@]}" -q "select sleep(0.4)" || exit 1
$CLICKHOUSE_CLIENT -q "system flush logs" || exit 1
$CLICKHOUSE_CLIENT -q "
select count()
from system.query_log
where
query_id = '$query_id'
and current_database = currentDatabase()
and query_duration_ms between 400 and 800
and event_date >= yesterday()
and event_time >= now() - interval 1 minute;
" || exit 1
$CLICKHOUSE_CLIENT -q "
-- at least two threads for processing
-- (but one just waits for another, sigh)
select count() == 2
from system.query_thread_log
where
query_id = '$query_id'
and current_database = currentDatabase()
and query_duration_ms between 400 and 800
and event_date >= yesterday()
and event_time >= now() - interval 1 minute;
" || exit 1
}
function main()
{
# retries, since there is no guarantee that every time query will take ~0.4 second.
local retries=20 i=0
while [ "$(test_query_duration_ms | xargs)" != '1 1' ] && [[ $i < $retries ]]; do
((++i))
done
}
main "$@"

View File

@ -1,27 +0,0 @@
set log_query_threads=1;
set log_queries_min_type='QUERY_FINISH';
set log_queries=1;
select '01548_query_log_query_execution_ms', sleep(0.4) format Null;
set log_queries=0;
set log_query_threads=0;
system flush logs;
select count()
from system.query_log
where
query like '%01548_query_log_query_execution_ms%'
and current_database = currentDatabase()
and query_duration_ms between 100 and 800
and event_date >= yesterday();
-- at least two threads for processing
-- (but one just waits for another, sigh)
select count() == 2
from system.query_thread_log
where
query like '%01548_query_log_query_execution_ms%'
and current_database = currentDatabase()
and query_duration_ms between 100 and 800
and event_date = today()
and event_time >= now() - interval 1 minute;

View File

@ -1,86 +0,0 @@
#!/usr/bin/env bash
# NOTE: $SECONDS accuracy is second, so we need some delta, hence -1 in time conditions.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
max_delay_to_insert=5
${CLICKHOUSE_CLIENT} -nq "
drop table if exists dist_01675;
drop table if exists data_01675;
"
${CLICKHOUSE_CLIENT} -nq "
create table data_01675 (key Int) engine=Null();
create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert;
system stop distributed sends dist_01675;
"
#
# Case 1: max_delay_to_insert will throw.
#
echo "max_delay_to_insert will throw"
start_seconds=$SECONDS
${CLICKHOUSE_CLIENT} --testmode -nq "
-- first batch is always OK, since there is no pending bytes yet
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
-- second will fail, because of bytes_to_delay_insert=1 and max_delay_to_insert=5,
-- while distributed sends is stopped.
--
-- (previous block definitelly takes more, since it has header)
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; -- { serverError 574 }
system flush distributed dist_01675;
"
end_seconds=$SECONDS
if (( (end_seconds-start_seconds)<(max_delay_to_insert-1) )); then
echo "max_delay_to_insert was not satisfied ($end_seconds-$start_seconds)"
fi
#
# Case 2: max_delay_to_insert will finally finished.
#
echo "max_delay_to_insert will succeed"
max_delay_to_insert=10
${CLICKHOUSE_CLIENT} -nq "
drop table dist_01675;
create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert;
system stop distributed sends dist_01675;
"
flush_delay=4
function flush_distributed_worker()
{
sleep $flush_delay
${CLICKHOUSE_CLIENT} -q "system flush distributed dist_01675"
echo flushed
}
flush_distributed_worker &
start_seconds=$SECONDS
${CLICKHOUSE_CLIENT} --testmode -nq "
-- first batch is always OK, since there is no pending bytes yet
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
-- second will succcedd, due to SYSTEM FLUSH DISTRIBUTED in background.
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
"
end_seconds=$SECONDS
wait
if (( (end_seconds-start_seconds)<(flush_delay-1) )); then
echo "max_delay_to_insert was not wait flush_delay ($end_seconds-$start_seconds)"
fi
if (( (end_seconds-start_seconds)>=(max_delay_to_insert-1) )); then
echo "max_delay_to_insert was overcommited ($end_seconds-$start_seconds)"
fi
${CLICKHOUSE_CLIENT} -nq "
drop table dist_01675;
drop table data_01675;
"

View File

@ -0,0 +1,129 @@
#!/usr/bin/env bash
# NOTE: $SECONDS accuracy is second, so we need some delta, hence -1 in time conditions.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function drop_tables()
{
${CLICKHOUSE_CLIENT} -nq "
drop table if exists dist_01675;
drop table if exists data_01675;
"
}
#
# Case 1: max_delay_to_insert will throw.
#
function test_max_delay_to_insert_will_throw()
{
echo "max_delay_to_insert will throw"
local max_delay_to_insert=2
${CLICKHOUSE_CLIENT} -nq "
create table data_01675 (key Int) engine=Null();
create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert;
system stop distributed sends dist_01675;
"
local start_seconds=$SECONDS
${CLICKHOUSE_CLIENT} --testmode -nq "
-- first batch is always OK, since there is no pending bytes yet
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
-- second will fail, because of bytes_to_delay_insert=1 and max_delay_to_insert>0,
-- while distributed sends is stopped.
--
-- (previous block definitelly takes more, since it has header)
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; -- { serverError 574 }
system flush distributed dist_01675;
"
local end_seconds=$SECONDS
if (( (end_seconds-start_seconds)<(max_delay_to_insert-1) )); then
echo "max_delay_to_insert was not satisfied ($end_seconds-$start_seconds)"
fi
}
#
# Case 2: max_delay_to_insert will finally finished.
#
function test_max_delay_to_insert_will_succeed_once()
{
local max_delay_to_insert=4
local flush_delay=2
drop_tables
${CLICKHOUSE_CLIENT} -nq "
create table data_01675 (key Int) engine=Null();
create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert;
system stop distributed sends dist_01675;
"
function flush_distributed_worker()
{
sleep $flush_delay
${CLICKHOUSE_CLIENT} -q "system flush distributed dist_01675"
}
flush_distributed_worker &
local start_seconds=$SECONDS
# ignore stderr, since it may produce exception if flushing thread will be too slow
# (this is possible on CI)
${CLICKHOUSE_CLIENT} --testmode -nq "
-- first batch is always OK, since there is no pending bytes yet
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
-- second will succeed, due to SYSTEM FLUSH DISTRIBUTED in background.
insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0;
" >& /dev/null
local end_seconds=$SECONDS
wait
local diff=$(( end_seconds-start_seconds ))
if (( diff<(flush_delay-1) )); then
# this is fatal error, that should not be retriable
echo "max_delay_to_insert was not wait flush_delay ($diff)"
exit 1
fi
# retry the test until the diff will be satisfied
# (since we cannot assume that there will be no other lags)
if (( diff>=(max_delay_to_insert-1) )); then
return 1
fi
return 0
}
function test_max_delay_to_insert_will_succeed()
{
echo "max_delay_to_insert will succeed"
local retries=20 i=0
while (( (i++) < retries )); do
if test_max_delay_to_insert_will_succeed_once; then
return
fi
done
echo failed
}
function run_test()
{
local test_case=$1 && shift
drop_tables
$test_case
}
function main()
{
run_test test_max_delay_to_insert_will_throw
run_test test_max_delay_to_insert_will_succeed
drop_tables
}
main "$@"