mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Prevent conversion to Replicated if zookeeper path already exists
This commit is contained in:
parent
6d544d9417
commit
0ce90f4ef4
@ -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));
|
||||
|
@ -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>
|
||||
|
@ -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()
|
||||
|
Loading…
Reference in New Issue
Block a user