Merge pull request #27927 from ClickHouse/fix_27832

Сheck cluster name before creating Distributed
This commit is contained in:
alexey-milovidov 2021-08-21 10:40:23 +03:00 committed by GitHub
commit 5ac6a99542
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 19 additions and 16 deletions

View File

@ -1803,8 +1803,8 @@ std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) c
auto res = getClusters()->getCluster(cluster_name);
if (res)
return res;
res = tryGetReplicatedDatabaseCluster(cluster_name);
if (!cluster_name.empty())
res = tryGetReplicatedDatabaseCluster(cluster_name);
if (res)
return res;

View File

@ -327,11 +327,13 @@ StorageDistributed::StorageDistributed(
const String & relative_data_path_,
const DistributedSettings & distributed_settings_,
bool attach_,
ClusterPtr owned_cluster_)
ClusterPtr owned_cluster_,
ASTPtr remote_table_function_ptr_)
: IStorage(id_)
, WithContext(context_->getGlobalContext())
, remote_database(remote_database_)
, remote_table(remote_table_)
, remote_table_function_ptr(remote_table_function_ptr_)
, log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")"))
, owned_cluster(std::move(owned_cluster_))
, 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.
if (!attach_ && !cluster_name.empty())
if (!attach_)
{
size_t num_local_shards = getContext()->getCluster(cluster_name)->getLocalShardCount();
if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name)
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.");
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);
}
}
@ -399,9 +404,9 @@ StorageDistributed::StorageDistributed(
relative_data_path_,
distributed_settings_,
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(
@ -810,9 +815,6 @@ void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_co
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)
return;

View File

@ -136,7 +136,8 @@ private:
const String & relative_data_path_,
const DistributedSettings & distributed_settings_,
bool attach_,
ClusterPtr owned_cluster_ = {});
ClusterPtr owned_cluster_ = {},
ASTPtr remote_table_function_ptr_ = {});
StorageDistributed(
const StorageID & id_,

View File

@ -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 distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0);
SELECT * FROM distr0; -- { serverError 581 }
CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); -- { serverError 269 }
CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2);
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 distr2; -- { serverError 581 }
DROP TABLE distr0;
DROP TABLE distr1;
DROP TABLE distr2;

View File

@ -9,7 +9,9 @@ CREATE TABLE tt6
`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 }