mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Add current_database = currentDatabase() filter for tests with query_log/query_thread_log
This commit is contained in:
parent
9c5d83d6bc
commit
0a1fd29ee6
@ -48,7 +48,7 @@ SELECT
|
|||||||
threads_realtime >= threads_time_user_system_io,
|
threads_realtime >= threads_time_user_system_io,
|
||||||
any(length(thread_ids)) >= 1
|
any(length(thread_ids)) >= 1
|
||||||
FROM
|
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 event_date >= today()-1 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1)
|
||||||
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
|
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
|
||||||
|
|
||||||
# Clean
|
# Clean
|
||||||
|
@ -28,6 +28,6 @@ $CLICKHOUSE_CLIENT $settings -q "$touching_many_parts_query" &> /dev/null
|
|||||||
|
|
||||||
$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS"
|
$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT $settings -q "SELECT pi.Values FROM system.query_log ARRAY JOIN ProfileEvents as pi WHERE query='$touching_many_parts_query' and pi.Names = 'FileOpen' ORDER BY event_time DESC LIMIT 1;"
|
$CLICKHOUSE_CLIENT $settings -q "SELECT pi.Values FROM system.query_log ARRAY JOIN ProfileEvents as pi WHERE query='$touching_many_parts_query' and current_database = currentDatabase() and pi.Names = 'FileOpen' ORDER BY event_time DESC LIMIT 1;"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS merge_tree_table;"
|
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS merge_tree_table;"
|
||||||
|
@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompresse
|
|||||||
|
|
||||||
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS"
|
$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 current_database = currentDatabase() AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"
|
||||||
|
|
||||||
|
@ -97,7 +97,7 @@ echo 7
|
|||||||
# and finally querylog
|
# and finally querylog
|
||||||
$CLICKHOUSE_CLIENT \
|
$CLICKHOUSE_CLIENT \
|
||||||
--server_logs_file=/dev/null \
|
--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 current_database = currentDatabase() AND event_time > now() - 10 and query like '%TOPSECRET%';"
|
||||||
|
|
||||||
|
|
||||||
rm -f "$tmp_file" >/dev/null 2>&1
|
rm -f "$tmp_file" >/dev/null 2>&1
|
||||||
|
@ -7,7 +7,7 @@ SET log_queries = 0;
|
|||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) AS symbol
|
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) 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%';
|
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND 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_real_time_period_ns = 0;
|
||||||
SET query_profiler_cpu_time_period_ns = 1000000;
|
SET query_profiler_cpu_time_period_ns = 1000000;
|
||||||
@ -17,4 +17,4 @@ SET log_queries = 0;
|
|||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) AS symbol
|
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) 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%';
|
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%test cpu time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Source%';
|
||||||
|
@ -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();
|
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;
|
SELECT * FROM test_table_for_01070_exception_code_in_query_log_table;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT exception_code FROM system.query_log WHERE lower(query) LIKE lower('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 current_database = currentDatabase() AND lower(query) LIKE lower('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;
|
||||||
DROP TABLE IF EXISTS test_table_for_01070_exception_code_in_query_log_table;
|
DROP TABLE IF EXISTS test_table_for_01070_exception_code_in_query_log_table;
|
||||||
|
@ -5,6 +5,6 @@ SYSTEM FLUSH LOGS;
|
|||||||
SELECT arrayJoin AS kv_key
|
SELECT arrayJoin AS kv_key
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
ARRAY JOIN ProfileEvents.Names AS arrayJoin
|
ARRAY JOIN ProfileEvents.Names AS arrayJoin
|
||||||
PREWHERE has(arrayMap(key -> key, ProfileEvents.Names), 'Query')
|
PREWHERE current_database = currentDatabase() AND has(arrayMap(key -> key, ProfileEvents.Names), 'Query')
|
||||||
WHERE arrayJoin = 'Query'
|
WHERE arrayJoin = 'Query'
|
||||||
LIMIT 0;
|
LIMIT 0;
|
||||||
|
@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
QUERY_ID=$RANDOM
|
QUERY_ID=$RANDOM
|
||||||
$CLICKHOUSE_BENCHMARK <<< "SELECT 1" --query_id $QUERY_ID -i 10 2>/dev/null
|
$CLICKHOUSE_BENCHMARK <<< "SELECT 1" --query_id $QUERY_ID -i 10 2>/dev/null
|
||||||
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
|
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.query_log WHERE query_id='$QUERY_ID'"
|
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$QUERY_ID'"
|
||||||
|
@ -8,7 +8,7 @@ WITH
|
|||||||
(
|
(
|
||||||
SELECT query_id
|
SELECT query_id
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (normalizeQuery(query) like normalizeQuery('WITH 01091 AS id SELECT 1;')) AND (event_date >= (today() - 1))
|
WHERE current_database = currentDatabase() AND (normalizeQuery(query) like normalizeQuery('WITH 01091 AS id SELECT 1;')) AND (event_date >= (today() - 1))
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS id
|
) AS id
|
||||||
@ -23,7 +23,7 @@ WITH
|
|||||||
(
|
(
|
||||||
SELECT query_id
|
SELECT query_id
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers(1000000);')) AND (event_date >= (today() - 1))
|
WHERE current_database = currentDatabase() AND (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers(1000000);')) AND (event_date >= (today() - 1))
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS id
|
) AS id
|
||||||
@ -38,7 +38,7 @@ WITH
|
|||||||
(
|
(
|
||||||
SELECT query_id
|
SELECT query_id
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers_mt(1000000);')) AND (event_date >= (today() - 1))
|
WHERE current_database = currentDatabase() AND (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers_mt(1000000);')) AND (event_date >= (today() - 1))
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS id
|
) AS id
|
||||||
|
@ -3,4 +3,4 @@ SET allow_introspection_functions = 1;
|
|||||||
SET memory_profiler_step = 1000000;
|
SET memory_profiler_step = 1000000;
|
||||||
SELECT ignore(groupArray(number), 'test memory profiler') FROM numbers(10000000);
|
SELECT ignore(groupArray(number), 'test memory profiler') FROM numbers(10000000);
|
||||||
SYSTEM FLUSH LOGS;
|
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 event_date >= yesterday() AND trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);
|
||||||
|
@ -46,7 +46,7 @@ for i in {1..5}; do
|
|||||||
done
|
done
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
|
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE query_id LIKE '$db-%' AND type=2"
|
$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i"
|
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i"
|
||||||
|
|
||||||
|
@ -14,4 +14,4 @@ ${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,4" > /dev/null
|
|||||||
|
|
||||||
${CLICKHOUSE_CURL} -sS "$url" --data "SYSTEM FLUSH LOGS"
|
${CLICKHOUSE_CURL} -sS "$url" --data "SYSTEM FLUSH LOGS"
|
||||||
|
|
||||||
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+count(DISTINCT+query_id)+FROM+system.query_log+WHERE+query+LIKE+'SELECT+''test_01194'',$rnd%25'"
|
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+count(DISTINCT+query_id)+FROM+system.query_log+WHERE+current_database+LIKE+currentDatabase()+AND+query+LIKE+'SELECT+''test_01194'',$rnd%25'"
|
||||||
|
@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
# shellcheck source=../shell_config.sh
|
# shellcheck source=../shell_config.sh
|
||||||
. "$CURDIR"/../shell_config.sh
|
. "$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 current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'"
|
||||||
|
@ -2,22 +2,23 @@ set log_queries=1;
|
|||||||
|
|
||||||
select '01231_log_queries_min_type/QUERY_START';
|
select '01231_log_queries_min_type/QUERY_START';
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select count() from system.query_log where 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 '%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;
|
||||||
|
|
||||||
set log_queries_min_type='EXCEPTION_BEFORE_START';
|
set log_queries_min_type='EXCEPTION_BEFORE_START';
|
||||||
select '01231_log_queries_min_type/EXCEPTION_BEFORE_START';
|
select '01231_log_queries_min_type/EXCEPTION_BEFORE_START';
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select count() from system.query_log where 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 '%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;
|
||||||
|
|
||||||
set max_rows_to_read='100K';
|
set max_rows_to_read='100K';
|
||||||
set log_queries_min_type='EXCEPTION_WHILE_PROCESSING';
|
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; }
|
select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select count() from system.query_log where 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 '%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 '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
|
select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select count() from system.query_log where
|
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 like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and
|
||||||
query not like '%system.query_log%' and
|
query not like '%system.query_log%' and
|
||||||
event_date = today() and
|
event_date = today() and
|
||||||
|
@ -16,7 +16,7 @@ SELECT
|
|||||||
throwIf(count() != 1, 'no query was logged'),
|
throwIf(count() != 1, 'no query was logged'),
|
||||||
throwIf(length(thread_ids) != 2, 'too many threads used')
|
throwIf(length(thread_ids) != 2, 'too many threads used')
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%'
|
WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%'
|
||||||
GROUP BY thread_ids
|
GROUP BY thread_ids
|
||||||
FORMAT Null;
|
FORMAT Null;
|
||||||
|
|
||||||
|
@ -17,6 +17,7 @@ SYSTEM FLUSH LOGS;
|
|||||||
SELECT DISTINCT query_duration_ms >= 500
|
SELECT DISTINCT query_duration_ms >= 500
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE
|
WHERE
|
||||||
|
current_database = currentDatabase() AND
|
||||||
event_date >= yesterday() AND
|
event_date >= yesterday() AND
|
||||||
query LIKE '%special query for 01290_max_execution_speed_distributed%' AND
|
query LIKE '%special query for 01290_max_execution_speed_distributed%' AND
|
||||||
query NOT LIKE '%system.query_log%' AND
|
query NOT LIKE '%system.query_log%' AND
|
||||||
|
@ -9,11 +9,11 @@ set log_queries = 1;
|
|||||||
select x from table_01323_many_parts limit 10 format Null;
|
select x from table_01323_many_parts limit 10 format Null;
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select arrayUniq(thread_ids) <= 4 from system.query_log where event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select arrayUniq(thread_ids) <= 4 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
|
||||||
select x from table_01323_many_parts order by x limit 10 format Null;
|
select x from table_01323_many_parts order by x limit 10 format Null;
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select arrayUniq(thread_ids) <= 36 from system.query_log where event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select arrayUniq(thread_ids) <= 36 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
|
||||||
drop table if exists table_01323_many_parts;
|
drop table if exists table_01323_many_parts;
|
||||||
|
@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
|
|||||||
SELECT * FROM test_01343;
|
SELECT * FROM test_01343;
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' 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 event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
DROP TABLE test_01343;
|
DROP TABLE test_01343;
|
||||||
|
@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
|
|||||||
SELECT * FROM test_01344 WHERE x = 'Hello, world';
|
SELECT * FROM test_01344 WHERE x = 'Hello, world';
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE 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 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;
|
||||||
|
|
||||||
DROP TABLE test_01344;
|
DROP TABLE test_01344;
|
||||||
|
@ -7,6 +7,6 @@ set max_threads = 16;
|
|||||||
select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg;
|
select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg;
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and lower(query) like '%select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and lower(query) like '%select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
|
||||||
drop table if exists table_01356_view_threads;
|
drop table if exists table_01356_view_threads;
|
||||||
|
@ -2,4 +2,4 @@ set log_queries = 1;
|
|||||||
select count() > 0 from system.settings;
|
select count() > 0 from system.settings;
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select result_rows, result_bytes >= 8 from system.query_log where event_date >= today() - 1 and lower(query) like '%select count() > 0 from system.settings%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select result_rows, result_bytes >= 8 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and lower(query) like '%select count() > 0 from system.settings%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
@ -4,4 +4,4 @@ set max_threads = 16;
|
|||||||
SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100);
|
SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100);
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and query like '%SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100)%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query like '%SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100)%' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
@ -8,6 +8,7 @@ SET log_queries=1;
|
|||||||
SELECT 1;
|
SELECT 1;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
|
-- NOTE: can be rewritten using log_queries_min_query_duration_ms
|
||||||
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
|
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
|
||||||
(
|
(
|
||||||
SELECT * FROM
|
SELECT * FROM
|
||||||
@ -37,7 +38,7 @@ SELECT
|
|||||||
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
|
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
|
||||||
count()
|
count()
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE 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 event_time > now() - 20 and QUERY_GROUP_ID<>''
|
||||||
GROUP BY QUERY_GROUP_ID
|
GROUP BY QUERY_GROUP_ID
|
||||||
ORDER BY QUERY_GROUP_ID;
|
ORDER BY QUERY_GROUP_ID;
|
||||||
|
|
||||||
@ -47,6 +48,7 @@ SELECT
|
|||||||
QUERY_GROUP_ID,
|
QUERY_GROUP_ID,
|
||||||
count()
|
count()
|
||||||
FROM slow_log
|
FROM slow_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
GROUP BY QUERY_GROUP_ID
|
GROUP BY QUERY_GROUP_ID
|
||||||
ORDER BY QUERY_GROUP_ID;
|
ORDER BY QUERY_GROUP_ID;
|
||||||
|
|
||||||
|
@ -4,25 +4,25 @@ CREATE TABLE rows_events_test (k UInt32, v UInt32) ENGINE = MergeTree ORDER BY k
|
|||||||
INSERT INTO /* test 01413, query 1 */ rows_events_test VALUES (1,1);
|
INSERT INTO /* test 01413, query 1 */ rows_events_test VALUES (1,1);
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
SELECT written_rows FROM system.query_log WHERE query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
SELECT written_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
||||||
WHERE ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
INSERT INTO /* test 01413, query 2 */ rows_events_test VALUES (2,2), (3,3);
|
INSERT INTO /* test 01413, query 2 */ rows_events_test VALUES (2,2), (3,3);
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
SELECT written_rows FROM system.query_log WHERE query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
SELECT written_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
||||||
WHERE ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
SELECT * FROM /* test 01413, query 3 */ rows_events_test WHERE v = 2;
|
SELECT * FROM /* test 01413, query 3 */ rows_events_test WHERE v = 2;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
SELECT read_rows FROM system.query_log WHERE query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
SELECT read_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
|
||||||
WHERE ProfileEvents.Names = 'SelectedRows' AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'SelectedRows' AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
DROP TABLE rows_events_test;
|
DROP TABLE rows_events_test;
|
||||||
|
@ -4,4 +4,4 @@ set max_threads = 16;
|
|||||||
select sum(number) from remote('127.0.0.{1|2}', numbers_mt(1000000)) group by number % 2 order by number % 2;
|
select sum(number) from remote('127.0.0.{1|2}', numbers_mt(1000000)) group by number % 2 order by number % 2;
|
||||||
|
|
||||||
system flush logs;
|
system flush logs;
|
||||||
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and lower(query) like '%select sum(number) from remote(_127.0.0.{1|2}_, numbers_mt(1000000)) group by number %' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() and event_date >= today() - 1 and lower(query) like '%select sum(number) from remote(_127.0.0.{1|2}_, numbers_mt(1000000)) group by number %' and type = 'QueryFinish' order by query_start_time desc limit 1;
|
||||||
|
@ -29,7 +29,9 @@ select count(*) "'"'"initial query spans with proper parent"'"'"
|
|||||||
array join attribute.names as attribute_name,
|
array join attribute.names as attribute_name,
|
||||||
attribute.values as attribute_value) o
|
attribute.values as attribute_value) o
|
||||||
join system.query_log on query_id = o.attribute_value
|
join system.query_log on query_id = o.attribute_value
|
||||||
where trace_id = reinterpretAsUUID(reverse(unhex('$trace_id')))
|
where
|
||||||
|
trace_id = reinterpretAsUUID(reverse(unhex('$trace_id')))
|
||||||
|
and current_database = currentDatabase()
|
||||||
and operation_name = 'query'
|
and operation_name = 'query'
|
||||||
and parent_span_id = reinterpretAsUInt64(unhex('73'))
|
and parent_span_id = reinterpretAsUInt64(unhex('73'))
|
||||||
and o.attribute_name = 'clickhouse.query_id'
|
and o.attribute_name = 'clickhouse.query_id'
|
||||||
@ -61,7 +63,7 @@ trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString
|
|||||||
# https://github.com/ClickHouse/ClickHouse/issues/14228
|
# https://github.com/ClickHouse/ClickHouse/issues/14228
|
||||||
${CLICKHOUSE_CURL} \
|
${CLICKHOUSE_CURL} \
|
||||||
--header "traceparent: 00-$trace_id-0000000000000073-01" \
|
--header "traceparent: 00-$trace_id-0000000000000073-01" \
|
||||||
--header "tracestate: some custom state" "http://127.0.0.2:8123/" \
|
--header "tracestate: some custom state" "$CLICKHOUSE_URL" \
|
||||||
--get \
|
--get \
|
||||||
--data-urlencode "query=select 1 from remote('127.0.0.2', system, one) format Null"
|
--data-urlencode "query=select 1 from remote('127.0.0.2', system, one) format Null"
|
||||||
|
|
||||||
|
@ -6,12 +6,14 @@ WITH (
|
|||||||
(
|
(
|
||||||
SELECT query_start_time_microseconds
|
SELECT query_start_time_microseconds
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time_with_microseconds,
|
) AS time_with_microseconds,
|
||||||
(
|
(
|
||||||
SELECT query_start_time
|
SELECT query_start_time
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS t)
|
) AS t)
|
||||||
@ -24,13 +26,15 @@ WITH (
|
|||||||
(
|
(
|
||||||
SELECT query_start_time_microseconds
|
SELECT query_start_time_microseconds
|
||||||
FROM system.query_thread_log
|
FROM system.query_thread_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time_with_microseconds,
|
) AS time_with_microseconds,
|
||||||
(
|
(
|
||||||
SELECT query_start_time
|
SELECT query_start_time
|
||||||
FROM system.query_thread_log
|
FROM system.query_thread_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS t)
|
) AS t)
|
||||||
SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); --
|
SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); --
|
||||||
|
@ -33,6 +33,7 @@ SELECT '01473_query_log_table_event_start_time_microseconds_test';
|
|||||||
WITH (
|
WITH (
|
||||||
SELECT event_time_microseconds, event_time
|
SELECT event_time_microseconds, event_time
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time
|
) AS time
|
||||||
@ -42,6 +43,7 @@ SELECT '01473_query_thread_log_table_event_start_time_microseconds_test';
|
|||||||
WITH (
|
WITH (
|
||||||
SELECT event_time_microseconds, event_time
|
SELECT event_time_microseconds, event_time
|
||||||
FROM system.query_thread_log
|
FROM system.query_thread_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time
|
) AS time
|
||||||
@ -51,6 +53,7 @@ SELECT '01473_text_log_table_event_start_time_microseconds_test';
|
|||||||
WITH (
|
WITH (
|
||||||
SELECT event_time_microseconds, event_time
|
SELECT event_time_microseconds, event_time
|
||||||
FROM system.query_thread_log
|
FROM system.query_thread_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time
|
) AS time
|
||||||
|
@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} -n -q "
|
|||||||
system flush logs;
|
system flush logs;
|
||||||
select interface, initial_query_id = query_id
|
select interface, initial_query_id = query_id
|
||||||
from system.query_log
|
from system.query_log
|
||||||
where query_id = '$query_id' and type = 'QueryFinish'
|
where current_database = currentDatabase() AND query_id = '$query_id' and type = 'QueryFinish'
|
||||||
order by interface
|
order by interface
|
||||||
;
|
;
|
||||||
"
|
"
|
||||||
|
@ -15,7 +15,7 @@ min_trace_entries=2
|
|||||||
query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$"
|
query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$"
|
||||||
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
|
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
|
||||||
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
|
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
|
||||||
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE query LIKE '%$query_id_tcp_prefix%'")"
|
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%$query_id_tcp_prefix%'")"
|
||||||
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"
|
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"
|
||||||
|
|
||||||
# HTTP
|
# HTTP
|
||||||
|
@ -2,4 +2,4 @@
|
|||||||
SELECT 1;
|
SELECT 1;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 }
|
SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 }
|
||||||
|
@ -26,4 +26,4 @@ ${CLICKHOUSE_CLIENT} -q "drop table if exists test_log_queries.logtable3" "--que
|
|||||||
${CLICKHOUSE_CLIENT} -q "drop database if exists test_log_queries" "--query_id=01600_log_queries_with_extensive_info_018"
|
${CLICKHOUSE_CLIENT} -q "drop database if exists test_log_queries" "--query_id=01600_log_queries_with_extensive_info_018"
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} -q "system flush logs"
|
${CLICKHOUSE_CLIENT} -q "system flush logs"
|
||||||
${CLICKHOUSE_CLIENT} -q "select columns(query, normalized_query_hash, query_kind, databases, tables, columns) apply (any) from system.query_log where type = 'QueryFinish' and query_id like '01600_log_queries_with_extensive_info%' group by query_id order by query_id"
|
${CLICKHOUSE_CLIENT} -q "select columns(query, normalized_query_hash, query_kind, databases, tables, columns) apply (any) from system.query_log where current_database = currentDatabase() AND type = 'QueryFinish' and query_id like '01600_log_queries_with_extensive_info%' group by query_id order by query_id"
|
||||||
|
@ -51,7 +51,7 @@ WHERE (query_id =
|
|||||||
[NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL],
|
[NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL],
|
||||||
query_id
|
query_id
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (query LIKE '%test cpu time query profiler%') AND (query NOT LIKE '%system%')
|
WHERE current_database = currentDatabase() AND (query LIKE '%test cpu time query profiler%') AND (query NOT LIKE '%system%')
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
)) AND (symbol LIKE '%Source%');
|
)) AND (symbol LIKE '%Source%');
|
||||||
@ -66,7 +66,7 @@ WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterd
|
|||||||
ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''),
|
ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''),
|
||||||
query_id
|
query_id
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (event_date >= yesterday()) AND (query LIKE '%test memory profiler%')
|
WHERE current_database = currentDatabase() AND (event_date >= yesterday()) AND (query LIKE '%test memory profiler%')
|
||||||
ORDER BY event_time DESC
|
ORDER BY event_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
)); -- { serverError 42 }
|
)); -- { serverError 42 }
|
||||||
@ -79,6 +79,7 @@ WITH (
|
|||||||
(
|
(
|
||||||
SELECT query_start_time_microseconds
|
SELECT query_start_time_microseconds
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time_with_microseconds,
|
) AS time_with_microseconds,
|
||||||
@ -87,6 +88,7 @@ WITH (
|
|||||||
inf,
|
inf,
|
||||||
query_start_time
|
query_start_time
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS t)
|
) AS t)
|
||||||
@ -96,12 +98,14 @@ WITH (
|
|||||||
(
|
(
|
||||||
SELECT query_start_time_microseconds
|
SELECT query_start_time_microseconds
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS time_with_microseconds,
|
) AS time_with_microseconds,
|
||||||
(
|
(
|
||||||
SELECT query_start_time
|
SELECT query_start_time
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
|
WHERE current_database = currentDatabase()
|
||||||
ORDER BY query_start_time DESC
|
ORDER BY query_start_time DESC
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
) AS t)
|
) AS t)
|
||||||
|
@ -13,27 +13,27 @@ SELECT '';
|
|||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT arraySort(used_aggregate_functions)
|
SELECT arraySort(used_aggregate_functions)
|
||||||
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
SELECT arraySort(used_aggregate_function_combinators)
|
SELECT arraySort(used_aggregate_function_combinators)
|
||||||
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
SELECT arraySort(used_table_functions)
|
SELECT arraySort(used_table_functions)
|
||||||
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
SELECT arraySort(used_functions)
|
SELECT arraySort(used_functions)
|
||||||
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
SELECT arraySort(used_data_type_families)
|
SELECT arraySort(used_data_type_families)
|
||||||
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ CREATE database test_query_log_factories_info1 ENGINE=Atomic;
|
|||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT used_database_engines
|
SELECT used_database_engines
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE type == 'QueryFinish' AND (query LIKE '%database test_query_log_factories_info%')
|
WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query LIKE '%database test_query_log_factories_info%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
@ -52,7 +52,7 @@ CREATE OR REPLACE TABLE test_query_log_factories_info1.memory_table (id BIGINT,
|
|||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT arraySort(used_data_type_families), used_storages
|
SELECT arraySort(used_data_type_families), used_storages
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE type == 'QueryFinish' AND (query LIKE '%TABLE test%')
|
WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query LIKE '%TABLE test%')
|
||||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||||
SELECT '';
|
SELECT '';
|
||||||
|
|
||||||
|
@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
|
|
||||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://yandex.ru/'
|
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://yandex.ru/'
|
||||||
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
|
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE http_referer LIKE '%yandex%' LIMIT 1"
|
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE current_database = currentDatabase() AND http_referer LIKE '%yandex%' LIMIT 1"
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
SET log_comment = 'log_comment test', log_queries = 1;
|
SET log_comment = 'log_comment test', log_queries = 1;
|
||||||
SELECT 1;
|
SELECT 1;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT type, query FROM system.query_log WHERE log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 1 ORDER BY event_time DESC LIMIT 1;
|
SELECT type, query FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 1 ORDER BY event_time DESC LIMIT 1;
|
||||||
SELECT type, query FROM system.query_log WHERE log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;
|
SELECT type, query FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;
|
||||||
|
Loading…
Reference in New Issue
Block a user