Skip to content

Commit a34e5fd

Browse files
Backport ClickHouse#85966 to 25.8: Analyzer: check for valid correlated subqueries in the FROM clause
1 parent ea12745 commit a34e5fd

File tree

5 files changed

+172
-4
lines changed

5 files changed

+172
-4
lines changed

src/Analyzer/Resolve/QueryAnalyzer.cpp

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6094,6 +6094,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
60946094

60956095
expandGroupByAll(query_node_typed);
60966096

6097+
validateFromClause(query_node);
60976098
validateFilters(query_node);
60986099
validateAggregates(query_node, {.group_by_use_nulls = scope.group_by_use_nulls});
60996100

src/Analyzer/ValidationUtils.cpp

Lines changed: 81 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,27 +1,32 @@
11
#include <Analyzer/ValidationUtils.h>
22

33
#include <Analyzer/AggregationUtils.h>
4+
#include <Analyzer/ArrayJoinNode.h>
45
#include <Analyzer/ColumnNode.h>
56
#include <Analyzer/ConstantNode.h>
67
#include <Analyzer/FunctionNode.h>
78
#include <Analyzer/InDepthQueryTreeVisitor.h>
9+
#include <Analyzer/JoinNode.h>
810
#include <Analyzer/QueryNode.h>
911
#include <Analyzer/TableNode.h>
1012
#include <Analyzer/WindowFunctionsUtils.h>
1113
#include <Storages/IStorage.h>
1214

15+
#include <memory>
16+
1317
namespace DB
1418
{
1519

1620
namespace ErrorCodes
1721
{
18-
extern const int NOT_AN_AGGREGATE;
19-
extern const int NOT_IMPLEMENTED;
2022
extern const int BAD_ARGUMENTS;
21-
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
2223
extern const int ILLEGAL_PREWHERE;
23-
extern const int UNSUPPORTED_METHOD;
24+
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
25+
extern const int LOGICAL_ERROR;
26+
extern const int NOT_AN_AGGREGATE;
27+
extern const int NOT_IMPLEMENTED;
2428
extern const int UNEXPECTED_EXPRESSION;
29+
extern const int UNSUPPORTED_METHOD;
2530
}
2631

2732
namespace
@@ -541,4 +546,76 @@ void validateCorrelatedSubqueries(const QueryTreeNodePtr & node)
541546
}
542547
}
543548

549+
void validateFromClause(const QueryTreeNodePtr & node)
550+
{
551+
const auto & root_query_node = node->as<QueryNode &>();
552+
auto correlated_columns_set = root_query_node.getCorrelatedColumnsSet();
553+
554+
std::vector<QueryTreeNodePtr> nodes_to_process = { root_query_node.getJoinTree() };
555+
556+
while (!nodes_to_process.empty())
557+
{
558+
auto node_to_process = std::move(nodes_to_process.back());
559+
nodes_to_process.pop_back();
560+
561+
auto node_type = node_to_process->getNodeType();
562+
563+
switch (node_type)
564+
{
565+
case QueryTreeNodeType::TABLE:
566+
[[fallthrough]];
567+
case QueryTreeNodeType::TABLE_FUNCTION:
568+
break;
569+
case QueryTreeNodeType::QUERY:
570+
{
571+
auto & query_node = node_to_process->as<QueryNode &>();
572+
const auto & correlated_columns = query_node.getCorrelatedColumns();
573+
for (const auto & column : correlated_columns)
574+
{
575+
if (!correlated_columns_set.contains(std::static_pointer_cast<ColumnNode>(column)))
576+
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
577+
"Lateral joins are not supported. Correlated column '{}' is found in the FROM clause. In query {}",
578+
column->formatASTForErrorMessage(),
579+
node->formatASTForErrorMessage());
580+
}
581+
break;
582+
}
583+
case QueryTreeNodeType::UNION:
584+
{
585+
for (const auto & union_node : node_to_process->as<UnionNode>()->getQueries().getNodes())
586+
nodes_to_process.push_back(union_node);
587+
break;
588+
}
589+
case QueryTreeNodeType::ARRAY_JOIN:
590+
{
591+
auto & array_join_node = node_to_process->as<ArrayJoinNode &>();
592+
nodes_to_process.push_back(array_join_node.getTableExpression());
593+
break;
594+
}
595+
case QueryTreeNodeType::CROSS_JOIN:
596+
{
597+
auto & join_node = node_to_process->as<CrossJoinNode &>();
598+
for (const auto & expr : std::ranges::reverse_view(join_node.getTableExpressions()))
599+
nodes_to_process.push_back(expr);
600+
break;
601+
}
602+
case QueryTreeNodeType::JOIN:
603+
{
604+
auto & join_node = node_to_process->as<JoinNode &>();
605+
nodes_to_process.push_back(join_node.getRightTableExpression());
606+
nodes_to_process.push_back(join_node.getLeftTableExpression());
607+
break;
608+
}
609+
default:
610+
{
611+
throw Exception(ErrorCodes::LOGICAL_ERROR,
612+
"Unexpected node type for table expression. "
613+
"Expected table, table function, query, union, join or array join. Actual {}",
614+
node_to_process->getNodeTypeName());
615+
}
616+
}
617+
}
618+
619+
}
620+
544621
}

src/Analyzer/ValidationUtils.h

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,8 +41,16 @@ void validateTreeSize(const QueryTreeNodePtr & node,
4141
size_t max_size,
4242
std::unordered_map<QueryTreeNodePtr, size_t> & node_to_tree_size);
4343

44+
/**
45+
* Validate that correlated subqueries do not present in the context of distributed query.
46+
*/
4447
void validateCorrelatedSubqueries(const QueryTreeNodePtr & node);
4548

49+
/**
50+
* Validate that if correlated subquery appears in the FROM clause then it uses columns from outer query.
51+
*/
52+
void validateFromClause(const QueryTreeNodePtr & node);
53+
4654
/** Compare node with group by key node.
4755
* Such comparison does not take into account aliases, but checks types and column sources.
4856
*/

tests/queries/0_stateless/03595_analyzer_lateral_join.reference

Whitespace-only changes.
Lines changed: 82 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,82 @@
1+
SET enable_analyzer = 1;
2+
SET allow_experimental_correlated_subqueries = 1;
3+
4+
SELECT
5+
t.a,
6+
u.a
7+
FROM
8+
(
9+
SELECT
10+
1 AS a
11+
) AS t,
12+
(
13+
SELECT 1 AS a
14+
QUALIFY 0 = (t.a AS alias668)
15+
) AS u; -- { serverError NOT_IMPLEMENTED }
16+
17+
SELECT
18+
t.a,
19+
u.a
20+
FROM
21+
(
22+
SELECT
23+
1 AS a
24+
) AS t,
25+
(
26+
SELECT
27+
DISTINCT *,
28+
*,
29+
27,
30+
'======Before Truncate======',
31+
materialize(27),
32+
27,
33+
*,
34+
isZeroOrNull(27),
35+
27,
36+
materialize(27),
37+
*,
38+
* IS NOT NULL,
39+
*,
40+
27,
41+
*,
42+
toFixedString('======Before Truncate======', 27),
43+
27,
44+
27,
45+
27,
46+
27,
47+
toLowCardinality(27),
48+
toNullable(materialize(27)),
49+
* IS NOT NULL,
50+
1 AS a QUALIFY (
51+
(
52+
*,
53+
27,
54+
materialize(27),
55+
27,
56+
'======Before Truncate======',
57+
27,
58+
27,
59+
(27 IS NOT NULL),
60+
* IS NOT NULL
61+
) IS NULL
62+
) = (t.a AS alias668)
63+
) AS u; -- { serverError NOT_IMPLEMENTED }
64+
65+
SELECT
66+
c,
67+
a c
68+
FROM
69+
(
70+
SELECT
71+
1 a
72+
) X
73+
CROSS JOIN (
74+
SELECT
75+
1
76+
WHERE
77+
c = 1
78+
) Y
79+
CROSS JOIN (
80+
SELECT
81+
1 c
82+
) Z; -- { serverError NOT_IMPLEMENTED }

0 commit comments

Comments
 (0)