mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Analyzer support ARRAY JOIN with Nested
This commit is contained in:
parent
077555bd5d
commit
af38660cf5
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
171
src/Functions/nested.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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>())
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user