2017-07-25 21:07:05 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/ASTDropQuery.h>
|
2017-07-25 21:07:05 +00:00
|
|
|
#include <Parsers/ASTIdentifier.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>
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageMaterializedView.h>
|
|
|
|
#include <Storages/VirtualColumnFactory.h>
|
2013-11-15 09:43:50 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-10-24 19:32:23 +00:00
|
|
|
#include "StorageReplicatedMergeTree.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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-08-18 20:56:19 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-07-25 21:07:05 +00:00
|
|
|
static void extractDependentTable(const ASTSelectQuery & query, String & select_database_name, String & select_table_name)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2017-07-25 21:07:05 +00:00
|
|
|
auto query_table = query.table();
|
|
|
|
|
|
|
|
if (!query_table)
|
|
|
|
return;
|
|
|
|
|
2017-08-18 20:56:19 +00:00
|
|
|
if (auto ast_id = typeid_cast<const ASTIdentifier *>(query_table.get()))
|
2017-07-25 21:07:05 +00:00
|
|
|
{
|
|
|
|
auto query_database = query.database();
|
|
|
|
|
|
|
|
if (!query_database)
|
|
|
|
throw Exception("Logical error while creating StorageMaterializedView."
|
|
|
|
" Could not retrieve database name from select query.",
|
|
|
|
DB::ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
select_database_name = typeid_cast<const ASTIdentifier &>(*query_database).name;
|
|
|
|
select_table_name = ast_id->name;
|
|
|
|
}
|
2017-08-18 20:56:19 +00:00
|
|
|
else if (auto ast_select = typeid_cast<const ASTSelectQuery *>(query_table.get()))
|
2017-07-25 21:07:05 +00:00
|
|
|
{
|
|
|
|
extractDependentTable(*ast_select, select_database_name, select_table_name);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Logical error while creating StorageMaterializedView."
|
|
|
|
" Could not retrieve table name from select query.",
|
|
|
|
DB::ErrorCodes::LOGICAL_ERROR);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2017-07-25 21:07:05 +00:00
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageMaterializedView::StorageMaterializedView(
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & table_name_,
|
|
|
|
const String & database_name_,
|
2017-10-26 15:39:56 +00:00
|
|
|
Context & local_context,
|
2017-09-17 18:49:43 +00:00
|
|
|
const ASTCreateQuery & query,
|
2017-12-25 21:32:33 +00:00
|
|
|
const NamesAndTypes & columns_,
|
|
|
|
const NamesAndTypes & materialized_columns_,
|
|
|
|
const NamesAndTypes & alias_columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ColumnDefaults & column_defaults_,
|
|
|
|
bool attach_)
|
2017-07-25 21:07:05 +00:00
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_}, table_name(table_name_),
|
2017-10-26 15:39:56 +00:00
|
|
|
database_name(database_name_), global_context(local_context.getGlobalContext()), columns(columns_)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
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);
|
|
|
|
|
2017-10-30 17:53:01 +00:00
|
|
|
if (!query.storage && query.to_table.empty())
|
|
|
|
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
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
extractDependentTable(*query.select, select_database_name, select_table_name);
|
2017-07-25 21:07:05 +00:00
|
|
|
|
|
|
|
if (!select_table_name.empty())
|
2017-10-26 15:39:56 +00:00
|
|
|
global_context.addDependency(
|
2017-07-25 21:07:05 +00:00
|
|
|
DatabaseAndTableName(select_database_name, select_table_name),
|
|
|
|
DatabaseAndTableName(database_name, table_name));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-10-21 20:08:49 +00:00
|
|
|
// If the destination table is not set, use inner table
|
2017-10-30 17:53:01 +00:00
|
|
|
if (!query.to_table.empty())
|
2017-10-21 20:08:49 +00:00
|
|
|
{
|
2017-10-30 17:53:01 +00:00
|
|
|
target_database_name = query.to_database;
|
|
|
|
target_table_name = query.to_table;
|
2017-10-21 20:08:49 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
target_database_name = database_name;
|
|
|
|
target_table_name = ".inner." + table_name;
|
|
|
|
has_inner_table = true;
|
|
|
|
}
|
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
inner_query = query.select->ptr();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// If there is an ATTACH request, then the internal table must already be connected.
|
2017-10-21 20:08:49 +00:00
|
|
|
if (!attach_ && has_inner_table)
|
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>();
|
2017-10-21 20:08:49 +00:00
|
|
|
manual_create_query->database = target_database_name;
|
|
|
|
manual_create_query->table = target_table_name;
|
2017-09-17 18:49:43 +00:00
|
|
|
manual_create_query->set(manual_create_query->columns, query.columns->ptr());
|
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
|
|
|
|
|
|
|
/// Execute the query.
|
|
|
|
try
|
|
|
|
{
|
2017-10-26 15:39:56 +00:00
|
|
|
InterpreterCreateQuery create_interpreter(manual_create_query, local_context);
|
2017-04-01 07:20:54 +00:00
|
|
|
create_interpreter.execute();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2017-07-25 21:07:05 +00:00
|
|
|
/// In case of any error we should remove dependency to the view.
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!select_table_name.empty())
|
2017-10-26 15:39:56 +00:00
|
|
|
global_context.removeDependency(
|
2017-04-01 07:20:54 +00:00
|
|
|
DatabaseAndTableName(select_database_name, select_table_name),
|
|
|
|
DatabaseAndTableName(database_name, table_name));
|
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2017-12-25 21:32:33 +00:00
|
|
|
NameAndType StorageMaterializedView::getColumn(const String & column_name) const
|
2014-08-05 10:56:58 +00:00
|
|
|
{
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->getColumn(column_name);
|
2014-08-05 10:56:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool StorageMaterializedView::hasColumn(const String & column_name) const
|
|
|
|
{
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->hasColumn(column_name);
|
2014-08-05 10:56:58 +00:00
|
|
|
}
|
|
|
|
|
2013-11-08 17:43:03 +00:00
|
|
|
BlockInputStreams 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,
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
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
|
|
|
{
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2017-05-21 22:25:25 +00:00
|
|
|
BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Settings & settings)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->write(query, settings);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2014-03-20 13:28:49 +00:00
|
|
|
void StorageMaterializedView::drop()
|
|
|
|
{
|
2017-10-26 15:39:56 +00:00
|
|
|
global_context.removeDependency(
|
2017-04-01 07:20:54 +00:00
|
|
|
DatabaseAndTableName(select_database_name, select_table_name),
|
|
|
|
DatabaseAndTableName(database_name, table_name));
|
|
|
|
|
2017-10-27 20:21:27 +00:00
|
|
|
if (has_inner_table && global_context.tryGetTable(target_database_name, target_table_name))
|
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>();
|
2017-10-21 20:08:49 +00:00
|
|
|
drop_query->database = target_database_name;
|
|
|
|
drop_query->table = target_table_name;
|
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
|
|
|
}
|
|
|
|
|
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
|
|
|
{
|
2017-10-21 20:08:49 +00:00
|
|
|
return getTargetTable()->optimize(query, partition, final, deduplicate, context);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:41:06 +00:00
|
|
|
void StorageMaterializedView::shutdown()
|
|
|
|
{
|
|
|
|
/// Make sure the dependency is removed after DETACH TABLE
|
|
|
|
global_context.removeDependency(
|
|
|
|
DatabaseAndTableName(select_database_name, select_table_name),
|
|
|
|
DatabaseAndTableName(database_name, table_name));
|
|
|
|
}
|
|
|
|
|
2017-10-21 20:08:49 +00:00
|
|
|
StoragePtr StorageMaterializedView::getTargetTable() const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2017-10-27 20:21:27 +00:00
|
|
|
return global_context.getTable(target_database_name, target_table_name);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
|
|
|
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|