mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Update tests
This commit is contained in:
parent
22c5fa0974
commit
bcf495a107
@ -1,6 +1,7 @@
|
|||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeMap.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
#include <Interpreters/ProcessList.h>
|
#include <Interpreters/ProcessList.h>
|
||||||
#include <Storages/System/StorageSystemProcesses.h>
|
#include <Storages/System/StorageSystemProcesses.h>
|
||||||
@ -59,10 +60,8 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
|
|||||||
{"query", std::make_shared<DataTypeString>()},
|
{"query", std::make_shared<DataTypeString>()},
|
||||||
|
|
||||||
{"thread_ids", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
|
{"thread_ids", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
|
||||||
{"ProfileEvents.Names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
{"ProfileEvents", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>())},
|
||||||
{"ProfileEvents.Values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
|
{"Settings", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>())},
|
||||||
{"Settings.Names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
|
||||||
{"Settings.Values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -40,16 +40,15 @@ $CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS"
|
|||||||
$CLICKHOUSE_CLIENT $settings -q "
|
$CLICKHOUSE_CLIENT $settings -q "
|
||||||
WITH
|
WITH
|
||||||
any(query_duration_ms*1000) AS duration,
|
any(query_duration_ms*1000) AS duration,
|
||||||
sumIf(PV, PN = 'RealTimeMicroseconds') AS threads_realtime,
|
sum(ProfileEvents['RealTimeMicroseconds']) AS threads_realtime,
|
||||||
sumIf(PV, PN IN ('UserTimeMicroseconds', 'SystemTimeMicroseconds', 'OSIOWaitMicroseconds', 'OSCPUWaitMicroseconds')) AS threads_time_user_system_io
|
sum(ProfileEvents['UserTimeMicroseconds'] + ProfileEvents['SystemTimeMicroseconds'] + ProfileEvents['OSIOWaitMicroseconds'] + ProfileEvents['OSCPUWaitMicroseconds']) AS threads_time_user_system_io
|
||||||
SELECT
|
SELECT
|
||||||
-- duration, threads_realtime, threads_time_user_system_io,
|
-- duration, threads_realtime, threads_time_user_system_io,
|
||||||
threads_realtime >= 0.99 * duration,
|
threads_realtime >= 0.99 * duration,
|
||||||
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 type=2 ORDER BY event_time DESC LIMIT 1)"
|
||||||
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
|
|
||||||
|
|
||||||
# Clean
|
# Clean
|
||||||
rm "$server_logs_file"
|
rm "$server_logs_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 "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 ProfileEvents['FileOpen'] FROM system.query_log WHERE query='$touching_many_parts_query' 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['Seek'], ProfileEvents['ReadCompressedBytes'], ProfileEvents['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="DROP TABLE IF EXISTS small_table"
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"
|
||||||
|
|
||||||
|
@ -2,9 +2,6 @@ SET log_queries = 1;
|
|||||||
SELECT 1 LIMIT 0;
|
SELECT 1 LIMIT 0;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
SELECT arrayJoin AS kv_key
|
SELECT * FROM system.query_log
|
||||||
FROM system.query_log
|
PREWHERE ProfileEvents['Query'] > 0
|
||||||
ARRAY JOIN ProfileEvents.Names AS arrayJoin
|
|
||||||
PREWHERE has(arrayMap(key -> key, ProfileEvents.Names), 'Query')
|
|
||||||
WHERE arrayJoin = 'Query'
|
|
||||||
LIMIT 0;
|
LIMIT 0;
|
||||||
|
@ -22,5 +22,4 @@ select count() from system.query_log where
|
|||||||
query not like '%system.query_log%' and
|
query not like '%system.query_log%' and
|
||||||
event_date = today() and
|
event_date = today() and
|
||||||
event_time >= now() - interval 1 minute and
|
event_time >= now() - interval 1 minute and
|
||||||
type = 'ExceptionWhileProcessing' and
|
type = 'ExceptionWhileProcessing' and Settings['max_rows_to_read'] != '';
|
||||||
has(Settings.Names, 'max_rows_to_read');
|
|
||||||
|
@ -24,7 +24,7 @@ function show_processes_func()
|
|||||||
# These two system metrics for the generating query above are guaranteed to be nonzero when ProcFS is mounted at /proc
|
# These two system metrics for the generating query above are guaranteed to be nonzero when ProcFS is mounted at /proc
|
||||||
$CLICKHOUSE_CLIENT -q "
|
$CLICKHOUSE_CLIENT -q "
|
||||||
SELECT count() > 0 FROM system.processes\
|
SELECT count() > 0 FROM system.processes\
|
||||||
WHERE has(ProfileEvents.Names, 'OSCPUVirtualTimeMicroseconds') AND has(ProfileEvents.Names, 'OSReadChars')\
|
WHERE ProfileEvents['OSCPUVirtualTimeMicroseconds'] > 0 AND ProfileEvents['OSReadChars'] = 0 \
|
||||||
SETTINGS max_threads = 1
|
SETTINGS max_threads = 1
|
||||||
" | grep '1' && break;
|
" | grep '1' && break;
|
||||||
done
|
done
|
||||||
|
@ -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 ProfileEvents['CreatedReadBufferMMap'] as value FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' 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 ProfileEvents['CreatedReadBufferMMap'] as value FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND type = 2 ORDER BY event_time DESC LIMIT 1;
|
||||||
|
|
||||||
DROP TABLE test_01344;
|
DROP TABLE test_01344;
|
||||||
|
@ -8,6 +8,7 @@ SET log_queries=1;
|
|||||||
SELECT 1;
|
SELECT 1;
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
|
|
||||||
|
SET allow_experimental_map_type = 1;
|
||||||
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
|
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
|
||||||
(
|
(
|
||||||
SELECT * FROM
|
SELECT * FROM
|
||||||
|
@ -6,23 +6,20 @@ 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 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['InsertedRows'] as value 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;
|
||||||
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;
|
|
||||||
|
|
||||||
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 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['InsertedRows'] as value 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;
|
||||||
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;
|
|
||||||
|
|
||||||
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 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['SelectedRows'] as value 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;
|
||||||
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;
|
|
||||||
|
|
||||||
DROP TABLE rows_events_test;
|
DROP TABLE rows_events_test;
|
||||||
|
@ -7,7 +7,7 @@ SYSTEM DROP MARK CACHE;
|
|||||||
SELECT a.size0 FROM t_arr;
|
SELECT a.size0 FROM t_arr;
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
@ -24,7 +24,7 @@ SYSTEM DROP MARK CACHE;
|
|||||||
SELECT t.u FROM t_tup;
|
SELECT t.u FROM t_tup;
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
@ -38,7 +38,7 @@ SYSTEM DROP MARK CACHE;
|
|||||||
SELECT n.null FROM t_nul;
|
SELECT n.null FROM t_nul;
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
@ -57,7 +57,7 @@ SYSTEM DROP MARK CACHE;
|
|||||||
SELECT m.values FROM t_map;
|
SELECT m.values FROM t_map;
|
||||||
|
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
|
@ -8,7 +8,7 @@ CREATE TABLE nested
|
|||||||
col2 Nested(a UInt32, n Nested(s String, b UInt32)),
|
col2 Nested(a UInt32, n Nested(s String, b UInt32)),
|
||||||
col3 Nested(n1 Nested(a UInt32, b UInt32), n2 Nested(s String, t String))
|
col3 Nested(n1 Nested(a UInt32, b UInt32), n2 Nested(s String, t String))
|
||||||
)
|
)
|
||||||
ENGINE = MergeTree
|
ENGINE = MergeTree
|
||||||
ORDER BY tuple()
|
ORDER BY tuple()
|
||||||
SETTINGS min_bytes_for_wide_part = 0;
|
SETTINGS min_bytes_for_wide_part = 0;
|
||||||
|
|
||||||
@ -33,7 +33,7 @@ SELECT col1.a FROM nested FORMAT Null;
|
|||||||
|
|
||||||
-- 4 files: (col1.size0, col1.a) x2
|
-- 4 files: (col1.size0, col1.a) x2
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
@ -43,7 +43,7 @@ SELECT col3.n2.s FROM nested FORMAT Null;
|
|||||||
|
|
||||||
-- 6 files: (col3.size0, col3.n2.size1, col3.n2.s) x2
|
-- 6 files: (col3.size0, col3.n2.size1, col3.n2.s) x2
|
||||||
SYSTEM FLUSH LOGS;
|
SYSTEM FLUSH LOGS;
|
||||||
SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
SELECT ProfileEvents['FileOpen']
|
||||||
FROM system.query_log
|
FROM system.query_log
|
||||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%'))
|
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%'))
|
||||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||||
@ -55,7 +55,7 @@ CREATE TABLE nested
|
|||||||
id UInt32,
|
id UInt32,
|
||||||
col1 Nested(a UInt32, n Nested(s String, b UInt32))
|
col1 Nested(a UInt32, n Nested(s String, b UInt32))
|
||||||
)
|
)
|
||||||
ENGINE = MergeTree
|
ENGINE = MergeTree
|
||||||
ORDER BY id
|
ORDER BY id
|
||||||
SETTINGS min_bytes_for_wide_part = 0;
|
SETTINGS min_bytes_for_wide_part = 0;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user