Analyzer support ARRAY JOIN with Nested

This commit is contained in:
Maksim Kita 2023-02-04 16:51:43 +01:00
parent 077555bd5d
commit af38660cf5
14 changed files with 601 additions and 237 deletions

View File

@ -2,6 +2,7 @@
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/Utils.h>
namespace DB
{
@ -74,41 +75,9 @@ void assertNoAggregateFunctionNodes(const QueryTreeNodePtr & node, const String
visitor.visit(node);
}
namespace
{
class ValidateGroupingFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor<ValidateGroupingFunctionNodesVisitor>
{
public:
explicit ValidateGroupingFunctionNodesVisitor(String assert_no_grouping_function_place_message_)
: assert_no_grouping_function_place_message(std::move(assert_no_grouping_function_place_message_))
{}
void visitImpl(const QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (function_node && function_node->getFunctionName() == "grouping")
throw Exception(ErrorCodes::ILLEGAL_AGGREGATION,
"GROUPING function {} is found {} in query",
function_node->formatASTForErrorMessage(),
assert_no_grouping_function_place_message);
}
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node)
{
return !(child_node->getNodeType() == QueryTreeNodeType::QUERY || child_node->getNodeType() == QueryTreeNodeType::UNION);
}
private:
String assert_no_grouping_function_place_message;
};
}
void assertNoGroupingFunction(const QueryTreeNodePtr & node, const String & assert_no_grouping_function_place_message)
{
ValidateGroupingFunctionNodesVisitor visitor(assert_no_grouping_function_place_message);
visitor.visit(node);
assertNoFunction(node, "grouping", ErrorCodes::ILLEGAL_AGGREGATION, "GROUPING", assert_no_grouping_function_place_message);
}
}

View File

@ -14,6 +14,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeLowCardinality.h>
@ -108,6 +109,7 @@ namespace ErrorCodes
extern const int ALIAS_REQUIRED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_PREWHERE;
}
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before.
@ -1179,7 +1181,13 @@ private:
QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope);
static bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, const IdentifierResolveScope & scope);
static bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
const IdentifierResolveScope & scope);
static bool tryBindIdentifierToTableExpressions(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
const IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
@ -1189,6 +1197,10 @@ private:
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveExpressionFromArrayJoinExpressions(const QueryTreeNodePtr & resolved_expression,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromArrayJoin(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
@ -2595,6 +2607,25 @@ bool QueryAnalyzer::tryBindIdentifierToTableExpression(const IdentifierLookup &
return false;
}
bool QueryAnalyzer::tryBindIdentifierToTableExpressions(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node_to_ignore,
const IdentifierResolveScope & scope)
{
bool can_bind_identifier_to_table_expression = false;
for (const auto & [table_expression_node, _] : scope.table_expression_node_to_data)
{
if (table_expression_node.get() == table_expression_node_to_ignore.get())
continue;
can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(identifier_lookup, table_expression_node, scope);
if (can_bind_identifier_to_table_expression)
break;
}
return can_bind_identifier_to_table_expression;
}
QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope)
@ -2637,8 +2668,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id
auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr
{
auto identifier_view = IdentifierView(identifier);
identifier_view.popFirst(identifier_column_qualifier_parts);
auto identifier_without_column_qualifier = identifier;
identifier_without_column_qualifier.popFirst(identifier_column_qualifier_parts);
/** Compound identifier cannot be resolved directly from storage if storage is not table.
*
@ -2649,42 +2680,90 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id
* Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required.
*/
ColumnNodePtr result_column;
bool compound_identifier = identifier_view.getPartsSize() > 1;
QueryTreeNodePtr result_expression;
bool match_full_identifier = false;
auto it = table_expression_data.column_name_to_column_node.find(std::string(identifier_view.getFullName()));
auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.getFullName());
if (it != table_expression_data.column_name_to_column_node.end())
{
match_full_identifier = true;
result_column = it->second;
result_expression = it->second;
}
else
{
it = table_expression_data.column_name_to_column_node.find(std::string(identifier_view.at(0)));
it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0));
if (it != table_expression_data.column_name_to_column_node.end())
result_column = it->second;
result_expression = it->second;
}
QueryTreeNodePtr result_expression = result_column;
bool clone_is_needed = true;
String table_expression_source = table_expression_data.table_expression_description;
if (!table_expression_data.table_expression_name.empty())
table_expression_source += " with name " + table_expression_data.table_expression_name;
if (result_column && !match_full_identifier && compound_identifier)
if (result_expression && !match_full_identifier && identifier_without_column_qualifier.isCompound())
{
size_t identifier_bind_size = identifier_column_qualifier_parts + 1;
result_expression = tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier,
identifier_bind_size,
result_column,
result_expression,
table_expression_source,
scope);
clone_is_needed = false;
}
if (!result_expression)
{
QueryTreeNodes nested_column_nodes;
DataTypes nested_types;
Array nested_names_array;
for (auto & [column_name, _] : table_expression_data.column_names_and_types)
{
Identifier column_name_identifier_without_last_part(column_name);
auto column_name_identifier_last_part = column_name_identifier_without_last_part.getParts().back();
column_name_identifier_without_last_part.popLast();
if (identifier_without_column_qualifier.getFullName() != column_name_identifier_without_last_part.getFullName())
continue;
auto column_node_it = table_expression_data.column_name_to_column_node.find(column_name);
if (column_node_it == table_expression_data.column_name_to_column_node.end())
continue;
const auto & column_node = column_node_it->second;
const auto & column_type = column_node->getColumnType();
const auto * column_type_array = typeid_cast<const DataTypeArray *>(column_type.get());
if (!column_type_array)
continue;
nested_column_nodes.push_back(column_node);
nested_types.push_back(column_type_array->getNestedType());
nested_names_array.push_back(Field(std::move(column_name_identifier_last_part)));
}
if (!nested_types.empty())
{
auto nested_function_node = std::make_shared<FunctionNode>("nested");
auto & nested_function_node_arguments = nested_function_node->getArguments().getNodes();
auto nested_function_names_array_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
auto nested_function_names_constant_node = std::make_shared<ConstantNode>(std::move(nested_names_array),
std::move(nested_function_names_array_type));
nested_function_node_arguments.push_back(std::move(nested_function_names_constant_node));
nested_function_node_arguments.insert(nested_function_node_arguments.end(),
nested_column_nodes.begin(),
nested_column_nodes.end());
auto nested_function = FunctionFactory::instance().get(nested_function_node->getFunctionName(), scope.context);
nested_function_node->resolveAsFunction(nested_function->build(nested_function_node->getArgumentColumns()));
clone_is_needed = false;
result_expression = std::move(nested_function_node);
}
}
if (!result_expression)
{
std::unordered_set<Identifier> valid_identifiers;
@ -2715,32 +2794,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id
if (qualified_identifier_with_removed_part.empty())
break;
IdentifierLookup bind_to_aliases_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION};
if (tryBindIdentifierToAliases(bind_to_aliases_identifier_lookup, scope))
break;
bool can_remove_qualificator = true;
if (table_expression_data.should_qualify_columns)
{
for (auto & table_expression_to_check_data : scope.table_expression_node_to_data)
{
const auto & table_expression_to_check = table_expression_to_check_data.first;
if (table_expression_to_check.get() == table_expression_node.get())
continue;
IdentifierLookup column_identifier_lookup{qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION};
bool can_bind_identifier_to_table_expression = tryBindIdentifierToTableExpression(column_identifier_lookup, table_expression_to_check, scope);
if (can_bind_identifier_to_table_expression)
{
can_remove_qualificator = false;
break;
}
}
}
if (!can_remove_qualificator)
IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION};
if (tryBindIdentifierToAliases(column_identifier_lookup, scope) ||
tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope))
break;
qualified_identifier = std::move(qualified_identifier_with_removed_part);
@ -2917,6 +2973,79 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo
return resolved_identifier;
}
QueryTreeNodePtr QueryAnalyzer::tryResolveExpressionFromArrayJoinExpressions(const QueryTreeNodePtr & resolved_expression,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope)
{
const auto & array_join_node = table_expression_node->as<const ArrayJoinNode &>();
const auto & array_join_column_expressions_list = array_join_node.getJoinExpressions();
const auto & array_join_column_expressions_nodes = array_join_column_expressions_list.getNodes();
QueryTreeNodePtr array_join_resolved_expression;
/** Special case when qualified or unqualified identifier point to array join expression without alias.
*
* CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog;
* SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array;
*
* value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression.
*/
for (const auto & array_join_column_expression : array_join_column_expressions_nodes)
{
auto & array_join_column_expression_typed = array_join_column_expression->as<ColumnNode &>();
if (array_join_column_expression_typed.hasAlias())
continue;
auto & array_join_column_inner_expression = array_join_column_expression_typed.getExpressionOrThrow();
auto * array_join_column_inner_expression_function = array_join_column_inner_expression->as<FunctionNode>();
if (array_join_column_inner_expression_function &&
array_join_column_inner_expression_function->getFunctionName() == "nested" &&
array_join_column_inner_expression_function->getArguments().getNodes().size() > 1 &&
isTuple(array_join_column_expression_typed.getResultType()))
{
const auto & nested_function_arguments = array_join_column_inner_expression_function->getArguments().getNodes();
size_t nested_function_arguments_size = nested_function_arguments.size();
const auto & nested_keys_names_constant_node = nested_function_arguments[0]->as<ConstantNode & >();
const auto & nested_keys_names = nested_keys_names_constant_node.getValue().get<Array &>();
size_t nested_keys_names_size = nested_keys_names.size();
if (nested_keys_names_size == nested_function_arguments_size - 1)
{
for (size_t i = 1; i < nested_function_arguments_size; ++i)
{
if (!nested_function_arguments[i]->isEqual(*resolved_expression))
continue;
auto array_join_column = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
const auto & nested_key_name = nested_keys_names[i - 1].get<String &>();
Identifier nested_identifier = Identifier(nested_key_name);
auto tuple_element_function = wrapExpressionNodeInTupleElement(array_join_column, nested_identifier);
resolveFunction(tuple_element_function, scope);
array_join_resolved_expression = std::move(tuple_element_function);
break;
}
}
}
if (array_join_resolved_expression)
break;
if (array_join_column_inner_expression->isEqual(*resolved_expression))
{
array_join_resolved_expression = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
break;
}
}
return array_join_resolved_expression;
}
QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope)
@ -2932,43 +3061,27 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi
/** Allow JOIN with USING with ARRAY JOIN.
*
* SELECT * FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id
* SELECT * FROM test_table_1 AS t1 ARRAY JOIN t1.id AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id
* SELECT * FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] AS id INNER JOIN test_table_2 AS t2 USING (id);
* SELECT * FROM test_table_1 AS t1 ARRAY JOIN t1.id AS id INNER JOIN test_table_2 AS t2 USING (id);
*/
for (const auto & array_join_column_expression : array_join_column_expressions_nodes)
{
auto & array_join_column_expression_typed = array_join_column_expression->as<ColumnNode &>();
if (identifier_lookup.identifier.isShort() &&
array_join_column_expression_typed.getAlias() == identifier_lookup.identifier.getFullName())
return array_join_column_expression;
if (array_join_column_expression_typed.getAlias() == identifier_lookup.identifier.getFullName())
{
auto array_join_column = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
return array_join_column;
}
}
if (!resolved_identifier)
return nullptr;
/** Special case when qualified or unqualified identifier point to array join expression without alias.
*
* CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog;
* SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array;
*
* value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression.
*/
for (const auto & array_join_column_expression : array_join_column_expressions_nodes)
{
auto & array_join_column_expression_typed = array_join_column_expression->as<ColumnNode &>();
if (array_join_column_expression_typed.hasAlias())
continue;
auto & array_join_column_inner_expression = array_join_column_expression_typed.getExpressionOrThrow();
if (array_join_column_inner_expression.get() == resolved_identifier.get() ||
array_join_column_inner_expression->isEqual(*resolved_identifier))
{
resolved_identifier = array_join_column_expression;
break;
}
}
auto array_join_resolved_expression = tryResolveExpressionFromArrayJoinExpressions(resolved_identifier, table_expression_node, scope);
if (array_join_resolved_expression)
resolved_identifier = std::move(array_join_resolved_expression);
return resolved_identifier;
}
@ -3313,26 +3426,13 @@ void QueryAnalyzer::qualifyColumnNodesWithProjectionNames(const QueryTreeNodes &
/// Iterate over additional column qualifications and apply them if needed
for (size_t i = 0; i < additional_column_qualification_parts_size; ++i)
{
bool need_to_qualify = false;
auto identifier_to_check = Identifier(column_qualified_identifier_parts);
IdentifierLookup lookup{identifier_to_check, IdentifierLookupContext::EXPRESSION};
IdentifierLookup identifier_lookup{identifier_to_check, IdentifierLookupContext::EXPRESSION};
bool need_to_qualify = table_expression_data.should_qualify_columns;
if (need_to_qualify)
need_to_qualify = tryBindIdentifierToTableExpressions(identifier_lookup, table_expression_node, scope);
if (table_expression_data.should_qualify_columns)
{
for (const auto & scope_table_expression_data : scope.table_expression_node_to_data)
{
if (scope_table_expression_data.first.get() == table_expression_node.get())
continue;
if (tryBindIdentifierToTableExpression(lookup, scope_table_expression_data.first, scope))
{
need_to_qualify = true;
break;
}
}
}
if (tryBindIdentifierToAliases(lookup, scope))
if (tryBindIdentifierToAliases(identifier_lookup, scope))
need_to_qualify = true;
if (need_to_qualify)
@ -3575,55 +3675,35 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree());
std::vector<QueryTreeNodesWithNames> table_expressions_column_nodes_with_names_stack;
std::unordered_set<std::string> left_table_expression_column_names_to_skip;
std::unordered_set<std::string> right_table_expression_column_names_to_skip;
std::unordered_set<std::string> table_expression_column_names_to_skip;
for (auto & table_expression : table_expressions_stack)
{
bool table_expression_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression.get());
if (auto * array_join_node = table_expression->as<ArrayJoinNode>())
{
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");
if (table_expression_in_resolve_process)
continue;
auto & table_expression_column_nodes_with_names = table_expressions_column_nodes_with_names_stack.back();
const auto & array_join_column_list = array_join_node->getJoinExpressions();
const auto & array_join_column_nodes = array_join_column_list.getNodes();
/** Special case with ARRAY JOIN column without alias.
*
* CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog;
* SELECT * FROM test_table ARRAY JOIN value_array;
*
* In matched columns `value_array` must be resolved into array join column.
*/
for (const auto & array_join_column_node : array_join_column_nodes)
for (auto & [table_expression_column_node, _] : table_expression_column_nodes_with_names)
{
if (array_join_column_node->hasAlias())
continue;
auto array_join_column_inner_expression = array_join_column_node->as<ColumnNode &>().getExpressionOrThrow();
if (array_join_column_inner_expression->getNodeType() != QueryTreeNodeType::COLUMN)
continue;
for (auto & table_expressions_column_node_with_name : table_expression_column_nodes_with_names)
{
auto & table_expression_column_node = table_expressions_column_node_with_name.first;
if (table_expression_column_node.get() == array_join_column_inner_expression.get() ||
table_expression_column_node->isEqual(*array_join_column_inner_expression))
{
table_expression_column_node = array_join_column_node;
}
}
auto array_join_resolved_expression = tryResolveExpressionFromArrayJoinExpressions(table_expression_column_node,
table_expression,
scope);
if (array_join_resolved_expression)
table_expression_column_node = std::move(array_join_resolved_expression);
}
continue;
}
bool table_expression_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression.get());
auto * join_node = table_expression->as<JoinNode>();
if (join_node)
@ -3640,8 +3720,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
auto left_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back());
table_expressions_column_nodes_with_names_stack.pop_back();
left_table_expression_column_names_to_skip.clear();
right_table_expression_column_names_to_skip.clear();
table_expression_column_names_to_skip.clear();
QueryTreeNodesWithNames matched_expression_nodes_with_column_names;
@ -3656,13 +3735,13 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
for (auto & join_using_node : join_using_list.getNodes())
{
auto & column_node = join_using_node->as<ColumnNode &>();
const auto & using_column_name = column_node.getColumnName();
auto & join_using_column_node = join_using_node->as<ColumnNode &>();
const auto & join_using_column_name = join_using_column_node.getColumnName();
if (!matcher_node_typed.isMatchingColumn(using_column_name))
if (!matcher_node_typed.isMatchingColumn(join_using_column_name))
continue;
const auto & join_using_column_nodes_list = column_node.getExpressionOrThrow()->as<ListNode &>();
const auto & join_using_column_nodes_list = join_using_column_node.getExpressionOrThrow()->as<ListNode &>();
const auto & join_using_column_nodes = join_using_column_nodes_list.getNodes();
QueryTreeNodePtr matched_column_node;
@ -3672,40 +3751,17 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
else
matched_column_node = join_using_column_nodes.at(0);
/** It is possible that in USING there is JOIN with array joined column.
* SELECT * FROM (SELECT [0] AS value) AS t1 ARRAY JOIN value AS id INNER JOIN test_table USING (id);
* In such example match `value` column from t1, and all columns from test_table except `id`.
*
* SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN id INNER JOIN test_table USING (id);
* In such example, match `id` column from ARRAY JOIN, and all columns from test_table except `id`.
*
* SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN id AS id INNER JOIN test_table USING (id);
* In such example match `id` column from t1, and all columns from test_table except `id`.
*
* SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
* In such example match `id` column from t1, and all columns from test_table except `id`.
*/
auto matched_column_source = matched_column_node->as<ColumnNode &>().getColumnSource();
matched_column_node = matched_column_node->clone();
matched_column_node->as<ColumnNode &>().setColumnType(join_using_column_node.getResultType());
if (matched_column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN && matched_column_node->hasAlias())
{
if (isRight(join_node->getKind()))
left_table_expression_column_names_to_skip.insert(using_column_name);
else
right_table_expression_column_names_to_skip.insert(using_column_name);
}
else
{
left_table_expression_column_names_to_skip.insert(using_column_name);
right_table_expression_column_names_to_skip.insert(using_column_name);
matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), using_column_name);
}
table_expression_column_names_to_skip.insert(join_using_column_name);
matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), join_using_column_name);
}
}
for (auto && left_table_column_with_name : left_table_expression_columns)
{
if (left_table_expression_column_names_to_skip.contains(left_table_column_with_name.second))
if (table_expression_column_names_to_skip.contains(left_table_column_with_name.second))
continue;
matched_expression_nodes_with_column_names.push_back(std::move(left_table_column_with_name));
@ -3713,7 +3769,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
for (auto && right_table_column_with_name : right_table_expression_columns)
{
if (right_table_expression_column_names_to_skip.contains(right_table_column_with_name.second))
if (table_expression_column_names_to_skip.contains(right_table_column_with_name.second))
continue;
matched_expression_nodes_with_column_names.push_back(std::move(right_table_column_with_name));
@ -5854,6 +5910,10 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
auto & array_join_nodes = array_join_node_typed.getJoinExpressions().getNodes();
size_t array_join_nodes_size = array_join_nodes.size();
if (array_join_nodes_size == 0)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"ARRAY JOIN requires at least single expression");
std::vector<QueryTreeNodePtr> array_join_column_expressions;
array_join_column_expressions.reserve(array_join_nodes_size);
@ -5870,18 +5930,28 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
/// Add array join expression into scope
expressions_visitor.visit(array_join_expression);
std::string identifier_full_name;
if (auto * identifier_node = array_join_expression->as<IdentifierNode>())
identifier_full_name = identifier_node->getIdentifier().getFullName();
resolveExpressionNode(array_join_expression, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/);
auto result_type = array_join_expression->getResultType();
bool is_array_type = isArray(result_type);
bool is_map_type = isMap(result_type);
if (!isArray(result_type))
if (!is_array_type && !is_map_type)
throw Exception(ErrorCodes::TYPE_MISMATCH,
"ARRAY JOIN {} requires expression {} with Array type. Actual {}. In scope {}",
"ARRAY JOIN {} requires expression {} with Array or Map type. Actual {}. In scope {}",
array_join_node_typed.formatASTForErrorMessage(),
array_join_expression->formatASTForErrorMessage(),
result_type->getName(),
scope.scope_node->formatASTForErrorMessage());
if (is_map_type)
result_type = assert_cast<const DataTypeMap &>(*result_type).getNestedType();
result_type = assert_cast<const DataTypeArray &>(*result_type).getNestedType();
String array_join_column_name;
@ -5894,6 +5964,10 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
{
array_join_column_name = array_join_expression_inner_column->getColumnName();
}
else if (!identifier_full_name.empty())
{
array_join_column_name = identifier_full_name;
}
else
{
array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter);
@ -5925,11 +5999,17 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif
*/
for (size_t i = 0; i < array_join_nodes_size; ++i)
{
auto & array_join_expression = array_join_nodes[i];
array_join_expression = std::move(array_join_column_expressions[i]);
auto it = scope.alias_name_to_expression_node.find(array_join_expression->getAlias());
auto & array_join_column_expression = array_join_nodes[i];
array_join_column_expression = std::move(array_join_column_expressions[i]);
auto it = scope.alias_name_to_expression_node.find(array_join_column_expression->getAlias());
if (it != scope.alias_name_to_expression_node.end())
it->second = array_join_nodes[i];
{
auto & array_join_column_expression_typed = array_join_column_expression->as<ColumnNode &>();
auto array_join_column = std::make_shared<ColumnNode>(array_join_column_expression_typed.getColumn(),
array_join_column_expression_typed.getColumnSource());
it->second = std::move(array_join_column);
}
}
}
@ -5984,6 +6064,13 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS
identifier_full_name,
scope.scope_node->formatASTForErrorMessage());
if (result_left_table_expression->getNodeType() != QueryTreeNodeType::COLUMN)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"JOIN {} using identifier '{}' must be resolved into column node from left table expression. In scope {}",
join_node_typed.formatASTForErrorMessage(),
identifier_full_name,
scope.scope_node->formatASTForErrorMessage());
auto result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node_typed.getRightTableExpression(), scope);
if (!result_right_table_expression)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
@ -5992,6 +6079,13 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS
identifier_full_name,
scope.scope_node->formatASTForErrorMessage());
if (result_right_table_expression->getNodeType() != QueryTreeNodeType::COLUMN)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"JOIN {} using identifier '{}' must be resolved into column node from right table expression. In scope {}",
join_node_typed.formatASTForErrorMessage(),
identifier_full_name,
scope.scope_node->formatASTForErrorMessage());
auto expression_types = DataTypes{result_left_table_expression->getResultType(), result_right_table_expression->getResultType()};
DataTypePtr common_type = tryGetLeastSupertype(expression_types);
@ -6164,8 +6258,7 @@ public:
bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node)
{
auto * child_function_node = child_node->as<FunctionNode>();
if (child_function_node)
if (auto * child_function_node = child_node->as<FunctionNode>())
{
if (child_function_node->isAggregateFunction())
return false;
@ -6499,6 +6592,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
if (query_node_typed.isGroupByAll())
expandGroupByAll(query_node_typed);
if (query_node_typed.hasPrewhere())
assertNoFunction(query_node_typed.getPrewhere(),
"arrayJoin",
ErrorCodes::ILLEGAL_PREWHERE,
"ARRAY JOIN",
"in PREWHERE");
/** Validate aggregates
*
* 1. Check that there are no aggregate functions and GROUPING function in JOIN TREE, WHERE, PREWHERE, in another aggregate functions.

View File

@ -10,10 +10,12 @@
#include <Functions/FunctionHelpers.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/IdentifierNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/JoinNode.h>
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/TableFunctionNode.h>
#include <Analyzer/QueryNode.h>
@ -374,4 +376,56 @@ bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, Identifier
return true;
}
namespace
{
class ValidateFunctionNodesVisitor : public ConstInDepthQueryTreeVisitor<ValidateFunctionNodesVisitor>
{
public:
explicit ValidateFunctionNodesVisitor(std::string_view function_name_,
int exception_code_,
std::string_view exception_function_name_,
std::string_view exception_place_message_)
: function_name(function_name_)
, exception_code(exception_code_)
, exception_function_name(exception_function_name_)
, exception_place_message(exception_place_message_)
{}
void visitImpl(const QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (function_node && function_node->getFunctionName() == function_name)
throw Exception(exception_code,
"{} function {} is found {} in query",
exception_function_name,
function_node->formatASTForErrorMessage(),
exception_place_message);
}
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node)
{
auto child_node_type = child_node->getNodeType();
return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION);
}
private:
std::string_view function_name;
int exception_code = 0;
std::string_view exception_function_name;
std::string_view exception_place_message;
};
}
void assertNoFunction(const QueryTreeNodePtr & node,
std::string_view function_name,
int exception_code,
std::string_view exception_function_name,
std::string_view exception_place_message)
{
ValidateFunctionNodesVisitor visitor(function_name, exception_code, exception_function_name, exception_place_message);
visitor.visit(node);
}
}

View File

@ -42,4 +42,13 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod
*/
bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, IdentifierView nested_identifier);
/** Assert that there are no function with specified function name in node children.
* Do not visit subqueries.
*/
void assertNoFunction(const QueryTreeNodePtr & node,
std::string_view function_name,
int exception_code,
std::string_view exception_function_name,
std::string_view exception_place_message);
}

171
src/Functions/nested.cpp Normal file
View File

@ -0,0 +1,171 @@
#include <memory>
#include <Common/assert_cast.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
}
namespace
{
class FunctionNested : public IFunction
{
public:
static constexpr auto name = "nested";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionNested>();
}
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override
{
return 0;
}
bool useDefaultImplementationForConstants() const override
{
return true;
}
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override
{
return {0};
}
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
size_t arguments_size = arguments.size();
if (arguments_size < 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be at least 2",
getName(),
arguments_size);
Names nested_names = extractNestedNames(arguments[0].column);
if (nested_names.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"First argument for function {} must be constant column with array of strings",
getName());
if (nested_names.size() != arguments_size - 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Size of nested names array for function {} does not match arrays arguments size. Actual {}. Expected {}",
getName(),
nested_names.size(),
arguments_size - 1);
DataTypes nested_types;
nested_types.reserve(arguments_size);
for (size_t i = 1; i < arguments_size; ++i)
{
const auto & argument = arguments[i];
const auto * argument_type_array = typeid_cast<const DataTypeArray *>(argument.type.get());
if (!argument_type_array)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument {} for function {} must be array. Actual {}",
i + 1,
getName(),
argument.type->getName());
nested_types.push_back(argument_type_array->getNestedType());
}
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(nested_types, nested_names));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
size_t arguments_size = arguments.size();
const auto * lhs_array = assert_cast<const ColumnArray *>(arguments.at(1).column.get());
Columns data_columns;
data_columns.reserve(arguments_size);
data_columns.push_back(lhs_array->getDataPtr());
for (size_t i = 2; i < arguments_size; ++i)
{
const auto * rhs_array = assert_cast<const ColumnArray *>(arguments[i].column.get());
if (!lhs_array->hasEqualOffsets(*rhs_array))
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH,
"The argument 1 and argument {} of function {} have different array offsets",
i + 1,
getName());
data_columns.push_back(rhs_array->getDataPtr());
}
auto tuple_column = ColumnTuple::create(std::move(data_columns));
auto array_column = ColumnArray::create(std::move(tuple_column), lhs_array->getOffsetsPtr());
return array_column;
}
private:
static Names extractNestedNames(const ColumnPtr & column)
{
const auto * const_column = typeid_cast<const ColumnConst *>(column.get());
if (!const_column)
return {};
Field nested_names_field;
const_column->get(0, nested_names_field);
if (nested_names_field.getType() != Field::Types::Array)
return {};
const auto & nested_names_array = nested_names_field.get<const Array &>();
Names nested_names;
nested_names.reserve(nested_names_array.size());
for (const auto & nested_name_field : nested_names_array)
{
if (nested_name_field.getType() != Field::Types::String)
return {};
nested_names.push_back(nested_name_field.get<const String &>());
}
return nested_names;
}
};
}
REGISTER_FUNCTION(Nsted)
{
factory.registerFunction<FunctionNested>();
}
}

View File

@ -11,7 +11,7 @@ namespace DB
namespace
{
class CollectTopLevelColumnIdentifiersVisitor : public InDepthQueryTreeVisitor<CollectTopLevelColumnIdentifiersVisitor, true>
class CollectTopLevelColumnIdentifiersVisitor : public ConstInDepthQueryTreeVisitor<CollectTopLevelColumnIdentifiersVisitor>
{
public:
@ -20,7 +20,7 @@ public:
, planner_context(planner_context_)
{}
static bool needChildVisit(VisitQueryTreeNodeType &, VisitQueryTreeNodeType & child)
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child)
{
const auto & node_type = child->getNodeType();
return node_type != QueryTreeNodeType::TABLE

View File

@ -1132,6 +1132,13 @@ void Planner::buildPlanForQueryNode()
query_node.getPrewhere() = {};
}
if (query_node.hasWhere())
{
auto condition_constant = tryExtractConstantFromConditionNode(query_node.getWhere());
if (condition_constant.has_value() && *condition_constant)
query_node.getWhere() = {};
}
SelectQueryInfo select_query_info;
select_query_info.original_query = queryNodeToSelectQuery(query_tree);
select_query_info.query = select_query_info.original_query;

View File

@ -778,8 +778,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
}
auto drop_unused_columns_after_join_actions_dag = std::make_shared<ActionsDAG>(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs updated_outputs;
std::unordered_set<std::string_view> updated_outputs_names;
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
std::optional<size_t> first_skipped_column_node_index;
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag->getOutputs();
@ -790,7 +790,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i];
const auto & global_planner_context = planner_context->getGlobalPlannerContext();
if (updated_outputs_names.contains(output->result_name)
if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name)
|| !global_planner_context->hasColumnIdentifier(output->result_name))
continue;
@ -801,18 +801,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
continue;
}
updated_outputs.push_back(output);
updated_outputs_names.insert(output->result_name);
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output);
drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name);
}
/** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope.
*
* Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2;
*/
if (updated_outputs.empty() && first_skipped_column_node_index)
updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index)
drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]);
drop_unused_columns_after_join_actions_dag_outputs = std::move(updated_outputs);
drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs);
auto drop_unused_columns_after_join_transform_step = std::make_unique<ExpressionStep>(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join_actions_dag));
drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN");
@ -823,6 +823,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_join_table_expression,
JoinTreeQueryPlan join_tree_query_plan,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context)
{
auto & array_join_node = array_join_table_expression->as<ArrayJoinNode &>();
@ -837,6 +838,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
ActionsDAGPtr array_join_action_dag = std::make_shared<ActionsDAG>(plan_output_columns);
PlannerActionsVisitor actions_visitor(planner_context);
std::unordered_set<std::string> array_join_expressions_output_nodes;
NameSet array_join_column_names;
for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes())
@ -846,10 +848,12 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
auto & array_join_expression_column = array_join_expression->as<ColumnNode &>();
auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
for (auto & expression_dag_index_node : expression_dag_index_nodes)
{
const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier);
array_join_action_dag->getOutputs().push_back(array_join_column_node);
array_join_expressions_output_nodes.insert(array_join_column_node->result_name);
}
}
@ -858,6 +862,35 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
array_join_actions->setStepDescription("ARRAY JOIN actions");
plan.addStep(std::move(array_join_actions));
auto drop_unused_columns_before_array_join_actions_dag = std::make_shared<ActionsDAG>(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_before_array_join_actions_dag_updated_outputs_names;
auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag->getOutputs();
size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i)
{
const auto & output = drop_unused_columns_before_array_join_actions_dag_outputs[i];
if (drop_unused_columns_before_array_join_actions_dag_updated_outputs_names.contains(output->result_name))
continue;
if (!array_join_expressions_output_nodes.contains(output->result_name) &&
!outer_scope_columns.contains(output->result_name))
continue;
drop_unused_columns_before_array_join_actions_dag_updated_outputs.push_back(output);
drop_unused_columns_before_array_join_actions_dag_updated_outputs_names.insert(output->result_name);
}
drop_unused_columns_before_array_join_actions_dag_outputs = std::move(drop_unused_columns_before_array_join_actions_dag_updated_outputs);
auto drop_unused_columns_before_array_join_transform_step = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(),
std::move(drop_unused_columns_before_array_join_actions_dag));
drop_unused_columns_before_array_join_transform_step->setStepDescription("DROP unused columns before ARRAY JOIN");
plan.addStep(std::move(drop_unused_columns_before_array_join_transform_step));
auto array_join_action = std::make_shared<ArrayJoinAction>(array_join_column_names, array_join_node.isLeft(), planner_context->getQueryContext());
auto array_join_step = std::make_unique<ArrayJoinStep>(plan.getCurrentDataStream(), std::move(array_join_action));
array_join_step->setStepDescription("ARRAY JOIN");
@ -885,9 +918,13 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
for (Int64 i = static_cast<Int64>(table_expressions_stack_size) - 1; i >= 0; --i)
{
table_expressions_outer_scope_columns[i] = current_outer_scope_columns;
auto & table_expression = table_expressions_stack[i];
auto table_expression_type = table_expression->getNodeType();
if (table_expressions_stack[i]->getNodeType() == QueryTreeNodeType::JOIN)
collectTopLevelColumnIdentifiers(table_expressions_stack[i], planner_context, current_outer_scope_columns);
if (table_expression_type == QueryTreeNodeType::JOIN)
collectTopLevelColumnIdentifiers(table_expression, planner_context, current_outer_scope_columns);
else if (table_expression_type == QueryTreeNodeType::ARRAY_JOIN)
collectTopLevelColumnIdentifiers(table_expression, planner_context, current_outer_scope_columns);
}
std::vector<JoinTreeQueryPlan> query_plans_stack;
@ -906,6 +943,7 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
auto query_plan = std::move(query_plans_stack.back());
query_plans_stack.back() = buildQueryPlanForArrayJoinNode(table_expression,
std::move(query_plan),
table_expressions_outer_scope_columns[i],
planner_context);
}
else if (auto * join_node = table_expression->as<JoinNode>())

View File

@ -37,6 +37,7 @@
#include <Planner/PlannerActionsVisitor.h>
#include <Planner/PlannerContext.h>
#include <Planner/Utils.h>
namespace DB
{
@ -513,23 +514,7 @@ std::optional<bool> tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join
if (!join_node_typed.getJoinExpression())
return {};
const auto * constant_node = join_node_typed.getJoinExpression()->as<ConstantNode>();
if (!constant_node)
return {};
const auto & value = constant_node->getValue();
auto constant_type = constant_node->getResultType();
constant_type = removeNullable(removeLowCardinality(constant_type));
auto which_constant_type = WhichDataType(constant_type);
if (!which_constant_type.isUInt8() && !which_constant_type.isNothing())
return {};
if (value.isNull())
return false;
UInt8 predicate_value = value.safeGet<UInt8>();
return predicate_value > 0;
return tryExtractConstantFromConditionNode(join_node_typed.getJoinExpression());
}
namespace

View File

@ -4,6 +4,9 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/getLeastSuperColumn.h>
#include <IO/WriteBufferFromString.h>
@ -333,4 +336,25 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
return function_node;
}
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node)
{
const auto * constant_node = condition_node->as<ConstantNode>();
if (!constant_node)
return {};
const auto & value = constant_node->getValue();
auto constant_type = constant_node->getResultType();
constant_type = removeNullable(removeLowCardinality(constant_type));
auto which_constant_type = WhichDataType(constant_type);
if (!which_constant_type.isUInt8() && !which_constant_type.isNothing())
return {};
if (value.isNull())
return false;
UInt8 predicate_value = value.safeGet<UInt8>();
return predicate_value > 0;
}
}

View File

@ -63,4 +63,7 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no
/// Returns `and` function node that has condition nodes as its arguments
QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context);
/// Try extract boolean constant from condition node
std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node);
}

View File

@ -6,11 +6,15 @@
6 0
7 1 0
8 1 0
101 0
102 0
-
1 0
2 2 a2
1 0
2 2 a2
1
2 a2
0 0
0 0
0 0

View File

@ -1,4 +1,5 @@
SET joined_subquery_requires_alias = 0;
SET allow_experimental_analyzer = 1;
SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c);
SELECT ax, c FROM (SELECT [3,4] ax, 0 c) JOIN (SELECT 0 c) USING (c) ARRAY JOIN ax;
@ -9,7 +10,7 @@ SELECT ax, c, d FROM (SELECT [7,8] ax, 1 c, 0 d) s1 JOIN system.one s2 ON s1.c =
SELECT ax, c FROM (SELECT [101,102] ax, 0 c) s1
JOIN system.one s2 ON s1.c = s2.dummy
JOIN system.one s3 ON s1.c = s3.dummy
ARRAY JOIN ax; -- { serverError 48 }
ARRAY JOIN ax;
SELECT '-';
@ -28,8 +29,7 @@ INSERT INTO d VALUES (2, 'a2'), (3, 'a3');
SELECT d_ids, id, name FROM f LEFT ARRAY JOIN d_ids LEFT JOIN d ON d.id = d_ids ORDER BY id;
SELECT did, id, name FROM f LEFT ARRAY JOIN d_ids as did LEFT JOIN d ON d.id = did ORDER BY id;
-- name clash, doesn't work yet
SELECT id, name FROM f LEFT ARRAY JOIN d_ids as id LEFT JOIN d ON d.id = id ORDER BY id; -- { serverError 403 }
SELECT id, name FROM f LEFT ARRAY JOIN d_ids as id LEFT JOIN d ON d.id = id ORDER BY id;
SELECT * FROM ( SELECT [dummy, dummy] AS dummy FROM system.one ) AS x ARRAY JOIN dummy
JOIN system.one AS y ON x.dummy == y.dummy;

View File

@ -50,7 +50,7 @@ SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT
SELECT '--';
--
SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1,2,3] AS id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id);
[5]
1
SELECT '--';
--
SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table USING (id);
@ -61,68 +61,68 @@ SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table
SELECT '--';
--
SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3]
0 Value_0 [1,2,3]
SELECT '--';
--
SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3] 0
0 Value_0 [1,2,3] 0
SELECT '--';
--
SELECT *, id FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT * FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3]
0 [0] Value_0 [1,2,3]
SELECT '--';
--
SELECT * FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT *, id FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3] 0
0 [0] Value_0 [1,2,3] 0
SELECT '--';
--
SELECT *, id FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3]
0 Value_0 [1,2,3]
SELECT '--';
--
SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
[0] Value_0 [1,2,3] 0
0 Value_0 [1,2,3] 0
SELECT '--';
--
SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
[5] Value_0 [1,2,3]
0 Value_0 [1,2,3]
SELECT '--';
--
SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
[5] Value_0 [1,2,3] 0
0 Value_0 [1,2,3] 0
SELECT '--';
--
SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
SELECT '--';
--
SELECT * FROM (SELECT [5] AS id_array) AS subquery ARRAY JOIN id_array, [0] AS id INNER JOIN test_table USING (id);
5 Value_0 [1,2,3]
0 5 Value_0 [1,2,3]
SELECT '--';
--
SELECT * FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id);
[[0]] Value_0 [1,2,3]
0 Value_0 [1,2,3]
SELECT '--';
--
SELECT *, id FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id);
[[0]] Value_0 [1,2,3] 0
0 Value_0 [1,2,3] 0