ClickHouse/dbms/src/Interpreters/Cluster.cpp

286 lines
9.4 KiB
C++
Raw Normal View History

2013-12-07 16:51:29 +00:00
#include <DB/Interpreters/Cluster.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Common/isLocalAddress.h>
2013-12-07 16:51:29 +00:00
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/Application.h>
namespace DB
{
Cluster::Address::Address(const String & config_prefix)
2013-12-07 16:51:29 +00:00
{
auto & config = Poco::Util::Application::instance().config();
2013-12-07 16:51:29 +00:00
host_name = config.getString(config_prefix + ".host");
2015-05-28 21:41:28 +00:00
port = config.getInt(config_prefix + ".port");
resolved_address = Poco::Net::SocketAddress(host_name, port);
user = config.getString(config_prefix + ".user", "default");
password = config.getString(config_prefix + ".password", "");
}
Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_)
: user(user_), password(password_)
{
UInt16 default_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
/// Похоже на то, что строка host_port_ содержит порт. Если условие срабатывает - не обязательно значит, что порт есть (пример: [::]).
if (nullptr != strchr(host_port_.c_str(), ':') || !default_port)
2015-05-28 21:41:28 +00:00
{
resolved_address = Poco::Net::SocketAddress(host_port_);
host_name = host_port_.substr(0, host_port_.find(':'));
port = resolved_address.port();
}
else
2015-05-28 21:41:28 +00:00
{
resolved_address = Poco::Net::SocketAddress(host_port_, default_port);
host_name = host_port_;
port = default_port;
}
}
namespace
{
inline std::string addressToDirName(const Cluster::Address & address)
{
return
escapeForFileName(address.user) +
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
2015-05-28 21:41:28 +00:00
escapeForFileName(address.resolved_address.host().toString()) + ':' +
std::to_string(address.resolved_address.port());
}
}
Clusters::Clusters(const Settings & settings, const String & config_name)
2013-12-07 16:51:29 +00:00
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_name, config_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
impl.emplace(std::piecewise_construct,
std::forward_as_tuple(*it),
std::forward_as_tuple(settings, config_name + "." + *it));
2013-12-07 16:51:29 +00:00
}
Cluster::Cluster(const Settings & settings, const String & cluster_name)
2013-12-07 16:51:29 +00:00
{
/// Создать кластер.
2013-12-07 16:51:29 +00:00
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(cluster_name, config_keys);
const auto & config_prefix = cluster_name + ".";
2013-12-07 16:51:29 +00:00
UInt32 current_shard_num = 1;
2015-04-30 12:43:16 +00:00
for (auto it = config_keys.begin(); it != config_keys.end(); ++it)
2013-12-07 16:51:29 +00:00
{
if (0 == strncmp(it->c_str(), "node", strlen("node")))
{
const auto & prefix = config_prefix + *it;
const auto weight = config.getInt(prefix + ".weight", 1);
if (weight == 0)
continue;
addresses.emplace_back(prefix);
addresses.back().replica_num = 1;
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
if (const auto is_local = isLocal(addresses.back()))
2015-04-30 12:43:16 +00:00
shard_info_vec.push_back({{}, current_shard_num, weight, is_local});
else
2015-04-30 12:43:16 +00:00
shard_info_vec.push_back({{addressToDirName(addresses.back())}, current_shard_num, weight, is_local});
2013-12-07 16:51:29 +00:00
}
else if (0 == strncmp(it->c_str(), "shard", strlen("shard")))
{
Poco::Util::AbstractConfiguration::Keys replica_keys;
config.keys(config_prefix + *it, replica_keys);
addresses_with_failover.emplace_back();
2013-12-07 16:51:29 +00:00
Addresses & replica_addresses = addresses_with_failover.back();
UInt32 current_replica_num = 1;
2013-12-07 16:51:29 +00:00
const auto & partial_prefix = config_prefix + *it + ".";
const auto weight = config.getInt(partial_prefix + ".weight", 1);
if (weight == 0)
continue;
const auto internal_replication = config.getBool(partial_prefix + ".internal_replication", false);
/** in case of internal_replication we will be appending names to
* the first element of vector; otherwise we will just .emplace_back
*/
std::vector<std::string> dir_names{};
size_t num_local_nodes = 0;
auto first = true;
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
2013-12-07 16:51:29 +00:00
{
if (0 == strncmp(jt->data(), "weight", strlen("weight")) ||
0 == strncmp(jt->data(), "internal_replication", strlen("internal_replication")))
continue;
2013-12-07 16:51:29 +00:00
if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
{
replica_addresses.emplace_back(partial_prefix + *jt);
2015-04-30 12:43:16 +00:00
replica_addresses.back().replica_num = current_replica_num;
++current_replica_num;
if (isLocal(replica_addresses.back()))
{
++num_local_nodes;
}
else
{
if (internal_replication)
{
auto dir_name = addressToDirName(replica_addresses.back());
if (first)
dir_names.emplace_back(std::move(dir_name));
else
dir_names.front() += "," + dir_name;
}
else
dir_names.emplace_back(addressToDirName(replica_addresses.back()));
if (first) first = false;
}
}
2013-12-07 16:51:29 +00:00
else
throw Exception("Unknown element in config: " + *jt, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
2015-04-30 12:43:16 +00:00
shard_info_vec.push_back({std::move(dir_names), current_shard_num, weight, num_local_nodes});
2013-12-07 16:51:29 +00:00
}
else
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
2015-04-30 12:43:16 +00:00
++current_shard_num;
2013-12-07 16:51:29 +00:00
}
/// Создать соответствующие пулы соединений.
2013-12-07 16:51:29 +00:00
if (!addresses_with_failover.empty() && !addresses.empty())
2015-01-13 00:56:43 +00:00
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
2013-12-07 16:51:29 +00:00
if (!addresses_with_failover.empty())
2015-01-13 00:56:43 +00:00
{
for (const auto & shard : addresses_with_failover)
2013-12-07 16:51:29 +00:00
{
2015-01-13 00:56:43 +00:00
ConnectionPools replicas;
replicas.reserve(shard.size());
2013-12-07 16:51:29 +00:00
bool has_local_replica = false;
2015-01-13 00:56:43 +00:00
for (const auto & replica : shard)
2013-12-07 16:51:29 +00:00
{
2015-01-13 00:56:43 +00:00
if (isLocal(replica))
2013-12-07 16:51:29 +00:00
{
has_local_replica = true;
2015-10-12 14:53:16 +00:00
local_addresses.push_back(replica);
2015-10-16 15:00:50 +00:00
break;
2013-12-07 16:51:29 +00:00
}
else
{
2015-01-13 00:56:43 +00:00
replicas.emplace_back(new ConnectionPool(
2013-12-07 16:51:29 +00:00
settings.distributed_connections_pool_size,
2015-05-28 21:41:28 +00:00
replica.host_name, replica.port, replica.resolved_address,
"", replica.user, replica.password,
"server", Protocol::Compression::Enable,
2015-01-13 00:56:43 +00:00
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)));
2013-12-07 16:51:29 +00:00
}
}
2015-01-13 00:56:43 +00:00
if (has_local_replica)
2015-01-13 00:56:43 +00:00
++local_nodes_num;
2015-10-16 15:00:50 +00:00
else
2015-01-13 00:56:43 +00:00
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
}
}
else if (!addresses.empty())
2015-01-13 00:56:43 +00:00
{
for (const auto & address : addresses)
{
if (isLocal(address))
{
2015-10-12 14:53:16 +00:00
local_addresses.push_back(address);
2015-01-13 00:56:43 +00:00
++local_nodes_num;
}
else
{
pools.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size,
2015-05-28 21:41:28 +00:00
address.host_name, address.port, address.resolved_address,
"", address.user, address.password,
"server", Protocol::Compression::Enable,
2015-01-13 00:56:43 +00:00
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)));
}
2013-12-07 16:51:29 +00:00
}
2015-01-13 00:56:43 +00:00
}
else
throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
2013-12-07 16:51:29 +00:00
}
Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> names,
2015-01-13 00:56:43 +00:00
const String & username, const String & password)
{
2015-01-13 00:56:43 +00:00
for (const auto & shard : names)
{
Addresses current;
2015-01-13 00:56:43 +00:00
for (auto & replica : shard)
current.emplace_back(replica, username, password);
addresses_with_failover.emplace_back(current);
}
2015-01-13 00:56:43 +00:00
for (const auto & shard : addresses_with_failover)
{
ConnectionPools replicas;
2015-01-13 00:56:43 +00:00
replicas.reserve(shard.size());
2015-01-13 00:56:43 +00:00
for (const auto & replica : shard)
{
replicas.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size,
2015-05-28 21:41:28 +00:00
replica.host_name, replica.port, replica.resolved_address,
"", replica.user, replica.password,
"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)));
}
2015-01-13 00:56:43 +00:00
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
}
}
Poco::Timespan Cluster::saturate(const Poco::Timespan & v, const Poco::Timespan & limit)
2013-12-07 16:51:29 +00:00
{
if (limit.totalMicroseconds() == 0)
return v;
else
return v > limit ? limit : v;
}
bool Cluster::isLocal(const Address & address)
2013-12-07 16:51:29 +00:00
{
/// Если среди реплик существует такая, что:
/// - её порт совпадает с портом, который слушает сервер;
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
2015-05-28 21:41:28 +00:00
return isLocalAddress(address.resolved_address);
2013-12-07 16:51:29 +00:00
}
}