mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Backport #71538 to 24.8: Analyzer materialized view IN with CTE fix
This commit is contained in:
parent
97f87d758a
commit
682e6f0593
@ -602,9 +602,21 @@ public:
|
||||
return projection_columns;
|
||||
}
|
||||
|
||||
/// Returns true if query node is resolved, false otherwise
|
||||
bool isResolved() const
|
||||
{
|
||||
return !projection_columns.empty();
|
||||
}
|
||||
|
||||
/// Resolve query node projection columns
|
||||
void resolveProjectionColumns(NamesAndTypes projection_columns_value);
|
||||
|
||||
/// Clear query node projection columns
|
||||
void clearProjectionColumns()
|
||||
{
|
||||
projection_columns.clear();
|
||||
}
|
||||
|
||||
/// Remove unused projection columns
|
||||
void removeUnusedProjectionColumns(const std::unordered_set<std::string> & used_projection_columns);
|
||||
|
||||
|
@ -2899,27 +2899,29 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
/// Replace storage with values storage of insertion block
|
||||
if (StoragePtr storage = scope.context->getViewSource())
|
||||
{
|
||||
QueryTreeNodePtr table_expression;
|
||||
QueryTreeNodePtr table_expression = in_second_argument;
|
||||
|
||||
/// Process possibly nested sub-selects
|
||||
for (auto * query_node = in_second_argument->as<QueryNode>(); query_node; query_node = table_expression->as<QueryNode>())
|
||||
while (table_expression)
|
||||
{
|
||||
if (auto * query_node = table_expression->as<QueryNode>())
|
||||
table_expression = extractLeftTableExpression(query_node->getJoinTree());
|
||||
|
||||
if (table_expression)
|
||||
{
|
||||
if (auto * query_table_node = table_expression->as<TableNode>())
|
||||
{
|
||||
if (query_table_node->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted())
|
||||
{
|
||||
auto replacement_table_expression = std::make_shared<TableNode>(storage, scope.context);
|
||||
if (std::optional<TableExpressionModifiers> table_expression_modifiers = query_table_node->getTableExpressionModifiers())
|
||||
replacement_table_expression->setTableExpressionModifiers(*table_expression_modifiers);
|
||||
in_second_argument = in_second_argument->cloneAndReplace(table_expression, std::move(replacement_table_expression));
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (auto * union_node = table_expression->as<UnionNode>())
|
||||
table_expression = union_node->getQueries().getNodes().at(0);
|
||||
else
|
||||
break;
|
||||
}
|
||||
|
||||
resolveExpressionNode(in_second_argument, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/);
|
||||
TableNode * table_expression_table_node = table_expression ? table_expression->as<TableNode>() : nullptr;
|
||||
|
||||
if (table_expression_table_node &&
|
||||
table_expression_table_node->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted())
|
||||
{
|
||||
auto replacement_table_expression_table_node = table_expression_table_node->clone();
|
||||
replacement_table_expression_table_node->as<TableNode &>().updateStorage(storage, scope.context);
|
||||
in_second_argument = in_second_argument->cloneAndReplace(table_expression, std::move(replacement_table_expression_table_node));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Edge case when the first argument of IN is scalar subquery.
|
||||
@ -5259,6 +5261,16 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
|
||||
|
||||
auto & query_node_typed = query_node->as<QueryNode &>();
|
||||
|
||||
/** It is unsafe to call resolveQuery on already resolved query node, because during identifier resolution process
|
||||
* we replace identifiers with expressions without aliases, also at the end of resolveQuery all aliases from all nodes will be removed.
|
||||
* For subsequent resolveQuery executions it is possible to have wrong projection header, because for nodes
|
||||
* with aliases projection name is alias.
|
||||
*
|
||||
* If for client it is necessary to resolve query node after clone, client must clear projection columns from query node before resolve.
|
||||
*/
|
||||
if (query_node_typed.isResolved())
|
||||
return;
|
||||
|
||||
if (query_node_typed.isCTE())
|
||||
ctes_in_resolve_process.insert(query_node_typed.getCTEName());
|
||||
|
||||
@ -5624,6 +5636,9 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier
|
||||
{
|
||||
auto & union_node_typed = union_node->as<UnionNode &>();
|
||||
|
||||
if (union_node_typed.isResolved())
|
||||
return;
|
||||
|
||||
if (union_node_typed.isCTE())
|
||||
ctes_in_resolve_process.insert(union_node_typed.getCTEName());
|
||||
|
||||
|
@ -35,6 +35,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
UnionNode::UnionNode(ContextMutablePtr context_, SelectUnionMode union_mode_)
|
||||
@ -50,6 +51,26 @@ UnionNode::UnionNode(ContextMutablePtr context_, SelectUnionMode union_mode_)
|
||||
children[queries_child_index] = std::make_shared<ListNode>();
|
||||
}
|
||||
|
||||
bool UnionNode::isResolved() const
|
||||
{
|
||||
for (const auto & query_node : getQueries().getNodes())
|
||||
{
|
||||
bool is_resolved = false;
|
||||
|
||||
if (auto * query_node_typed = query_node->as<QueryNode>())
|
||||
is_resolved = query_node_typed->isResolved();
|
||||
else if (auto * union_node_typed = query_node->as<UnionNode>())
|
||||
is_resolved = union_node_typed->isResolved();
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected query tree node type in UNION node");
|
||||
|
||||
if (!is_resolved)
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
NamesAndTypes UnionNode::computeProjectionColumns() const
|
||||
{
|
||||
if (recursive_cte_table)
|
||||
|
@ -163,6 +163,9 @@ public:
|
||||
return children[queries_child_index];
|
||||
}
|
||||
|
||||
/// Returns true if union node is resolved, false otherwise
|
||||
bool isResolved() const;
|
||||
|
||||
/// Compute union node projection columns
|
||||
NamesAndTypes computeProjectionColumns() const;
|
||||
|
||||
|
@ -0,0 +1 @@
|
||||
1 2 \N test
|
@ -0,0 +1,63 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
DROP TABLE IF EXISTS mv_test;
|
||||
DROP TABLE IF EXISTS mv_test_target;
|
||||
DROP VIEW IF EXISTS mv_test_mv;
|
||||
|
||||
CREATE TABLE mv_test
|
||||
(
|
||||
`id` UInt64,
|
||||
`ref_id` UInt64,
|
||||
`final_id` Nullable(UInt64),
|
||||
`display` String
|
||||
)
|
||||
ENGINE = Log;
|
||||
|
||||
CREATE TABLE mv_test_target
|
||||
(
|
||||
`id` UInt64,
|
||||
`ref_id` UInt64,
|
||||
`final_id` Nullable(UInt64),
|
||||
`display` String
|
||||
)
|
||||
ENGINE = Log;
|
||||
|
||||
CREATE MATERIALIZED VIEW mv_test_mv TO mv_test_target
|
||||
(
|
||||
`id` UInt64,
|
||||
`ref_id` UInt64,
|
||||
`final_id` Nullable(UInt64),
|
||||
`display` String
|
||||
)
|
||||
AS WITH
|
||||
tester AS
|
||||
(
|
||||
SELECT
|
||||
id,
|
||||
ref_id,
|
||||
final_id,
|
||||
display
|
||||
FROM mv_test
|
||||
),
|
||||
id_set AS
|
||||
(
|
||||
SELECT
|
||||
display,
|
||||
max(id) AS max_id
|
||||
FROM mv_test
|
||||
GROUP BY display
|
||||
)
|
||||
SELECT *
|
||||
FROM tester
|
||||
WHERE id IN (
|
||||
SELECT max_id
|
||||
FROM id_set
|
||||
);
|
||||
|
||||
INSERT INTO mv_test ( id, ref_id, display) values ( 1, 2, 'test');
|
||||
|
||||
SELECT * FROM mv_test_target;
|
||||
|
||||
DROP VIEW mv_test_mv;
|
||||
DROP TABLE mv_test_target;
|
||||
DROP TABLE mv_test;
|
@ -0,0 +1 @@
|
||||
2
|
@ -0,0 +1,19 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
DROP VIEW IF EXISTS test_mv;
|
||||
|
||||
CREATE TABLE test_table ENGINE = MergeTree ORDER BY tuple() AS SELECT 1 as col1;
|
||||
|
||||
CREATE MATERIALIZED VIEW test_mv ENGINE = MergeTree ORDER BY tuple() AS
|
||||
WITH
|
||||
subquery_on_source AS (SELECT col1 AS aliased FROM test_table),
|
||||
output AS (SELECT * FROM test_table WHERE col1 IN (SELECT aliased FROM subquery_on_source))
|
||||
SELECT * FROM output;
|
||||
|
||||
INSERT INTO test_table VALUES (2);
|
||||
|
||||
SELECT * FROM test_mv;
|
||||
|
||||
DROP VIEW test_mv;
|
||||
DROP TABLE test_table;
|
Loading…
Reference in New Issue
Block a user