ClickHouse/dbms/src/Storages/StorageMerge.cpp

278 lines
10 KiB
C++
Raw Normal View History

#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
2015-10-01 03:30:50 +00:00
#include <DB/DataStreams/narrowBlockInputStreams.h>
#include <DB/DataStreams/LazyBlockInputStream.h>
#include <DB/DataStreams/NullBlockInputStream.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>
#include <DB/Interpreters/InterpreterAlterQuery.h>
2015-05-04 17:52:19 +00:00
#include <DB/Interpreters/ExpressionActions.h>
2014-07-31 09:11:49 +00:00
#include <DB/Storages/VirtualColumnFactory.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/Columns/ColumnString.h>
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
#include <DB/Databases/IDatabase.h>
2012-05-30 05:53:09 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_PREWHERE;
extern const int INCOMPATIBLE_SOURCE_TABLES;
}
2012-05-30 05:53:09 +00:00
StorageMerge::StorageMerge(
const std::string & name_,
NamesAndTypesListPtr columns_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
: name(name_), columns(columns_), source_database(source_database_),
table_name_regexp(table_name_regexp_), context(context_)
{
}
StorageMerge::StorageMerge(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
name(name_), columns(columns_), source_database(source_database_),
table_name_regexp(table_name_regexp_), context(context_)
2012-05-30 05:53:09 +00:00
{
}
StoragePtr StorageMerge::create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
{
return make_shared(
name_, columns_,
source_database_, table_name_regexp_, context_
);
}
StoragePtr StorageMerge::create(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
const String & source_database_,
const String & table_name_regexp_,
const Context & context_)
{
return make_shared(
name_, columns_, materialized_columns_, alias_columns_, column_defaults_,
source_database_, table_name_regexp_, context_
);
}
2014-07-31 09:11:49 +00:00
NameAndTypePair StorageMerge::getColumn(const String & column_name) const
2014-01-16 14:52:13 +00:00
{
auto type = VirtualColumnFactory::tryGetType(column_name);
if (type)
return NameAndTypePair(column_name, type);
2014-07-31 09:11:49 +00:00
return IStorage::getColumn(column_name);
2014-01-16 14:52:13 +00:00
}
2014-07-31 09:11:49 +00:00
bool StorageMerge::hasColumn(const String & column_name) const
2014-01-16 14:52:13 +00:00
{
return VirtualColumnFactory::hasColumn(column_name) || IStorage::hasColumn(column_name);
2014-01-16 14:52:13 +00:00
}
2012-05-30 05:53:09 +00:00
BlockInputStreams StorageMerge::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
const unsigned threads)
2012-05-30 05:53:09 +00:00
{
BlockInputStreams res;
Names virt_column_names, real_column_names;
for (const auto & it : column_names)
if (it != "_table")
real_column_names.push_back(it);
else
virt_column_names.push_back(it);
std::experimental::optional<QueryProcessingStage::Enum> processed_stage_in_source_tables;
/** First we make list of selected tables to find out its size.
* This is necessary to correctly pass the recommended number of threads to each table.
*/
StorageListWithLocks selected_tables = getSelectedTables();
/// If PREWHERE is used in query, you need to make sure that all tables support this.
if (typeid_cast<const ASTSelectQuery &>(*query).prewhere_expression)
for (const auto & elem : selected_tables)
if (!elem.first->supportsPrewhere())
throw Exception("Storage " + elem.first->getName() + " doesn't support PREWHERE.", ErrorCodes::ILLEGAL_PREWHERE);
Block virtual_columns_block = getBlockWithVirtualColumns(selected_tables);
/// If at least one virtual column is requested, try indexing
if (!virt_column_names.empty())
{
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
/// Remove unused tables from the list
selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getTableName()) == values.end(); });
}
/** Just in case, turn off optimization "transfer to PREWHERE",
* since there is no certainty that it works when one of table is MergeTree and other is not.
*/
Settings modified_settings = settings;
modified_settings.optimize_move_to_prewhere = false;
size_t tables_count = selected_tables.size();
size_t curr_table_number = 0;
for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it, ++curr_table_number)
{
StoragePtr table = it->first;
auto & table_lock = it->second;
/// If there are only virtual columns in query, you must request at least one other column.
if (real_column_names.size() == 0)
real_column_names.push_back(ExpressionActions::getSmallestColumn(table->getColumnsList()));
/// Substitute virtual column for its value
ASTPtr modified_query_ast = query->clone();
VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, "_table", table->getTableName());
BlockInputStreams source_streams;
if (curr_table_number < threads)
{
QueryProcessingStage::Enum processed_stage_in_source_table = processed_stage;
source_streams = table->read(
real_column_names,
modified_query_ast,
context,
modified_settings,
processed_stage_in_source_table,
max_block_size,
tables_count >= threads ? 1 : (threads / tables_count));
if (!processed_stage_in_source_tables)
processed_stage_in_source_tables.emplace(processed_stage_in_source_table);
else if (processed_stage_in_source_table != processed_stage_in_source_tables.value())
throw Exception("Source tables for Merge table are processing data up to different stages",
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
}
else
{
/// If many streams, initialize it lazily, to avoid long delay before start of query processing.
source_streams.emplace_back(std::make_shared<LazyBlockInputStream>([=]
{
QueryProcessingStage::Enum processed_stage_in_source_table = processed_stage;
BlockInputStreams streams = table->read(
real_column_names,
modified_query_ast,
context,
modified_settings,
processed_stage_in_source_table,
max_block_size,
1);
if (!processed_stage_in_source_tables)
throw Exception("Logical error: unknown processed stage in source tables",
ErrorCodes::LOGICAL_ERROR);
else if (processed_stage_in_source_table != processed_stage_in_source_tables.value())
throw Exception("Source tables for Merge table are processing data up to different stages",
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
return streams.empty() ? std::make_shared<NullBlockInputStream>() : streams.front();
}));
}
for (auto & stream : source_streams)
stream->addTableLock(table_lock);
for (auto & virtual_column : virt_column_names)
{
if (virtual_column == "_table")
{
for (auto & stream : source_streams)
stream = std::make_shared<AddingConstColumnBlockInputStream<String>>(
stream, std::make_shared<DataTypeString>(), table->getTableName(), "_table");
}
}
res.insert(res.end(), source_streams.begin(), source_streams.end());
}
if (processed_stage_in_source_tables)
processed_stage = processed_stage_in_source_tables.value();
return narrowBlockInputStreams(res, threads);
2012-05-30 05:53:09 +00:00
}
/// Construct a block consisting only of possible values of virtual columns
Block StorageMerge::getBlockWithVirtualColumns(const StorageListWithLocks & selected_tables) const
{
Block res;
ColumnWithTypeAndName _table(std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "_table");
for (const auto & elem : selected_tables)
_table.column->insert(elem.first->getTableName());
res.insert(_table);
return res;
}
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables() const
2013-09-23 12:01:19 +00:00
{
StorageListWithLocks selected_tables;
auto database = context.getDatabase(source_database);
auto iterator = database->getIterator();
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
auto & table = iterator->table();
if (table.get() != this)
selected_tables.emplace_back(table, table->lockStructure(false));
}
iterator->next();
}
return selected_tables;
2012-05-30 05:53:09 +00:00
}
2013-09-23 12:01:19 +00:00
2016-01-28 01:00:27 +00:00
void StorageMerge::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
2013-09-23 12:01:19 +00:00
{
for (const auto & param : params)
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED);
auto lock = lockStructureForAlter();
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
2016-05-13 21:08:19 +00:00
context.getDatabase(database_name)->alterTable(
context, table_name,
*columns, materialized_columns, alias_columns, column_defaults, {});
2013-09-23 12:01:19 +00:00
}
2013-09-23 12:01:19 +00:00
}