save username and password and take internal_replication into account. [#METR-12221]

This commit is contained in:
Andrey Mironov 2014-08-14 14:38:06 +04:00
parent b801907980
commit 61af8baa77
3 changed files with 54 additions and 46 deletions

View File

@ -33,6 +33,7 @@ public:
{ {
std::string dir_name; std::string dir_name;
int weight; int weight;
bool internal_replication;
}; };
std::vector<ShardInfo> shard_info_vec; std::vector<ShardInfo> shard_info_vec;
std::vector<size_t> slot_to_shard; std::vector<size_t> slot_to_shard;
@ -40,7 +41,6 @@ public:
/// используеться для выставления ограничения на размер таймаута /// используеться для выставления ограничения на размер таймаута
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit); static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
private:
struct Address struct Address
{ {
/** В конфиге адреса либо находятся в узлах <node>: /** В конфиге адреса либо находятся в узлах <node>:
@ -64,12 +64,12 @@ private:
String password; String password;
Address(const String & config_prefix); Address(const String & config_prefix);
Address(const String & host, int port, const String & config_prefix);
Address(const String & host_port_, const String & user_, const String & password_); Address(const String & host_port_, const String & user_, const String & password_);
}; };
static bool isLocal(const Address & address); static bool isLocal(const Address & address);
// private:
/// Массив шардов. Каждый шард - адреса одного сервера. /// Массив шардов. Каждый шард - адреса одного сервера.
typedef std::vector<Address> Addresses; typedef std::vector<Address> Addresses;

View File

@ -10,18 +10,12 @@ namespace DB
Cluster::Address::Address(const String & config_prefix) Cluster::Address::Address(const String & config_prefix)
{ {
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config(); auto & config = Poco::Util::Application::instance().config();
host_port = Poco::Net::SocketAddress(config.getString(config_prefix + ".host"),
config.getInt(config_prefix + ".port"));
user = config.getString(config_prefix + ".user", "default"); host_port = Poco::Net::SocketAddress(
password = config.getString(config_prefix + ".password", ""); config.getString(config_prefix + ".host"),
} config.getInt(config_prefix + ".port")
);
Cluster::Address::Address(const String & host, const int port, const String & config_prefix)
: host_port(host, port)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
user = config.getString(config_prefix + ".user", "default"); user = config.getString(config_prefix + ".user", "default");
password = config.getString(config_prefix + ".password", ""); password = config.getString(config_prefix + ".password", "");
@ -39,6 +33,16 @@ Cluster::Address::Address(const String & host_port_, const String & user_, const
host_port = Poco::Net::SocketAddress(host_port_, default_port); host_port = Poco::Net::SocketAddress(host_port_, default_port);
} }
namespace
{
inline std::string addressToDirName(const Cluster::Address & address)
{
return
address.user + (address.password.empty() ? "" : (':' + address.password)) + '@' +
address.host_port.host().toString() + ':' + std::to_string(address.host_port.port());
}
}
Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_factory, const String & config_name) Clusters::Clusters(const Settings & settings, const DataTypeFactory & data_type_factory, const String & config_name)
{ {
@ -60,31 +64,32 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
Poco::Util::AbstractConfiguration::Keys config_keys; Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(cluster_name, config_keys); config.keys(cluster_name, config_keys);
String config_prefix = cluster_name + "."; const auto & config_prefix = cluster_name + ".";
for (auto it = config_keys.begin(); it != config_keys.end(); ++it) for (auto it = config_keys.begin(); it != config_keys.end(); ++it)
{ {
if (0 == strncmp(it->c_str(), "node", strlen("node"))) if (0 == strncmp(it->c_str(), "node", strlen("node")))
{ {
const auto & prefix = config_prefix + *it; const auto & prefix = config_prefix + *it;
const auto & host = config.getString(prefix + ".host");
const auto port = config.getInt(prefix + ".port");
const auto weight = config.getInt(prefix + ".weight", 1); const auto weight = config.getInt(prefix + ".weight", 1);
const auto internal_replication = config.getBool(prefix + ".internal_replication", false);
addresses.emplace_back(host, port, prefix); addresses.emplace_back(prefix);
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size()); slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
shard_info_vec.push_back({host + ':' + std::to_string(port), weight}); shard_info_vec.push_back({addressToDirName(addresses.back()), weight, internal_replication});
} }
else if (0 == strncmp(it->c_str(), "shard", strlen("shard"))) else if (0 == strncmp(it->c_str(), "shard", strlen("shard")))
{ {
Poco::Util::AbstractConfiguration::Keys replica_keys; Poco::Util::AbstractConfiguration::Keys replica_keys;
config.keys(config_prefix + *it, replica_keys); config.keys(config_prefix + *it, replica_keys);
addresses_with_failover.push_back(Addresses()); addresses_with_failover.emplace_back();
Addresses & replica_addresses = addresses_with_failover.back(); Addresses & replica_addresses = addresses_with_failover.back();
const auto & partial_prefix = config_prefix + *it + "."; const auto & partial_prefix = config_prefix + *it + ".";
const auto weight = config.getInt(partial_prefix + ".weight", 1); const auto weight = config.getInt(partial_prefix + ".weight", 1);
const auto internal_replication = config.getBool(partial_prefix + ".internal_replication", false);
std::string dir_name{}; std::string dir_name{};
auto first = true; auto first = true;
@ -95,12 +100,9 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
if (0 == strncmp(jt->c_str(), "replica", strlen("replica"))) if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
{ {
const auto & prefix = partial_prefix + *jt; replica_addresses.emplace_back(partial_prefix + *jt);
const auto & host = config.getString(prefix + ".host");
const auto port = config.getInt(prefix + ".port");
replica_addresses.emplace_back(host, port, prefix); dir_name += (first ? "" : ",") + addressToDirName(replica_addresses.back());
dir_name += (first ? "" : ",") + host + ':' + std::to_string(port);
if (first) first = false; if (first) first = false;
} }
@ -109,7 +111,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()); slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
shard_info_vec.push_back({dir_name, weight}); shard_info_vec.push_back({dir_name, weight, internal_replication});
} }
else else
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
@ -135,7 +137,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
} }
else else
{ {
replicas.push_back(new ConnectionPool( replicas.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size, 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, 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.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
@ -147,7 +149,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
if (has_local_replics) if (has_local_replics)
++local_nodes_num; ++local_nodes_num;
else else
pools.push_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries)); pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
} }
} }
else if (addresses.size()) else if (addresses.size())
@ -160,7 +162,7 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
} }
else else
{ {
pools.push_back(new ConnectionPool( pools.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size, 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, 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), saturate(settings.connect_timeout, settings.limits.max_execution_time),
@ -181,8 +183,8 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
{ {
Addresses current; Addresses current;
for (size_t j = 0; j < names[i].size(); ++j) for (size_t j = 0; j < names[i].size(); ++j)
current.push_back(Address(names[i][j], username, password)); current.emplace_back(names[i][j], username, password);
addresses_with_failover.push_back(current); addresses_with_failover.emplace_back(current);
} }
for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it) for (AddressesWithFailover::const_iterator it = addresses_with_failover.begin(); it != addresses_with_failover.end(); ++it)
@ -192,14 +194,14 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt) for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
{ {
replicas.push_back(new ConnectionPool( replicas.emplace_back(new ConnectionPool(
settings.distributed_connections_pool_size, 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, 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.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
saturate(settings.receive_timeout, settings.limits.max_execution_time), saturate(settings.receive_timeout, settings.limits.max_execution_time),
saturate(settings.send_timeout, settings.limits.max_execution_time))); saturate(settings.send_timeout, settings.limits.max_execution_time)));
} }
pools.push_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries)); pools.emplace_back(new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries));
} }
} }

View File

@ -93,12 +93,6 @@ StorageDistributed::StorageDistributed(
path(data_path_ + escapeForFileName(name) + '/') path(data_path_ + escapeForFileName(name) + '/')
{ {
std::cout << "table `" << name << "` in " << path << std::endl; std::cout << "table `" << name << "` in " << path << std::endl;
for (auto & shard_info : cluster.shard_info_vec) {
std::cout
<< "\twill write to " << path + shard_info.dir_name
<< " with weight " << shard_info.weight
<< std::endl;
}
createDirectoryMonitors(); createDirectoryMonitors();
} }
@ -262,27 +256,39 @@ void StorageDistributed::createDirectoryMonitor(const std::string & name)
void StorageDistributed::directoryMonitorFunc(const std::string & name) void StorageDistributed::directoryMonitorFunc(const std::string & name)
{ {
const auto path = this->path + name + '/'; const auto & path = this->path + name + '/';
std::cout << "created monitor for directory " << path << std::endl; std::cout << "created monitor for directory " << path << std::endl;
ConnectionPools pools; // ConnectionPools pools;
for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it) for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it)
{ {
const auto & address = boost::copy_range<std::string>(*it); const auto & address = boost::copy_range<std::string>(*it);
/// lookup end of hostname const auto user_pw_end = strchr(address.data(), '@');
const auto host_end = strchr(address.data(), ':'); const auto colon = strchr(address.data(), ':');
if (!user_pw_end || !colon)
throw Exception{"Shard address '" + address + "' does not match to 'user[:password]@host:port' pattern"};
const auto has_pw = colon < user_pw_end;
const auto host_end = has_pw ? colon : strchr(user_pw_end + 1, ':');
if (!host_end) if (!host_end)
throw Exception{"Shard address '" + address + "' does not contain port"}; throw Exception{"Shard address '" + address + "' does not contain port"};
const std::string host{address.data(), host_end}; const std::string user{address.data(), has_pw ? colon : user_pw_end};
const auto password = has_pw ? std::string{colon + 1, user_pw_end} : std::string{};
const std::string host{user_pw_end + 1, host_end};
const auto port = DB::parse<UInt16>(host_end + 1); const auto port = DB::parse<UInt16>(host_end + 1);
pools.emplace_back(new ConnectionPool(1, host, port, remote_database, "default", "", getName() + '_' + name)); // pools.emplace_back(new ConnectionPool(1, host, port, remote_database, "default", "", getName() + '_' + name));
std::cout << "\taddress " << host << " port " << port << std::endl; std::cout
<< "\taddress " << host
<< " port " << port
<< " user " << user
<< " password " << password
<< std::endl;
} }
auto pool = pools.size() == 1 ? pools[0] : new ConnectionPoolWithFailover(pools, DB::LoadBalancing::RANDOM); // auto pool = pools.size() == 1 ? pools[0] : new ConnectionPoolWithFailover(pools, DB::LoadBalancing::RANDOM);
while (!quit.load(std::memory_order_relaxed)) while (!quit.load(std::memory_order_relaxed))
{ {