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>
|
2015-02-10 13:22:13 +00:00
|
|
|
|
#include <DB/Common/isLocalAddress.h>
|
2015-10-16 16:10:10 +00:00
|
|
|
|
#include <DB/Common/SimpleCache.h>
|
2013-12-07 16:51:29 +00:00
|
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
|
|
|
|
#include <Poco/Util/Application.h>
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
|
|
|
|
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
|
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
|
extern const int SHARD_HAS_NO_CONNECTIONS;
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
namespace
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
/// Вес шарда по-умолчанию.
|
|
|
|
|
static constexpr int default_weight = 1;
|
|
|
|
|
|
|
|
|
|
inline bool isLocal(const Cluster::Address & address)
|
|
|
|
|
{
|
|
|
|
|
/// Если среди реплик существует такая, что:
|
|
|
|
|
/// - её порт совпадает с портом, который слушает сервер;
|
|
|
|
|
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
|
|
|
|
|
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
|
|
|
|
|
return isLocalAddress(address.resolved_address);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
inline std::string addressToDirName(const Cluster::Address & address)
|
|
|
|
|
{
|
|
|
|
|
return
|
|
|
|
|
escapeForFileName(address.user) +
|
|
|
|
|
(address.password.empty() ? "" : (':' + escapeForFileName(address.password))) + '@' +
|
|
|
|
|
escapeForFileName(address.resolved_address.host().toString()) + ':' +
|
|
|
|
|
std::to_string(address.resolved_address.port());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
inline bool beginsWith(const std::string & str1, const char * str2)
|
|
|
|
|
{
|
|
|
|
|
if (str2 == nullptr)
|
|
|
|
|
throw Exception("Passed null pointer to function beginsWith", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
return 0 == strncmp(str1.data(), str2, strlen(str2));
|
|
|
|
|
}
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2015-10-16 16:10:10 +00:00
|
|
|
|
/// Для кэширования DNS запросов.
|
2015-10-20 14:59:29 +00:00
|
|
|
|
Poco::Net::SocketAddress resolveSocketAddressImpl1(const String & host, UInt16 port)
|
2015-10-16 16:10:10 +00:00
|
|
|
|
{
|
|
|
|
|
return Poco::Net::SocketAddress(host, port);
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
Poco::Net::SocketAddress resolveSocketAddressImpl2(const String & host_and_port)
|
2015-10-16 16:10:10 +00:00
|
|
|
|
{
|
|
|
|
|
return Poco::Net::SocketAddress(host_and_port);
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
Poco::Net::SocketAddress resolveSocketAddress(const String & host, UInt16 port)
|
2015-10-16 16:10:10 +00:00
|
|
|
|
{
|
|
|
|
|
static SimpleCache<decltype(resolveSocketAddressImpl1), &resolveSocketAddressImpl1> cache;
|
|
|
|
|
return cache(host, port);
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
Poco::Net::SocketAddress resolveSocketAddress(const String & host_and_port)
|
2015-10-16 16:10:10 +00:00
|
|
|
|
{
|
|
|
|
|
static SimpleCache<decltype(resolveSocketAddressImpl2), &resolveSocketAddressImpl2> cache;
|
|
|
|
|
return cache(host_and_port);
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Реализация класса Cluster::Address
|
2015-10-16 16:10:10 +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
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
const auto & config = Poco::Util::Application::instance().config();
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2015-05-05 10:13:45 +00:00
|
|
|
|
host_name = config.getString(config_prefix + ".host");
|
2015-05-28 21:41:28 +00:00
|
|
|
|
port = config.getInt(config_prefix + ".port");
|
2015-10-16 16:10:10 +00:00
|
|
|
|
resolved_address = resolveSocketAddress(host_name, 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
|
|
|
|
|
2015-10-16 16:10:10 +00:00
|
|
|
|
|
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_ содержит порт. Если условие срабатывает - не обязательно значит, что порт есть (пример: [::]).
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if ((nullptr != strchr(host_port_.c_str(), ':')) || !default_port)
|
2015-05-28 21:41:28 +00:00
|
|
|
|
{
|
2015-10-16 16:10:10 +00:00
|
|
|
|
resolved_address = resolveSocketAddress(host_port_);
|
2015-05-28 21:41:28 +00:00
|
|
|
|
host_name = host_port_.substr(0, host_port_.find(':'));
|
|
|
|
|
port = resolved_address.port();
|
|
|
|
|
}
|
2014-02-22 21:50:27 +00:00
|
|
|
|
else
|
2015-05-28 21:41:28 +00:00
|
|
|
|
{
|
2015-10-16 16:10:10 +00:00
|
|
|
|
resolved_address = resolveSocketAddress(host_port_, default_port);
|
2015-05-28 21:41:28 +00:00
|
|
|
|
host_name = host_port_;
|
|
|
|
|
port = default_port;
|
|
|
|
|
}
|
2014-02-22 21:50:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
/// Реализация класса Clusters
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2015-05-28 03:49:28 +00:00
|
|
|
|
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);
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
for (const auto & key : config_keys)
|
2014-02-22 21:50:27 +00:00
|
|
|
|
impl.emplace(std::piecewise_construct,
|
2015-10-20 14:59:29 +00:00
|
|
|
|
std::forward_as_tuple(key),
|
|
|
|
|
std::forward_as_tuple(settings, config_name + "." + key));
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
/// Реализация класса Cluster
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2015-05-28 03:49:28 +00:00
|
|
|
|
Cluster::Cluster(const Settings & settings, const String & cluster_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(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
|
|
|
|
|
2015-05-05 11:53:09 +00:00
|
|
|
|
UInt32 current_shard_num = 1;
|
2015-04-30 12:43:16 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
for (const auto & key : config_keys)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if (beginsWith(key, "node"))
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
/// Шард без реплик.
|
|
|
|
|
|
|
|
|
|
const auto & prefix = config_prefix + key;
|
|
|
|
|
const auto weight = config.getInt(prefix + ".weight", default_weight);
|
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);
|
2015-05-05 11:53:09 +00:00
|
|
|
|
addresses.back().replica_num = 1;
|
2015-10-20 14:59:29 +00:00
|
|
|
|
const auto & address = addresses.back();
|
|
|
|
|
|
|
|
|
|
ShardInfo info;
|
|
|
|
|
info.shard_num = current_shard_num;
|
|
|
|
|
info.weight = weight;
|
2014-08-15 09:50:05 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if (isLocal(address))
|
|
|
|
|
info.local_addresses.push_back(address);
|
2014-08-15 09:50:05 +00:00
|
|
|
|
else
|
2015-10-20 14:59:29 +00:00
|
|
|
|
{
|
|
|
|
|
info.dir_names.push_back(addressToDirName(address));
|
|
|
|
|
info.pool = new ConnectionPool(
|
|
|
|
|
settings.distributed_connections_pool_size,
|
|
|
|
|
address.host_name, address.port, address.resolved_address,
|
|
|
|
|
"", address.user, address.password,
|
|
|
|
|
"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));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
|
|
|
|
shards_info.push_back(info);
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
2015-10-20 14:59:29 +00:00
|
|
|
|
else if (beginsWith(key, "shard"))
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
/// Шард с репликами.
|
|
|
|
|
|
2013-12-07 16:51:29 +00:00
|
|
|
|
Poco::Util::AbstractConfiguration::Keys replica_keys;
|
2015-10-20 14:59:29 +00:00
|
|
|
|
config.keys(config_prefix + key, replica_keys);
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
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();
|
2015-05-05 11:53:09 +00:00
|
|
|
|
UInt32 current_replica_num = 1;
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
const auto & partial_prefix = config_prefix + key + ".";
|
|
|
|
|
const auto weight = config.getInt(partial_prefix + ".weight", default_weight);
|
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-12 13:46:46 +00:00
|
|
|
|
|
2014-08-13 15:07:53 +00:00
|
|
|
|
auto first = true;
|
2015-10-20 14:59:29 +00:00
|
|
|
|
for (const auto & replica_key : replica_keys)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if (beginsWith(replica_key, "weight") || beginsWith(replica_key, "internal_replication"))
|
2014-08-12 13:46:46 +00:00
|
|
|
|
continue;
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if (beginsWith(replica_key, "replica"))
|
2014-08-12 13:46:46 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
replica_addresses.emplace_back(partial_prefix + replica_key);
|
2015-04-30 12:43:16 +00:00
|
|
|
|
replica_addresses.back().replica_num = current_replica_num;
|
|
|
|
|
++current_replica_num;
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
if (!isLocal(replica_addresses.back()))
|
2014-08-15 09:50:05 +00:00
|
|
|
|
{
|
|
|
|
|
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
|
2015-10-20 14:59:29 +00:00
|
|
|
|
throw Exception("Unknown element in config: " + replica_key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
2014-08-12 13:46:46 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
Addresses shard_local_addresses;
|
2013-12-07 16:51:29 +00:00
|
|
|
|
|
2015-01-13 00:56:43 +00:00
|
|
|
|
ConnectionPools replicas;
|
2015-10-20 14:59:29 +00:00
|
|
|
|
replicas.reserve(replica_addresses.size());
|
2015-10-16 11:02:26 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
for (const auto & replica : replica_addresses)
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-01-13 00:56:43 +00:00
|
|
|
|
if (isLocal(replica))
|
2015-10-20 14:59:29 +00:00
|
|
|
|
shard_local_addresses.push_back(replica);
|
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,
|
2015-05-28 03:49:28 +00:00
|
|
|
|
"server", Protocol::Compression::Enable,
|
2015-01-13 00:56:43 +00:00
|
|
|
|
saturate(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
|
2014-02-22 21:50:27 +00:00
|
|
|
|
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
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
ConnectionPoolPtr shard_pool;
|
|
|
|
|
if (!replicas.empty())
|
|
|
|
|
shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
|
|
|
|
|
|
|
|
|
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
|
|
|
|
shards_info.push_back({std::move(dir_names), current_shard_num, weight, shard_local_addresses, shard_pool});
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
2015-10-20 14:59:29 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown element in config: " + key, 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);
|
|
|
|
|
|
|
|
|
|
++current_shard_num;
|
2015-01-13 00:56:43 +00:00
|
|
|
|
}
|
2015-10-20 14:59:29 +00:00
|
|
|
|
|
|
|
|
|
initMisc();
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2015-05-28 03:49:28 +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)
|
2014-02-07 15:11:57 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
UInt32 current_shard_num = 1;
|
|
|
|
|
|
2015-01-13 00:56:43 +00:00
|
|
|
|
for (const auto & shard : names)
|
2014-02-07 15:11:57 +00:00
|
|
|
|
{
|
|
|
|
|
Addresses current;
|
2015-01-13 00:56:43 +00:00
|
|
|
|
for (auto & replica : shard)
|
|
|
|
|
current.emplace_back(replica, username, password);
|
2015-10-16 16:10:10 +00:00
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
|
addresses_with_failover.emplace_back(current);
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2014-02-07 15:11:57 +00:00
|
|
|
|
ConnectionPools replicas;
|
2015-10-16 16:10:10 +00:00
|
|
|
|
replicas.reserve(current.size());
|
2014-02-07 15:11:57 +00:00
|
|
|
|
|
2015-10-16 16:10:10 +00:00
|
|
|
|
for (const auto & replica : current)
|
2014-02-07 15:11:57 +00:00
|
|
|
|
{
|
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,
|
2015-05-28 21:41:28 +00:00
|
|
|
|
replica.host_name, replica.port, replica.resolved_address,
|
|
|
|
|
"", replica.user, replica.password,
|
2015-05-28 03:49:28 +00:00
|
|
|
|
"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
|
|
|
|
}
|
2015-01-13 00:56:43 +00:00
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
ConnectionPoolPtr shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
|
|
|
|
|
|
|
|
|
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
|
|
|
|
|
shards_info.push_back({{}, current_shard_num, default_weight, {}, shard_pool});
|
|
|
|
|
++current_shard_num;
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
2015-10-20 14:59:29 +00:00
|
|
|
|
|
|
|
|
|
initMisc();
|
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
|
2015-10-20 14:59:29 +00:00
|
|
|
|
return (v > limit) ? limit : v;
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-10-20 14:59:29 +00:00
|
|
|
|
void Cluster::initMisc()
|
2013-12-07 16:51:29 +00:00
|
|
|
|
{
|
2015-10-20 14:59:29 +00:00
|
|
|
|
for (const auto & shard_info : shards_info)
|
|
|
|
|
{
|
|
|
|
|
if (!shard_info.isLocal() && !shard_info.hasRemoteConnections())
|
|
|
|
|
throw Exception("Found shard without any specified connection",
|
|
|
|
|
ErrorCodes::SHARD_HAS_NO_CONNECTIONS);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
for (const auto & shard_info : shards_info)
|
|
|
|
|
{
|
|
|
|
|
if (shard_info.isLocal())
|
|
|
|
|
++local_shard_count;
|
|
|
|
|
else
|
|
|
|
|
++remote_shard_count;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
for (auto & shard_info : shards_info)
|
|
|
|
|
{
|
|
|
|
|
if (!shard_info.isLocal())
|
|
|
|
|
{
|
|
|
|
|
any_remote_shard_info = &shard_info;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-12-07 16:51:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|