Merge pull request #43321 from kitaisreal/analyzer-small-fixes

Analyzer small fixes
This commit is contained in:
Maksim Kita 2022-11-23 13:19:32 +03:00 committed by GitHub
commit 43a451d55e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 135 additions and 86 deletions

60
src/Analyzer/HashUtils.h Normal file
View File

@ -0,0 +1,60 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** This structure holds query tree node ptr and its hash. It can be used as hash map key to avoid unnecessary hash
* recalculations.
*
* Example of usage:
* std::unordered_map<QueryTreeNodeConstRawPtrWithHash, std::string> map;
*/
template <typename QueryTreeNodePtrType>
struct QueryTreeNodeWithHash
{
QueryTreeNodeWithHash(QueryTreeNodePtrType node_) /// NOLINT
: node(std::move(node_))
, hash(node->getTreeHash().first)
{}
QueryTreeNodePtrType node = nullptr;
size_t hash = 0;
};
template <typename T>
inline bool operator==(const QueryTreeNodeWithHash<T> & lhs, const QueryTreeNodeWithHash<T> & rhs)
{
return lhs.hash == rhs.hash && lhs.node->isEqual(*rhs.node);
}
template <typename T>
inline bool operator!=(const QueryTreeNodeWithHash<T> & lhs, const QueryTreeNodeWithHash<T> & rhs)
{
return !(lhs == rhs);
}
using QueryTreeNodePtrWithHash = QueryTreeNodeWithHash<QueryTreeNodePtr>;
using QueryTreeNodeRawPtrWithHash = QueryTreeNodeWithHash<IQueryTreeNode *>;
using QueryTreeNodeConstRawPtrWithHash = QueryTreeNodeWithHash<const IQueryTreeNode *>;
using QueryTreeNodePtrWithHashSet = std::unordered_set<QueryTreeNodePtrWithHash>;
using QueryTreeNodeConstRawPtrWithHashSet = std::unordered_set<QueryTreeNodeConstRawPtrWithHash>;
template <typename Value>
using QueryTreeNodePtrWithHashMap = std::unordered_map<QueryTreeNodePtrWithHash, Value>;
template <typename Value>
using QueryTreeNodeConstRawPtrWithHashMap = std::unordered_map<QueryTreeNodeConstRawPtrWithHash, Value>;
}
template <typename T>
struct std::hash<DB::QueryTreeNodeWithHash<T>>
{
size_t operator()(const DB::QueryTreeNodeWithHash<T> & node_with_hash) const
{
return node_with_hash.hash;
}
};

View File

@ -8,6 +8,7 @@
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/HashUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
@ -48,43 +49,24 @@ public:
/// Do not apply for `count()` with without arguments or `count(*)`, only `count(x)` is supported.
return;
mapping[QueryTreeNodeWithHash(argument_nodes[0])].push_back(&node);
argument_to_functions_mapping[argument_nodes[0]].push_back(&node);
}
struct QueryTreeNodeWithHash
{
const QueryTreeNodePtr & node;
IQueryTreeNode::Hash hash;
explicit QueryTreeNodeWithHash(const QueryTreeNodePtr & node_)
: node(node_)
, hash(node->getTreeHash())
{}
bool operator==(const QueryTreeNodeWithHash & rhs) const
{
return hash == rhs.hash && node->isEqual(*rhs.node);
}
struct Hash
{
size_t operator() (const QueryTreeNodeWithHash & key) const { return key.hash.first ^ key.hash.second; }
};
};
/// argument -> list of sum/count/avg functions with this argument
std::unordered_map<QueryTreeNodeWithHash, std::vector<QueryTreeNodePtr *>, QueryTreeNodeWithHash::Hash> mapping;
QueryTreeNodePtrWithHashMap<std::vector<QueryTreeNodePtr *>> argument_to_functions_mapping;
private:
std::unordered_set<String> names_to_collect;
};
QueryTreeNodePtr createResolvedFunction(ContextPtr context, const String & name, DataTypePtr result_type, QueryTreeNodes arguments)
QueryTreeNodePtr createResolvedFunction(const ContextPtr & context, const String & name, const DataTypePtr & result_type, QueryTreeNodes arguments)
{
auto function_node = std::make_shared<FunctionNode>(name);
auto function = FunctionFactory::instance().get(name, context);
function_node->resolveAsFunction(std::move(function), result_type);
function_node->getArguments().getNodes() = std::move(arguments);
return function_node;
}
@ -94,21 +76,20 @@ FunctionNodePtr createResolvedAggregateFunction(const String & name, const Query
AggregateFunctionProperties properties;
auto aggregate_function = AggregateFunctionFactory::instance().get(name, {argument->getResultType()}, parameters, properties);
function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType());
function_node->getArguments().getNodes() = { argument };
function_node->getArgumentsNode() = std::make_shared<ListNode>(QueryTreeNodes{argument});
return function_node;
}
QueryTreeNodePtr createTupleElementFunction(ContextPtr context, DataTypePtr result_type, QueryTreeNodePtr argument, UInt64 index)
QueryTreeNodePtr createTupleElementFunction(const ContextPtr & context, const DataTypePtr & result_type, QueryTreeNodePtr argument, UInt64 index)
{
return createResolvedFunction(context, "tupleElement", result_type, {argument, std::make_shared<ConstantNode>(index)});
return createResolvedFunction(context, "tupleElement", result_type, {std::move(argument), std::make_shared<ConstantNode>(index)});
}
QueryTreeNodePtr createArrayElementFunction(ContextPtr context, DataTypePtr result_type, QueryTreeNodePtr argument, UInt64 index)
QueryTreeNodePtr createArrayElementFunction(const ContextPtr & context, const DataTypePtr & result_type, QueryTreeNodePtr argument, UInt64 index)
{
return createResolvedFunction(context, "arrayElement", result_type, {argument, std::make_shared<ConstantNode>(index)});
return createResolvedFunction(context, "arrayElement", result_type, {std::move(argument), std::make_shared<ConstantNode>(index)});
}
void replaceWithSumCount(QueryTreeNodePtr & node, const FunctionNodePtr & sum_count_node, ContextPtr context)
@ -151,6 +132,7 @@ FunctionNodePtr createFusedQuantilesNode(const std::vector<QueryTreeNodePtr *> n
{
Array parameters;
parameters.reserve(nodes.size());
for (const auto * node : nodes)
{
const FunctionNode & function_node = (*node)->as<const FunctionNode &>();
@ -172,6 +154,7 @@ FunctionNodePtr createFusedQuantilesNode(const std::vector<QueryTreeNodePtr *> n
parameters.push_back(constant_value->getValue());
}
return createResolvedAggregateFunction("quantiles", argument, parameters);
}
@ -181,7 +164,7 @@ void tryFuseSumCountAvg(QueryTreeNodePtr query_tree_node, ContextPtr context)
FuseFunctionsVisitor visitor({"sum", "count", "avg"});
visitor.visit(query_tree_node);
for (auto & [argument, nodes] : visitor.mapping)
for (auto & [argument, nodes] : visitor.argument_to_functions_mapping)
{
if (nodes.size() < 2)
continue;
@ -199,24 +182,22 @@ void tryFuseQuantiles(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
FuseFunctionsVisitor visitor_quantile({"quantile"});
visitor_quantile.visit(query_tree_node);
for (auto & [argument, nodes] : visitor_quantile.mapping)
for (auto & [argument, nodes] : visitor_quantile.argument_to_functions_mapping)
{
if (nodes.size() < 2)
size_t nodes_size = nodes.size();
if (nodes_size < 2)
continue;
auto quantiles_node = createFusedQuantilesNode(nodes, argument.node);
auto result_array_type = std::dynamic_pointer_cast<const DataTypeArray>(quantiles_node->getResultType());
if (!result_array_type)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected return type '{}' of function '{}', should be array",
quantiles_node->getResultType(), quantiles_node->getFunctionName());
}
for (size_t i = 0; i < nodes.size(); ++i)
{
for (size_t i = 0; i < nodes_size; ++i)
*nodes[i] = createArrayElementFunction(context, result_array_type->getNestedType(), quantiles_node, i + 1);
}
}
}

View File

@ -3,6 +3,7 @@
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/SortNode.h>
#include <Analyzer/HashUtils.h>
namespace DB
{
@ -10,35 +11,6 @@ namespace DB
namespace
{
struct QueryTreeNodeWithHash
{
explicit QueryTreeNodeWithHash(const IQueryTreeNode * node_)
: node(node_)
, hash(node->getTreeHash().first)
{}
const IQueryTreeNode * node = nullptr;
size_t hash = 0;
};
struct QueryTreeNodeWithHashHash
{
size_t operator()(const QueryTreeNodeWithHash & node_with_hash) const
{
return node_with_hash.hash;
}
};
struct QueryTreeNodeWithHashEqualTo
{
bool operator()(const QueryTreeNodeWithHash & lhs_node, const QueryTreeNodeWithHash & rhs_node) const
{
return lhs_node.hash == rhs_node.hash && lhs_node.node->isEqual(*rhs_node.node);
}
};
using QueryTreeNodeWithHashSet = std::unordered_set<QueryTreeNodeWithHash, QueryTreeNodeWithHashHash, QueryTreeNodeWithHashEqualTo>;
class OrderByLimitByDuplicateEliminationVisitor : public InDepthQueryTreeVisitor<OrderByLimitByDuplicateEliminationVisitor>
{
public:
@ -93,7 +65,7 @@ public:
}
private:
QueryTreeNodeWithHashSet unique_expressions_nodes_set;
QueryTreeNodeConstRawPtrWithHashSet unique_expressions_nodes_set;
};
}

View File

@ -519,7 +519,7 @@ public:
private:
QueryTreeNodes expressions;
std::unordered_map<std::string, std::vector<QueryTreeNodePtr>> alias_name_to_expressions;
std::unordered_map<std::string, QueryTreeNodes> alias_name_to_expressions;
};
/** Projection names is name of query tree node that is used in projection part of query node.
@ -2239,18 +2239,19 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
auto & alias_identifier_node = it->second->as<IdentifierNode &>();
auto identifier = alias_identifier_node.getIdentifier();
auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings);
if (!lookup_result.isResolved())
if (!lookup_result.resolved_identifier)
{
std::unordered_set<Identifier> valid_identifiers;
collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers);
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}' in scope {}{}",
toStringLowercase(IdentifierLookupContext::EXPRESSION),
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}'. In scope {}{}",
toStringLowercase(identifier_lookup.lookup_context),
identifier.getFullName(),
scope.scope_node->formatASTForErrorMessage(),
getHintsErrorMessageSuffix(hints));
}
it->second = lookup_result.resolved_identifier;
/** During collection of aliases if node is identifier and has alias, we cannot say if it is
@ -2261,9 +2262,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
* If we resolved identifier node as function, we must remove identifier node alias from
* expression alias map.
*/
if (identifier_lookup.isExpressionLookup() && it->second)
if (identifier_lookup.isExpressionLookup())
scope.alias_name_to_lambda_node.erase(identifier_bind_part);
else if (identifier_lookup.isFunctionLookup() && it->second)
else if (identifier_lookup.isFunctionLookup())
scope.alias_name_to_expression_node.erase(identifier_bind_part);
scope.expressions_in_resolve_process_stack.popNode();
@ -3271,11 +3272,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
if (auto * array_join_node = table_expression->as<ArrayJoinNode>())
{
size_t table_expressions_column_nodes_with_names_stack_size = table_expressions_column_nodes_with_names_stack.size();
if (table_expressions_column_nodes_with_names_stack_size < 1)
if (table_expressions_column_nodes_with_names_stack.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected at least 1 table expressions on stack before ARRAY JOIN processing. Actual {}",
table_expressions_column_nodes_with_names_stack_size);
"Expected at least 1 table expressions on stack before ARRAY JOIN processing");
auto & table_expression_column_nodes_with_names = table_expressions_column_nodes_with_names_stack.back();

View File

@ -87,8 +87,8 @@ void JoinClause::dump(WriteBuffer & buffer) const
{
const auto & asof_condition = asof_conditions[i];
buffer << "key_index: " << asof_condition.key_index;
buffer << "inequality: " << toString(asof_condition.asof_inequality);
buffer << " key_index: " << asof_condition.key_index;
buffer << " inequality: " << toString(asof_condition.asof_inequality);
if (i + 1 != asof_conditions_size)
buffer << ',';

View File

@ -183,19 +183,19 @@ public:
}
private:
/// Valid for table, table function, query, union, array join table expression nodes
/// Valid for table, table function, array join, query, union nodes
NamesAndTypesList columns;
/// Valid for table, table function, query, union, array join table expression nodes
/// Valid for table, table function, array join, query, union nodes
NameSet columns_names;
/// Valid only for table table expression node
/// Valid only for table node
NameSet alias_columns_names;
/// Valid for table, table function, query, union table, array join expression nodes
/// Valid for table, table function, array join, query, union nodes
ColumnNameToColumnIdentifier column_name_to_column_identifier;
/// Valid for table, table function, query, union table, array join expression nodes
/// Valid for table, table function, array join, query, union nodes
ColumnIdentifierToColumnName column_identifier_to_column_name;
/// Is storage remote

View File

@ -0,0 +1,36 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS test_table_join_1;
CREATE TABLE test_table_join_1
(
id UInt8,
value String
)
ENGINE = TinyLog;
INSERT INTO test_table_join_1 VALUES (0, 'Value_0');
DROP TABLE IF EXISTS test_table_join_2;
CREATE TABLE test_table_join_2
(
id UInt16,
value String
)
ENGINE = TinyLog;
INSERT INTO test_table_join_2 VALUES (0, 'Value_1');
SELECT
toTypeName(t2_value),
t2.value AS t2_value
FROM test_table_join_1 AS t1
INNER JOIN test_table_join_2 USING (id); -- { serverError 47 };
SELECT
toTypeName(t2_value),
t2.value AS t2_value
FROM test_table_join_1 AS t1
INNER JOIN test_table_join_2 AS t2 USING (id);
DROP TABLE test_table_join_1;
DROP TABLE test_table_join_2;