2018-08-16 02:55:52 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2018-02-25 06:34:20 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2019-07-21 02:13:42 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2018-08-16 02:55:52 +00:00
|
|
|
#include <Parsers/ASTSubquery.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2019-03-23 22:45:28 +00:00
|
|
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageView.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2020-06-05 11:54:54 +00:00
|
|
|
#include <Storages/SelectQueryDescription.h>
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2018-08-16 02:55:52 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-07-21 02:13:42 +00:00
|
|
|
|
2020-02-17 15:27:51 +00:00
|
|
|
#include <Processors/Pipe.h>
|
|
|
|
#include <Processors/Transforms/MaterializingTransform.h>
|
2020-11-17 17:16:55 +00:00
|
|
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
2020-09-17 13:22:24 +00:00
|
|
|
#include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
|
2021-03-04 17:38:12 +00:00
|
|
|
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
|
|
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
2013-10-30 13:52:02 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-09-19 19:18:34 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2018-08-16 02:55:52 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2013-10-30 13:52:02 +00:00
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageView::StorageView(
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2017-09-17 18:49:43 +00:00
|
|
|
const ASTCreateQuery & query,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_)
|
2013-11-08 17:43:03 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2017-09-19 19:18:34 +00:00
|
|
|
if (!query.select)
|
|
|
|
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2020-06-05 11:54:54 +00:00
|
|
|
SelectQueryDescription description;
|
|
|
|
|
|
|
|
description.inner_query = query.select->ptr();
|
2020-06-19 15:39:41 +00:00
|
|
|
storage_metadata.setSelectQuery(description);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2014-06-12 23:21:38 +00:00
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageView::read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
2020-09-18 11:39:07 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
const size_t max_block_size,
|
|
|
|
const unsigned num_streams)
|
2013-10-30 13:52:02 +00:00
|
|
|
{
|
2020-09-18 11:39:07 +00:00
|
|
|
QueryPlan plan;
|
|
|
|
read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
2021-03-04 17:38:12 +00:00
|
|
|
return plan.convertToPipe(
|
|
|
|
QueryPlanOptimizationSettings::fromContext(context),
|
|
|
|
BuildQueryPipelineSettings::fromContext(context));
|
2013-11-08 17:43:03 +00:00
|
|
|
}
|
|
|
|
|
2020-09-17 13:22:24 +00:00
|
|
|
void StorageView::read(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
2020-09-18 11:39:07 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-11-10 12:02:22 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2020-09-18 11:39:07 +00:00
|
|
|
const Context & context,
|
2020-09-17 13:22:24 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
const size_t /*max_block_size*/,
|
|
|
|
const unsigned /*num_streams*/)
|
|
|
|
{
|
|
|
|
ASTPtr current_inner_query = metadata_snapshot->getSelectQuery().inner_query;
|
|
|
|
|
|
|
|
if (query_info.view_query)
|
|
|
|
{
|
|
|
|
if (!query_info.view_query->as<ASTSelectWithUnionQuery>())
|
|
|
|
throw Exception("Unexpected optimized VIEW query", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
current_inner_query = query_info.view_query->clone();
|
|
|
|
}
|
|
|
|
|
2020-09-18 11:39:07 +00:00
|
|
|
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
|
2020-09-17 13:22:24 +00:00
|
|
|
interpreter.buildQueryPlan(query_plan);
|
|
|
|
|
|
|
|
/// It's expected that the columns read from storage are not constant.
|
|
|
|
/// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery.
|
2021-02-18 11:15:16 +00:00
|
|
|
auto materializing_actions = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
|
|
|
|
materializing_actions->addMaterializingOutputActions();
|
|
|
|
|
2021-03-04 17:38:12 +00:00
|
|
|
auto materializing = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(materializing_actions));
|
2020-09-17 13:22:24 +00:00
|
|
|
materializing->setStepDescription("Materialize constants after VIEW subquery");
|
|
|
|
query_plan.addStep(std::move(materializing));
|
|
|
|
|
|
|
|
/// And also convert to expected structure.
|
|
|
|
auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
|
2020-11-17 17:16:55 +00:00
|
|
|
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
|
|
|
|
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
|
|
|
|
header.getColumnsWithTypeAndName(),
|
|
|
|
ActionsDAG::MatchColumnsMode::Name);
|
|
|
|
|
2021-03-04 17:38:12 +00:00
|
|
|
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
|
2020-09-17 13:22:24 +00:00
|
|
|
converting->setStepDescription("Convert VIEW subquery result to VIEW table structure");
|
|
|
|
query_plan.addStep(std::move(converting));
|
|
|
|
}
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)
|
2020-03-18 00:57:00 +00:00
|
|
|
{
|
2021-02-25 14:43:58 +00:00
|
|
|
if (!select_query.tables() || select_query.tables()->children.empty())
|
|
|
|
throw Exception("Logical error: no table expression in view select AST", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
auto * select_element = select_query.tables()->children[0]->as<ASTTablesInSelectQueryElement>();
|
2020-03-18 00:57:00 +00:00
|
|
|
|
|
|
|
if (!select_element->table_expression)
|
|
|
|
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
return select_element->table_expression->as<ASTTableExpression>();
|
|
|
|
}
|
|
|
|
|
|
|
|
void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name)
|
|
|
|
{
|
|
|
|
ASTTableExpression * table_expression = getFirstTableExpression(outer_query);
|
2020-03-18 00:57:00 +00:00
|
|
|
|
|
|
|
if (!table_expression->database_and_table_name)
|
2020-08-28 14:07:14 +00:00
|
|
|
{
|
|
|
|
// If it's a view table function, add a fake db.table name.
|
|
|
|
if (table_expression->table_function && table_expression->table_function->as<ASTFunction>()->name == "view")
|
|
|
|
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>("__view");
|
|
|
|
else
|
|
|
|
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
2020-03-18 00:57:00 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name);
|
|
|
|
String alias = db_table.alias.empty() ? db_table.table : db_table.alias;
|
|
|
|
|
|
|
|
view_name = table_expression->database_and_table_name;
|
2020-03-18 00:57:00 +00:00
|
|
|
table_expression->database_and_table_name = {};
|
|
|
|
table_expression->subquery = std::make_shared<ASTSubquery>();
|
2020-06-15 12:36:10 +00:00
|
|
|
table_expression->subquery->children.push_back(view_query);
|
|
|
|
table_expression->subquery->setAlias(alias);
|
2020-03-18 00:57:00 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
for (auto & child : table_expression->children)
|
|
|
|
if (child.get() == view_name.get())
|
|
|
|
child = view_query;
|
|
|
|
}
|
2020-03-18 00:57:00 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
ASTTableExpression * table_expression = getFirstTableExpression(select_query);
|
2020-02-05 08:22:25 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
if (!table_expression->subquery)
|
|
|
|
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
ASTPtr subquery = table_expression->subquery;
|
|
|
|
table_expression->subquery = {};
|
|
|
|
table_expression->database_and_table_name = view_name;
|
2020-02-05 08:22:25 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
for (auto & child : table_expression->children)
|
|
|
|
if (child.get() == subquery.get())
|
|
|
|
child = view_name;
|
|
|
|
return subquery->children[0];
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
void registerStorageView(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("View", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
if (args.query.storage)
|
|
|
|
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2019-12-04 16:06:55 +00:00
|
|
|
return StorageView::create(args.table_id, args.query, args.columns);
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2013-10-30 13:52:02 +00:00
|
|
|
}
|