mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Analyzer improve ARRAY JOIN with JOIN
This commit is contained in:
parent
2c568df714
commit
3b3e27f22a
@ -2330,6 +2330,26 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi
|
||||
const auto & from_array_join_node = table_expression_node->as<const ArrayJoinNode &>();
|
||||
auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope);
|
||||
|
||||
if (scope.table_expressions_in_resolve_process.contains(table_expression_node.get()))
|
||||
return 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())
|
||||
{
|
||||
resolved_identifier = array_join_expression;
|
||||
break;
|
||||
}
|
||||
|
||||
if (!resolved_identifier)
|
||||
continue;
|
||||
|
||||
/** 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;
|
||||
@ -2337,10 +2357,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi
|
||||
*
|
||||
* 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())
|
||||
{
|
||||
auto & array_join_column_expression = array_join_expression->as<ColumnNode &>();
|
||||
if (array_join_column_expression.hasAlias())
|
||||
continue;
|
||||
@ -2349,11 +2365,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi
|
||||
if (array_join_column_inner_expression.get() == resolved_identifier.get() ||
|
||||
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;
|
||||
}
|
||||
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());
|
||||
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)
|
||||
{
|
||||
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());
|
||||
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())
|
||||
{
|
||||
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())
|
||||
{
|
||||
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;
|
||||
|
||||
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);
|
||||
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);
|
||||
QueryTreeNodePtr matched_column_node;
|
||||
|
||||
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)
|
||||
{
|
||||
if (column_names_to_skip.contains(left_table_column.second))
|
||||
if (left_table_expression_column_names_to_skip.contains(left_table_column.second))
|
||||
continue;
|
||||
|
||||
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)
|
||||
{
|
||||
if (column_names_to_skip.contains(right_table_column.second))
|
||||
if (right_table_expression_column_names_to_skip.contains(right_table_column.second))
|
||||
continue;
|
||||
|
||||
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)
|
||||
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(),
|
||||
result_left_table_expression->getResultType()->getName(),
|
||||
result_right_table_expression->getResultType()->getName(),
|
||||
identifier_full_name,
|
||||
scope.scope_node->formatASTForErrorMessage());
|
||||
|
||||
|
@ -289,41 +289,4 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -31,9 +31,4 @@ QueryTreeNodes extractTableExpressions(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);
|
||||
|
||||
}
|
||||
|
@ -39,17 +39,16 @@ public:
|
||||
auto column_source_node = column_node->getColumnSource();
|
||||
auto column_source_node_type = column_source_node->getNodeType();
|
||||
|
||||
if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN ||
|
||||
column_source_node_type == QueryTreeNodeType::LAMBDA)
|
||||
if (column_source_node_type == QueryTreeNodeType::LAMBDA)
|
||||
return;
|
||||
|
||||
/// JOIN using expression
|
||||
if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN)
|
||||
if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN)
|
||||
return;
|
||||
|
||||
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
|
||||
table_expression_data.addAliasColumnName(column_node->getColumnName());
|
||||
@ -61,9 +60,10 @@ public:
|
||||
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)
|
||||
column_source_node_type != QueryTreeNodeType::UNION &&
|
||||
column_source_node_type != QueryTreeNodeType::ARRAY_JOIN)
|
||||
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());
|
||||
|
||||
bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName());
|
||||
|
@ -185,18 +185,19 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
|
||||
}
|
||||
|
||||
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())
|
||||
{
|
||||
const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name);
|
||||
|
||||
if (!column_identifier)
|
||||
continue;
|
||||
|
||||
const auto * node_to_rename = output_node;
|
||||
output_node = &rename_actions_dag->addAlias(*node_to_rename, *column_identifier);
|
||||
updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *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);
|
||||
rename_step->setStepDescription("Change column names to column identifiers");
|
||||
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);
|
||||
PlannerActionsVisitor actions_visitor(planner_context);
|
||||
|
||||
NameSet array_join_columns;
|
||||
NameSet array_join_column_names;
|
||||
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_name = array_join_expression_column.getColumnName();
|
||||
array_join_columns.insert(array_join_column_name);
|
||||
const auto & array_join_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(array_join_expression);
|
||||
array_join_column_names.insert(array_join_column_identifier);
|
||||
|
||||
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_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);
|
||||
}
|
||||
}
|
||||
@ -659,7 +660,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression,
|
||||
array_join_actions->setStepDescription("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));
|
||||
array_join_step->setStepDescription("ARRAY JOIN");
|
||||
plan.addStep(std::move(array_join_step));
|
||||
|
@ -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 has information about columns that participate in query, their name to identifier mapping,
|
||||
* 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
|
||||
{
|
||||
@ -164,19 +183,19 @@ public:
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
/// 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;
|
||||
|
||||
/// Valid only for table table expression node
|
||||
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;
|
||||
|
||||
/// 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;
|
||||
|
||||
/// Is storage remote
|
||||
|
@ -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
|
@ -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;
|
@ -0,0 +1,9 @@
|
||||
1 3
|
||||
--
|
||||
2 4
|
||||
--
|
||||
1 3
|
||||
--
|
||||
2 4
|
||||
--
|
||||
1 2 3 4
|
@ -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;
|
Loading…
Reference in New Issue
Block a user