提交 a7957914 编写于 作者: I Ivan Lezhankin

Check for orphaned databases too

上级 398e1edd
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.big_array;
CREATE TABLE test.big_array (x Array(UInt8)) ENGINE=TinyLog;
DROP TABLE IF EXISTS big_array;
CREATE TABLE big_array (x Array(UInt8)) ENGINE=TinyLog;
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test.big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
SELECT sum(y) AS s FROM remote('127.0.0.{2,3}', test, big_array) ARRAY JOIN x AS y;
INSERT INTO big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
SELECT sum(y) AS s FROM remote('127.0.0.{2,3}', currentDatabase(), big_array) ARRAY JOIN x AS y;
SELECT sum(s) FROM (SELECT y AS s FROM remote('127.0.0.{2,3}', currentDatabase(), big_array) ARRAY JOIN x AS y);
DROP TABLE big_array;
SELECT sum(s) FROM (SELECT y AS s FROM remote('127.0.0.{2,3}', test, big_array) ARRAY JOIN x AS y);
DROP TABLE test.big_array;
DROP TABLE IF EXISTS test.storage;
CREATE TABLE test.storage(UserID UInt64) ENGINE=Memory;
INSERT INTO test.storage(UserID) values (6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(402895971392036118)(402895971392036118)(402895971392036118);
DROP TABLE IF EXISTS storage;
CREATE TABLE storage(UserID UInt64) ENGINE=Memory;
INSERT INTO storage(UserID) values (6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(402895971392036118)(402895971392036118)(402895971392036118);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{2,3}', test.storage))) FROM remote('127.0.0.{2,3}', test.storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.storage)) FROM remote('127.0.0.{2,3}', test.storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{2,3}', currentDatabase(), storage))) FROM remote('127.0.0.{2,3}', currentDatabase(), storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM storage)) FROM remote('127.0.0.{2,3}', currentDatabase(), storage);
DROP TABLE storage;
drop table if exists test.tab;
create table test.tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192);
desc test.tab;
drop table if exists tab;
create table tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192);
desc tab;
select '-';
desc table test.tab;
desc table tab;
select '-';
desc remote('127.0.0.2', test.tab);
desc remote('127.0.0.2', currentDatabase(), tab);
select '-';
desc table remote('127.0.0.2', test.tab);
desc table remote('127.0.0.2', currentDatabase(), tab);
select '-';
desc (select 1);
select '-';
......@@ -14,4 +14,4 @@ desc table (select 1);
select '-';
desc (select * from system.numbers);
select '-';
drop table if exists test.tab;
drop table if exists tab;
......@@ -2,28 +2,30 @@ DROP TABLE IF EXISTS tab_00625;
CREATE TABLE tab_00625
(
date Date,
key UInt32,
date Date,
key UInt32,
testMap Nested(
k UInt16,
k UInt16,
v UInt64)
)
ENGINE = SummingMergeTree(date, (date, key), 1);
INSERT INTO tab_00625 SELECT
today(),
number,
[toUInt16(number)],
INSERT INTO tab_00625 SELECT
today(),
number,
[toUInt16(number)],
[number]
FROM system.numbers
FROM system.numbers
LIMIT 8190;
INSERT INTO tab_00625 SELECT
today(),
number + 8190,
[toUInt16(number)],
INSERT INTO tab_00625 SELECT
today(),
number + 8190,
[toUInt16(number)],
[number + 8190]
FROM system.numbers
FROM system.numbers
LIMIT 10;
OPTIMIZE TABLE tab_00625;
DROP TABLE tab_00625;
----- Group of very similar simple tests ------
DROP TABLE IF EXISTS test.zero_rows_per_granule;
DROP TABLE IF EXISTS zero_rows_per_granule;
CREATE TABLE test.zero_rows_per_granule (
CREATE TABLE zero_rows_per_granule (
p Date,
k UInt64,
v1 UInt64,
......@@ -14,26 +14,26 @@ CREATE TABLE test.zero_rows_per_granule (
vertical_merge_algorithm_min_rows_to_activate=0,
vertical_merge_algorithm_min_columns_to_activate=0;
INSERT INTO test.zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
SELECT COUNT(*) FROM test.zero_rows_per_granule;
SELECT COUNT(*) FROM zero_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database=currentDatabase() and active=1;
INSERT INTO test.zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, 1, 1), ('2018-05-16', 6, 3000, 4000, 1, 1), ('2018-05-17', 7, 5000, 6000, 1, 1), ('2018-05-19', 8, 7000, 8000, 1, 1);
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, 1, 1), ('2018-05-16', 6, 3000, 4000, 1, 1), ('2018-05-17', 7, 5000, 6000, 1, 1), ('2018-05-19', 8, 7000, 8000, 1, 1);
OPTIMIZE TABLE test.zero_rows_per_granule FINAL;
OPTIMIZE TABLE zero_rows_per_granule FINAL;
SELECT COUNT(*) FROM test.zero_rows_per_granule FINAL;
SELECT COUNT(*) FROM zero_rows_per_granule FINAL;
SELECT sum(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database='test' and active=1;
SELECT sum(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database=currentDatabase() and active=1;
DROP TABLE IF EXISTS test.zero_rows_per_granule;
DROP TABLE IF EXISTS zero_rows_per_granule;
SELECT '-----';
DROP TABLE IF EXISTS test.four_rows_per_granule;
DROP TABLE IF EXISTS four_rows_per_granule;
CREATE TABLE test.four_rows_per_granule (
CREATE TABLE four_rows_per_granule (
p Date,
k UInt64,
v1 UInt64,
......@@ -46,39 +46,39 @@ CREATE TABLE test.four_rows_per_granule (
vertical_merge_algorithm_min_rows_to_activate=0,
vertical_merge_algorithm_min_columns_to_activate=0;
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
SELECT COUNT(*) FROM test.four_rows_per_granule;
SELECT COUNT(*) FROM four_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, -1, 1), ('2018-05-16', 2, 3000, 4000, -1, 1), ('2018-05-17', 3, 5000, 6000, -1, 1), ('2018-05-18', 4, 7000, 8000, -1, 1);
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, -1, 1), ('2018-05-16', 2, 3000, 4000, -1, 1), ('2018-05-17', 3, 5000, 6000, -1, 1), ('2018-05-18', 4, 7000, 8000, -1, 1);
OPTIMIZE TABLE test.four_rows_per_granule FINAL;
OPTIMIZE TABLE four_rows_per_granule FINAL;
SELECT COUNT(*) FROM test.four_rows_per_granule;
SELECT COUNT(*) FROM four_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1);
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, 1, 1), ('2018-05-16', 6, 3000, 4000, 1, 1), ('2018-05-17', 7, 5000, 6000, 1, 1), ('2018-05-18', 8, 7000, 8000, 1, 1);
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, 1, 1), ('2018-05-16', 6, 3000, 4000, 1, 1), ('2018-05-17', 7, 5000, 6000, 1, 1), ('2018-05-18', 8, 7000, 8000, 1, 1);
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, -1, 1), ('2018-05-17', 7, 5000, 6000, -1, 1);
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 5, 1000, 2000, -1, 1), ('2018-05-17', 7, 5000, 6000, -1, 1);
OPTIMIZE TABLE test.four_rows_per_granule FINAL;
OPTIMIZE TABLE four_rows_per_granule FINAL;
SELECT COUNT(*) FROM test.four_rows_per_granule;
SELECT COUNT(*) FROM four_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
DROP TABLE IF EXISTS test.four_rows_per_granule;
DROP TABLE IF EXISTS four_rows_per_granule;
SELECT '-----';
DROP TABLE IF EXISTS test.six_rows_per_granule;
DROP TABLE IF EXISTS six_rows_per_granule;
CREATE TABLE test.six_rows_per_granule (
CREATE TABLE six_rows_per_granule (
p Date,
k UInt64,
v1 UInt64,
......@@ -92,28 +92,28 @@ CREATE TABLE test.six_rows_per_granule (
vertical_merge_algorithm_min_columns_to_activate=0;
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 1, 1000, 2000, -1, 2);
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 2, 1000, 2000, 1, 1), ('2018-05-16', 2, 1000, 2000, -1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 2, 1000, 2000, 1, 1), ('2018-05-16', 2, 1000, 2000, -1, 2);
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 3, 1000, 2000, 1, 1), ('2018-05-16', 3, 1000, 2000, -1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 3, 1000, 2000, 1, 1), ('2018-05-16', 3, 1000, 2000, -1, 2);
OPTIMIZE TABLE test.six_rows_per_granule FINAL;
OPTIMIZE TABLE six_rows_per_granule FINAL;
SELECT COUNT(*) FROM test.six_rows_per_granule;
SELECT COUNT(*) FROM six_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'six_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'six_rows_per_granule' and database=currentDatabase() and active=1;
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, -1, 1), ('2018-05-16', 1, 1000, 2000, 1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, -1, 1), ('2018-05-16', 1, 1000, 2000, 1, 2);
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 2, 1000, 2000, -1, 1), ('2018-05-16', 2, 1000, 2000, 1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 2, 1000, 2000, -1, 1), ('2018-05-16', 2, 1000, 2000, 1, 2);
INSERT INTO test.six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 3, 1000, 2000, -1, 1), ('2018-05-16', 3, 1000, 2000, 1, 2);
INSERT INTO six_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 3, 1000, 2000, -1, 1), ('2018-05-16', 3, 1000, 2000, 1, 2);
OPTIMIZE TABLE test.six_rows_per_granule FINAL;
OPTIMIZE TABLE six_rows_per_granule FINAL;
SELECT COUNT(*) FROM test.six_rows_per_granule;
SELECT COUNT(*) FROM six_rows_per_granule;
SELECT distinct(marks) from system.parts WHERE table = 'six_rows_per_granule' and database='test' and active=1;
SELECT distinct(marks) from system.parts WHERE table = 'six_rows_per_granule' and database=currentDatabase() and active=1;
DROP TABLE IF EXISTS test.six_rows_per_granule;
DROP TABLE IF EXISTS six_rows_per_granule;
DROP TABLE IF EXISTS test.partitions;
CREATE TABLE test.partitions (x UInt64) ENGINE = MergeTree ORDER BY x PARTITION BY x;
DROP TABLE IF EXISTS partitions;
CREATE TABLE partitions (x UInt64) ENGINE = MergeTree ORDER BY x PARTITION BY x;
INSERT INTO test.partitions SELECT * FROM system.numbers LIMIT 100;
SELECT count() FROM system.parts WHERE database = 'test' AND table = 'partitions';
INSERT INTO test.partitions SELECT * FROM system.numbers LIMIT 100;
SELECT count() FROM system.parts WHERE database = 'test' AND table = 'partitions';
INSERT INTO partitions SELECT * FROM system.numbers LIMIT 100;
SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'partitions';
INSERT INTO partitions SELECT * FROM system.numbers LIMIT 100;
SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'partitions';
SET max_partitions_per_insert_block = 1;
INSERT INTO test.partitions SELECT * FROM system.numbers LIMIT 1;
INSERT INTO test.partitions SELECT * FROM system.numbers LIMIT 2; -- { serverError 252 }
INSERT INTO partitions SELECT * FROM system.numbers LIMIT 1;
INSERT INTO partitions SELECT * FROM system.numbers LIMIT 2; -- { serverError 252 }
DROP TABLE test.partitions;
DROP TABLE partitions;
......@@ -51,7 +51,10 @@ def test_query(bin_prefix, sql_query, standalone_server):
query = 'CREATE DATABASE {random}; USE {random}; {query}'.format(random=random_name, query=query)
run_client(bin_prefix, tcp_port, query, reference, {random_name: 'default'})
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT database, name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
run_client(bin_prefix, tcp_port, query, 'SHOW ORPHANED TABLES\n')
run_client(bin_prefix, tcp_port, 'DROP DATABASE {random};'.format(random=random_name), '')
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
run_client(bin_prefix, tcp_port, query, 'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册