mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
dbms: Cluster: cleanups [#METR-2944].
This commit is contained in:
parent
a7a988bdc3
commit
b5b1693573
@ -18,8 +18,8 @@ class Cluster : private boost::noncopyable
|
||||
public:
|
||||
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name);
|
||||
|
||||
/// Построить кластер по именам шардов и реплик, локальные обрабатываются так же как удаленные
|
||||
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector< std::vector<String> > names,
|
||||
/// Построить кластер по именам шардов и реплик. Локальные обрабатываются так же как удаленные.
|
||||
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector<std::vector<String>> names,
|
||||
const String & username, const String & password);
|
||||
|
||||
/// количество узлов clickhouse сервера, расположенных локально
|
||||
@ -80,7 +80,7 @@ private:
|
||||
Addresses addresses;
|
||||
AddressesWithFailover addresses_with_failover;
|
||||
|
||||
size_t local_nodes_num;
|
||||
size_t local_nodes_num = 0;
|
||||
};
|
||||
|
||||
struct Clusters
|
||||
|
@ -59,8 +59,7 @@ Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_
|
||||
}
|
||||
|
||||
|
||||
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name):
|
||||
local_nodes_num(0)
|
||||
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
@ -81,7 +80,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
||||
if (const auto is_local = isLocal(addresses.back()))
|
||||
shard_info_vec.push_back({{}, weight, is_local });
|
||||
shard_info_vec.push_back({{}, weight, is_local});
|
||||
else
|
||||
shard_info_vec.push_back({{addressToDirName(addresses.back())}, weight, is_local});
|
||||
}
|
||||
@ -149,89 +148,93 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
|
||||
}
|
||||
|
||||
if (!addresses_with_failover.empty() && !addresses.empty())
|
||||
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
if (addresses_with_failover.size())
|
||||
if (addresses_with_failover.size())
|
||||
{
|
||||
for (const auto & shard : addresses_with_failover)
|
||||
{
|
||||
for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it)
|
||||
{
|
||||
ConnectionPools replicas;
|
||||
replicas.reserve(it->size());
|
||||
ConnectionPools replicas;
|
||||
replicas.reserve(shard.size());
|
||||
|
||||
bool has_local_replics = false;
|
||||
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
|
||||
{
|
||||
if (isLocal(*jt))
|
||||
{
|
||||
has_local_replics = true;
|
||||
break;
|
||||
}
|
||||
else
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
jt->host_port.host().toString(), jt->host_port.port(), "", jt->user, jt->password, data_type_factory, "server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
}
|
||||
|
||||
if (has_local_replics)
|
||||
++local_nodes_num;
|
||||
else
|
||||
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
||||
}
|
||||
}
|
||||
else if (addresses.size())
|
||||
{
|
||||
for (Addresses::const_iterator it = addresses.begin(); it != addresses.end(); ++it)
|
||||
bool has_local_replics = false;
|
||||
for (const auto & replica : shard)
|
||||
{
|
||||
if (isLocal(*it))
|
||||
if (isLocal(replica))
|
||||
{
|
||||
++local_nodes_num;
|
||||
has_local_replics = true;
|
||||
break;
|
||||
}
|
||||
else
|
||||
{
|
||||
pools.emplace_back(new ConnectionPool(
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
it->host_port.host().toString(), it->host_port.port(), "", it->user, it->password, data_type_factory, "server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout, settings.limits.max_execution_time),
|
||||
replica.host_port.host().toString(), replica.host_port.port(), "", replica.user, replica.password,
|
||||
data_type_factory, "server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
}
|
||||
|
||||
if (has_local_replics)
|
||||
++local_nodes_num;
|
||||
else
|
||||
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
||||
}
|
||||
else
|
||||
throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
}
|
||||
else if (addresses.size())
|
||||
{
|
||||
for (const auto & address : addresses)
|
||||
{
|
||||
if (isLocal(address))
|
||||
{
|
||||
++local_nodes_num;
|
||||
}
|
||||
else
|
||||
{
|
||||
pools.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_port.host().toString(), address.host_port.port(), "", address.user, address.password,
|
||||
data_type_factory, "server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
}
|
||||
|
||||
|
||||
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector< std::vector<String> > names,
|
||||
const String & username, const String & password): local_nodes_num(0)
|
||||
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector<std::vector<String>> names,
|
||||
const String & username, const String & password)
|
||||
{
|
||||
for (size_t i = 0; i < names.size(); ++i)
|
||||
for (const auto & shard : names)
|
||||
{
|
||||
Addresses current;
|
||||
for (size_t j = 0; j < names[i].size(); ++j)
|
||||
current.emplace_back(names[i][j], username, password);
|
||||
for (auto & replica : shard)
|
||||
current.emplace_back(replica, username, password);
|
||||
addresses_with_failover.emplace_back(current);
|
||||
}
|
||||
|
||||
for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it)
|
||||
for (const auto & shard : addresses_with_failover)
|
||||
{
|
||||
ConnectionPools replicas;
|
||||
replicas.reserve(it->size());
|
||||
replicas.reserve(shard.size());
|
||||
|
||||
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
|
||||
for (const auto & replica : shard)
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
settings.distributed_connections_pool_size,
|
||||
jt->host_port.host().toString(), jt->host_port.port(), "", jt->user, jt->password, data_type_factory, "server", Protocol::Compression::Enable,
|
||||
replica.host_port.host().toString(), replica.host_port.port(), "", replica.user, replica.password,
|
||||
data_type_factory, "server", Protocol::Compression::Enable,
|
||||
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
|
||||
saturate(settings.receive_timeout, settings.limits.max_execution_time),
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
|
||||
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
||||
}
|
||||
}
|
||||
@ -259,9 +262,11 @@ bool Cluster::isLocal(const Address & address)
|
||||
interfaces.end() != std::find_if(interfaces.begin(), interfaces.end(),
|
||||
[&](const Poco::Net::NetworkInterface & interface) { return interface.address() == address.host_port.host(); }))
|
||||
{
|
||||
LOG_INFO(&Poco::Util::Application::instance().logger(), "Replica with address " << address.host_port.toString() << " will be processed as local.");
|
||||
LOG_INFO(&Poco::Util::Application::instance().logger(),
|
||||
"Replica with address " << address.host_port.toString() << " will be processed as local.");
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user