Do not add event_date/event_time condition for system.*_log

Also some of them can be even more simplified, since the system.*_log
are empty most of the time there should not be any conditions at all
(i.e. `query` LIKE and so on).
This commit is contained in:
Azat Khuzhin 2020-05-18 21:18:37 +03:00
parent bdbfa08e02
commit 32071ba080
10 changed files with 22 additions and 22 deletions

View File

@ -47,7 +47,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 type=2 ORDER BY event_time DESC LIMIT 1)
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE type=2 ORDER BY event_time DESC LIMIT 1)
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
# Check per-thread and per-query ProfileEvents consistency
@ -58,7 +58,7 @@ SELECT PN, PVq, PVt FROM
SELECT PN, sum(PV) AS PVt
FROM system.query_thread_log
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE event_date >= today()-1 AND query_id='$query_id'
WHERE query_id='$query_id'
GROUP BY PN
) js1
ANY INNER JOIN
@ -66,7 +66,7 @@ ANY INNER JOIN
SELECT PN, PV AS PVq
FROM system.query_log
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE event_date >= today()-1 AND query_id='$query_id'
WHERE query_id='$query_id'
) js2
USING PN
WHERE

View File

@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompresse
sleep 1
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"

View File

@ -95,7 +95,7 @@ echo 7
# and finally querylog
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from system.query_log where event_time>now() - 10 and query like '%TOPSECRET%';"
--query="select * from system.query_log where query like '%TOPSECRET%';"
rm -f $tmp_file >/dev/null 2>&1
@ -117,8 +117,8 @@ sleep 0.1;
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_time>now() - 60 and message like '%find_me%';
select * from system.text_log where event_time>now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE message like '%find_me%';
select * from system.text_log where message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
echo 'finish'
rm -f $tmp_file >/dev/null 2>&1

View File

@ -5,7 +5,7 @@ SET log_queries = 1;
SELECT sleep(0.5), ignore('test real time query profiler');
SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test real time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%';
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test real time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%';
SET query_profiler_real_time_period_ns = 0;
SET query_profiler_cpu_time_period_ns = 1000000;
@ -13,4 +13,4 @@ SET log_queries = 1;
SELECT count(), ignore('test cpu time query profiler') FROM numbers(1000000000);
SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test cpu time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Source%';
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test cpu time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Source%';

View File

@ -10,7 +10,7 @@ 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") == 1 ]]; then echo 1; exit; fi;
done;

View File

@ -3,5 +3,5 @@ SELECT * FROM test_table_for_01070_exception_code_in_query_log_table; -- { serve
CREATE TABLE test_table_for_01070_exception_code_in_query_log_table (value UInt64) ENGINE=Memory();
SELECT * FROM test_table_for_01070_exception_code_in_query_log_table;
SYSTEM FLUSH LOGS;
SELECT exception_code FROM system.query_log WHERE query = 'SELECT * FROM test_table_for_01070_exception_code_in_query_log_table' AND event_date >= yesterday() AND event_time > now() - INTERVAL 5 MINUTE ORDER BY exception_code;
SELECT exception_code FROM system.query_log WHERE query = 'SELECT * FROM test_table_for_01070_exception_code_in_query_log_table' ORDER BY exception_code;
DROP TABLE IF EXISTS test_table_for_01070_exception_code_in_query_log_table;

View File

@ -8,13 +8,13 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT 1') AND (event_date >= (today() - 1))
WHERE (query = 'SELECT 1')
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id)
FROM system.query_thread_log
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
WHERE (query_id = id) AND (thread_id != master_thread_id);
select sum(number) from numbers(1000000);
SYSTEM FLUSH LOGS;
@ -23,13 +23,13 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT sum(number) FROM numbers(1000000)') AND (event_date >= (today() - 1))
WHERE (query = 'SELECT sum(number) FROM numbers(1000000)')
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id)
FROM system.query_thread_log
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
WHERE (query_id = id) AND (thread_id != master_thread_id);
select sum(number) from numbers_mt(1000000);
SYSTEM FLUSH LOGS;
@ -38,10 +38,10 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT sum(number) FROM numbers_mt(1000000)') AND (event_date >= (today() - 1))
WHERE (query = 'SELECT sum(number) FROM numbers_mt(1000000)')
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id) > 2
FROM system.query_thread_log
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
WHERE (query_id = id) AND (thread_id != master_thread_id);

View File

@ -3,4 +3,4 @@ SET allow_introspection_functions = 1;
SET memory_profiler_step = 1000000;
SELECT ignore(groupArray(number), 'test memory profiler') FROM numbers(10000000);
SYSTEM FLUSH LOGS;
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);

View File

@ -3,4 +3,4 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'"
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE query_id = 'test_quota_key'"

View File

@ -2,14 +2,14 @@ set log_queries=1;
select '01231_log_queries_min_type/QUERY_START';
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/%' 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 query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';
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 query like '%01231_log_queries_min_type/%' 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 query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';
set log_queries_min_type='EXCEPTION_WHILE_PROCESSING';
select '01231_log_queries_min_type/', max(number) from system.numbers limit 1e6 settings max_rows_to_read='100K'; -- { serverError 158; }
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/%' 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 query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';