2012-05-21 20:38:34 +00:00
|
|
|
|
#include <DB/Parsers/formatAST.h>
|
|
|
|
|
|
|
|
|
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
|
|
|
|
|
|
|
|
|
#include <DB/Storages/StorageDistributed.h>
|
|
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
|
#include <Poco/Net/NetworkInterface.h>
|
|
|
|
|
|
|
|
|
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
|
|
|
|
#include <boost/bind.hpp>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2013-12-05 17:39:33 +00:00
|
|
|
|
|
|
|
|
|
static Poco::Timespan saturation(const Poco::Timespan & v, const Poco::Timespan & limit)
|
2013-12-05 17:24:05 +00:00
|
|
|
|
{
|
2013-12-05 17:39:33 +00:00
|
|
|
|
if (limit.totalMicroseconds() == 0)
|
|
|
|
|
return v;
|
|
|
|
|
else
|
|
|
|
|
return v > limit ? limit : v;
|
2013-12-05 17:24:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2012-11-06 17:04:38 +00:00
|
|
|
|
const StorageDistributed::Addresses & addresses,
|
2012-05-21 20:38:34 +00:00
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2012-08-02 17:33:31 +00:00
|
|
|
|
const DataTypeFactory & data_type_factory_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const Settings & settings,
|
2013-09-23 12:01:19 +00:00
|
|
|
|
const Context & context_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const String & sign_column_name_)
|
2012-11-06 17:04:38 +00:00
|
|
|
|
: name(name_), columns(columns_),
|
2012-05-21 20:38:34 +00:00
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
2013-05-08 10:30:29 +00:00
|
|
|
|
data_type_factory(data_type_factory_),
|
2013-09-23 12:01:19 +00:00
|
|
|
|
sign_column_name(sign_column_name_),
|
2013-11-28 10:31:17 +00:00
|
|
|
|
context(context_),
|
|
|
|
|
local_replics_num(0)
|
2012-05-21 20:38:34 +00:00
|
|
|
|
{
|
2013-12-05 17:24:05 +00:00
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
for (Addresses::const_iterator it = addresses.begin(); it != addresses.end(); ++it)
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
|
|
|
|
if (checkLocalReplics(*it))
|
|
|
|
|
{
|
|
|
|
|
++local_replics_num;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
pools.push_back(new ConnectionPool(
|
|
|
|
|
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,
|
2013-12-05 17:24:05 +00:00
|
|
|
|
saturation(settings.connect_timeout, settings.limits.max_execution_time),
|
|
|
|
|
saturation(settings.receive_timeout, settings.limits.max_execution_time),
|
|
|
|
|
saturation(settings.send_timeout, settings.limits.max_execution_time)));
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2012-05-21 20:38:34 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-11-06 17:04:38 +00:00
|
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const StorageDistributed::AddressesWithFailover & addresses,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
|
|
|
|
const DataTypeFactory & data_type_factory_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const Settings & settings,
|
2013-09-23 12:01:19 +00:00
|
|
|
|
const Context & context_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const String & sign_column_name_)
|
2012-11-06 17:04:38 +00:00
|
|
|
|
: name(name_), columns(columns_),
|
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
2013-05-08 10:30:29 +00:00
|
|
|
|
data_type_factory(data_type_factory_),
|
2013-09-23 12:01:19 +00:00
|
|
|
|
sign_column_name(sign_column_name_),
|
2013-11-28 10:31:17 +00:00
|
|
|
|
context(context_),
|
|
|
|
|
local_replics_num(0)
|
2012-11-06 17:04:38 +00:00
|
|
|
|
{
|
|
|
|
|
for (AddressesWithFailover::const_iterator it = addresses.begin(); it != addresses.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
ConnectionPools replicas;
|
|
|
|
|
replicas.reserve(it->size());
|
|
|
|
|
|
2013-11-28 10:31:21 +00:00
|
|
|
|
bool has_local_replics = false;
|
2012-11-06 17:04:38 +00:00
|
|
|
|
for (Addresses::const_iterator jt = it->begin(); jt != it->end(); ++jt)
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
|
|
|
|
if (checkLocalReplics(*jt))
|
|
|
|
|
{
|
2013-11-28 10:31:21 +00:00
|
|
|
|
has_local_replics = true;
|
|
|
|
|
break;
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
replicas.push_back(new ConnectionPool(
|
|
|
|
|
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,
|
2013-12-05 17:24:05 +00:00
|
|
|
|
saturation(settings.connect_timeout_with_failover_ms, settings.limits.max_execution_time),
|
|
|
|
|
saturation(settings.receive_timeout, settings.limits.max_execution_time),
|
|
|
|
|
saturation(settings.send_timeout, settings.limits.max_execution_time)));
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2012-11-06 17:04:38 +00:00
|
|
|
|
|
2013-11-28 10:31:21 +00:00
|
|
|
|
if (has_local_replics)
|
|
|
|
|
++local_replics_num;
|
|
|
|
|
else
|
|
|
|
|
pools.push_back(new ConnectionPoolWithFailover(replicas, settings.connections_with_failover_max_tries));
|
2012-11-06 17:04:38 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
|
static bool interfaceEqual(const Poco::Net::NetworkInterface & interface, Poco::Net::IPAddress & address)
|
|
|
|
|
{
|
|
|
|
|
return interface.address() == address;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool StorageDistributed::checkLocalReplics(const Address & address)
|
|
|
|
|
{
|
|
|
|
|
/// Если среди реплик существует такая, что:
|
|
|
|
|
/// - её порт совпадает с портом, который слушает сервер;
|
|
|
|
|
/// - её хост резолвится в набор адресов, один из которых совпадает с одним из адресов сетевых интерфейсов сервера
|
|
|
|
|
/// то нужно всегда ходить на этот шард без межпроцессного взаимодействия
|
|
|
|
|
UInt16 clickhouse_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
|
|
|
|
|
Poco::Net::NetworkInterface::NetworkInterfaceList interfaces = Poco::Net::NetworkInterface::list();
|
|
|
|
|
|
|
|
|
|
if (clickhouse_port == address.host_port.port() &&
|
|
|
|
|
interfaces.end() != std::find_if(interfaces.begin(), interfaces.end(),
|
|
|
|
|
boost::bind(interfaceEqual, _1, address.host_port.host())))
|
|
|
|
|
{
|
|
|
|
|
LOG_INFO(&Poco::Util::Application::instance().logger(), "Replica with address " << address.host_port.toString() << " will be processed as local.");
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const StorageDistributed::Addresses & addresses,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
|
|
|
|
const DataTypeFactory & data_type_factory_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const Settings & settings,
|
2013-09-23 12:01:19 +00:00
|
|
|
|
const Context & context_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const String & sign_column_name_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
2013-09-23 12:01:19 +00:00
|
|
|
|
return (new StorageDistributed(name_, columns_, addresses, remote_database_, remote_table_, data_type_factory_, settings, context_, sign_column_name_))->thisPtr();
|
2013-02-06 11:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const StorageDistributed::AddressesWithFailover & addresses,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
|
|
|
|
const DataTypeFactory & data_type_factory_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const Settings & settings,
|
2013-09-23 12:01:19 +00:00
|
|
|
|
const Context & context_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const String & sign_column_name_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
2013-09-23 12:01:19 +00:00
|
|
|
|
return (new StorageDistributed(name_, columns_, addresses, remote_database_, remote_table_, data_type_factory_, settings, context_, sign_column_name_))->thisPtr();
|
2013-02-06 11:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageDistributed::read(
|
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
2013-02-01 19:02:04 +00:00
|
|
|
|
const Settings & settings,
|
2012-05-22 18:32:45 +00:00
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2012-05-21 20:38:34 +00:00
|
|
|
|
size_t max_block_size,
|
2012-05-30 04:45:49 +00:00
|
|
|
|
unsigned threads)
|
2012-05-21 20:38:34 +00:00
|
|
|
|
{
|
2013-11-28 10:31:21 +00:00
|
|
|
|
processed_stage = (pools.size() + local_replics_num) == 1
|
2012-05-22 20:05:43 +00:00
|
|
|
|
? QueryProcessingStage::Complete
|
|
|
|
|
: QueryProcessingStage::WithMergeableState;
|
2013-11-28 10:31:17 +00:00
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
/// Заменим в запросе имена БД и таблицы.
|
|
|
|
|
ASTPtr modified_query_ast = query->clone();
|
|
|
|
|
ASTSelectQuery & select = dynamic_cast<ASTSelectQuery &>(*modified_query_ast);
|
|
|
|
|
select.database = new ASTIdentifier(StringRange(), remote_database, ASTIdentifier::Database);
|
|
|
|
|
select.table = new ASTIdentifier(StringRange(), remote_table, ASTIdentifier::Table);
|
2013-11-28 10:31:17 +00:00
|
|
|
|
|
2013-05-08 10:30:29 +00:00
|
|
|
|
/// Установим sign_rewrite = 0, чтобы второй раз не переписывать запрос
|
|
|
|
|
Settings new_settings = settings;
|
|
|
|
|
new_settings.sign_rewrite = false;
|
2013-12-05 17:24:05 +00:00
|
|
|
|
new_settings.queue_max_wait_ms = saturation(new_settings.queue_max_wait_ms, settings.limits.max_execution_time);
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
|
|
|
|
std::stringstream s;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
formatAST(select, s, 0, false, true);
|
2012-05-21 20:38:34 +00:00
|
|
|
|
String modified_query = s.str();
|
|
|
|
|
|
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
2012-10-22 19:55:19 +00:00
|
|
|
|
for (ConnectionPools::iterator it = pools.begin(); it != pools.end(); ++it)
|
2013-05-08 10:30:29 +00:00
|
|
|
|
res.push_back(new RemoteBlockInputStream((*it)->get(), modified_query, &new_settings, processed_stage));
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
|
|
|
|
|
|
/// добавляем запросы к локальному clickhouse
|
2013-11-28 10:31:21 +00:00
|
|
|
|
DB::Context new_context = context;
|
|
|
|
|
new_context.setSettings(new_settings);
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
|
|
|
|
DB::Context new_context = context;
|
|
|
|
|
new_context.setSettings(new_settings);
|
|
|
|
|
for(size_t i = 0; i < local_replics_num; ++i)
|
|
|
|
|
{
|
|
|
|
|
InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage);
|
|
|
|
|
res.push_back(interpreter.execute());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-23 12:01:19 +00:00
|
|
|
|
void StorageDistributed::alter(const ASTAlterQuery::Parameters ¶ms)
|
|
|
|
|
{
|
2013-11-13 09:47:12 +00:00
|
|
|
|
alterColumns(params, columns, context);
|
2013-09-23 12:01:19 +00:00
|
|
|
|
}
|
2012-05-21 20:38:34 +00:00
|
|
|
|
}
|