add custom partitioning tests [#CLICKHOUSE-3000]

This commit is contained in:
Alexey Zatelepin 2017-09-11 20:33:01 +03:00 committed by alexey-milovidov
parent 1be62b567e
commit 4160bcfdf5
4 changed files with 302 additions and 0 deletions

View File

@ -0,0 +1,50 @@
*** Not partitioned ***
Parts before OPTIMIZE:
tuple() all_1_1_0
tuple() all_2_2_0
Parts after OPTIMIZE:
tuple() all_1_2_1
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
*** Partitioned by week ***
Parts before OPTIMIZE:
\'1999-12-27\' 19991227_1_1_0
\'2000-01-03\' 20000103_2_2_0
\'2000-01-03\' 20000103_3_3_0
Parts after OPTIMIZE:
\'1999-12-27\' 19991227_1_1_0
\'2000-01-03\' 20000103_2_3_1
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
12
*** Partitioned by a (Date, UInt8) tuple ***
Parts before OPTIMIZE:
(\'2000-01-01\', 1) 20000101-1_1_1_0
(\'2000-01-01\', 1) 20000101-1_5_5_0
(\'2000-01-01\', 2) 20000101-2_2_2_0
(\'2000-01-02\', 1) 20000102-1_3_3_0
(\'2000-01-02\', 1) 20000102-1_4_4_0
Parts after OPTIMIZE:
(\'2000-01-01\', 1) 20000101-1_1_5_1
(\'2000-01-01\', 2) 20000101-2_2_2_0
(\'2000-01-02\', 1) 20000102-1_3_4_1
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
9
*** Partitioned by String ***
Parts before OPTIMIZE:
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_1_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_4_4_0
Parts after OPTIMIZE:
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_4_1
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
8

View File

@ -0,0 +1,94 @@
-- IMPORTANT: Don't use this setting just yet.
-- It is for testing purposes, the syntax will likely change soon and the server will not be able
-- to load the tables created this way. You have been warned.
SET experimental_merge_tree_allow_custom_partitions = 1;
SELECT '*** Not partitioned ***';
DROP TABLE IF EXISTS test.not_partitioned;
CREATE TABLE test.not_partitioned(x UInt8) ENGINE = MergeTree(tuple(), x, 8192);
INSERT INTO test.not_partitioned VALUES (1), (2), (3);
INSERT INTO test.not_partitioned VALUES (4), (5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned' AND active ORDER BY name;
OPTIMIZE TABLE test.not_partitioned PARTITION tuple() FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned;
ALTER TABLE test.not_partitioned DETACH PARTITION ID 'all';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned;
DROP TABLE test.not_partitioned;
SELECT '*** Partitioned by week ***';
DROP TABLE IF EXISTS test.partitioned_by_week;
CREATE TABLE test.partitioned_by_week(d Date, x UInt8) ENGINE = MergeTree(toMonday(d), x, 8192);
-- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02
INSERT INTO test.partitioned_by_week VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3);
INSERT INTO test.partitioned_by_week VALUES ('2000-01-03', 4), ('2000-01-03', 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_week PARTITION '2000-01-03' FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week;
ALTER TABLE test.partitioned_by_week DROP PARTITION '1999-12-27';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week;
DROP TABLE test.partitioned_by_week;
SELECT '*** Partitioned by a (Date, UInt8) tuple ***';
DROP TABLE IF EXISTS test.partitioned_by_tuple;
CREATE TABLE test.partitioned_by_tuple(d Date, x UInt8, y UInt8) ENGINE = MergeTree((d, x), x, 8192);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_tuple PARTITION ('2000-01-01', 1) FINAL;
OPTIMIZE TABLE test.partitioned_by_tuple PARTITION ('2000-01-02', 1) FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple;
ALTER TABLE test.partitioned_by_tuple DETACH PARTITION ID '20000101-1';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple;
DROP TABLE test.partitioned_by_tuple;
SELECT '*** Partitioned by String ***';
DROP TABLE IF EXISTS test.partitioned_by_string;
CREATE TABLE test.partitioned_by_string(s String, x UInt8) ENGINE = MergeTree(tuple(s), x, 8192);
INSERT INTO test.partitioned_by_string VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3);
INSERT INTO test.partitioned_by_string VALUES ('bbb', 4), ('aaa', 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_string PARTITION 'aaa' FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string;
ALTER TABLE test.partitioned_by_string DROP PARTITION 'bbb';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string;
DROP TABLE test.partitioned_by_string;

View File

@ -0,0 +1,50 @@
*** Not partitioned ***
Parts before OPTIMIZE:
tuple() all_0_0_0
tuple() all_1_1_0
Parts after OPTIMIZE:
tuple() all_0_1_1
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
*** Partitioned by week ***
Parts before OPTIMIZE:
\'1999-12-27\' 19991227_0_0_0
\'2000-01-03\' 20000103_0_0_0
\'2000-01-03\' 20000103_1_1_0
Parts after OPTIMIZE:
\'1999-12-27\' 19991227_0_0_0
\'2000-01-03\' 20000103_0_1_1
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
12
*** Partitioned by a (Date, UInt8) tuple ***
Parts before OPTIMIZE:
(\'2000-01-01\', 1) 20000101-1_0_0_0
(\'2000-01-01\', 1) 20000101-1_1_1_0
(\'2000-01-01\', 2) 20000101-2_0_0_0
(\'2000-01-02\', 1) 20000102-1_0_0_0
(\'2000-01-02\', 1) 20000102-1_1_1_0
Parts after OPTIMIZE:
(\'2000-01-01\', 1) 20000101-1_0_1_1
(\'2000-01-01\', 2) 20000101-2_0_0_0
(\'2000-01-02\', 1) 20000102-1_0_1_1
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
9
*** Partitioned by String ***
Parts before OPTIMIZE:
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_0_0_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_1_0
Parts after OPTIMIZE:
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0
\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0
\'aaa\' 9b50856126a8a6064f11f027d455bf58_0_1_1
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
8

View File

@ -0,0 +1,108 @@
-- IMPORTANT: Don't use this setting just yet.
-- It is for testing purposes, the syntax will likely change soon and the server will not be able
-- to load the tables created this way. You have been warned.
SET experimental_merge_tree_allow_custom_partitions = 1;
SET replication_alter_partitions_sync = 2;
SELECT '*** Not partitioned ***';
DROP TABLE IF EXISTS test.not_partitioned_replica1;
DROP TABLE IF EXISTS test.not_partitioned_replica2;
CREATE TABLE test.not_partitioned_replica1(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned', '1', tuple(), x, 8192);
CREATE TABLE test.not_partitioned_replica2(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/not_partitioned', '2', tuple(), x, 8192);
INSERT INTO test.not_partitioned_replica1 VALUES (1), (2), (3);
INSERT INTO test.not_partitioned_replica1 VALUES (4), (5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned_replica1' AND active ORDER BY name;
OPTIMIZE TABLE test.not_partitioned_replica1 PARTITION tuple() FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned_replica2' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned_replica2;
ALTER TABLE test.not_partitioned_replica1 DETACH PARTITION ID 'all';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned_replica2;
DROP TABLE test.not_partitioned_replica1;
DROP TABLE test.not_partitioned_replica2;
SELECT '*** Partitioned by week ***';
DROP TABLE IF EXISTS test.partitioned_by_week_replica1;
DROP TABLE IF EXISTS test.partitioned_by_week_replica2;
CREATE TABLE test.partitioned_by_week_replica1(d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week', '1', toMonday(d), x, 8192);
CREATE TABLE test.partitioned_by_week_replica2(d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_week', '2', toMonday(d), x, 8192);
-- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02
INSERT INTO test.partitioned_by_week_replica1 VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3);
INSERT INTO test.partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03', 5);
SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication.
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week_replica1' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL;
SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync.
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week_replica2' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week_replica2;
ALTER TABLE test.partitioned_by_week_replica1 DROP PARTITION '1999-12-27';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week_replica2;
DROP TABLE test.partitioned_by_week_replica1;
DROP TABLE test.partitioned_by_week_replica2;
SELECT '*** Partitioned by a (Date, UInt8) tuple ***';
DROP TABLE IF EXISTS test.partitioned_by_tuple_replica1;
DROP TABLE IF EXISTS test.partitioned_by_tuple_replica2;
CREATE TABLE test.partitioned_by_tuple_replica1(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple', '1', (d, x), x, 8192);
CREATE TABLE test.partitioned_by_tuple_replica2(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_tuple', '2', (d, x), x, 8192);
INSERT INTO test.partitioned_by_tuple_replica1 VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3);
INSERT INTO test.partitioned_by_tuple_replica1 VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple_replica1' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_tuple_replica1 PARTITION ('2000-01-01', 1) FINAL;
OPTIMIZE TABLE test.partitioned_by_tuple_replica1 PARTITION ('2000-01-02', 1) FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple_replica2' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple_replica2;
ALTER TABLE test.partitioned_by_tuple_replica1 DETACH PARTITION ID '20000101-1';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple_replica2;
DROP TABLE test.partitioned_by_tuple_replica1;
DROP TABLE test.partitioned_by_tuple_replica2;
SELECT '*** Partitioned by String ***';
DROP TABLE IF EXISTS test.partitioned_by_string_replica1;
DROP TABLE IF EXISTS test.partitioned_by_string_replica2;
CREATE TABLE test.partitioned_by_string_replica1(s String, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string', '1', tuple(s), x, 8192);
CREATE TABLE test.partitioned_by_string_replica2(s String, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/partitioned_by_string', '2', tuple(s), x, 8192);
INSERT INTO test.partitioned_by_string_replica1 VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3);
INSERT INTO test.partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string_replica1' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_string_replica2 PARTITION 'aaa' FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string_replica2' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string_replica2;
ALTER TABLE test.partitioned_by_string_replica1 DROP PARTITION 'bbb';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string_replica2;
DROP TABLE test.partitioned_by_string_replica1;
DROP TABLE test.partitioned_by_string_replica2;