mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Backport #63688 to 24.3: Analyzer view read only necessary columns
This commit is contained in:
parent
db75a12c99
commit
4ebf48eb8a
@ -5,6 +5,8 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
@ -38,22 +40,47 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
ASTPtr normalizeAndValidateQuery(const ASTPtr & query)
|
||||
ASTPtr normalizeAndValidateQuery(const ASTPtr & query, const Names & column_names)
|
||||
{
|
||||
ASTPtr result_query;
|
||||
|
||||
if (query->as<ASTSelectWithUnionQuery>() || query->as<ASTSelectQuery>())
|
||||
{
|
||||
return query;
|
||||
}
|
||||
result_query = query;
|
||||
else if (auto * subquery = query->as<ASTSubquery>())
|
||||
{
|
||||
return subquery->children[0];
|
||||
}
|
||||
result_query = subquery->children[0];
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
|
||||
"Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}",
|
||||
query->formatForErrorMessage());
|
||||
}
|
||||
|
||||
if (column_names.empty())
|
||||
return result_query;
|
||||
|
||||
/// The initial query the VIEW references to is wrapped here with another SELECT query to allow reading only necessary columns.
|
||||
auto select_query = std::make_shared<ASTSelectQuery>();
|
||||
|
||||
auto result_table_expression_ast = std::make_shared<ASTTableExpression>();
|
||||
result_table_expression_ast->children.push_back(std::make_shared<ASTSubquery>(std::move(result_query)));
|
||||
result_table_expression_ast->subquery = result_table_expression_ast->children.back();
|
||||
|
||||
auto tables_in_select_query_element_ast = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
tables_in_select_query_element_ast->children.push_back(std::move(result_table_expression_ast));
|
||||
tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back();
|
||||
|
||||
ASTPtr tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
|
||||
tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast));
|
||||
|
||||
select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast));
|
||||
|
||||
auto projection_expression_list_ast = std::make_shared<ASTExpressionList>();
|
||||
projection_expression_list_ast->children.reserve(column_names.size());
|
||||
|
||||
for (const auto & column_name : column_names)
|
||||
projection_expression_list_ast->children.push_back(std::make_shared<ASTIdentifier>(column_name));
|
||||
|
||||
select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(projection_expression_list_ast));
|
||||
|
||||
return select_query;
|
||||
}
|
||||
|
||||
ContextMutablePtr buildContext(const ContextPtr & context, const SelectQueryOptions & select_query_options)
|
||||
@ -125,8 +152,9 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query,
|
||||
InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const ContextPtr & context_,
|
||||
const SelectQueryOptions & select_query_options_)
|
||||
: query(normalizeAndValidateQuery(query_))
|
||||
const SelectQueryOptions & select_query_options_,
|
||||
const Names & column_names)
|
||||
: query(normalizeAndValidateQuery(query_, column_names))
|
||||
, context(buildContext(context_, select_query_options_))
|
||||
, select_query_options(select_query_options_)
|
||||
, query_tree(buildQueryTreeAndRunPasses(query, select_query_options, context, nullptr /*storage*/))
|
||||
@ -138,8 +166,9 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const ContextPtr & context_,
|
||||
const StoragePtr & storage_,
|
||||
const SelectQueryOptions & select_query_options_)
|
||||
: query(normalizeAndValidateQuery(query_))
|
||||
const SelectQueryOptions & select_query_options_,
|
||||
const Names & column_names)
|
||||
: query(normalizeAndValidateQuery(query_, column_names))
|
||||
, context(buildContext(context_, select_query_options_))
|
||||
, select_query_options(select_query_options_)
|
||||
, query_tree(buildQueryTreeAndRunPasses(query, select_query_options, context, storage_))
|
||||
|
@ -16,7 +16,8 @@ public:
|
||||
/// Initialize interpreter with query AST
|
||||
InterpreterSelectQueryAnalyzer(const ASTPtr & query_,
|
||||
const ContextPtr & context_,
|
||||
const SelectQueryOptions & select_query_options_);
|
||||
const SelectQueryOptions & select_query_options_,
|
||||
const Names & column_names = {});
|
||||
|
||||
/** Initialize interpreter with query AST and storage.
|
||||
* After query tree is built left most table expression is replaced with table node that
|
||||
@ -25,7 +26,8 @@ public:
|
||||
InterpreterSelectQueryAnalyzer(const ASTPtr & query_,
|
||||
const ContextPtr & context_,
|
||||
const StoragePtr & storage_,
|
||||
const SelectQueryOptions & select_query_options_);
|
||||
const SelectQueryOptions & select_query_options_,
|
||||
const Names & column_names = {});
|
||||
|
||||
/// Initialize interpreter with query tree
|
||||
InterpreterSelectQueryAnalyzer(const QueryTreeNodePtr & query_tree_,
|
||||
|
@ -164,7 +164,7 @@ void StorageView::read(
|
||||
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
InterpreterSelectQueryAnalyzer interpreter(current_inner_query, getViewContext(context, storage_snapshot), options);
|
||||
InterpreterSelectQueryAnalyzer interpreter(current_inner_query, getViewContext(context, storage_snapshot), options, column_names);
|
||||
interpreter.addStorageLimits(*query_info.storage_limits);
|
||||
query_plan = std::move(interpreter).extractQueryPlan();
|
||||
}
|
||||
|
@ -77,7 +77,7 @@ Expression (Project names)
|
||||
Distinct (DISTINCT)
|
||||
Union
|
||||
Distinct (Preliminary DISTINCT)
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized
|
||||
@ -85,7 +85,7 @@ Union
|
||||
Expression (Project names)
|
||||
Distinct (DISTINCT)
|
||||
Distinct (Preliminary DISTINCT)
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized
|
||||
@ -94,7 +94,7 @@ Expression (Project names)
|
||||
Union
|
||||
Expression (Before LIMIT BY)
|
||||
LimitBy
|
||||
Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))))
|
||||
Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))))
|
||||
ReadFromSystemNumbers
|
||||
Expression
|
||||
ReadFromRemote (Read from remote replica)
|
||||
@ -102,7 +102,7 @@ explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1,
|
||||
Union
|
||||
Expression (Project names)
|
||||
LimitBy
|
||||
Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))))
|
||||
Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized
|
||||
@ -113,7 +113,7 @@ Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
Distinct (Preliminary DISTINCT)
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
|
||||
@ -124,7 +124,7 @@ Expression (Project names)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
Distinct (Preliminary DISTINCT)
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))
|
||||
Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized
|
||||
@ -134,9 +134,9 @@ Expression (Project names)
|
||||
Sorting (Merge sorted streams for ORDER BY, without aggregation)
|
||||
Union
|
||||
LimitBy
|
||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) [lifted up part]))
|
||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))) [lifted up part]))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))))
|
||||
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
|
||||
@ -144,8 +144,8 @@ Expression (Project names)
|
||||
Sorting (Merge sorted streams after aggregation stage for ORDER BY)
|
||||
Union
|
||||
LimitBy
|
||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) [lifted up part]))
|
||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))) [lifted up part]))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))))
|
||||
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))))
|
||||
ReadFromSystemNumbers
|
||||
ReadFromRemote (Read from remote replica)
|
||||
|
@ -0,0 +1,8 @@
|
||||
Expression ((Project names + Projection))
|
||||
Header: sum(id) UInt64
|
||||
Aggregating
|
||||
Header: sum(__table1.id) UInt64
|
||||
Expression ((Before GROUP BY + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))
|
||||
Header: __table1.id UInt64
|
||||
ReadFromPreparedSource (Read from NullSource)
|
||||
Header: id UInt64
|
@ -0,0 +1,14 @@
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
CREATE TABLE test_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=MergeTree ORDER BY id;
|
||||
|
||||
DROP VIEW IF EXISTS test_view;
|
||||
CREATE VIEW test_view AS SELECT id, value FROM test_table;
|
||||
|
||||
EXPLAIN header = 1 SELECT sum(id) FROM test_view settings allow_experimental_analyzer=1;
|
||||
|
||||
DROP VIEW test_view;
|
||||
DROP TABLE test_table;
|
Loading…
Reference in New Issue
Block a user