ClickHouse/dbms/src/Storages/StorageMerge.cpp

290 lines
11 KiB
C++
Raw Normal View History

#include <DataStreams/AddingConstColumnBlockInputStream.h>
#include <DataStreams/narrowBlockInputStreams.h>
#include <DataStreams/LazyBlockInputStream.h>
#include <DataStreams/NullBlockInputStream.h>
#include <Storages/StorageMerge.h>
#include <Common/VirtualColumnUtils.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/ExpressionActions.h>
#include <Storages/VirtualColumnFactory.h>
#include <Parsers/ASTSelectQuery.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Databases/IDatabase.h>
#include <DataStreams/CastTypeBlockInputStream.h>
#include <DataStreams/FilterColumnsBlockInputStream.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
{
}
bool StorageMerge::isRemote() const
{
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 && table->isRemote())
return true;
}
iterator->next();
}
return false;
}
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,
const ASTPtr & query,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
2017-06-02 15:54:39 +00:00
const unsigned num_streams)
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.
*/
Context modified_context = context;
modified_context.getSettingsRef().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;
2017-06-02 15:54:39 +00:00
if (curr_table_number < num_streams)
{
QueryProcessingStage::Enum processed_stage_in_source_table = processed_stage;
source_streams = table->read(
real_column_names,
modified_query_ast,
modified_context,
processed_stage_in_source_table,
max_block_size,
2017-06-02 15:54:39 +00:00
tables_count >= num_streams ? 1 : (num_streams / 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);
2017-04-29 03:37:21 +00:00
/// Subordinary tables could have different but convertible types, like numeric types of different width.
/// We must return streams with structure equals to structure of Merge table.
for (auto & stream : source_streams)
{
/// will throw if some columns not convertible
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
}
}
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,
modified_context,
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);
auto stream = streams.empty() ? std::make_shared<NullBlockInputStream>() : streams.front();
if (!streams.empty())
{
/// will throw if some columns not convertible
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
}
return stream;
}));
}
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();
res = narrowBlockInputStreams(res, num_streams);
/// Added to avoid different block structure from different sources
bool throw_if_column_not_found = !processed_stage_in_source_tables
|| processed_stage_in_source_tables.value() == QueryProcessingStage::FetchColumns;
for (auto & stream : res)
stream = std::make_shared<FilterColumnsBlockInputStream>(stream, column_names, throw_if_column_not_found);
return res;
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
}