This commit is contained in:
Amos Bird 2020-09-13 01:00:04 +08:00
parent 87b3984d17
commit 882b2a3348
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
13 changed files with 266 additions and 1 deletions

View File

@ -0,0 +1,90 @@
#include <Interpreters/ApplyWithSubqueryVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/StorageID.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTWithElement.h>
namespace DB
{
void ApplyWithSubqueryVisitor::visit(ASTPtr & ast, const Data & data)
{
if (auto * node_select = ast->as<ASTSelectQuery>())
{
auto with = node_select->with();
std::optional<Data> new_data;
if (with)
{
for (auto & child : with->children)
visit(child, data);
for (auto & child : with->children)
{
if (auto * ast_with_elem = child->as<ASTWithElement>())
{
if (!new_data)
new_data = data;
new_data->subqueries[ast_with_elem->name] = ast_with_elem->subquery;
}
}
}
for (auto & child : node_select->children)
{
if (child != with)
visit(child, new_data ? *new_data : data);
}
return;
}
for (auto & child : ast->children)
visit(child, data);
if (auto * node_func = ast->as<ASTFunction>())
visit(*node_func, data);
else if (auto * node_table = ast->as<ASTTableExpression>())
visit(*node_table, data);
}
void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & data)
{
if (table.database_and_table_name)
{
auto table_id = IdentifierSemantic::extractDatabaseAndTable(table.database_and_table_name->as<ASTIdentifier &>());
if (table_id.database_name.empty())
{
auto subquery_it = data.subqueries.find(table_id.table_name);
if (subquery_it != data.subqueries.end())
{
table.children.clear();
table.database_and_table_name.reset();
table.subquery = subquery_it->second->clone();
dynamic_cast<ASTWithAlias &>(*table.subquery).alias = table_id.table_name;
table.children.emplace_back(table.subquery);
}
}
}
}
void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data)
{
if (checkFunctionIsInOrGlobalInOperator(func))
{
auto & ast = func.arguments->children.at(1);
if (const auto * ident = ast->as<ASTIdentifier>())
{
auto table_id = IdentifierSemantic::extractDatabaseAndTable(*ident);
if (table_id.database_name.empty())
{
auto subquery_it = data.subqueries.find(table_id.table_name);
if (subquery_it != data.subqueries.end())
{
func.arguments->children[1] = subquery_it->second->clone();
dynamic_cast<ASTWithAlias &>(*func.arguments->children[1]).alias = table_id.table_name;
}
}
}
}
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <map>
#include <Parsers/IAST.h>
namespace DB
{
// TODO After we support `union_with_global`, this visitor should also be extended to match ASTSelectQueryWithUnion.
class ASTSelectQuery;
class ASTFunction;
struct ASTTableExpression;
class ApplyWithSubqueryVisitor
{
public:
struct Data
{
std::map<String, ASTPtr> subqueries;
};
static void visit(ASTPtr & ast) { visit(ast, {}); }
private:
static void visit(ASTPtr & ast, const Data & data);
static void visit(ASTTableExpression & table, const Data & data);
static void visit(ASTFunction & func, const Data & data);
};
}

View File

@ -4,6 +4,7 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTWithElement.h>
#include <Interpreters/Context.h>
#include <Interpreters/misc.h>
@ -40,6 +41,10 @@ bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr
if (node->as<ASTTableExpression>())
return false;
/// Do not go to subqueries defined in with statement
if (node->as<ASTWithElement>())
return false;
if (node->as<ASTSelectQuery>())
{
/// Do not go to FROM, JOIN, UNION.

View File

@ -14,6 +14,7 @@
#include <Access/AccessFlags.h>
#include <Interpreters/ApplyWithSubqueryVisitor.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSetQuery.h>
@ -249,6 +250,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = input_pipe->getHeader();
}
ApplyWithSubqueryVisitor().visit(query_ptr);
JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery());
if (!has_input && !storage)

View File

@ -23,6 +23,7 @@ SRCS(
addTypeConversionToAST.cpp
AggregateDescription.cpp
Aggregator.cpp
ApplyWithSubqueryVisitor.cpp
ArithmeticOperationsInAgrFuncOptimize.cpp
ArrayJoinAction.cpp
AsynchronousMetricLog.cpp

View File

@ -0,0 +1,21 @@
#include <Parsers/ASTWithElement.h>
namespace DB
{
ASTPtr ASTWithElement::clone() const
{
const auto res = std::make_shared<ASTWithElement>(*this);
res->name = name;
res->subquery = subquery->clone();
res->children.emplace_back(res->subquery);
return res;
}
void ASTWithElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
subquery->formatImpl(settings, state, frame);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
/** subquery in with statement
*/
class ASTWithElement : public IAST
{
public:
String name;
ASTPtr subquery;
/** Get the text that identifies this element. */
String getID(char) const override { return "WithElement"; }
ASTPtr clone() const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -8,6 +8,7 @@
#include <Parsers/ParserSampleRatio.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/ParserWithElement.h>
namespace DB
@ -74,7 +75,10 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (s_with.ignore(pos, expected))
{
if (!exp_list_for_with_clause.parse(pos, with_expression_list, expected))
if (!ParserList(std::make_unique<ParserWithElement>(), std::make_unique<ParserToken>(TokenType::Comma))
.parse(pos, with_expression_list, expected))
return false;
if (with_expression_list->children.empty())
return false;
}
}

View File

@ -0,0 +1,39 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTWithElement.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserWithElement.h>
namespace DB
{
bool ParserWithElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier s_ident;
ParserKeyword s_as("AS");
ParserSubquery s_subquery;
auto old_pos = pos;
if (ASTPtr name, subquery;
s_ident.parse(pos, name, expected) && s_as.ignore(pos, expected) && s_subquery.parse(pos, subquery, expected))
{
auto with_element = std::make_shared<ASTWithElement>();
tryGetIdentifierNameInto(name, with_element->name);
with_element->subquery = subquery;
node = with_element;
}
else
{
pos = old_pos;
ParserExpressionWithOptionalAlias s_expr(false);
if (!s_expr.parse(pos, node, expected))
return false;
}
return true;
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** WITH (scalar query) AS identifier
* or WITH identifier AS (subquery)
*/
class ParserWithElement : public IParserBase
{
protected:
const char * getName() const override { return "WITH element"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -61,6 +61,7 @@ SRCS(
ASTTTLElement.cpp
ASTUserNameWithHost.cpp
ASTWithAlias.cpp
ASTWithElement.cpp
CommonParsers.cpp
ExpressionElementParsers.cpp
ExpressionListParsers.cpp
@ -133,6 +134,7 @@ SRCS(
ParserUseQuery.cpp
ParserUserNameWithHost.cpp
ParserWatchQuery.cpp
ParserWithElement.cpp
parseUserName.cpp
queryToString.cpp
QueryWithOutputSettingsPushDownVisitor.cpp

View File

@ -0,0 +1,14 @@
0
1
2
3
4
2 3
4 5
2 3
4 5
1 1 2
3 3 4
4 5
4 5
4 5

View File

@ -0,0 +1,13 @@
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(i int, j int) ENGINE Log;
INSERT INTO test1 VALUES (1, 2), (3, 4);
WITH test1 AS (SELECT * FROM numbers(5)) SELECT * FROM test1;
WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM test1;
WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM (SELECT * FROM test1);
SELECT * FROM (WITH test1 AS (SELECT toInt32(*) i FROM numbers(5)) SELECT * FROM test1) l ANY INNER JOIN test1 r on (l.i == r.i);
WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT toInt64(4) i, toInt64(5) j FROM numbers(3) WHERE (i, j) IN test1;
DROP TABLE IF EXISTS test1;