Analyzer improve ARRAY JOIN with JOIN

This commit is contained in:
Maksim Kita 2022-11-08 14:25:23 +01:00
parent 2c568df714
commit 3b3e27f22a
10 changed files with 262 additions and 91 deletions

View File

@ -2330,30 +2330,43 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi
const auto & from_array_join_node = table_expression_node->as<const ArrayJoinNode &>(); const auto & from_array_join_node = table_expression_node->as<const ArrayJoinNode &>();
auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope);
/** Special case when qualified or unqualified identifier point to array join expression without alias. if (scope.table_expressions_in_resolve_process.contains(table_expression_node.get()))
* return resolved_identifier;
* 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; for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes())
*
* value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression.
*/
if (!scope.table_expressions_in_resolve_process.contains(table_expression_node.get()) && resolved_identifier)
{ {
for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) /** 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
*/
if (identifier_lookup.identifier.isShort() &&
array_join_expression->getAlias() == identifier_lookup.identifier.getFullName())
{ {
auto & array_join_column_expression = array_join_expression->as<ColumnNode &>(); resolved_identifier = array_join_expression;
if (array_join_column_expression.hasAlias()) break;
continue; }
auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow(); if (!resolved_identifier)
if (array_join_column_inner_expression.get() == resolved_identifier.get() || continue;
array_join_column_inner_expression->isEqual(*resolved_identifier))
{
auto array_join_column = array_join_column_expression.getColumn();
auto result = std::make_shared<ColumnNode>(array_join_column, table_expression_node);
return result; /** 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.
*/
auto & array_join_column_expression = array_join_expression->as<ColumnNode &>();
if (array_join_column_expression.hasAlias())
continue;
auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow();
if (array_join_column_inner_expression.get() == resolved_identifier.get() ||
array_join_column_inner_expression->isEqual(*resolved_identifier))
{
resolved_identifier = array_join_expression;
break;
} }
} }
@ -2868,6 +2881,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree()); auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree());
std::vector<QueryTreeNodesWithNames> table_expressions_column_nodes_with_names_stack; 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;
for (auto & table_expression : table_expressions_stack) for (auto & table_expression : table_expressions_stack)
{ {
QueryTreeNodesWithNames matched_expression_nodes_with_column_names; QueryTreeNodesWithNames matched_expression_nodes_with_column_names;
@ -2893,8 +2909,14 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
auto left_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back()); auto left_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back());
table_expressions_column_nodes_with_names_stack.pop_back(); table_expressions_column_nodes_with_names_stack.pop_back();
std::unordered_set<std::string> column_names_to_skip; left_table_expression_column_names_to_skip.clear();
right_table_expression_column_names_to_skip.clear();
/** If there is JOIN with USING we need to match only single USING column and do not use left table expression
* and right table expression column with same name.
*
* Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id);
*/
if (!table_expression_in_resolve_process && join_node->isUsingJoinExpression()) if (!table_expression_in_resolve_process && join_node->isUsingJoinExpression())
{ {
auto & join_using_list = join_node->getJoinExpression()->as<ListNode &>(); auto & join_using_list = join_node->getJoinExpression()->as<ListNode &>();
@ -2902,22 +2924,64 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
for (auto & join_using_node : join_using_list.getNodes()) for (auto & join_using_node : join_using_list.getNodes())
{ {
auto & column_node = join_using_node->as<ColumnNode &>(); auto & column_node = join_using_node->as<ColumnNode &>();
const auto & column_name = column_node.getColumnName(); const auto & using_column_name = column_node.getColumnName();
if (!matcher_node_typed.isMatchingColumn(column_name)) if (!matcher_node_typed.isMatchingColumn(using_column_name))
continue; continue;
column_names_to_skip.insert(column_name); const auto & join_using_column_nodes_list = column_node.getExpressionOrThrow()->as<ListNode &>();
const auto & join_using_column_nodes = join_using_column_nodes_list.getNodes();
QueryTreeNodePtr column_source = getColumnSourceForJoinNodeWithUsing(table_expression); QueryTreeNodePtr matched_column_node;
auto matched_column_node = std::make_shared<ColumnNode>(column_node.getColumn(), column_source);
matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), column_name); if (isRight(join_node->getKind()))
matched_column_node = join_using_column_nodes.at(1);
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 ARRAY JOIN, 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_node_typed = matched_column_node->as<ColumnNode &>();
auto matched_column_source = matched_column_node->as<ColumnNode &>().getColumnSource();
if (matched_column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN)
{
auto * array_join_inner_expression_column = matched_column_node_typed.getExpressionOrThrow()->as<ColumnNode>();
if (array_join_inner_expression_column && array_join_inner_expression_column->getColumnName() == using_column_name)
{
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);
}
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);
}
} }
} }
for (auto && left_table_column : left_table_expression_columns) for (auto && left_table_column : left_table_expression_columns)
{ {
if (column_names_to_skip.contains(left_table_column.second)) if (left_table_expression_column_names_to_skip.contains(left_table_column.second))
continue; continue;
matched_expression_nodes_with_column_names.push_back(std::move(left_table_column)); matched_expression_nodes_with_column_names.push_back(std::move(left_table_column));
@ -2925,7 +2989,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher(
for (auto && right_table_column : right_table_expression_columns) for (auto && right_table_column : right_table_expression_columns)
{ {
if (column_names_to_skip.contains(right_table_column.second)) if (right_table_expression_column_names_to_skip.contains(right_table_column.second))
continue; continue;
matched_expression_nodes_with_column_names.push_back(std::move(right_table_column)); matched_expression_nodes_with_column_names.push_back(std::move(right_table_column));
@ -5056,8 +5120,10 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node,
if (!common_type) if (!common_type)
throw Exception(ErrorCodes::NO_COMMON_TYPE, throw Exception(ErrorCodes::NO_COMMON_TYPE,
"JOIN {} cannot infer common type in USING for identifier '{}'. In scope {}", "JOIN {} cannot infer common type for {} and {} in USING for identifier '{}'. In scope {}",
join_node.formatASTForErrorMessage(), join_node.formatASTForErrorMessage(),
result_left_table_expression->getResultType()->getName(),
result_right_table_expression->getResultType()->getName(),
identifier_full_name, identifier_full_name,
scope.scope_node->formatASTForErrorMessage()); scope.scope_node->formatASTForErrorMessage());

View File

@ -289,41 +289,4 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod
return result; return result;
} }
QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node)
{
QueryTreeNodePtr column_source_node = join_node;
while (true)
{
auto column_source_node_type = column_source_node->getNodeType();
if (column_source_node_type == QueryTreeNodeType::TABLE ||
column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION ||
column_source_node_type == QueryTreeNodeType::QUERY ||
column_source_node_type == QueryTreeNodeType::UNION)
{
break;
}
else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN)
{
auto & array_join_node = column_source_node->as<ArrayJoinNode &>();
column_source_node = array_join_node.getTableExpression();
continue;
}
else if (column_source_node_type == QueryTreeNodeType::JOIN)
{
auto & join_node_typed = column_source_node->as<JoinNode &>();
column_source_node = isRight(join_node_typed.getKind()) ? join_node_typed.getRightTableExpression() : join_node_typed.getLeftTableExpression();
continue;
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}",
column_source_node->getNodeTypeName());
}
}
return column_source_node;
}
} }

View File

@ -31,9 +31,4 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node);
*/ */
QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node);
/** Get column source for JOIN node with USING.
* Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id);
*/
QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node);
} }

View File

@ -39,17 +39,16 @@ public:
auto column_source_node = column_node->getColumnSource(); auto column_source_node = column_node->getColumnSource();
auto column_source_node_type = column_source_node->getNodeType(); auto column_source_node_type = column_source_node->getNodeType();
if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || if (column_source_node_type == QueryTreeNodeType::LAMBDA)
column_source_node_type == QueryTreeNodeType::LAMBDA)
return; return;
/// JOIN using expression /// JOIN using expression
if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN)
return; return;
auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node); auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node);
if (column_node->hasExpression()) if (column_node->hasExpression() && column_source_node_type != QueryTreeNodeType::ARRAY_JOIN)
{ {
/// Replace ALIAS column with expression /// Replace ALIAS column with expression
table_expression_data.addAliasColumnName(column_node->getColumnName()); table_expression_data.addAliasColumnName(column_node->getColumnName());
@ -61,9 +60,10 @@ public:
if (column_source_node_type != QueryTreeNodeType::TABLE && if (column_source_node_type != QueryTreeNodeType::TABLE &&
column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION &&
column_source_node_type != QueryTreeNodeType::QUERY && column_source_node_type != QueryTreeNodeType::QUERY &&
column_source_node_type != QueryTreeNodeType::UNION) column_source_node_type != QueryTreeNodeType::UNION &&
column_source_node_type != QueryTreeNodeType::ARRAY_JOIN)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected table, table function, query or union column source. Actual {}", "Expected table, table function, array join, query or union column source. Actual {}",
column_source_node->formatASTForErrorMessage()); column_source_node->formatASTForErrorMessage());
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName()); bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());

View File

@ -185,18 +185,19 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
} }
auto rename_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); auto rename_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs;
for (auto & output_node : rename_actions_dag->getOutputs()) for (auto & output_node : rename_actions_dag->getOutputs())
{ {
const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name);
if (!column_identifier) if (!column_identifier)
continue; continue;
const auto * node_to_rename = output_node; updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier));
output_node = &rename_actions_dag->addAlias(*node_to_rename, *column_identifier);
} }
rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs);
auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), rename_actions_dag); auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), rename_actions_dag);
rename_step->setStepDescription("Change column names to column identifiers"); rename_step->setStepDescription("Change column names to column identifiers");
query_plan.addStep(std::move(rename_step)); query_plan.addStep(std::move(rename_step));
@ -639,17 +640,17 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
ActionsDAGPtr array_join_action_dag = std::make_shared<ActionsDAG>(plan_output_columns); ActionsDAGPtr array_join_action_dag = std::make_shared<ActionsDAG>(plan_output_columns);
PlannerActionsVisitor actions_visitor(planner_context); PlannerActionsVisitor actions_visitor(planner_context);
NameSet array_join_columns; NameSet array_join_column_names;
for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes())
{ {
auto & array_join_expression_column = array_join_expression->as<ColumnNode &>(); const auto & array_join_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(array_join_expression);
const auto & array_join_column_name = array_join_expression_column.getColumnName(); array_join_column_names.insert(array_join_column_identifier);
array_join_columns.insert(array_join_column_name);
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()); 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) 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_name); 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_action_dag->getOutputs().push_back(array_join_column_node);
} }
} }
@ -659,7 +660,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
array_join_actions->setStepDescription("ARRAY JOIN actions"); array_join_actions->setStepDescription("ARRAY JOIN actions");
plan.addStep(std::move(array_join_actions)); plan.addStep(std::move(array_join_actions));
auto array_join_action = std::make_shared<ArrayJoinAction>(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); 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)); auto array_join_step = std::make_unique<ArrayJoinStep>(plan.getCurrentDataStream(), std::move(array_join_action));
array_join_step->setStepDescription("ARRAY JOIN"); array_join_step->setStepDescription("ARRAY JOIN");
plan.addStep(std::move(array_join_step)); plan.addStep(std::move(array_join_step));

View File

@ -16,6 +16,25 @@ using ColumnIdentifier = std::string;
/** Table expression data is created for each table expression that take part in query. /** Table expression data is created for each table expression that take part in query.
* Table expression data has information about columns that participate in query, their name to identifier mapping, * Table expression data has information about columns that participate in query, their name to identifier mapping,
* and additional table expression properties. * and additional table expression properties.
*
* Table expression can be table, table function, query, union, array join node.
*
* Examples:
* SELECT * FROM (SELECT 1);
* (SELECT 1) - table expression.
*
* SELECT * FROM test_table;
* test_table - table expression.
*
* SELECT * FROM view(SELECT 1);
* view(SELECT 1) - table expression.
*
* SELECT * FROM (SELECT 1) JOIN (SELECT 2);
* (SELECT 1) - table expression.
* (SELECT 2) - table expression.
*
* SELECT array, a FROM (SELECT [1] AS array) ARRAY JOIN array AS a;
* ARRAY JOIN array AS a - table expression.
*/ */
class TableExpressionData class TableExpressionData
{ {
@ -164,19 +183,19 @@ public:
} }
private: private:
/// Valid for table, table function, query, union table expression nodes /// Valid for table, table function, query, union, array join table expression nodes
NamesAndTypesList columns; NamesAndTypesList columns;
/// Valid for table, table function, query, union table expression nodes /// Valid for table, table function, query, union, array join table expression nodes
NameSet columns_names; NameSet columns_names;
/// Valid only for table table expression node /// Valid only for table table expression node
NameSet alias_columns_names; NameSet alias_columns_names;
/// Valid for table, table function, query, union table expression nodes /// Valid for table, table function, query, union table, array join expression nodes
ColumnNameToColumnIdentifier column_name_to_column_identifier; ColumnNameToColumnIdentifier column_name_to_column_identifier;
/// Valid for table, table function, query, union table expression nodes /// Valid for table, table function, query, union table, array join expression nodes
ColumnIdentifierToColumnName column_identifier_to_column_name; ColumnIdentifierToColumnName column_identifier_to_column_name;
/// Is storage remote /// Is storage remote

View File

@ -0,0 +1,28 @@
0
0
--
0
--
--
--
1
--
0 Value_0
--
1 Value_1
--
0 Value_0
--
1 Value_1
--
[0] Value_0
--
[1] Value_1
--
[0] Value_0
--
[0] Value_1
--
[5] Value_0
--
[5] Value_1

View File

@ -0,0 +1,71 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
INSERT INTO test_table VALUES (0, 'Value_0');
INSERT INTO test_table VALUES (1, 'Value_1');
SELECT * FROM (SELECT [dummy, dummy] AS dummy FROM system.one) AS subquery ARRAY JOIN dummy INNER JOIN system.one USING (dummy);
SELECT '--';
SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy);
SELECT '--';
SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy);
SELECT '--';
SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy);
SELECT '--';
SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy);
SELECT '--';
SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [0] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [1] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id);
SELECT '--';
SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id);
DROP TABLE test_table;

View File

@ -0,0 +1,9 @@
1 3
--
2 4
--
1 3
--
2 4
--
1 2 3 4

View File

@ -0,0 +1,19 @@
SET allow_experimental_analyzer = 1;
SELECT subquery_1.id, subquery_2.id FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2;
SELECT '--';
SELECT subquery_1.value, subquery_2.value FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2;
SELECT '--';
SELECT COLUMNS('id') FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2;
SELECT '--';
SELECT COLUMNS('value') FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2;
SELECT '--';
SELECT * FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2;