2012-06-25 03:01:37 +00:00
|
|
|
|
#include <DB/DataStreams/narrowBlockInputStreams.h>
|
2012-05-30 05:53:09 +00:00
|
|
|
|
#include <DB/Storages/StorageMerge.h>
|
2014-01-17 15:19:20 +00:00
|
|
|
|
#include <DB/Common/VirtualColumnUtils.h>
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
StorageMerge::StorageMerge(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & source_database_,
|
|
|
|
|
const String & table_name_regexp_,
|
2013-05-05 18:02:05 +00:00
|
|
|
|
const Context & context_)
|
2012-05-30 05:53:09 +00:00
|
|
|
|
: name(name_), columns(columns_), source_database(source_database_), table_name_regexp(table_name_regexp_), context(context_)
|
|
|
|
|
{
|
2014-01-16 15:25:54 +00:00
|
|
|
|
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
|
2012-05-30 05:53:09 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
|
StoragePtr StorageMerge::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & source_database_,
|
|
|
|
|
const String & table_name_regexp_,
|
2013-05-05 18:02:05 +00:00
|
|
|
|
const Context & context_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
|
|
|
|
return (new StorageMerge(name_, columns_, source_database_, table_name_regexp_, context_))->thisPtr();
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-16 14:52:13 +00:00
|
|
|
|
NameAndTypePair StorageMerge::getColumn(const String &column_name) const
|
|
|
|
|
{
|
|
|
|
|
if (column_name == _table_column_name) return std::make_pair(_table_column_name, new DataTypeString);
|
|
|
|
|
return getRealColumn(column_name);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool StorageMerge::hasColumn(const String &column_name) const
|
|
|
|
|
{
|
|
|
|
|
if (column_name == _table_column_name) return true;
|
|
|
|
|
return hasRealColumn(column_name);
|
|
|
|
|
}
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageMerge::read(
|
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
2013-02-01 19:02:04 +00:00
|
|
|
|
const Settings & settings,
|
2012-05-30 05:53:09 +00:00
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
|
size_t max_block_size,
|
|
|
|
|
unsigned threads)
|
|
|
|
|
{
|
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
2014-01-16 14:52:13 +00:00
|
|
|
|
Names virt_column_names, real_column_names;
|
2014-01-22 14:24:05 +00:00
|
|
|
|
for (const auto & it : column_names)
|
2014-01-16 14:52:13 +00:00
|
|
|
|
if (it != _table_column_name)
|
|
|
|
|
real_column_names.push_back(it);
|
|
|
|
|
else
|
|
|
|
|
virt_column_names.push_back(it);
|
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
StorageVector selected_tables;
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
|
|
|
|
/// Среди всех стадий, до которых обрабатывается запрос в таблицах-источниках, выберем минимальную.
|
|
|
|
|
processed_stage = QueryProcessingStage::Complete;
|
|
|
|
|
QueryProcessingStage::Enum tmp_processed_stage = QueryProcessingStage::Complete;
|
|
|
|
|
|
2012-08-02 17:33:31 +00:00
|
|
|
|
/// Список таблиц могут менять в другом потоке.
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
|
|
|
|
context.assertDatabaseExists(source_database);
|
2013-09-23 12:01:19 +00:00
|
|
|
|
|
2012-08-02 17:33:31 +00:00
|
|
|
|
/** Сначала составим список выбранных таблиц, чтобы узнать его размер.
|
|
|
|
|
* Это нужно, чтобы правильно передать в каждую таблицу рекомендацию по количеству потоков.
|
|
|
|
|
*/
|
2013-09-23 12:01:19 +00:00
|
|
|
|
getSelectedTables(selected_tables);
|
2012-08-02 17:33:31 +00:00
|
|
|
|
}
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
TableLocks table_locks;
|
|
|
|
|
|
|
|
|
|
/// Нельзя, чтобы эти таблицы кто-нибудь удалил, пока мы их читаем.
|
|
|
|
|
for (auto table : selected_tables)
|
|
|
|
|
{
|
|
|
|
|
table_locks.push_back(table->lockStructure(false));
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-11 18:38:21 +00:00
|
|
|
|
Block virtual_columns_block = getBlockWithVirtualColumns(selected_tables);
|
2014-02-26 10:59:56 +00:00
|
|
|
|
BlockInputStreamPtr virtual_columns;
|
|
|
|
|
|
|
|
|
|
/// Если запрошен хотя бы один виртуальный столбец, пробуем индексировать
|
|
|
|
|
if (!virt_column_names.empty())
|
|
|
|
|
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<String> values = VirtualColumnUtils::extractSingleValueFromBlocks<String>(virtual_columns, _table_column_name);
|
2014-02-11 18:38:21 +00:00
|
|
|
|
bool all_inclusive = (values.size() == virtual_columns_block.rows());
|
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
for (size_t i = 0; i < selected_tables.size(); ++i)
|
2012-05-30 05:53:09 +00:00
|
|
|
|
{
|
2014-03-20 10:59:45 +00:00
|
|
|
|
StoragePtr table = selected_tables[i];
|
|
|
|
|
auto table_lock = table_locks[i];
|
|
|
|
|
|
|
|
|
|
if (!all_inclusive && values.find(table->getTableName()) == values.end())
|
2014-02-11 18:38:21 +00:00
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
/// Если в запросе только виртуальные столбцы, надо запросить хотя бы один любой другой.
|
|
|
|
|
if (real_column_names.size() == 0)
|
2014-03-20 10:59:45 +00:00
|
|
|
|
real_column_names.push_back(ExpressionActions::getSmallestColumn(table->getColumnsList()));
|
2014-02-11 18:38:21 +00:00
|
|
|
|
|
2014-02-26 10:59:56 +00:00
|
|
|
|
/// Подменяем виртуальный столбец на его значение
|
|
|
|
|
ASTPtr modified_query_ast = query->clone();
|
2014-03-20 10:59:45 +00:00
|
|
|
|
VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, _table_column_name, table->getTableName());
|
2014-02-26 10:59:56 +00:00
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
BlockInputStreams source_streams = table->read(
|
2014-01-16 14:52:13 +00:00
|
|
|
|
real_column_names,
|
2014-02-26 10:59:56 +00:00
|
|
|
|
modified_query_ast,
|
2013-02-01 19:02:04 +00:00
|
|
|
|
settings,
|
2012-05-30 05:53:09 +00:00
|
|
|
|
tmp_processed_stage,
|
|
|
|
|
max_block_size,
|
|
|
|
|
selected_tables.size() > threads ? 1 : (threads / selected_tables.size()));
|
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
for (auto & stream : source_streams)
|
|
|
|
|
{
|
|
|
|
|
stream->addTableLock(table_lock);
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-16 14:52:13 +00:00
|
|
|
|
for (auto & virtual_column : virt_column_names)
|
|
|
|
|
{
|
|
|
|
|
if (virtual_column == _table_column_name)
|
|
|
|
|
{
|
|
|
|
|
for (auto & stream : source_streams)
|
2014-03-20 10:59:45 +00:00
|
|
|
|
stream = new AddingConstColumnBlockInputStream<String>(stream, new DataTypeString, table->getTableName(), _table_column_name);
|
2014-01-16 14:52:13 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
res.insert(res.end(), source_streams.begin(), source_streams.end());
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
|
|
|
|
if (tmp_processed_stage < processed_stage)
|
|
|
|
|
processed_stage = tmp_processed_stage;
|
|
|
|
|
}
|
|
|
|
|
|
2012-06-25 03:01:37 +00:00
|
|
|
|
/** Если истчоников слишком много, то склеим их в threads источников.
|
|
|
|
|
*/
|
|
|
|
|
if (res.size() > threads)
|
|
|
|
|
res = narrowBlockInputStreams(res, threads);
|
|
|
|
|
|
2012-05-30 05:53:09 +00:00
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-11 18:38:21 +00:00
|
|
|
|
/// Построить блок состоящий только из возможных значений виртуальных столбцов
|
|
|
|
|
Block StorageMerge::getBlockWithVirtualColumns(const std::vector<StoragePtr> & selected_tables) const
|
|
|
|
|
{
|
|
|
|
|
Block res;
|
|
|
|
|
ColumnWithNameAndType _table(new ColumnString, new DataTypeString, _table_column_name);
|
|
|
|
|
|
2014-03-20 10:59:45 +00:00
|
|
|
|
for (StorageVector::const_iterator it = selected_tables.begin(); it != selected_tables.end(); ++it)
|
2014-02-11 18:38:21 +00:00
|
|
|
|
_table.column->insert((*it)->getTableName());
|
|
|
|
|
|
|
|
|
|
res.insert(_table);
|
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-23 12:01:19 +00:00
|
|
|
|
void StorageMerge::getSelectedTables(StorageVector & selected_tables)
|
|
|
|
|
{
|
|
|
|
|
const Tables & tables = context.getDatabases().at(source_database);
|
|
|
|
|
for (Tables::const_iterator it = tables.begin(); it != tables.end(); ++it)
|
2014-03-20 10:59:45 +00:00
|
|
|
|
if (it->second.get() != this && table_name_regexp.match(it->first))
|
2013-09-23 12:01:19 +00:00
|
|
|
|
selected_tables.push_back(it->second);
|
2012-05-30 05:53:09 +00:00
|
|
|
|
}
|
2013-09-23 12:01:19 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void StorageMerge::alter(const ASTAlterQuery::Parameters & params)
|
|
|
|
|
{
|
2013-11-13 09:47:12 +00:00
|
|
|
|
alterColumns(params, columns, context);
|
2013-09-23 12:01:19 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|