Change push down logic in VIEW (#11513)

This commit is contained in:
Artem Zuikov 2020-06-15 15:36:10 +03:00 committed by GitHub
parent 10566e2b43
commit 24059efad5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 171 additions and 130 deletions

View File

@ -202,11 +202,11 @@ private:
{ {
std::optional<size_t> left_table_pos = IdentifierSemantic::getMembership(left); std::optional<size_t> left_table_pos = IdentifierSemantic::getMembership(left);
if (!left_table_pos) if (!left_table_pos)
left_table_pos = IdentifierSemantic::chooseTable(left, tables); left_table_pos = IdentifierSemantic::chooseTableColumnMatch(left, tables);
std::optional<size_t> right_table_pos = IdentifierSemantic::getMembership(right); std::optional<size_t> right_table_pos = IdentifierSemantic::getMembership(right);
if (!right_table_pos) if (!right_table_pos)
right_table_pos = IdentifierSemantic::chooseTable(right, tables); right_table_pos = IdentifierSemantic::chooseTableColumnMatch(right, tables);
if (left_table_pos && right_table_pos && (*left_table_pos != *right_table_pos)) if (left_table_pos && right_table_pos && (*left_table_pos != *right_table_pos))
{ {

View File

@ -16,7 +16,8 @@ namespace
{ {
template <typename T> template <typename T>
std::optional<size_t> tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables, bool allow_ambiguous) std::optional<size_t> tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables,
bool allow_ambiguous, bool column_match [[maybe_unused]] = false)
{ {
using ColumnMatch = IdentifierSemantic::ColumnMatch; using ColumnMatch = IdentifierSemantic::ColumnMatch;
@ -27,6 +28,13 @@ std::optional<size_t> tryChooseTable(const ASTIdentifier & identifier, const std
for (size_t i = 0; i < tables.size(); ++i) for (size_t i = 0; i < tables.size(); ++i)
{ {
auto match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i]); auto match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i]);
if constexpr (std::is_same_v<T, TableWithColumnNamesAndTypes>)
{
if (column_match && match == ColumnMatch::NoMatch && identifier.isShort() && tables[i].hasColumn(identifier.shortName()))
match = ColumnMatch::ColumnName;
}
if (match != ColumnMatch::NoMatch) if (match != ColumnMatch::NoMatch)
{ {
if (match > best_match) if (match > best_match)
@ -125,12 +133,17 @@ std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & iden
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, ambiguous); return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, ambiguous);
} }
std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNamesAndTypes> & tables, std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const TablesWithColumns & tables, bool ambiguous)
bool ambiguous)
{ {
return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous); return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous);
} }
std::optional<size_t> IdentifierSemantic::chooseTableColumnMatch(const ASTIdentifier & identifier, const TablesWithColumns & tables,
bool ambiguous)
{
return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous, true);
}
StorageID IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier) StorageID IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
{ {
if (identifier.name_parts.size() > 2) if (identifier.name_parts.size() > 2)
@ -191,14 +204,9 @@ IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const
} }
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
const TableWithColumnNamesAndTypes & db_and_table) const TableWithColumnNamesAndTypes & table_with_columns)
{ {
ColumnMatch match = canReferColumnToTable(identifier, db_and_table.table); return canReferColumnToTable(identifier, table_with_columns.table);
#if 0
if (match == ColumnMatch::NoMatch && identifier.isShort() && db_and_table.hasColumn(identifier.shortName()))
match = ColumnMatch::ColumnName;
#endif
return match;
} }
/// Strip qualificators from left side of column name. /// Strip qualificators from left side of column name.

View File

@ -41,7 +41,7 @@ struct IdentifierSemantic
static std::optional<String> extractNestedName(const ASTIdentifier & identifier, const String & table_name); static std::optional<String> extractNestedName(const ASTIdentifier & identifier, const String & table_name);
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const TableWithColumnNamesAndTypes & db_and_table); static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const TableWithColumnNamesAndTypes & table_with_columns);
static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
@ -52,7 +52,9 @@ struct IdentifierSemantic
static std::optional<size_t> getMembership(const ASTIdentifier & identifier); static std::optional<size_t> getMembership(const ASTIdentifier & identifier);
static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables, static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables,
bool allow_ambiguous = false); bool allow_ambiguous = false);
static std::optional<size_t> chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNamesAndTypes> & tables, static std::optional<size_t> chooseTable(const ASTIdentifier &, const TablesWithColumns & tables,
bool allow_ambiguous = false);
static std::optional<size_t> chooseTableColumnMatch(const ASTIdentifier &, const TablesWithColumns & tables,
bool allow_ambiguous = false); bool allow_ambiguous = false);
private: private:

View File

@ -4,20 +4,15 @@
#include <DataStreams/OneBlockInputStream.h> #include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Interpreters/InDepthNodeVisitor.h> #include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h> #include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Parsers/DumpASTNode.h> #include <Parsers/DumpASTNode.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTExplainQuery.h> #include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Core/Field.h>
#include <Common/typeid_cast.h>
#include <Storages/StorageView.h> #include <Storages/StorageView.h>
#include <sstream> #include <sstream>
@ -31,56 +26,30 @@ namespace
{ {
struct Data struct Data
{ {
bool analyzed = false;
const Context & context; const Context & context;
}; };
static bool needChildVisit(ASTPtr &, ASTPtr &) { return true; } static bool needChildVisit(ASTPtr & node, ASTPtr &)
{
return !node->as<ASTSelectQuery>();
}
static void visit(ASTPtr & ast, Data & data) static void visit(ASTPtr & ast, Data & data)
{ {
if (auto * select_query = ast->as<ASTSelectQuery>()) if (auto * select = ast->as<ASTSelectQuery>())
visit(*select_query, ast, data); visit(*select, ast, data);
if (auto * union_select_query = ast->as<ASTSelectWithUnionQuery>())
visit(*union_select_query, ast, data);
} }
static void visit(ASTSelectQuery & select_query, ASTPtr &, Data & data) static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
{ {
if (!select_query.tables()) InterpreterSelectQuery interpreter(
return;
for (const auto & child : select_query.tables()->children)
{
auto * tables_element = child->as<ASTTablesInSelectQueryElement>();
if (tables_element && tables_element->table_expression)
visit(*tables_element->table_expression->as<ASTTableExpression>(), select_query, data);
}
}
static void visit(ASTSelectWithUnionQuery &, ASTPtr & node, Data & data)
{
if (!data.analyzed)
{
data.analyzed = true;
InterpreterSelectWithUnionQuery interpreter(
node, data.context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify()); node, data.context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify());
}
}
static void visit(ASTTableExpression & expression, ASTSelectQuery & select_query, Data & data) const SelectQueryInfo & query_info = interpreter.getQueryInfo();
if (query_info.view_query)
{ {
if (data.context.getSettingsRef().enable_optimize_predicate_expression && expression.database_and_table_name) ASTPtr tmp;
{ StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp);
if (const auto * identifier = expression.database_and_table_name->as<ASTIdentifier>())
{
auto table_id = data.context.resolveStorageID(*identifier);
const auto & storage = DatabaseCatalog::instance().getTable(table_id, data.context);
if (auto * storage_view = dynamic_cast<StorageView *>(storage.get()))
storage_view->getRuntimeViewQuery(&select_query, data.context, true);
}
} }
} }
}; };

View File

@ -30,6 +30,7 @@
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h> #include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Storages/StorageView.h>
#include <TableFunctions/ITableFunction.h> #include <TableFunctions/ITableFunction.h>
@ -186,6 +187,26 @@ static Context getSubqueryContext(const Context & context)
return subquery_context; return subquery_context;
} }
static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings)
{
ASTSelectQuery & select = query->as<ASTSelectQuery &>();
Aliases aliases;
if (ASTPtr with = select.with())
QueryAliasesNoSubqueriesVisitor(aliases).visit(with);
QueryAliasesNoSubqueriesVisitor(aliases).visit(select.select());
CrossToInnerJoinVisitor::Data cross_to_inner{tables, aliases, database};
CrossToInnerJoinVisitor(cross_to_inner).visit(query);
size_t rewriter_version = settings.multiple_joins_rewriter_version;
if (!rewriter_version || rewriter_version > 2)
throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(),
ErrorCodes::INVALID_SETTING_VALUE);
JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases, rewriter_version};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query);
}
InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_, const ASTPtr & query_ptr_,
const Context & context_, const Context & context_,
@ -242,29 +263,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(
/// Rewrite JOINs /// Rewrite JOINs
if (!has_input && joined_tables.tablesCount() > 1) if (!has_input && joined_tables.tablesCount() > 1)
{ {
ASTSelectQuery & select = getSelectQuery(); rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase(), settings);
Aliases aliases; joined_tables.reset(getSelectQuery());
if (ASTPtr with = select.with())
QueryAliasesNoSubqueriesVisitor(aliases).visit(with);
QueryAliasesNoSubqueriesVisitor(aliases).visit(select.select());
CrossToInnerJoinVisitor::Data cross_to_inner{joined_tables.tablesWithColumns(), aliases, context->getCurrentDatabase()};
CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr);
size_t rewriter_version = settings.multiple_joins_rewriter_version;
if (!rewriter_version || rewriter_version > 2)
throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(),
ErrorCodes::INVALID_SETTING_VALUE);
JoinToSubqueryTransformVisitor::Data join_to_subs_data{joined_tables.tablesWithColumns(), aliases, rewriter_version};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
joined_tables.reset(select);
joined_tables.resolveTables(); joined_tables.resolveTables();
if (storage && joined_tables.isLeftTableSubquery()) if (storage && joined_tables.isLeftTableSubquery())
{ {
/// Rewritten with subquery. Free storage here locks here. /// Rewritten with subquery. Free storage locks here.
storage = {}; storage = {};
table_lock.release(); table_lock.release();
table_id = StorageID::createEmpty(); table_id = StorageID::createEmpty();
@ -288,12 +294,28 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (storage) if (storage)
row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
StorageView * view = nullptr;
if (storage)
view = dynamic_cast<StorageView *>(storage.get());
auto analyze = [&] (bool try_move_to_prewhere) auto analyze = [&] (bool try_move_to_prewhere)
{ {
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
ASTPtr view_table;
if (view)
view->replaceWithSubquery(getSelectQuery(), view_table);
syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect( syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect(
query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage), query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage),
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join); options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
if (view)
{
/// Restore original view name. Save rewritten subquery for future usage in StorageView.
query_info.view_query = view->restoreViewName(getSelectQuery(), view_table);
view = nullptr;
}
if (try_move_to_prewhere && storage && !row_policy_filter && query.where() && !query.prewhere() && !query.final()) if (try_move_to_prewhere && storage && !row_policy_filter && query.where() && !query.prewhere() && !query.final())
{ {
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable

View File

@ -88,6 +88,8 @@ public:
size_t getMaxStreams() const { return max_streams; } size_t getMaxStreams() const { return max_streams; }
const SelectQueryInfo & getQueryInfo() const { return query_info; }
private: private:
InterpreterSelectQuery( InterpreterSelectQuery(
const ASTPtr & query_ptr_, const ASTPtr & query_ptr_,

View File

@ -585,8 +585,9 @@ std::vector<TableNeededColumns> normalizeColumnNamesExtractNeeded(
for (ASTIdentifier * ident : identifiers) for (ASTIdentifier * ident : identifiers)
{ {
bool got_alias = aliases.count(ident->name); bool got_alias = aliases.count(ident->name);
bool allow_ambiguous = got_alias; /// allow ambiguous column overridden by an alias
if (auto table_pos = IdentifierSemantic::chooseTable(*ident, tables)) if (auto table_pos = IdentifierSemantic::chooseTableColumnMatch(*ident, tables, allow_ambiguous))
{ {
if (!ident->isShort()) if (!ident->isShort())
{ {

View File

@ -34,7 +34,8 @@ public:
void makeFakeTable(StoragePtr storage, const Block & source_header); void makeFakeTable(StoragePtr storage, const Block & source_header);
std::shared_ptr<TableJoin> makeTableJoin(const ASTSelectQuery & select_query); std::shared_ptr<TableJoin> makeTableJoin(const ASTSelectQuery & select_query);
const std::vector<TableWithColumnNamesAndTypes> & tablesWithColumns() const { return tables_with_columns; } const TablesWithColumns & tablesWithColumns() const { return tables_with_columns; }
TablesWithColumns moveTablesWithColumns() { return std::move(tables_with_columns); }
bool isLeftTableSubquery() const; bool isLeftTableSubquery() const;
bool isLeftTableFunction() const; bool isLeftTableFunction() const;
@ -49,7 +50,7 @@ public:
private: private:
Context context; Context context;
std::vector<const ASTTableExpression *> table_expressions; std::vector<const ASTTableExpression *> table_expressions;
std::vector<TableWithColumnNamesAndTypes> tables_with_columns; TablesWithColumns tables_with_columns;
/// Legacy (duplicated left table values) /// Legacy (duplicated left table values)
ASTPtr left_table_expression; ASTPtr left_table_expression;

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Interpreters/PreparedSets.h> #include <Interpreters/PreparedSets.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Core/SortDescription.h> #include <Core/SortDescription.h>
#include <Core/Names.h> #include <Core/Names.h>
#include <memory> #include <memory>
@ -70,6 +71,7 @@ using ReadInOrderOptimizerPtr = std::shared_ptr<const ReadInOrderOptimizer>;
struct SelectQueryInfo struct SelectQueryInfo
{ {
ASTPtr query; ASTPtr query;
ASTPtr view_query; /// Optimized VIEW query
SyntaxAnalyzerResultPtr syntax_analyzer_result; SyntaxAnalyzerResultPtr syntax_analyzer_result;

View File

@ -1,8 +1,6 @@
#include <Interpreters/InterpreterSelectQuery.h> #include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h> #include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/getTableExpressions.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
@ -30,7 +28,6 @@ namespace ErrorCodes
{ {
extern const int INCORRECT_QUERY; extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int ALIAS_REQUIRED;
} }
@ -60,9 +57,12 @@ Pipes StorageView::read(
Pipes pipes; Pipes pipes;
ASTPtr current_inner_query = inner_query; ASTPtr current_inner_query = inner_query;
if (query_info.view_query)
if (context.getSettings().enable_optimize_predicate_expression) {
current_inner_query = getRuntimeViewQuery(*query_info.query->as<const ASTSelectQuery>(), context); 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); InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
@ -87,60 +87,52 @@ Pipes StorageView::read(
return pipes; return pipes;
} }
ASTPtr StorageView::getRuntimeViewQuery(const ASTSelectQuery & outer_query, const Context & context) static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)
{ {
auto temp_outer_query = outer_query.clone(); auto * select_element = select_query.tables()->children[0]->as<ASTTablesInSelectQueryElement>();
auto * new_outer_select = temp_outer_query->as<ASTSelectQuery>();
return getRuntimeViewQuery(new_outer_select, context, false);
}
static void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
{
auto * select_element = select_query->tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression) if (!select_element->table_expression)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
auto * table_expression = select_element->table_expression->as<ASTTableExpression>(); 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 (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
const auto alias = table_expression->database_and_table_name->tryGetAlias(); 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->database_and_table_name = {};
table_expression->subquery = std::make_shared<ASTSubquery>(); table_expression->subquery = std::make_shared<ASTSubquery>();
table_expression->subquery->children.push_back(subquery); table_expression->subquery->children.push_back(view_query);
table_expression->children.push_back(table_expression->subquery);
if (!alias.empty())
table_expression->subquery->setAlias(alias); 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)
ASTPtr StorageView::getRuntimeViewQuery(ASTSelectQuery * outer_query, const Context & context, bool normalize)
{ {
auto runtime_view_query = inner_query->clone(); ASTTableExpression * table_expression = getFirstTableExpression(select_query);
/// TODO: remove getTableExpressions and getTablesWithColumns if (!table_expression->subquery)
{ throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
const auto & table_expressions = getTableExpressions(*outer_query);
const auto & tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context);
replaceTableNameWithSubquery(outer_query, runtime_view_query); ASTPtr subquery = table_expression->subquery;
if (context.getSettingsRef().joined_subquery_requires_alias && tables_with_columns.size() > 1) table_expression->subquery = {};
{ table_expression->database_and_table_name = view_name;
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) for (auto & child : table_expression->children)
InterpreterSelectWithUnionQuery( if (child.get() == subquery.get())
runtime_view_query, context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify(), {}); child = view_name;
} return subquery->children[0];
return runtime_view_query;
} }
void registerStorageView(StorageFactory & factory) void registerStorageView(StorageFactory & factory)

View File

@ -29,9 +29,13 @@ public:
size_t max_block_size, size_t max_block_size,
unsigned num_streams) override; unsigned num_streams) override;
ASTPtr getRuntimeViewQuery(const ASTSelectQuery & outer_query, const Context & context); void replaceWithSubquery(ASTSelectQuery & select_query, ASTPtr & view_name) const
{
replaceWithSubquery(select_query, inner_query->clone(), view_name);
}
ASTPtr getRuntimeViewQuery(ASTSelectQuery * outer_query, const Context & context, bool normalize); static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name);
static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name);
private: private:
ASTPtr inner_query; ASTPtr inner_query;

View File

@ -1,4 +1,4 @@
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM default.test\n WHERE id = 1\n)\nWHERE id = 1 SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS test_view\nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM default.test\n WHERE id = 2\n)\nWHERE id = 2 SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 2\n) AS test_view\nWHERE id = 2
SELECT id\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM default.test\n WHERE id = 1\n)\nWHERE id = 1 SELECT id\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS test_view\nWHERE id = 1
SELECT id\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM default.test\n WHERE id = 1\n) AS s\nWHERE id = 1 SELECT id\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS s\nWHERE id = 1

View File

@ -0,0 +1,3 @@
1 1 1
2 2 0
1 val11 val21 val31

View File

@ -0,0 +1,35 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
DROP TABLE IF EXISTS view1;
CREATE TABLE t1 (id UInt32, value1 String) ENGINE MergeTree() ORDER BY id;
CREATE TABLE t2 (id UInt32, value2 String) ENGINE MergeTree() ORDER BY id;
CREATE TABLE t3 (id UInt32, value3 String) ENGINE MergeTree() ORDER BY id;
INSERT INTO t1 (id, value1) VALUES (1, 'val11');
INSERT INTO t2 (id, value2) VALUES (1, 'val21');
INSERT INTO t3 (id, value3) VALUES (1, 'val31');
SET multiple_joins_rewriter_version = 2;
SET enable_optimize_predicate_expression = 1;
SELECT t1.id, t2.id as id, t3.id as value
FROM (select number as id, 42 as value from numbers(4)) t1
LEFT JOIN (select number as id, 42 as value from numbers(3)) t2 ON t1.id = t2.id
LEFT JOIN (select number as id, 42 as value from numbers(2)) t3 ON t1.id = t3.id
WHERE id > 0 AND value < 42;
CREATE VIEW IF NOT EXISTS view1 AS
SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3
FROM t1
LEFT JOIN t2 ON t1.id = t2.id
LEFT JOIN t3 ON t1.id = t3.id
WHERE t1.id > 0;
SELECT * FROM view1 WHERE id = 1;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
DROP TABLE IF EXISTS view1;