mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-14 10:22:10 +00:00
review fix
This commit is contained in:
parent
a80e11f21b
commit
08ec69736f
@ -173,18 +173,7 @@ Result:
|
||||
└───┴────┴─────┘
|
||||
```
|
||||
|
||||
## [experimental] Join with inequality conditions for columns from different tables
|
||||
|
||||
:::note
|
||||
This feature is experimental. To use it, set `allow_experimental_join_condition` to 1 in your configuration files or by using the `SET` command:
|
||||
|
||||
```sql
|
||||
SET allow_experimental_join_condition=1
|
||||
```
|
||||
|
||||
Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`.
|
||||
|
||||
:::
|
||||
## Join with inequality conditions for columns from different tables
|
||||
|
||||
Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`.
|
||||
|
||||
|
@ -600,15 +600,15 @@ void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, Que
|
||||
|
||||
}
|
||||
|
||||
QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query)
|
||||
QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query)
|
||||
{
|
||||
QueryTreeNodes result;
|
||||
|
||||
QueryTreeNodes nearest_query_tmp;
|
||||
if (out_nearest_query == nullptr)
|
||||
out_nearest_query = &nearest_query_tmp;
|
||||
if (nearest_query == nullptr)
|
||||
nearest_query = &nearest_query_tmp;
|
||||
|
||||
buildTableExpressionsStackImpl(query_node->as<QueryNode>()->getJoinTree(), query_node, result, *out_nearest_query);
|
||||
buildTableExpressionsStackImpl(query_node->as<QueryNode>()->getJoinTree(), query_node, result, *nearest_query);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -67,6 +67,9 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node,
|
||||
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);
|
||||
|
||||
/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree.
|
||||
* Parameters:
|
||||
* query_node - query node that contains join tree.
|
||||
* nearest_query - if not nullptr, then nearest query node will be stored in this variable.
|
||||
*
|
||||
* Example: SELECT * FROM t1 INNER JOIN t2 INNER JOIN t3.
|
||||
* Result table expressions stack:
|
||||
@ -75,8 +78,10 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n
|
||||
* 3. t1 INNER JOIN t2
|
||||
* 4. t2
|
||||
* 5. t1
|
||||
*
|
||||
* Nearest query is the query itself in this case.
|
||||
*/
|
||||
QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query = nullptr);
|
||||
QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query = nullptr);
|
||||
|
||||
|
||||
/** Assert that there are no function nodes with specified function name in node children.
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Core/Joins.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -168,4 +169,18 @@ private:
|
||||
void setRightIndex(size_t right_pos, size_t result_position);
|
||||
};
|
||||
|
||||
/// Call the same func twice - for left arguments and then right arguments
|
||||
template <typename Func, typename... Args>
|
||||
void forJoinSides(Func && func, std::tuple<Args...> && left, std::tuple<Args...> && right) {
|
||||
std::apply([&](auto &&... args)
|
||||
{
|
||||
func(JoinTableSide::Left, std::forward<decltype(args)>(args)...);
|
||||
}, std::forward<std::tuple<Args...>>(left));
|
||||
|
||||
std::apply([&](auto &&... args)
|
||||
{
|
||||
func(JoinTableSide::Right, std::forward<decltype(args)>(args)...);
|
||||
}, std::forward<std::tuple<Args...>>(right));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1291,12 +1291,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
planner_context);
|
||||
|
||||
const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes();
|
||||
if (!left_pre_filters.empty() && left_pre_filters.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one left pre-filter condition node. Actual [{}]", fmt::join(left_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", "));
|
||||
|
||||
const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes();
|
||||
if (!right_pre_filters.empty() && right_pre_filters.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", "));
|
||||
forJoinSides([](JoinTableSide side, const auto & pre_filters)
|
||||
{
|
||||
if (!pre_filters.empty() && pre_filters.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one {} pre-filter condition node. Actual [{}]",
|
||||
side, fmt::join(pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", "));
|
||||
}, std::make_tuple(std::ref(left_pre_filters)), std::make_tuple(std::ref(right_pre_filters)));
|
||||
|
||||
can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1
|
||||
&& join_strictness == JoinStrictness::All
|
||||
@ -1304,29 +1305,21 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
&& (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type))
|
||||
&& (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type));
|
||||
|
||||
join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader());
|
||||
appendSetsFromActionsDAG(join_clauses_and_actions.left_join_expressions_actions, left_join_tree_query_plan.useful_sets);
|
||||
forJoinSides([&](JoinTableSide, ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters)
|
||||
{
|
||||
join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader());
|
||||
appendSetsFromActionsDAG(join_expressions_actions, left_join_tree_query_plan.useful_sets);
|
||||
|
||||
QueryPlanStepPtr left_join_expressions_actions_step;
|
||||
QueryPlanStepPtr join_expressions_actions_step;
|
||||
if (can_move_out_residuals && !left_pre_filters.empty())
|
||||
left_join_expressions_actions_step = std::make_unique<FilterStep>(left_plan.getCurrentHeader(), std::move(join_clauses_and_actions.left_join_expressions_actions), left_pre_filters[0]->result_name, false);
|
||||
join_expressions_actions_step = std::make_unique<FilterStep>(plan.getCurrentHeader(), std::move(join_expressions_actions), pre_filters[0]->result_name, false);
|
||||
else
|
||||
left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentHeader(), std::move(join_clauses_and_actions.left_join_expressions_actions));
|
||||
join_expressions_actions_step = std::make_unique<ExpressionStep>(plan.getCurrentHeader(), std::move(join_expressions_actions));
|
||||
|
||||
left_join_expressions_actions_step->setStepDescription("JOIN actions");
|
||||
left_plan.addStep(std::move(left_join_expressions_actions_step));
|
||||
|
||||
join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentHeader());
|
||||
appendSetsFromActionsDAG(join_clauses_and_actions.right_join_expressions_actions, right_join_tree_query_plan.useful_sets);
|
||||
|
||||
QueryPlanStepPtr right_join_expressions_actions_step;
|
||||
if (can_move_out_residuals && !right_pre_filters.empty())
|
||||
right_join_expressions_actions_step = std::make_unique<FilterStep>(right_plan.getCurrentHeader(), std::move(join_clauses_and_actions.right_join_expressions_actions), right_pre_filters[0]->result_name, false);
|
||||
else
|
||||
right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentHeader(), std::move(join_clauses_and_actions.right_join_expressions_actions));
|
||||
|
||||
right_join_expressions_actions_step->setStepDescription("JOIN actions");
|
||||
right_plan.addStep(std::move(right_join_expressions_actions_step));
|
||||
join_expressions_actions_step->setStepDescription("JOIN actions");
|
||||
plan.addStep(std::move(join_expressions_actions_step));
|
||||
}, std::make_tuple(std::ref(join_clauses_and_actions.left_join_expressions_actions), std::ref(left_plan), std::ref(left_pre_filters)),
|
||||
std::make_tuple(std::ref(join_clauses_and_actions.right_join_expressions_actions), std::ref(right_plan), std::ref(right_pre_filters)));
|
||||
}
|
||||
|
||||
std::unordered_map<ColumnIdentifier, DataTypePtr> left_plan_column_name_to_cast_type;
|
||||
@ -1460,6 +1453,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
if (join_clause_key_nodes_size == 0 && !can_move_out_residuals)
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot determine join keys in {}", join_node.formatASTForErrorMessage());
|
||||
|
||||
/// If there are no keys, but only conditions that cannot be used as keys, then it is a cross join.
|
||||
/// Example: SELECT * FROM t1 JOIN t2 ON t1.x > t2.y
|
||||
/// Same as: SELECT * FROM t1 CROSS JOIN t2 WHERE t1.x > t2.y
|
||||
if (join_clause_key_nodes_size == 0 && can_move_out_residuals)
|
||||
{
|
||||
join_kind = JoinKind::Cross;
|
||||
@ -1531,15 +1527,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
}
|
||||
}
|
||||
|
||||
if (!can_move_out_residuals && join_clauses_and_actions.mixed_join_expressions_actions)
|
||||
if (!can_move_out_residuals && join_clauses_and_actions.residual_join_expressions_actions)
|
||||
{
|
||||
/// Let join algorithm handle residual conditions
|
||||
ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression();
|
||||
mixed_join_expression = std::make_shared<ExpressionActions>(
|
||||
std::move(*join_clauses_and_actions.mixed_join_expressions_actions),
|
||||
std::move(*join_clauses_and_actions.residual_join_expressions_actions),
|
||||
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
|
||||
|
||||
appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets);
|
||||
join_clauses_and_actions.mixed_join_expressions_actions.reset();
|
||||
join_clauses_and_actions.residual_join_expressions_actions.reset();
|
||||
}
|
||||
}
|
||||
else if (join_node.isUsingJoinExpression())
|
||||
@ -1688,26 +1685,28 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
result_plan.unitePlans(std::move(join_step), {std::move(plans)});
|
||||
}
|
||||
|
||||
if (join_clauses_and_actions.mixed_join_expressions_actions)
|
||||
/// If residuals were not moved to JOIN algorithm,
|
||||
/// we need to process add then as WHERE condition after JOIN
|
||||
if (join_clauses_and_actions.residual_join_expressions_actions)
|
||||
{
|
||||
auto outputs = join_clauses_and_actions.mixed_join_expressions_actions->getOutputs();
|
||||
auto outputs = join_clauses_and_actions.residual_join_expressions_actions->getOutputs();
|
||||
if (outputs.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 output column in JOIN actions, got {}",
|
||||
join_clauses_and_actions.mixed_join_expressions_actions->dumpDAG());
|
||||
join_clauses_and_actions.residual_join_expressions_actions->dumpDAG());
|
||||
|
||||
join_clauses_and_actions.mixed_join_expressions_actions->appendInputsForUnusedColumns(result_plan.getCurrentHeader());
|
||||
for (const auto * input_node : join_clauses_and_actions.mixed_join_expressions_actions->getInputs())
|
||||
join_clauses_and_actions.mixed_join_expressions_actions->addOrReplaceInOutputs(*input_node);
|
||||
join_clauses_and_actions.residual_join_expressions_actions->appendInputsForUnusedColumns(result_plan.getCurrentHeader());
|
||||
for (const auto * input_node : join_clauses_and_actions.residual_join_expressions_actions->getInputs())
|
||||
join_clauses_and_actions.residual_join_expressions_actions->addOrReplaceInOutputs(*input_node);
|
||||
|
||||
appendSetsFromActionsDAG(*join_clauses_and_actions.mixed_join_expressions_actions, left_join_tree_query_plan.useful_sets);
|
||||
appendSetsFromActionsDAG(*join_clauses_and_actions.residual_join_expressions_actions, left_join_tree_query_plan.useful_sets);
|
||||
auto filter_step = std::make_unique<FilterStep>(result_plan.getCurrentHeader(),
|
||||
std::move(*join_clauses_and_actions.mixed_join_expressions_actions),
|
||||
std::move(*join_clauses_and_actions.residual_join_expressions_actions),
|
||||
outputs[0]->result_name,
|
||||
/* remove_column = */ false); /// Unused columns will be removed by next step
|
||||
filter_step->setStepDescription("Residual JOIN filter");
|
||||
result_plan.addStep(std::move(filter_step));
|
||||
|
||||
join_clauses_and_actions.mixed_join_expressions_actions.reset();
|
||||
join_clauses_and_actions.residual_join_expressions_actions.reset();
|
||||
}
|
||||
|
||||
ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName());
|
||||
|
@ -378,8 +378,8 @@ void buildJoinClause(
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"OUTER JOIN ON expression {} contains column from left and right table, which is not supported with join_use_nulls",
|
||||
join_expression->formatASTForErrorMessage());
|
||||
"{} JOIN ON expression {} contains column from left and right table, which is not supported with `join_use_nulls`",
|
||||
toString(join_node.getKind()), join_expression->formatASTForErrorMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -474,7 +474,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
join_right_table_expressions,
|
||||
join_node,
|
||||
result.join_clauses.back());
|
||||
has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty();
|
||||
has_residual_filters |= !result.join_clauses.back().getResidualFilterConditionNodes().empty();
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -491,7 +491,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
join_right_table_expressions,
|
||||
join_node,
|
||||
result.join_clauses.back());
|
||||
has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty();
|
||||
has_residual_filters |= !result.join_clauses.back().getResidualFilterConditionNodes().empty();
|
||||
}
|
||||
|
||||
auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext());
|
||||
@ -618,29 +618,29 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
/// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined.
|
||||
if (result.join_clauses.size() > 1)
|
||||
{
|
||||
ActionsDAG mixed_join_expressions_actions(result_relation_columns);
|
||||
ActionsDAG residual_join_expressions_actions(result_relation_columns);
|
||||
PlannerActionsVisitor join_expression_visitor(planner_context);
|
||||
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression);
|
||||
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(residual_join_expressions_actions, join_expression);
|
||||
if (join_expression_dag_node_raw_pointers.size() != 1)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage());
|
||||
|
||||
mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]);
|
||||
residual_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]);
|
||||
Names required_names{join_expression_dag_node_raw_pointers[0]->result_name};
|
||||
mixed_join_expressions_actions.removeUnusedActions(required_names);
|
||||
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
|
||||
residual_join_expressions_actions.removeUnusedActions(required_names);
|
||||
result.residual_join_expressions_actions = std::move(residual_join_expressions_actions);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & join_clause = result.join_clauses.front();
|
||||
const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes();
|
||||
auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true);
|
||||
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
|
||||
const auto & residual_filter_condition_nodes = join_clause.getResidualFilterConditionNodes();
|
||||
auto residual_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(residual_filter_condition_nodes, {}, true);
|
||||
result.residual_join_expressions_actions = std::move(residual_join_expressions_actions);
|
||||
}
|
||||
auto outputs = result.mixed_join_expressions_actions->getOutputs();
|
||||
auto outputs = result.residual_join_expressions_actions->getOutputs();
|
||||
if (outputs.size() != 1)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one output is expected, got: {}", result.mixed_join_expressions_actions->dumpDAG());
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one output is expected, got: {}", result.residual_join_expressions_actions->dumpDAG());
|
||||
}
|
||||
auto output_type = removeNullable(outputs[0]->result_type);
|
||||
WhichDataType which_type(output_type);
|
||||
@ -648,8 +648,8 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
{
|
||||
DataTypePtr uint8_ty = std::make_shared<DataTypeUInt8>();
|
||||
auto true_col = ColumnWithTypeAndName(uint8_ty->createColumnConst(1, 1), uint8_ty, "true");
|
||||
const auto * true_node = &result.mixed_join_expressions_actions->addColumn(true_col);
|
||||
result.mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node});
|
||||
const auto * true_node = &result.residual_join_expressions_actions->addColumn(true_col);
|
||||
result.residual_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -140,7 +140,7 @@ public:
|
||||
return right_filter_condition_nodes;
|
||||
}
|
||||
|
||||
ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes()
|
||||
ActionsDAG::NodeRawConstPtrs & getResidualFilterConditionNodes()
|
||||
{
|
||||
return residual_filter_condition_nodes;
|
||||
}
|
||||
@ -150,7 +150,7 @@ public:
|
||||
residual_filter_condition_nodes.push_back(condition_node);
|
||||
}
|
||||
|
||||
const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const
|
||||
const ActionsDAG::NodeRawConstPtrs & getResidualFilterConditionNodes() const
|
||||
{
|
||||
return residual_filter_condition_nodes;
|
||||
}
|
||||
@ -190,7 +190,7 @@ struct JoinClausesAndActions
|
||||
ActionsDAG right_join_expressions_actions;
|
||||
/// Originally used for inequal join. it's the total join expression.
|
||||
/// If there is no inequal join conditions, it's null.
|
||||
std::optional<ActionsDAG> mixed_join_expressions_actions;
|
||||
std::optional<ActionsDAG> residual_join_expressions_actions;
|
||||
};
|
||||
|
||||
/** Calculate join clauses and actions for JOIN ON section.
|
||||
|
@ -17,7 +17,7 @@ namespace ErrorCodes
|
||||
|
||||
bool FilterTransform::canUseType(const DataTypePtr & filter_type)
|
||||
{
|
||||
return filter_type->onlyNull() || isUInt8(removeNullable(removeLowCardinality(filter_type)));
|
||||
return filter_type->onlyNull() || isUInt8(removeLowCardinalityAndNullable(filter_type));
|
||||
}
|
||||
|
||||
Block FilterTransform::transformHeader(
|
||||
|
Loading…
Reference in New Issue
Block a user