mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Add test, comments, rename
This commit is contained in:
parent
f3702e9279
commit
6c5b4458cb
@ -1,4 +1,4 @@
|
||||
#include <Planner/CollectUsedIndetifiers.h>
|
||||
#include <Planner/CollectColumnIndetifiers.h>
|
||||
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
@ -11,11 +11,11 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
class CollectUsedIdentifiersVisitor : public InDepthQueryTreeVisitor<CollectUsedIdentifiersVisitor, true>
|
||||
class CollectTopLevelColumnIdentifiersVisitor : public InDepthQueryTreeVisitor<CollectTopLevelColumnIdentifiersVisitor, true>
|
||||
{
|
||||
public:
|
||||
|
||||
explicit CollectUsedIdentifiersVisitor(const PlannerContextPtr & planner_context_, ColumnIdentifierSet & used_identifiers_)
|
||||
explicit CollectTopLevelColumnIdentifiersVisitor(const PlannerContextPtr & planner_context_, ColumnIdentifierSet & used_identifiers_)
|
||||
: used_identifiers(used_identifiers_)
|
||||
, planner_context(planner_context_)
|
||||
{}
|
||||
@ -49,16 +49,16 @@ public:
|
||||
|
||||
}
|
||||
|
||||
void collectUsedIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out)
|
||||
void collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out)
|
||||
{
|
||||
CollectUsedIdentifiersVisitor visitor(planner_context, out);
|
||||
CollectTopLevelColumnIdentifiersVisitor visitor(planner_context, out);
|
||||
visitor.visit(node);
|
||||
}
|
||||
|
||||
ColumnIdentifierSet collectUsedIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context)
|
||||
ColumnIdentifierSet collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context)
|
||||
{
|
||||
ColumnIdentifierSet out;
|
||||
collectUsedIdentifiers(node, planner_context, out);
|
||||
collectTopLevelColumnIdentifiers(node, planner_context, out);
|
||||
return out;
|
||||
}
|
||||
|
22
src/Planner/CollectColumnIndetifiers.h
Normal file
22
src/Planner/CollectColumnIndetifiers.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Planner/Planner.h>
|
||||
#include <Planner/TableExpressionData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ColumnIdentifierSet = std::unordered_set<ColumnIdentifier>;
|
||||
|
||||
/// Collect all top level column identifiers from query tree node.
|
||||
/// Top level column identifiers are in the SELECT list or GROUP BY/ORDER BY/WHERE/HAVING clause, but not in child nodes of join tree.
|
||||
/// For example, in the following query:
|
||||
/// SELECT sum(b) FROM (SELECT x AS a, y AS b FROM t) AS t1 JOIN t2 ON t1.a = t2.key GROUP BY t2.y
|
||||
/// The top level column identifiers are: `t1.b`, `t2.y`
|
||||
ColumnIdentifierSet collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context);
|
||||
|
||||
void collectTopLevelColumnIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out);
|
||||
|
||||
}
|
||||
|
@ -1,17 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Planner/Planner.h>
|
||||
#include <Planner/TableExpressionData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ColumnIdentifierSet = std::unordered_set<ColumnIdentifier>;
|
||||
|
||||
ColumnIdentifierSet collectUsedIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context);
|
||||
void collectUsedIdentifiers(const QueryTreeNodePtr & node, const PlannerContextPtr & planner_context, ColumnIdentifierSet & out);
|
||||
|
||||
|
||||
}
|
||||
|
@ -64,7 +64,7 @@
|
||||
#include <Planner/CollectTableExpressionData.h>
|
||||
#include <Planner/PlannerJoinTree.h>
|
||||
#include <Planner/PlannerExpressionAnalysis.h>
|
||||
#include <Planner/CollectUsedIndetifiers.h>
|
||||
#include <Planner/CollectColumnIndetifiers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -375,7 +375,7 @@ void Planner::buildQueryPlanIfNeeded()
|
||||
|
||||
collectSets(query_tree, *planner_context);
|
||||
|
||||
auto top_level_identifiers = collectUsedIdentifiers(query_tree, planner_context);
|
||||
auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context);
|
||||
|
||||
query_plan = buildQueryPlanForJoinTreeNode(query_node.getJoinTree(), select_query_info, select_query_options, top_level_identifiers, planner_context);
|
||||
auto expression_analysis_result = buildExpressionAnalysisResult(query_tree, query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), planner_context);
|
||||
|
@ -33,7 +33,7 @@
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Interpreters/ArrayJoinAction.h>
|
||||
|
||||
#include <Planner/CollectUsedIndetifiers.h>
|
||||
#include <Planner/CollectColumnIndetifiers.h>
|
||||
#include <Planner/Planner.h>
|
||||
#include <Planner/PlannerJoins.h>
|
||||
#include <Planner/PlannerActionsVisitor.h>
|
||||
@ -214,7 +214,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node,
|
||||
auto & join_node = join_tree_node->as<JoinNode &>();
|
||||
|
||||
ColumnIdentifierSet current_scope_columns = outer_scope_columns;
|
||||
collectUsedIdentifiers(join_tree_node, planner_context, current_scope_columns);
|
||||
collectTopLevelColumnIdentifiers(join_tree_node, planner_context, current_scope_columns);
|
||||
|
||||
auto left_plan = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(),
|
||||
select_query_info,
|
||||
|
141
tests/queries/0_stateless/02514_analyzer_drop_join_on.reference
Normal file
141
tests/queries/0_stateless/02514_analyzer_drop_join_on.reference
Normal file
@ -0,0 +1,141 @@
|
||||
Expression ((Project names + Projection))
|
||||
Header: count() UInt64
|
||||
Aggregating
|
||||
Header: default.a.a2_4 String
|
||||
count() UInt64
|
||||
Expression ((Before GROUP BY + DROP unused columns after JOIN))
|
||||
Header: default.a.a2_4 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_4 String
|
||||
default.c.c1_2 UInt64
|
||||
default.d.d1_3 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
Header: default.a.a2_4 String
|
||||
default.c.c1_2 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_4 String
|
||||
default.b.b1_0 UInt64
|
||||
default.c.c1_2 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
Header: default.a.a2_4 String
|
||||
default.b.b1_0 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_4 String
|
||||
default.a.a1_1 UInt64
|
||||
default.b.b1_0 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.a.a2_4 String
|
||||
default.a.a1_1 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: a2 String
|
||||
a1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.b.b1_0 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: b1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.c.c1_2 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: c1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.d.d1_3 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: d1 UInt64
|
||||
Expression ((Project names + (Projection + DROP unused columns after JOIN)))
|
||||
Header: a2 String
|
||||
d2 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
default.d.d2_1 String
|
||||
default.d.k_5 UInt64
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
default.c.k_4 UInt64
|
||||
Expression (DROP unused columns after JOIN)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
default.b.k_3 UInt64
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: default.a.k_2 UInt64
|
||||
default.a.a2_0 String
|
||||
ReadFromStorage (Memory)
|
||||
Header: k UInt64
|
||||
a2 String
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: default.b.k_3 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: k UInt64
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: default.c.k_4 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: k UInt64
|
||||
Expression (Change column names to column identifiers)
|
||||
Header: default.d.k_5 UInt64
|
||||
default.d.d2_1 String
|
||||
ReadFromStorage (Memory)
|
||||
Header: k UInt64
|
||||
d2 String
|
||||
Expression (Project names)
|
||||
Header: bx String
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
Expression ((Before ORDER BY + (Projection + )))
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
default.c.c2_5 String
|
||||
default.c.c1_3 UInt64
|
||||
d.d1_4 UInt64
|
||||
Filter (( + (JOIN actions + DROP unused columns after JOIN)))
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
default.c.c2_5 String
|
||||
default.c.c1_3 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
b.b1_1 UInt64
|
||||
default.c.c2_5 String
|
||||
default.c.c1_3 UInt64
|
||||
Expression ((JOIN actions + DROP unused columns after JOIN))
|
||||
Header: default.a.a2_6 String
|
||||
b.bx_0 String
|
||||
b.b1_1 UInt64
|
||||
Join (JOIN FillRightFirst)
|
||||
Header: default.a.a2_6 String
|
||||
default.a.a1_2 UInt64
|
||||
b.bx_0 String
|
||||
b.b1_1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.a.a2_6 String
|
||||
default.a.a1_2 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: a2 String
|
||||
a1 UInt64
|
||||
Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
|
||||
Header: b.b1_1 UInt64
|
||||
b.bx_0 String
|
||||
ReadFromStorage (Memory)
|
||||
Header: b2 String
|
||||
b1 UInt64
|
||||
Expression ((JOIN actions + Change column names to column identifiers))
|
||||
Header: default.c.c2_5 String
|
||||
default.c.c1_3 UInt64
|
||||
ReadFromStorage (Memory)
|
||||
Header: c2 String
|
||||
c1 UInt64
|
||||
Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
|
||||
Header: d.d1_4 UInt64
|
||||
ReadFromStorage (SystemNumbers)
|
||||
Header: number UInt64
|
46
tests/queries/0_stateless/02514_analyzer_drop_join_on.sql
Normal file
46
tests/queries/0_stateless/02514_analyzer_drop_join_on.sql
Normal file
@ -0,0 +1,46 @@
|
||||
DROP TABLE IF EXISTS a;
|
||||
DROP TABLE IF EXISTS b;
|
||||
DROP TABLE IF EXISTS c;
|
||||
DROP TABLE IF EXISTS d;
|
||||
|
||||
CREATE TABLE a (k UInt64, a1 UInt64, a2 String) ENGINE = Memory;
|
||||
INSERT INTO a VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
|
||||
|
||||
CREATE TABLE b (k UInt64, b1 UInt64, b2 String) ENGINE = Memory;
|
||||
INSERT INTO b VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
|
||||
|
||||
CREATE TABLE c (k UInt64, c1 UInt64, c2 String) ENGINE = Memory;
|
||||
INSERT INTO c VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
|
||||
|
||||
CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory;
|
||||
INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
|
||||
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
EXPLAIN PLAN header = 1
|
||||
SELECT count()
|
||||
FROM a
|
||||
JOIN b ON b.b1 = a.a1
|
||||
JOIN c ON c.c1 = b.b1
|
||||
JOIN d ON d.d1 = c.c1
|
||||
GROUP BY a.a2
|
||||
;
|
||||
|
||||
EXPLAIN PLAN header = 1
|
||||
SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k)
|
||||
;
|
||||
|
||||
EXPLAIN PLAN header = 1
|
||||
SELECT b.bx
|
||||
FROM a
|
||||
JOIN (SELECT b1, b2 || 'x' AS bx FROM b ) AS b ON b.b1 = a.a1
|
||||
JOIN c ON c.c1 = b.b1
|
||||
JOIN (SELECT number AS d1 from numbers(10)) AS d ON d.d1 = c.c1
|
||||
WHERE c.c2 != ''
|
||||
ORDER BY a.a2
|
||||
;
|
||||
|
||||
DROP TABLE IF EXISTS a;
|
||||
DROP TABLE IF EXISTS b;
|
||||
DROP TABLE IF EXISTS c;
|
||||
DROP TABLE IF EXISTS d;
|
Loading…
Reference in New Issue
Block a user