2013-12-07 16:51:29 +00:00
|
|
|
|
#include <DB/Interpreters/Cluster.h>
|
2014-08-14 11:50:36 +00:00
|
|
|
|
#include <DB/Common/escapeForFileName.h>
|
2013-12-07 16:51:29 +00:00
|
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
|
|
|
|
#include <Poco/Util/Application.h>
|
|
|
|
|
#include <Poco/Net/NetworkInterface.h>
|
|
|
|
|
#include <boost/bind.hpp>
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
Cluster::Address::Address(const String & config_prefix)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
|
auto & config = Poco::Util::Application::instance().config();
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
host_port = Poco::Net::SocketAddress(
|
|
|
|
|
config.getString(config_prefix + ".host"),
|
|
|
|
|
config.getInt(config_prefix + ".port")
|
|
|
|
|
);
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
|
|
|
|
user = config.getString(config_prefix + ".user", "default");
|
|
|
|
|
password = config.getString(config_prefix + ".password", "");
|
|
|
|
|
}
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
host_port = Poco::Net::SocketAddress(host_port_);
|
|
|
|
|
else
|
|
|
|
|
host_port = Poco::Net::SocketAddress(host_port_, default_port);
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
namespace
|
|
|
|
|
{
|
|
|
|
|
inline std::string addressToDirName(const Cluster::Address & address)
|
|
|
|
|
{
|
|
|
|
|
return
|
2014-08-14 11:50:36 +00:00
|
|
|
|
escapeForFileName(address.user) +
|
|
|
|
|
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
|
|
|
|
|
escapeForFileName(address.host_port.host().toString()) + ':' +
|
|
|
|
|
std::to_string(address.host_port.port());
|
2014-08-14 10:38:06 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
|
|
|
|
Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_factory, 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)
|
2014-02-22 21:50:27 +00:00
|
|
|
|
impl.emplace(std::piecewise_construct,
|
|
|
|
|
std::forward_as_tuple(*it),
|
|
|
|
|
std::forward_as_tuple(settings, data_type_factory, config_name + "." + *it));
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name):
|
2013-12-07 16:51:29 +00:00
|
|
|
|
local_nodes_num(0)
|
|
|
|
|
{
|
|
|
|
|
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
|
|
|
|
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
|
|
|
config.keys(cluster_name, config_keys);
|
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
const auto & config_prefix = cluster_name + ".";
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2014-08-12 13:46:46 +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")))
|
|
|
|
|
{
|
2014-08-12 13:46:46 +00:00
|
|
|
|
const auto & prefix = config_prefix + *it;
|
|
|
|
|
const auto weight = config.getInt(prefix + ".weight", 1);
|
2014-08-14 11:50:36 +00:00
|
|
|
|
if (weight == 0)
|
|
|
|
|
continue;
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
addresses.emplace_back(prefix);
|
2014-08-15 09:50:05 +00:00
|
|
|
|
|
2014-08-12 13:46:46 +00:00
|
|
|
|
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (const auto is_local = isLocal(addresses.back()))
|
|
|
|
|
shard_info_vec.push_back({{}, weight, is_local });
|
|
|
|
|
else
|
|
|
|
|
shard_info_vec.push_back({{addressToDirName(addresses.back())}, 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);
|
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
addresses_with_failover.emplace_back();
|
2013-12-07 16:51:29 +00:00
|
|
|
|
Addresses & replica_addresses = addresses_with_failover.back();
|
|
|
|
|
|
2014-08-12 13:46:46 +00:00
|
|
|
|
const auto & partial_prefix = config_prefix + *it + ".";
|
|
|
|
|
const auto weight = config.getInt(partial_prefix + ".weight", 1);
|
2014-08-14 11:50:36 +00:00
|
|
|
|
if (weight == 0)
|
|
|
|
|
continue;
|
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
const auto internal_replication = config.getBool(partial_prefix + ".internal_replication", false);
|
2014-08-14 10:38:06 +00:00
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
/** in case of internal_replication we will be appending names to
|
2014-08-15 12:07:50 +00:00
|
|
|
|
* the first element of vector; otherwise we will just .emplace_back
|
2014-08-15 09:50:05 +00:00
|
|
|
|
*/
|
2014-08-15 12:07:50 +00:00
|
|
|
|
std::vector<std::string> dir_names{};
|
2014-08-15 09:50:05 +00:00
|
|
|
|
auto has_local_node = false;
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2014-08-13 15:07:53 +00:00
|
|
|
|
auto first = true;
|
2014-08-12 13:46:46 +00:00
|
|
|
|
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2014-08-14 11:50:36 +00:00
|
|
|
|
if (0 == strncmp(jt->data(), "weight", strlen("weight")) ||
|
|
|
|
|
0 == strncmp(jt->data(), "internal_replication", strlen("internal_replication")))
|
2014-08-12 13:46:46 +00:00
|
|
|
|
continue;
|
|
|
|
|
|
2013-12-07 16:51:29 +00:00
|
|
|
|
if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
|
2014-08-12 13:46:46 +00:00
|
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
|
replica_addresses.emplace_back(partial_prefix + *jt);
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (isLocal(replica_addresses.back()))
|
|
|
|
|
{
|
|
|
|
|
has_local_node = true;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
if (internal_replication)
|
2014-08-15 12:07:50 +00:00
|
|
|
|
{
|
|
|
|
|
auto dir_name = addressToDirName(replica_addresses.back());
|
|
|
|
|
if (first)
|
|
|
|
|
dir_names.emplace_back(std::move(dir_name));
|
|
|
|
|
else
|
|
|
|
|
dir_names.front() += "," + dir_name;
|
|
|
|
|
}
|
2014-08-15 09:50:05 +00:00
|
|
|
|
else
|
|
|
|
|
dir_names.emplace_back(addressToDirName(replica_addresses.back()));
|
2014-08-13 15:07:53 +00:00
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (first) first = false;
|
|
|
|
|
}
|
2014-08-12 13:46:46 +00:00
|
|
|
|
}
|
2013-12-07 16:51:29 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown element in config: " + *jt, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
|
|
|
|
}
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
|
|
|
|
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
|
2014-08-15 09:50:05 +00:00
|
|
|
|
shard_info_vec.push_back({std::move(dir_names), weight, has_local_node});
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!addresses_with_failover.empty() && !addresses.empty())
|
|
|
|
|
throw Exception("There must be either 'node' or 'shard' elements in config", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
|
|
|
|
|
|
|
|
|
if (addresses_with_failover.size())
|
|
|
|
|
{
|
|
|
|
|
for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
ConnectionPools replicas;
|
|
|
|
|
replicas.reserve(it->size());
|
|
|
|
|
|
|
|
|
|
bool has_local_replics = false;
|
|
|
|
|
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
|
|
|
|
|
{
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (isLocal(*jt))
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
|
|
|
|
has_local_replics = true;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
|
replicas.emplace_back(new ConnectionPool(
|
2013-12-07 16:51:29 +00:00
|
|
|
|
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,
|
2014-02-22 21:50:27 +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
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (has_local_replics)
|
|
|
|
|
++local_nodes_num;
|
|
|
|
|
else
|
2014-08-14 10:38:06 +00:00
|
|
|
|
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else if (addresses.size())
|
|
|
|
|
{
|
|
|
|
|
for (Addresses::const_iterator it = addresses.begin(); it != addresses.end(); ++it)
|
|
|
|
|
{
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (isLocal(*it))
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
|
|
|
|
++local_nodes_num;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
|
pools.emplace_back(new ConnectionPool(
|
2013-12-07 16:51:29 +00:00
|
|
|
|
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,
|
2014-02-22 21:50:27 +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
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("No addresses listed in config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2014-02-11 17:11:57 +00:00
|
|
|
|
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)
|
2014-02-07 15:11:57 +00:00
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < names.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
Addresses current;
|
|
|
|
|
for (size_t j = 0; j < names[i].size(); ++j)
|
2014-08-14 10:38:06 +00:00
|
|
|
|
current.emplace_back(names[i][j], username, password);
|
|
|
|
|
addresses_with_failover.emplace_back(current);
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2014-02-07 15:11:57 +00:00
|
|
|
|
for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
ConnectionPools replicas;
|
|
|
|
|
replicas.reserve(it->size());
|
|
|
|
|
|
|
|
|
|
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
|
|
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
|
replicas.emplace_back(new ConnectionPool(
|
2014-02-07 15:11:57 +00:00
|
|
|
|
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,
|
2014-02-22 21:50:27 +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)));
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
2014-08-14 10:38:06 +00:00
|
|
|
|
pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
bool Cluster::isLocal(const Address & address)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
|
|
|
|
/// Если среди реплик существует такая, что:
|
|
|
|
|
/// - её порт совпадает с портом, который слушает сервер;
|
|
|
|
|
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
|
|
|
|
|
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
|
2014-08-14 11:50:36 +00:00
|
|
|
|
const UInt16 clickhouse_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
|
|
|
|
|
static auto interfaces = Poco::Net::NetworkInterface::list();
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (clickhouse_port == address.host_port.port() &&
|
2013-12-07 16:51:29 +00:00
|
|
|
|
interfaces.end() != std::find_if(interfaces.begin(), interfaces.end(),
|
2014-08-15 09:50:05 +00:00
|
|
|
|
[&](const Poco::Net::NetworkInterface & interface) { return interface.address() == address.host_port.host(); }))
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2014-08-15 09:50:05 +00:00
|
|
|
|
LOG_INFO(&Poco::Util::Application::instance().logger(), "Replica with address " << address.host_port.toString() << " will be processed as local.");
|
2013-12-07 16:51:29 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|