mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
propagate insertion values storage into subquery
This commit is contained in:
parent
243207aa4e
commit
b680b1ac1e
@ -1056,7 +1056,7 @@ private:
|
||||
class QueryAnalyzer
|
||||
{
|
||||
public:
|
||||
void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context)
|
||||
void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context, const StoragePtr & storage = nullptr)
|
||||
{
|
||||
IdentifierResolveScope scope(node, nullptr /*parent_scope*/);
|
||||
|
||||
@ -1073,7 +1073,7 @@ public:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"For query analysis table expression must be empty");
|
||||
|
||||
resolveQuery(node, scope);
|
||||
resolveQuery(node, scope, storage);
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::UNION:
|
||||
@ -1187,7 +1187,7 @@ private:
|
||||
|
||||
QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context);
|
||||
|
||||
void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope);
|
||||
void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr);
|
||||
|
||||
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
|
||||
|
||||
@ -1292,9 +1292,9 @@ private:
|
||||
|
||||
ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope);
|
||||
|
||||
ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression);
|
||||
ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr);
|
||||
|
||||
ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression);
|
||||
ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage = nullptr);
|
||||
|
||||
ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope);
|
||||
|
||||
@ -1302,7 +1302,7 @@ private:
|
||||
|
||||
void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope);
|
||||
|
||||
NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope);
|
||||
NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr);
|
||||
|
||||
void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope);
|
||||
|
||||
@ -1316,7 +1316,7 @@ private:
|
||||
|
||||
void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor);
|
||||
|
||||
void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope);
|
||||
void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage = nullptr);
|
||||
|
||||
void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope);
|
||||
|
||||
@ -1834,7 +1834,7 @@ QueryTreeNodePtr QueryAnalyzer::tryGetLambdaFromSQLUserDefinedFunctions(const st
|
||||
}
|
||||
|
||||
/// Evaluate scalar subquery and perform constant folding if scalar subquery does not have constant value
|
||||
void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope)
|
||||
void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, IdentifierResolveScope & scope, const StoragePtr & storage)
|
||||
{
|
||||
auto * query_node = node->as<QueryNode>();
|
||||
auto * union_node = node->as<UnionNode>();
|
||||
@ -1870,7 +1870,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
|
||||
subquery_context->setSettings(subquery_settings);
|
||||
|
||||
auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/);
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(node->toAST(), subquery_context, options);
|
||||
auto interpreter = std::make_unique<InterpreterSelectQueryAnalyzer>(node->toAST(), subquery_context, storage, options);
|
||||
|
||||
auto io = interpreter->execute();
|
||||
|
||||
@ -5230,7 +5230,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
*
|
||||
* 4. If node has alias, update its value in scope alias map. Deregister alias from expression_aliases_in_resolve_process.
|
||||
*/
|
||||
ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression)
|
||||
ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
@ -5487,7 +5487,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id
|
||||
resolveUnion(node, subquery_scope);
|
||||
|
||||
if (!allow_table_expression)
|
||||
evaluateScalarSubqueryIfNeeded(node, subquery_scope);
|
||||
evaluateScalarSubqueryIfNeeded(node, subquery_scope, storage);
|
||||
|
||||
if (result_projection_names.empty())
|
||||
result_projection_names.push_back(std::move(projection_name));
|
||||
@ -5572,7 +5572,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id
|
||||
* Example: CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog; SELECT plus(*) FROM test_table;
|
||||
* Example: SELECT *** FROM system.one;
|
||||
*/
|
||||
ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression)
|
||||
ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, const StoragePtr & storage)
|
||||
{
|
||||
auto & node_list_typed = node_list->as<ListNode &>();
|
||||
size_t node_list_size = node_list_typed.getNodes().size();
|
||||
@ -5585,7 +5585,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node
|
||||
for (auto & node : node_list_typed.getNodes())
|
||||
{
|
||||
auto node_to_resolve = node;
|
||||
auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression);
|
||||
auto expression_node_projection_names = resolveExpressionNode(node_to_resolve, scope, allow_lambda_expression, allow_table_expression, storage);
|
||||
|
||||
size_t expected_projection_names_size = 1;
|
||||
if (auto * expression_list = node_to_resolve->as<ListNode>())
|
||||
@ -5772,9 +5772,9 @@ void QueryAnalyzer::resolveWindowNodeList(QueryTreeNodePtr & window_node_list, I
|
||||
resolveWindow(node, scope);
|
||||
}
|
||||
|
||||
NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope)
|
||||
NamesAndTypes QueryAnalyzer::resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope, const StoragePtr & storage)
|
||||
{
|
||||
ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
|
||||
ProjectionNames projection_names = resolveExpressionNodeList(projection_node_list, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/, storage);
|
||||
|
||||
auto projection_nodes = projection_node_list->as<ListNode &>().getNodes();
|
||||
size_t projection_nodes_size = projection_nodes.size();
|
||||
@ -6681,7 +6681,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
|
||||
* 10. Remove aliases from expression and lambda nodes.
|
||||
* 11. Resolve query tree node with projection columns.
|
||||
*/
|
||||
void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope)
|
||||
void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope, const StoragePtr & storage)
|
||||
{
|
||||
size_t max_subquery_depth = scope.context->getSettingsRef().max_subquery_depth;
|
||||
if (max_subquery_depth && scope.subquery_depth > max_subquery_depth)
|
||||
@ -6838,7 +6838,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
|
||||
|
||||
if (!scope.group_by_use_nulls)
|
||||
{
|
||||
projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope);
|
||||
projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope, storage);
|
||||
if (query_node_typed.getProjection().getNodes().empty())
|
||||
throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED,
|
||||
"Empty list of columns in projection. In scope {}",
|
||||
@ -7077,7 +7077,7 @@ QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_)
|
||||
void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
|
||||
{
|
||||
QueryAnalyzer analyzer;
|
||||
analyzer.resolve(query_tree_node, table_expression, context);
|
||||
analyzer.resolve(query_tree_node, table_expression, context, storage);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Analyzer/IQueryTreePass.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -71,7 +72,7 @@ public:
|
||||
/** Construct query analysis pass for query or union analysis.
|
||||
* Available columns are extracted from query node join tree.
|
||||
*/
|
||||
QueryAnalysisPass() = default;
|
||||
explicit QueryAnalysisPass(const StoragePtr & storage_ = nullptr) : storage(storage_) {}
|
||||
|
||||
/** Construct query analysis pass for expression or list of expressions analysis.
|
||||
* Available expression columns are extracted from table expression.
|
||||
@ -93,6 +94,7 @@ public:
|
||||
|
||||
private:
|
||||
QueryTreeNodePtr table_expression;
|
||||
const StoragePtr storage = nullptr;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -238,9 +238,9 @@ void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index)
|
||||
}
|
||||
}
|
||||
|
||||
void addQueryTreePasses(QueryTreePassManager & manager)
|
||||
void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage)
|
||||
{
|
||||
manager.addPass(std::make_unique<QueryAnalysisPass>());
|
||||
manager.addPass(std::make_unique<QueryAnalysisPass>(storage));
|
||||
manager.addPass(std::make_unique<FunctionToSubcolumnsPass>());
|
||||
|
||||
manager.addPass(std::make_unique<ConvertLogicalExpressionToCNFPass>());
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Analyzer/IQueryTreePass.h>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -44,6 +45,6 @@ private:
|
||||
std::vector<QueryTreePassPtr> passes;
|
||||
};
|
||||
|
||||
void addQueryTreePasses(QueryTreePassManager & manager);
|
||||
void addQueryTreePasses(QueryTreePassManager & manager, const StoragePtr & storage = nullptr);
|
||||
|
||||
}
|
||||
|
@ -109,6 +109,9 @@ void replaceStorageInQueryTree(QueryTreeNodePtr & query_tree, const ContextPtr &
|
||||
}
|
||||
}
|
||||
|
||||
if (auto * table_node = table_expression_to_replace->as<TableNode>(); table_node && table_node->getStorageID().getFullNameNotQuoted() != storage->getStorageID().getFullTableName())
|
||||
return;
|
||||
|
||||
auto replacement_table_expression = std::make_shared<TableNode>(storage, context);
|
||||
std::optional<TableExpressionModifiers> table_expression_modifiers;
|
||||
|
||||
@ -133,7 +136,7 @@ QueryTreeNodePtr buildQueryTreeAndRunPasses(const ASTPtr & query,
|
||||
auto query_tree = buildQueryTree(query, context);
|
||||
|
||||
QueryTreePassManager query_tree_pass_manager(context);
|
||||
addQueryTreePasses(query_tree_pass_manager);
|
||||
addQueryTreePasses(query_tree_pass_manager, storage);
|
||||
|
||||
if (select_query_options.ignore_ast_optimizations)
|
||||
query_tree_pass_manager.run(query_tree, 1 /*up_to_pass_index*/);
|
||||
|
Loading…
Reference in New Issue
Block a user