This commit is contained in:
Alexander Tokmakov 2024-07-09 16:45:15 +02:00
parent b948d5ad85
commit 6cb361413e
9 changed files with 16 additions and 5 deletions

View File

@ -1008,6 +1008,9 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
query_context->setSetting("allow_create_index_without_type", 1);
query_context->setSetting("allow_experimental_s3queue", 1);
query_context->setSetting("database_replicated_allow_explicit_uuid", 3);
query_context->setSetting("database_replicated_allow_replicated_engine_arguments", 3);
auto txn = std::make_shared<ZooKeeperMetadataTransaction>(current_zookeeper, zookeeper_path, false, "");
query_context->initZooKeeperMetadataTransaction(txn);
return query_context;

View File

@ -311,7 +311,8 @@ def test_replicated_database(cluster):
SETTINGS storage_policy = 'web';
""".format(
uuids[0]
)
),
settings={"database_replicated_allow_explicit_uuid": 3},
)
node2 = cluster.instances["node2"]

View File

@ -6,6 +6,8 @@ DROP TABLE IF EXISTS without_deduplication;
DROP TABLE IF EXISTS with_deduplication_mv;
DROP TABLE IF EXISTS without_deduplication_mv;
SET database_replicated_allow_explicit_uuid=3;
SET database_replicated_allow_replicated_engine_arguments=3;
CREATE TABLE with_deduplication(x UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_00510/with_deduplication', 'r1') ORDER BY x;
CREATE TABLE without_deduplication(x UInt32)

View File

@ -6,6 +6,7 @@ DROP TABLE IF EXISTS test_mv_00609;
create table test_00609 (a Int8) engine=Memory;
insert into test_00609 values (1);
set database_replicated_allow_explicit_uuid=3;
set allow_deprecated_syntax_for_merge_tree=1;
create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609;

View File

@ -7,6 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3"
# there are some issues with Atomic database, let's generate it uniq
# otherwise flaky check will not pass.
uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT reinterpretAsUUID(currentDatabase())")

View File

@ -14,6 +14,7 @@ INSERT INTO src VALUES (3), (4);
SELECT * FROM mv ORDER BY n;
DROP TABLE mv SYNC;
SET database_replicated_allow_explicit_uuid=3;
SET show_table_uuid_in_table_create_query_if_not_nil=1;
CREATE TABLE ".inner_id.e15f3ab5-6cae-4df3-b879-f40deafd82c2" (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n;
ATTACH MATERIALIZED VIEW mv UUID 'e15f3ab5-6cae-4df3-b879-f40deafd82c2' (n Int32, n2 Int64) ENGINE = MergeTree PARTITION BY n % 10 ORDER BY n AS SELECT n, n * n AS n2 FROM src;

View File

@ -6,5 +6,5 @@ rmt1
rmt2
02858000-1000-4000-8000-000000000
0
CREATE TABLE rdb_default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE rdb_default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt1\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/02858_explicit_uuid_and_zk_path_default/rmt/{shard}\', \'_{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt2\n(\n `n` Int32\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/{uuid}/{shard}\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192

View File

@ -38,7 +38,7 @@ $CLICKHOUSE_CLIENT -q "SELECT name FROM system.tables WHERE database='$db' ORDER
$CLICKHOUSE_CLIENT -q "SELECT substring(toString(uuid) as s, 1, length(s) - 3) FROM system.tables WHERE database='$db' and name='m1'"
$CLICKHOUSE_CLIENT -q "SELECT toString(uuid) LIKE '02858000%' FROM system.tables WHERE database='$db' and name='m2'"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt1" | sed "s/$db/default/g"
$CLICKHOUSE_CLIENT -q "SHOW CREATE $db.rmt2" | sed "s/$db/default/g"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS rdb_$CLICKHOUSE_DATABASE"

View File

@ -5,6 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --database_replicated_allow_explicit_uuid 3 --database_replicated_allow_replicated_engine_arguments 3"
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test_exception_replicated SYNC"