Merge remote-tracking branch 'origin/aku/flaky-stateless' into trying_parallel_func_tests

This commit is contained in:
alesapin 2021-03-29 17:25:23 +03:00
commit a423540452
21 changed files with 93 additions and 86 deletions

View File

@ -74,7 +74,13 @@ timeout "$MAX_RUN_TIME" bash -c run_tests ||:
./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz ||:
clickhouse-client -q "sytem flush logs" ||:
pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz &
clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz &
clickhouse-client -q "select * from system.query_thread_log format TSVWithNamesAndTypes" | pigz > /test_output/query-thread-log.tsv.gz &
wait ||:
mv /var/log/clickhouse-server/stderr.log /test_output/ ||:
if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then
tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||:

View File

@ -170,7 +170,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
if need_drop_database:
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10)
seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20)
try:
clickhouse_proc_create.communicate(("DROP DATABASE " + database), timeout=seconds_left)
except TimeoutExpired:
@ -207,7 +207,7 @@ def need_retry(stderr):
def get_processlist(args):
try:
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10)
(stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=20)
return False, stdout.decode('utf-8')
except Exception as ex:
print("Exception", ex)
@ -352,7 +352,7 @@ def run_tests_array(all_tests_with_params):
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
failed_to_check = False
try:
clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=10)
clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=20)
except:
failed_to_check = True

View File

@ -48,7 +48,7 @@ SELECT
threads_realtime >= threads_time_user_system_io,
any(length(thread_ids)) >= 1
FROM
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1)
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-2 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1)
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
# Clean

View File

@ -97,7 +97,7 @@ echo 7
# and finally querylog
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from system.query_log where current_database = currentDatabase() AND event_time > now() - 10 and query like '%TOPSECRET%';"
--query="select * from system.query_log where current_database = currentDatabase() AND event_date >= yesterday() and query like '%TOPSECRET%';"
rm -f "$tmp_file" >/dev/null 2>&1

View File

@ -2,26 +2,33 @@ set log_queries=1;
select '01231_log_queries_min_type/QUERY_START';
system flush logs;
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
select count() from system.query_log where current_database = currentDatabase()
and query like 'select \'01231_log_queries_min_type/QUERY_START%'
and event_date >= yesterday();
set log_queries_min_type='EXCEPTION_BEFORE_START';
select '01231_log_queries_min_type/EXCEPTION_BEFORE_START';
system flush logs;
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
select count() from system.query_log where current_database = currentDatabase()
and query like 'select \'01231_log_queries_min_type/EXCEPTION_BEFORE_START%'
and event_date >= yesterday();
set max_rows_to_read='100K';
set log_queries_min_type='EXCEPTION_WHILE_PROCESSING';
select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
set max_rows_to_read=0;
system flush logs;
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing';
select count() from system.query_log where current_database = currentDatabase()
and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%'
and event_date >= yesterday() and type = 'ExceptionWhileProcessing';
set max_rows_to_read='100K';
select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
system flush logs;
set max_rows_to_read=0;
select count() from system.query_log where
current_database = currentDatabase() and
query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and
query not like '%system.query_log%' and
event_date = today() and
event_time >= now() - interval 1 minute and
query like 'select \'01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and
event_date >= yesterday() and
type = 'ExceptionWhileProcessing' and
has(Settings.Names, 'max_rows_to_read');

View File

@ -9,12 +9,12 @@ $CLICKHOUSE_CLIENT --multiquery --query "
CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date;
INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000);
OPTIMIZE TABLE bug FINAL;"
LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE"
$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$LOG"
cat "$LOG" | grep Exception
cat "$LOG" | grep Loaded
$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err
cat "$CLICKHOUSE_TMP"/err | grep Exception
cat "$CLICKHOUSE_TMP"/err | grep Loaded
rm "$CLICKHOUSE_TMP"/err
rm "$LOG"
$CLICKHOUSE_CLIENT --multiquery --query "
DROP TABLE bug;"

View File

@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
SELECT * FROM test_01344 WHERE x = 'Hello, world';
SYSTEM FLUSH LOGS;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
DROP TABLE test_01344;

View File

@ -17,7 +17,7 @@ CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
*
FROM system.query_log
WHERE type<>1 and event_date >= yesterday() and event_time > now() - 120
WHERE type<>1 and event_date >= yesterday()
) as ql
INNER JOIN expected_times USING (QUERY_GROUP_ID)
WHERE query_duration_ms > max_query_duration_ms
@ -38,7 +38,7 @@ SELECT
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
count()
FROM system.query_log
WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>''
WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and QUERY_GROUP_ID<>''
GROUP BY QUERY_GROUP_ID
ORDER BY QUERY_GROUP_ID;

View File

@ -10,7 +10,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND current_database = currentDatabase();
SELECT '====tuple====';
DROP TABLE IF EXISTS t_tup;
@ -27,7 +27,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND current_database = currentDatabase();
SELECT '====nullable====';
DROP TABLE IF EXISTS t_nul;
@ -41,7 +41,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND current_database = currentDatabase();
SELECT '====map====';
SET allow_experimental_map_type = 1;
@ -60,7 +60,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND current_database = currentDatabase();
DROP TABLE t_arr;
DROP TABLE t_nul;

View File

@ -1,20 +1,20 @@
set log_queries=1;
set log_queries_min_type='QUERY_FINISH';
set enable_global_with_statement=1;
set enable_global_with_statement=0;
select /* test=01531, enable_global_with_statement=0 */ 2;
system flush logs;
select count() from system.query_log
where event_time >= now() - interval 5 minute
and query like '%select /* test=01531, enable_global_with_statement=0 */ 2%'
where event_date >= yesterday()
and query like 'select /* test=01531, enable_global_with_statement=0 */ 2%'
and current_database = currentDatabase()
;
set enable_global_with_statement=1;
select /* test=01531 enable_global_with_statement=1 */ 2;
select /* test=01531, enable_global_with_statement=1 */ 2;
system flush logs;
select count() from system.query_log
where event_time >= now() - interval 5 minute
and query like '%select /* test=01531 enable_global_with_statement=1 */ 2%'
where event_date >= yesterday()
and query like 'select /* test=01531, enable_global_with_statement=1 */ 2%'
and current_database = currentDatabase()
;

View File

@ -13,16 +13,16 @@ col3
read files
4
6
0 899984 7199412
1 899987 7199877
2 899990 7200255
3 899993 7199883
4 899996 7199798
5 899999 7200306
6 900002 7200064
7 900005 7199429
8 900008 7200067
9 899992 7199993
0 89982 719752
1 89988 720017
2 89994 720152
3 90000 720157
4 90006 720100
5 90012 720168
6 90018 720106
7 90005 719891
8 89992 719854
9 89979 719706
0 []
0 [0]
1 [0,2]

View File

@ -36,7 +36,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND event_date >= yesterday() AND current_database = currentDatabase();
SYSTEM DROP MARK CACHE;
SELECT col3.n2.s FROM nested FORMAT Null;
@ -46,7 +46,7 @@ SYSTEM FLUSH LOGS;
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
FROM system.query_log
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%'))
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
AND event_date >= yesterday() AND current_database = currentDatabase();
DROP TABLE nested;
@ -59,7 +59,7 @@ ENGINE = MergeTree
ORDER BY id
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(1000000);
INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(100000);
SELECT id % 10, sum(length(col1)), sumArray(arrayMap(x -> length(x), col1.n.b)) FROM nested GROUP BY id % 10;
SELECT arraySum(col1.a), arrayMap(x -> x * x * 2, col1.a) FROM nested ORDER BY id LIMIT 5;

View File

@ -12,19 +12,15 @@ system flush logs;
select count()
from system.query_log
where
query like '%01546_log_queries_min_query_duration_ms-fast%'
and query not like '%system.query_log%'
query like 'select \'01546_log_queries_min_query_duration_ms-fast%'
and current_database = currentDatabase()
and event_date = today()
and event_time >= now() - interval 1 minute;
and event_date >= yesterday();
select count()
from system.query_thread_log
where
query like '%01546_log_queries_min_query_duration_ms-fast%'
and query not like '%system.query_thread_log%'
query like 'select \'01546_log_queries_min_query_duration_ms-fast%'
and current_database = currentDatabase()
and event_date = today()
and event_time >= now() - interval 1 minute;
and event_date >= yesterday();
--
-- slow -- query logged
@ -37,18 +33,14 @@ system flush logs;
select count()
from system.query_log
where
query like '%01546_log_queries_min_query_duration_ms-slow%'
and query not like '%system.query_log%'
query like 'select \'01546_log_queries_min_query_duration_ms-slow%'
and current_database = currentDatabase()
and event_date = today()
and event_time >= now() - interval 1 minute;
and event_date >= yesterday();
-- There at least two threads involved in a simple query
-- (one thread just waits another, sigh)
select count() == 2
select if(count() == 2, 'OK', 'Fail: ' || toString(count()))
from system.query_thread_log
where
query like '%01546_log_queries_min_query_duration_ms-slow%'
and query not like '%system.query_thread_log%'
query like 'select \'01546_log_queries_min_query_duration_ms-slow%'
and current_database = currentDatabase()
and event_date = today()
and event_time >= now() - interval 1 minute;
and event_date >= yesterday();

View File

@ -23,8 +23,7 @@ from system.query_log
where
query like '%01547_query_log_current_database%'
and current_database = currentDatabase()
and event_date = today()
and event_time >= now() - interval 1 minute;
and event_date >= yesterday();
-- at least two threads for processing
-- (but one just waits for another, sigh)

View File

@ -13,8 +13,7 @@ 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;
and event_date >= yesterday();
-- at least two threads for processing
-- (but one just waits for another, sigh)

View File

@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err
LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE"
$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$LOG"
cat "$CLICKHOUSE_TMP"/err | grep Exception
cat "$CLICKHOUSE_TMP"/err | grep Loaded
cat "$LOG" | grep Exception
cat "$LOG" | grep Loaded
rm "$CLICKHOUSE_TMP"/err
rm "$LOG"

View File

@ -15,6 +15,8 @@ SYSTEM FLUSH LOGS;
WITH (
SELECT (event_time, event_time_microseconds)
FROM system.part_log
WHERE "table" = 'table_with_single_pk'
AND "database" = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time

View File

@ -136,7 +136,7 @@ SELECT 'ACTUAL LOG CONTENT:';
-- Try to filter out all possible previous junk events by excluding old log entries,
SELECT query_kind, query FROM system.query_log
WHERE
log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_time >= now() - 10
log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_date >= yesterday()
AND current_database == currentDatabase()
ORDER BY event_time_microseconds;

View File

@ -525,8 +525,8 @@
"00571_non_exist_database_when_create_materializ_view",
"00575_illegal_column_exception_when_drop_depen_column",
"00599_create_view_with_subquery",
"00604_show_create_database",
"00600_replace_running_query",
"00604_show_create_database",
"00612_http_max_query_size",
"00619_union_highlite",
"00620_optimize_on_nonleader_replica_zookeeper",
@ -577,6 +577,7 @@
"00933_test_fix_extra_seek_on_compressed_cache",
"00933_ttl_replicated_zookeeper",
"00933_ttl_with_default",
"00950_dict_get",
"00955_test_final_mark",
"00976_ttl_with_old_parts",
"00980_merge_alter_settings",
@ -740,8 +741,8 @@
"01530_drop_database_atomic_sync",
"01541_max_memory_usage_for_user_long",
"01542_dictionary_load_exception_race",
"01560_optimize_on_insert_zookeeper",
"01545_system_errors", // looks at the difference of values in system.errors
"01560_optimize_on_insert_zookeeper",
"01575_disable_detach_table_of_dictionary",
"01593_concurrent_alter_mutations_kill",
"01593_concurrent_alter_mutations_kill_many_replicas",
@ -754,11 +755,23 @@
"01603_rename_overwrite_bug",
"01646_system_restart_replicas_smoke", // system restart replicas is a global query
"01656_test_query_log_factories_info",
"01658_read_file_to_stringcolumn",
"01669_columns_declaration_serde",
"01676_dictget_in_default_expression",
"01681_cache_dictionary_simple_key",
"01682_cache_dictionary_complex_key",
"01683_flat_dictionary",
"01684_ssd_cache_dictionary_simple_key",
"01685_ssd_cache_dictionary_complex_key",
"01700_system_zookeeper_path_in",
"01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test.
"01702_system_query_log", // Runs many global system queries
"01715_background_checker_blather_zookeeper",
"01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test.
"01747_alter_partition_key_enum_zookeeper",
"01748_dictionary_table_dot", // creates database
"01760_polygon_dictionaries",
"01760_system_dictionaries",
"01761_alter_decimal_zookeeper",
"attach",
"ddl_dictionaries",
@ -767,18 +780,6 @@
"live_view",
"memory_leak",
"memory_limit",
"polygon_dicts", // they use an explicitly specified database
"01658_read_file_to_stringcolumn",
"01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test.
"01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test.
"01748_dictionary_table_dot", // creates database
"00950_dict_get",
"01683_flat_dictionary",
"01681_cache_dictionary_simple_key",
"01682_cache_dictionary_complex_key",
"01684_ssd_cache_dictionary_simple_key",
"01685_ssd_cache_dictionary_complex_key",
"01760_system_dictionaries",
"01760_polygon_dictionaries"
"polygon_dicts" // they use an explicitly specified database
]
}