ClickHouse/src/Storages/StorageView.cpp

236 lines
8.7 KiB
C++

#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Storages/StorageView.h>
#include <Storages/StorageFactory.h>
#include <Storages/SelectQueryDescription.h>
#include <Common/typeid_cast.h>
#include <Processors/Pipe.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR;
}
namespace
{
void addSettingsChanges(ASTPtr ast, const Settings & settings)
{
auto * settings_ast = ast->as<ASTSetQuery>();
if (!settings_ast)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ASTSetQuery expected");
settings_ast->is_standalone = false;
if (settings_ast->changes.tryGet("join_use_nulls") == nullptr)
settings_ast->changes.emplace_back("join_use_nulls", Field(settings.join_use_nulls));
}
/// Save to AST settings from context that affects view behaviour.
void saveSettingsToAst(ASTSelectWithUnionQuery * select, const Settings & settings)
{
/// Check SETTINGS section on the top level
if (select->settings_ast)
{
addSettingsChanges(select->settings_ast, settings);
return;
}
/// We cannot add SETTINGS on the top level because it will clash with section from inner SELECT
/// and will got query: SELECT ... SETTINGS ... SETTINGS ...
/// Process every select in ast and add SETTINGS section to each
for (const auto & child : select->list_of_selects->children)
{
auto * child_select = child->as<ASTSelectQuery>();
if (!child_select)
continue;
ASTPtr ast_set_query = child_select->settings();
if (ast_set_query)
{
/// Modify existing SETTINGS section
addSettingsChanges(ast_set_query, settings);
}
else
{
/// Add SETTINGS section to query
ast_set_query = std::make_shared<ASTSetQuery>();
addSettingsChanges(ast_set_query, settings);
child_select->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(ast_set_query));
}
}
}
}
StorageView::StorageView(
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
const String & comment,
const Settings & settings)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setComment(comment);
if (!query.select)
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
saveSettingsToAst(query.select, settings);
SelectQueryDescription description;
description.inner_query = query.select->ptr();
storage_metadata.setSelectQuery(description);
setInMemoryMetadata(storage_metadata);
}
Pipe StorageView::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
const size_t max_block_size,
const unsigned num_streams)
{
QueryPlan plan;
read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
return plan.convertToPipe(
QueryPlanOptimizationSettings::fromContext(context),
BuildQueryPipelineSettings::fromContext(context));
}
void StorageView::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
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();
}
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
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.
auto materializing_actions = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
materializing_actions->addMaterializingOutputActions();
auto materializing = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(materializing_actions));
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());
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
converting->setStepDescription("Convert VIEW subquery result to VIEW table structure");
query_plan.addStep(std::move(converting));
}
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)
{
if (!select_query.tables() || select_query.tables()->children.empty())
throw Exception("Logical error: no table expression in view select AST", ErrorCodes::LOGICAL_ERROR);
auto * select_element = select_query.tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
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);
if (!table_expression->database_and_table_name)
{
// 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);
}
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;
table_expression->database_and_table_name = {};
table_expression->subquery = std::make_shared<ASTSubquery>();
table_expression->subquery->children.push_back(view_query);
table_expression->subquery->setAlias(alias);
for (auto & child : table_expression->children)
if (child.get() == view_name.get())
child = view_query;
}
ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name)
{
ASTTableExpression * table_expression = getFirstTableExpression(select_query);
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;
for (auto & child : table_expression->children)
if (child.get() == subquery.get())
child = view_name;
return subquery->children[0];
}
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);
return StorageView::create(args.table_id, args.query, args.columns, args.comment, args.getLocalContext()->getSettingsRef());
});
}
}