未验证 提交 2028228f 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #18684 from amosbird/betterwith

better CTE and with global
#!/usr/bin/python3
#!/usr/bin/env python3
import argparse
import clickhouse_driver
......
......@@ -405,7 +405,7 @@ class IColumn;
M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \
M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \
M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \
M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \
......
......@@ -12,12 +12,10 @@ void ApplyWithAliasVisitor::visit(ASTPtr & ast, const Data & data)
std::optional<Data> new_data;
if (auto with = node_select->with())
{
for (auto & child : with->children)
visit(child, data);
std::set<String> current_names;
for (auto & child : with->children)
{
visit(child, new_data ? *new_data : data);
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
{
if (!new_data)
......
......@@ -3,51 +3,77 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTWithAlias.h>
#include <map>
namespace DB
{
void ApplyWithGlobalVisitor::visit(ASTSelectQuery & select, const std::map<String, ASTPtr> & exprs, const ASTPtr & with_expression_list)
{
auto with = select.with();
if (with)
{
std::set<String> current_names;
for (const auto & child : with->children)
{
if (const auto * ast_with_alias = dynamic_cast<const ASTWithAlias *>(child.get()))
current_names.insert(ast_with_alias->alias);
}
for (const auto & with_alias : exprs)
{
if (!current_names.count(with_alias.first))
with->children.push_back(with_alias.second->clone());
}
}
else
select.setExpression(ASTSelectQuery::Expression::WITH, with_expression_list->clone());
}
void ApplyWithGlobalVisitor::visit(
ASTSelectWithUnionQuery & selects, const std::map<String, ASTPtr> & exprs, const ASTPtr & with_expression_list)
{
for (auto & select : selects.list_of_selects->children)
{
if (ASTSelectWithUnionQuery * node_union = select->as<ASTSelectWithUnionQuery>())
{
visit(*node_union, exprs, with_expression_list);
}
else if (ASTSelectQuery * node_select = select->as<ASTSelectQuery>())
{
visit(*node_select, exprs, with_expression_list);
}
}
}
void ApplyWithGlobalVisitor::visit(ASTPtr & ast)
{
if (ASTSelectWithUnionQuery * node_union = ast->as<ASTSelectWithUnionQuery>())
{
auto & first_select = node_union->list_of_selects->children[0]->as<ASTSelectQuery &>();
ASTPtr with_expression_list = first_select.with();
if (with_expression_list)
if (auto * first_select = node_union->list_of_selects->children[0]->as<ASTSelectQuery>())
{
std::map<String, ASTPtr> exprs;
for (auto & child : with_expression_list->children)
ASTPtr with_expression_list = first_select->with();
if (with_expression_list)
{
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
exprs[ast_with_alias->alias] = child;
}
for (auto it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it)
{
auto & select = (*it)->as<ASTSelectQuery &>();
auto with = select.with();
if (with)
std::map<String, ASTPtr> exprs;
for (auto & child : with_expression_list->children)
{
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
exprs[ast_with_alias->alias] = child;
}
for (auto it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it)
{
std::set<String> current_names;
for (auto & child : with->children)
{
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
current_names.insert(ast_with_alias->alias);
}
for (auto & with_alias : exprs)
{
if (!current_names.count(with_alias.first))
with->children.push_back(with_alias.second->clone());
}
if (auto * union_child = (*it)->as<ASTSelectWithUnionQuery>())
visit(*union_child, exprs, with_expression_list);
else if (auto * select_child = (*it)->as<ASTSelectQuery>())
visit(*select_child, exprs, with_expression_list);
}
else
select.setExpression(ASTSelectQuery::Expression::WITH, with_expression_list->clone());
}
}
}
for (auto & child : ast->children)
visit(child);
else
{
// Other non-SELECT queries that contains SELECT children, such as EXPLAIN or INSERT
for (auto & child : ast->children)
visit(child);
}
}
}
#pragma once
#include <Parsers/IAST.h>
#include <map>
namespace DB
{
class ASTSelectWithUnionQuery;
class ASTSelectQuery;
/// Pull out the WITH statement from the first child of ASTSelectWithUnion query if any.
class ApplyWithGlobalVisitor
{
public:
static void visit(ASTPtr & ast);
private:
static void visit(ASTSelectWithUnionQuery & selects, const std::map<String, ASTPtr> & exprs, const ASTPtr & with_expression_list);
static void visit(ASTSelectQuery & select, const std::map<String, ASTPtr> & exprs, const ASTPtr & with_expression_list);
};
}
......@@ -4,6 +4,7 @@
#include <Interpreters/misc.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTWithElement.h>
......@@ -55,10 +56,13 @@ void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & da
auto subquery_it = data.subqueries.find(table_id.table_name);
if (subquery_it != data.subqueries.end())
{
auto old_alias = table.database_and_table_name->tryGetAlias();
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.subquery->as<ASTSubquery &>().cte_name = table_id.table_name;
if (!old_alias.empty())
table.subquery->setAlias(old_alias);
table.children.emplace_back(table.subquery);
}
}
......@@ -78,8 +82,11 @@ void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data)
auto subquery_it = data.subqueries.find(table_id.table_name);
if (subquery_it != data.subqueries.end())
{
auto old_alias = func.arguments->children[1]->tryGetAlias();
func.arguments->children[1] = subquery_it->second->clone();
dynamic_cast<ASTWithAlias &>(*func.arguments->children[1]).alias = table_id.table_name;
func.arguments->children[1]->as<ASTSubquery &>().cte_name = table_id.table_name;
if (!old_alias.empty())
func.arguments->children[1]->setAlias(old_alias);
}
}
}
......
......@@ -44,7 +44,15 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression &
else if (table_expression.table_function)
alias = table_expression.table_function->tryGetAlias();
else if (table_expression.subquery)
{
const auto & cte_name = table_expression.subquery->as<const ASTSubquery &>().cte_name;
if (!cte_name.empty())
{
database = current_database;
table = cte_name;
}
alias = table_expression.subquery->tryGetAlias();
}
else
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
}
......
......@@ -55,6 +55,7 @@
#include <IO/WriteBufferFromString.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Parsers/formatAST.h>
namespace DB
{
......
......@@ -31,6 +31,12 @@ public:
{
if (child->as<ASTSubquery>() || child->as<ASTSelectQuery>())
return false;
if (auto * select = node->as<ASTSelectQuery>())
{
// We don't analysis WITH statement because it might contain useless aggregates
if (child == select->with())
return false;
}
if (auto * func = node->as<ASTFunction>())
{
if (isAggregateFunction(*func))
......
......@@ -294,9 +294,13 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = input_pipe->getHeader();
}
if (context->getSettingsRef().enable_global_with_statement)
ApplyWithAliasVisitor().visit(query_ptr);
ApplyWithSubqueryVisitor().visit(query_ptr);
// Only propagate WITH elements to subqueries if we're not a subquery
if (options.subquery_depth == 0)
{
if (context->getSettingsRef().enable_global_with_statement)
ApplyWithAliasVisitor().visit(query_ptr);
ApplyWithSubqueryVisitor().visit(query_ptr);
}
JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery());
......
......@@ -123,8 +123,12 @@ void RequiredSourceColumnsMatcher::visit(const ASTSelectQuery & select, const AS
std::vector<ASTPtr *> out;
for (const auto & node : select.children)
if (node != select.select())
{
// We should not go into WITH statement because all needed aliases are already expanded to
// the right place after normalization. And it might contain unused unknown columns.
if (node != select.select() && node != select.with())
Visitor(data).visit(node);
}
/// revisit select_expression_list (with children) when all the aliases are set
Visitor(data).visit(select.select());
......
......@@ -73,8 +73,12 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number
return nullptr;
}
static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression & table_expression, const Context & context,
NamesAndTypesList & materialized, NamesAndTypesList & aliases, NamesAndTypesList & virtuals)
static NamesAndTypesList getColumnsFromTableExpression(
const ASTTableExpression & table_expression,
const Context & context,
NamesAndTypesList & materialized,
NamesAndTypesList & aliases,
NamesAndTypesList & virtuals)
{
NamesAndTypesList names_and_type_list;
if (table_expression.subquery)
......
#include <Parsers/ASTSubquery.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Common/SipHash.h>
namespace DB
{
......@@ -12,6 +13,10 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const
{
writeString(alias, ostr);
}
else if (!cte_name.empty())
{
writeString(cte_name, ostr);
}
else
{
Hash hash = getTreeHash();
......@@ -24,6 +29,14 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const
void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!cte_name.empty())
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(cte_name);
settings.ostr << (settings.hilite ? hilite_none : "");
return;
}
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
std::string nl_or_nothing = settings.one_line ? "" : "\n";
......@@ -35,5 +48,12 @@ void ASTSubquery::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << nl_or_nothing << indent_str << ")";
}
void ASTSubquery::updateTreeHashImpl(SipHash & hash_state) const
{
if (!cte_name.empty())
hash_state.update(cte_name);
IAST::updateTreeHashImpl(hash_state);
}
}
......@@ -12,6 +12,10 @@ namespace DB
class ASTSubquery : public ASTWithAlias
{
public:
// Stored the name when the subquery is defined in WITH clause. For example:
// WITH (SELECT 1) AS a SELECT * FROM a AS b; cte_name will be `a`.
std::string cte_name;
/** Get the text that identifies this element. */
String getID(char) const override { return "Subquery"; }
......@@ -28,6 +32,8 @@ public:
return ptr;
}
void updateTreeHashImpl(SipHash & hash_state) const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
......
#include <Parsers/ASTWithElement.h>
#include <Parsers/ASTWithAlias.h>
#include <IO/Operators.h>
namespace DB
......@@ -8,7 +9,6 @@ ASTPtr ASTWithElement::clone() const
{
const auto res = std::make_shared<ASTWithElement>(*this);
res->children.clear();
res->name = name;
res->subquery = subquery->clone();
res->children.emplace_back(res->subquery);
return res;
......@@ -18,6 +18,7 @@ void ASTWithElement::formatImpl(const FormatSettings & settings, FormatState & s
{
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
subquery->formatImpl(settings, state, frame);
dynamic_cast<const ASTWithAlias *>(subquery.get())->formatImplWithoutAlias(settings, state, frame);
}
}
......@@ -23,6 +23,7 @@ bool ParserWithElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto with_element = std::make_shared<ASTWithElement>();
tryGetIdentifierNameInto(name, with_element->name);
with_element->subquery = subquery;
with_element->children.push_back(with_element->subquery);
node = with_element;
}
else
......
......@@ -38,7 +38,8 @@ ColumnsDescription TableFunctionView::getActualTableStructure(const Context & co
return ColumnsDescription(sample.getNamesAndTypesList());
}
StoragePtr TableFunctionView::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
StoragePtr TableFunctionView::executeImpl(
const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
auto columns = getActualTableStructure(context);
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns);
......
0 0
WITH it AS
(
SELECT *
FROM numbers(1)
)
SELECT
number,
number
FROM it AS i
with it as ( select * from numbers(1) ) select it.number, i.number from it as i;
explain syntax with it as ( select * from numbers(1) ) select it.number, i.number from it as i;
......@@ -5,6 +5,10 @@
1
1
2
106 6
107 7
108 8
109 9
WITH 1 AS x
SELECT x
WITH 1 AS x
......@@ -33,3 +37,18 @@ SELECT x
UNION ALL
WITH 2 AS x
SELECT x
WITH
5 AS q1,
x AS
(
WITH 5 AS q1
SELECT
number + 100 AS b,
number AS a
FROM numbers(10)
WHERE number > q1
)
SELECT
b,
a
FROM x
SET enable_global_with_statement = 1;
WITH 1 AS x SELECT x;
WITH 1 AS x SELECT * FROM (SELECT x);
WITH 1 AS x SELECT *, x FROM (WITH 2 AS x SELECT x AS y);
WITH 1 AS x SELECT x UNION ALL SELECT x;
select x from (WITH 1 AS x SELECT x UNION ALL WITH 2 AS x SELECT x) order by x;
with 1 as x select x;
with 1 as x select * from (select x);
with 1 as x select *, x from (with 2 as x select x as y);
with 1 as x select x union all select x;
select x from (with 1 as x select x union all with 2 as x select x) order by x;
with 5 as q1, x as (select number+100 as b, number as a from numbers(10) where number > q1) select * from x;
EXPLAIN SYNTAX WITH 1 AS x SELECT x;
EXPLAIN SYNTAX WITH 1 AS x SELECT * FROM (SELECT x);
EXPLAIN SYNTAX WITH 1 AS x SELECT *, x FROM (WITH 2 AS x SELECT x AS y);
EXPLAIN SYNTAX WITH 1 AS x SELECT x UNION ALL SELECT x;
EXPLAIN SYNTAX WITH 1 AS x SELECT x UNION ALL WITH 2 AS x SELECT x;
explain syntax with 1 as x select x;
explain syntax with 1 as x select * from (select x);
explain syntax with 1 as x select *, x from (with 2 as x select x as y);
explain syntax with 1 as x select x union all select x;
explain syntax with 1 as x select x union all with 2 as x select x;
explain syntax with 5 as q1, x as (select number + 100 as b, number as a from numbers(10) where number > q1) select * from x;
SET union_default_mode = 'DISTINCT';
set enable_global_with_statement = 1;
EXPLAIN SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1;
EXPLAIN (SELECT 1 UNION ALL SELECT 1) UNION ALL SELECT 1;
EXPLAIN SELECT 1 UNION (SELECT 1 UNION ALL SELECT 1);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册