2012-05-21 20:38:34 +00:00
|
|
|
|
#include <DB/Parsers/formatAST.h>
|
|
|
|
|
|
|
|
|
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
2014-02-11 17:10:48 +00:00
|
|
|
|
#include <DB/DataStreams/RemoveColumnsBlockInputStream.h>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Storages/StorageDistributed.h>
|
|
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
|
#include <Poco/Net/NetworkInterface.h>
|
2014-01-22 14:24:05 +00:00
|
|
|
|
#include <DB/Client/ConnectionPool.h>
|
2013-11-28 10:31:17 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
|
|
|
|
#include <boost/bind.hpp>
|
2014-02-11 17:10:48 +00:00
|
|
|
|
#include <DB/Core/Field.h>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2012-11-06 17:04:38 +00:00
|
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2013-12-07 16:51:29 +00:00
|
|
|
|
Cluster & cluster_,
|
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-09-23 12:01:19 +00:00
|
|
|
|
sign_column_name(sign_column_name_),
|
2013-11-28 10:31:17 +00:00
|
|
|
|
context(context_),
|
2013-12-07 16:51:29 +00:00
|
|
|
|
cluster(cluster_)
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
2014-01-22 14:24:05 +00:00
|
|
|
|
std::vector<String> virtual_columns;
|
|
|
|
|
virtual_columns.push_back("_host");
|
|
|
|
|
virtual_columns.push_back("_port");
|
|
|
|
|
String suffix = VirtualColumnUtils::chooseSuffixForSet(getColumnsList(), virtual_columns);
|
|
|
|
|
_host_column_name = virtual_columns[0] + suffix;
|
|
|
|
|
_port_column_name = virtual_columns[1] + suffix;
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2013-12-10 17:06:57 +00:00
|
|
|
|
const String & cluster_name,
|
|
|
|
|
Context & context_,
|
2013-05-08 10:30:29 +00:00
|
|
|
|
const String & sign_column_name_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
2013-12-10 17:06:57 +00:00
|
|
|
|
context_.initClusters();
|
2014-03-10 04:17:17 +00:00
|
|
|
|
return (new StorageDistributed(name_, columns_, remote_database_, remote_table_, context_.getCluster(cluster_name), context_, sign_column_name_))->thisPtr();
|
2013-02-06 11:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-07 15:11:57 +00:00
|
|
|
|
|
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2014-02-23 02:27:09 +00:00
|
|
|
|
SharedPtr<Cluster> & owned_cluster_,
|
2014-02-07 15:11:57 +00:00
|
|
|
|
Context & context_,
|
|
|
|
|
const String & sign_column_name_)
|
|
|
|
|
{
|
2014-03-10 04:17:17 +00:00
|
|
|
|
auto res = new StorageDistributed(name_, columns_, remote_database_, remote_table_, *owned_cluster_, context_, sign_column_name_);
|
2014-02-23 02:27:09 +00:00
|
|
|
|
|
|
|
|
|
/// Захватываем владение объектом-кластером.
|
|
|
|
|
res->owned_cluster = owned_cluster_;
|
|
|
|
|
|
|
|
|
|
return res->thisPtr();
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-22 14:24:05 +00:00
|
|
|
|
NameAndTypePair StorageDistributed::getColumn(const String &column_name) const
|
|
|
|
|
{
|
2014-02-04 15:44:15 +00:00
|
|
|
|
if (column_name == _host_column_name)
|
|
|
|
|
return std::make_pair(_host_column_name, new DataTypeString);
|
|
|
|
|
if (column_name == _port_column_name)
|
|
|
|
|
return std::make_pair(_port_column_name, new DataTypeUInt16);
|
|
|
|
|
|
2014-01-22 14:24:05 +00:00
|
|
|
|
return getRealColumn(column_name);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool StorageDistributed::hasColumn(const String &column_name) const
|
|
|
|
|
{
|
2014-02-04 15:44:15 +00:00
|
|
|
|
if (column_name == _host_column_name)
|
|
|
|
|
return true;
|
|
|
|
|
if (column_name == _port_column_name)
|
|
|
|
|
return true;
|
|
|
|
|
|
2014-01-22 14:24:05 +00:00
|
|
|
|
return hasRealColumn(column_name);
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-27 13:52:01 +00:00
|
|
|
|
ASTPtr StorageDistributed::remakeQuery(ASTPtr query, const String & host, size_t port)
|
|
|
|
|
{
|
|
|
|
|
/// Создаем копию запроса.
|
|
|
|
|
ASTPtr modified_query_ast = query->clone();
|
|
|
|
|
|
2014-02-04 15:44:15 +00:00
|
|
|
|
/// Добавляем в запрос значения хоста и порта, если требуется.
|
|
|
|
|
if (!host.empty())
|
|
|
|
|
VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, _host_column_name, host);
|
|
|
|
|
if (port != 0)
|
|
|
|
|
VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, _port_column_name, port);
|
2014-01-27 13:52:01 +00:00
|
|
|
|
|
|
|
|
|
/// Меняем имена таблицы и базы данных
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
return modified_query_ast;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static String selectToString(ASTPtr query)
|
|
|
|
|
{
|
|
|
|
|
ASTSelectQuery & select = dynamic_cast<ASTSelectQuery &>(*query);
|
|
|
|
|
std::stringstream s;
|
|
|
|
|
formatAST(select, s, 0, false, true);
|
|
|
|
|
return s.str();
|
|
|
|
|
}
|
|
|
|
|
|
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
|
|
|
|
{
|
2014-02-11 17:10:48 +00:00
|
|
|
|
/// Узнаем на каком порту слушает сервер
|
|
|
|
|
UInt16 clickhouse_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
|
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;
|
2014-02-22 21:50:27 +00:00
|
|
|
|
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.limits.max_execution_time);
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
2014-02-04 15:44:15 +00:00
|
|
|
|
/** Запрошены ли виртуальные столбцы?
|
|
|
|
|
* Если да - будем добавлять их в виде констант в запрос, предназначенный для выполнения на удалённом сервере,
|
|
|
|
|
* а также при получении результата с удалённого сервера.
|
|
|
|
|
*/
|
|
|
|
|
bool need_host_column = false;
|
|
|
|
|
bool need_port_column = false;
|
|
|
|
|
for (const auto & it : column_names)
|
|
|
|
|
{
|
|
|
|
|
if (it == _host_column_name)
|
|
|
|
|
need_host_column = true;
|
|
|
|
|
else if (it == _port_column_name)
|
|
|
|
|
need_port_column = true;
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
/** Есть ли виртуальные столбцы в секции SELECT?
|
2014-02-11 17:10:48 +00:00
|
|
|
|
* Если нет - в случае вычисления запроса до стадии Complete, необходимо удалить их из блока.
|
|
|
|
|
*/
|
|
|
|
|
bool select_host_column = false;
|
|
|
|
|
bool select_port_column = false;
|
2014-02-20 20:05:06 +00:00
|
|
|
|
const ASTExpressionList & select_list = dynamic_cast<const ASTExpressionList &>(*(dynamic_cast<const ASTSelectQuery &>(*query)).select_expression_list);
|
2014-02-11 17:10:48 +00:00
|
|
|
|
for (const auto & it : select_list.children)
|
|
|
|
|
{
|
2014-02-20 20:05:06 +00:00
|
|
|
|
if (const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(&*it))
|
2014-02-11 17:10:48 +00:00
|
|
|
|
{
|
|
|
|
|
if (identifier->name == _host_column_name)
|
|
|
|
|
select_host_column = true;
|
|
|
|
|
else if (identifier->name == _port_column_name)
|
|
|
|
|
select_port_column = true;
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-02-20 20:05:06 +00:00
|
|
|
|
|
2014-02-11 17:10:48 +00:00
|
|
|
|
Names columns_to_remove;
|
2014-02-26 10:59:56 +00:00
|
|
|
|
if (!select_host_column && need_host_column)
|
2014-02-11 17:10:48 +00:00
|
|
|
|
columns_to_remove.push_back(_host_column_name);
|
2014-02-26 10:59:56 +00:00
|
|
|
|
if (!select_port_column && need_port_column)
|
2014-02-11 17:10:48 +00:00
|
|
|
|
columns_to_remove.push_back(_port_column_name);
|
|
|
|
|
|
|
|
|
|
Block virtual_columns_block = getBlockWithVirtualColumns();
|
2014-02-26 10:59:56 +00:00
|
|
|
|
BlockInputStreamPtr virtual_columns;
|
|
|
|
|
|
|
|
|
|
/// Если запрошен хотя бы один виртуальный столбец, пробуем индексировать
|
|
|
|
|
if (need_host_column || need_port_column)
|
|
|
|
|
virtual_columns = VirtualColumnUtils::getVirtualColumnsBlocks(query->clone(), virtual_columns_block, context);
|
|
|
|
|
else /// Иначе, считаем допустимыми все возможные значения
|
|
|
|
|
virtual_columns = new OneBlockInputStream(virtual_columns_block);
|
|
|
|
|
|
2014-03-19 12:35:27 +00:00
|
|
|
|
std::multiset< std::pair<String, UInt16> > values =
|
2014-02-11 17:10:48 +00:00
|
|
|
|
VirtualColumnUtils::extractTwoValuesFromBlocks<String, UInt16>(virtual_columns, _host_column_name, _port_column_name);
|
2014-02-22 21:50:27 +00:00
|
|
|
|
bool all_inclusive = values.size() == virtual_columns_block.rows();
|
2014-02-11 17:10:48 +00:00
|
|
|
|
|
|
|
|
|
size_t result_size = values.size();
|
2014-03-19 12:35:27 +00:00
|
|
|
|
if (cluster.getLocalNodesNum() > 0 && values.find(std::make_pair("localhost", clickhouse_port)) != values.end())
|
2014-02-11 17:10:48 +00:00
|
|
|
|
result_size += cluster.getLocalNodesNum() - 1;
|
|
|
|
|
|
|
|
|
|
processed_stage = result_size == 1
|
|
|
|
|
? QueryProcessingStage::Complete
|
|
|
|
|
: QueryProcessingStage::WithMergeableState;
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
2014-02-23 02:27:09 +00:00
|
|
|
|
for (auto & conn_pool : cluster.pools)
|
2014-01-22 14:24:05 +00:00
|
|
|
|
{
|
2014-02-23 02:27:09 +00:00
|
|
|
|
String current_host = conn_pool->get()->getHost();
|
|
|
|
|
UInt16 current_port = conn_pool->get()->getPort();
|
2014-02-11 17:10:48 +00:00
|
|
|
|
|
|
|
|
|
if (!all_inclusive && values.find(std::make_pair(current_host, current_port)) == values.end())
|
|
|
|
|
continue;
|
|
|
|
|
|
2014-02-04 15:44:15 +00:00
|
|
|
|
String modified_query = selectToString(remakeQuery(
|
|
|
|
|
query,
|
2014-02-11 17:10:48 +00:00
|
|
|
|
need_host_column ? current_host : "",
|
|
|
|
|
need_port_column ? current_port : 0));
|
2014-02-04 15:44:15 +00:00
|
|
|
|
|
2014-02-11 17:10:48 +00:00
|
|
|
|
BlockInputStreamPtr temp = new RemoteBlockInputStream(
|
2014-02-23 02:27:09 +00:00
|
|
|
|
conn_pool->get(&new_settings),
|
2014-02-04 15:44:15 +00:00
|
|
|
|
modified_query,
|
|
|
|
|
&new_settings,
|
|
|
|
|
need_host_column ? _host_column_name : "",
|
|
|
|
|
need_port_column ? _port_column_name : "",
|
2014-03-13 15:00:06 +00:00
|
|
|
|
external_tables,
|
2014-02-11 17:10:48 +00:00
|
|
|
|
processed_stage);
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
if (processed_stage == QueryProcessingStage::WithMergeableState || columns_to_remove.empty())
|
2014-02-11 17:10:48 +00:00
|
|
|
|
res.push_back(temp);
|
|
|
|
|
else
|
|
|
|
|
res.push_back(new RemoveColumnsBlockInputStream(temp, columns_to_remove));
|
2014-01-22 14:24:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-03-19 15:07:29 +00:00
|
|
|
|
if (cluster.getLocalNodesNum() > 0 && (all_inclusive || values.find(std::make_pair("localhost", clickhouse_port)) != values.end()))
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
2014-02-11 17:10:48 +00:00
|
|
|
|
ASTPtr modified_query_ast = remakeQuery(
|
|
|
|
|
query,
|
|
|
|
|
need_host_column ? "localhost" : "",
|
|
|
|
|
need_port_column ? clickhouse_port : 0);
|
|
|
|
|
|
2014-02-22 21:50:27 +00:00
|
|
|
|
/// Добавляем запросы к локальному ClickHouse
|
2013-11-28 10:31:17 +00:00
|
|
|
|
DB::Context new_context = context;
|
|
|
|
|
new_context.setSettings(new_settings);
|
2014-03-13 15:00:06 +00:00
|
|
|
|
for (auto & it : external_tables)
|
2014-03-19 15:07:29 +00:00
|
|
|
|
if (!new_context.tryGetExternalTable(it.first))
|
|
|
|
|
new_context.addExternalTable(it.first, it.second);
|
2014-02-20 20:05:06 +00:00
|
|
|
|
|
|
|
|
|
for(size_t i = 0; i < cluster.getLocalNodesNum(); ++i)
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
2014-02-20 20:05:06 +00:00
|
|
|
|
InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage);
|
2014-02-22 21:50:27 +00:00
|
|
|
|
if (processed_stage == QueryProcessingStage::WithMergeableState || columns_to_remove.empty())
|
2014-02-20 20:05:06 +00:00
|
|
|
|
res.push_back(interpreter.execute());
|
|
|
|
|
else
|
|
|
|
|
res.push_back(new RemoveColumnsBlockInputStream(interpreter.execute(), columns_to_remove));
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-03-13 15:00:06 +00:00
|
|
|
|
external_tables.clear();
|
2014-02-23 02:27:09 +00:00
|
|
|
|
|
2014-02-11 17:10:48 +00:00
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Построить блок состоящий только из возможных значений виртуальных столбцов
|
|
|
|
|
Block StorageDistributed::getBlockWithVirtualColumns()
|
|
|
|
|
{
|
|
|
|
|
Block res;
|
|
|
|
|
ColumnWithNameAndType _host(new ColumnString, new DataTypeString, _host_column_name);
|
|
|
|
|
ColumnWithNameAndType _port(new ColumnUInt16, new DataTypeUInt16, _port_column_name);
|
|
|
|
|
|
|
|
|
|
for (ConnectionPools::iterator it = cluster.pools.begin(); it != cluster.pools.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
_host.column->insert((*it)->get()->getHost());
|
2014-02-22 21:50:27 +00:00
|
|
|
|
_port.column->insert(static_cast<UInt64>((*it)->get()->getPort()));
|
2014-02-11 17:10:48 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (cluster.getLocalNodesNum() > 0)
|
|
|
|
|
{
|
|
|
|
|
/// Узнаем на каком порту слушает сервер
|
|
|
|
|
UInt64 clickhouse_port = Poco::Util::Application::instance().config().getInt("tcp_port", 0);
|
|
|
|
|
String clockhouse_host = "localhost";
|
|
|
|
|
_host.column->insert(clockhouse_host);
|
|
|
|
|
_port.column->insert(clickhouse_port);
|
|
|
|
|
}
|
2014-02-22 21:50:27 +00:00
|
|
|
|
|
2014-02-11 17:10:48 +00:00
|
|
|
|
res.insert(_host);
|
|
|
|
|
res.insert(_port);
|
2013-11-28 10:31:17 +00:00
|
|
|
|
|
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
|
|
|
|
}
|
2014-02-04 15:44:15 +00:00
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
}
|