2017-04-01 09:19:00 +00:00
|
|
|
#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>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2017-04-29 02:03:38 +00:00
|
|
|
#include <DataStreams/CastTypeBlockInputStream.h>
|
2017-07-04 14:59:01 +00:00
|
|
|
#include <DataStreams/FilterColumnsBlockInputStream.h>
|
2017-07-12 16:57:35 +00:00
|
|
|
#include <DataStreams/RemoveColumnsBlockInputStream.h>
|
2017-07-12 17:41:37 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
2012-05-30 05:53:09 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int ILLEGAL_PREWHERE;
|
|
|
|
extern const int INCOMPATIBLE_SOURCE_TABLES;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageMerge::StorageMerge(
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-06-15 14:07:31 +00:00
|
|
|
bool StorageMerge::isRemote() const
|
|
|
|
{
|
|
|
|
auto database = context.getDatabase(source_database);
|
2017-09-11 12:39:01 +00:00
|
|
|
auto iterator = database->getIterator(context);
|
2017-06-15 14:07:31 +00:00
|
|
|
|
|
|
|
while (iterator->isValid())
|
|
|
|
{
|
2017-06-16 16:19:15 +00:00
|
|
|
if (table_name_regexp.match(iterator->name()))
|
|
|
|
{
|
|
|
|
auto & table = iterator->table();
|
|
|
|
if (table.get() != this && table->isRemote())
|
|
|
|
return true;
|
|
|
|
}
|
2017-06-16 00:21:00 +00:00
|
|
|
|
|
|
|
iterator->next();
|
2017-06-15 14:07:31 +00:00
|
|
|
}
|
2017-06-16 16:19:15 +00:00
|
|
|
|
2017-06-15 14:07:31 +00:00
|
|
|
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
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto type = VirtualColumnFactory::tryGetType(column_name);
|
|
|
|
if (type)
|
|
|
|
return NameAndTypePair(column_name, type);
|
2014-07-31 09:11:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +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
|
|
|
{
|
2017-04-01 07:20:54 +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
|
|
|
|
2017-07-12 18:44:27 +00:00
|
|
|
static Names collectIdentifiersInFirstLevelOfSelectQuery(ASTPtr ast)
|
|
|
|
{
|
|
|
|
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*ast);
|
|
|
|
ASTExpressionList & node = typeid_cast<ASTExpressionList &>(*select.select_expression_list);
|
|
|
|
ASTs & asts = node.children;
|
|
|
|
|
|
|
|
Names names;
|
|
|
|
for (size_t i = 0; i < asts.size(); ++i)
|
|
|
|
{
|
|
|
|
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(&* asts[i]))
|
|
|
|
{
|
|
|
|
if (identifier->kind == ASTIdentifier::Kind::Column)
|
|
|
|
names.push_back(identifier->name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return names;
|
|
|
|
}
|
|
|
|
|
2017-07-21 20:59:01 +00:00
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
using NodeHashToSet = std::map<IAST::Hash, SetPtr>;
|
|
|
|
|
|
|
|
void relinkSetsImpl(const ASTPtr & query, const NodeHashToSet & node_hash_to_set, PreparedSets & new_sets)
|
|
|
|
{
|
|
|
|
auto hash = query->getTreeHash();
|
|
|
|
auto it = node_hash_to_set.find(hash);
|
|
|
|
if (node_hash_to_set.end() != it)
|
|
|
|
new_sets[query.get()] = it->second;
|
|
|
|
|
|
|
|
for (const auto & child : query->children)
|
|
|
|
relinkSetsImpl(child, node_hash_to_set, new_sets);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Re-link prepared sets onto cloned and modified AST.
|
|
|
|
void relinkSets(const ASTPtr & query, const PreparedSets & old_sets, PreparedSets & new_sets)
|
|
|
|
{
|
|
|
|
NodeHashToSet node_hash_to_set;
|
|
|
|
for (const auto & node_set : old_sets)
|
|
|
|
node_hash_to_set.emplace(node_set.first->getTreeHash(), node_set.second);
|
|
|
|
|
|
|
|
relinkSetsImpl(query, node_hash_to_set, new_sets);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-05-30 05:53:09 +00:00
|
|
|
BlockInputStreams StorageMerge::read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
{
|
2017-04-01 07:20:54 +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);
|
|
|
|
|
2017-11-20 04:15:43 +00:00
|
|
|
std::optional<QueryProcessingStage::Enum> processed_stage_in_source_tables;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** 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();
|
|
|
|
|
2017-07-15 03:48:36 +00:00
|
|
|
const ASTPtr & query = query_info.query;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// 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.
|
|
|
|
*/
|
2017-05-24 21:06:29 +00:00
|
|
|
Context modified_context = context;
|
|
|
|
modified_context.getSettingsRef().optimize_move_to_prewhere = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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());
|
|
|
|
|
2017-07-21 20:59:01 +00:00
|
|
|
SelectQueryInfo modified_query_info;
|
|
|
|
modified_query_info.query = modified_query_ast;
|
|
|
|
|
|
|
|
relinkSets(modified_query_info.query, query_info.sets, modified_query_info.sets);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockInputStreams source_streams;
|
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
if (curr_table_number < num_streams)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
QueryProcessingStage::Enum processed_stage_in_source_table = processed_stage;
|
|
|
|
source_streams = table->read(
|
|
|
|
real_column_names,
|
2017-07-21 20:59:01 +00:00
|
|
|
modified_query_info,
|
2017-05-24 21:06:29 +00:00
|
|
|
modified_context,
|
2017-04-01 07:20:54 +00:00
|
|
|
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));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!processed_stage_in_source_tables)
|
|
|
|
processed_stage_in_source_tables.emplace(processed_stage_in_source_table);
|
2017-10-04 00:22:00 +00:00
|
|
|
else if (processed_stage_in_source_table != *processed_stage_in_source_tables)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Source tables for Merge table are processing data up to different stages",
|
|
|
|
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
|
2017-04-29 01:48:05 +00:00
|
|
|
|
2017-04-29 03:37:21 +00:00
|
|
|
/// Subordinary tables could have different but convertible types, like numeric types of different width.
|
2017-05-25 01:12:41 +00:00
|
|
|
/// We must return streams with structure equals to structure of Merge table.
|
2017-04-29 01:48:05 +00:00
|
|
|
for (auto & stream : source_streams)
|
2017-05-04 04:08:59 +00:00
|
|
|
{
|
|
|
|
/// will throw if some columns not convertible
|
2017-05-20 14:47:40 +00:00
|
|
|
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
|
2017-05-04 04:08:59 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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,
|
2017-07-21 20:59:01 +00:00
|
|
|
modified_query_info,
|
2017-05-24 21:06:29 +00:00
|
|
|
modified_context,
|
2017-04-01 07:20:54 +00:00
|
|
|
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);
|
2017-10-04 00:22:00 +00:00
|
|
|
else if (processed_stage_in_source_table != *processed_stage_in_source_tables)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Source tables for Merge table are processing data up to different stages",
|
|
|
|
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
|
|
|
|
|
2017-04-29 01:48:05 +00:00
|
|
|
auto stream = streams.empty() ? std::make_shared<NullBlockInputStream>() : streams.front();
|
|
|
|
if (!streams.empty())
|
2017-05-04 04:08:59 +00:00
|
|
|
{
|
|
|
|
/// will throw if some columns not convertible
|
2017-05-20 14:47:40 +00:00
|
|
|
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
|
2017-05-04 04:08:59 +00:00
|
|
|
}
|
2017-04-29 01:48:05 +00:00
|
|
|
return stream;
|
2017-04-01 07:20:54 +00:00
|
|
|
}));
|
|
|
|
}
|
|
|
|
|
|
|
|
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)
|
2017-10-04 00:22:00 +00:00
|
|
|
processed_stage = *processed_stage_in_source_tables;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-04 14:59:01 +00:00
|
|
|
res = narrowBlockInputStreams(res, num_streams);
|
|
|
|
|
|
|
|
/// Added to avoid different block structure from different sources
|
2017-10-04 00:22:00 +00:00
|
|
|
if (!processed_stage_in_source_tables || *processed_stage_in_source_tables == QueryProcessingStage::FetchColumns)
|
2017-07-12 16:57:35 +00:00
|
|
|
{
|
|
|
|
for (auto & stream : res)
|
|
|
|
stream = std::make_shared<FilterColumnsBlockInputStream>(stream, column_names, true);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-07-12 17:41:37 +00:00
|
|
|
/// Blocks from distributed tables may have extra columns.
|
|
|
|
/// We need to remove them to make blocks compatible.
|
|
|
|
auto identifiers = collectIdentifiersInFirstLevelOfSelectQuery(query);
|
|
|
|
std::set<String> identifiers_set(identifiers.begin(), identifiers.end());
|
2017-07-12 16:57:35 +00:00
|
|
|
Names columns_to_remove;
|
|
|
|
for (const auto & column : column_names)
|
2017-07-12 17:41:37 +00:00
|
|
|
if (!identifiers_set.count(column))
|
2017-07-12 16:57:35 +00:00
|
|
|
columns_to_remove.push_back(column);
|
|
|
|
|
|
|
|
if (!columns_to_remove.empty())
|
|
|
|
for (auto & stream : res)
|
|
|
|
stream = std::make_shared<RemoveColumnsBlockInputStream>(stream, columns_to_remove);
|
|
|
|
}
|
2017-07-04 14:59:01 +00:00
|
|
|
|
|
|
|
return res;
|
2012-05-30 05:53:09 +00:00
|
|
|
}
|
|
|
|
|
2017-03-13 18:02:45 +00:00
|
|
|
/// Construct a block consisting only of possible values of virtual columns
|
2017-02-02 15:54:27 +00:00
|
|
|
Block StorageMerge::getBlockWithVirtualColumns(const StorageListWithLocks & selected_tables) const
|
2014-02-11 18:38:21 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Block res;
|
|
|
|
ColumnWithTypeAndName _table(std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "_table");
|
2014-02-11 18:38:21 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & elem : selected_tables)
|
|
|
|
_table.column->insert(elem.first->getTableName());
|
2014-02-11 18:38:21 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
res.insert(_table);
|
|
|
|
return res;
|
2014-02-11 18:38:21 +00:00
|
|
|
}
|
|
|
|
|
2017-02-02 15:54:27 +00:00
|
|
|
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables() const
|
2013-09-23 12:01:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageListWithLocks selected_tables;
|
|
|
|
auto database = context.getDatabase(source_database);
|
2017-09-11 12:39:01 +00:00
|
|
|
auto iterator = database->getIterator(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
while (iterator->isValid())
|
|
|
|
{
|
|
|
|
if (table_name_regexp.match(iterator->name()))
|
|
|
|
{
|
|
|
|
auto & table = iterator->table();
|
|
|
|
if (table.get() != this)
|
2017-09-01 15:05:23 +00:00
|
|
|
selected_tables.emplace_back(table, table->lockStructure(false, __PRETTY_FUNCTION__));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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
|
|
|
{
|
2017-04-01 07:20:54 +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);
|
2016-05-05 18:28:46 +00:00
|
|
|
|
2017-09-01 15:05:23 +00:00
|
|
|
auto lock = lockStructureForAlter(__PRETTY_FUNCTION__);
|
2017-04-01 07:20:54 +00:00
|
|
|
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
|
2016-05-13 21:08:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +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
|
|
|
}
|
2014-07-11 08:12:03 +00:00
|
|
|
|
2013-09-23 12:01:19 +00:00
|
|
|
}
|