ClickHouse/src/Storages/StorageView.cpp

161 lines
5.8 KiB
C++
Raw Normal View History

#include <Interpreters/InterpreterSelectQuery.h>
2018-02-25 06:34:20 +00:00
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
2019-07-21 02:13:42 +00:00
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/Context.h>
2019-12-18 03:56:03 +00:00
#include <Interpreters/getTableExpressions.h>
2019-07-21 02:13:42 +00:00
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
2019-03-23 22:45:28 +00:00
#include <Parsers/ASTSelectWithUnionQuery.h>
2019-07-21 02:13:42 +00:00
#include <Parsers/queryToString.h>
#include <Storages/StorageView.h>
#include <Storages/StorageFactory.h>
#include <Common/typeid_cast.h>
2019-07-21 02:13:42 +00:00
#include <Processors/Pipe.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Transforms/MaterializingTransform.h>
2020-04-12 21:07:11 +00:00
#include <Processors/Transforms/ConvertingTransform.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR;
2019-12-18 03:56:03 +00:00
extern const int ALIAS_REQUIRED;
}
StorageView::StorageView(
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_)
2019-12-04 16:06:55 +00:00
: IStorage(table_id_)
{
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);
inner_query = query.select->ptr();
}
2014-06-12 23:21:38 +00:00
Pipes StorageView::read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum /*processed_stage*/,
2017-12-01 21:13:25 +00:00
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
Pipes pipes;
ASTPtr current_inner_query = inner_query;
if (context.getSettings().enable_optimize_predicate_expression)
current_inner_query = getRuntimeViewQuery(*query_info.query->as<const ASTSelectQuery>(), context);
QueryPipeline pipeline;
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
/// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return
if (query_info.force_tree_shaped_pipeline)
{
BlockInputStreams streams = interpreter.executeWithMultipleStreams(pipeline);
for (auto & stream : streams)
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::move(stream)));
}
else
/// TODO: support multiple streams here. Need more general interface than pipes.
pipes.emplace_back(interpreter.executeWithProcessors().getPipe());
2018-02-28 19:47:33 +00:00
/// 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.
for (auto & pipe : pipes)
2020-04-12 21:07:11 +00:00
{
pipe.addSimpleTransform(std::make_shared<MaterializingTransform>(pipe.getHeader()));
2018-02-28 19:47:33 +00:00
2020-04-12 21:07:11 +00:00
/// And also convert to expected structure.
pipe.addSimpleTransform(std::make_shared<ConvertingTransform>(
pipe.getHeader(), getSampleBlockForColumns(column_names),
2020-04-14 21:05:45 +00:00
ConvertingTransform::MatchColumnsMode::Name));
2020-04-12 21:07:11 +00:00
}
return pipes;
}
ASTPtr StorageView::getRuntimeViewQuery(const ASTSelectQuery & outer_query, const Context & context)
{
auto temp_outer_query = outer_query.clone();
auto * new_outer_select = temp_outer_query->as<ASTSelectQuery>();
return getRuntimeViewQuery(new_outer_select, context, false);
}
2020-03-18 00:57:00 +00:00
static void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
{
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);
auto * table_expression = select_element->table_expression->as<ASTTableExpression>();
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
const auto alias = table_expression->database_and_table_name->tryGetAlias();
table_expression->database_and_table_name = {};
table_expression->subquery = std::make_shared<ASTSubquery>();
table_expression->subquery->children.push_back(subquery);
table_expression->children.push_back(table_expression->subquery);
if (!alias.empty())
table_expression->subquery->setAlias(alias);
}
ASTPtr StorageView::getRuntimeViewQuery(ASTSelectQuery * outer_query, const Context & context, bool normalize)
{
auto runtime_view_query = inner_query->clone();
/// TODO: remove getTableExpressions and getTablesWithColumns
{
const auto & table_expressions = getTableExpressions(*outer_query);
const auto & tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context);
replaceTableNameWithSubquery(outer_query, runtime_view_query);
if (context.getSettingsRef().joined_subquery_requires_alias && tables_with_columns.size() > 1)
{
2020-04-22 06:22:14 +00:00
for (const auto & pr : tables_with_columns)
if (pr.table.table.empty() && pr.table.alias.empty())
throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).",
ErrorCodes::ALIAS_REQUIRED);
}
if (PredicateExpressionsOptimizer(context, tables_with_columns, context.getSettings()).optimize(*outer_query) && normalize)
InterpreterSelectWithUnionQuery(
runtime_view_query, context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify(), {});
}
return runtime_view_query;
}
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);
});
}
}