Prevent conversion to Replicated if zookeeper path already exists

This commit is contained in:
Кирилл Гарбар 2024-05-12 18:13:17 +03:00
parent 6d544d9417
commit 0ce90f4ef4
3 changed files with 59 additions and 4 deletions

View File

@ -76,6 +76,20 @@ static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr contex
String replica_path = server_settings.default_replica_path;
String replica_name = server_settings.default_replica_name;
/// Check that replica path doesn't exist
Macros::MacroExpansionInfo info;
StorageID table_id = StorageID(create_query->getDatabase(), create_query->getTable(), create_query->uuid);
info.table_id = table_id;
info.expand_special_macros_only = false;
String zookeeper_path = context->getMacros()->expand(replica_path, info);
if (context->getZooKeeper()->exists(zookeeper_path))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Found existing ZooKeeper path {} while trying to convert table {} to replicated. Table will not be converted.",
zookeeper_path, backQuote(table_id.getFullTableName())
);
auto args = std::make_shared<ASTExpressionList>();
args->children.push_back(std::make_shared<ASTLiteral>(replica_path));
args->children.push_back(std::make_shared<ASTLiteral>(replica_name));

View File

@ -15,8 +15,10 @@
</cluster>
</remote_servers>
<default_replica_path>/clickhouse/tables/{database}/{table}/{uuid}</default_replica_path>
<macros>
<shard>01</shard>
</macros>
</clickhouse>
</clickhouse>

View File

@ -1,5 +1,9 @@
import pytest
from test_modify_engine_on_restart.common import check_flags_deleted, set_convert_flags
from test_modify_engine_on_restart.common import (
check_flags_deleted,
set_convert_flags,
get_table_path,
)
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
@ -120,7 +124,7 @@ def check_replica_added():
q(
ch2,
f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{uuid}/{{shard}}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A",
f"CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/rmt/{uuid}', '{{replica}}') PARTITION BY toYYYYMM(D) ORDER BY A",
)
ch2.query(database=database_name, sql="SYSTEM SYNC REPLICA rmt", timeout=20)
@ -136,7 +140,7 @@ def check_replica_added():
def test_modify_engine_on_restart(started_cluster):
ch1.query("CREATE DATABASE " + database_name + " ON CLUSTER cluster")
ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster")
create_tables()
@ -159,3 +163,38 @@ def test_modify_engine_on_restart(started_cluster):
ch1.restart_clickhouse()
check_tables(True)
def test_modify_engine_fails_if_zk_path_exists(started_cluster):
ch1.query("CREATE DATABASE IF NOT EXISTS " + database_name + " ON CLUSTER cluster")
q(
ch1,
"CREATE TABLE already_exists_1 ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A;",
)
uuid = q(
ch1,
f"SELECT uuid FROM system.tables WHERE table = 'already_exists_1' and database = '{database_name}'",
).strip("'[]\n")
q(
ch1,
f"CREATE TABLE already_exists_2 ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database_name}/already_exists_1/{uuid}', 'r2') PARTITION BY toYYYYMM(D) ORDER BY A;",
)
set_convert_flags(ch1, database_name, ["already_exists_1"])
table_data_path = get_table_path(ch1, "already_exists_1", database_name)
ch1.stop_clickhouse()
ch1.start_clickhouse(retry_start=False, expected_to_fail=True)
# Check if we can cancel convertation
ch1.exec_in_container(
[
"bash",
"-c",
f"rm {table_data_path}convert_to_replicated",
]
)
ch1.start_clickhouse()