mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge pull request #27927 from ClickHouse/fix_27832
Сheck cluster name before creating Distributed
This commit is contained in:
commit
5ac6a99542
@ -1803,8 +1803,8 @@ std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) c
|
|||||||
auto res = getClusters()->getCluster(cluster_name);
|
auto res = getClusters()->getCluster(cluster_name);
|
||||||
if (res)
|
if (res)
|
||||||
return res;
|
return res;
|
||||||
|
if (!cluster_name.empty())
|
||||||
res = tryGetReplicatedDatabaseCluster(cluster_name);
|
res = tryGetReplicatedDatabaseCluster(cluster_name);
|
||||||
if (res)
|
if (res)
|
||||||
return res;
|
return res;
|
||||||
|
|
||||||
|
@ -327,11 +327,13 @@ StorageDistributed::StorageDistributed(
|
|||||||
const String & relative_data_path_,
|
const String & relative_data_path_,
|
||||||
const DistributedSettings & distributed_settings_,
|
const DistributedSettings & distributed_settings_,
|
||||||
bool attach_,
|
bool attach_,
|
||||||
ClusterPtr owned_cluster_)
|
ClusterPtr owned_cluster_,
|
||||||
|
ASTPtr remote_table_function_ptr_)
|
||||||
: IStorage(id_)
|
: IStorage(id_)
|
||||||
, WithContext(context_->getGlobalContext())
|
, WithContext(context_->getGlobalContext())
|
||||||
, remote_database(remote_database_)
|
, remote_database(remote_database_)
|
||||||
, remote_table(remote_table_)
|
, remote_table(remote_table_)
|
||||||
|
, remote_table_function_ptr(remote_table_function_ptr_)
|
||||||
, log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")"))
|
, log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")"))
|
||||||
, owned_cluster(std::move(owned_cluster_))
|
, owned_cluster(std::move(owned_cluster_))
|
||||||
, cluster_name(getContext()->getMacros()->expand(cluster_name_))
|
, cluster_name(getContext()->getMacros()->expand(cluster_name_))
|
||||||
@ -363,10 +365,13 @@ StorageDistributed::StorageDistributed(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Sanity check. Skip check if the table is already created to allow the server to start.
|
/// Sanity check. Skip check if the table is already created to allow the server to start.
|
||||||
if (!attach_ && !cluster_name.empty())
|
if (!attach_)
|
||||||
{
|
{
|
||||||
size_t num_local_shards = getContext()->getCluster(cluster_name)->getLocalShardCount();
|
if (remote_database.empty() && !remote_table_function_ptr && !getCluster()->maybeCrossReplication())
|
||||||
if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name)
|
LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly.");
|
||||||
|
|
||||||
|
size_t num_local_shards = getCluster()->getLocalShardCount();
|
||||||
|
if (num_local_shards && (remote_database.empty() || remote_database == id_.database_name) && remote_table == id_.table_name)
|
||||||
throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP);
|
throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -399,9 +404,9 @@ StorageDistributed::StorageDistributed(
|
|||||||
relative_data_path_,
|
relative_data_path_,
|
||||||
distributed_settings_,
|
distributed_settings_,
|
||||||
attach,
|
attach,
|
||||||
std::move(owned_cluster_))
|
std::move(owned_cluster_),
|
||||||
|
remote_table_function_ptr_)
|
||||||
{
|
{
|
||||||
remote_table_function_ptr = std::move(remote_table_function_ptr_);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
|
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
|
||||||
@ -810,9 +815,6 @@ void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_co
|
|||||||
|
|
||||||
void StorageDistributed::startup()
|
void StorageDistributed::startup()
|
||||||
{
|
{
|
||||||
if (remote_database.empty() && !remote_table_function_ptr && !getCluster()->maybeCrossReplication())
|
|
||||||
LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly.");
|
|
||||||
|
|
||||||
if (!storage_policy)
|
if (!storage_policy)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
@ -136,7 +136,8 @@ private:
|
|||||||
const String & relative_data_path_,
|
const String & relative_data_path_,
|
||||||
const DistributedSettings & distributed_settings_,
|
const DistributedSettings & distributed_settings_,
|
||||||
bool attach_,
|
bool attach_,
|
||||||
ClusterPtr owned_cluster_ = {});
|
ClusterPtr owned_cluster_ = {},
|
||||||
|
ASTPtr remote_table_function_ptr_ = {});
|
||||||
|
|
||||||
StorageDistributed(
|
StorageDistributed(
|
||||||
const StorageID & id_,
|
const StorageID & id_,
|
||||||
|
@ -4,8 +4,7 @@ DROP TABLE IF EXISTS distr2;
|
|||||||
|
|
||||||
CREATE TABLE distr (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr); -- { serverError 269 }
|
CREATE TABLE distr (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr); -- { serverError 269 }
|
||||||
|
|
||||||
CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0);
|
CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); -- { serverError 269 }
|
||||||
SELECT * FROM distr0; -- { serverError 581 }
|
|
||||||
|
|
||||||
CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2);
|
CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2);
|
||||||
CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1);
|
CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1);
|
||||||
@ -13,6 +12,5 @@ CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, current
|
|||||||
SELECT * FROM distr1; -- { serverError 581 }
|
SELECT * FROM distr1; -- { serverError 581 }
|
||||||
SELECT * FROM distr2; -- { serverError 581 }
|
SELECT * FROM distr2; -- { serverError 581 }
|
||||||
|
|
||||||
DROP TABLE distr0;
|
|
||||||
DROP TABLE distr1;
|
DROP TABLE distr1;
|
||||||
DROP TABLE distr2;
|
DROP TABLE distr2;
|
||||||
|
@ -9,7 +9,9 @@ CREATE TABLE tt6
|
|||||||
`status` String
|
`status` String
|
||||||
|
|
||||||
)
|
)
|
||||||
ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand());
|
ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand());
|
||||||
|
|
||||||
|
CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand());
|
||||||
|
|
||||||
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 }
|
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 }
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user