mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Backport #70617 to 24.9: Increase max_rows_to_read limit in some tests
This commit is contained in:
parent
f9e7d4c8b2
commit
f0e202bc63
@ -293,6 +293,7 @@ def clickhouse_execute_http(
|
||||
"http_receive_timeout": timeout,
|
||||
"http_send_timeout": timeout,
|
||||
"output_format_parallel_formatting": 0,
|
||||
"max_rows_to_read": 0, # Some queries read from system.text_log which might get too big
|
||||
}
|
||||
if settings is not None:
|
||||
params.update(settings)
|
||||
|
@ -9,7 +9,7 @@ system flush logs;
|
||||
drop table if exists logs;
|
||||
create view logs as select * from system.text_log where now() - toIntervalMinute(120) < event_time;
|
||||
|
||||
SET max_rows_to_read = 0;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
|
||||
-- Check that we don't have too many messages formatted with fmt::runtime or strings concatenation.
|
||||
-- 0.001 threshold should be always enough, the value was about 0.00025
|
||||
|
@ -144,8 +144,7 @@ echo 9
|
||||
$CLICKHOUSE_CLIENT \
|
||||
--server_logs_file=/dev/null \
|
||||
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_date >= yesterday() and message like '%find_me%';
|
||||
select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%';" --ignore-error
|
||||
|
||||
select * from system.text_log where event_date >= yesterday() and message like '%TOPSECRET=TOPSECRET%' SETTINGS max_rows_to_read = 0" --ignore-error
|
||||
echo 'finish'
|
||||
rm -f "$tmp_file" >/dev/null 2>&1
|
||||
rm -f "$tmp_file2" >/dev/null 2>&1
|
||||
|
@ -6,12 +6,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT 6103"
|
||||
|
||||
for (( i=1; i <= 50; i++ ))
|
||||
for (( i=1; i <= 50; i++ ))
|
||||
do
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SYSTEM FLUSH LOGS"
|
||||
sleep 0.1;
|
||||
if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0 AND event_date >= yesterday()") == 1 ]]; then echo 1; exit; fi;
|
||||
if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0 AND event_date >= yesterday() SETTINGS max_rows_to_read = 0") == 1 ]]; then echo 1; exit; fi;
|
||||
|
||||
done;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: zookeeper
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
|
||||
create table rmt1 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '1') order by n partition by toYYYYMMDD(d);
|
||||
create table rmt2 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '2') order by n partition by toYYYYMMDD(d);
|
||||
|
||||
|
@ -2,5 +2,6 @@ SeLeCt 'ab
|
||||
cd' /* hello */ -- world
|
||||
, 1;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT extract(message, 'SeL.+?;') FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%SeLeCt \'ab\n%' ORDER BY event_time DESC LIMIT 1 FORMAT TSVRaw;
|
||||
|
@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --max_block_siz
|
||||
# Now wait for cleanup thread
|
||||
for _ in {1..60}; do
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS"
|
||||
[[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break;
|
||||
[[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%' SETTINGS max_rows_to_read = 0") -gt $((SCALE - 10)) ]] && break;
|
||||
sleep 1
|
||||
done
|
||||
|
||||
|
@ -72,6 +72,6 @@ ${CLICKHOUSE_CLIENT} --query_id "$query_id" --query "select i from simple where
|
||||
# We have to search the server's error log because the following warning message
|
||||
# is generated during pipeline destruction and thus is not sent to the client.
|
||||
${CLICKHOUSE_CLIENT} --query "system flush logs"
|
||||
if [[ $(${CLICKHOUSE_CLIENT} --query "select count() > 0 from system.text_log where query_id = '$query_id' and level = 'Warning' and message like '%We have query_id removed but it\'s not recorded. This is a bug%' format TSVRaw") == 1 ]]; then echo "We have query_id removed but it's not recorded. This is a bug." >&2; exit 1; fi
|
||||
if [[ $(${CLICKHOUSE_CLIENT} --query "select count() > 0 from system.text_log where query_id = '$query_id' and level = 'Warning' and message like '%We have query_id removed but it\'s not recorded. This is a bug%' format TSVRaw SETTINGS max_rows_to_read = 0") == 1 ]]; then echo "We have query_id removed but it's not recorded. This is a bug." >&2; exit 1; fi
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "drop table simple"
|
||||
|
@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -m -q "
|
||||
-- OPTIMIZE TABLE x FINAL will be done in background
|
||||
-- attach to it's log, via table UUID in query_id (see merger/mutator code).
|
||||
create materialized view this_text_log engine=Memory() as
|
||||
select * from system.text_log where query_id like '%${ttl_02262_uuid}%';
|
||||
select * from system.text_log where query_id like '%${ttl_02262_uuid}%' SETTINGS max_rows_to_read = 0;
|
||||
|
||||
optimize table ttl_02262 final;
|
||||
system flush logs;
|
||||
|
@ -64,5 +64,6 @@ drop table rmt;
|
||||
drop table rmt2;
|
||||
|
||||
system flush logs;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
select count() > 0 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts from filesystem (concurrently): Parts:%';
|
||||
select count() > 1, countDistinct(thread_id) > 1 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts in blocks range%';
|
||||
|
@ -99,4 +99,4 @@ $CLICKHOUSE_CLIENT -q 'system flush logs'
|
||||
$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and (
|
||||
message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes',
|
||||
'Query was cancelled or a client has unexpectedly dropped the connection') or
|
||||
message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')"
|
||||
message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%') SETTINGS max_rows_to_read = 0"
|
||||
|
@ -15,6 +15,7 @@ alter table t materialize projection p_norm settings mutations_sync = 1;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT * FROM system.text_log WHERE event_time >= now() - 30 and level == 'Error' and message like '%BAD_DATA_PART_NAME%'and message like '%p_norm%';
|
||||
|
||||
DROP TABLE IF EXISTS t;
|
||||
|
@ -12,5 +12,5 @@ do
|
||||
query_id=$(echo "select queryID() from (select sum(s), k from remote('127.0.0.{1,2}', view(select sum(number) s, bitAnd(number, 3) k from numbers_mt(1000000) group by k)) group by k) limit 1 settings group_by_two_level_threshold=1, max_threads=3, prefer_localhost_replica=1" | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- 2>&1)
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="system flush logs"
|
||||
${CLICKHOUSE_CLIENT} --query="select count() from system.text_log where event_date >= yesterday() and query_id = '${query_id}' and message like '%Converting aggregation data to two-level%'" | grep -P '^6$' && break;
|
||||
${CLICKHOUSE_CLIENT} --query="select count() from system.text_log where event_date >= yesterday() and query_id = '${query_id}' and message like '%Converting aggregation data to two-level%' SETTINGS max_rows_to_read = 0" | grep -P '^6$' && break;
|
||||
done
|
||||
|
@ -48,6 +48,7 @@ $CLICKHOUSE_CLIENT --query "
|
||||
SELECT 'id_' || splitByChar('_', query_id)[1] AS id FROM system.text_log
|
||||
WHERE query_id LIKE '%$query_id_suffix' AND message LIKE '%$message%'
|
||||
ORDER BY id
|
||||
SETTINGS max_rows_to_read = 0
|
||||
"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t_async_insert_fallback"
|
||||
|
@ -30,6 +30,7 @@ SELECT count() FROM 02581_trips SETTINGS select_sequential_consistency = 1;
|
||||
DELETE FROM 02581_trips WHERE id IN (SELECT (number*10 + 9)::UInt32 FROM numbers(10000000)) SETTINGS lightweight_deletes_sync = 2;
|
||||
SELECT count(), _part from 02581_trips WHERE description = '' GROUP BY _part ORDER BY _part SETTINGS select_sequential_consistency=1;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' )
|
||||
-- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' )
|
||||
|
@ -58,6 +58,7 @@ WHERE
|
||||
SETTINGS mutations_sync=2;
|
||||
SELECT count() from 02581_trips WHERE description = '';
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SYSTEM FLUSH LOGS;
|
||||
-- Check that in every mutation there were parts that built sets (log messages like 'Created Set with 10000000 entries from 10000000 rows in 0.388989187 sec.' )
|
||||
-- and parts that shared sets (log messages like 'Got set from cache in 0.388930505 sec.' )
|
||||
|
@ -30,5 +30,5 @@ ORDER BY column;
|
||||
DROP TABLE t_sparse_columns_clear SYNC;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count(), groupArray(message) FROM system.text_log WHERE logger_name LIKE '%' || currentDatabase() || '.t_sparse_columns_clear' || '%' AND level = 'Error';
|
||||
|
@ -1,5 +1,6 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT 'Hello', throwIf(1); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
|
@ -1,2 +1,3 @@
|
||||
SYSTEM FLUSH LOGS;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Starting ClickHouse%';
|
||||
|
@ -8,6 +8,7 @@ SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_lo
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas=0;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count() > 0 FROM system.text_log
|
||||
WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_190aed82-2423-413b-ad4c-24dcca50f65b')
|
||||
AND message LIKE '%Parallel reading from replicas is disabled for cluster%';
|
||||
|
@ -8,6 +8,7 @@ SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), tt) settings log
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas=0;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count() > 0 FROM system.text_log
|
||||
WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '02875_89f3c39b-1919-48cb-b66e-ef9904e73146')
|
||||
AND message LIKE '%Parallel reading from replicas is disabled for cluster%';
|
||||
|
@ -34,6 +34,7 @@ SELECT count() FROM checksums_r3;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT * FROM system.text_log WHERE event_time >= now() - INTERVAL 120 SECOND and level == 'Error' and message like '%CHECKSUM_DOESNT_MATCH%' and logger_name like ('%' || currentDatabase() || '%checksums_r%');
|
||||
|
||||
DROP TABLE IF EXISTS checksums_r3;
|
||||
|
@ -21,6 +21,7 @@ SELECT count(), min(k), max(k), avg(k) FROM t1 SETTINGS log_comment='02898_defau
|
||||
|
||||
-- check logs
|
||||
SYSTEM FLUSH LOGS;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count() > 0 FROM system.text_log
|
||||
WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_default_190aed82-2423-413b-ad4c-24dcca50f65b')
|
||||
AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
@ -56,7 +56,8 @@ for _ in {0..50}; do
|
||||
(
|
||||
(logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or
|
||||
(logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%')
|
||||
);
|
||||
)
|
||||
SETTINGS max_rows_to_read = 0;
|
||||
")
|
||||
if [[ $no_active_repilica_messages -gt 0 ]]; then
|
||||
break
|
||||
@ -78,5 +79,6 @@ $CLICKHOUSE_CLIENT -m -q "
|
||||
(logger_name = 'MergeTreeBackgroundExecutor' and message like '%{$table_uuid::$part_name}%No active replica has part $part_name or covering part%') or
|
||||
(logger_name like '$table_uuid::$part_name (MergeFromLogEntryTask)' and message like '%No active replica has part $part_name or covering part%')
|
||||
)
|
||||
group by level;
|
||||
group by level
|
||||
SETTINGS max_rows_to_read = 0;
|
||||
"
|
||||
|
@ -11,7 +11,7 @@ SET cluster_for_parallel_replicas='parallel_replicas';
|
||||
SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*) settings log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f';
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
SELECT count() > 0 FROM system.text_log
|
||||
WHERE yesterday() <= event_date
|
||||
AND query_id in (select query_id from system.query_log where current_database=currentDatabase() AND log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f')
|
||||
|
@ -6,6 +6,7 @@ select conut(); -- { serverError UNKNOWN_FUNCTION }
|
||||
|
||||
system flush logs;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB';
|
||||
|
||||
select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = '`count`' and value4 = 'SELECT count';
|
||||
|
@ -11,6 +11,7 @@ system disable failpoint replicated_sends_failpoint;
|
||||
system sync replica data_r2;
|
||||
|
||||
system flush logs;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
select event_time_microseconds, logger_name, message from system.text_log where level = 'Error' and message like '%Malformed chunked encoding%' order by 1 format LineAsString;
|
||||
|
||||
-- { echoOn }
|
||||
|
@ -24,7 +24,7 @@ SELECT count() FROM t_ind_merge_1 WHERE b < 100 SETTINGS force_data_skipping_ind
|
||||
EXPLAIN indexes = 1 SELECT count() FROM t_ind_merge_1 WHERE b < 100;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
WITH
|
||||
(SELECT uuid FROM system.tables WHERE database = currentDatabase() AND table = 't_ind_merge_1') AS uuid,
|
||||
extractAllGroupsVertical(message, 'containing (\\d+) columns \((\\d+) merged, (\\d+) gathered\)')[1] AS groups
|
||||
|
@ -26,6 +26,7 @@ INSERT INTO t_ind_merge_2 SELECT number, number, rand(), rand(), rand(), rand()
|
||||
|
||||
OPTIMIZE TABLE t_ind_merge_2 FINAL;
|
||||
SYSTEM FLUSH LOGS;
|
||||
SET max_rows_to_read = 0; -- system.text_log can be really big
|
||||
|
||||
--- merged: a, c, d; gathered: b, e, f
|
||||
WITH
|
||||
|
@ -15,7 +15,8 @@ check_replicas_read_in_order() {
|
||||
SELECT COUNT() > 0
|
||||
FROM system.text_log
|
||||
WHERE query_id IN (SELECT query_id FROM system.query_log WHERE query_id != '$1' AND initial_query_id = '$1' AND event_date >= yesterday())
|
||||
AND event_date >= yesterday() AND message ILIKE '%Reading%ranges in order%'"
|
||||
AND event_date >= yesterday() AND message ILIKE '%Reading%ranges in order%'
|
||||
SETTINGS max_rows_to_read=0"
|
||||
}
|
||||
|
||||
# replicas should use reading in order following initiator's decision to execute aggregation in order.
|
||||
|
Loading…
Reference in New Issue
Block a user