Add current_database = currentDatabase() filter for tests with query_log/query_thread_log

This commit is contained in:
Azat Khuzhin 2021-01-26 22:56:11 +03:00
parent 9c5d83d6bc
commit 0a1fd29ee6
36 changed files with 71 additions and 54 deletions

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 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"
# Clean

View File

@ -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 "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;"

View File

@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompresse
$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"

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 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

View File

@ -7,7 +7,7 @@ SET log_queries = 0;
SYSTEM FLUSH LOGS;
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_cpu_time_period_ns = 1000000;
@ -17,4 +17,4 @@ SET log_queries = 0;
SYSTEM FLUSH LOGS;
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%';

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 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;

View File

@ -5,6 +5,6 @@ SYSTEM FLUSH LOGS;
SELECT arrayJoin AS kv_key
FROM system.query_log
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'
LIMIT 0;

View File

@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
QUERY_ID=$RANDOM
$CLICKHOUSE_BENCHMARK <<< "SELECT 1" --query_id $QUERY_ID -i 10 2>/dev/null
$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'"

View File

@ -8,7 +8,7 @@ WITH
(
SELECT query_id
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
LIMIT 1
) AS id
@ -23,7 +23,7 @@ WITH
(
SELECT query_id
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
LIMIT 1
) AS id
@ -38,7 +38,7 @@ WITH
(
SELECT query_id
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
LIMIT 1
) AS 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 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);

View File

@ -46,7 +46,7 @@ for i in {1..5}; do
done
$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"

View File

@ -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&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'"

View File

@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../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'"

View File

@ -2,22 +2,23 @@ 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/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';
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/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 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; }
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; }
system flush logs;
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

View File

@ -16,7 +16,7 @@ SELECT
throwIf(count() != 1, 'no query was logged'),
throwIf(length(thread_ids) != 2, 'too many threads used')
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
FORMAT Null;

View File

@ -17,6 +17,7 @@ SYSTEM FLUSH LOGS;
SELECT DISTINCT query_duration_ms >= 500
FROM system.query_log
WHERE
current_database = currentDatabase() AND
event_date >= yesterday() AND
query LIKE '%special query for 01290_max_execution_speed_distributed%' AND
query NOT LIKE '%system.query_log%' AND

View File

@ -9,11 +9,11 @@ set log_queries = 1;
select x from table_01323_many_parts limit 10 format Null;
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;
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;

View File

@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
SELECT * FROM test_01343;
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;

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 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;

View File

@ -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;
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;

View File

@ -2,4 +2,4 @@ set log_queries = 1;
select count() > 0 from system.settings;
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;

View File

@ -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);
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;

View File

@ -8,6 +8,7 @@ SET log_queries=1;
SELECT 1;
SYSTEM FLUSH LOGS;
-- NOTE: can be rewritten using log_queries_min_query_duration_ms
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
(
SELECT * FROM
@ -37,7 +38,7 @@ SELECT
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
count()
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
ORDER BY QUERY_GROUP_ID;
@ -47,6 +48,7 @@ SELECT
QUERY_GROUP_ID,
count()
FROM slow_log
WHERE current_database = currentDatabase()
GROUP BY QUERY_GROUP_ID
ORDER BY QUERY_GROUP_ID;

View File

@ -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);
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
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);
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
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;
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
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;

View File

@ -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;
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;

View File

@ -29,7 +29,9 @@ select count(*) "'"'"initial query spans with proper parent"'"'"
array join attribute.names as attribute_name,
attribute.values as attribute_value) o
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 parent_span_id = reinterpretAsUInt64(unhex('73'))
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
${CLICKHOUSE_CURL} \
--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 \
--data-urlencode "query=select 1 from remote('127.0.0.2', system, one) format Null"

View File

@ -6,12 +6,14 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)
@ -24,13 +26,15 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) 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'); --

View File

@ -33,6 +33,7 @@ SELECT '01473_query_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time
@ -42,6 +43,7 @@ SELECT '01473_query_thread_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time
@ -51,6 +53,7 @@ SELECT '01473_text_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time

View File

@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} -n -q "
system flush logs;
select interface, initial_query_id = query_id
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
;
"

View File

@ -15,7 +15,7 @@ min_trace_entries=2
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} -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"
# HTTP

View File

@ -2,4 +2,4 @@
SELECT 1;
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 }

View File

@ -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 "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"

View File

@ -51,7 +51,7 @@ WHERE (query_id =
[NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL],
query_id
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
LIMIT 1
)) 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)), ''),
query_id
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
LIMIT 1
)); -- { serverError 42 }
@ -79,6 +79,7 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
@ -87,6 +88,7 @@ WITH (
inf,
query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)
@ -96,12 +98,14 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)

View File

@ -13,27 +13,27 @@ SELECT '';
SYSTEM FLUSH LOGS;
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;
SELECT '';
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;
SELECT '';
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;
SELECT '';
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;
SELECT '';
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;
SELECT '';
@ -43,7 +43,7 @@ CREATE database test_query_log_factories_info1 ENGINE=Atomic;
SYSTEM FLUSH LOGS;
SELECT used_database_engines
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;
SELECT '';
@ -52,7 +52,7 @@ CREATE OR REPLACE TABLE test_query_log_factories_info1.memory_table (id BIGINT,
SYSTEM FLUSH LOGS;
SELECT arraySort(used_data_type_families), used_storages
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;
SELECT '';

View File

@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://yandex.ru/'
${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"

View File

@ -1,5 +1,5 @@
SET log_comment = 'log_comment test', log_queries = 1;
SELECT 1;
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 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 = 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 = 2 ORDER BY event_time DESC LIMIT 1;