ClickHouse/dbms/include/DB/Interpreters/Cluster.h

96 lines
3.4 KiB
C
Raw Normal View History

2014-02-22 18:53:42 +00:00
#pragma once
2013-12-07 16:51:29 +00:00
#include <map>
#include <DB/Interpreters/Settings.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Client/ConnectionPool.h>
#include <DB/Client/ConnectionPoolWithFailover.h>
#include <Poco/Net/SocketAddress.h>
namespace DB
{
/// Cluster содержит пулы соединений до каждого из узлов
/// С локальными узлами соединение не устанавливается, а выполяется запрос напрямую.
/// Поэтому храним только количество локальных узлов
/// В конфиге кластер включает в себя узлы <node> или <shard>
class Cluster : private boost::noncopyable
2013-12-07 16:51:29 +00:00
{
public:
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, const String & cluster_name);
2013-12-07 16:51:29 +00:00
2015-01-13 00:56:43 +00:00
/// Построить кластер по именам шардов и реплик. Локальные обрабатываются так же как удаленные.
Cluster(const Settings & settings, const DataTypeFactory & data_type_factory, std::vector<std::vector<String>> names,
const String & username, const String & password);
2013-12-07 16:51:29 +00:00
/// количество узлов clickhouse сервера, расположенных локально
/// к локальным узлам обращаемся напрямую
size_t getLocalNodesNum() const { return local_nodes_num; }
/// Соединения с удалёнными серверами.
ConnectionPools pools;
struct ShardInfo
{
2014-08-21 12:07:29 +00:00
/// contains names of directories for asynchronous write to StorageDistributed
std::vector<std::string> dir_names;
int weight;
size_t num_local_nodes;
};
std::vector<ShardInfo> shard_info_vec;
std::vector<size_t> slot_to_shard;
2013-12-07 16:51:29 +00:00
/// используеться для выставления ограничения на размер таймаута
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
2013-12-07 16:51:29 +00:00
struct Address
{
/** В конфиге адреса либо находятся в узлах <node>:
* <node>
* <host>example01-01-1</host>
* <port>9000</port>
* <!-- <user>, <password>, если нужны -->
* </node>
* ...
* либо в узлах <shard>, и внутри - <replica>
* <shard>
* <replica>
* <host>example01-01-1</host>
* <port>9000</port>
* <!-- <user>, <password>, если нужны -->
2013-12-10 09:35:30 +00:00
* </replica>
2013-12-07 16:51:29 +00:00
* </shard>
*/
Poco::Net::SocketAddress host_port;
String user;
String password;
Address(const String & config_prefix);
Address(const String & host_port_, const String & user_, const String & password_);
2013-12-07 16:51:29 +00:00
};
private:
static bool isLocal(const Address & address);
2013-12-07 16:51:29 +00:00
/// Массив шардов. Каждый шард - адреса одного сервера.
typedef std::vector<Address> Addresses;
/// Массив шардов. Для каждого шарда - массив адресов реплик (серверов, считающихся идентичными).
typedef std::vector<Addresses> AddressesWithFailover;
Addresses addresses;
AddressesWithFailover addresses_with_failover;
2015-01-13 00:56:43 +00:00
size_t local_nodes_num = 0;
2013-12-07 16:51:29 +00:00
};
struct Clusters
2013-12-07 16:51:29 +00:00
{
typedef std::map<String, Cluster> Impl;
Impl impl;
2013-12-07 16:51:29 +00:00
Clusters(const Settings & settings, const DataTypeFactory & data_type_factory,
const String & config_name = "remote_servers");
2013-12-07 16:51:29 +00:00
};
2013-12-07 16:51:29 +00:00
}