2018-11-13 13:48:53 +00:00
|
|
|
#include <Storages/StorageMaterializedView.h>
|
|
|
|
|
2017-07-25 21:07:05 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2018-03-16 01:23:37 +00:00
|
|
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/ASTDropQuery.h>
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2017-05-23 18:33:48 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
|
|
|
#include <Interpreters/InterpreterDropQuery.h>
|
2019-05-07 06:54:55 +00:00
|
|
|
#include <Interpreters/InterpreterRenameQuery.h>
|
2019-12-12 15:28:24 +00:00
|
|
|
#include <Interpreters/getTableExpressions.h>
|
2018-11-01 14:49:37 +00:00
|
|
|
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
2020-01-26 09:49:53 +00:00
|
|
|
#include <Access/AccessFlags.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2020-01-29 17:44:16 +00:00
|
|
|
#include <Storages/AlterCommands.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2019-12-11 13:09:46 +00:00
|
|
|
#include <Storages/ReadInOrderOptimizer.h>
|
2020-06-05 11:54:54 +00:00
|
|
|
#include <Storages/SelectQueryDescription.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2020-01-31 15:16:06 +00:00
|
|
|
#include <Processors/Sources/SourceFromInputStream.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
|
2013-11-15 09:43:50 +00:00
|
|
|
|
2013-11-08 17:43:03 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2017-08-18 20:56:19 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2018-03-16 01:23:37 +00:00
|
|
|
extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2020-03-18 17:38:52 +00:00
|
|
|
static inline String generateInnerTableName(const StorageID & view_id)
|
2019-05-07 06:54:55 +00:00
|
|
|
{
|
2020-03-18 17:38:52 +00:00
|
|
|
if (view_id.hasUUID())
|
2020-03-20 00:07:52 +00:00
|
|
|
return ".inner_id." + toString(view_id.uuid);
|
2020-03-18 17:38:52 +00:00
|
|
|
return ".inner." + view_id.getTableName();
|
2019-05-07 06:54:55 +00:00
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2018-03-16 01:23:37 +00:00
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageMaterializedView::StorageMaterializedView(
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2017-10-26 15:39:56 +00:00
|
|
|
Context & local_context,
|
2017-09-17 18:49:43 +00:00
|
|
|
const ASTCreateQuery & query,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool attach_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_), global_context(local_context.getGlobalContext())
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2019-08-24 21:20:20 +00:00
|
|
|
setColumns(columns_);
|
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
if (!query.select)
|
2017-08-18 20:56:19 +00:00
|
|
|
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
|
2019-12-10 19:48:16 +00:00
|
|
|
has_inner_table = query.to_table_id.empty();
|
2019-12-10 19:48:16 +00:00
|
|
|
if (has_inner_table && !query.storage)
|
2017-10-30 17:53:01 +00:00
|
|
|
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);
|
2017-08-18 20:56:19 +00:00
|
|
|
|
2018-02-25 07:39:45 +00:00
|
|
|
if (query.select->list_of_selects->children.size() != 1)
|
2018-03-16 01:23:37 +00:00
|
|
|
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
|
|
|
|
2020-06-05 11:54:54 +00:00
|
|
|
auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), local_context);
|
|
|
|
setSelectQuery(select);
|
2020-01-31 17:12:18 +00:00
|
|
|
|
2019-12-10 19:48:16 +00:00
|
|
|
if (!has_inner_table)
|
2019-12-10 19:48:16 +00:00
|
|
|
target_table_id = query.to_table_id;
|
2019-12-10 19:48:16 +00:00
|
|
|
else if (attach_)
|
2017-10-21 20:08:49 +00:00
|
|
|
{
|
2019-12-10 19:48:16 +00:00
|
|
|
/// If there is an ATTACH request, then the internal table must already be created.
|
2020-03-18 17:38:52 +00:00
|
|
|
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID()));
|
2017-10-21 20:08:49 +00:00
|
|
|
}
|
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-06-25 00:17:08 +00:00
|
|
|
/// We will create a query to create an internal table.
|
2017-04-01 07:20:54 +00:00
|
|
|
auto manual_create_query = std::make_shared<ASTCreateQuery>();
|
2020-01-31 17:12:18 +00:00
|
|
|
manual_create_query->database = getStorageID().database_name;
|
2020-03-18 17:38:52 +00:00
|
|
|
manual_create_query->table = generateInnerTableName(getStorageID());
|
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);
|
2017-10-25 19:52:32 +00:00
|
|
|
manual_create_query->set(manual_create_query->storage, query.storage->ptr());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-12-10 19:48:16 +00:00
|
|
|
InterpreterCreateQuery create_interpreter(manual_create_query, local_context);
|
|
|
|
create_interpreter.setInternal(true);
|
|
|
|
create_interpreter.execute();
|
|
|
|
|
2020-05-28 23:01:18 +00:00
|
|
|
target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-12-10 19:48:16 +00:00
|
|
|
|
2020-06-05 11:54:54 +00:00
|
|
|
if (!select.select_table_id.empty())
|
|
|
|
DatabaseCatalog::instance().addDependency(select.select_table_id, getStorageID());
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2020-04-28 10:38:57 +00:00
|
|
|
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum to_stage, const ASTPtr & query_ptr) const
|
2018-04-19 14:47:09 +00:00
|
|
|
{
|
2020-04-01 18:38:01 +00:00
|
|
|
return getTargetTable()->getQueryProcessingStage(context, to_stage, query_ptr);
|
2018-04-19 14:47:09 +00:00
|
|
|
}
|
|
|
|
|
2020-02-19 16:07:28 +00:00
|
|
|
Pipes StorageMaterializedView::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,
|
2018-04-19 14:47:09 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2019-02-18 23:38:44 +00:00
|
|
|
const size_t max_block_size,
|
2017-06-02 15:54:39 +00:00
|
|
|
const unsigned num_streams)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2018-11-28 18:40:02 +00:00
|
|
|
auto storage = getTargetTable();
|
2020-04-09 18:10:27 +00:00
|
|
|
auto lock = storage->lockStructureForShare(
|
|
|
|
false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
|
|
|
|
2020-05-13 13:49:10 +00:00
|
|
|
if (query_info.order_optimizer)
|
|
|
|
query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage);
|
2019-12-11 13:09:46 +00:00
|
|
|
|
2020-02-19 16:02:37 +00:00
|
|
|
Pipes pipes = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
|
2020-01-31 15:16:06 +00:00
|
|
|
|
|
|
|
for (auto & pipe : pipes)
|
|
|
|
pipe.addTableLock(lock);
|
|
|
|
|
|
|
|
return pipes;
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2019-02-27 18:26:24 +00:00
|
|
|
BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Context & context)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2018-11-28 18:40:02 +00:00
|
|
|
auto storage = getTargetTable();
|
2020-04-09 18:10:27 +00:00
|
|
|
auto lock = storage->lockStructureForShare(
|
|
|
|
true, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2019-02-27 18:26:24 +00:00
|
|
|
auto stream = storage->write(query, context);
|
2018-11-28 18:40:02 +00:00
|
|
|
stream->addTableLock(lock);
|
|
|
|
return stream;
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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-05-28 23:01:18 +00:00
|
|
|
if (DatabaseCatalog::instance().tryGetTable(target_table_id, global_context))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-06-25 00:17:08 +00:00
|
|
|
/// We create and execute `drop` query for internal table.
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2020-04-12 18:52:59 +00:00
|
|
|
drop_query->no_delay = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTPtr ast_drop_query = drop_query;
|
2017-10-26 15:39:56 +00:00
|
|
|
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
|
2017-04-01 07:20:54 +00:00
|
|
|
drop_interpreter.execute();
|
|
|
|
}
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2018-06-09 15:48:22 +00:00
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
void StorageMaterializedView::drop()
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2019-12-03 16:25:32 +00:00
|
|
|
auto table_id = getStorageID();
|
2020-06-05 11:54:54 +00:00
|
|
|
const auto & select_query = getSelectQuery();
|
|
|
|
if (!select_query.select_table_id.empty())
|
|
|
|
DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id);
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2018-06-09 18:17:27 +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
|
|
|
}
|
|
|
|
|
2018-03-20 13:42:44 +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() + ". "
|
2018-03-20 13:42:44 +00:00
|
|
|
+ "Execute the statement directly on it.", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
}
|
|
|
|
|
2017-09-06 20:34:26 +00:00
|
|
|
bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
2015-04-10 17:09:16 +00:00
|
|
|
{
|
2018-03-20 13:42:44 +00:00
|
|
|
checkStatementCanBeForwarded();
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->optimize(query, partition, final, deduplicate, context);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2020-01-29 17:44:16 +00:00
|
|
|
void StorageMaterializedView::alter(
|
|
|
|
const AlterCommands & params,
|
|
|
|
const Context & context,
|
|
|
|
TableStructureWriteLockHolder & table_lock_holder)
|
|
|
|
{
|
2020-04-09 18:10:27 +00:00
|
|
|
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-01-29 17:44:16 +00:00
|
|
|
auto table_id = getStorageID();
|
2020-06-09 21:22:01 +00:00
|
|
|
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
|
|
|
params.apply(new_metadata, context);
|
2020-01-31 17:12:18 +00:00
|
|
|
|
|
|
|
/// start modify query
|
|
|
|
if (context.getSettingsRef().allow_experimental_alter_materialized_view_structure)
|
|
|
|
{
|
2020-06-09 21:22:01 +00:00
|
|
|
const auto & new_select = new_metadata.select;
|
2020-06-05 11:54:54 +00:00
|
|
|
const auto & old_select = getSelectQuery();
|
2020-01-31 17:12:18 +00:00
|
|
|
|
2020-06-05 11:54:54 +00:00
|
|
|
DatabaseCatalog::instance().updateDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id);
|
2020-01-31 17:12:18 +00:00
|
|
|
|
2020-06-05 11:54:54 +00:00
|
|
|
setSelectQuery(new_select);
|
2020-01-31 17:12:18 +00:00
|
|
|
}
|
|
|
|
/// end modify query
|
|
|
|
|
2020-06-09 21:22:01 +00:00
|
|
|
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
|
|
|
|
setColumns(std::move(new_metadata.columns));
|
2020-01-29 17:44:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-06-10 11:16:31 +00:00
|
|
|
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
2020-01-29 17:44:16 +00:00
|
|
|
{
|
|
|
|
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);
|
|
|
|
}
|
2020-01-29 17:44:16 +00:00
|
|
|
}
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-11-26 14:43:40 +00:00
|
|
|
void StorageMaterializedView::alterPartition(const ASTPtr & query, const PartitionCommands &commands, const Context &context)
|
2018-03-20 13:42:44 +00:00
|
|
|
{
|
|
|
|
checkStatementCanBeForwarded();
|
2018-11-26 14:43:40 +00:00
|
|
|
getTargetTable()->alterPartition(query, commands, context);
|
2018-03-20 13:42:44 +00:00
|
|
|
}
|
|
|
|
|
2018-08-20 16:28:30 +00:00
|
|
|
void StorageMaterializedView::mutate(const MutationCommands & commands, const Context & context)
|
|
|
|
{
|
|
|
|
checkStatementCanBeForwarded();
|
|
|
|
getTargetTable()->mutate(commands, context);
|
|
|
|
}
|
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
2019-05-07 06:54:55 +00:00
|
|
|
{
|
2020-03-18 17:38:52 +00:00
|
|
|
auto old_table_id = getStorageID();
|
2020-04-10 01:35:37 +00:00
|
|
|
bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID();
|
2020-04-07 14:05:51 +00:00
|
|
|
|
2020-04-10 01:35:37 +00:00
|
|
|
if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database)
|
2019-05-07 06:54:55 +00:00
|
|
|
{
|
2020-04-08 01:02:00 +00:00
|
|
|
auto new_target_table_name = generateInnerTableName(new_table_id);
|
2019-12-10 19:48:16 +00:00
|
|
|
auto rename = std::make_shared<ASTRenameQuery>();
|
2019-05-07 06:54:55 +00:00
|
|
|
|
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-05-07 06:54:55 +00:00
|
|
|
|
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-05-07 06:54:55 +00:00
|
|
|
|
2019-12-10 19:48:16 +00:00
|
|
|
ASTRenameQuery::Element elem;
|
|
|
|
elem.from = from;
|
|
|
|
elem.to = to;
|
|
|
|
rename->elements.emplace_back(elem);
|
2019-05-07 06:54:55 +00:00
|
|
|
|
2019-12-10 19:48:16 +00:00
|
|
|
InterpreterRenameQuery(rename, global_context).execute();
|
|
|
|
target_table_id.table_name = new_target_table_name;
|
2019-05-07 06:54:55 +00:00
|
|
|
}
|
2019-05-09 06:12:02 +00:00
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
IStorage::renameInMemory(new_table_id);
|
2020-06-05 11:54:54 +00:00
|
|
|
const auto & select_query = getSelectQuery();
|
2020-03-16 11:38:50 +00:00
|
|
|
// TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
|
2020-06-05 11:54:54 +00:00
|
|
|
DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID());
|
2019-05-07 06:54:55 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:41:06 +00:00
|
|
|
void StorageMaterializedView::shutdown()
|
|
|
|
{
|
2020-06-05 11:54:54 +00:00
|
|
|
const auto & select_query = getSelectQuery();
|
2017-12-16 01:41:06 +00:00
|
|
|
/// Make sure the dependency is removed after DETACH TABLE
|
2020-06-05 11:54:54 +00:00
|
|
|
if (!select_query.select_table_id.empty())
|
|
|
|
DatabaseCatalog::instance().removeDependency(select_query.select_table_id, getStorageID());
|
2017-12-16 01:41:06 +00:00
|
|
|
}
|
|
|
|
|
2017-10-21 20:08:49 +00:00
|
|
|
StoragePtr StorageMaterializedView::getTargetTable() const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2020-05-28 23:01:18 +00:00
|
|
|
return DatabaseCatalog::instance().getTable(target_table_id, global_context);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
|
|
|
|
2018-06-09 18:17:27 +00:00
|
|
|
StoragePtr StorageMaterializedView::tryGetTargetTable() const
|
|
|
|
{
|
2020-05-28 23:01:18 +00:00
|
|
|
return DatabaseCatalog::instance().tryGetTable(target_table_id, global_context);
|
2018-06-09 18:17:27 +00:00
|
|
|
}
|
|
|
|
|
2019-04-04 13:13:59 +00:00
|
|
|
Strings StorageMaterializedView::getDataPaths() const
|
2018-06-09 18:17:27 +00:00
|
|
|
{
|
|
|
|
if (auto table = tryGetTargetTable())
|
2019-04-04 13:13:59 +00:00
|
|
|
return table->getDataPaths();
|
2018-06-09 18:17:27 +00:00
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2018-08-03 09:39:01 +00:00
|
|
|
void StorageMaterializedView::checkTableCanBeDropped() const
|
2018-02-12 19:26:16 +00:00
|
|
|
{
|
|
|
|
/// Don't drop the target table if it was created manually via 'TO inner_table' statement
|
2018-06-09 18:17:27 +00:00
|
|
|
if (!has_inner_table)
|
2018-08-03 09:39:01 +00:00
|
|
|
return;
|
2018-06-09 18:17:27 +00:00
|
|
|
|
|
|
|
auto target_table = tryGetTargetTable();
|
|
|
|
if (!target_table)
|
2018-08-03 09:39:01 +00:00
|
|
|
return;
|
2018-06-09 18:17:27 +00:00
|
|
|
|
2018-08-03 09:39:01 +00:00
|
|
|
target_table->checkTableCanBeDropped();
|
2018-02-12 19:26:16 +00:00
|
|
|
}
|
|
|
|
|
2018-08-03 09:39:01 +00:00
|
|
|
void StorageMaterializedView::checkPartitionCanBeDropped(const ASTPtr & partition)
|
2018-08-01 17:41:18 +00:00
|
|
|
{
|
|
|
|
/// Don't drop the partition in target table if it was created manually via 'TO inner_table' statement
|
|
|
|
if (!has_inner_table)
|
2018-08-03 09:39:01 +00:00
|
|
|
return;
|
2018-08-01 17:41:18 +00:00
|
|
|
|
|
|
|
auto target_table = tryGetTargetTable();
|
|
|
|
if (!target_table)
|
2018-08-03 09:39:01 +00:00
|
|
|
return;
|
2018-08-01 17:41:18 +00:00
|
|
|
|
2018-08-03 09:39:01 +00:00
|
|
|
target_table->checkPartitionCanBeDropped(partition);
|
2018-08-01 17:41:18 +00:00
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2019-04-08 05:13:16 +00:00
|
|
|
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
|
|
|
|
{
|
|
|
|
return has_inner_table ? getTargetTable()->getActionLock(type) : ActionLock{};
|
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
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,
|
2018-03-06 20:18:34 +00:00
|
|
|
args.columns, args.attach);
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|