ClickHouse/dbms/src/Storages/StorageMaterializedView.cpp

425 lines
15 KiB
C++
Raw Normal View History

#include <Storages/StorageMaterializedView.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTDropQuery.h>
2017-05-23 18:33:48 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/InterpreterRenameQuery.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Access/AccessFlags.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Storages/AlterCommands.h>
#include <Storages/StorageFactory.h>
#include <Storages/ReadInOrderOptimizer.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Processors/Sources/SourceFromInputStream.h>
2017-07-13 20:58:19 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int INCORRECT_QUERY;
extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW;
}
static inline String generateInnerTableName(const String & table_name)
{
return ".inner." + table_name;
}
2020-01-31 17:12:18 +00:00
static StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, const Context & context, bool add_default_db = true)
{
2019-12-30 18:20:43 +00:00
if (add_default_db)
2019-12-10 19:48:16 +00:00
{
AddDefaultDatabaseVisitor visitor(context.getCurrentDatabase(), nullptr);
visitor.visit(query);
}
2019-12-30 18:20:43 +00:00
if (auto db_and_table = getDatabaseAndTable(query, 0))
{
2019-12-10 19:48:16 +00:00
return StorageID(db_and_table->database, db_and_table->table/*, db_and_table->uuid*/);
}
2019-12-30 18:20:43 +00:00
else if (auto subquery = extractTableExpression(query, 0))
{
2019-12-30 18:20:43 +00:00
auto * ast_select = subquery->as<ASTSelectWithUnionQuery>();
if (!ast_select)
throw Exception("Logical error while creating StorageMaterializedView. "
"Could not retrieve table name from select query.",
2019-12-30 18:20:43 +00:00
DB::ErrorCodes::LOGICAL_ERROR);
if (ast_select->list_of_selects->children.size() != 1)
throw Exception("UNION is not supported for MATERIALIZED VIEW",
ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
auto & inner_query = ast_select->list_of_selects->children.at(0);
return extractDependentTableFromSelectQuery(inner_query->as<ASTSelectQuery &>(), context, false);
}
else
2019-12-30 18:20:43 +00:00
return StorageID::createEmpty();
}
static void checkAllowedQueries(const ASTSelectQuery & query)
{
if (query.prewhere() || query.final() || query.sampleSize())
throw Exception("MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL.", DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
2019-01-15 18:29:54 +00:00
ASTPtr subquery = extractTableExpression(query, 0);
if (!subquery)
return;
2019-03-11 13:22:51 +00:00
if (const auto * ast_select = subquery->as<ASTSelectWithUnionQuery>())
{
if (ast_select->list_of_selects->children.size() != 1)
throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
const auto & inner_query = ast_select->list_of_selects->children.at(0);
2019-03-15 17:09:14 +00:00
checkAllowedQueries(inner_query->as<ASTSelectQuery &>());
}
}
StorageMaterializedView::StorageMaterializedView(
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
Context & local_context,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
bool attach_)
2019-12-04 16:06:55 +00:00
: IStorage(table_id_), global_context(local_context.getGlobalContext())
{
2019-08-24 21:20:20 +00:00
setColumns(columns_);
if (!query.select)
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
2019-12-10 19:48:16 +00:00
/// If the destination table is not set, use inner table
has_inner_table = query.to_table.empty();
if (has_inner_table && !query.storage)
throw Exception(
"You must specify where to save results of a MaterializedView query: either ENGINE or an existing table in a TO clause",
ErrorCodes::INCORRECT_QUERY);
2018-02-25 07:39:45 +00:00
if (query.select->list_of_selects->children.size() != 1)
throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
2018-02-25 06:34:20 +00:00
select = query.select->clone();
2018-02-25 07:39:45 +00:00
inner_query = query.select->list_of_selects->children.at(0);
2018-02-25 06:34:20 +00:00
2019-03-15 17:09:14 +00:00
auto & select_query = inner_query->as<ASTSelectQuery &>();
checkAllowedQueries(select_query);
2020-01-31 17:12:18 +00:00
select_table_id = extractDependentTableFromSelectQuery(select_query, local_context);
2019-12-10 19:48:16 +00:00
if (!has_inner_table)
target_table_id = StorageID(query.to_database, query.to_table);
else if (attach_)
{
2019-12-10 19:48:16 +00:00
/// If there is an ATTACH request, then the internal table must already be created.
2020-01-31 17:12:18 +00:00
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID().table_name));
}
else
{
/// We will create a query to create an internal table.
auto manual_create_query = std::make_shared<ASTCreateQuery>();
2020-01-31 17:12:18 +00:00
manual_create_query->database = getStorageID().database_name;
manual_create_query->table = generateInnerTableName(getStorageID().table_name);
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
auto new_columns_list = std::make_shared<ASTColumns>();
new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr());
manual_create_query->set(manual_create_query->columns_list, new_columns_list);
manual_create_query->set(manual_create_query->storage, query.storage->ptr());
2019-12-10 19:48:16 +00:00
InterpreterCreateQuery create_interpreter(manual_create_query, local_context);
create_interpreter.setInternal(true);
create_interpreter.execute();
2020-02-17 19:28:25 +00:00
target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table})->getStorageID();
}
2019-12-10 19:48:16 +00:00
if (!select_table_id.empty())
DatabaseCatalog::instance().addDependency(select_table_id, getStorageID());
}
NameAndTypePair StorageMaterializedView::getColumn(const String & column_name) const
{
return getTargetTable()->getColumn(column_name);
}
bool StorageMaterializedView::hasColumn(const String & column_name) const
{
return getTargetTable()->hasColumn(column_name);
}
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
{
2020-02-14 13:17:50 +00:00
StorageInMemoryMetadata result(getColumns(), getIndices(), getConstraints());
result.select = getSelectQuery();
return result;
}
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const
{
return getTargetTable()->getQueryProcessingStage(context, query_ptr);
}
Pipes StorageMaterializedView::read(
const Names & column_names,
const SelectQueryInfo & query_info,
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)
{
auto storage = getTargetTable();
2020-03-28 02:09:29 +00:00
auto lock = storage->lockStructureForShare(context.getCurrentQueryId());
if (query_info.order_by_optimizer)
query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage);
Pipes pipes = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
for (auto & pipe : pipes)
pipe.addTableLock(lock);
return pipes;
}
BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Context & context)
{
auto storage = getTargetTable();
2020-03-28 02:09:29 +00:00
auto lock = storage->lockStructureForShare(context.getCurrentQueryId());
auto stream = storage->write(query, context);
stream->addTableLock(lock);
return stream;
}
2019-12-10 19:48:16 +00:00
static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id)
2018-06-09 15:48:22 +00:00
{
2020-02-17 13:52:59 +00:00
if (DatabaseCatalog::instance().tryGetTable(target_table_id))
{
/// We create and execute `drop` query for internal table.
auto drop_query = std::make_shared<ASTDropQuery>();
2019-12-10 19:48:16 +00:00
drop_query->database = target_table_id.database_name;
drop_query->table = target_table_id.table_name;
2018-06-09 15:48:22 +00:00
drop_query->kind = kind;
ASTPtr ast_drop_query = drop_query;
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
drop_interpreter.execute();
}
}
2018-06-09 15:48:22 +00:00
2019-08-27 23:47:30 +00:00
void StorageMaterializedView::drop(TableStructureWriteLockHolder &)
2018-04-21 00:35:20 +00:00
{
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID();
2019-12-30 18:20:43 +00:00
if (!select_table_id.empty())
DatabaseCatalog::instance().removeDependency(select_table_id, table_id);
2018-04-21 00:35:20 +00:00
if (has_inner_table && tryGetTargetTable())
2019-12-10 19:48:16 +00:00
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id);
2018-06-09 15:48:22 +00:00
}
2019-08-27 23:47:30 +00:00
void StorageMaterializedView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
2018-06-09 15:48:22 +00:00
{
if (has_inner_table)
2019-12-10 19:48:16 +00:00
executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_table_id);
2018-04-21 00:35:20 +00:00
}
void StorageMaterializedView::checkStatementCanBeForwarded() const
{
if (!has_inner_table)
throw Exception(
2019-12-10 19:48:16 +00:00
"MATERIALIZED VIEW targets existing table " + target_table_id.getNameForLogs() + ". "
+ "Execute the statement directly on it.", ErrorCodes::INCORRECT_QUERY);
}
bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
{
checkStatementCanBeForwarded();
return getTargetTable()->optimize(query, partition, final, deduplicate, context);
}
void StorageMaterializedView::alter(
const AlterCommands & params,
const Context & context,
TableStructureWriteLockHolder & table_lock_holder)
{
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
auto table_id = getStorageID();
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
2020-01-31 17:12:18 +00:00
/// start modify query
if (context.getSettingsRef().allow_experimental_alter_materialized_view_structure)
{
auto & new_select = metadata.select->as<ASTSelectWithUnionQuery &>();
if (new_select.list_of_selects->children.size() != 1)
throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
auto & new_inner_query = new_select.list_of_selects->children.at(0);
auto & select_query = new_inner_query->as<ASTSelectQuery &>();
checkAllowedQueries(select_query);
auto new_select_table_id = extractDependentTableFromSelectQuery(select_query, context);
DatabaseCatalog::instance().updateDependency(select_table_id, table_id, new_select_table_id, table_id);
2020-01-31 17:12:18 +00:00
select_table_id = new_select_table_id;
select = metadata.select;
inner_query = new_inner_query;
2020-01-31 17:12:18 +00:00
}
/// end modify query
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
setColumns(std::move(metadata.columns));
}
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings)
{
if (settings.allow_experimental_alter_materialized_view_structure)
{
2020-01-31 17:12:18 +00:00
for (const auto & command : commands)
{
if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_QUERY)
throw Exception(
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
ErrorCodes::NOT_IMPLEMENTED);
}
}
else
{
for (const auto & command : commands)
{
if (!command.isCommentAlter())
throw Exception(
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
ErrorCodes::NOT_IMPLEMENTED);
}
}
}
void StorageMaterializedView::alterPartition(const ASTPtr & query, const PartitionCommands &commands, const Context &context)
{
checkStatementCanBeForwarded();
getTargetTable()->alterPartition(query, commands, context);
}
void StorageMaterializedView::mutate(const MutationCommands & commands, const Context & context)
{
checkStatementCanBeForwarded();
getTargetTable()->mutate(commands, context);
}
2019-12-10 19:48:16 +00:00
void StorageMaterializedView::rename(
const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
{
2019-12-10 19:48:16 +00:00
if (has_inner_table && tryGetTargetTable())
{
2019-12-10 19:48:16 +00:00
auto new_target_table_name = generateInnerTableName(new_table_name);
auto rename = std::make_shared<ASTRenameQuery>();
2019-12-10 19:48:16 +00:00
ASTRenameQuery::Table from;
from.database = target_table_id.database_name;
from.table = target_table_id.table_name;
2019-12-10 19:48:16 +00:00
ASTRenameQuery::Table to;
to.database = target_table_id.database_name;
to.table = new_target_table_name;
2019-12-10 19:48:16 +00:00
ASTRenameQuery::Element elem;
elem.from = from;
elem.to = to;
rename->elements.emplace_back(elem);
2019-12-10 19:48:16 +00:00
InterpreterRenameQuery(rename, global_context).execute();
target_table_id.table_name = new_target_table_name;
}
auto old_table_id = getStorageID();
2019-12-03 16:25:32 +00:00
IStorage::renameInMemory(new_database_name, new_table_name);
DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID());
}
void StorageMaterializedView::shutdown()
{
/// Make sure the dependency is removed after DETACH TABLE
2019-12-30 18:20:43 +00:00
if (!select_table_id.empty())
DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID());
}
StoragePtr StorageMaterializedView::getTargetTable() const
{
2020-02-17 13:52:59 +00:00
return DatabaseCatalog::instance().getTable(target_table_id);
}
StoragePtr StorageMaterializedView::tryGetTargetTable() const
{
2020-02-17 13:52:59 +00:00
return DatabaseCatalog::instance().tryGetTable(target_table_id);
}
2019-04-04 13:13:59 +00:00
Strings StorageMaterializedView::getDataPaths() const
{
if (auto table = tryGetTargetTable())
2019-04-04 13:13:59 +00:00
return table->getDataPaths();
return {};
}
void StorageMaterializedView::checkTableCanBeDropped() const
{
/// Don't drop the target table if it was created manually via 'TO inner_table' statement
if (!has_inner_table)
return;
auto target_table = tryGetTargetTable();
if (!target_table)
return;
target_table->checkTableCanBeDropped();
}
void StorageMaterializedView::checkPartitionCanBeDropped(const ASTPtr & partition)
{
/// Don't drop the partition in target table if it was created manually via 'TO inner_table' statement
if (!has_inner_table)
return;
auto target_table = tryGetTargetTable();
if (!target_table)
return;
target_table->checkPartitionCanBeDropped(partition);
}
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
{
return has_inner_table ? getTargetTable()->getActionLock(type) : ActionLock{};
}
void registerStorageMaterializedView(StorageFactory & factory)
{
factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args)
{
/// Pass local_context here to convey setting for inner table
return StorageMaterializedView::create(
2019-12-04 16:06:55 +00:00
args.table_id, args.local_context, args.query,
args.columns, args.attach);
});
}
}