mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
tests: fix flakiness with system.{parts,detached_parts,columns,parts_clumns,tables}
This commit is contained in:
parent
12cc3fc4af
commit
8058166d55
@ -25,6 +25,6 @@ DROP DATABASE IF EXISTS test_DatabaseMemory;
|
|||||||
CREATE DATABASE test_DatabaseMemory ENGINE = Memory;
|
CREATE DATABASE test_DatabaseMemory ENGINE = Memory;
|
||||||
CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null;
|
CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null;
|
||||||
|
|
||||||
SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables;
|
SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = 'test_DatabaseMemory';
|
||||||
|
|
||||||
DROP DATABASE test_DatabaseMemory;
|
DROP DATABASE test_DatabaseMemory;
|
||||||
|
@ -57,7 +57,8 @@ ${CLICKHOUSE_LOCAL} -q "CREATE TABLE sophisticated_default
|
|||||||
a UInt8 DEFAULT 3,
|
a UInt8 DEFAULT 3,
|
||||||
b UInt8 ALIAS a + 5,
|
b UInt8 ALIAS a + 5,
|
||||||
c UInt8
|
c UInt8
|
||||||
) ENGINE = Memory; SELECT count() FROM system.tables WHERE name='sophisticated_default';"
|
) ENGINE = Memory;
|
||||||
|
SELECT count() FROM system.tables WHERE name='sophisticated_default' AND database = currentDatabase();"
|
||||||
|
|
||||||
# Help is not skipped
|
# Help is not skipped
|
||||||
[[ $(${CLICKHOUSE_LOCAL} --help | wc -l) -gt 100 ]]
|
[[ $(${CLICKHOUSE_LOCAL} --help | wc -l) -gt 100 ]]
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
|
|
||||||
SELECT avg(blockSize()) <= 10 FROM system.tables SETTINGS max_block_size = 10;
|
SELECT avg(blockSize()) <= 10 FROM system.tables SETTINGS max_block_size = 10;
|
||||||
SELECT avg(blockSize()) <= 10 FROM system.tables LIMIT 10 SETTINGS max_block_size = 10;
|
SELECT avg(blockSize()) <= 10 FROM system.tables LIMIT 10 SETTINGS max_block_size = 10;
|
||||||
SELECT (SELECT count() FROM system.tables SETTINGS max_block_size = 10) = (SELECT count() FROM system.tables SETTINGS max_block_size = 9);
|
SELECT (SELECT count() FROM system.tables SETTINGS max_block_size = 10) = (SELECT count() FROM system.tables SETTINGS max_block_size = 9);
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
|
|
||||||
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 0);
|
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 0);
|
||||||
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 1);
|
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 1);
|
||||||
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 2);
|
SELECT count() > 0 FROM (SELECT * FROM system.columns LIMIT 2);
|
||||||
|
@ -13,17 +13,15 @@ CREATE TABLE check_system_tables
|
|||||||
SETTINGS min_bytes_for_wide_part = 0;
|
SETTINGS min_bytes_for_wide_part = 0;
|
||||||
|
|
||||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows
|
SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows
|
||||||
FROM system.tables
|
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE name = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||||
FROM system.columns
|
FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE table = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
INSERT INTO check_system_tables VALUES (1, 1, 1);
|
INSERT INTO check_system_tables VALUES (1, 1, 1);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
|
|
||||||
DROP TABLE IF EXISTS check_system_tables;
|
DROP TABLE IF EXISTS check_system_tables;
|
||||||
|
|
||||||
@ -39,13 +37,11 @@ CREATE TABLE check_system_tables
|
|||||||
ORDER BY date;
|
ORDER BY date;
|
||||||
|
|
||||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||||
FROM system.tables
|
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE name = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||||
FROM system.columns
|
FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE table = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS check_system_tables;
|
DROP TABLE IF EXISTS check_system_tables;
|
||||||
@ -59,29 +55,27 @@ CREATE TABLE check_system_tables
|
|||||||
) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192);
|
) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192);
|
||||||
|
|
||||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||||
FROM system.tables
|
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE name = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||||
FROM system.columns
|
FROM system.columns WHERE table = 'check_system_tables' AND database = currentDatabase()
|
||||||
WHERE table = 'check_system_tables'
|
|
||||||
FORMAT PrettyCompactNoEscapes;
|
FORMAT PrettyCompactNoEscapes;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS check_system_tables;
|
DROP TABLE IF EXISTS check_system_tables;
|
||||||
|
|
||||||
SELECT 'Check total_bytes/total_rows for TinyLog';
|
SELECT 'Check total_bytes/total_rows for TinyLog';
|
||||||
CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog();
|
CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog();
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables VALUES (1);
|
INSERT INTO check_system_tables VALUES (1);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
DROP TABLE check_system_tables;
|
DROP TABLE check_system_tables;
|
||||||
|
|
||||||
SELECT 'Check total_bytes/total_rows for Memory';
|
SELECT 'Check total_bytes/total_rows for Memory';
|
||||||
CREATE TABLE check_system_tables (key UInt16) ENGINE = Memory();
|
CREATE TABLE check_system_tables (key UInt16) ENGINE = Memory();
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables VALUES (1);
|
INSERT INTO check_system_tables VALUES (1);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
DROP TABLE check_system_tables;
|
DROP TABLE check_system_tables;
|
||||||
|
|
||||||
SELECT 'Check total_bytes/total_rows for Buffer';
|
SELECT 'Check total_bytes/total_rows for Buffer';
|
||||||
@ -96,33 +90,33 @@ CREATE TABLE check_system_tables (key UInt16) ENGINE = Buffer(
|
|||||||
100, 100, /* min_rows /max_rows */
|
100, 100, /* min_rows /max_rows */
|
||||||
0, 1e6 /* min_bytes/max_bytes */
|
0, 1e6 /* min_bytes/max_bytes */
|
||||||
);
|
);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables SELECT * FROM numbers_mt(50);
|
INSERT INTO check_system_tables SELECT * FROM numbers_mt(50);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
|
|
||||||
SELECT 'Check lifetime_bytes/lifetime_rows for Buffer';
|
SELECT 'Check lifetime_bytes/lifetime_rows for Buffer';
|
||||||
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
OPTIMIZE TABLE check_system_tables; -- flush
|
OPTIMIZE TABLE check_system_tables; -- flush
|
||||||
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables SELECT * FROM numbers_mt(50);
|
INSERT INTO check_system_tables SELECT * FROM numbers_mt(50);
|
||||||
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
OPTIMIZE TABLE check_system_tables; -- flush
|
OPTIMIZE TABLE check_system_tables; -- flush
|
||||||
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables SELECT * FROM numbers_mt(101); -- direct block write (due to min_rows exceeded)
|
INSERT INTO check_system_tables SELECT * FROM numbers_mt(101); -- direct block write (due to min_rows exceeded)
|
||||||
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT lifetime_bytes, lifetime_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
DROP TABLE check_system_tables;
|
DROP TABLE check_system_tables;
|
||||||
DROP TABLE check_system_tables_null;
|
DROP TABLE check_system_tables_null;
|
||||||
|
|
||||||
SELECT 'Check total_bytes/total_rows for Set';
|
SELECT 'Check total_bytes/total_rows for Set';
|
||||||
CREATE TABLE check_system_tables Engine=Set() AS SELECT * FROM numbers(50);
|
CREATE TABLE check_system_tables Engine=Set() AS SELECT * FROM numbers(50);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables SELECT number+50 FROM numbers(50);
|
INSERT INTO check_system_tables SELECT number+50 FROM numbers(50);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
DROP TABLE check_system_tables;
|
DROP TABLE check_system_tables;
|
||||||
|
|
||||||
SELECT 'Check total_bytes/total_rows for Join';
|
SELECT 'Check total_bytes/total_rows for Join';
|
||||||
CREATE TABLE check_system_tables Engine=Join(ANY, LEFT, number) AS SELECT * FROM numbers(50);
|
CREATE TABLE check_system_tables Engine=Join(ANY, LEFT, number) AS SELECT * FROM numbers(50);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
INSERT INTO check_system_tables SELECT number+50 FROM numbers(50);
|
INSERT INTO check_system_tables SELECT number+50 FROM numbers(50);
|
||||||
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
|
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
|
||||||
DROP TABLE check_system_tables;
|
DROP TABLE check_system_tables;
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
|
|
||||||
DROP TABLE IF EXISTS table1;
|
DROP TABLE IF EXISTS table1;
|
||||||
DROP TABLE IF EXISTS table2;
|
DROP TABLE IF EXISTS table2;
|
||||||
|
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#!/usr/bin/env bash
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
|
|
||||||
set -e
|
set -e
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
@ -25,8 +25,8 @@ ALTER TABLE alter_compression_codec1 ADD COLUMN alter_column String DEFAULT 'def
|
|||||||
SYSTEM SYNC REPLICA alter_compression_codec1;
|
SYSTEM SYNC REPLICA alter_compression_codec1;
|
||||||
SYSTEM SYNC REPLICA alter_compression_codec2;
|
SYSTEM SYNC REPLICA alter_compression_codec2;
|
||||||
|
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
|
|
||||||
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 3, '3');
|
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 3, '3');
|
||||||
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 4, '4');
|
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 4, '4');
|
||||||
@ -37,8 +37,8 @@ SELECT * FROM alter_compression_codec1 ORDER BY id;
|
|||||||
SELECT * FROM alter_compression_codec2 ORDER BY id;
|
SELECT * FROM alter_compression_codec2 ORDER BY id;
|
||||||
|
|
||||||
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE);
|
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE);
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
|
|
||||||
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 5, '5');
|
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 5, '5');
|
||||||
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 6, '6');
|
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 6, '6');
|
||||||
@ -50,8 +50,8 @@ SET allow_suspicious_codecs = 1;
|
|||||||
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
|
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
|
||||||
SYSTEM SYNC REPLICA alter_compression_codec1;
|
SYSTEM SYNC REPLICA alter_compression_codec1;
|
||||||
SYSTEM SYNC REPLICA alter_compression_codec2;
|
SYSTEM SYNC REPLICA alter_compression_codec2;
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
|
|
||||||
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 7, '7');
|
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 7, '7');
|
||||||
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 8, '8');
|
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 8, '8');
|
||||||
@ -62,8 +62,8 @@ SELECT * FROM alter_compression_codec2 ORDER BY id;
|
|||||||
|
|
||||||
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100);
|
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100);
|
||||||
SYSTEM SYNC REPLICA alter_compression_codec2;
|
SYSTEM SYNC REPLICA alter_compression_codec2;
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
|
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column' AND database = currentDatabase();
|
||||||
|
|
||||||
DROP TABLE IF EXISTS alter_compression_codec1;
|
DROP TABLE IF EXISTS alter_compression_codec1;
|
||||||
DROP TABLE IF EXISTS alter_compression_codec2;
|
DROP TABLE IF EXISTS alter_compression_codec2;
|
||||||
|
@ -8,6 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
|
|
||||||
set -e
|
set -e
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
for _ in {1..100}; do \
|
for _ in {1..100}; do \
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables UNION ALL SELECT name FROM system.columns format Null";
|
$CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables UNION ALL SELECT name FROM system.columns format Null";
|
||||||
done
|
done
|
||||||
|
@ -13,6 +13,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d
|
|||||||
|
|
||||||
function thread1()
|
function thread1()
|
||||||
{
|
{
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
while true; do $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null"; done
|
while true; do $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null"; done
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -15,6 +15,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d
|
|||||||
|
|
||||||
function thread1()
|
function thread1()
|
||||||
{
|
{
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done
|
while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -16,6 +16,7 @@ $CLICKHOUSE_CLIENT -n -q "
|
|||||||
|
|
||||||
function thread1()
|
function thread1()
|
||||||
{
|
{
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done
|
while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -8,7 +8,8 @@ set -e
|
|||||||
|
|
||||||
function thread1()
|
function thread1()
|
||||||
{
|
{
|
||||||
while true; do
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
|
while true; do
|
||||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null";
|
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null";
|
||||||
done
|
done
|
||||||
}
|
}
|
||||||
|
@ -28,6 +28,7 @@ function thread2()
|
|||||||
function thread3()
|
function thread3()
|
||||||
{
|
{
|
||||||
while true; do
|
while true; do
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null
|
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null
|
||||||
done
|
done
|
||||||
}
|
}
|
||||||
|
@ -15,14 +15,16 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE b (x UInt8) ENGINE = MergeTree ORDER BY
|
|||||||
|
|
||||||
function thread1()
|
function thread1()
|
||||||
{
|
{
|
||||||
while true; do
|
while true; do
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n
|
seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n
|
||||||
done
|
done
|
||||||
}
|
}
|
||||||
|
|
||||||
function thread2()
|
function thread2()
|
||||||
{
|
{
|
||||||
while true; do
|
while true; do
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n
|
seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n
|
||||||
done
|
done
|
||||||
}
|
}
|
||||||
|
@ -82,14 +82,14 @@ wait
|
|||||||
|
|
||||||
echo "Finishing alters"
|
echo "Finishing alters"
|
||||||
|
|
||||||
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null)
|
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null)
|
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null)
|
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
|
|
||||||
while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do
|
while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do
|
||||||
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null)
|
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null)
|
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null)
|
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3' and database='$CLICKHOUSE_DATABASE'" 2> /dev/null)
|
||||||
|
|
||||||
sleep 1
|
sleep 1
|
||||||
done
|
done
|
||||||
|
@ -10,6 +10,7 @@ url="${url_without_session}session_id=test_01098"
|
|||||||
${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE IF EXISTS tmp_table"
|
${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE IF EXISTS tmp_table"
|
||||||
${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "CREATE TEMPORARY TABLE tmp_table AS SELECT number AS n FROM numbers(42)"
|
${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "CREATE TEMPORARY TABLE tmp_table AS SELECT number AS n FROM numbers(42)"
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
id=$(echo "SELECT uuid FROM system.tables WHERE name='tmp_table' AND is_temporary" | ${CLICKHOUSE_CURL} -m 31 -sSgk "$url" -d @-)
|
id=$(echo "SELECT uuid FROM system.tables WHERE name='tmp_table' AND is_temporary" | ${CLICKHOUSE_CURL} -m 31 -sSgk "$url" -d @-)
|
||||||
internal_table_name="_temporary_and_external_tables.\`_tmp_$id\`"
|
internal_table_name="_temporary_and_external_tables.\`_tmp_$id\`"
|
||||||
|
|
||||||
|
@ -4,6 +4,7 @@ CREATE DATABASE test_01109 ENGINE=Atomic;
|
|||||||
USE test_01109;
|
USE test_01109;
|
||||||
|
|
||||||
CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables'])));
|
CREATE TABLE t0 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks(), * FROM (SELECT toLowCardinality(arrayJoin(['exchange', 'tables'])));
|
||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name;
|
CREATE TABLE t1 ENGINE=Log() AS SELECT * FROM system.tables AS t JOIN system.databases AS d ON t.database=d.name;
|
||||||
CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world']));
|
CREATE TABLE t2 ENGINE=MergeTree() ORDER BY tuple() AS SELECT rowNumberInAllBlocks() + (SELECT count() FROM t0), * FROM (SELECT arrayJoin(['hello', 'world']));
|
||||||
|
|
||||||
|
@ -32,11 +32,11 @@ $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_rename_replicated;"
|
|||||||
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_rename_replicated RENAME COLUMN value1 to renamed_value1" --replication_alter_partitions_sync=0
|
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_rename_replicated RENAME COLUMN value1 to renamed_value1" --replication_alter_partitions_sync=0
|
||||||
|
|
||||||
|
|
||||||
while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'" 2>/dev/null) ]]; do
|
while [[ -z $($CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'" 2>/dev/null) ]]; do
|
||||||
sleep 0.5
|
sleep 0.5
|
||||||
done
|
done
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated'"
|
$CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns WHERE name = 'renamed_value1' and table = 'table_for_rename_replicated' AND database = '$CLICKHOUSE_DATABASE'"
|
||||||
|
|
||||||
# SHOW CREATE TABLE takes query from .sql file on disk.
|
# SHOW CREATE TABLE takes query from .sql file on disk.
|
||||||
# previous select take metadata from memory. So, when previous select says, that return renamed_value1 already exists in table, it's still can have old version on disk.
|
# previous select take metadata from memory. So, when previous select says, that return renamed_value1 already exists in table, it's still can have old version on disk.
|
||||||
|
@ -6,6 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
# shellcheck source=../shell_config.sh
|
# shellcheck source=../shell_config.sh
|
||||||
. "$CUR_DIR"/../shell_config.sh
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10);" 2>&1 | grep -F "Code: 57" > /dev/null && echo 'OK' || echo 'FAIL'
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10);" 2>&1 | grep -F "Code: 57" > /dev/null && echo 'OK' || echo 'FAIL'
|
||||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns engine=Memory AS numbers(10);" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL'
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns engine=Memory AS numbers(10);" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL'
|
||||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10) engine=Memory;" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL'
|
${CLICKHOUSE_CLIENT} --query "CREATE TABLE system.columns AS numbers(10) engine=Memory;" 2>&1 | grep -F "Code: 62" > /dev/null && echo 'OK' || echo 'FAIL'
|
||||||
|
@ -15,13 +15,13 @@ ALTER TABLE mt_01451 DETACH PART 'all_2_2_0';
|
|||||||
|
|
||||||
SELECT v FROM mt_01451 ORDER BY v;
|
SELECT v FROM mt_01451 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.detached_parts WHERE table = 'mt_01451';
|
SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase();
|
||||||
|
|
||||||
ALTER TABLE mt_01451 ATTACH PART 'all_2_2_0';
|
ALTER TABLE mt_01451 ATTACH PART 'all_2_2_0';
|
||||||
|
|
||||||
SELECT v FROM mt_01451 ORDER BY v;
|
SELECT v FROM mt_01451 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.detached_parts WHERE table = 'mt_01451';
|
SELECT name FROM system.detached_parts WHERE table = 'mt_01451' AND database = currentDatabase();
|
||||||
|
|
||||||
SELECT '-- drop part --';
|
SELECT '-- drop part --';
|
||||||
|
|
||||||
@ -37,6 +37,6 @@ OPTIMIZE TABLE mt_01451 FINAL;
|
|||||||
|
|
||||||
SELECT v FROM mt_01451 ORDER BY v;
|
SELECT v FROM mt_01451 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.parts WHERE table = 'mt_01451' AND active;
|
SELECT name FROM system.parts WHERE table = 'mt_01451' AND active AND database = currentDatabase();
|
||||||
|
|
||||||
DROP TABLE mt_01451;
|
DROP TABLE mt_01451;
|
||||||
|
@ -19,14 +19,14 @@ ALTER TABLE replica2 DETACH PART 'all_1_1_0';
|
|||||||
|
|
||||||
SELECT v FROM replica1 ORDER BY v;
|
SELECT v FROM replica1 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.detached_parts WHERE table = 'replica2';
|
SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase();
|
||||||
|
|
||||||
ALTER TABLE replica2 ATTACH PART 'all_1_1_0';
|
ALTER TABLE replica2 ATTACH PART 'all_1_1_0';
|
||||||
|
|
||||||
SYSTEM SYNC REPLICA replica1;
|
SYSTEM SYNC REPLICA replica1;
|
||||||
SELECT v FROM replica1 ORDER BY v;
|
SELECT v FROM replica1 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.detached_parts WHERE table = 'replica2';
|
SELECT name FROM system.detached_parts WHERE table = 'replica2' AND database = currentDatabase();
|
||||||
|
|
||||||
SELECT '-- drop part --';
|
SELECT '-- drop part --';
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ OPTIMIZE TABLE replica1 FINAL;
|
|||||||
|
|
||||||
SELECT v FROM replica1 ORDER BY v;
|
SELECT v FROM replica1 ORDER BY v;
|
||||||
|
|
||||||
SELECT name FROM system.parts WHERE table = 'replica2' AND active;
|
SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase();
|
||||||
|
|
||||||
DROP TABLE replica1;
|
DROP TABLE replica1;
|
||||||
DROP TABLE replica2;
|
DROP TABLE replica2;
|
||||||
|
@ -1,7 +1,8 @@
|
|||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
|
|
||||||
SELECT database FROM system.tables WHERE database LIKE '%' format Null;
|
SELECT database FROM system.tables WHERE database LIKE '%' format Null;
|
||||||
SELECT database AS db FROM system.tables WHERE db LIKE '%' format Null;
|
SELECT database AS db FROM system.tables WHERE db LIKE '%' format Null;
|
||||||
SELECT CAST(database, 'String') AS db FROM system.tables WHERE db LIKE '%' format Null;
|
SELECT CAST(database, 'String') AS db FROM system.tables WHERE db LIKE '%' format Null;
|
||||||
SELECT CAST('a string', 'Nullable(String)') AS str WHERE str LIKE '%' format Null;
|
SELECT CAST('a string', 'Nullable(String)') AS str WHERE str LIKE '%' format Null;
|
||||||
SELECT CAST(database, 'Nullable(String)') AS ndb FROM system.tables WHERE ndb LIKE '%' format Null;
|
SELECT CAST(database, 'Nullable(String)') AS ndb FROM system.tables WHERE ndb LIKE '%' format Null;
|
||||||
SELECT 'all tests passed';
|
SELECT 'all tests passed';
|
||||||
|
|
||||||
|
@ -14,7 +14,7 @@ INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(
|
|||||||
|
|
||||||
OPTIMIZE TABLE optimize_final FINAL;
|
OPTIMIZE TABLE optimize_final FINAL;
|
||||||
|
|
||||||
SELECT table, partition, active, level from system.parts where table = 'optimize_final' and active = 1;
|
SELECT table, partition, active, level from system.parts where table = 'optimize_final' and database = currentDatabase() and active = 1;
|
||||||
|
|
||||||
DROP TABLE optimize_final;
|
DROP TABLE optimize_final;
|
||||||
|
|
||||||
|
@ -38,5 +38,5 @@ DROP TABLE IF EXISTS empty;
|
|||||||
CREATE TABLE empty (key UInt32, val UInt32, date Datetime) ENGINE=SummingMergeTree(val) PARTITION BY date ORDER BY key;
|
CREATE TABLE empty (key UInt32, val UInt32, date Datetime) ENGINE=SummingMergeTree(val) PARTITION BY date ORDER BY key;
|
||||||
INSERT INTO empty VALUES (1, 1, '2020-01-01'), (1, 1, '2020-01-01'), (1, -2, '2020-01-01');
|
INSERT INTO empty VALUES (1, 1, '2020-01-01'), (1, 1, '2020-01-01'), (1, -2, '2020-01-01');
|
||||||
SELECT * FROM empty ORDER BY key;
|
SELECT * FROM empty ORDER BY key;
|
||||||
SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1;
|
SELECT table, partition, active FROM system.parts where table = 'empty' and active = 1 and database = currentDatabase();
|
||||||
DROP TABLE empty;
|
DROP TABLE empty;
|
||||||
|
@ -4,6 +4,7 @@ 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
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
verify_sql="SELECT
|
verify_sql="SELECT
|
||||||
(SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics)
|
(SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics)
|
||||||
= (SELECT sum(active), sum(NOT active) FROM system.parts)"
|
= (SELECT sum(active), sum(NOT active) FROM system.parts)"
|
||||||
|
@ -7,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
set -e
|
set -e
|
||||||
set -o pipefail
|
set -o pipefail
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||||
verify_sql="SELECT
|
verify_sql="SELECT
|
||||||
(SELECT sumIf(value, metric = 'PartsInMemory'), sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) =
|
(SELECT sumIf(value, metric = 'PartsInMemory'), sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) =
|
||||||
(SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide') FROM system.parts)"
|
(SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide') FROM system.parts)"
|
||||||
|
@ -4,6 +4,8 @@ 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
|
||||||
|
|
||||||
|
# NOTE: database = $CLICKHOUSE_DATABASE is superfluous
|
||||||
|
|
||||||
function test_completion_word()
|
function test_completion_word()
|
||||||
{
|
{
|
||||||
local w=$1 && shift
|
local w=$1 && shift
|
||||||
|
@ -28,7 +28,7 @@ SELECT
|
|||||||
name,
|
name,
|
||||||
comment
|
comment
|
||||||
FROM system.tables
|
FROM system.tables
|
||||||
WHERE name IN ('t1', 't2', 't3') order by name;
|
WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name;
|
||||||
|
|
||||||
SHOW CREATE TABLE t1;
|
SHOW CREATE TABLE t1;
|
||||||
|
|
||||||
|
@ -1 +1,2 @@
|
|||||||
|
-- NOTE: database = currentDatabase() is not mandatory
|
||||||
SELECT sum(data_compressed_bytes) > 0, sum(data_uncompressed_bytes) > 0, sum(marks_bytes) > 0 FROM system.columns;
|
SELECT sum(data_compressed_bytes) > 0, sum(data_uncompressed_bytes) > 0, sum(marks_bytes) > 0 FROM system.columns;
|
||||||
|
Loading…
Reference in New Issue
Block a user