From 2692784585650311c78ae6c5f22dbdfa48be9254 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 29 Feb 2024 19:53:23 +0800 Subject: [PATCH 01/11] support inequal join for left/right/inner all --- src/Columns/IColumn.h | 1 + src/Core/Block.cpp | 14 +- src/Core/Block.h | 4 +- src/Interpreters/HashJoin.cpp | 314 +++++++++++++++++++++++++++++++- src/Interpreters/HashJoin.h | 3 + src/Interpreters/RowRefs.h | 7 +- src/Interpreters/TableJoin.h | 8 + src/Planner/PlannerJoinTree.cpp | 9 + src/Planner/PlannerJoins.cpp | 145 +++++++++++---- src/Planner/PlannerJoins.h | 29 +++ 10 files changed, 484 insertions(+), 50 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index cea8d7c9f55..20d76e8edd0 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 77dbad5443e..34984f122a8 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -14,6 +14,9 @@ #include +#include +#include + namespace DB { @@ -428,6 +431,16 @@ std::string Block::dumpIndex() const return out.str(); } +std::string Block::dumpContent() const +{ +WriteBufferFromOwnString buf; + auto output_format = Context::getGlobalContextInstance()->getOutputFormat("PrettyCompact", buf, *this); + output_format->write(materializeBlock(*this)); + output_format->flush(); + buf.finalize(); + return buf.str(); +} + Block Block::cloneEmpty() const { Block res; @@ -853,5 +866,4 @@ Block concatenateBlocks(const std::vector & blocks) out.setColumns(std::move(columns)); return out; } - } diff --git a/src/Core/Block.h b/src/Core/Block.h index c8bebb4552a..fb12e64fcb7 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -126,6 +126,9 @@ public: /** List of column names and positions from index */ std::string dumpIndex() const; + /// Print all the values in this block. + std::string dumpContent() const; + /** Get the same block, but empty. */ Block cloneEmpty() const; @@ -210,5 +213,4 @@ Block materializeBlock(const Block & block); void materializeBlockInplace(Block & block); Block concatenateBlocks(const std::vector & blocks); - } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 73498b39ead..dc2ddd5a365 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -16,6 +17,7 @@ #include +#include #include #include @@ -32,6 +34,8 @@ #include #include #include +#include "ExpressionActions.h" +#include "RowRefs.h" #include #include @@ -251,6 +255,8 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); + validateAdditionalFilterExpression(table_join->getFullJoinExpression()); + if (isCrossOrComma(kind)) { data->type = Type::CROSS; @@ -1043,14 +1049,17 @@ public: }; AddedColumns( - const Block & left_block, + const Block & left_block_, const Block & block_with_columns_to_add, const Block & saved_block_sample, const HashJoin & join, std::vector && join_on_keys_, + ExpressionActionsPtr additional_filter_expression_, bool is_asof_join, bool is_join_get_) - : join_on_keys(join_on_keys_) + : left_block(left_block_) + , join_on_keys(join_on_keys_) + , additional_filter_expression(additional_filter_expression_) , rows_to_add(left_block.rows()) , is_join_get(is_join_get_) { @@ -1119,7 +1128,9 @@ public: const IColumn & leftAsofKey() const { return *left_asof_key; } + Block left_block; std::vector join_on_keys; + ExpressionActionsPtr additional_filter_expression; size_t max_joined_block_rows = 0; size_t rows_to_add; @@ -1220,7 +1231,7 @@ void AddedColumns::buildOutput() { if (!lazy_output.blocks[j]) { - default_count ++; + default_count++; continue; } apply_default(); @@ -1481,6 +1492,240 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse filter[pos] = 1; } +template +ColumnPtr buildAdditionFilter( + const std::vector> & selected_rows, + const std::vector & row_replicate_offset, + AddedColumns & added_columns) +{ + if (selected_rows.empty()) + return ColumnUInt8::create(); + const Block & sample_right_block = *selected_rows.begin()->first; + if (!sample_right_block) + return ColumnUInt8::create(); + + auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + NameSet required_column_names; + for (auto & col : required_cols) + { + required_column_names.insert(col.name); + } + Block executed_block; + size_t right_col_pos = 0; + for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) + { + if (required_column_names.contains(col.name)) + { + auto new_col = col.column->cloneEmpty(); + for (const auto & selected_row : selected_rows) + { + const auto & src_col = selected_row.first->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row.second); + } + executed_block.insert({std::move(new_col), col.type, col.name}); + } + right_col_pos += 1; + } + if (!executed_block) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected at least one column from right table"); + } + + for (const auto & col_name : required_column_names) + { + const auto * src_col = added_columns.left_block.findByName(col_name); + if (!src_col) + continue; + auto new_col = src_col->column->cloneEmpty(); + size_t prev_left_offset = 0; + for (size_t i = 1; i < row_replicate_offset.size(); ++i) + { + const size_t & left_offset = row_replicate_offset[i]; + size_t rows = left_offset - prev_left_offset; + if (rows) + { + new_col->insertManyFrom(*src_col->column, i - 1, rows); + } + prev_left_offset = left_offset; + } + executed_block.insert({std::move(new_col), src_col->type, col_name}); + } + // LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent()); + added_columns.additional_filter_expression->execute(executed_block); + // LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent()); + return executed_block.getByPosition(0).column; +} + +template +void appendFoundRowAll( + const RowRefList & row_list, + std::vector> & selected_rows, + IColumn::Offset & current_offset, + KnownRowsHolder & known_rows [[maybe_unused]], + JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) +{ + if constexpr (multiple_disjuncts) + { + std::unique_ptr::Type>> new_known_rows_ptr; + for (auto it = row_list.begin(); it.ok(); ++it) + { + auto row_ref = std::make_pair(it->block, it->row_num); + if (!known_rows.isKnown(row_ref)) + { + + selected_rows.push_back(row_ref); + ++current_offset; + if (!new_known_rows_ptr) + { + new_known_rows_ptr = std::make_unique::Type>>(); + } + new_known_rows_ptr->push_back(row_ref); + if constexpr (need_flags) + { + used_flags->JoinStuff::JoinUsedFlags::setUsedOnce( + FindResultImpl(*it, true, 0)); + } + } + } + + if (new_known_rows_ptr) + known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr)); + } + else + { + for (auto it = row_list.begin(); it.ok(); ++it) + { + selected_rows.emplace_back(std::pair(it->block, it->row_num)); + ++current_offset; + } + } +} + +template +NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) +{ + constexpr JoinFeatures join_features; + + size_t rows = added_columns.rows_to_add; + if constexpr (need_filter) + added_columns.filter = IColumn::Filter(rows, 0); + + Arena pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(rows); + + std::vector row_replicate_offset; + row_replicate_offset.reserve(rows); + row_replicate_offset.push_back(0); + + using FindResult = typename KeyGetter::FindResult; + IColumn::Offset current_offset = 0; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t i = 0; + std::vector> selected_rows; + selected_rows.reserve(rows); + std::vector find_results; + /// First, collect matched row refs. + for (; i < rows; ++i) + { + if constexpr (join_features.need_replication) + { + if (unlikely(current_offset >= max_joined_block_rows)) + { + break; + } + } + KnownRowsHolder known_rows; + for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) + { + const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; + if (join_keys.null_map && (*join_keys.null_map)[i]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(i); + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + find_results.push_back(find_result); + if constexpr (join_features.is_all_join) + { + appendFoundRowAll(mapped, selected_rows, current_offset, known_rows, &used_flags); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported join type. kind:{}, strictness:{}", KIND, STRICTNESS); + } + } + } + row_replicate_offset.push_back(current_offset); + } + + /// Second. filtout rows which is not true in additional filter expression. + size_t prev_offset = 0; + const PaddedPODArray * filter_flags = nullptr; + auto filter = buildAdditionFilter(selected_rows, row_replicate_offset, added_columns); + if (filter->isNullable()) + { + auto nested_col = typeid_cast(*filter).getNestedColumnPtr(); + filter_flags = &(dynamic_cast(*nested_col).getData()); + } + else + { + filter_flags = &(dynamic_cast(*filter).getData()); + } + + current_offset = 0; + auto row_it = selected_rows.begin(); + for (size_t j = 1; j < row_replicate_offset.size(); ++j) + { + bool any_matched = false; + for (size_t k = prev_offset; k < row_replicate_offset[j]; ++k) + { + if ((*filter_flags)[k]) + { + any_matched = true; + added_columns.appendFromBlock(*row_it->first, row_it->second, join_features.add_missing); + current_offset += 1; + } + ++row_it; + } + if (!any_matched) + { + if constexpr (join_features.is_anti_join && join_features.left) + setUsed(added_columns.filter, j - 1); + addNotFoundRow(added_columns, current_offset); + } + else + { + if constexpr (join_features.is_all_join) + { + used_flags.template setUsed(find_results[j - 1]); + setUsed(added_columns.filter, j - 1); + } + } + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[j-1] = current_offset; + } + prev_offset = row_replicate_offset[j]; + } + if constexpr (join_features.need_replication) + { + added_columns.offsets_to_replicate->resize_assume_reserved(i); + added_columns.filter.resize_assume_reserved(i); + } + added_columns.applyLazyDefaults(); + return i; +} + /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). template @@ -1497,7 +1742,7 @@ NO_INLINE size_t joinRightColumns( added_columns.filter = IColumn::Filter(rows, 0); Arena pool; - + if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); @@ -1629,9 +1874,20 @@ size_t joinRightColumnsSwitchMultipleDisjuncts( AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - return mapv.size() > 1 - ? joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags); + if (added_columns.additional_filter_expression) + { + return mapv.size() > 1 ? joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } + else + { + return mapv.size() > 1 ? joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumns( + std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } } template @@ -1787,8 +2043,14 @@ Block HashJoin::joinBlockImpl( * For ASOF, the last column is used as the ASOF column */ AddedColumns added_columns( - block, block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), join_features.is_asof_join, is_join_get); - + block, + block_with_columns_to_add, + savedBlockSample(), + *this, + std::move(join_on_keys), + table_join->getFullJoinExpression(), + join_features.is_asof_join, + is_join_get); bool has_required_right_keys = (required_right_keys.columns() != 0); added_columns.need_filter = join_features.need_filter || has_required_right_keys; @@ -1855,11 +2117,15 @@ Block HashJoin::joinBlockImpl( /// If ALL ... JOIN - we replicate all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) + { block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); + } /// Replicate additional right keys for (size_t pos : right_keys_to_replicate) + { block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); + } } return remaining_block; @@ -2393,4 +2659,34 @@ const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); } +void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additional_filter_expression) +{ + if (!additional_filter_expression) + return; + Block expression_sample_block = additional_filter_expression->getSampleBlock(); + + if (expression_sample_block.columns() != 1) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected expression in JOIN ON section. Expected single column, got '{}'", + expression_sample_block.dumpStructure()); + } + + auto type = removeNullable(expression_sample_block.getByPosition(0).type); + if (!type->equals(*std::make_shared())) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'", + expression_sample_block.getByPosition(0).type->getName()); + } + + bool is_supported = (strictness == JoinStrictness::All) && (kind == JoinKind::Inner || kind == JoinKind::Left || kind == JoinKind::Right); + if (!is_supported) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/RIGHT JOINs", + expression_sample_block.getByPosition(0).name); + } +} + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 3e11517edad..2e5c7c12323 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -470,6 +471,8 @@ private: static Type chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes); bool empty() const; + + void validateAdditionalFilterExpression(ExpressionActionsPtr additional_filter_expression); }; } diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 650b2311ba7..b391b77c0b7 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -122,7 +123,7 @@ struct RowRefList : RowRef }; RowRefList() {} /// NOLINT - RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {} + RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {} ForwardIterator begin() const { return ForwardIterator(this); } @@ -135,10 +136,14 @@ struct RowRefList : RowRef *next = Batch(nullptr); } next = next->insert(std::move(row_ref), pool); + rows += 1; } + size_t toalRows() const { return rows; } + private: Batch * next = nullptr; + size_t rows = 0; }; /** diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 75e2342d1e9..6a449f967fc 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -11,6 +11,8 @@ #include #include +#include "ActionsDAG.h" +#include "ExpressionActions.h" #include #include @@ -153,6 +155,8 @@ private: ASTs key_asts_right; Clauses clauses; + /// Originally used for inequal join. If there is no any inequal join condition, it will be nullptr. + ExpressionActionsPtr full_join_expression = nullptr; ASTTableJoin table_join; @@ -298,6 +302,10 @@ public: std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } + const ExpressionActionsPtr & getFullJoinExpression() const { return full_join_expression;} + ExpressionActionsPtr & getFullJoinExpression() { return full_join_expression;} + + Names getAllNames(JoinTableSide side) const; void resetCollected(); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index ba4f9718107..9c58511f070 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -1317,6 +1318,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ std::swap(table_join_clause.key_names_right.at(asof_condition.key_index), table_join_clause.key_names_right.back()); } } + + if (join_clauses_and_actions.full_join_expressions_actions) + { + ExpressionActionsPtr & full_join_expression = table_join->getFullJoinExpression(); + full_join_expression = std::make_shared( + join_clauses_and_actions.full_join_expressions_actions, + ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::no)); + } } else if (join_node.isUsingJoinExpression()) { diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 78b0e966ca4..44a56827450 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -41,6 +42,7 @@ #include #include #include +#include "PlannerActionsVisitor.h" namespace DB { @@ -125,13 +127,13 @@ TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node) return res; } -std::optional extractJoinTableSideFromExpression( +std::set extractJoinTableSidesFromExpression(//const ActionsDAG::Node * expression_root_node, const IQueryTreeNode * expression_root_node, const TableExpressionSet & left_table_expressions, const TableExpressionSet & right_table_expressions, const JoinNode & join_node) { - std::optional table_side; + std::set table_sides; std::vector nodes_to_process; nodes_to_process.push_back(expression_root_node); @@ -169,15 +171,10 @@ std::optional extractJoinTableSideFromExpression( join_node.getRightTableExpression()->formatASTForErrorMessage()); auto input_table_side = is_column_from_left_expr ? JoinTableSide::Left : JoinTableSide::Right; - if (table_side && (*table_side) != input_table_side) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} join expression contains column from left and right table", - join_node.formatASTForErrorMessage()); - - table_side = input_table_side; + table_sides.insert(input_table_side); } - return table_side; + return table_sides; } const ActionsDAG::Node * appendExpression( @@ -199,6 +196,7 @@ const ActionsDAG::Node * appendExpression( void buildJoinClause( ActionsDAGPtr & left_dag, ActionsDAGPtr & right_dag, + ActionsDAGPtr & mixed_dag, const PlannerContextPtr & planner_context, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, @@ -219,6 +217,7 @@ void buildJoinClause( buildJoinClause( left_dag, right_dag, + mixed_dag, planner_context, child, left_table_expressions, @@ -235,41 +234,52 @@ void buildJoinClause( if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality) { + if (function_node->getArguments().getNodes().size() != 2) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} ON expression expected two arguments", + join_node.formatASTForErrorMessage()); + } + const auto left_child = function_node->getArguments().getNodes().at(0); const auto right_child = function_node->getArguments().getNodes().at(1); - auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(), + auto left_expression_sides = extractJoinTableSidesFromExpression(left_child.get(), left_table_expressions, right_table_expressions, join_node); - auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(), + auto right_expression_sides = extractJoinTableSidesFromExpression(right_child.get(), left_table_expressions, right_table_expressions, join_node); - - if (!left_expression_side_optional && !right_expression_side_optional) + + if (left_expression_sides.empty() && right_expression_sides.empty()) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} ON expression with constants is not supported", + "JOIN {} ON expression expected non-empty left and right table expressions", join_node.formatASTForErrorMessage()); } - else if (left_expression_side_optional && !right_expression_side_optional) + else if (left_expression_sides.size() == 1 && right_expression_sides.empty()) { - auto & dag = *left_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag; + auto expression_side = *left_expression_sides.begin(); + auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag; const auto * node = appendExpression(dag, join_expression, planner_context, join_node); - join_clause.addCondition(*left_expression_side_optional, node); + join_clause.addCondition(expression_side, node); + } - else if (!left_expression_side_optional && right_expression_side_optional) + else if (left_expression_sides.empty() && right_expression_sides.size() == 1) { - auto & dag = *right_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag; + auto expression_side = *right_expression_sides.begin(); + auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag; const auto * node = appendExpression(dag, join_expression, planner_context, join_node); - join_clause.addCondition(*right_expression_side_optional, node); + join_clause.addCondition(expression_side, node); + } - else + else if (left_expression_sides.size() == 1 && right_expression_sides.size() == 1) { - auto left_expression_side = *left_expression_side_optional; - auto right_expression_side = *right_expression_side_optional; + auto left_expression_side = *left_expression_sides.begin(); + auto right_expression_side = *right_expression_sides.begin(); if (left_expression_side != right_expression_side) { @@ -310,23 +320,36 @@ void buildJoinClause( join_clause.addCondition(left_expression_side, node); } } + else + { + /// expression involves both tables. + /// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)` + const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); + join_clause.addMixedCondition(node); + } - return; } + else + { + auto expression_sides = extractJoinTableSidesFromExpression(join_expression.get(), + left_table_expressions, + right_table_expressions, + join_node); - auto expression_side_optional = extractJoinTableSideFromExpression( - join_expression.get(), - left_table_expressions, - right_table_expressions, - join_node); - - if (!expression_side_optional) - expression_side_optional = JoinTableSide::Right; - - auto expression_side = *expression_side_optional; - auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag; - const auto * node = appendExpression(dag, join_expression, planner_context, join_node); - join_clause.addCondition(expression_side, node); + if (expression_sides.empty() || expression_sides.size() == 1) + { + auto expression_side = expression_sides.empty() ? JoinTableSide::Right : *expression_sides.begin(); + auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag; + const auto * node = appendExpression(dag, join_expression, planner_context, join_node); + join_clause.addCondition(expression_side, node); + } + else + { + /// expression involves both tables. + const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); + join_clause.addMixedCondition(node); + } + } } JoinClausesAndActions buildJoinClausesAndActions( @@ -337,6 +360,16 @@ JoinClausesAndActions buildJoinClausesAndActions( { ActionsDAGPtr left_join_actions = std::make_shared(left_table_expression_columns); ActionsDAGPtr right_join_actions = std::make_shared(right_table_expression_columns); + ColumnsWithTypeAndName mixed_table_expression_columns; + for (const auto & left_column : left_table_expression_columns) + { + mixed_table_expression_columns.push_back(left_column); + } + for (const auto & right_column : right_table_expression_columns) + { + mixed_table_expression_columns.push_back(right_column); + } + ActionsDAGPtr mixed_join_actions = std::make_shared(mixed_table_expression_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -390,6 +423,7 @@ JoinClausesAndActions buildJoinClausesAndActions( JoinClausesAndActions result; + bool is_inequal_join = false; const auto & function_name = function_node->getFunction()->getName(); if (function_name == "or") { @@ -400,12 +434,14 @@ JoinClausesAndActions buildJoinClausesAndActions( buildJoinClause( left_join_actions, right_join_actions, + mixed_join_actions, planner_context, child, join_left_table_expressions, join_right_table_expressions, join_node, result.join_clauses.back()); + is_inequal_join |= result.join_clauses.back().hasMixedFilterCondition(); } } else @@ -415,12 +451,14 @@ JoinClausesAndActions buildJoinClausesAndActions( buildJoinClause( left_join_actions, right_join_actions, + mixed_join_actions, planner_context, join_expression, join_left_table_expressions, join_right_table_expressions, join_node, result.join_clauses.back()); + is_inequal_join |= result.join_clauses.back().hasMixedFilterCondition(); } auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext()); @@ -441,7 +479,6 @@ JoinClausesAndActions buildJoinClausesAndActions( if (!left_filter_condition_nodes.empty()) { const ActionsDAG::Node * dag_filter_condition_node = nullptr; - if (left_filter_condition_nodes.size() > 1) dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {}); else @@ -540,6 +577,32 @@ JoinClausesAndActions buildJoinClausesAndActions( result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + /// If there is any inequal join condition, we need to build full join expressions actions. + if (is_inequal_join) + { + if (result.join_clauses.size() > 1) + { + auto full_join_expressions_actions = std::make_shared(mixed_table_expression_columns); + PlannerActionsVisitor join_expression_visitor(planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(full_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()); + + full_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); + Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; + full_join_expressions_actions->removeUnusedActions(required_names); + result.full_join_expressions_actions = full_join_expressions_actions; + } + else + { + const auto & join_clause = result.join_clauses.front(); + const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes(); + auto full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); + result.full_join_expressions_actions = full_join_expressions_actions; + } + } + return result; } @@ -751,6 +814,12 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo const Block & right_table_expression_header, const PlannerContextPtr & planner_context) { + if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "JOIN with mixed conditions supports only hash join algorithm"); + } + trySetStorageInTableJoin(right_table_expression, table_join); auto & right_table_expression_data = planner_context->getTableExpressionDataOrThrow(right_table_expression); diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 7bc65cfb544..79d8de6a538 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -9,6 +9,9 @@ #include +#include +#include + namespace DB { @@ -71,6 +74,7 @@ public: /// Add condition for table side void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) { + LOG_ERROR(getLogger("JoinClause"), "xxx add condition. {}. {}", table_side, condition_node->result_name); auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; filter_condition_nodes.push_back(condition_node); } @@ -140,6 +144,26 @@ public: return right_filter_condition_nodes; } + ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() + { + return mixed_filter_condition_nodes; + } + + void addMixedCondition(const ActionsDAG::Node * condition_node) + { + mixed_filter_condition_nodes.push_back(condition_node); + } + + const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const + { + return mixed_filter_condition_nodes; + } + + bool hasMixedFilterCondition() const + { + // return has_mixed_filter_condition; + return !mixed_filter_condition_nodes.empty(); + } /// Dump clause into buffer void dump(WriteBuffer & buffer) const; @@ -154,6 +178,8 @@ private: ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; + /// conditions which involve both left and right tables + ActionsDAG::NodeRawConstPtrs mixed_filter_condition_nodes; std::unordered_set nullsafe_compare_key_indexes; }; @@ -171,6 +197,9 @@ struct JoinClausesAndActions ActionsDAGPtr left_join_expressions_actions; /// Right join expressions actions ActionsDAGPtr 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. + ActionsDAGPtr full_join_expressions_actions; }; /** Calculate join clauses and actions for JOIN ON section. From 4a2ad7beb5cae0489d87fe66c8b80c7b14ac505e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Mar 2024 09:23:34 +0800 Subject: [PATCH 02/11] fixed bugs --- src/Columns/IColumn.h | 1 - src/Core/Block.cpp | 16 ++++++++-------- src/Core/Block.h | 1 + src/Interpreters/HashJoin.cpp | 6 +++--- src/Interpreters/RowRefs.h | 7 +------ src/Interpreters/TableJoin.h | 9 ++++----- src/Planner/PlannerJoinTree.cpp | 1 - src/Planner/PlannerJoins.cpp | 11 +---------- src/Planner/PlannerJoins.h | 4 ---- 9 files changed, 18 insertions(+), 38 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 20d76e8edd0..cea8d7c9f55 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 34984f122a8..96d26e5fbd3 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -10,12 +10,12 @@ #include #include +#include + #include #include -#include -#include namespace DB @@ -433,12 +433,12 @@ std::string Block::dumpIndex() const std::string Block::dumpContent() const { -WriteBufferFromOwnString buf; - auto output_format = Context::getGlobalContextInstance()->getOutputFormat("PrettyCompact", buf, *this); - output_format->write(materializeBlock(*this)); - output_format->flush(); - buf.finalize(); - return buf.str(); + WriteBufferFromOwnString buf; + auto output_format = Context::getGlobalContextInstance()->getOutputFormat("PrettyCompact", buf, *this); + output_format->write(materializeBlock(*this)); + output_format->flush(); + buf.finalize(); + return buf.str(); } Block Block::cloneEmpty() const diff --git a/src/Core/Block.h b/src/Core/Block.h index fb12e64fcb7..9c054b7aea7 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -213,4 +213,5 @@ Block materializeBlock(const Block & block); void materializeBlockInplace(Block & block); Block concatenateBlocks(const std::vector & blocks); + } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index dc2ddd5a365..744d904353a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -21,11 +21,13 @@ #include #include +#include #include #include #include #include #include +#include #include @@ -34,8 +36,6 @@ #include #include #include -#include "ExpressionActions.h" -#include "RowRefs.h" #include #include @@ -1742,7 +1742,7 @@ NO_INLINE size_t joinRightColumns( added_columns.filter = IColumn::Filter(rows, 0); Arena pool; - + if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index b391b77c0b7..650b2311ba7 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -123,7 +122,7 @@ struct RowRefList : RowRef }; RowRefList() {} /// NOLINT - RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {} + RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {} ForwardIterator begin() const { return ForwardIterator(this); } @@ -136,14 +135,10 @@ struct RowRefList : RowRef *next = Batch(nullptr); } next = next->insert(std::move(row_ref), pool); - rows += 1; } - size_t toalRows() const { return rows; } - private: Batch * next = nullptr; - size_t rows = 0; }; /** diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 6a449f967fc..fec5c47c954 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -11,8 +13,6 @@ #include #include -#include "ActionsDAG.h" -#include "ExpressionActions.h" #include #include @@ -302,9 +302,8 @@ public: std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } - const ExpressionActionsPtr & getFullJoinExpression() const { return full_join_expression;} - ExpressionActionsPtr & getFullJoinExpression() { return full_join_expression;} - + const ExpressionActionsPtr & getFullJoinExpression() const { return full_join_expression; } + ExpressionActionsPtr & getFullJoinExpression() { return full_join_expression; } Names getAllNames(JoinTableSide side) const; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9c58511f070..1f13bed3694 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 44a56827450..8e5fca418f9 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -42,7 +41,6 @@ #include #include #include -#include "PlannerActionsVisitor.h" namespace DB { @@ -234,13 +232,6 @@ void buildJoinClause( if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality) { - if (function_node->getArguments().getNodes().size() != 2) - { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} ON expression expected two arguments", - join_node.formatASTForErrorMessage()); - } - const auto left_child = function_node->getArguments().getNodes().at(0); const auto right_child = function_node->getArguments().getNodes().at(1); @@ -253,7 +244,7 @@ void buildJoinClause( left_table_expressions, right_table_expressions, join_node); - + if (left_expression_sides.empty() && right_expression_sides.empty()) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 79d8de6a538..7d3df48af8e 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -9,9 +9,6 @@ #include -#include -#include - namespace DB { @@ -74,7 +71,6 @@ public: /// Add condition for table side void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) { - LOG_ERROR(getLogger("JoinClause"), "xxx add condition. {}. {}", table_side, condition_node->result_name); auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; filter_condition_nodes.push_back(condition_node); } From f7dd4deca0513357ff0d37ac689ac272a3302b9b Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Mar 2024 14:47:08 +0800 Subject: [PATCH 03/11] Optimization for cutting left blocks into processed and not_processed parts. --- src/Interpreters/HashJoin.cpp | 212 +++++++++++++++++++++------------- 1 file changed, 130 insertions(+), 82 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 744d904353a..d92601adf40 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -1494,6 +1495,7 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse template ColumnPtr buildAdditionFilter( + size_t left_start_row, const std::vector> & selected_rows, const std::vector & row_replicate_offset, AddedColumns & added_columns) @@ -1544,7 +1546,7 @@ ColumnPtr buildAdditionFilter( size_t rows = left_offset - prev_left_offset; if (rows) { - new_col->insertManyFrom(*src_col->column, i - 1, rows); + new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); } prev_left_offset = left_offset; } @@ -1601,6 +1603,7 @@ void appendFoundRowAll( } } +/// First to collect all matched rows by join keys, then filter out rows which is not true in additional filter expression. template NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, @@ -1610,120 +1613,165 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { constexpr JoinFeatures join_features; - size_t rows = added_columns.rows_to_add; + size_t left_block_rows = added_columns.rows_to_add; if constexpr (need_filter) - added_columns.filter = IColumn::Filter(rows, 0); + added_columns.filter = IColumn::Filter(left_block_rows, 0); - Arena pool; + std::unique_ptr pool; if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(rows); + added_columns.offsets_to_replicate = std::make_unique(left_block_rows); std::vector row_replicate_offset; - row_replicate_offset.reserve(rows); - row_replicate_offset.push_back(0); + row_replicate_offset.reserve(left_block_rows); using FindResult = typename KeyGetter::FindResult; - IColumn::Offset current_offset = 0; size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t i = 0; + size_t left_row_iter = 0; std::vector> selected_rows; - selected_rows.reserve(rows); + selected_rows.reserve(left_block_rows); std::vector find_results; - /// First, collect matched row refs. - for (; i < rows; ++i) - { - if constexpr (join_features.need_replication) - { - if (unlikely(current_offset >= max_joined_block_rows)) - { - break; - } - } - KnownRowsHolder known_rows; - for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) - { - const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; - if (join_keys.null_map && (*join_keys.null_map)[i]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(i); - auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + find_results.reserve(left_block_rows); + bool exceeded_max_block_rows = false; + IColumn::Offset total_added_rows = 0; + IColumn::Offset current_added_rows = 0; - if (find_result.isFound()) + auto collect_keys_matched_rows_refs = [&]() + { + pool = std::make_unique(); + find_results.clear(); + row_replicate_offset.clear(); + row_replicate_offset.push_back(0); + current_added_rows = 0; + selected_rows.clear(); + for (; left_row_iter < left_block_rows; ++left_row_iter) + { + if constexpr (join_features.need_replication) { - auto & mapped = find_result.getMapped(); - find_results.push_back(find_result); - if constexpr (join_features.is_all_join) + if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) { - appendFoundRowAll(mapped, selected_rows, current_offset, known_rows, &used_flags); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported join type. kind:{}, strictness:{}", KIND, STRICTNESS); + break; } } - } - row_replicate_offset.push_back(current_offset); - } - - /// Second. filtout rows which is not true in additional filter expression. - size_t prev_offset = 0; - const PaddedPODArray * filter_flags = nullptr; - auto filter = buildAdditionFilter(selected_rows, row_replicate_offset, added_columns); - if (filter->isNullable()) - { - auto nested_col = typeid_cast(*filter).getNestedColumnPtr(); - filter_flags = &(dynamic_cast(*nested_col).getData()); - } - else - { - filter_flags = &(dynamic_cast(*filter).getData()); - } - - current_offset = 0; - auto row_it = selected_rows.begin(); - for (size_t j = 1; j < row_replicate_offset.size(); ++j) - { - bool any_matched = false; - for (size_t k = prev_offset; k < row_replicate_offset[j]; ++k) - { - if ((*filter_flags)[k]) + KnownRowsHolder known_rows; + for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) { - any_matched = true; - added_columns.appendFromBlock(*row_it->first, row_it->second, join_features.add_missing); - current_offset += 1; + const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; + if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); + auto find_result = row_acceptable + ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) + : FindResult(); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + find_results.push_back(find_result); + if constexpr (join_features.is_all_join) + { + appendFoundRowAll( + mapped, selected_rows, current_added_rows, known_rows, &used_flags); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported join type. kind:{}, strictness:{}", KIND, STRICTNESS); + } + } } - ++row_it; + row_replicate_offset.push_back(current_added_rows); } - if (!any_matched) + }; + + auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) + { + const PaddedPODArray * filter_flags = nullptr; + if (filter_col->isNullable()) { - if constexpr (join_features.is_anti_join && join_features.left) - setUsed(added_columns.filter, j - 1); - addNotFoundRow(added_columns, current_offset); + auto nested_col = typeid_cast(*filter_col).getNestedColumnPtr(); + filter_flags = &(dynamic_cast(*nested_col).getData()); } else { - if constexpr (join_features.is_all_join) - { - used_flags.template setUsed(find_results[j - 1]); - setUsed(added_columns.filter, j - 1); - } + filter_flags = &(dynamic_cast(*filter_col).getData()); } + size_t prev_replicated_row = 0; + auto selected_right_row_it = selected_rows.begin(); + for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) + { + bool any_matched = false; + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if ((*filter_flags)[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->first, selected_right_row_it->second, join_features.add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + if (!any_matched) + { + if constexpr (join_features.is_anti_join && join_features.left) + setUsed(added_columns.filter, i - 1); + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if constexpr (join_features.is_all_join) + { + used_flags.template setUsed(find_results[i - 1]); + setUsed(added_columns.filter, left_start_row + i - 1); + } + } + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; + } + prev_replicated_row = row_replicate_offset[i]; + } + + }; + + while (left_row_iter < left_block_rows && !exceeded_max_block_rows) + { + auto left_start_row = left_row_iter; + collect_keys_matched_rows_refs(); + if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " + "left_start_row: {}", + selected_rows.size(), + current_added_rows, + row_replicate_offset.size(), + left_row_iter, + left_start_row); + } + auto filter_col = buildAdditionFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + copy_final_matched_rows(left_start_row, filter_col); + if constexpr (join_features.need_replication) { - (*added_columns.offsets_to_replicate)[j-1] = current_offset; + // Add a check for current_added_rows to avoid run the filter expression on too small size batch. + if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) + { + exceeded_max_block_rows = true; + } } - prev_offset = row_replicate_offset[j]; } + if constexpr (join_features.need_replication) { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); + added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); + added_columns.filter.resize_assume_reserved(left_row_iter); } added_columns.applyLazyDefaults(); - return i; + return left_row_iter; } /// Joins right table columns which indexes are present in right_indexes using specified map. From cd9c87811fe2f5291da5ac5175e11e40f079acfa Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 7 Mar 2024 19:51:54 +0800 Subject: [PATCH 04/11] fixed bugs for all right join add test cases fixed typos add document fixed stype error and rename some variables fixed test error add test cases try to recude hash join code size debug fixed fixed coredump fixed. condition result type is not bool enable when allow_experimental_analyzer=0 fixed tests fixed update tests update tests roll backup when use the old analyzer fixed typos fixed test large obj check test test --- .../sql-reference/statements/select/join.md | 45 +++ src/Core/Block.cpp | 2 +- src/Core/Settings.h | 1 + src/Interpreters/CollectJoinOnKeysVisitor.h | 1 - src/Interpreters/HashJoin.cpp | 282 ++++++++------ src/Interpreters/HashJoin.h | 13 +- src/Interpreters/TableJoin.h | 9 +- src/Planner/PlannerJoinTree.cpp | 2 +- src/Planner/PlannerJoins.cpp | 58 ++- src/Planner/PlannerJoins.h | 3 +- ..._join_on_inequal_expression_fast.reference | 366 ++++++++++++++++++ ...006_join_on_inequal_expression_fast.sql.j2 | 51 +++ utils/check-style/check-large-objects.sh | 2 +- 13 files changed, 695 insertions(+), 140 deletions(-) create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference create mode 100644 tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 95d0489d694..141bb80ceb4 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -164,6 +164,51 @@ Result: │ 4 │ -4 │ 4 │ └───┴────┴─────┘ ``` +## Join with inequality conditions +Clickhouse currently supports inner, left, right and full join with inequality conditions, including with `OR` operator. You need to set `allow_experimental_analyzer = 1` and select `hash` or `grace_hash` join algorithm. + +**Example** + +Table `t1`: +``` +┌─key──┬─attr─┬─a─┬─b─┬─c─┐ +│ key1 │ a │ 1 │ 1 │ 2 │ +│ key1 │ b │ 2 │ 3 │ 2 │ +│ key1 │ c │ 3 │ 2 │ 1 │ +│ key1 │ d │ 4 │ 7 │ 2 │ +│ key1 │ e │ 5 │ 5 │ 5 │ +│ key2 │ a2 │ 1 │ 1 │ 1 │ +│ key4 │ f │ 2 │ 3 │ 4 │ +└──────┴──────┴───┴───┴───┘ +``` + +Table `t2` +``` +┌─key──┬─attr─┬─a─┬─b─┬─c─┐ +│ key1 │ A │ 1 │ 2 │ 1 │ +│ key1 │ B │ 2 │ 1 │ 2 │ +│ key1 │ C │ 3 │ 4 │ 5 │ +│ key1 │ D │ 4 │ 1 │ 6 │ +│ key3 │ a3 │ 1 │ 1 │ 1 │ +│ key4 │ F │ 1 │ 1 │ 1 │ +└──────┴──────┴───┴───┴───┘ +``` +```sql +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +``` +``` +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +``` + ## NULL values in JOIN keys diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 96d26e5fbd3..bb8225af562 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -17,7 +17,6 @@ #include - namespace DB { @@ -866,4 +865,5 @@ Block concatenateBlocks(const std::vector & blocks) out.setColumns(std::move(columns)); return out; } + } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 86ac53f07e8..1690add4e2d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -321,6 +321,7 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ + M(Bool, enable_mixed_join_condition, false, "Support conditions involve columns from both tables in on join expression.", IMPORTANT) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index ff10db3beb5..7218f96d781 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -8,7 +8,6 @@ #include #include - namespace DB { diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index d92601adf40..823c5d48db9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -55,6 +55,7 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INVALID_JOIN_ON_EXPRESSION; } namespace @@ -124,14 +125,14 @@ namespace JoinStuff } } - template + template void JoinUsedFlags::setUsed(const FindResult & f) { if constexpr (!use_flags) return; /// Could be set simultaneously from different threads. - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { auto & mapped = f.getMapped(); flags[mapped.block][mapped.row_num].store(true, std::memory_order_relaxed); @@ -142,14 +143,14 @@ namespace JoinStuff } } - template + template void JoinUsedFlags::setUsed(const Block * block, size_t row_num, size_t offset) { if constexpr (!use_flags) return; /// Could be set simultaneously from different threads. - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { flags[block][row_num].store(true, std::memory_order_relaxed); } @@ -159,13 +160,13 @@ namespace JoinStuff } } - template + template bool JoinUsedFlags::getUsed(const FindResult & f) { if constexpr (!use_flags) return true; - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { auto & mapped = f.getMapped(); return flags[mapped.block][mapped.row_num].load(); @@ -176,13 +177,13 @@ namespace JoinStuff } } - template + template bool JoinUsedFlags::setUsedOnce(const FindResult & f) { if constexpr (!use_flags) return true; - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { auto & mapped = f.getMapped(); @@ -711,7 +712,8 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || isRightOrFull(kind) || - multiple_disjuncts; + multiple_disjuncts || + table_join->getFullJoinExpression(); if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -841,7 +843,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (rows) data->empty = false; - bool multiple_disjuncts = !table_join->oneDisjunct(); + bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); const auto & onexprs = table_join->getClauses(); for (size_t onexpr_idx = 0; onexpr_idx < onexprs.size(); ++onexpr_idx) { @@ -865,7 +867,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) auto join_mask_col = JoinCommon::getColumnAsMask(source_block, onexprs[onexpr_idx].condColumnNames().second); /// Save blocks that do not hold conditions in ON section ColumnUInt8::MutablePtr not_joined_map = nullptr; - if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col.hasData()) + if (!flag_per_row && isRightOrFull(kind) && join_mask_col.hasData()) { const auto & join_mask = join_mask_col.getData(); /// Save rows that do not hold conditions @@ -895,7 +897,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) join_mask_col.getData(), data->pool, is_inserted); - if (multiple_disjuncts) + if (flag_per_row) used_flags.reinit(stored_block); else if (is_inserted) /// Number of buckets + 1 value from zero storage @@ -903,19 +905,19 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) }); } - if (!multiple_disjuncts && save_nullmap && is_inserted) + if (!flag_per_row && save_nullmap && is_inserted) { data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); } - if (!multiple_disjuncts && not_joined_map && is_inserted) + if (!flag_per_row && not_joined_map && is_inserted) { data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } - if (!multiple_disjuncts && !is_inserted) + if (!flag_per_row && !is_inserted) { LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); @@ -1351,7 +1353,7 @@ struct JoinFeatures static constexpr bool need_flags = MapGetter::flagged; }; -template +template class KnownRowsHolder; /// Keep already joined rows to prevent duplication if many disjuncts @@ -1426,18 +1428,18 @@ public: } }; -template +template void addFoundRowAll( const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset, - KnownRowsHolder & known_rows [[maybe_unused]], + KnownRowsHolder & known_rows [[maybe_unused]], JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) { if constexpr (add_missing) added.applyLazyDefaults(); - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { std::unique_ptr::Type>> new_known_rows_ptr; @@ -1454,7 +1456,7 @@ void addFoundRowAll( new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num)); if (used_flags) { - used_flags->JoinStuff::JoinUsedFlags::setUsedOnce( + used_flags->JoinStuff::JoinUsedFlags::setUsedOnce( FindResultImpl(*it, true, 0)); } } @@ -1494,24 +1496,31 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse } template -ColumnPtr buildAdditionFilter( +ColumnPtr buildAdditionalFilter( size_t left_start_row, - const std::vector> & selected_rows, + const std::vector & selected_rows, const std::vector & row_replicate_offset, AddedColumns & added_columns) { if (selected_rows.empty()) return ColumnUInt8::create(); - const Block & sample_right_block = *selected_rows.begin()->first; + const Block & sample_right_block = *selected_rows.begin()->block; if (!sample_right_block) return ColumnUInt8::create(); auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + if (required_cols.empty()) + { + Block block; + added_columns.additional_filter_expression->execute(block); + return block.getByPosition(0).column->cloneResized(selected_rows.size()); + } NameSet required_column_names; for (auto & col : required_cols) { required_column_names.insert(col.name); } + Block executed_block; size_t right_col_pos = 0; for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) @@ -1521,8 +1530,8 @@ ColumnPtr buildAdditionFilter( auto new_col = col.column->cloneEmpty(); for (const auto & selected_row : selected_rows) { - const auto & src_col = selected_row.first->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row.second); + const auto & src_col = selected_row.block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row.row_num); } executed_block.insert({std::move(new_col), col.type, col.name}); } @@ -1530,7 +1539,7 @@ ColumnPtr buildAdditionFilter( } if (!executed_block) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected at least one column from right table"); + return ColumnUInt8::create(); } for (const auto & col_name : required_column_names) @@ -1552,21 +1561,20 @@ ColumnPtr buildAdditionFilter( } executed_block.insert({std::move(new_col), src_col->type, col_name}); } - // LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent()); + LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent()); added_columns.additional_filter_expression->execute(executed_block); - // LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent()); + LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent()); return executed_block.getByPosition(0).column; } -template -void appendFoundRowAll( +template +void addFoundRowRefAll( const RowRefList & row_list, - std::vector> & selected_rows, + std::vector & selected_rows, IColumn::Offset & current_offset, - KnownRowsHolder & known_rows [[maybe_unused]], - JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) + KnownRowsHolder & known_rows [[maybe_unused]]) { - if constexpr (multiple_disjuncts) + if constexpr (flag_per_row) { std::unique_ptr::Type>> new_known_rows_ptr; for (auto it = row_list.begin(); it.ok(); ++it) @@ -1574,19 +1582,13 @@ void appendFoundRowAll( auto row_ref = std::make_pair(it->block, it->row_num); if (!known_rows.isKnown(row_ref)) { - - selected_rows.push_back(row_ref); + selected_rows.emplace_back(row_ref.first, row_ref.second); ++current_offset; if (!new_known_rows_ptr) { new_known_rows_ptr = std::make_unique::Type>>(); } new_known_rows_ptr->push_back(row_ref); - if constexpr (need_flags) - { - used_flags->JoinStuff::JoinUsedFlags::setUsedOnce( - FindResultImpl(*it, true, 0)); - } } } @@ -1597,38 +1599,44 @@ void appendFoundRowAll( { for (auto it = row_list.begin(); it.ok(); ++it) { - selected_rows.emplace_back(std::pair(it->block, it->row_num)); + selected_rows.emplace_back(it->block, it->row_num); ++current_offset; } } } -/// First to collect all matched rows by join keys, then filter out rows which is not true in additional filter expression. -template +/// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. +template < + typename KeyGetter, + typename Map, + bool need_filter, + bool need_replication, + bool need_flags, + bool add_missing, + bool flag_per_row, + typename AddedColumns> NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - constexpr JoinFeatures join_features; - size_t left_block_rows = added_columns.rows_to_add; if constexpr (need_filter) added_columns.filter = IColumn::Filter(left_block_rows, 0); std::unique_ptr pool; - if constexpr (join_features.need_replication) + if constexpr (need_replication) added_columns.offsets_to_replicate = std::make_unique(left_block_rows); std::vector row_replicate_offset; row_replicate_offset.reserve(left_block_rows); - + using FindResult = typename KeyGetter::FindResult; size_t max_joined_block_rows = added_columns.max_joined_block_rows; size_t left_row_iter = 0; - std::vector> selected_rows; + std::vector selected_rows; selected_rows.reserve(left_block_rows); std::vector find_results; find_results.reserve(left_block_rows); @@ -1646,14 +1654,14 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( selected_rows.clear(); for (; left_row_iter < left_block_rows; ++left_row_iter) { - if constexpr (join_features.need_replication) + if constexpr (need_replication) { if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) { break; } } - KnownRowsHolder known_rows; + KnownRowsHolder known_rows; for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) { const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; @@ -1669,15 +1677,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { auto & mapped = find_result.getMapped(); find_results.push_back(find_result); - if constexpr (join_features.is_all_join) - { - appendFoundRowAll( - mapped, selected_rows, current_added_rows, known_rows, &used_flags); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported join type. kind:{}, strictness:{}", KIND, STRICTNESS); - } + addFoundRowRefAll(mapped, selected_rows, current_added_rows, known_rows); } } row_replicate_offset.push_back(current_added_rows); @@ -1687,6 +1687,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) { const PaddedPODArray * filter_flags = nullptr; + filter_col = filter_col->convertToFullIfNeeded(); if (filter_col->isNullable()) { auto nested_col = typeid_cast(*filter_col).getNestedColumnPtr(); @@ -1699,41 +1700,58 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( size_t prev_replicated_row = 0; auto selected_right_row_it = selected_rows.begin(); + size_t find_result_index = 0; for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) { bool any_matched = false; - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + /// For all right join, flag_per_row is true, we need mark used flags for each row. + if constexpr (flag_per_row) { - if ((*filter_flags)[replicated_row]) + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it->first, selected_right_row_it->second, join_features.add_missing); - total_added_rows += 1; + if ((*filter_flags)[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); + total_added_rows += 1; + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + } + ++selected_right_row_it; } - ++selected_right_row_it; - } - if (!any_matched) - { - if constexpr (join_features.is_anti_join && join_features.left) - setUsed(added_columns.filter, i - 1); - addNotFoundRow(added_columns, total_added_rows); } else { - if constexpr (join_features.is_all_join) + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - used_flags.template setUsed(find_results[i - 1]); - setUsed(added_columns.filter, left_start_row + i - 1); + if ((*filter_flags)[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; } } + if (!any_matched) + { + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if constexpr (!flag_per_row) + used_flags.template setUsed(find_results[find_result_index]); + setUsed(added_columns.filter, left_start_row + i - 1); + if constexpr (add_missing) + added_columns.applyLazyDefaults(); + } + find_result_index += (prev_replicated_row != row_replicate_offset[i]); - if constexpr (join_features.need_replication) + if constexpr (need_replication) { (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; } prev_replicated_row = row_replicate_offset[i]; } - }; while (left_row_iter < left_block_rows && !exceeded_max_block_rows) @@ -1752,10 +1770,10 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( left_row_iter, left_start_row); } - auto filter_col = buildAdditionFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); copy_final_matched_rows(left_start_row, filter_col); - if constexpr (join_features.need_replication) + if constexpr (need_replication) { // Add a check for current_added_rows to avoid run the filter expression on too small size batch. if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) @@ -1765,7 +1783,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( } } - if constexpr (join_features.need_replication) + if constexpr (need_replication) { added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); added_columns.filter.resize_assume_reserved(left_row_iter); @@ -1776,7 +1794,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template +template NO_INLINE size_t joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, @@ -1811,7 +1829,7 @@ NO_INLINE size_t joinRightColumns( bool right_row_found = false; - KnownRowsHolder known_rows; + KnownRowsHolder known_rows; for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) { const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; @@ -1834,10 +1852,10 @@ NO_INLINE size_t joinRightColumns( if (row_ref.block) { setUsed(added_columns.filter, i); - if constexpr (multiple_disjuncts) - used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); + if constexpr (flag_per_row) + used_flags.template setUsed(row_ref.block, row_ref.row_num, 0); else - used_flags.template setUsed(find_result); + used_flags.template setUsed(find_result); added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing); } @@ -1847,14 +1865,14 @@ NO_INLINE size_t joinRightColumns( else if constexpr (join_features.is_all_join) { setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); + used_flags.template setUsed(find_result); auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) { /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result); + bool used_once = used_flags.template setUsedOnce(find_result); if (used_once) { auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; @@ -1864,7 +1882,7 @@ NO_INLINE size_t joinRightColumns( } else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) { - bool used_once = used_flags.template setUsedOnce(find_result); + bool used_once = used_flags.template setUsedOnce(find_result); /// Use first appeared left key only if (used_once) @@ -1882,12 +1900,12 @@ NO_INLINE size_t joinRightColumns( else if constexpr (join_features.is_anti_join) { if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(find_result); + used_flags.template setUsed(find_result); } else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) { setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); + used_flags.template setUsed(find_result); added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing); if (join_features.is_any_or_semi_join) @@ -1922,19 +1940,46 @@ size_t joinRightColumnsSwitchMultipleDisjuncts( AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - if (added_columns.additional_filter_expression) - { - return mapv.size() > 1 ? joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - } - else + auto join_without_additional_filter = [&]() { return mapv.size() > 1 ? joinRightColumns( std::forward>(key_getter_vector), mapv, added_columns, used_flags) : joinRightColumns( std::forward>(key_getter_vector), mapv, added_columns, used_flags); + }; + + constexpr JoinFeatures join_features; + if constexpr (join_features.is_all_join) + { + if (added_columns.additional_filter_expression) + { + constexpr bool mark_per_row_used = join_features.right || join_features.full; + return mapv.size() > 1 + ? joinRightColumnsWithAddtitionalFilter< + KeyGetter, + Map, + need_filter, + join_features.need_replication, + join_features.need_flags, + join_features.add_missing, + true>(std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumnsWithAddtitionalFilter< + KeyGetter, + Map, + need_filter, + join_features.need_replication, + join_features.need_flags, + join_features.add_missing, + mark_per_row_used>(std::forward>(key_getter_vector), mapv, added_columns, used_flags); + } + else + { + return join_without_additional_filter(); + } + } + else + { + return join_without_additional_filter(); } } @@ -2421,10 +2466,10 @@ struct AdderNonJoined class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller { public: - NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool multiple_disjuncts_) + NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_, bool flag_per_row_) : parent(parent_) , max_block_size(max_block_size_) - , multiple_disjuncts(multiple_disjuncts_) + , flag_per_row(flag_per_row_) , current_block_start(0) { if (parent.data == nullptr) @@ -2451,7 +2496,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness); } - if (!multiple_disjuncts) + if (!flag_per_row) { fillNullsFromBlocks(columns_right, rows_added); } @@ -2462,7 +2507,7 @@ public: private: const HashJoin & parent; UInt64 max_block_size; - bool multiple_disjuncts; + bool flag_per_row; size_t current_block_start; @@ -2528,7 +2573,7 @@ private: { size_t rows_added = 0; - if (multiple_disjuncts) + if (flag_per_row) { if (!used_position.has_value()) used_position = parent.data->blocks.begin(); @@ -2620,8 +2665,8 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, return {}; size_t left_columns_count = left_sample_block.columns(); - bool multiple_disjuncts = !table_join->oneDisjunct(); - if (!multiple_disjuncts) + bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); + if (!flag_per_row) { /// With multiple disjuncts, all keys are in sample_block_with_columns_to_add, so invariant is not held size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns(); @@ -2633,7 +2678,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, } } - auto non_joined = std::make_unique(*this, max_block_size, multiple_disjuncts); + auto non_joined = std::make_unique(*this, max_block_size, flag_per_row); return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, *table_join); } @@ -2642,8 +2687,8 @@ void HashJoin::reuseJoinedData(const HashJoin & join) data = join.data; from_storage_join = true; - bool multiple_disjuncts = !table_join->oneDisjunct(); - if (multiple_disjuncts) + bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); + if (flag_per_row) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported"); for (auto & map : data->maps) @@ -2724,17 +2769,28 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona if (!type->equals(*std::make_shared())) { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'", - expression_sample_block.getByPosition(0).type->getName()); + "Unexpected expression in JOIN ON section. Expected boolean (UInt8), got '{}'. expression:\n{}", + expression_sample_block.getByPosition(0).type->getName(), + additional_filter_expression->dumpActions()); } - bool is_supported = (strictness == JoinStrictness::All) && (kind == JoinKind::Inner || kind == JoinKind::Left || kind == JoinKind::Right); + bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind)); if (!is_supported) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/RIGHT JOINs", + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs.", expression_sample_block.getByPosition(0).name); } } +bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const +{ + if (!table_join_->oneDisjunct()) + return true; + /// If it'a a all right join with inequal conditions, we need to mark each row + if (table_join_->getFullJoinExpression() && isRightOrFull(table_join_->kind())) + return true; + return false; +} + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 2e5c7c12323..aa9bb0a5aa1 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -9,7 +9,6 @@ #include #include -#include #include #include @@ -32,6 +31,7 @@ namespace DB { class TableJoin; +class ExpressionActions; namespace JoinStuff { @@ -61,16 +61,16 @@ public: bool getUsedSafe(size_t i) const; bool getUsedSafe(const Block * block_ptr, size_t row_idx) const; - template + template void setUsed(const T & f); - template + template void setUsed(const Block * block, size_t row_num, size_t offset); - template + template bool getUsed(const T & f); - template + template bool setUsedOnce(const T & f); }; @@ -472,7 +472,8 @@ private: bool empty() const; - void validateAdditionalFilterExpression(ExpressionActionsPtr additional_filter_expression); + void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); + bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; }; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index fec5c47c954..191e4fb1908 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include #include @@ -30,6 +28,7 @@ class ASTSelectQuery; struct DatabaseAndTableWithAlias; class Block; class DictionaryJoinAdapter; +class ExpressionActions; class StorageJoin; class StorageDictionary; class IKeyValueEntity; @@ -156,7 +155,7 @@ private: Clauses clauses; /// Originally used for inequal join. If there is no any inequal join condition, it will be nullptr. - ExpressionActionsPtr full_join_expression = nullptr; + std::shared_ptr full_join_expression = nullptr; ASTTableJoin table_join; @@ -302,8 +301,8 @@ public: std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } - const ExpressionActionsPtr & getFullJoinExpression() const { return full_join_expression; } - ExpressionActionsPtr & getFullJoinExpression() { return full_join_expression; } + const std::shared_ptr & getFullJoinExpression() const { return full_join_expression; } + std::shared_ptr & getFullJoinExpression() { return full_join_expression; } Names getAllNames(JoinTableSide side) const; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1f13bed3694..7b21cb4bef4 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1323,7 +1323,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ ExpressionActionsPtr & full_join_expression = table_join->getFullJoinExpression(); full_join_expression = std::make_shared( join_clauses_and_actions.full_join_expressions_actions, - ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::no)); + ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); } } else if (join_node.isUsingJoinExpression()) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 8e5fca418f9..8fc1a187701 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -313,10 +314,21 @@ void buildJoinClause( } else { - /// expression involves both tables. - /// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)` - const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); - join_clause.addMixedCondition(node); + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition; + if (support_mixed_join_condition) + { + /// expression involves both tables. + /// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)` + const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); + join_clause.addMixedCondition(node); + } + else + { + throw Exception( + ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression contains column from left and right table", + join_node.formatASTForErrorMessage()); + } } } @@ -326,7 +338,7 @@ void buildJoinClause( left_table_expressions, right_table_expressions, join_node); - + // expression_sides.empty() = true, the expression is constant if (expression_sides.empty() || expression_sides.size() == 1) { auto expression_side = expression_sides.empty() ? JoinTableSide::Right : *expression_sides.begin(); @@ -336,9 +348,20 @@ void buildJoinClause( } else { - /// expression involves both tables. - const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); - join_clause.addMixedCondition(node); + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition; + if (support_mixed_join_condition) + { + /// expression involves both tables. + const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); + join_clause.addMixedCondition(node); + } + else + { + throw Exception( + ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression contains column from left and right table", + join_node.formatASTForErrorMessage()); + } } } } @@ -592,6 +615,20 @@ JoinClausesAndActions buildJoinClausesAndActions( auto full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); result.full_join_expressions_actions = full_join_expressions_actions; } + auto outputs = result.full_join_expressions_actions->getOutputs(); + if (outputs.size() != 1) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Only one output is expected. but got:\n{}", result.full_join_expressions_actions->dumpDAG()); + } + auto output_type = removeNullable(outputs[0]->result_type); + WhichDataType which_type(output_type); + if (!which_type.isUInt8()) + { + DataTypePtr uint8_ty = std::make_shared(); + auto true_col = ColumnWithTypeAndName(uint8_ty->createColumnConst(1, 1), uint8_ty, "true"); + const auto * true_node = &result.full_join_expressions_actions->addColumn(true_col); + result.full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node}); + } } return result; @@ -805,10 +842,11 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo const Block & right_table_expression_header, const PlannerContextPtr & planner_context) { - if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) + if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) + && !(table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) && table_join->oneDisjunct())) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "JOIN with mixed conditions supports only hash join algorithm"); + "JOIN with mixed conditions supports only hash join or grace hash join with one disjunct."); } trySetStorageInTableJoin(right_table_expression, table_join); diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 7d3df48af8e..41736047a14 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -149,7 +149,7 @@ public: { mixed_filter_condition_nodes.push_back(condition_node); } - + const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const { return mixed_filter_condition_nodes; @@ -157,7 +157,6 @@ public: bool hasMixedFilterCondition() const { - // return has_mixed_filter_condition; return !mixed_filter_condition_nodes.empty(); } /// Dump clause into buffer diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference new file mode 100644 index 00000000000..806596f8a63 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.reference @@ -0,0 +1,366 @@ +-- { echoOn } +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='grace_hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 INNER JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +SELECT t1.*, t2.* from t1 RIGHT JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 key4 F 1 1 1 +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 A 1 2 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 c 3 2 1 key1 B 2 1 2 +key1 c 3 2 1 key1 C 3 4 5 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); + 0 0 \N key1 A 1 2 1 + 0 0 \N key3 a3 1 1 1 + 0 0 \N key4 F 1 1 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 0 0 \N +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 0 0 \N +key4 f 2 3 4 0 0 \N +SELECT t1.*, t2.* from t1 FULL JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 FULL JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +1 1 1 1 1 1 +SET join_algorithm='hash'; +SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 a 1 1 2 key3 a3 1 1 1 +key1 a 1 1 2 key4 F 1 1 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 key1 A 1 2 1 +key2 a2 1 1 1 key3 a3 1 1 1 +key2 a2 1 1 1 key4 F 1 1 1 +key4 f 2 3 4 key1 B 2 1 2 +SELECT t1.*, t2.* FROM t1 INNER JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 a 1 1 2 key3 a3 1 1 1 +key1 a 1 1 2 key4 F 1 1 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key2 a2 1 1 1 key1 A 1 2 1 +key2 a2 1 1 1 key3 a3 1 1 1 +key2 a2 1 1 1 key4 F 1 1 1 +key4 f 2 3 4 key1 B 2 1 2 +SELECT t1.*, t2.* FROM t1 RIGHT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 a 1 1 2 key3 a3 1 1 1 +key1 a 1 1 2 key4 F 1 1 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key2 a2 1 1 1 key1 A 1 2 1 +key2 a2 1 1 1 key3 a3 1 1 1 +key2 a2 1 1 1 key4 F 1 1 1 +key4 f 2 3 4 key1 B 2 1 2 +SELECT t1.*, t2.* FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +key1 a 1 1 2 key1 A 1 2 1 +key1 a 1 1 2 key1 B 2 1 2 +key1 a 1 1 2 key1 C 3 4 5 +key1 a 1 1 2 key1 D 4 1 6 +key1 a 1 1 2 key3 a3 1 1 1 +key1 a 1 1 2 key4 F 1 1 1 +key1 b 2 3 2 key1 B 2 1 2 +key1 b 2 3 2 key1 C 3 4 5 +key1 b 2 3 2 key1 D 4 1 6 +key1 c 3 2 1 key1 C 3 4 5 +key1 c 3 2 1 key1 D 4 1 6 +key1 d 4 7 2 key1 D 4 1 6 +key1 e 5 5 5 0 0 \N +key2 a2 1 1 1 key1 A 1 2 1 +key2 a2 1 1 1 key3 a3 1 1 1 +key2 a2 1 1 1 key4 F 1 1 1 +key4 f 2 3 4 key1 B 2 1 2 diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 new file mode 100644 index 00000000000..a85b7558b21 --- /dev/null +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -0,0 +1,51 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key; +INSERT INTO t1 VALUES ('key1', 'a', 1, 1, 2), ('key1', 'b', 2, 3, 2), ('key1', 'c', 3, 2, 1), ('key1', 'd', 4, 7, 2), ('key1', 'e', 5, 5, 5), ('key2', 'a2', 1, 1, 1), ('key4', 'f', 2, 3, 4); +CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64)) ENGINE = MergeTree ORDER BY key; +INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); + +SET allow_experimental_analyzer=1; +SET enable_mixed_join_condition=1; +-- { echoOn } +{% for algorithm in ['hash', 'grace_hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and t1.c ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); +{% endfor -%} +{% endfor -%} + + +{% for algorithm in ['hash'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); +{% endfor -%} +{% endfor -%} +-- { echoOff } + +-- test error messages +{% for algorithm in ['partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } +SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } +SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } +{% endfor -%} +{% endfor -%} + +{% for algorithm in ['grace_hash', 'partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%} +SET join_algorithm='{{ algorithm }}'; +{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } +{% endfor -%} +{% endfor -%} + + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; diff --git a/utils/check-style/check-large-objects.sh b/utils/check-style/check-large-objects.sh index 2122cca911e..a1e05c6ad05 100755 --- a/utils/check-style/check-large-objects.sh +++ b/utils/check-style/check-large-objects.sh @@ -9,7 +9,7 @@ TU_EXCLUDES=( Aggregator ) -if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 50000000) print }' \ +if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 100000000) print }' \ | grep -v -f <(printf "%s\n" "${TU_EXCLUDES[@]}") then echo "^ It's not allowed to have so large translation units." From c51e9163a56c6699e28d1577843900352d828e4c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 19 Mar 2024 15:04:34 +0800 Subject: [PATCH 05/11] reduce hash join code size add check debug fixed fixed fixed fixed fixed debug debug update rename full_expression rollback ExecutionContext experiment: support join_use_nulls --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 +- src/Core/Block.cpp | 12 -- src/Core/Block.h | 3 - src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/ExpressionActions.cpp | 15 ++- src/Interpreters/HashJoin.cpp | 118 ++++++++++++------ src/Interpreters/TableJoin.h | 6 +- src/Planner/PlannerJoinTree.cpp | 49 +++++++- src/Planner/PlannerJoins.cpp | 30 ++--- src/Planner/PlannerJoins.h | 2 +- ...006_join_on_inequal_expression_fast.sql.j2 | 2 +- utils/check-style/check-large-objects.sh | 2 +- 13 files changed, 161 insertions(+), 84 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a3c60c76132..f932a0de006 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3651,7 +3651,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo } } - if (join_node_in_resolve_process || !resolved_identifier) + // if (join_node_in_resolve_process || !resolved_identifier) + if (!resolved_identifier) return resolved_identifier; if (scope.join_use_nulls) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index bb8225af562..77dbad5443e 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -10,8 +10,6 @@ #include #include -#include - #include #include @@ -430,16 +428,6 @@ std::string Block::dumpIndex() const return out.str(); } -std::string Block::dumpContent() const -{ - WriteBufferFromOwnString buf; - auto output_format = Context::getGlobalContextInstance()->getOutputFormat("PrettyCompact", buf, *this); - output_format->write(materializeBlock(*this)); - output_format->flush(); - buf.finalize(); - return buf.str(); -} - Block Block::cloneEmpty() const { Block res; diff --git a/src/Core/Block.h b/src/Core/Block.h index 9c054b7aea7..c8bebb4552a 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -126,9 +126,6 @@ public: /** List of column names and positions from index */ std::string dumpIndex() const; - /// Print all the values in this block. - std::string dumpContent() const; - /** Get the same block, but empty. */ Block cloneEmpty() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1690add4e2d..616bbee131b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -321,7 +321,7 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ - M(Bool, enable_mixed_join_condition, false, "Support conditions involve columns from both tables in on join expression.", IMPORTANT) \ + M(Bool, allow_experimental_join_condition, false, "Support conditions involve columns from both tables in on join expression.", IMPORTANT) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index d2edf6ce9bd..b8ced4318d8 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -106,6 +106,7 @@ static std::map sett {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, + {"allow_experimental_join_condition", false, false, "Support conditions involve columns from both tables in on join expression."}, {"log_processors_profiles", false, true, "Enable by default"}, {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 1d193b3499c..aa2b677531a 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -18,6 +18,7 @@ #include #include #include +#include "ExpressionActions.h" #include @@ -615,12 +616,20 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon res_column.column = action.node->function->execute(arguments, res_column.type, num_rows, dry_run); if (res_column.column->getDataType() != res_column.type->getColumnType()) + { + WriteBufferFromOwnString out; + for (const auto & arg : arguments) + out << arg.dumpStructure() << ","; + throw Exception( ErrorCodes::LOGICAL_ERROR, - "Unexpected return type from {}. Expected {}. Got {}", + "Unexpected return type from {}. Expected {}. Got {}. Action:\n{},\ninput block structure:{}", action.node->function->getName(), - res_column.type->getColumnType(), - res_column.column->getDataType()); + res_column.type->getName(), //res_column.type->getColumnType(), + res_column.column->getName(), //res_column.column->getDataType(), + action.toString(), + out.str()); + } } break; } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 823c5d48db9..938f53c47e8 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -257,7 +257,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s LOG_TRACE(log, "{}Keys: {}, datatype: {}, kind: {}, strictness: {}, right header: {}", instance_log_id, TableJoin::formatClauses(table_join->getClauses(), true), data->type, kind, strictness, right_sample_block.dumpStructure()); - validateAdditionalFilterExpression(table_join->getFullJoinExpression()); + validateAdditionalFilterExpression(table_join->getMixedJoinExpression()); if (isCrossOrComma(kind)) { @@ -713,7 +713,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || isRightOrFull(kind) || multiple_disjuncts || - table_join->getFullJoinExpression(); + table_join->getMixedJoinExpression(); if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -1506,8 +1506,18 @@ ColumnPtr buildAdditionalFilter( return ColumnUInt8::create(); const Block & sample_right_block = *selected_rows.begin()->block; if (!sample_right_block) - return ColumnUInt8::create(); + { + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + return filter; + } + if (!added_columns.additional_filter_expression) + { + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + return filter; + } auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); if (required_cols.empty()) { @@ -1561,9 +1571,29 @@ ColumnPtr buildAdditionalFilter( } executed_block.insert({std::move(new_col), src_col->type, col_name}); } - LOG_TRACE(getLogger("HashJoin"), "Additional filter execute block:\n{}", executed_block.dumpContent()); + if (!executed_block) + { + WriteBufferFromOwnString buf; + for (const auto & col : required_cols) + { + buf << col.name << ", "; + } + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "required columns: {}. but not found any in left/right table. right table: {}, left table: {}", + buf.str(), + sample_right_block.dumpNames(), + added_columns.left_block.dumpNames()); + } + // Debug + for (const auto & col : executed_block.getColumnsWithTypeAndName()) + { + if (!col.column || !col.type) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Null column in input block. {}", executed_block.dumpStructure()); + } + } added_columns.additional_filter_expression->execute(executed_block); - LOG_TRACE(getLogger("HashJoin"), "Addition filter execute result block:\n{}", executed_block.dumpContent()); return executed_block.getByPosition(0).column; } @@ -1609,20 +1639,20 @@ void addFoundRowRefAll( template < typename KeyGetter, typename Map, - bool need_filter, bool need_replication, - bool need_flags, - bool add_missing, - bool flag_per_row, typename AddedColumns> NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) + JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], + bool need_filter [[maybe_unused]], + bool need_flags [[maybe_unused]], + bool add_missing [[maybe_unused]], + bool flag_per_row [[maybe_unused]]) { size_t left_block_rows = added_columns.rows_to_add; - if constexpr (need_filter) + if (need_filter) added_columns.filter = IColumn::Filter(left_block_rows, 0); std::unique_ptr pool; @@ -1661,7 +1691,8 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( break; } } - KnownRowsHolder known_rows; + KnownRowsHolder all_flag_known_rows; + KnownRowsHolder single_flag_know_rows; for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) { const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; @@ -1677,7 +1708,10 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { auto & mapped = find_result.getMapped(); find_results.push_back(find_result); - addFoundRowRefAll(mapped, selected_rows, current_added_rows, known_rows); + if (flag_per_row) + addFoundRowRefAll(mapped, selected_rows, current_added_rows, all_flag_known_rows); + else + addFoundRowRefAll(mapped, selected_rows, current_added_rows, single_flag_know_rows); } } row_replicate_offset.push_back(current_added_rows); @@ -1705,7 +1739,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { bool any_matched = false; /// For all right join, flag_per_row is true, we need mark used flags for each row. - if constexpr (flag_per_row) + if (flag_per_row) { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { @@ -1714,7 +1748,8 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( any_matched = true; added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); total_added_rows += 1; - used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); + if (need_flags) + used_flags.template setUsed(selected_right_row_it->block, selected_right_row_it->row_num, 0); } ++selected_right_row_it; } @@ -1734,14 +1769,18 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( } if (!any_matched) { - addNotFoundRow(added_columns, total_added_rows); + if (add_missing) + addNotFoundRow(added_columns, total_added_rows); + else + addNotFoundRow(added_columns, total_added_rows); } else { - if constexpr (!flag_per_row) - used_flags.template setUsed(find_results[find_result_index]); - setUsed(added_columns.filter, left_start_row + i - 1); - if constexpr (add_missing) + if (!flag_per_row && need_flags) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if (add_missing) added_columns.applyLazyDefaults(); } find_result_index += (prev_replicated_row != row_replicate_offset[i]); @@ -1954,23 +1993,24 @@ size_t joinRightColumnsSwitchMultipleDisjuncts( if (added_columns.additional_filter_expression) { constexpr bool mark_per_row_used = join_features.right || join_features.full; - return mapv.size() > 1 - ? joinRightColumnsWithAddtitionalFilter< - KeyGetter, - Map, - need_filter, - join_features.need_replication, - join_features.need_flags, - join_features.add_missing, - true>(std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumnsWithAddtitionalFilter< - KeyGetter, - Map, - need_filter, - join_features.need_replication, - join_features.need_flags, - join_features.add_missing, - mark_per_row_used>(std::forward>(key_getter_vector), mapv, added_columns, used_flags); + return mapv.size() > 1 ? joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), + mapv, + added_columns, + used_flags, + need_filter, + join_features.need_flags, + join_features.add_missing, + true) + : joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), + mapv, + added_columns, + used_flags, + need_filter, + join_features.need_flags, + join_features.add_missing, + mark_per_row_used); } else { @@ -2141,7 +2181,7 @@ Block HashJoin::joinBlockImpl( savedBlockSample(), *this, std::move(join_on_keys), - table_join->getFullJoinExpression(), + table_join->getMixedJoinExpression(), join_features.is_asof_join, is_join_get); @@ -2788,7 +2828,7 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table if (!table_join_->oneDisjunct()) return true; /// If it'a a all right join with inequal conditions, we need to mark each row - if (table_join_->getFullJoinExpression() && isRightOrFull(table_join_->kind())) + if (table_join_->getMixedJoinExpression() && isRightOrFull(table_join_->kind())) return true; return false; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 191e4fb1908..1d65330da61 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -155,7 +155,7 @@ private: Clauses clauses; /// Originally used for inequal join. If there is no any inequal join condition, it will be nullptr. - std::shared_ptr full_join_expression = nullptr; + std::shared_ptr mixed_join_expression = nullptr; ASTTableJoin table_join; @@ -301,8 +301,8 @@ public: std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } - const std::shared_ptr & getFullJoinExpression() const { return full_join_expression; } - std::shared_ptr & getFullJoinExpression() { return full_join_expression; } + const std::shared_ptr & getMixedJoinExpression() const { return mixed_join_expression; } + std::shared_ptr & getMixedJoinExpression() { return mixed_join_expression; } Names getAllNames(JoinTableSide side) const; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 7b21cb4bef4..81ba47596c8 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1042,6 +1042,31 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres }; } +static ColumnsWithTypeAndName joinCastPlanColumnsToNullable(const ColumnsWithTypeAndName & cols, PlannerContextPtr & planner_context) +{ + ColumnsWithTypeAndName res; + for (const auto & col : cols) + { + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(col.name)) + { + DataTypePtr type_to_check = col.type; + if (const auto * type_to_check_low_cardinality = typeid_cast(type_to_check.get())) + type_to_check = type_to_check_low_cardinality->getDictionaryType(); + + if (type_to_check->canBeInsideNullable()) + { + type_to_check = makeNullable(type_to_check); + } + res.push_back(ColumnWithTypeAndName(type_to_check->createColumn(), type_to_check, col.name)); + } + else + { + res.push_back(col); + } + } + return res; +} + void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); @@ -1112,6 +1137,22 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (!join_constant && join_node.isOnJoinExpression()) { + if (planner_context->getQueryContext()->getSettingsRef().join_use_nulls) + { + if (join_kind == JoinKind::Full) + { + left_plan_output_columns = joinCastPlanColumnsToNullable(left_plan_output_columns, planner_context); + right_plan_output_columns = joinCastPlanColumnsToNullable(right_plan_output_columns, planner_context); + } + else if (join_kind == JoinKind::Left) + { + right_plan_output_columns = joinCastPlanColumnsToNullable(right_plan_output_columns, planner_context); + } + else if (join_kind == JoinKind::Right) + { + left_plan_output_columns = joinCastPlanColumnsToNullable(left_plan_output_columns, planner_context); + } + } join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, join_table_expression, @@ -1318,11 +1359,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } } - if (join_clauses_and_actions.full_join_expressions_actions) + if (join_clauses_and_actions.mixed_join_expressions_actions) { - ExpressionActionsPtr & full_join_expression = table_join->getFullJoinExpression(); - full_join_expression = std::make_shared( - join_clauses_and_actions.full_join_expressions_actions, + ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); + mixed_join_expression = std::make_shared( + join_clauses_and_actions.mixed_join_expressions_actions, ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); } } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 8fc1a187701..6078df01506 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -314,7 +314,7 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition; + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; if (support_mixed_join_condition) { /// expression involves both tables. @@ -348,7 +348,7 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().enable_mixed_join_condition; + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; if (support_mixed_join_condition) { /// expression involves both tables. @@ -596,29 +596,29 @@ JoinClausesAndActions buildJoinClausesAndActions( { if (result.join_clauses.size() > 1) { - auto full_join_expressions_actions = std::make_shared(mixed_table_expression_columns); + auto mixed_join_expressions_actions = std::make_shared(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(full_join_expressions_actions, join_expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_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()); - full_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); + mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; - full_join_expressions_actions->removeUnusedActions(required_names); - result.full_join_expressions_actions = full_join_expressions_actions; + mixed_join_expressions_actions->removeUnusedActions(required_names); + result.mixed_join_expressions_actions = mixed_join_expressions_actions; } else { const auto & join_clause = result.join_clauses.front(); const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes(); - auto full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); - result.full_join_expressions_actions = full_join_expressions_actions; + auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); + result.mixed_join_expressions_actions = mixed_join_expressions_actions; } - auto outputs = result.full_join_expressions_actions->getOutputs(); + auto outputs = result.mixed_join_expressions_actions->getOutputs(); if (outputs.size() != 1) { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Only one output is expected. but got:\n{}", result.full_join_expressions_actions->dumpDAG()); + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Only one output is expected. but got:\n{}", result.mixed_join_expressions_actions->dumpDAG()); } auto output_type = removeNullable(outputs[0]->result_type); WhichDataType which_type(output_type); @@ -626,8 +626,8 @@ JoinClausesAndActions buildJoinClausesAndActions( { DataTypePtr uint8_ty = std::make_shared(); auto true_col = ColumnWithTypeAndName(uint8_ty->createColumnConst(1, 1), uint8_ty, "true"); - const auto * true_node = &result.full_join_expressions_actions->addColumn(true_col); - result.full_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node}); + const auto * true_node = &result.mixed_join_expressions_actions->addColumn(true_col); + result.mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node}); } } @@ -842,8 +842,8 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo const Block & right_table_expression_header, const PlannerContextPtr & planner_context) { - if (table_join->getFullJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) - && !(table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) && table_join->oneDisjunct())) + if (table_join->getMixedJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) + && !table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH)) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "JOIN with mixed conditions supports only hash join or grace hash join with one disjunct."); diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index 41736047a14..c96941a3c16 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -194,7 +194,7 @@ struct JoinClausesAndActions ActionsDAGPtr 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. - ActionsDAGPtr full_join_expressions_actions; + ActionsDAGPtr mixed_join_expressions_actions; }; /** Calculate join clauses and actions for JOIN ON section. diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index a85b7558b21..6084e6dbc76 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -7,7 +7,7 @@ CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64) INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); SET allow_experimental_analyzer=1; -SET enable_mixed_join_condition=1; +SET allow_experimental_join_condition=1; -- { echoOn } {% for algorithm in ['hash', 'grace_hash'] -%} SET join_algorithm='{{ algorithm }}'; diff --git a/utils/check-style/check-large-objects.sh b/utils/check-style/check-large-objects.sh index a1e05c6ad05..2122cca911e 100755 --- a/utils/check-style/check-large-objects.sh +++ b/utils/check-style/check-large-objects.sh @@ -9,7 +9,7 @@ TU_EXCLUDES=( Aggregator ) -if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 100000000) print }' \ +if find $1 -name '*.o' | xargs wc -c | grep --regexp='\.o$' | sort -rn | awk '{ if ($1 > 50000000) print }' \ | grep -v -f <(printf "%s\n" "${TU_EXCLUDES[@]}") then echo "^ It's not allowed to have so large translation units." From 6c30774ce580351ffeb9985ea23dffdfc6d192e7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 Apr 2024 08:22:43 +0800 Subject: [PATCH 06/11] disable when join_use_nulls=1 update setting update --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Planner/PlannerJoinTree.cpp | 41 ------------------- src/Planner/PlannerJoins.cpp | 12 ++++-- ...006_join_on_inequal_expression_fast.sql.j2 | 3 +- 6 files changed, 13 insertions(+), 50 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index f932a0de006..a3c60c76132 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3651,8 +3651,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo } } - // if (join_node_in_resolve_process || !resolved_identifier) - if (!resolved_identifier) + if (join_node_in_resolve_process || !resolved_identifier) return resolved_identifier; if (scope.join_use_nulls) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 616bbee131b..6618e595974 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -321,7 +321,7 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ - M(Bool, allow_experimental_join_condition, false, "Support conditions involve columns from both tables in on join expression.", IMPORTANT) \ + M(Bool, allow_mixed_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", IMPORTANT) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b8ced4318d8..397ada59986 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,6 +90,7 @@ static std::map sett {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, + {"allow_mixed_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, {"first_day_of_week", "Monday", "Monday", "Added a setting for the first day of the week for date/time functions"}, {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, @@ -106,7 +107,6 @@ static std::map sett {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"allow_experimental_join_condition", false, false, "Support conditions involve columns from both tables in on join expression."}, {"log_processors_profiles", false, true, "Enable by default"}, {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 81ba47596c8..4e97a3125d4 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1042,31 +1042,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres }; } -static ColumnsWithTypeAndName joinCastPlanColumnsToNullable(const ColumnsWithTypeAndName & cols, PlannerContextPtr & planner_context) -{ - ColumnsWithTypeAndName res; - for (const auto & col : cols) - { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(col.name)) - { - DataTypePtr type_to_check = col.type; - if (const auto * type_to_check_low_cardinality = typeid_cast(type_to_check.get())) - type_to_check = type_to_check_low_cardinality->getDictionaryType(); - - if (type_to_check->canBeInsideNullable()) - { - type_to_check = makeNullable(type_to_check); - } - res.push_back(ColumnWithTypeAndName(type_to_check->createColumn(), type_to_check, col.name)); - } - else - { - res.push_back(col); - } - } - return res; -} - void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) { auto cast_actions_dag = std::make_shared(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); @@ -1137,22 +1112,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (!join_constant && join_node.isOnJoinExpression()) { - if (planner_context->getQueryContext()->getSettingsRef().join_use_nulls) - { - if (join_kind == JoinKind::Full) - { - left_plan_output_columns = joinCastPlanColumnsToNullable(left_plan_output_columns, planner_context); - right_plan_output_columns = joinCastPlanColumnsToNullable(right_plan_output_columns, planner_context); - } - else if (join_kind == JoinKind::Left) - { - right_plan_output_columns = joinCastPlanColumnsToNullable(right_plan_output_columns, planner_context); - } - else if (join_kind == JoinKind::Right) - { - left_plan_output_columns = joinCastPlanColumnsToNullable(left_plan_output_columns, planner_context); - } - } join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, join_table_expression, diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 6078df01506..197edde97be 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -314,8 +314,10 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; - if (support_mixed_join_condition) + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_mixed_join_condition; + auto join_use_nulls = planner_context->getQueryContext()->getSettingsRef().join_use_nulls; + /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not right. + if (support_mixed_join_condition && !join_use_nulls) { /// expression involves both tables. /// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)` @@ -348,8 +350,10 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; - if (support_mixed_join_condition) + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_mixed_join_condition; + auto join_use_nulls = planner_context->getQueryContext()->getSettingsRef().join_use_nulls; + /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not right. + if (support_mixed_join_condition && !join_use_nulls) { /// expression involves both tables. const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 6084e6dbc76..89df825b32b 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -7,7 +7,8 @@ CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64) INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); SET allow_experimental_analyzer=1; -SET allow_experimental_join_condition=1; +SET allow_mixed_join_condition=1; +SET join_use_nulls=0; -- { echoOn } {% for algorithm in ['hash', 'grace_hash'] -%} SET join_algorithm='{{ algorithm }}'; From 7ee720ffb00c8626215cab8acb21556f4db74b1e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 25 Apr 2024 14:43:05 +0800 Subject: [PATCH 07/11] update settting --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Interpreters/CollectJoinOnKeysVisitor.h | 1 + src/Planner/PlannerJoins.cpp | 4 ++-- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6618e595974..6454d60c15e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -321,7 +321,7 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ - M(Bool, allow_mixed_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", IMPORTANT) \ + M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", IMPORTANT) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 397ada59986..23192ba7b21 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,12 +90,12 @@ static std::map sett {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"allow_mixed_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, {"first_day_of_week", "Monday", "Monday", "Added a setting for the first day of the week for date/time functions"}, {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, + {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 7218f96d781..ff10db3beb5 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -8,6 +8,7 @@ #include #include + namespace DB { diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 197edde97be..930881ec7d1 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -314,7 +314,7 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_mixed_join_condition; + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; auto join_use_nulls = planner_context->getQueryContext()->getSettingsRef().join_use_nulls; /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not right. if (support_mixed_join_condition && !join_use_nulls) @@ -350,7 +350,7 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_mixed_join_condition; + auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef().allow_experimental_join_condition; auto join_use_nulls = planner_context->getQueryContext()->getSettingsRef().join_use_nulls; /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not right. if (support_mixed_join_condition && !join_use_nulls) From 6fa6c0261b41ce63091b9ef353de14823469c5a8 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 25 Apr 2024 16:20:27 +0800 Subject: [PATCH 08/11] apply 27a2b19 --- .../sql-reference/statements/select/join.md | 23 ++- src/Interpreters/ExpressionActions.cpp | 11 +- src/Interpreters/HashJoin.cpp | 164 +++++++----------- src/Planner/PlannerJoins.cpp | 14 +- src/Planner/PlannerJoins.h | 4 - ...006_join_on_inequal_expression_fast.sql.j2 | 2 +- 6 files changed, 96 insertions(+), 122 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 141bb80ceb4..4ef407a4d13 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -164,12 +164,26 @@ Result: │ 4 │ -4 │ 4 │ └───┴────┴─────┘ ``` -## Join with inequality conditions -Clickhouse currently supports inner, left, right and full join with inequality conditions, including with `OR` operator. You need to set `allow_experimental_analyzer = 1` and select `hash` or `grace_hash` join algorithm. + +## [experimental] Join with inequality conditions + +:::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`. + +::: + +Clickhouse currently supports `ALL 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`. **Example** Table `t1`: + ``` ┌─key──┬─attr─┬─a─┬─b─┬─c─┐ │ key1 │ a │ 1 │ 1 │ 2 │ @@ -183,6 +197,7 @@ Table `t1`: ``` Table `t2` + ``` ┌─key──┬─attr─┬─a─┬─b─┬─c─┐ │ key1 │ A │ 1 │ 2 │ 1 │ @@ -193,9 +208,11 @@ Table `t2` │ key4 │ F │ 1 │ 1 │ 1 │ └──────┴──────┴───┴───┴───┘ ``` + ```sql SELECT t1.*, t2.* from t1 LEFT JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); ``` + ``` key1 a 1 1 2 key1 B 2 1 2 key1 a 1 1 2 key1 C 3 4 5 @@ -318,7 +335,7 @@ For example, consider the following tables: ## PASTE JOIN Usage The result of `PASTE JOIN` is a table that contains all columns from left subquery followed by all columns from the right subquery. -The rows are matched based on their positions in the original tables (the order of rows should be defined). +The rows are matched based on their positions in the original tables (the order of rows should be defined). If the subqueries return a different number of rows, extra rows will be cut. Example: diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index aa2b677531a..04f29f35c3c 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -18,7 +18,6 @@ #include #include #include -#include "ExpressionActions.h" #include @@ -617,18 +616,14 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon res_column.column = action.node->function->execute(arguments, res_column.type, num_rows, dry_run); if (res_column.column->getDataType() != res_column.type->getColumnType()) { - WriteBufferFromOwnString out; - for (const auto & arg : arguments) - out << arg.dumpStructure() << ","; - throw Exception( ErrorCodes::LOGICAL_ERROR, "Unexpected return type from {}. Expected {}. Got {}. Action:\n{},\ninput block structure:{}", action.node->function->getName(), - res_column.type->getName(), //res_column.type->getColumnType(), - res_column.column->getName(), //res_column.column->getDataType(), + res_column.type->getName(), + res_column.column->getName(), action.toString(), - out.str()); + Block(arguments).dumpStructure()); } } break; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 938f53c47e8..56955066191 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1573,67 +1573,58 @@ ColumnPtr buildAdditionalFilter( } if (!executed_block) { - WriteBufferFromOwnString buf; - for (const auto & col : required_cols) - { - buf << col.name << ", "; - } throw Exception( ErrorCodes::LOGICAL_ERROR, - "required columns: {}. but not found any in left/right table. right table: {}, left table: {}", - buf.str(), + "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", + required_cols.toString(), sample_right_block.dumpNames(), added_columns.left_block.dumpNames()); } - // Debug + for (const auto & col : executed_block.getColumnsWithTypeAndName()) { if (!col.column || !col.type) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Null column in input block. {}", executed_block.dumpStructure()); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); } + added_columns.additional_filter_expression->execute(executed_block); - return executed_block.getByPosition(0).column; + + ColumnPtr result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); + executed_block.clear(); + + if (result_column->isNullable()) + { + /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros + /// Trying to avoid copying data, since we are the only owner of the column. + ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); + + MutableColumnPtr mutable_column; + { + ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); + result_column.reset(); + mutable_column = IColumn::mutate(std::move(nested_column)); + } + + auto & column_data = assert_cast(*mutable_column).getData(); + const auto & mask_column_data = assert_cast(*mask_column).getData(); + for (size_t i = 0; i < column_data.size(); ++i) + { + if (mask_column_data[i]) + column_data[i] = 0; + } + return mutable_column; + } + return result_column; } -template -void addFoundRowRefAll( - const RowRefList & row_list, - std::vector & selected_rows, - IColumn::Offset & current_offset, - KnownRowsHolder & known_rows [[maybe_unused]]) +/// Adapter class to pass into addFoundRowAll +/// In joinRightColumnsWithAdditionalFilter we don't want to add rows directly into AddedColumns, +/// because they need to be filtered by additional_filter_expression. +class PreSelectedRows : public std::vector { - if constexpr (flag_per_row) - { - std::unique_ptr::Type>> new_known_rows_ptr; - for (auto it = row_list.begin(); it.ok(); ++it) - { - auto row_ref = std::make_pair(it->block, it->row_num); - if (!known_rows.isKnown(row_ref)) - { - selected_rows.emplace_back(row_ref.first, row_ref.second); - ++current_offset; - if (!new_known_rows_ptr) - { - new_known_rows_ptr = std::make_unique::Type>>(); - } - new_known_rows_ptr->push_back(row_ref); - } - } - - if (new_known_rows_ptr) - known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr)); - } - else - { - for (auto it = row_list.begin(); it.ok(); ++it) - { - selected_rows.emplace_back(it->block, it->row_num); - ++current_offset; - } - } -} +public: + void appendFromBlock(const Block & block, size_t row_num, bool /* has_default */) { this->emplace_back(&block, row_num); } +}; /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template < @@ -1666,7 +1657,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( using FindResult = typename KeyGetter::FindResult; size_t max_joined_block_rows = added_columns.max_joined_block_rows; size_t left_row_iter = 0; - std::vector selected_rows; + PreSelectedRows selected_rows; selected_rows.reserve(left_block_rows); std::vector find_results; find_results.reserve(left_block_rows); @@ -1709,9 +1700,9 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( auto & mapped = find_result.getMapped(); find_results.push_back(find_result); if (flag_per_row) - addFoundRowRefAll(mapped, selected_rows, current_added_rows, all_flag_known_rows); + addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); else - addFoundRowRefAll(mapped, selected_rows, current_added_rows, single_flag_know_rows); + addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); } } row_replicate_offset.push_back(current_added_rows); @@ -1720,17 +1711,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) { - const PaddedPODArray * filter_flags = nullptr; - filter_col = filter_col->convertToFullIfNeeded(); - if (filter_col->isNullable()) - { - auto nested_col = typeid_cast(*filter_col).getNestedColumnPtr(); - filter_flags = &(dynamic_cast(*nested_col).getData()); - } - else - { - filter_flags = &(dynamic_cast(*filter_col).getData()); - } + const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); size_t prev_replicated_row = 0; auto selected_right_row_it = selected_rows.begin(); @@ -1743,7 +1724,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - if ((*filter_flags)[replicated_row]) + if (filter_flags[replicated_row]) { any_matched = true; added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); @@ -1758,7 +1739,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( { for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) { - if ((*filter_flags)[replicated_row]) + if (filter_flags[replicated_row]) { any_matched = true; added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing); @@ -1979,48 +1960,30 @@ size_t joinRightColumnsSwitchMultipleDisjuncts( AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - auto join_without_additional_filter = [&]() - { - return mapv.size() > 1 ? joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags) - : joinRightColumns( - std::forward>(key_getter_vector), mapv, added_columns, used_flags); - }; - constexpr JoinFeatures join_features; if constexpr (join_features.is_all_join) { if (added_columns.additional_filter_expression) { - constexpr bool mark_per_row_used = join_features.right || join_features.full; - return mapv.size() > 1 ? joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - join_features.need_flags, - join_features.add_missing, - true) - : joinRightColumnsWithAddtitionalFilter( - std::forward>(key_getter_vector), - mapv, - added_columns, - used_flags, - need_filter, - join_features.need_flags, - join_features.add_missing, - mark_per_row_used); - } - else - { - return join_without_additional_filter(); + bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; + return joinRightColumnsWithAddtitionalFilter( + std::forward>(key_getter_vector), + mapv, + added_columns, + used_flags, + need_filter, + join_features.need_flags, + join_features.add_missing, + mark_per_row_used); } } - else - { - return join_without_additional_filter(); - } + + if (added_columns.additional_filter_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN"); + + return mapv.size() > 1 + ? joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags); } template @@ -2796,6 +2759,7 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona { if (!additional_filter_expression) return; + Block expression_sample_block = additional_filter_expression->getSampleBlock(); if (expression_sample_block.columns() != 1) @@ -2818,7 +2782,7 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona if (!is_supported) { throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs.", + "Non equi condition '{}' from JOIN ON section is supported only for ALL INNER/LEFT/FULL/RIGHT JOINs", expression_sample_block.getByPosition(0).name); } } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 930881ec7d1..1fdf51f399f 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -459,7 +459,7 @@ JoinClausesAndActions buildJoinClausesAndActions( join_right_table_expressions, join_node, result.join_clauses.back()); - is_inequal_join |= result.join_clauses.back().hasMixedFilterCondition(); + is_inequal_join |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); } } else @@ -476,7 +476,7 @@ JoinClausesAndActions buildJoinClausesAndActions( join_right_table_expressions, join_node, result.join_clauses.back()); - is_inequal_join |= result.join_clauses.back().hasMixedFilterCondition(); + is_inequal_join |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); } auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext()); @@ -595,9 +595,10 @@ JoinClausesAndActions buildJoinClausesAndActions( result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); - /// If there is any inequal join condition, we need to build full join expressions actions. if (is_inequal_join) { + /// In case of multiple disjuncts and any inequal join condition, we need to build full join on expression actions. + /// 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) { auto mixed_join_expressions_actions = std::make_shared(mixed_table_expression_columns); @@ -622,7 +623,7 @@ JoinClausesAndActions buildJoinClausesAndActions( auto outputs = result.mixed_join_expressions_actions->getOutputs(); if (outputs.size() != 1) { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Only one output is expected. but got:\n{}", result.mixed_join_expressions_actions->dumpDAG()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one output is expected, got: {}", result.mixed_join_expressions_actions->dumpDAG()); } auto output_type = removeNullable(outputs[0]->result_type); WhichDataType which_type(output_type); @@ -846,11 +847,12 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo const Block & right_table_expression_header, const PlannerContextPtr & planner_context) { - if (table_join->getMixedJoinExpression() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) + if (table_join->getMixedJoinExpression() + && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) && !table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH)) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "JOIN with mixed conditions supports only hash join or grace hash join with one disjunct."); + "JOIN with mixed conditions supports only hash join or grace hash join"); } trySetStorageInTableJoin(right_table_expression, table_join); diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index c96941a3c16..8adf6edd7ea 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -155,10 +155,6 @@ public: return mixed_filter_condition_nodes; } - bool hasMixedFilterCondition() const - { - return !mixed_filter_condition_nodes.empty(); - } /// Dump clause into buffer void dump(WriteBuffer & buffer) const; diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index 89df825b32b..d3aa74f5c38 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -7,7 +7,7 @@ CREATE TABLE t2 (key String, attr String, a UInt64, b UInt64, c Nullable(UInt64) INSERT INTO t2 VALUES ('key1', 'A', 1, 2, 1), ('key1', 'B', 2, 1, 2), ('key1', 'C', 3, 4, 5), ('key1', 'D', 4, 1, 6), ('key3', 'a3', 1, 1, 1), ('key4', 'F', 1,1,1); SET allow_experimental_analyzer=1; -SET allow_mixed_join_condition=1; +SET allow_experimental_join_condition=1; SET join_use_nulls=0; -- { echoOn } {% for algorithm in ['hash', 'grace_hash'] -%} From 5431e272c950b4bfc4b35175a60577fef7ae47ce Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 25 Apr 2024 16:39:26 +0800 Subject: [PATCH 09/11] update --- src/Interpreters/HashJoin.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 56955066191..fc211ab5a83 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,8 +1,6 @@ #include #include -#include #include -#include #include #include @@ -1505,19 +1503,13 @@ ColumnPtr buildAdditionalFilter( if (selected_rows.empty()) return ColumnUInt8::create(); const Block & sample_right_block = *selected_rows.begin()->block; - if (!sample_right_block) + if (!sample_right_block || !added_columns.additional_filter_expression) { auto filter = ColumnUInt8::create(); filter->insertMany(1, selected_rows.size()); return filter; } - if (!added_columns.additional_filter_expression) - { - auto filter = ColumnUInt8::create(); - filter->insertMany(1, selected_rows.size()); - return filter; - } auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); if (required_cols.empty()) { From 352d7a443b1e37205cba8fb256ec0ab770745aaa Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 25 Apr 2024 18:15:10 +0800 Subject: [PATCH 10/11] fixed: filter column is a const column --- src/Interpreters/HashJoin.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fc211ab5a83..f8dc7af3264 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1783,6 +1783,7 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( left_start_row); } auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + filter_col = filter_col->convertToFullIfNeeded(); copy_final_matched_rows(left_start_row, filter_col); if constexpr (need_replication) From 0070d6fc71ab630d4cb288ff05c7e5640a61b602 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 26 Apr 2024 09:35:29 +0800 Subject: [PATCH 11/11] fixed: converting filter column type --- src/Interpreters/HashJoin.cpp | 155 +++++++++++++++++----------------- 1 file changed, 79 insertions(+), 76 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f8dc7af3264..f016b222f1b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1500,90 +1500,94 @@ ColumnPtr buildAdditionalFilter( const std::vector & row_replicate_offset, AddedColumns & added_columns) { - if (selected_rows.empty()) - return ColumnUInt8::create(); - const Block & sample_right_block = *selected_rows.begin()->block; - if (!sample_right_block || !added_columns.additional_filter_expression) + ColumnPtr result_column; + do { - auto filter = ColumnUInt8::create(); - filter->insertMany(1, selected_rows.size()); - return filter; - } - - auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); - if (required_cols.empty()) - { - Block block; - added_columns.additional_filter_expression->execute(block); - return block.getByPosition(0).column->cloneResized(selected_rows.size()); - } - NameSet required_column_names; - for (auto & col : required_cols) - { - required_column_names.insert(col.name); - } - - Block executed_block; - size_t right_col_pos = 0; - for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) - { - if (required_column_names.contains(col.name)) + if (selected_rows.empty()) { - auto new_col = col.column->cloneEmpty(); - for (const auto & selected_row : selected_rows) - { - const auto & src_col = selected_row.block->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row.row_num); - } - executed_block.insert({std::move(new_col), col.type, col.name}); + result_column = ColumnUInt8::create(); + break; } - right_col_pos += 1; - } - if (!executed_block) - { - return ColumnUInt8::create(); - } - - for (const auto & col_name : required_column_names) - { - const auto * src_col = added_columns.left_block.findByName(col_name); - if (!src_col) - continue; - auto new_col = src_col->column->cloneEmpty(); - size_t prev_left_offset = 0; - for (size_t i = 1; i < row_replicate_offset.size(); ++i) + const Block & sample_right_block = *selected_rows.begin()->block; + if (!sample_right_block || !added_columns.additional_filter_expression) { - const size_t & left_offset = row_replicate_offset[i]; - size_t rows = left_offset - prev_left_offset; - if (rows) - { - new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); - } - prev_left_offset = left_offset; + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + result_column = std::move(filter); + break; } - executed_block.insert({std::move(new_col), src_col->type, col_name}); - } - if (!executed_block) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", - required_cols.toString(), - sample_right_block.dumpNames(), - added_columns.left_block.dumpNames()); - } - for (const auto & col : executed_block.getColumnsWithTypeAndName()) - { - if (!col.column || !col.type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); - } + auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + if (required_cols.empty()) + { + Block block; + added_columns.additional_filter_expression->execute(block); + result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); + break; + } + NameSet required_column_names; + for (auto & col : required_cols) + required_column_names.insert(col.name); - added_columns.additional_filter_expression->execute(executed_block); + Block executed_block; + size_t right_col_pos = 0; + for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) + { + if (required_column_names.contains(col.name)) + { + auto new_col = col.column->cloneEmpty(); + for (const auto & selected_row : selected_rows) + { + const auto & src_col = selected_row.block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row.row_num); + } + executed_block.insert({std::move(new_col), col.type, col.name}); + } + right_col_pos += 1; + } + if (!executed_block) + { + result_column = ColumnUInt8::create(); + break; + } - ColumnPtr result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); - executed_block.clear(); + for (const auto & col_name : required_column_names) + { + const auto * src_col = added_columns.left_block.findByName(col_name); + if (!src_col) + continue; + auto new_col = src_col->column->cloneEmpty(); + size_t prev_left_offset = 0; + for (size_t i = 1; i < row_replicate_offset.size(); ++i) + { + const size_t & left_offset = row_replicate_offset[i]; + size_t rows = left_offset - prev_left_offset; + if (rows) + new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); + prev_left_offset = left_offset; + } + executed_block.insert({std::move(new_col), src_col->type, col_name}); + } + if (!executed_block) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", + required_cols.toString(), + sample_right_block.dumpNames(), + added_columns.left_block.dumpNames()); + } + for (const auto & col : executed_block.getColumnsWithTypeAndName()) + if (!col.column || !col.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); + + added_columns.additional_filter_expression->execute(executed_block); + result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); + executed_block.clear(); + } while (false); + + result_column = result_column->convertToFullIfNeeded(); if (result_column->isNullable()) { /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros @@ -1783,7 +1787,6 @@ NO_INLINE size_t joinRightColumnsWithAddtitionalFilter( left_start_row); } auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); - filter_col = filter_col->convertToFullIfNeeded(); copy_final_matched_rows(left_start_row, filter_col); if constexpr (need_replication)