Merge pull request #38999 from azat/tests-improve-zk-path-check

tests: unique ZooKeeper path for Replicated.*MergeTree tables
This commit is contained in:
Alexander Tokmakov 2022-07-08 15:47:44 +03:00 committed by GitHub
commit 435a109242
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 37 additions and 39 deletions

View File

@ -63,28 +63,28 @@ CREATE TABLE aggregating_merge_tree_with_sampling
CREATE TABLE replicated_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE replicated_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y);
CREATE TABLE replicated_versioned_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_versioned_collapsing_merge_tree/', 'r1', d, (a, b), 111, y, b);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_versioned_collapsing_merge_tree/', 'r1', d, (a, b), 111, y, b);
CREATE TABLE replicated_summing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test_00083/01/replicated_summing_merge_tree/', 'r1', d, (a, b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_summing_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE replicated_summing_merge_tree_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test_00083/01/replicated_summing_merge_tree_with_list_of_columns_to_sum/', 'r1', d, (a, b), 111, (y, z));
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_summing_merge_tree_with_list_of_columns_to_sum/', 'r1', d, (a, b), 111, (y, z));
CREATE TABLE replicated_aggregating_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00083/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE replicated_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE replicated_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y);
CREATE TABLE replicated_versioned_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test_00083/01/replicated_versioned_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b, b), 111, y, b);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_versioned_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b, b), 111, y, b);
CREATE TABLE replicated_summing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test_00083/01/replicated_summing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_summing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test_00083/01/replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, (y, z));
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, (y, z));
CREATE TABLE replicated_aggregating_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00083/01/replicated_aggregating_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{database}/test_00083/01/replicated_aggregating_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
INSERT INTO merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);

View File

@ -1,9 +1,9 @@
-- Tags: disabled, zookeeper, no-parallel
-- Tags: disabled, zookeeper
DROP TABLE IF EXISTS r1;
DROP TABLE IF EXISTS r2;
CREATE TABLE r1 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r1', d, x, 111);
CREATE TABLE r1 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/01/r/', 'r1', d, x, 111);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 1;
@ -157,7 +157,7 @@ ALTER TABLE r1 DETACH PARTITION 201601;
SELECT count() FROM r1;
CREATE TABLE r2 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r2', d, x, 111);
CREATE TABLE r2 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/01/r/', 'r2', d, x, 111);
SELECT count() FROM r2;

View File

@ -38,7 +38,7 @@ SELECT '*** Replicated Collapsing ***';
DROP TABLE IF EXISTS replicated_collapsing;
CREATE TABLE replicated_collapsing(d Date, x UInt32, sign Int8)
ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test_00509/replicated_collapsing', 'r1', sign)
ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/{database}/test_00509/replicated_collapsing', 'r1', sign)
PARTITION BY toYYYYMM(d) ORDER BY d;
INSERT INTO replicated_collapsing VALUES ('2017-10-23', 1, 1);
@ -56,7 +56,7 @@ SELECT '*** Replicated VersionedCollapsing ***';
DROP TABLE IF EXISTS replicated_versioned_collapsing;
CREATE TABLE replicated_versioned_collapsing(d Date, x UInt32, sign Int8, version UInt8)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test_00509/replicated_versioned_collapsing', 'r1', sign, version)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/{database}/test_00509/replicated_versioned_collapsing', 'r1', sign, version)
PARTITION BY toYYYYMM(d) ORDER BY (d, version);
INSERT INTO replicated_versioned_collapsing VALUES ('2017-10-23', 1, 1, 0);

View File

@ -12,10 +12,10 @@ CREATE TABLE without_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0;
CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001'
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy
AS SELECT 0 AS dummy, countState(x) AS cnt FROM with_deduplication;
CREATE MATERIALIZED VIEW without_deduplication_mv UUID '00000510-1000-4000-8000-000000000002'
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/without_deduplication_mv', 'r1') ORDER BY dummy
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{database}/test_00510/without_deduplication_mv', 'r1') ORDER BY dummy
AS SELECT 0 AS dummy, countState(x) AS cnt FROM without_deduplication;
INSERT INTO with_deduplication VALUES (42);

View File

@ -1,4 +1,4 @@
-- Tags: replica, no-replicated-database, no-parallel
-- Tags: replica, no-replicated-database
-- Tag no-replicated-database: Fails due to additional replicas or shards
-- The test is mostly outdated as now every replica is leader and can do OPTIMIZE locally.
@ -6,8 +6,8 @@
DROP TABLE IF EXISTS rename1;
DROP TABLE IF EXISTS rename2;
DROP TABLE IF EXISTS rename3;
CREATE TABLE rename1 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/rename', '1', v) PARTITION BY p ORDER BY i;
CREATE TABLE rename2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/rename', '2', v) PARTITION BY p ORDER BY i;
CREATE TABLE rename1 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{database}/rename', '1', v) PARTITION BY p ORDER BY i;
CREATE TABLE rename2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{database}/rename', '2', v) PARTITION BY p ORDER BY i;
INSERT INTO rename1 VALUES (0, 1, 0);
INSERT INTO rename1 VALUES (0, 1, 1);

View File

@ -4,8 +4,8 @@ SET optimize_on_insert = 0;
DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661;
DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661;
CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w);
CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w);
CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w);
CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w);
INSERT INTO partitioned_by_tuple_replica1_00661 VALUES ('2000-01-02', 1, 'first', 3);
INSERT INTO partitioned_by_tuple_replica1_00661 VALUES ('2000-01-01', 2, 'first', 2);

View File

@ -1,4 +1,4 @@
-- Tags: zookeeper, no-parallel
-- Tags: zookeeper
DROP TABLE IF EXISTS versioned_collapsing_table;
@ -10,13 +10,13 @@ CREATE TABLE versioned_collapsing_table(
sign Int8,
version UInt16
)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table/{shard}', '{replica}', sign, version)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/{database}/versioned_collapsing_table/{shard}', '{replica}', sign, version)
PARTITION BY d
ORDER BY (key1, key2);
INSERT INTO versioned_collapsing_table VALUES (toDate('2019-10-10'), 1, 1, 'Hello', -1, 1);
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table/s1' and name = 'metadata';
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/' || currentDatabase() || '/versioned_collapsing_table/s1' and name = 'metadata';
SELECT COUNT() FROM versioned_collapsing_table;

View File

@ -12,7 +12,7 @@ CREATE TABLE default.merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String
2 b 0
2 e 555
CREATE TABLE default.merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String,\n `key2` UInt64\n)\nENGINE = ReplacingMergeTree\nPRIMARY KEY key\nORDER BY (key, key2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.replicated_merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String\n)\nENGINE = ReplicatedReplacingMergeTree(\'/clickhouse/test/01532_primary_key_without\', \'r1\')\nPRIMARY KEY key\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.replicated_merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String\n)\nENGINE = ReplicatedReplacingMergeTree(\'/clickhouse/tables/default/01532_primary_key_without\', \'r1\')\nPRIMARY KEY key\nORDER BY key\nSETTINGS index_granularity = 8192
1 a
2 b
1 c
@ -20,4 +20,4 @@ CREATE TABLE default.replicated_merge_tree_pk_sql\n(\n `key` UInt64,\n `va
1 c 0
2 b 0
2 e 555
CREATE TABLE default.replicated_merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String,\n `key2` UInt64\n)\nENGINE = ReplicatedReplacingMergeTree(\'/clickhouse/test/01532_primary_key_without\', \'r1\')\nPRIMARY KEY key\nORDER BY (key, key2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.replicated_merge_tree_pk_sql\n(\n `key` UInt64,\n `value` String,\n `key2` UInt64\n)\nENGINE = ReplicatedReplacingMergeTree(\'/clickhouse/tables/default/01532_primary_key_without\', \'r1\')\nPRIMARY KEY key\nORDER BY (key, key2)\nSETTINGS index_granularity = 8192

View File

@ -1,4 +1,4 @@
-- Tags: zookeeper, no-parallel
-- Tags: zookeeper
DROP TABLE IF EXISTS merge_tree_pk;
@ -70,7 +70,7 @@ CREATE TABLE replicated_merge_tree_pk_sql
value String,
PRIMARY KEY (key)
)
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/01532_primary_key_without', 'r1');
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{database}/01532_primary_key_without', 'r1');
SHOW CREATE TABLE replicated_merge_tree_pk_sql;

View File

@ -1,4 +1,4 @@
-- Tags: zookeeper, no-parallel
-- Tags: zookeeper
DROP TABLE IF EXISTS empty1;
DROP TABLE IF EXISTS empty2;
@ -6,11 +6,11 @@ DROP TABLE IF EXISTS empty2;
SELECT 'Check creating empty parts';
CREATE TABLE empty1 (key UInt32, val UInt32, date Datetime)
ENGINE=ReplicatedSummingMergeTree('/clickhouse/01560_optimize_on_insert', '1', val)
ENGINE=ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_01560_optimize_on_insert', '1', val)
PARTITION BY date ORDER BY key;
CREATE TABLE empty2 (key UInt32, val UInt32, date Datetime)
ENGINE=ReplicatedSummingMergeTree('/clickhouse/01560_optimize_on_insert', '2', val)
ENGINE=ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test_01560_optimize_on_insert', '2', val)
PARTITION BY date ORDER BY key;
INSERT INTO empty2 VALUES (1, 1, '2020-01-01'), (1, 1, '2020-01-01'), (1, -2, '2020-01-01');

View File

@ -1,9 +1,7 @@
-- Tags: no-parallel
SELECT 'simple partition key:';
DROP TABLE IF EXISTS table1 SYNC;
CREATE TABLE table1 (id Int64, v UInt64)
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/table12', '1', v)
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{database}/test_table12', '1', v)
PARTITION BY id % 200 ORDER BY id;
INSERT INTO table1 SELECT number-205, number FROM numbers(10);
INSERT INTO table1 SELECT number-205, number FROM numbers(400, 10);

View File

@ -201,7 +201,7 @@ done
tests_with_replicated_merge_tree=( $(
find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' |
grep -vP $EXCLUDE_DIRS |
xargs grep --with-filename -e ReplicatedMergeTree | cut -d: -f1 | sort -u
xargs grep --with-filename -e "Replicated.*MergeTree[ ]*(.*" | cut -d: -f1 | sort -u
) )
for test_case in "${tests_with_replicated_merge_tree[@]}"; do
case "$test_case" in
@ -209,11 +209,11 @@ for test_case in "${tests_with_replicated_merge_tree[@]}"; do
;;
*.sh)
test_case_zk_prefix="\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX"
grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
;;
*.sql|*.sql.j2)
test_case_zk_prefix="\({database}\|currentDatabase()\)"
grep -q -e "ReplicatedMergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "ReplicatedMergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
test_case_zk_prefix="\({database}\|currentDatabase()\|{uuid}\|{default_path_test}\)"
grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
;;
*.py)
# Right now there is not such tests anyway