mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
CTE
This commit is contained in:
parent
87b3984d17
commit
882b2a3348
90
src/Interpreters/ApplyWithSubqueryVisitor.cpp
Normal file
90
src/Interpreters/ApplyWithSubqueryVisitor.cpp
Normal 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
30
src/Interpreters/ApplyWithSubqueryVisitor.h
Normal file
30
src/Interpreters/ApplyWithSubqueryVisitor.h
Normal 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);
|
||||
};
|
||||
|
||||
}
|
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -23,6 +23,7 @@ SRCS(
|
||||
addTypeConversionToAST.cpp
|
||||
AggregateDescription.cpp
|
||||
Aggregator.cpp
|
||||
ApplyWithSubqueryVisitor.cpp
|
||||
ArithmeticOperationsInAgrFuncOptimize.cpp
|
||||
ArrayJoinAction.cpp
|
||||
AsynchronousMetricLog.cpp
|
||||
|
21
src/Parsers/ASTWithElement.cpp
Normal file
21
src/Parsers/ASTWithElement.cpp
Normal 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);
|
||||
}
|
||||
}
|
25
src/Parsers/ASTWithElement.h
Normal file
25
src/Parsers/ASTWithElement.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
39
src/Parsers/ParserWithElement.cpp
Normal file
39
src/Parsers/ParserWithElement.cpp
Normal 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;
|
||||
}
|
||||
|
||||
|
||||
}
|
18
src/Parsers/ParserWithElement.h
Normal file
18
src/Parsers/ParserWithElement.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user