diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 9a67ced4cf..98ce9d9950 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -60,6 +60,7 @@ #include "util/runtime_profile.h" #include "vec/core/block.h" #include "vec/exec/join/vhash_join_node.h" +#include "vec/exec/join/vnested_loop_join_node.h" #include "vec/exec/scan/new_es_scan_node.h" #include "vec/exec/scan/new_file_scan_node.h" #include "vec/exec/scan/new_jdbc_scan_node.h" @@ -75,7 +76,6 @@ #include "vec/exec/vexchange_node.h" #include "vec/exec/vintersect_node.h" #include "vec/exec/vmysql_scan_node.h" -#include "vec/exec/vnested_loop_join_node.h" #include "vec/exec/vrepeat_node.h" #include "vec/exec/vschema_scan_node.h" #include "vec/exec/vselect_node.h" @@ -221,7 +221,7 @@ Status ExecNode::prepare(RuntimeState* state) { SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); if (_vconjunct_ctx_ptr) { - RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->prepare(state, _row_descriptor)); + RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->prepare(state, intermediate_row_desc())); } // For vectorized olap scan node, the conjuncts is prepared in _vconjunct_ctx_ptr. @@ -230,7 +230,7 @@ Status ExecNode::prepare(RuntimeState* state) { if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { RETURN_IF_ERROR(Expr::prepare(_conjunct_ctxs, state, _row_descriptor)); } - RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, _row_descriptor)); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, intermediate_row_desc())); for (int i = 0; i < _children.size(); ++i) { RETURN_IF_ERROR(_children[i]->prepare(state)); diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index 296726b924..62a6a0ccae 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -185,6 +185,7 @@ public: virtual const RowDescriptor& row_desc() const { return _output_row_descriptor ? *_output_row_descriptor : _row_descriptor; } + virtual const RowDescriptor& intermediate_row_desc() const { return _row_descriptor; } int64_t rows_returned() const { return _num_rows_returned; } int64_t limit() const { return _limit; } bool reached_limit() const { return _limit != -1 && _num_rows_returned >= _limit; } diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 02bef14dd2..991b2d1217 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -95,7 +95,6 @@ set(VEC_FILES exec/vintersect_node.cpp exec/vexcept_node.cpp exec/vselect_node.cpp - exec/vnested_loop_join_node.cpp exec/vmysql_scan_node.cpp exec/vschema_scan_node.cpp exec/vempty_set_node.cpp @@ -110,6 +109,8 @@ set(VEC_FILES exec/vparquet_scanner.cpp exec/vorc_scanner.cpp exec/join/vhash_join_node.cpp + exec/join/vjoin_node_base.cpp + exec/join/vnested_loop_join_node.cpp exec/data_gen_functions/vnumbers_tvf.cpp exec/vdata_gen_scan_node.cpp exprs/vectorized_agg_fn.cpp diff --git a/be/src/vec/exec/join/vhash_join_node.cpp b/be/src/vec/exec/join/vhash_join_node.cpp index f6a8db2e2f..5bbf0905cb 100644 --- a/be/src/vec/exec/join/vhash_join_node.cpp +++ b/be/src/vec/exec/join/vhash_join_node.cpp @@ -17,10 +17,9 @@ #include "vec/exec/join/vhash_join_node.h" +#include "exprs/runtime_filter_slots.h" #include "gen_cpp/PlanNodes_types.h" #include "gutil/strings/substitute.h" -#include "runtime/memory/mem_tracker.h" -#include "runtime/runtime_filter_mgr.h" #include "util/defer_op.h" #include "vec/data_types/data_type_number.h" #include "vec/exprs/vexpr.h" @@ -867,32 +866,15 @@ Status ProcessHashTableProbe::process_data_in_hashtable(HashTableTyp } HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ExecNode(pool, tnode, descs), - _join_op(tnode.hash_join_node.join_op), + : VJoinNodeBase(pool, tnode, descs), _mem_used(0), - _have_other_join_conjunct(tnode.hash_join_node.__isset.vother_join_conjunct), - _match_all_probe(_join_op == TJoinOp::LEFT_OUTER_JOIN || - _join_op == TJoinOp::FULL_OUTER_JOIN), - _match_all_build(_join_op == TJoinOp::RIGHT_OUTER_JOIN || - _join_op == TJoinOp::FULL_OUTER_JOIN), - _build_unique(!_have_other_join_conjunct && - (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || - _join_op == TJoinOp::LEFT_ANTI_JOIN || - _join_op == TJoinOp::LEFT_SEMI_JOIN)), - _is_right_semi_anti(_join_op == TJoinOp::RIGHT_ANTI_JOIN || - _join_op == TJoinOp::RIGHT_SEMI_JOIN), - _is_outer_join(_match_all_build || _match_all_probe), _is_broadcast_join(tnode.hash_join_node.__isset.is_broadcast_join && tnode.hash_join_node.is_broadcast_join), _hash_output_slot_ids(tnode.hash_join_node.__isset.hash_output_slot_ids ? tnode.hash_join_node.hash_output_slot_ids - : std::vector {}), - _intermediate_row_desc( - descs, tnode.hash_join_node.vintermediate_tuple_id_list, - std::vector(tnode.hash_join_node.vintermediate_tuple_id_list.size())), - _output_row_desc(descs, {tnode.hash_join_node.voutput_tuple_id}, {false}) { + : std::vector {}) { _runtime_filter_descs = tnode.runtime_filters; - init_join_op(); + _init_join_op(); // avoid vector expand change block address. // one block can store 4g data, _build_blocks can store 128*4g data. @@ -902,22 +884,8 @@ HashJoinNode::HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const Descr HashJoinNode::~HashJoinNode() = default; -void HashJoinNode::init_join_op() { - switch (_join_op) { -#define M(NAME) \ - case TJoinOp::NAME: \ - _join_op_variants.emplace>(); \ - break; - APPLY_FOR_JOINOP_VARIANTS(M); -#undef M - default: - //do nothing - break; - } -} - Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(ExecNode::init(tnode, state)); + RETURN_IF_ERROR(VJoinNodeBase::init(tnode, state)); DCHECK(tnode.__isset.hash_join_node); const bool build_stores_null = _join_op == TJoinOp::RIGHT_OUTER_JOIN || @@ -968,13 +936,6 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { DCHECK(_have_other_join_conjunct); } - const auto& output_exprs = tnode.hash_join_node.srcExprList; - for (const auto& expr : output_exprs) { - VExprContext* ctx = nullptr; - RETURN_IF_ERROR(VExpr::create_expr_tree(_pool, expr, &ctx)); - _output_expr_ctxs.push_back(ctx); - } - _runtime_filters.resize(_runtime_filter_descs.size()); for (size_t i = 0; i < _runtime_filter_descs.size(); i++) { RETURN_IF_ERROR(state->runtime_filter_mgr()->regist_filter( @@ -1010,25 +971,8 @@ Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { } Status HashJoinNode::prepare(RuntimeState* state) { - DCHECK(_runtime_profile.get() != nullptr); - _rows_returned_counter = ADD_COUNTER(_runtime_profile, "RowsReturned", TUnit::UNIT); - _rows_returned_rate = runtime_profile()->add_derived_counter( - ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, - std::bind(&RuntimeProfile::units_per_second, _rows_returned_counter, - runtime_profile()->total_time_counter()), - ""); - _mem_tracker = std::make_unique("ExecNode:" + _runtime_profile->name(), - _runtime_profile.get()); - - if (_vconjunct_ctx_ptr) { - RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->prepare(state, _intermediate_row_desc)); - } - - for (int i = 0; i < _children.size(); ++i) { - RETURN_IF_ERROR(_children[i]->prepare(state)); - } + RETURN_IF_ERROR(VJoinNodeBase::prepare(state)); SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - // Build phase auto build_phase_profile = runtime_profile()->create_child("BuildPhase", true, true); runtime_profile()->add_child(build_phase_profile, false, nullptr); @@ -1069,7 +1013,6 @@ Status HashJoinNode::prepare(RuntimeState* state) { RETURN_IF_ERROR((*_vother_join_conjunct_ptr)->prepare(state, _intermediate_row_desc)); } RETURN_IF_ERROR(VExpr::prepare(_output_expr_ctxs, state, _intermediate_row_desc)); - RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, _intermediate_row_desc)); // right table data types _right_table_data_types = VectorizedUtils::get_data_types(child(1)->row_desc()); @@ -1093,14 +1036,12 @@ Status HashJoinNode::close(RuntimeState* state) { _shared_hashtable_controller->wait_for_closable(state, id()); } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "ashJoinNode::close"); + START_AND_SCOPE_SPAN(state->get_tracer(), span, "HashJoinNode::close"); VExpr::close(_build_expr_ctxs, state); VExpr::close(_probe_expr_ctxs, state); if (_vother_join_conjunct_ptr) (*_vother_join_conjunct_ptr)->close(state); - VExpr::close(_output_expr_ctxs, state); - - return ExecNode::close(state); + return VJoinNodeBase::close(state); } Status HashJoinNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { @@ -1291,16 +1232,6 @@ void HashJoinNode::_prepare_probe_block() { release_block_memory(_probe_block); } -void HashJoinNode::_construct_mutable_join_block() { - const auto& mutable_block_desc = _intermediate_row_desc; - for (const auto tuple_desc : mutable_block_desc.tuple_descriptors()) { - for (const auto slot_desc : tuple_desc->slots()) { - auto type_ptr = slot_desc->get_data_type_ptr(); - _join_block.insert({type_ptr->create_column(), type_ptr, slot_desc->col_name()}); - } - } -} - Status HashJoinNode::open(RuntimeState* state) { START_AND_SCOPE_SPAN(state->get_tracer(), span, "HashJoinNode::open"); SCOPED_TIMER(_runtime_profile->total_time_counter()); @@ -1309,43 +1240,18 @@ Status HashJoinNode::open(RuntimeState* state) { RETURN_IF_ERROR(bf->init_with_fixed_length()); } } - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(VExpr::open(_build_expr_ctxs, state)); RETURN_IF_ERROR(VExpr::open(_probe_expr_ctxs, state)); if (_vother_join_conjunct_ptr) { RETURN_IF_ERROR((*_vother_join_conjunct_ptr)->open(state)); } - RETURN_IF_ERROR(VExpr::open(_output_expr_ctxs, state)); - - std::promise thread_status; - std::thread([this, state, thread_status_p = &thread_status, - parent_span = opentelemetry::trace::Tracer::GetCurrentSpan()] { - OpentelemetryScope scope {parent_span}; - this->_probe_side_open_thread(state, thread_status_p); - }).detach(); - - // Open the probe-side child so that it may perform any initialisation in parallel. - // Don't exit even if we see an error, we still need to wait for the build thread - // to finish. - // ISSUE-1247, check open_status after buildThread execute. - // If this return first, build thread will use 'thread_status' - // which is already destructor and then coredump. - Status status = _hash_table_build(state); - RETURN_IF_ERROR(thread_status.get_future().get()); - return status; + RETURN_IF_ERROR(VJoinNodeBase::open(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); + RETURN_IF_CANCELLED(state); + return Status::OK(); } -void HashJoinNode::_probe_side_open_thread(RuntimeState* state, std::promise* status) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "HashJoinNode::_hash_table_build_thread"); - SCOPED_ATTACH_TASK(state); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_shared()); - status->set_value(child(0)->open(state)); -} - -Status HashJoinNode::_hash_table_build(RuntimeState* state) { +Status HashJoinNode::_materialize_build_side(RuntimeState* state) { RETURN_IF_ERROR(child(1)->open(state)); SCOPED_TIMER(_build_timer); MutableBlock mutable_block(child(1)->row_desc().tuple_descriptors()); @@ -1744,53 +1650,6 @@ std::vector HashJoinNode::_convert_block_to_null(Block& block) { return results; } -Status HashJoinNode::_build_output_block(Block* origin_block, Block* output_block) { - auto is_mem_reuse = output_block->mem_reuse(); - MutableBlock mutable_block = - is_mem_reuse ? MutableBlock(output_block) - : MutableBlock(VectorizedUtils::create_empty_columnswithtypename( - _output_row_desc)); - auto rows = origin_block->rows(); - // TODO: After FE plan support same nullable of output expr and origin block and mutable column - // we should replace `insert_column_datas` by `insert_range_from` - - auto insert_column_datas = [](auto& to, const auto& from, size_t rows) { - if (to->is_nullable() && !from.is_nullable()) { - auto& null_column = reinterpret_cast(*to); - null_column.get_nested_column().insert_range_from(from, 0, rows); - null_column.get_null_map_column().get_data().resize_fill(rows, 0); - } else { - to->insert_range_from(from, 0, rows); - } - }; - if (rows != 0) { - auto& mutable_columns = mutable_block.mutable_columns(); - if (_output_expr_ctxs.empty()) { - DCHECK(mutable_columns.size() == _output_row_desc.num_materialized_slots()); - for (int i = 0; i < mutable_columns.size(); ++i) { - insert_column_datas(mutable_columns[i], *origin_block->get_by_position(i).column, - rows); - } - } else { - DCHECK(mutable_columns.size() == _output_row_desc.num_materialized_slots()); - for (int i = 0; i < mutable_columns.size(); ++i) { - auto result_column_id = -1; - RETURN_IF_ERROR(_output_expr_ctxs[i]->execute(origin_block, &result_column_id)); - auto column_ptr = origin_block->get_by_position(result_column_id) - .column->convert_to_full_column_if_const(); - insert_column_datas(mutable_columns[i], *column_ptr, rows); - } - } - - if (!is_mem_reuse) { - output_block->swap(mutable_block.to_block()); - } - DCHECK(output_block->rows() == rows); - } - - return Status::OK(); -} - void HashJoinNode::_add_tuple_is_null_column(Block* block) { if (_is_outer_join) { auto p0 = _tuple_is_null_left_flag_column->assume_mutable(); diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h index 277795a4a7..ca930ccda8 100644 --- a/be/src/vec/exec/join/vhash_join_node.h +++ b/be/src/vec/exec/join/vhash_join_node.h @@ -20,17 +20,18 @@ #include #include "common/object_pool.h" -#include "exec/exec_node.h" #include "exprs/runtime_filter_slots.h" #include "vec/common/columns_hashing.h" #include "vec/common/hash_table/hash_map.h" #include "vec/common/hash_table/hash_table.h" #include "vec/exec/join/join_op.h" #include "vec/exec/join/vacquire_list.hpp" +#include "vec/exec/join/vjoin_node_base.h" #include "vec/functions/function.h" #include "vec/runtime/shared_hash_table_controller.h" namespace doris { +class IRuntimeFilter; namespace vectorized { template @@ -160,30 +161,6 @@ using HashTableVariants = std::variant< I256FixedKeyHashTableContext, I256FixedKeyHashTableContext>; -using JoinOpVariants = - std::variant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant, - std::integral_constant>; - -#define APPLY_FOR_JOINOP_VARIANTS(M) \ - M(INNER_JOIN) \ - M(LEFT_SEMI_JOIN) \ - M(LEFT_ANTI_JOIN) \ - M(LEFT_OUTER_JOIN) \ - M(FULL_OUTER_JOIN) \ - M(RIGHT_OUTER_JOIN) \ - M(CROSS_JOIN) \ - M(RIGHT_SEMI_JOIN) \ - M(RIGHT_ANTI_JOIN) \ - M(NULL_AWARE_LEFT_ANTI_JOIN) - class VExprContext; class HashJoinNode; @@ -257,7 +234,7 @@ using HashTableCtxVariants = std::variant< ProcessHashTableProbe< std::integral_constant>>; -class HashJoinNode : public ::doris::ExecNode { +class HashJoinNode final : public VJoinNodeBase { public: HashJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); ~HashJoinNode() override; @@ -268,24 +245,15 @@ public: Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; Status get_next(RuntimeState* state, Block* block, bool* eos) override; Status close(RuntimeState* state) override; - void init_join_op(); - - const RowDescriptor& row_desc() const override { return _output_row_desc; } private: using VExprContexts = std::vector; - - TJoinOp::type _join_op; - - JoinOpVariants _join_op_variants; // probe expr VExprContexts _probe_expr_ctxs; // build expr VExprContexts _build_expr_ctxs; // other expr std::unique_ptr _vother_join_conjunct_ptr; - // output expr - VExprContexts _output_expr_ctxs; // mark the join column whether support null eq std::vector _is_null_safe_eq_join; @@ -299,19 +267,15 @@ private: DataTypes _right_table_data_types; DataTypes _left_table_data_types; - RuntimeProfile::Counter* _build_timer; RuntimeProfile::Counter* _build_table_timer; RuntimeProfile::Counter* _build_expr_call_timer; RuntimeProfile::Counter* _build_table_insert_timer; RuntimeProfile::Counter* _build_table_expanse_timer; - RuntimeProfile::Counter* _probe_timer; RuntimeProfile::Counter* _probe_expr_call_timer; RuntimeProfile::Counter* _probe_next_timer; RuntimeProfile::Counter* _build_buckets_counter; RuntimeProfile::Counter* _push_down_timer; RuntimeProfile::Counter* _push_compute_timer; - RuntimeProfile::Counter* _build_rows_counter; - RuntimeProfile::Counter* _probe_rows_counter; RuntimeProfile::Counter* _search_hashtable_timer; RuntimeProfile::Counter* _build_side_output_timer; RuntimeProfile::Counter* _probe_side_output_timer; @@ -343,14 +307,6 @@ private: Sizes _probe_key_sz; Sizes _build_key_sz; - bool _have_other_join_conjunct; - const bool _match_all_probe; // output all rows coming from the probe input. Full/Left Join - const bool _match_all_build; // output all rows coming from the build input. Full/Right Join - bool _build_unique; // build a hash table without duplicated rows. Left semi/anti Join - - const bool _is_right_semi_anti; - const bool _is_outer_join; - // For null aware left anti join, we apply a short circuit strategy. // 1. Set _short_circuit_for_null_in_build_side to true if join operator is null aware left anti join. // 2. In build phase, we stop building hash table when we meet the first null value and set _short_circuit_for_null_in_probe_side to true. @@ -361,22 +317,15 @@ private: SharedHashTableController* _shared_hashtable_controller = nullptr; VRuntimeFilterSlots* _runtime_filter_slots; - Block _join_block; - std::vector _hash_output_slot_ids; std::vector _left_output_slot_flags; std::vector _right_output_slot_flags; - RowDescriptor _intermediate_row_desc; - RowDescriptor _output_row_desc; - MutableColumnPtr _tuple_is_null_left_flag_column; MutableColumnPtr _tuple_is_null_right_flag_column; private: - void _probe_side_open_thread(RuntimeState* state, std::promise* status); - - Status _hash_table_build(RuntimeState* state); + Status _materialize_build_side(RuntimeState* state) override; Status _process_build_block(RuntimeState* state, Block& block, uint8_t offset); @@ -398,10 +347,6 @@ private: void _prepare_probe_block(); - void _construct_mutable_join_block(); - - Status _build_output_block(Block* origin_block, Block* output_block); - // add tuple is null flag column to Block for filter conjunct and output expr void _add_tuple_is_null_column(Block* block); diff --git a/be/src/vec/exec/join/vjoin_node_base.cpp b/be/src/vec/exec/join/vjoin_node_base.cpp new file mode 100644 index 0000000000..2c59b66b04 --- /dev/null +++ b/be/src/vec/exec/join/vjoin_node_base.cpp @@ -0,0 +1,196 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/exec/join/vjoin_node_base.h" + +#include + +#include "gen_cpp/PlanNodes_types.h" +#include "runtime/runtime_state.h" +#include "vec/utils/util.hpp" + +namespace doris::vectorized { + +VJoinNodeBase::VJoinNodeBase(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) + : ExecNode(pool, tnode, descs), + _join_op(tnode.__isset.hash_join_node ? tnode.hash_join_node.join_op + : tnode.nested_loop_join_node.join_op), + _have_other_join_conjunct(tnode.__isset.hash_join_node + ? tnode.hash_join_node.__isset.vother_join_conjunct + : false), + _match_all_probe(_join_op == TJoinOp::LEFT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _match_all_build(_join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN), + _build_unique(!_have_other_join_conjunct && + (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_ANTI_JOIN || + _join_op == TJoinOp::LEFT_SEMI_JOIN)), + _is_right_semi_anti(_join_op == TJoinOp::RIGHT_ANTI_JOIN || + _join_op == TJoinOp::RIGHT_SEMI_JOIN), + _is_outer_join(_match_all_build || _match_all_probe), + _output_row_desc( + descs, + {tnode.__isset.hash_join_node ? tnode.hash_join_node.voutput_tuple_id + : tnode.nested_loop_join_node.voutput_tuple_id}, + {false}), + _intermediate_row_desc( + descs, + tnode.__isset.hash_join_node + ? tnode.hash_join_node.vintermediate_tuple_id_list + : tnode.nested_loop_join_node.vintermediate_tuple_id_list, + std::vector( + tnode.__isset.hash_join_node + ? tnode.hash_join_node.vintermediate_tuple_id_list.size() + : tnode.nested_loop_join_node.vintermediate_tuple_id_list + .size())) {} + +Status VJoinNodeBase::close(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VJoinNodeBase::close"); + VExpr::close(_output_expr_ctxs, state); + return ExecNode::close(state); +} + +void VJoinNodeBase::_construct_mutable_join_block() { + const auto& mutable_block_desc = _intermediate_row_desc; + for (const auto tuple_desc : mutable_block_desc.tuple_descriptors()) { + for (const auto slot_desc : tuple_desc->slots()) { + auto type_ptr = slot_desc->get_data_type_ptr(); + _join_block.insert({type_ptr->create_column(), type_ptr, slot_desc->col_name()}); + } + } +} + +Status VJoinNodeBase::_build_output_block(Block* origin_block, Block* output_block) { + auto is_mem_reuse = output_block->mem_reuse(); + MutableBlock mutable_block = + is_mem_reuse ? MutableBlock(output_block) + : MutableBlock(VectorizedUtils::create_empty_columnswithtypename( + _output_row_desc)); + auto rows = origin_block->rows(); + // TODO: After FE plan support same nullable of output expr and origin block and mutable column + // we should replace `insert_column_datas` by `insert_range_from` + + auto insert_column_datas = [](auto& to, const auto& from, size_t rows) { + if (to->is_nullable() && !from.is_nullable()) { + auto& null_column = reinterpret_cast(*to); + null_column.get_nested_column().insert_range_from(from, 0, rows); + null_column.get_null_map_column().get_data().resize_fill(rows, 0); + } else { + to->insert_range_from(from, 0, rows); + } + }; + if (rows != 0) { + auto& mutable_columns = mutable_block.mutable_columns(); + if (_output_expr_ctxs.empty()) { + DCHECK(mutable_columns.size() == _output_row_desc.num_materialized_slots()); + for (int i = 0; i < mutable_columns.size(); ++i) { + insert_column_datas(mutable_columns[i], *origin_block->get_by_position(i).column, + rows); + } + } else { + DCHECK(mutable_columns.size() == _output_row_desc.num_materialized_slots()); + for (int i = 0; i < mutable_columns.size(); ++i) { + auto result_column_id = -1; + RETURN_IF_ERROR(_output_expr_ctxs[i]->execute(origin_block, &result_column_id)); + auto column_ptr = origin_block->get_by_position(result_column_id) + .column->convert_to_full_column_if_const(); + insert_column_datas(mutable_columns[i], *column_ptr, rows); + } + } + + if (!is_mem_reuse) { + output_block->swap(mutable_block.to_block()); + } + DCHECK(output_block->rows() == rows); + } + + return Status::OK(); +} + +Status VJoinNodeBase::init(const TPlanNode& tnode, RuntimeState* state) { + const auto& output_exprs = tnode.__isset.hash_join_node + ? tnode.hash_join_node.srcExprList + : tnode.nested_loop_join_node.srcExprList; + for (const auto& expr : output_exprs) { + VExprContext* ctx = nullptr; + RETURN_IF_ERROR(VExpr::create_expr_tree(_pool, expr, &ctx)); + _output_expr_ctxs.push_back(ctx); + } + return ExecNode::init(tnode, state); +} + +Status VJoinNodeBase::open(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VJoinNodeBase::open"); + RETURN_IF_ERROR(ExecNode::open(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); + RETURN_IF_CANCELLED(state); + + std::promise thread_status; + std::thread([this, state, thread_status_p = &thread_status, + parent_span = opentelemetry::trace::Tracer::GetCurrentSpan()] { + OpentelemetryScope scope {parent_span}; + this->_probe_side_open_thread(state, thread_status_p); + }).detach(); + + // Open the probe-side child so that it may perform any initialisation in parallel. + // Don't exit even if we see an error, we still need to wait for the build thread + // to finish. + // ISSUE-1247, check open_status after buildThread execute. + // If this return first, build thread will use 'thread_status' + // which is already destructor and then coredump. + Status status = _materialize_build_side(state); + RETURN_IF_ERROR(thread_status.get_future().get()); + + RETURN_IF_ERROR(VExpr::open(_output_expr_ctxs, state)); + return status; +} + +void VJoinNodeBase::_probe_side_open_thread(RuntimeState* state, std::promise* status) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VJoinNodeBase::_hash_table_build_thread"); + SCOPED_ATTACH_TASK(state); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker_shared()); + status->set_value(child(0)->open(state)); +} + +#define APPLY_FOR_JOINOP_VARIANTS(M) \ + M(INNER_JOIN) \ + M(LEFT_SEMI_JOIN) \ + M(LEFT_ANTI_JOIN) \ + M(LEFT_OUTER_JOIN) \ + M(FULL_OUTER_JOIN) \ + M(RIGHT_OUTER_JOIN) \ + M(CROSS_JOIN) \ + M(RIGHT_SEMI_JOIN) \ + M(RIGHT_ANTI_JOIN) \ + M(NULL_AWARE_LEFT_ANTI_JOIN) + +void VJoinNodeBase::_init_join_op() { + switch (_join_op) { +#define M(NAME) \ + case TJoinOp::NAME: \ + _join_op_variants.emplace>(); \ + break; + APPLY_FOR_JOINOP_VARIANTS(M); +#undef M + default: + //do nothing + break; + } +} + +} // namespace doris::vectorized diff --git a/be/src/vec/exec/join/vjoin_node_base.h b/be/src/vec/exec/join/vjoin_node_base.h new file mode 100644 index 0000000000..dd41f5ca1f --- /dev/null +++ b/be/src/vec/exec/join/vjoin_node_base.h @@ -0,0 +1,96 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include + +#include "exec/exec_node.h" +#include "gen_cpp/PlanNodes_types.h" +#include "runtime/descriptors.h" + +namespace doris::vectorized { + +using JoinOpVariants = + std::variant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant, + std::integral_constant>; + +class VJoinNodeBase : public ExecNode { +public: + VJoinNodeBase(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + + virtual Status close(RuntimeState* state) override; + + virtual Status open(RuntimeState* state) override; + + const RowDescriptor& row_desc() const override { return _output_row_desc; } + + const RowDescriptor& intermediate_row_desc() const override { return _intermediate_row_desc; } + + virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + +protected: + // Construct the intermediate blocks to store the results from join operation. + void _construct_mutable_join_block(); + // Convert the intermediate blocks to the final result. For example, if the block from probe + // side is non-nullable and the join op is righter outer join, we need to convert the non-nullable + // columns from probe side to a nullable column. + Status _build_output_block(Block* origin_block, Block* output_block); + // Open probe side asynchronously. + void _probe_side_open_thread(RuntimeState* state, std::promise* status); + + // Initialize the join operation. + void _init_join_op(); + + // Materialize build relation. For HashJoin, it will build a hash table while a list of build blocks for NLJoin. + virtual Status _materialize_build_side(RuntimeState* state) = 0; + + TJoinOp::type _join_op; + JoinOpVariants _join_op_variants; + + bool _have_other_join_conjunct; + const bool _match_all_probe; // output all rows coming from the probe input. Full/Left Join + const bool _match_all_build; // output all rows coming from the build input. Full/Right Join + bool _build_unique; // build a hash table without duplicated rows. Left semi/anti Join + + const bool _is_right_semi_anti; + const bool _is_outer_join; + + RowDescriptor _output_row_desc; + RowDescriptor _intermediate_row_desc; + // output expr + std::vector _output_expr_ctxs; + + Block _join_block; + + RuntimeProfile::Counter* _build_timer; + RuntimeProfile::Counter* _probe_timer; + RuntimeProfile::Counter* _build_rows_counter; + RuntimeProfile::Counter* _probe_rows_counter; +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/exec/join/vnested_loop_join_node.cpp b/be/src/vec/exec/join/vnested_loop_join_node.cpp new file mode 100644 index 0000000000..c966b9ab2c --- /dev/null +++ b/be/src/vec/exec/join/vnested_loop_join_node.cpp @@ -0,0 +1,487 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/exec/join/vnested_loop_join_node.h" + +#include + +#include "common/status.h" +#include "exprs/expr.h" +#include "gen_cpp/PlanNodes_types.h" +#include "runtime/row_batch.h" +#include "runtime/runtime_state.h" +#include "util/runtime_profile.h" +#include "util/simd/bits.h" +#include "vec/columns/column_const.h" +#include "vec/common/typeid_cast.h" +#include "vec/utils/template_helpers.hpp" +#include "vec/utils/util.hpp" + +namespace doris::vectorized { + +VNestedLoopJoinNode::VNestedLoopJoinNode(ObjectPool* pool, const TPlanNode& tnode, + const DescriptorTbl& descs) + : VJoinNodeBase(pool, tnode, descs), + _cur_probe_row_visited_flags(false), + _matched_rows_done(false), + _left_block_pos(0), + _left_side_eos(false) {} + +Status VNestedLoopJoinNode::prepare(RuntimeState* state) { + DCHECK(_join_op == TJoinOp::CROSS_JOIN || _join_op == TJoinOp::INNER_JOIN || + _join_op == TJoinOp::LEFT_OUTER_JOIN || _join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(VJoinNodeBase::prepare(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); + + _build_timer = ADD_TIMER(runtime_profile(), "BuildTime"); + _build_rows_counter = ADD_COUNTER(runtime_profile(), "BuildRows", TUnit::UNIT); + _probe_rows_counter = ADD_COUNTER(runtime_profile(), "ProbeRows", TUnit::UNIT); + _probe_timer = ADD_TIMER(runtime_profile(), "ProbeTime"); + + // pre-compute the tuple index of build tuples in the output row + int num_build_tuples = child(1)->row_desc().tuple_descriptors().size(); + + for (int i = 0; i < num_build_tuples; ++i) { + TupleDescriptor* build_tuple_desc = child(1)->row_desc().tuple_descriptors()[i]; + auto tuple_idx = _row_descriptor.get_tuple_idx(build_tuple_desc->id()); + RETURN_IF_INVALID_TUPLE_IDX(build_tuple_desc->id(), tuple_idx); + } + + _num_probe_side_columns = child(0)->row_desc().num_materialized_slots(); + _num_build_side_columns = child(1)->row_desc().num_materialized_slots(); + RETURN_IF_ERROR(VExpr::prepare(_output_expr_ctxs, state, _intermediate_row_desc)); + _construct_mutable_join_block(); + return Status::OK(); +} + +Status VNestedLoopJoinNode::close(RuntimeState* state) { + // avoid double close + if (is_closed()) { + return Status::OK(); + } + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VNestedLoopJoinNode::close"); + return VJoinNodeBase::close(state); +} + +Status VNestedLoopJoinNode::_materialize_build_side(RuntimeState* state) { + // Do a full scan of child(1) and store all build row batches. + RETURN_IF_ERROR(child(1)->open(state)); + + bool eos = false; + while (true) { + SCOPED_TIMER(_build_timer); + RETURN_IF_CANCELLED(state); + + Block block; + RETURN_IF_ERROR_AND_CHECK_SPAN(child(1)->get_next_after_projects(state, &block, &eos), + child(1)->get_next_span(), eos); + auto rows = block.rows(); + auto mem_usage = block.allocated_bytes(); + + if (rows != 0) { + _build_rows += rows; + _total_mem_usage += mem_usage; + _build_blocks.emplace_back(std::move(block)); + if (_match_all_build) { + _build_side_visited_flags.emplace_back(ColumnUInt8::create(rows, 0)); + } + } + + if (eos) { + break; + } + } + + COUNTER_UPDATE(_build_rows_counter, _build_rows); + return Status::OK(); +} + +Status VNestedLoopJoinNode::get_next(RuntimeState* state, Block* block, bool* eos) { + INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, + "VNestedLoopJoinNode::get_next"); + SCOPED_TIMER(_probe_timer); + RETURN_IF_CANCELLED(state); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); + if ((_match_all_build && _matched_rows_done && + _output_null_idx_build_side == _build_blocks.size()) || + _matched_rows_done) { + *eos = true; + return Status::OK(); + } + + _join_block.clear_column_data(); + MutableBlock mutable_join_block(&_join_block); + auto& dst_columns = mutable_join_block.mutable_columns(); + + std::stack offset_stack; + RETURN_IF_ERROR(std::visit( + [&](auto&& join_op_variants, auto set_build_side_flag, auto set_probe_side_flag) { + while (mutable_join_block.rows() < state->batch_size() && !_matched_rows_done) { + // If this left block is exhausted or empty, we need to pull data from left child. + if (_left_block_pos == _left_block.rows()) { + _left_block_pos = 0; + + if (_left_side_eos) { + _matched_rows_done = true; + } else { + do { + release_block_memory(_left_block); + RETURN_IF_ERROR_AND_CHECK_SPAN( + child(0)->get_next_after_projects(state, &_left_block, + &_left_side_eos), + child(0)->get_next_span(), _left_side_eos); + } while (_left_block.rows() == 0 && !_left_side_eos); + COUNTER_UPDATE(_probe_rows_counter, _left_block.rows()); + if (_left_block.rows() == 0) { + _matched_rows_done = _left_side_eos; + } + } + } + + // We should try to join rows if there still are some rows from probe side. + if (!_matched_rows_done && _current_build_pos < _build_blocks.size()) { + do { + const auto& now_process_build_block = + _build_blocks[_current_build_pos++]; + if constexpr (set_build_side_flag) { + offset_stack.push(mutable_join_block.rows()); + } + _process_left_child_block(dst_columns, now_process_build_block); + } while (mutable_join_block.rows() < state->batch_size() && + _current_build_pos < _build_blocks.size()); + } + + if constexpr (set_probe_side_flag) { + Block tmp_block = mutable_join_block.to_block(0); + Status status = _do_filtering_and_update_visited_flags( + &tmp_block, offset_stack); + if (!status.OK()) { + return status; + } + mutable_join_block = MutableBlock(std::move(tmp_block)); + // If this join operation is left outer join or full outer join, when + // `_current_build_pos == _build_blocks.size()`, means all rows from build + // side have been joined with the current probe row, we should output current + // probe row with null from build side. + if (_current_build_pos == _build_blocks.size()) { + if (!_matched_rows_done) { + _output_null_data(dst_columns, state->batch_size()); + _reset_with_next_probe_row(dst_columns); + } + break; + } + } + + if (!_matched_rows_done && _current_build_pos == _build_blocks.size()) { + _reset_with_next_probe_row(dst_columns); + } + } + if constexpr (!set_probe_side_flag) { + Block tmp_block = mutable_join_block.to_block(0); + Status status = _do_filtering_and_update_visited_flags( + &tmp_block, offset_stack); + mutable_join_block = MutableBlock(std::move(tmp_block)); + if (!status.OK()) { + return status; + } + } + + if constexpr (set_build_side_flag) { + if (_matched_rows_done && _output_null_idx_build_side < _build_blocks.size()) { + auto& cols = mutable_join_block.mutable_columns(); + _output_null_data(cols, state->batch_size()); + } + } + return Status::OK(); + }, + _join_op_variants, make_bool_variant(_match_all_build), + make_bool_variant(_match_all_probe))); + *eos = _match_all_build + ? _output_null_idx_build_side == _build_blocks.size() && _matched_rows_done + : _matched_rows_done; + + Block tmp_block = mutable_join_block.to_block(0); + RETURN_IF_ERROR(_build_output_block(&tmp_block, block)); + reached_limit(block, eos); + return Status::OK(); +} + +void VNestedLoopJoinNode::_process_left_child_block(MutableColumns& dst_columns, + const Block& now_process_build_block) const { + const int max_added_rows = now_process_build_block.rows(); + for (size_t i = 0; i < _num_probe_side_columns; ++i) { + const ColumnWithTypeAndName& src_column = _left_block.get_by_position(i); + if (!src_column.column->is_nullable() && dst_columns[i]->is_nullable()) { + auto origin_sz = dst_columns[i]->size(); + DCHECK(_join_op == TJoinOp::RIGHT_OUTER_JOIN || _join_op == TJoinOp::FULL_OUTER_JOIN); + assert_cast(dst_columns[i].get()) + ->get_nested_column_ptr() + ->insert_many_from(*src_column.column, _left_block_pos, max_added_rows); + assert_cast(dst_columns[i].get()) + ->get_null_map_column() + .get_data() + .resize_fill(origin_sz + max_added_rows, 0); + } else { + dst_columns[i]->insert_many_from(*src_column.column, _left_block_pos, max_added_rows); + } + } + for (size_t i = 0; i < _num_build_side_columns; ++i) { + const ColumnWithTypeAndName& src_column = now_process_build_block.get_by_position(i); + if (!src_column.column->is_nullable() && + dst_columns[_num_probe_side_columns + i]->is_nullable()) { + auto origin_sz = dst_columns[_num_probe_side_columns + i]->size(); + DCHECK(_join_op == TJoinOp::LEFT_OUTER_JOIN || _join_op == TJoinOp::FULL_OUTER_JOIN); + assert_cast(dst_columns[_num_probe_side_columns + i].get()) + ->get_nested_column_ptr() + ->insert_range_from(*src_column.column.get(), 0, max_added_rows); + assert_cast(dst_columns[_num_probe_side_columns + i].get()) + ->get_null_map_column() + .get_data() + .resize_fill(origin_sz + max_added_rows, 0); + } else { + dst_columns[_num_probe_side_columns + i]->insert_range_from(*src_column.column.get(), 0, + max_added_rows); + } + } +} + +template +void VNestedLoopJoinNode::_output_null_data(MutableColumns& dst_columns, size_t batch_size) { + if constexpr (BuildSide) { + auto build_block_sz = _build_blocks.size(); + size_t i = _output_null_idx_build_side; + for (; i < build_block_sz; i++) { + const auto& cur_block = _build_blocks[i]; + const auto* __restrict cur_visited_flags = + assert_cast(_build_side_visited_flags[i].get()) + ->get_data() + .data(); + const auto num_rows = cur_block.rows(); + + std::vector selector(num_rows); + size_t selector_idx = 0; + for (size_t j = 0; j < num_rows; j++) { + if (!cur_visited_flags[j]) { + selector[selector_idx++] = j; + } + } + for (size_t j = 0; j < _num_probe_side_columns; ++j) { + DCHECK(_join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN); + dst_columns[j]->insert_many_defaults(selector_idx); + } + + for (size_t j = 0; j < _num_build_side_columns; ++j) { + auto src_column = cur_block.get_by_position(j); + if (!src_column.column->is_nullable() && + dst_columns[_num_probe_side_columns + j]->is_nullable()) { + auto origin_sz = dst_columns[_num_probe_side_columns + j]->size(); + DCHECK(_join_op == TJoinOp::LEFT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN); + assert_cast(dst_columns[_num_probe_side_columns + j].get()) + ->get_nested_column_ptr() + ->insert_indices_from(*src_column.column, selector.data(), + selector.data() + selector_idx); + assert_cast(dst_columns[_num_probe_side_columns + j].get()) + ->get_null_map_column() + .get_data() + .resize_fill(origin_sz + selector_idx, 0); + } else { + dst_columns[_num_probe_side_columns + j]->insert_indices_from( + *src_column.column.get(), selector.data(), + selector.data() + selector_idx); + } + } + if (dst_columns[0]->size() > batch_size) { + i++; + break; + } + } + _output_null_idx_build_side = i; + } else { + if (_cur_probe_row_visited_flags) { + return; + } + + DCHECK_LT(_left_block_pos, _left_block.rows()); + for (size_t i = 0; i < _num_probe_side_columns; ++i) { + const ColumnWithTypeAndName src_column = _left_block.get_by_position(i); + if (!src_column.column->is_nullable() && dst_columns[i]->is_nullable()) { + auto origin_sz = dst_columns[i]->size(); + DCHECK(_join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::FULL_OUTER_JOIN); + assert_cast(dst_columns[i].get()) + ->get_nested_column_ptr() + ->insert_many_from(*src_column.column, _left_block_pos, 1); + assert_cast(dst_columns[i].get()) + ->get_null_map_column() + .get_data() + .resize_fill(origin_sz + 1, 0); + } else { + dst_columns[i]->insert_many_from(*src_column.column, _left_block_pos, 1); + } + } + for (size_t i = 0; i < _num_build_side_columns; ++i) { + dst_columns[_num_probe_side_columns + i]->insert_default(); + } + } +} + +void VNestedLoopJoinNode::_reset_with_next_probe_row(MutableColumns& dst_columns) { + _cur_probe_row_visited_flags = false; + _current_build_pos = 0; + _left_block_pos++; +} + +template +Status VNestedLoopJoinNode::_do_filtering_and_update_visited_flags( + Block* block, std::stack& offset_stack) { + auto column_to_keep = block->columns(); + // If we need to set visited flags for build side, + // 1. Execute conjuncts and get a column with bool type to do filtering. + // 2. Use bool column to update build-side visited flags. + // 3. Use bool column to do filtering. + size_t build_block_idx = + _current_build_pos == 0 ? _build_blocks.size() - 1 : _current_build_pos - 1; + size_t processed_blocks_num = offset_stack.size(); + if (LIKELY(_vconjunct_ctx_ptr != nullptr && block->rows() > 0)) { + DCHECK((*_vconjunct_ctx_ptr) != nullptr); + int result_column_id = -1; + RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->execute(block, &result_column_id)); + ColumnPtr filter_column = block->get_by_position(result_column_id).column; + if (auto* nullable_column = check_and_get_column(*filter_column)) { + ColumnPtr nested_column = nullable_column->get_nested_column_ptr(); + + MutableColumnPtr mutable_holder = + nested_column->use_count() == 1 + ? nested_column->assume_mutable() + : nested_column->clone_resized(nested_column->size()); + + ColumnUInt8* concrete_column = assert_cast(mutable_holder.get()); + auto* __restrict null_map = nullable_column->get_null_map_data().data(); + IColumn::Filter& filter = concrete_column->get_data(); + auto* __restrict filter_data = filter.data(); + + const size_t size = filter.size(); + for (size_t i = 0; i < size; ++i) { + filter_data[i] &= !null_map[i]; + } + if constexpr (SetBuildSideFlag) { + for (size_t i = 0; i < processed_blocks_num; i++) { + auto& build_side_flag = + assert_cast( + _build_side_visited_flags[build_block_idx].get()) + ->get_data(); + auto* __restrict build_side_flag_data = build_side_flag.data(); + auto cur_sz = build_side_flag.size(); + const size_t offset = offset_stack.top(); + offset_stack.pop(); + for (size_t j = 0; j < cur_sz; j++) { + build_side_flag_data[j] |= filter_data[offset + j]; + } + build_block_idx = + build_block_idx == 0 ? _build_blocks.size() - 1 : build_block_idx - 1; + } + } + if constexpr (SetProbeSideFlag) { + _cur_probe_row_visited_flags |= simd::contain_byte(filter_data, size, 1); + } + Block::filter_block_internal(block, filter, column_to_keep); + } else if (auto* const_column = check_and_get_column(*filter_column)) { + bool ret = const_column->get_bool(0); + if (!ret) { + for (size_t i = 0; i < column_to_keep; ++i) { + std::move(*block->get_by_position(i).column).assume_mutable()->clear(); + } + } else { + if constexpr (SetBuildSideFlag) { + for (size_t i = 0; i < processed_blocks_num; i++) { + auto& build_side_flag = + assert_cast( + _build_side_visited_flags[build_block_idx].get()) + ->get_data(); + auto* __restrict build_side_flag_data = build_side_flag.data(); + auto cur_sz = build_side_flag.size(); + offset_stack.pop(); + memset(reinterpret_cast(build_side_flag_data), 1, cur_sz); + build_block_idx = build_block_idx == 0 ? _build_blocks.size() - 1 + : build_block_idx - 1; + } + } + if constexpr (SetProbeSideFlag) { + _cur_probe_row_visited_flags |= ret; + } + } + } else { + const IColumn::Filter& filter = + assert_cast&>(*filter_column) + .get_data(); + if constexpr (SetBuildSideFlag) { + for (size_t i = 0; i < processed_blocks_num; i++) { + auto& build_side_flag = + assert_cast( + _build_side_visited_flags[build_block_idx].get()) + ->get_data(); + auto* __restrict build_side_flag_data = build_side_flag.data(); + auto cur_sz = build_side_flag.size(); + const size_t offset = offset_stack.top(); + offset_stack.pop(); + for (size_t j = 0; j < cur_sz; j++) { + build_side_flag_data[j] |= filter[offset + j]; + } + build_block_idx = + build_block_idx == 0 ? _build_blocks.size() - 1 : build_block_idx - 1; + } + } + if constexpr (SetProbeSideFlag) { + _cur_probe_row_visited_flags |= + simd::contain_byte(filter.data(), filter.size(), 1); + } + Block::filter_block_internal(block, filter, column_to_keep); + } + Block::erase_useless_column(block, column_to_keep); + } + return Status::OK(); +} + +Status VNestedLoopJoinNode::open(RuntimeState* state) { + START_AND_SCOPE_SPAN(state->get_tracer(), span, "VNestedLoopJoinNode::open") + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(VJoinNodeBase::open(state)); + SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); + RETURN_IF_CANCELLED(state); + // We can close the right child to release its resources because its input has been + // fully consumed. + child(1)->close(state); + return Status::OK(); +} + +void VNestedLoopJoinNode::debug_string(int indentation_level, std::stringstream* out) const { + *out << std::string(indentation_level * 2, ' '); + *out << "VNestedLoopJoinNode"; + *out << "(eos=" << (_matched_rows_done ? "true" : "false") + << " left_block_pos=" << _left_block_pos; + VJoinNodeBase::debug_string(indentation_level, out); + *out << ")"; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/exec/vnested_loop_join_node.h b/be/src/vec/exec/join/vnested_loop_join_node.h similarity index 50% rename from be/src/vec/exec/vnested_loop_join_node.h rename to be/src/vec/exec/join/vnested_loop_join_node.h index 7cd9d95eee..313b33e626 100644 --- a/be/src/vec/exec/vnested_loop_join_node.h +++ b/be/src/vec/exec/join/vnested_loop_join_node.h @@ -15,29 +15,22 @@ // specific language governing permissions and limitations // under the License. -#ifndef DORIS_BE_SRC_QUERY_EXEC_VNESTED_LOOP_JOIN_NODE_H -#define DORIS_BE_SRC_QUERY_EXEC_VNESTED_LOOP_JOIN_NODE_H +#pragma once #include #include +#include #include -#include "exec/exec_node.h" -#include "exec/row_batch_list.h" #include "gen_cpp/PlanNodes_types.h" #include "runtime/descriptors.h" -#include "runtime/mem_pool.h" #include "vec/core/block.h" +#include "vec/exec/join/vjoin_node_base.h" namespace doris::vectorized { -// Node for cross joins. -// Iterates over the left child rows and then the right child rows and, for -// each combination, writes the output row if the conjuncts are satisfied. The -// build batches are kept in a list that is fully constructed from the right child in -// construct_build_side() (called by BlockingJoinNode::open()) while rows are fetched from -// the left child as necessary in get_next(). -class VNestedLoopJoinNode final : public ExecNode { +// Node for nested loop joins. +class VNestedLoopJoinNode final : public VJoinNodeBase { public: VNestedLoopJoinNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); @@ -47,11 +40,6 @@ public: Status close(RuntimeState* state) override; - Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; - - // Open prepares the build side structures (subclasses should implement - // construct_build_side()) and then prepares for GetNext with the first left child row - // (subclasses should implement init_get_next()). Status open(RuntimeState* state) override; Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override { @@ -61,42 +49,38 @@ public: void debug_string(int indentation_level, std::stringstream* out) const override; private: - // Supervises ConstructBuildSide in a separate thread, and returns its status in the - // promise parameter. - void _build_side_thread(RuntimeState* state, std::promise* status); - // Init the build-side state for a new left child row (e.g. hash table iterator or list - // iterator) given the first row. Used in open() to prepare for get_next(). - // -1 for left_side_pos indicates the left child eos. - void _init_get_next(int first_left_row); - - // We parallelize building the build-side with Opening the - // left child. If, for example, the left child is another - // join node, it can start to build its own build-side at the - // same time. - Status _construct_build_side(RuntimeState* state); - - // Build mutable columns to insert data. - // if block can mem reuse, just clear data in block - // else build a new block and alloc mem of column from left and right child block - MutableColumns _get_mutable_columns(Block* block); + Status _materialize_build_side(RuntimeState* state) override; // Processes a block from the left child. // dst_columns: left_child_row and now_process_build_block to construct a bundle column of new block // now_process_build_block: right child block now to process void _process_left_child_block(MutableColumns& dst_columns, - const Block& now_process_build_block); + const Block& now_process_build_block) const; + + template + Status _do_filtering_and_update_visited_flags(Block* block, std::stack& offset_stack); + + template + void _output_null_data(MutableColumns& dst_columns, size_t batch_size); + + void _reset_with_next_probe_row(MutableColumns& dst_columns); // List of build blocks, constructed in prepare() Blocks _build_blocks; + // Visited flags for each row in build side. + MutableColumns _build_side_visited_flags; + // Visited flags for current row in probe side. + bool _cur_probe_row_visited_flags; size_t _current_build_pos = 0; - size_t _num_existing_columns = 0; - size_t _num_columns_to_add = 0; + size_t _num_probe_side_columns = 0; + size_t _num_build_side_columns = 0; uint64_t _build_rows = 0; uint64_t _total_mem_usage = 0; - TJoinOp::type _join_op; - bool _eos; // if true, nothing left to return in get_next() + uint64_t _output_null_idx_build_side = 0; + + bool _matched_rows_done; // _left_block must be cleared before calling get_next(). The child node // does not initialize all tuple ptrs in the row, only the ones that it @@ -105,13 +89,6 @@ private: int _left_block_pos; // current scan pos in _left_block bool _left_side_eos; // if true, left child has no more rows to process - - RuntimeProfile::Counter* _build_timer; // time to prepare build side - RuntimeProfile::Counter* _left_child_timer; // time to process left child batch - RuntimeProfile::Counter* _build_row_counter; // num build rows - RuntimeProfile::Counter* _left_child_row_counter; // num left child rows }; } // namespace doris::vectorized - -#endif diff --git a/be/src/vec/exec/vnested_loop_join_node.cpp b/be/src/vec/exec/vnested_loop_join_node.cpp deleted file mode 100644 index 337d25d9b8..0000000000 --- a/be/src/vec/exec/vnested_loop_join_node.cpp +++ /dev/null @@ -1,276 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "vec/exec/vnested_loop_join_node.h" - -#include - -#include "exprs/expr.h" -#include "gen_cpp/PlanNodes_types.h" -#include "runtime/row_batch.h" -#include "runtime/runtime_state.h" -#include "util/runtime_profile.h" - -namespace doris::vectorized { - -VNestedLoopJoinNode::VNestedLoopJoinNode(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs) - : ExecNode(pool, tnode, descs), _join_op(TJoinOp::CROSS_JOIN), _left_side_eos(false) {} - -Status VNestedLoopJoinNode::prepare(RuntimeState* state) { - DCHECK(_join_op == TJoinOp::CROSS_JOIN); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - _build_timer = ADD_TIMER(runtime_profile(), "BuildTime"); - _left_child_timer = ADD_TIMER(runtime_profile(), "LeftChildTime"); - _build_row_counter = ADD_COUNTER(runtime_profile(), "BuildRows", TUnit::UNIT); - _left_child_row_counter = ADD_COUNTER(runtime_profile(), "LeftChildRows", TUnit::UNIT); - - // pre-compute the tuple index of build tuples in the output row - int num_build_tuples = child(1)->row_desc().tuple_descriptors().size(); - - for (int i = 0; i < num_build_tuples; ++i) { - TupleDescriptor* build_tuple_desc = child(1)->row_desc().tuple_descriptors()[i]; - auto tuple_idx = _row_descriptor.get_tuple_idx(build_tuple_desc->id()); - RETURN_IF_INVALID_TUPLE_IDX(build_tuple_desc->id(), tuple_idx); - } - - _num_existing_columns = child(0)->row_desc().num_materialized_slots(); - _num_columns_to_add = child(1)->row_desc().num_materialized_slots(); - return Status::OK(); -} - -Status VNestedLoopJoinNode::close(RuntimeState* state) { - // avoid double close - if (is_closed()) { - return Status::OK(); - } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VNestedLoopJoinNode::close"); - ExecNode::close(state); - return Status::OK(); -} - -Status VNestedLoopJoinNode::_construct_build_side(RuntimeState* state) { - // Do a full scan of child(1) and store all build row batches. - RETURN_IF_ERROR(child(1)->open(state)); - - bool eos = false; - while (true) { - SCOPED_TIMER(_build_timer); - RETURN_IF_CANCELLED(state); - - Block block; - RETURN_IF_ERROR_AND_CHECK_SPAN(child(1)->get_next_after_projects(state, &block, &eos), - child(1)->get_next_span(), eos); - auto rows = block.rows(); - auto mem_usage = block.allocated_bytes(); - - if (rows != 0) { - _build_rows += rows; - _total_mem_usage += mem_usage; - _build_blocks.emplace_back(std::move(block)); - } - - if (eos) { - break; - } - } - - COUNTER_UPDATE(_build_row_counter, _build_rows); - // If right table in join is empty, the node is eos - _eos = _build_rows == 0; - return Status::OK(); -} - -void VNestedLoopJoinNode::_init_get_next(int left_batch_row) { - _current_build_pos = 0; -} - -Status VNestedLoopJoinNode::get_next(RuntimeState* state, Block* block, bool* eos) { - INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, - "VNestedLoopJoinNode::get_next"); - RETURN_IF_CANCELLED(state); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - *eos = false; - - if (_eos) { - *eos = true; - return Status::OK(); - } - - auto dst_columns = _get_mutable_columns(block); - ScopedTimer timer(_left_child_timer); - - while (block->rows() < state->batch_size() && !_eos) { - // Check to see if we're done processing the current left child batch - if (_current_build_pos == _build_blocks.size()) { - _current_build_pos = 0; - _left_block_pos++; - - if (_left_block_pos == _left_block.rows()) { - _left_block_pos = 0; - - if (_left_side_eos) { - *eos = _eos = true; - } else { - do { - release_block_memory(_left_block); - timer.stop(); - RETURN_IF_ERROR_AND_CHECK_SPAN( - child(0)->get_next_after_projects(state, &_left_block, - &_left_side_eos), - child(0)->get_next_span(), _left_side_eos); - timer.start(); - } while (_left_block.rows() == 0 && !_left_side_eos); - COUNTER_UPDATE(_left_child_row_counter, _left_block.rows()); - if (_left_block.rows() == 0) { - *eos = _eos = _left_side_eos; - } - } - } - } - - if (!_eos) { - do { - const auto& now_process_build_block = _build_blocks[_current_build_pos++]; - _process_left_child_block(dst_columns, now_process_build_block); - } while (block->rows() < state->batch_size() && - _current_build_pos < _build_blocks.size()); - } - } - dst_columns.clear(); - RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns())); - - reached_limit(block, eos); - return Status::OK(); -} - -MutableColumns VNestedLoopJoinNode::_get_mutable_columns(Block* block) { - bool mem_reuse = block->mem_reuse(); - if (!mem_reuse) { - for (size_t i = 0; i < _num_existing_columns; ++i) { - const ColumnWithTypeAndName& src_column = _left_block.get_by_position(i); - block->insert({src_column.type->create_column(), src_column.type, src_column.name}); - } - - for (size_t i = 0; i < _num_columns_to_add; ++i) { - const ColumnWithTypeAndName& src_column = _build_blocks[0].get_by_position(i); - block->insert({src_column.type->create_column(), src_column.type, src_column.name}); - } - } - return block->mutate_columns(); -} - -void VNestedLoopJoinNode::_process_left_child_block(MutableColumns& dst_columns, - const Block& now_process_build_block) { - const int max_added_rows = now_process_build_block.rows(); - for (size_t i = 0; i < _num_existing_columns; ++i) { - const ColumnWithTypeAndName& src_column = _left_block.get_by_position(i); - dst_columns[i]->insert_many_from(*src_column.column, _left_block_pos, max_added_rows); - } - for (size_t i = 0; i < _num_columns_to_add; ++i) { - const ColumnWithTypeAndName& src_column = now_process_build_block.get_by_position(i); - dst_columns[_num_existing_columns + i]->insert_range_from(*src_column.column.get(), 0, - max_added_rows); - } -} - -Status VNestedLoopJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { - return ExecNode::init(tnode, state); -} - -void VNestedLoopJoinNode::_build_side_thread(RuntimeState* state, std::promise* status) { - SCOPED_ATTACH_TASK(state); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_shared()); - status->set_value(_construct_build_side(state)); - // Release the thread token as soon as possible (before the main thread joins - // on it). This way, if we had a chain of 10 joins using 1 additional thread, - // we'd keep the additional thread busy the whole time. -} - -Status VNestedLoopJoinNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VNestedLoopJoinNode::open") - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - RETURN_IF_CANCELLED(state); - - _eos = false; - - // Kick-off the construction of the build-side table in a separate - // thread, so that the left child can do any initialisation in parallel. - // Only do this if we can get a thread token. Otherwise, do this in the - // main thread - std::promise build_side_status; - - add_runtime_exec_option("Join Build-Side Prepared Asynchronously"); - std::thread(bind(&VNestedLoopJoinNode::_build_side_thread, this, state, &build_side_status)) - .detach(); - - // Open the left child so that it may perform any initialisation in parallel. - // Don't exit even if we see an error, we still need to wait for the build thread - // to finish. - Status open_status = child(0)->open(state); - - // Blocks until ConstructBuildSide has returned, after which the build side structures - // are fully constructed. - RETURN_IF_ERROR(build_side_status.get_future().get()); - // We can close the right child to release its resources because its input has been - // fully consumed. - child(1)->close(state); - - RETURN_IF_ERROR(open_status); - - // Seed left child in preparation for get_next(). - while (true) { - release_block_memory(_left_block); - RETURN_IF_ERROR_AND_CHECK_SPAN( - child(0)->get_next_after_projects(state, &_left_block, &_left_side_eos), - child(0)->get_next_span(), _left_side_eos); - COUNTER_UPDATE(_left_child_row_counter, _left_block.rows()); - _left_block_pos = 0; - - if (_left_block.rows() == 0) { - if (_left_side_eos) { - _init_get_next(-1); - _eos = true; - break; - } - - continue; - } else { - _init_get_next(_left_block_pos); - break; - } - } - - return Status::OK(); -} - -void VNestedLoopJoinNode::debug_string(int indentation_level, std::stringstream* out) const { - *out << std::string(indentation_level * 2, ' '); - *out << "VNestedLoopJoinNode"; - *out << "(eos=" << (_eos ? "true" : "false") << " left_block_pos=" << _left_block_pos; - ExecNode::debug_string(indentation_level, out); - *out << ")"; -} - -} // namespace doris::vectorized diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 3db5cf8358..dfa32985bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -691,10 +691,87 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor leftChildOutputMap = Maps.newHashMap(); + nestedLoopJoin.child(0).getOutput().stream() + .map(SlotReference.class::cast) + .forEach(s -> leftChildOutputMap.put(s.getExprId(), s)); + Map rightChildOutputMap = Maps.newHashMap(); + nestedLoopJoin.child(1).getOutput().stream() + .map(SlotReference.class::cast) + .forEach(s -> rightChildOutputMap.put(s.getExprId(), s)); + // make intermediate tuple + List leftIntermediateSlotDescriptor = Lists.newArrayList(); + List rightIntermediateSlotDescriptor = Lists.newArrayList(); + TupleDescriptor intermediateDescriptor = context.generateTupleDesc(); + + // Nereids does not care about output order of join, + // but BE need left child's output must be before right child's output. + // So we need to swap the output order of left and right child if necessary. + // TODO: revert this after Nereids could ensure the output order is correct. + List leftSlotDescriptors = leftTuples.stream() + .map(TupleDescriptor::getSlots) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + List rightSlotDescriptors = rightTuples.stream() + .map(TupleDescriptor::getSlots) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + Map outputSlotReferenceMap = Maps.newHashMap(); + + nestedLoopJoin.getOutput().stream() + .map(SlotReference.class::cast) + .forEach(s -> outputSlotReferenceMap.put(s.getExprId(), s)); + List outputSlotReferences = Stream.concat(leftTuples.stream(), rightTuples.stream()) + .map(TupleDescriptor::getSlots) + .flatMap(Collection::stream) + .map(sd -> context.findExprId(sd.getId())) + .map(outputSlotReferenceMap::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + + JoinType joinType = nestedLoopJoin.getJoinType(); + if (crossJoinNode.getConjuncts().isEmpty() + && (joinType == JoinType.LEFT_ANTI_JOIN || joinType == JoinType.LEFT_SEMI_JOIN)) { + for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) { + SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId())); + SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf); + leftIntermediateSlotDescriptor.add(sd); + } + } else if (crossJoinNode.getConjuncts().isEmpty() + && (joinType == JoinType.RIGHT_ANTI_JOIN || joinType == JoinType.RIGHT_SEMI_JOIN)) { + for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) { + SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId())); + SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf); + rightIntermediateSlotDescriptor.add(sd); + } + } else { + for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) { + SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId())); + SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf); + leftIntermediateSlotDescriptor.add(sd); + } + for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) { + SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId())); + SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf); + rightIntermediateSlotDescriptor.add(sd); + } + } + + // set slots as nullable for outer join + if (joinType == JoinType.LEFT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) { + rightIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true)); + } + if (joinType == JoinType.RIGHT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) { + leftIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true)); + } + + crossJoinNode.setvIntermediateTupleDescList(Lists.newArrayList(intermediateDescriptor)); + rightFragment.getPlanRoot().setCompactData(false); crossJoinNode.setChild(0, leftFragment.getPlanRoot()); connectChildFragment(crossJoinNode, 1, leftFragment, rightFragment, context); @@ -702,6 +779,21 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor ExpressionTranslator.translate(e, context)).forEach(crossJoinNode::addConjunct); + if (nestedLoopJoin.isShouldTranslateOutput()) { + // translate output expr on intermediate tuple + List srcToOutput = outputSlotReferences.stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toList()); + + TupleDescriptor outputDescriptor = context.generateTupleDesc(); + outputSlotReferences.forEach(s -> context.createSlotDesc(outputDescriptor, s)); + + crossJoinNode.setvOutputTupleDesc(outputDescriptor); + crossJoinNode.setvSrcToOutputSMap(srcToOutput); + } + if (nestedLoopJoin.getStats() != null) { + crossJoinNode.setCardinality((long) nestedLoopJoin.getStats().getRowCount()); + } return leftFragment; } else { throw new RuntimeException("Physical nested loop join could not execute with equal join condition."); @@ -714,10 +806,16 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor) project.child(0)).setShouldTranslateOutput(false); } + if (project.child(0) instanceof PhysicalNestedLoopJoin) { + ((PhysicalNestedLoopJoin) project.child(0)).setShouldTranslateOutput(false); + } if (project.child(0) instanceof PhysicalFilter) { if (project.child(0).child(0) instanceof PhysicalHashJoin) { ((PhysicalHashJoin) project.child(0).child(0)).setShouldTranslateOutput(false); } + if (project.child(0).child(0) instanceof PhysicalNestedLoopJoin) { + ((PhysicalNestedLoopJoin) project.child(0).child(0)).setShouldTranslateOutput(false); + } } PlanFragment inputFragment = project.child(0).accept(this, context); @@ -738,6 +836,12 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor otherJoinConjuncts; + protected boolean shouldTranslateOutput = true; + /** * Constructor of PhysicalJoin. */ @@ -77,6 +79,14 @@ public abstract class AbstractPhysicalJoin< return hashJoinConjuncts; } + public boolean isShouldTranslateOutput() { + return shouldTranslateOutput; + } + + public void setShouldTranslateOutput(boolean shouldTranslateOutput) { + this.shouldTranslateOutput = shouldTranslateOutput; + } + public JoinType getJoinType() { return joinType; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java index 8f38c06f8f..cfc7cdbfe8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashJoin.java @@ -42,8 +42,6 @@ public class PhysicalHashJoin< RIGHT_CHILD_TYPE extends Plan> extends AbstractPhysicalJoin { - private boolean shouldTranslateOutput = true; - // TODO: What's purpose? it's alway empty. private final List filterConjuncts = Lists.newArrayList(); @@ -123,14 +121,6 @@ public class PhysicalHashJoin< Optional.empty(), getLogicalProperties(), physicalProperties, statsDeriveResult, left(), right()); } - public boolean isShouldTranslateOutput() { - return shouldTranslateOutput; - } - - public void setShouldTranslateOutput(boolean shouldTranslateOutput) { - this.shouldTranslateOutput = shouldTranslateOutput; - } - public List getFilterConjuncts() { return filterConjuncts; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/CrossJoinNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/CrossJoinNode.java index 6b563b0878..d5763fac61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/CrossJoinNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/CrossJoinNode.java @@ -18,85 +18,125 @@ package org.apache.doris.planner; import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ExprSubstitutionMap; +import org.apache.doris.analysis.JoinOperator; +import org.apache.doris.analysis.SlotId; +import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TableRef; +import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.analysis.TupleId; +import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.VectorizedUtil; import org.apache.doris.statistics.StatisticalType; -import org.apache.doris.statistics.StatsRecursiveDerive; import org.apache.doris.thrift.TExplainLevel; +import org.apache.doris.thrift.TNestedLoopJoinNode; import org.apache.doris.thrift.TPlanNode; import org.apache.doris.thrift.TPlanNodeType; import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Collections; import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; /** * Cross join between left child and right child. */ -public class CrossJoinNode extends PlanNode { +public class CrossJoinNode extends JoinNodeBase { private static final Logger LOG = LogManager.getLogger(CrossJoinNode.class); - // Default per-host memory requirement used if no valid stats are available. - // TODO: Come up with a more useful heuristic (e.g., based on scanned partitions). - private static final long DEFAULT_PER_HOST_MEM = 2L * 1024L * 1024L * 1024L; - private final TableRef innerRef; - public CrossJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, TableRef innerRef) { - super(id, "CROSS JOIN", StatisticalType.CROSS_JOIN_NODE); - this.innerRef = innerRef; + super(id, "CROSS JOIN", StatisticalType.CROSS_JOIN_NODE, outer, inner, innerRef); tupleIds.addAll(outer.getTupleIds()); tupleIds.addAll(inner.getTupleIds()); - tblRefIds.addAll(outer.getTblRefIds()); - tblRefIds.addAll(inner.getTblRefIds()); - children.add(outer); - children.add(inner); + } - // Inherits all the nullable tuple from the children - // Mark tuples that form the "nullable" side of the outer join as nullable. - nullableTupleIds.addAll(outer.getNullableTupleIds()); - nullableTupleIds.addAll(inner.getNullableTupleIds()); + @Override + public Set computeInputSlotIds(Analyzer analyzer) throws NotImplementedException { + Set result = Sets.newHashSet(); + Preconditions.checkState(outputSlotIds != null); + // step1: change output slot id to src slot id + if (vSrcToOutputSMap != null) { + for (SlotId slotId : outputSlotIds) { + SlotRef slotRef = new SlotRef(analyzer.getDescTbl().getSlotDesc(slotId)); + Expr srcExpr = vSrcToOutputSMap.mappingForRhsExpr(slotRef); + if (srcExpr == null) { + result.add(slotId); + } else { + List srcSlotRefList = Lists.newArrayList(); + srcExpr.collect(SlotRef.class, srcSlotRefList); + result.addAll(srcSlotRefList.stream().map(e -> e.getSlotId()).collect(Collectors.toList())); + } + } + } + // conjunct + List conjunctSlotIds = Lists.newArrayList(); + Expr.getIds(conjuncts, null, conjunctSlotIds); + result.addAll(conjunctSlotIds); + return result; + } + + @Override + protected Pair needToCopyRightAndLeft() { + boolean copyleft = true; + boolean copyRight = true; + switch (joinOp) { + case LEFT_ANTI_JOIN: + case LEFT_SEMI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + copyRight = false; + break; + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + copyleft = false; + break; + default: + break; + } + return Pair.of(copyleft, copyRight); } /** * Only for Nereids. */ - public CrossJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, List tupleIds) { - super(id, "CROSS JOIN", StatisticalType.CROSS_JOIN_NODE); - this.innerRef = null; + public CrossJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, List tupleIds, + List srcToOutputList, TupleDescriptor intermediateTuple, TupleDescriptor outputTuple) { + super(id, "CROSS JOIN", StatisticalType.CROSS_JOIN_NODE, JoinOperator.CROSS_JOIN); this.tupleIds.addAll(tupleIds); children.add(outer); children.add(inner); + // TODO: need to set joinOp by Nereids // Inherits all the nullable tuple from the children // Mark tuples that form the "nullable" side of the outer join as nullable. nullableTupleIds.addAll(outer.getNullableTupleIds()); nullableTupleIds.addAll(inner.getNullableTupleIds()); + if (joinOp.equals(JoinOperator.FULL_OUTER_JOIN)) { + nullableTupleIds.addAll(outer.getTupleIds()); + nullableTupleIds.addAll(inner.getTupleIds()); + } else if (joinOp.equals(JoinOperator.LEFT_OUTER_JOIN)) { + nullableTupleIds.addAll(inner.getTupleIds()); + } else if (joinOp.equals(JoinOperator.RIGHT_OUTER_JOIN)) { + nullableTupleIds.addAll(outer.getTupleIds()); + } + vIntermediateTupleDescList = Lists.newArrayList(intermediateTuple); + vOutputTupleDesc = outputTuple; + vSrcToOutputSMap = new ExprSubstitutionMap(srcToOutputList, Collections.emptyList()); } public TableRef getInnerRef() { return innerRef; } - @Override - public void init(Analyzer analyzer) throws UserException { - super.init(analyzer); - assignedConjuncts = analyzer.getAssignedConjuncts(); - computeStats(analyzer); - } - - @Override - public void computeStats(Analyzer analyzer) throws UserException { - super.computeStats(analyzer); - if (!analyzer.safeIsEnableJoinReorderBasedCost()) { - return; - } - StatsRecursiveDerive.getStatsRecursiveDerive().statsRecursiveDerive(this); - cardinality = (long) statsDeriveResult.getRowCount(); - } - @Override protected void computeOldCardinality() { if (getChild(0).cardinality == -1 || getChild(1).cardinality == -1) { @@ -117,28 +157,74 @@ public class CrossJoinNode extends PlanNode { @Override protected void toThrift(TPlanNode msg) { + msg.nested_loop_join_node = new TNestedLoopJoinNode(); + msg.nested_loop_join_node.join_op = joinOp.toThrift(); + if (vSrcToOutputSMap != null) { + for (int i = 0; i < vSrcToOutputSMap.size(); i++) { + // TODO: Enable it after we support new optimizers + // if (ConnectContext.get().getSessionVariable().isEnableNereidsPlanner()) { + // msg.addToProjections(vSrcToOutputSMap.getLhs().get(i).treeToThrift()); + // } else + msg.nested_loop_join_node.addToSrcExprList(vSrcToOutputSMap.getLhs().get(i).treeToThrift()); + } + } + if (vOutputTupleDesc != null) { + msg.nested_loop_join_node.setVoutputTupleId(vOutputTupleDesc.getId().asInt()); + // TODO Enable it after we support new optimizers + // msg.setOutputTupleId(vOutputTupleDesc.getId().asInt()); + } + if (vIntermediateTupleDescList != null) { + for (TupleDescriptor tupleDescriptor : vIntermediateTupleDescList) { + msg.nested_loop_join_node.addToVintermediateTupleIdList(tupleDescriptor.getId().asInt()); + } + } msg.node_type = TPlanNodeType.CROSS_JOIN_NODE; } + @Override + public void init(Analyzer analyzer) throws UserException { + super.init(analyzer); + + // Only for Vec: create new tuple for join result + if (VectorizedUtil.isVectorized()) { + computeOutputTuple(analyzer); + } + } + @Override public String getNodeExplainString(String detailPrefix, TExplainLevel detailLevel) { - StringBuilder output = new StringBuilder().append(detailPrefix).append("cross join:").append("\n"); + String distrModeStr = ""; + StringBuilder output = + new StringBuilder().append(detailPrefix).append("join op: ").append(joinOp.toString()).append("(") + .append(distrModeStr).append(")\n"); + if (detailLevel == TExplainLevel.BRIEF) { output.append(detailPrefix).append(String.format("cardinality=%s", cardinality)).append("\n"); return output.toString(); } + if (!conjuncts.isEmpty()) { - output.append(detailPrefix).append("predicates: ").append(getExplainString(conjuncts)).append("\n"); - } else { - output.append(detailPrefix).append("predicates is NULL."); + output.append(detailPrefix).append("other predicates: ").append(getExplainString(conjuncts)).append("\n"); } output.append(detailPrefix).append(String.format("cardinality=%s", cardinality)).append("\n"); + // todo unify in plan node + if (vOutputTupleDesc != null) { + output.append(detailPrefix).append("vec output tuple id: ").append(vOutputTupleDesc.getId()).append("\n"); + } + if (vIntermediateTupleDescList != null) { + output.append(detailPrefix).append("vIntermediate tuple ids: "); + for (TupleDescriptor tupleDescriptor : vIntermediateTupleDescList) { + output.append(tupleDescriptor.getId()).append(" "); + } + output.append("\n"); + } + if (outputSlotIds != null) { + output.append(detailPrefix).append("output slot ids: "); + for (SlotId slotId : outputSlotIds) { + output.append(slotId).append(" "); + } + output.append("\n"); + } return output.toString(); } - - @Override - public int getNumInstances() { - return Math.max(children.get(0).getNumInstances(), children.get(1).getNumInstances()); - } - } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java index d04c5b28c1..01c6292253 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java @@ -31,29 +31,24 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TableRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.analysis.TupleId; -import org.apache.doris.analysis.TupleIsNullPredicate; import org.apache.doris.catalog.ColumnStats; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableIf; -import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CheckedMath; import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.VectorizedUtil; import org.apache.doris.statistics.StatisticalType; -import org.apache.doris.statistics.StatsRecursiveDerive; import org.apache.doris.thrift.TEqJoinCondition; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.THashJoinNode; -import org.apache.doris.thrift.TNullSide; import org.apache.doris.thrift.TPlanNode; import org.apache.doris.thrift.TPlanNodeType; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -61,7 +56,6 @@ import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -73,11 +67,9 @@ import java.util.stream.Collectors; * The right child must be a leaf node, ie, can only materialize * a single input tuple. */ -public class HashJoinNode extends PlanNode { +public class HashJoinNode extends JoinNodeBase { private static final Logger LOG = LogManager.getLogger(HashJoinNode.class); - private TableRef innerRef; - private final JoinOperator joinOp; // predicates of the form 'a=b' or 'a<=>b' private List eqJoinConjuncts = Lists.newArrayList(); // join conjuncts from the JOIN clause that aren't equi-join predicates @@ -85,29 +77,20 @@ public class HashJoinNode extends PlanNode { // join conjunct from the JOIN clause that aren't equi-join predicates, only use in // vec exec engine private Expr votherJoinConjunct = null; - private DistributionMode distrMode; private boolean isColocate = false; //the flag for colocate join private String colocateReason = ""; // if can not do colocate join, set reason here - private boolean isBucketShuffle = false; // the flag for bucket shuffle join private List hashOutputSlotIds = new ArrayList<>(); //init for nereids - private TupleDescriptor vOutputTupleDesc; - private ExprSubstitutionMap vSrcToOutputSMap; - private List vIntermediateTupleDescList; /** * Constructor of HashJoinNode. */ public HashJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, TableRef innerRef, List eqJoinConjuncts, List otherJoinConjuncts) { - super(id, "HASH JOIN", StatisticalType.HASH_JOIN_NODE); + super(id, "HASH JOIN", StatisticalType.HASH_JOIN_NODE, outer, inner, innerRef); Preconditions.checkArgument(eqJoinConjuncts != null && !eqJoinConjuncts.isEmpty()); Preconditions.checkArgument(otherJoinConjuncts != null); - tblRefIds.addAll(outer.getTblRefIds()); - tblRefIds.addAll(inner.getTblRefIds()); - this.innerRef = innerRef; - this.joinOp = innerRef.getJoinOp(); // TODO: Support not vec exec engine cut unless tupleid in semi/anti join if (VectorizedUtil.isVectorized()) { @@ -138,21 +121,6 @@ public class HashJoinNode extends PlanNode { } this.distrMode = DistributionMode.NONE; this.otherJoinConjuncts = otherJoinConjuncts; - children.add(outer); - children.add(inner); - - // Inherits all the nullable tuple from the children - // Mark tuples that form the "nullable" side of the outer join as nullable. - nullableTupleIds.addAll(inner.getNullableTupleIds()); - nullableTupleIds.addAll(outer.getNullableTupleIds()); - if (joinOp.equals(JoinOperator.FULL_OUTER_JOIN)) { - nullableTupleIds.addAll(outer.getTupleIds()); - nullableTupleIds.addAll(inner.getTupleIds()); - } else if (joinOp.equals(JoinOperator.LEFT_OUTER_JOIN)) { - nullableTupleIds.addAll(inner.getTupleIds()); - } else if (joinOp.equals(JoinOperator.RIGHT_OUTER_JOIN)) { - nullableTupleIds.addAll(outer.getTupleIds()); - } } /** @@ -161,12 +129,12 @@ public class HashJoinNode extends PlanNode { public HashJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, JoinOperator joinOp, List eqJoinConjuncts, List otherJoinConjuncts, List srcToOutputList, TupleDescriptor intermediateTuple, TupleDescriptor outputTuple) { - super(id, "HASH JOIN", StatisticalType.HASH_JOIN_NODE); + super(id, "HASH JOIN", StatisticalType.HASH_JOIN_NODE, joinOp); Preconditions.checkArgument(eqJoinConjuncts != null && !eqJoinConjuncts.isEmpty()); Preconditions.checkArgument(otherJoinConjuncts != null); tblRefIds.addAll(outer.getTblRefIds()); tblRefIds.addAll(inner.getTblRefIds()); - this.joinOp = joinOp; + // TODO: Support not vec exec engine cut unless tupleid in semi/anti join if (VectorizedUtil.isVectorized()) { if (joinOp.equals(JoinOperator.LEFT_ANTI_JOIN) || joinOp.equals(JoinOperator.LEFT_SEMI_JOIN) @@ -220,14 +188,6 @@ public class HashJoinNode extends PlanNode { return eqJoinConjuncts; } - public JoinOperator getJoinOp() { - return joinOp; - } - - public TableRef getInnerRef() { - return innerRef; - } - public DistributionMode getDistributionMode() { return distrMode; } @@ -292,63 +252,18 @@ public class HashJoinNode extends PlanNode { @Override public void initOutputSlotIds(Set requiredSlotIdSet, Analyzer analyzer) { - outputSlotIds = Lists.newArrayList(); - List outputTupleDescList = Lists.newArrayList(); - if (vOutputTupleDesc != null) { - outputTupleDescList.add(vOutputTupleDesc); - } else { - for (TupleId tupleId : tupleIds) { - outputTupleDescList.add(analyzer.getTupleDesc(tupleId)); - } - } - SlotId firstMaterializedSlotId = null; - for (TupleDescriptor tupleDescriptor : outputTupleDescList) { - for (SlotDescriptor slotDescriptor : tupleDescriptor.getSlots()) { - if (slotDescriptor.isMaterialized()) { - if ((requiredSlotIdSet == null || requiredSlotIdSet.contains(slotDescriptor.getId()))) { - outputSlotIds.add(slotDescriptor.getId()); - } - if (firstMaterializedSlotId == null) { - firstMaterializedSlotId = slotDescriptor.getId(); - } - } - } - } - - // be may be possible to output correct row number without any column data in future - // but for now, in order to have correct output row number, should keep at least one slot. - // use first materialized slot if outputSlotIds is empty. - if (outputSlotIds.isEmpty() && firstMaterializedSlotId != null) { - outputSlotIds.add(firstMaterializedSlotId); - } + super.initOutputSlotIds(requiredSlotIdSet, analyzer); initHashOutputSlotIds(outputSlotIds, analyzer); } @Override - public void projectOutputTuple() throws NotImplementedException { - if (vOutputTupleDesc == null) { - return; + protected void computeOtherConjuncts(Analyzer analyzer, ExprSubstitutionMap originToIntermediateSmap) { + otherJoinConjuncts = Expr.substituteList(otherJoinConjuncts, originToIntermediateSmap, analyzer, false); + if (votherJoinConjunct != null) { + votherJoinConjunct = + Expr.substituteList(Arrays.asList(votherJoinConjunct), originToIntermediateSmap, analyzer, false) + .get(0); } - if (vOutputTupleDesc.getSlots().size() == outputSlotIds.size()) { - return; - } - Iterator iterator = vOutputTupleDesc.getSlots().iterator(); - while (iterator.hasNext()) { - SlotDescriptor slotDescriptor = iterator.next(); - boolean keep = false; - for (SlotId outputSlotId : outputSlotIds) { - if (slotDescriptor.getId().equals(outputSlotId)) { - keep = true; - break; - } - } - if (!keep) { - iterator.remove(); - SlotRef slotRef = new SlotRef(slotDescriptor); - vSrcToOutputSMap.removeByRhsExpr(slotRef); - } - } - vOutputTupleDesc.computeStatAndMemLayout(); } // output slots + predicate slots = input slots @@ -388,11 +303,6 @@ public class HashJoinNode extends PlanNode { @Override public void init(Analyzer analyzer) throws UserException { super.init(analyzer); - assignedConjuncts = analyzer.getAssignedConjuncts(); - // outSmap replace in outer join may cause NULL be replace by literal - // so need replace the outsmap in nullableTupleID - replaceOutputSmapForOuterJoin(); - computeStats(analyzer); ExprSubstitutionMap combinedChildSmap = getCombinedChildWithoutTupleIsNullSmap(); List newEqJoinConjuncts = Expr.substituteList(eqJoinConjuncts, combinedChildSmap, analyzer, false); @@ -407,201 +317,11 @@ public class HashJoinNode extends PlanNode { } } - private void computeOutputTuple(Analyzer analyzer) throws UserException { - // 1. create new tuple - vOutputTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); - boolean copyLeft = false; - boolean copyRight = false; - boolean leftNullable = false; - boolean rightNullable = false; - switch (joinOp) { - case INNER_JOIN: - case CROSS_JOIN: - copyLeft = true; - copyRight = true; - break; - case LEFT_OUTER_JOIN: - copyLeft = true; - copyRight = true; - rightNullable = true; - break; - case RIGHT_OUTER_JOIN: - copyLeft = true; - copyRight = true; - leftNullable = true; - break; - case FULL_OUTER_JOIN: - copyLeft = true; - copyRight = true; - leftNullable = true; - rightNullable = true; - break; - case LEFT_ANTI_JOIN: - case LEFT_SEMI_JOIN: - case NULL_AWARE_LEFT_ANTI_JOIN: - copyLeft = true; - break; - case RIGHT_ANTI_JOIN: - case RIGHT_SEMI_JOIN: - copyRight = true; - break; - default: - break; - } - ExprSubstitutionMap srcTblRefToOutputTupleSmap = new ExprSubstitutionMap(); - int leftNullableNumber = 0; - int rightNullableNumber = 0; - if (copyLeft) { - //cross join do not have OutputTblRefIds - List srcTupleIds = getChild(0) instanceof CrossJoinNode ? getChild(0).getOutputTupleIds() - : getChild(0).getOutputTblRefIds(); - for (TupleDescriptor leftTupleDesc : analyzer.getDescTbl().getTupleDesc(srcTupleIds)) { - // if the child is cross join node, the only way to get the correct nullable info of its output slots - // is to check if the output tuple ids are outer joined or not. - // then pass this nullable info to hash join node will be correct. - boolean needSetToNullable = - getChild(0) instanceof CrossJoinNode && analyzer.isOuterJoined(leftTupleDesc.getId()); - for (SlotDescriptor leftSlotDesc : leftTupleDesc.getSlots()) { - if (!isMaterailizedByChild(leftSlotDesc, getChild(0).getOutputSmap())) { - continue; - } - SlotDescriptor outputSlotDesc = - analyzer.getDescTbl().copySlotDescriptor(vOutputTupleDesc, leftSlotDesc); - if (leftNullable) { - outputSlotDesc.setIsNullable(true); - leftNullableNumber++; - } - if (needSetToNullable) { - outputSlotDesc.setIsNullable(true); - } - srcTblRefToOutputTupleSmap.put(new SlotRef(leftSlotDesc), new SlotRef(outputSlotDesc)); - } - } - } - if (copyRight) { - List srcTupleIds = getChild(1) instanceof CrossJoinNode ? getChild(1).getOutputTupleIds() - : getChild(1).getOutputTblRefIds(); - for (TupleDescriptor rightTupleDesc : analyzer.getDescTbl().getTupleDesc(srcTupleIds)) { - boolean needSetToNullable = - getChild(1) instanceof CrossJoinNode && analyzer.isOuterJoined(rightTupleDesc.getId()); - for (SlotDescriptor rightSlotDesc : rightTupleDesc.getSlots()) { - if (!isMaterailizedByChild(rightSlotDesc, getChild(1).getOutputSmap())) { - continue; - } - SlotDescriptor outputSlotDesc = - analyzer.getDescTbl().copySlotDescriptor(vOutputTupleDesc, rightSlotDesc); - if (rightNullable) { - outputSlotDesc.setIsNullable(true); - rightNullableNumber++; - } - if (needSetToNullable) { - outputSlotDesc.setIsNullable(true); - } - srcTblRefToOutputTupleSmap.put(new SlotRef(rightSlotDesc), new SlotRef(outputSlotDesc)); - } - } - } - // 2. compute srcToOutputMap - vSrcToOutputSMap = ExprSubstitutionMap.subtraction(outputSmap, srcTblRefToOutputTupleSmap, analyzer); - for (int i = 0; i < vSrcToOutputSMap.size(); i++) { - Preconditions.checkState(vSrcToOutputSMap.getRhs().get(i) instanceof SlotRef); - SlotRef rSlotRef = (SlotRef) vSrcToOutputSMap.getRhs().get(i); - if (vSrcToOutputSMap.getLhs().get(i) instanceof SlotRef) { - SlotRef lSlotRef = (SlotRef) vSrcToOutputSMap.getLhs().get(i); - rSlotRef.getDesc().setIsMaterialized(lSlotRef.getDesc().isMaterialized()); - } else { - rSlotRef.getDesc().setIsMaterialized(true); - } - } - vOutputTupleDesc.computeStatAndMemLayout(); - // 3. add tupleisnull in null-side - Preconditions.checkState(srcTblRefToOutputTupleSmap.getLhs().size() == vSrcToOutputSMap.getLhs().size()); - // Condition1: the left child is null-side - // Condition2: the left child is a inline view - // Then: add tuple is null in left child columns - if (leftNullable && getChild(0).tblRefIds.size() == 1 && analyzer.isInlineView(getChild(0).tblRefIds.get(0))) { - List tupleIsNullLhs = TupleIsNullPredicate - .wrapExprs(vSrcToOutputSMap.getLhs().subList(0, leftNullableNumber), new ArrayList<>(), - TNullSide.LEFT, analyzer); - tupleIsNullLhs - .addAll(vSrcToOutputSMap.getLhs().subList(leftNullableNumber, vSrcToOutputSMap.getLhs().size())); - vSrcToOutputSMap.updateLhsExprs(tupleIsNullLhs); - } - // Condition1: the right child is null-side - // Condition2: the right child is a inline view - // Then: add tuple is null in right child columns - if (rightNullable && getChild(1).tblRefIds.size() == 1 && analyzer.isInlineView(getChild(1).tblRefIds.get(0))) { - if (rightNullableNumber != 0) { - int rightBeginIndex = vSrcToOutputSMap.size() - rightNullableNumber; - List tupleIsNullLhs = TupleIsNullPredicate - .wrapExprs(vSrcToOutputSMap.getLhs().subList(rightBeginIndex, vSrcToOutputSMap.size()), - new ArrayList<>(), TNullSide.RIGHT, analyzer); - List newLhsList = Lists.newArrayList(); - if (rightBeginIndex > 0) { - newLhsList.addAll(vSrcToOutputSMap.getLhs().subList(0, rightBeginIndex)); - } - newLhsList.addAll(tupleIsNullLhs); - vSrcToOutputSMap.updateLhsExprs(newLhsList); - } - } - // 4. change the outputSmap - outputSmap = ExprSubstitutionMap.composeAndReplace(outputSmap, srcTblRefToOutputTupleSmap, analyzer); - } - - private void replaceOutputSmapForOuterJoin() { - if (joinOp.isOuterJoin() && !VectorizedUtil.isVectorized()) { - List lhs = new ArrayList<>(); - List rhs = new ArrayList<>(); - - for (int i = 0; i < outputSmap.size(); i++) { - Expr expr = outputSmap.getLhs().get(i); - boolean isInNullableTuple = false; - for (TupleId tupleId : nullableTupleIds) { - if (expr.isBound(tupleId)) { - isInNullableTuple = true; - break; - } - } - - if (!isInNullableTuple) { - lhs.add(outputSmap.getLhs().get(i)); - rhs.add(outputSmap.getRhs().get(i)); - } - } - outputSmap = new ExprSubstitutionMap(lhs, rhs); - } - } - @Override - public void finalize(Analyzer analyzer) throws UserException { - super.finalize(analyzer); - if (VectorizedUtil.isVectorized()) { - computeIntermediateTuple(analyzer); - } - } - - private void computeIntermediateTuple(Analyzer analyzer) throws AnalysisException { - // 1. create new tuple - TupleDescriptor vIntermediateLeftTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); - TupleDescriptor vIntermediateRightTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); - vIntermediateTupleDescList = new ArrayList<>(); - vIntermediateTupleDescList.add(vIntermediateLeftTupleDesc); - vIntermediateTupleDescList.add(vIntermediateRightTupleDesc); - boolean leftNullable = false; - boolean rightNullable = false; + protected Pair needToCopyRightAndLeft() { boolean copyleft = true; boolean copyRight = true; switch (joinOp) { - case LEFT_OUTER_JOIN: - rightNullable = true; - break; - case RIGHT_OUTER_JOIN: - leftNullable = true; - break; - case FULL_OUTER_JOIN: - leftNullable = true; - rightNullable = true; - break; case LEFT_ANTI_JOIN: case LEFT_SEMI_JOIN: case NULL_AWARE_LEFT_ANTI_JOIN: @@ -618,58 +338,7 @@ public class HashJoinNode extends PlanNode { default: break; } - // 2. exprsmap: - ExprSubstitutionMap originToIntermediateSmap = new ExprSubstitutionMap(); - Map, TupleId> originTidsToIntermediateTidMap = Maps.newHashMap(); - // left - if (copyleft) { - originTidsToIntermediateTidMap.put(getChild(0).getOutputTupleIds(), vIntermediateLeftTupleDesc.getId()); - for (TupleDescriptor tupleDescriptor : analyzer.getDescTbl() - .getTupleDesc(getChild(0).getOutputTupleIds())) { - for (SlotDescriptor slotDescriptor : tupleDescriptor.getMaterializedSlots()) { - SlotDescriptor intermediateSlotDesc = - analyzer.getDescTbl().copySlotDescriptor(vIntermediateLeftTupleDesc, slotDescriptor); - if (leftNullable) { - intermediateSlotDesc.setIsNullable(true); - } - originToIntermediateSmap.put(new SlotRef(slotDescriptor), new SlotRef(intermediateSlotDesc)); - } - } - } - vIntermediateLeftTupleDesc.computeMemLayout(); - // right - if (copyRight) { - originTidsToIntermediateTidMap.put(getChild(1).getOutputTupleIds(), vIntermediateRightTupleDesc.getId()); - for (TupleDescriptor tupleDescriptor : analyzer.getDescTbl() - .getTupleDesc(getChild(1).getOutputTupleIds())) { - for (SlotDescriptor slotDescriptor : tupleDescriptor.getMaterializedSlots()) { - SlotDescriptor intermediateSlotDesc = - analyzer.getDescTbl().copySlotDescriptor(vIntermediateRightTupleDesc, slotDescriptor); - if (rightNullable) { - intermediateSlotDesc.setIsNullable(true); - } - originToIntermediateSmap.put(new SlotRef(slotDescriptor), new SlotRef(intermediateSlotDesc)); - } - } - } - vIntermediateRightTupleDesc.computeMemLayout(); - // 3. replace srcExpr by intermediate tuple - Preconditions.checkState(vSrcToOutputSMap != null); - // Set `preserveRootTypes` to true because we should keep the consistent for types. See Issue-11314. - vSrcToOutputSMap.substituteLhs(originToIntermediateSmap, analyzer, true); - // 4. replace other conjuncts and conjuncts - otherJoinConjuncts = Expr.substituteList(otherJoinConjuncts, originToIntermediateSmap, analyzer, false); - if (votherJoinConjunct != null) { - votherJoinConjunct = - Expr.substituteList(Arrays.asList(votherJoinConjunct), originToIntermediateSmap, analyzer, false) - .get(0); - } - conjuncts = Expr.substituteList(conjuncts, originToIntermediateSmap, analyzer, false); - if (vconjunct != null) { - vconjunct = Expr.substituteList(Arrays.asList(vconjunct), originToIntermediateSmap, analyzer, false).get(0); - } - // 5. replace tuple is null expr - TupleIsNullPredicate.substitueListForTupleIsNull(vSrcToOutputSMap.getLhs(), originTidsToIntermediateTidMap); + return Pair.of(copyleft, copyRight); } /** @@ -849,14 +518,6 @@ public class HashJoinNode extends PlanNode { @Override public void computeStats(Analyzer analyzer) throws UserException { super.computeStats(analyzer); - - if (!analyzer.safeIsEnableJoinReorderBasedCost()) { - return; - } - - StatsRecursiveDerive.getStatsRecursiveDerive().statsRecursiveDerive(this); - cardinality = (long) statsDeriveResult.getRowCount(); - if (LOG.isDebugEnabled()) { LOG.debug("stats HashJoin:" + id + ", cardinality: " + cardinality); } @@ -1169,15 +830,6 @@ public class HashJoinNode extends PlanNode { return output.toString(); } - @Override - public int getNumInstances() { - return Math.max(children.get(0).getNumInstances(), children.get(1).getNumInstances()); - } - - public boolean isShuffleJoin() { - return distrMode == DistributionMode.PARTITIONED; - } - public enum DistributionMode { NONE("NONE"), BROADCAST("BROADCAST"), PARTITIONED("PARTITIONED"), BUCKET_SHUFFLE("BUCKET_SHUFFLE"); @@ -1202,102 +854,10 @@ public class HashJoinNode extends PlanNode { super.convertToVectoriezd(); } - /** - * If parent wants to get hash join node tupleids, - * it will call this function instead of read properties directly. - * The reason is that the tuple id of vOutputTupleDesc the real output tuple id for hash join node. - *

- * If you read the properties of @tupleids directly instead of this function, - * it reads the input id of the current node. - */ - @Override - public ArrayList getTupleIds() { - Preconditions.checkState(tupleIds != null); - if (vOutputTupleDesc != null) { - return Lists.newArrayList(vOutputTupleDesc.getId()); - } - return tupleIds; - } - - @Override - public ArrayList getOutputTblRefIds() { - if (vOutputTupleDesc != null) { - return Lists.newArrayList(vOutputTupleDesc.getId()); - } - switch (joinOp) { - case LEFT_SEMI_JOIN: - case LEFT_ANTI_JOIN: - case NULL_AWARE_LEFT_ANTI_JOIN: - return getChild(0).getOutputTblRefIds(); - case RIGHT_SEMI_JOIN: - case RIGHT_ANTI_JOIN: - return getChild(1).getOutputTblRefIds(); - default: - return getTblRefIds(); - } - } - - @Override - public List getOutputTupleIds() { - if (vOutputTupleDesc != null) { - return Lists.newArrayList(vOutputTupleDesc.getId()); - } - switch (joinOp) { - case LEFT_SEMI_JOIN: - case LEFT_ANTI_JOIN: - case NULL_AWARE_LEFT_ANTI_JOIN: - return getChild(0).getOutputTupleIds(); - case RIGHT_SEMI_JOIN: - case RIGHT_ANTI_JOIN: - return getChild(1).getOutputTupleIds(); - default: - return tupleIds; - } - } - - private boolean isMaterailizedByChild(SlotDescriptor slotDesc, ExprSubstitutionMap smap) { - if (slotDesc.isMaterialized()) { - return true; - } - Expr child = smap.get(new SlotRef(slotDesc)); - if (child == null) { - return false; - } - List slotRefList = Lists.newArrayList(); - child.collect(SlotRef.class, slotRefList); - for (SlotRef slotRef : slotRefList) { - if (slotRef.getDesc() != null && !slotRef.getDesc().isMaterialized()) { - return false; - } - } - return true; - } - /** * Used by nereids. */ public void setOtherJoinConjuncts(List otherJoinConjuncts) { this.otherJoinConjuncts = otherJoinConjuncts; } - - /** - * Used by nereids. - */ - public void setvOutputTupleDesc(TupleDescriptor vOutputTupleDesc) { - this.vOutputTupleDesc = vOutputTupleDesc; - } - - /** - * Used by nereids. - */ - public void setvIntermediateTupleDescList(List vIntermediateTupleDescList) { - this.vIntermediateTupleDescList = vIntermediateTupleDescList; - } - - /** - * Used by nereids. - */ - public void setvSrcToOutputSMap(List lhs) { - this.vSrcToOutputSMap = new ExprSubstitutionMap(lhs, Collections.emptyList()); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/JoinNodeBase.java b/fe/fe-core/src/main/java/org/apache/doris/planner/JoinNodeBase.java new file mode 100644 index 0000000000..b10a6a96e6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/JoinNodeBase.java @@ -0,0 +1,534 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.planner; + +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ExprSubstitutionMap; +import org.apache.doris.analysis.JoinOperator; +import org.apache.doris.analysis.SlotDescriptor; +import org.apache.doris.analysis.SlotId; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.analysis.TableRef; +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.analysis.TupleId; +import org.apache.doris.analysis.TupleIsNullPredicate; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.VectorizedUtil; +import org.apache.doris.statistics.StatisticalType; +import org.apache.doris.statistics.StatsRecursiveDerive; +import org.apache.doris.thrift.TNullSide; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public abstract class JoinNodeBase extends PlanNode { + private static final Logger LOG = LogManager.getLogger(JoinNodeBase.class); + + protected final TableRef innerRef; + protected final JoinOperator joinOp; + protected TupleDescriptor vOutputTupleDesc; + protected ExprSubstitutionMap vSrcToOutputSMap; + protected List vIntermediateTupleDescList; + + public JoinNodeBase(PlanNodeId id, String planNodeName, StatisticalType statisticalType, + PlanNode outer, PlanNode inner, TableRef innerRef) { + super(id, planNodeName, statisticalType); + this.innerRef = innerRef; + tblRefIds.addAll(outer.getTblRefIds()); + tblRefIds.addAll(inner.getTblRefIds()); + children.add(outer); + children.add(inner); + + // Inherits all the nullable tuple from the children + // Mark tuples that form the "nullable" side of the outer join as nullable. + nullableTupleIds.addAll(outer.getNullableTupleIds()); + nullableTupleIds.addAll(inner.getNullableTupleIds()); + + joinOp = innerRef.getJoinOp(); + if (joinOp.equals(JoinOperator.FULL_OUTER_JOIN)) { + nullableTupleIds.addAll(outer.getTupleIds()); + nullableTupleIds.addAll(inner.getTupleIds()); + } else if (joinOp.equals(JoinOperator.LEFT_OUTER_JOIN)) { + nullableTupleIds.addAll(inner.getTupleIds()); + } else if (joinOp.equals(JoinOperator.RIGHT_OUTER_JOIN)) { + nullableTupleIds.addAll(outer.getTupleIds()); + } + } + + public JoinOperator getJoinOp() { + return joinOp; + } + + protected boolean isMaterializedByChild(SlotDescriptor slotDesc, ExprSubstitutionMap smap) { + if (slotDesc.isMaterialized()) { + return true; + } + Expr child = smap.get(new SlotRef(slotDesc)); + if (child == null) { + return false; + } + List slotRefList = Lists.newArrayList(); + child.collect(SlotRef.class, slotRefList); + for (SlotRef slotRef : slotRefList) { + if (slotRef.getDesc() != null && !slotRef.getDesc().isMaterialized()) { + return false; + } + } + return true; + } + + protected void computeOutputTuple(Analyzer analyzer) throws UserException { + // 1. create new tuple + vOutputTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); + boolean copyLeft = false; + boolean copyRight = false; + boolean leftNullable = false; + boolean rightNullable = false; + switch (joinOp) { + case INNER_JOIN: + case CROSS_JOIN: + copyLeft = true; + copyRight = true; + break; + case LEFT_OUTER_JOIN: + copyLeft = true; + copyRight = true; + rightNullable = true; + break; + case RIGHT_OUTER_JOIN: + copyLeft = true; + copyRight = true; + leftNullable = true; + break; + case FULL_OUTER_JOIN: + copyLeft = true; + copyRight = true; + leftNullable = true; + rightNullable = true; + break; + case LEFT_ANTI_JOIN: + case LEFT_SEMI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + copyLeft = true; + break; + case RIGHT_ANTI_JOIN: + case RIGHT_SEMI_JOIN: + copyRight = true; + break; + default: + break; + } + ExprSubstitutionMap srcTblRefToOutputTupleSmap = new ExprSubstitutionMap(); + int leftNullableNumber = 0; + int rightNullableNumber = 0; + if (copyLeft) { + //cross join do not have OutputTblRefIds + List srcTupleIds = getChild(0) instanceof JoinNodeBase ? getChild(0).getOutputTupleIds() + : getChild(0).getOutputTblRefIds(); + for (TupleDescriptor leftTupleDesc : analyzer.getDescTbl().getTupleDesc(srcTupleIds)) { + // if the child is cross join node, the only way to get the correct nullable info of its output slots + // is to check if the output tuple ids are outer joined or not. + // then pass this nullable info to hash join node will be correct. + boolean needSetToNullable = + getChild(0) instanceof JoinNodeBase && analyzer.isOuterJoined(leftTupleDesc.getId()); + for (SlotDescriptor leftSlotDesc : leftTupleDesc.getSlots()) { + if (!isMaterializedByChild(leftSlotDesc, getChild(0).getOutputSmap())) { + continue; + } + SlotDescriptor outputSlotDesc = + analyzer.getDescTbl().copySlotDescriptor(vOutputTupleDesc, leftSlotDesc); + if (leftNullable) { + outputSlotDesc.setIsNullable(true); + leftNullableNumber++; + } + if (needSetToNullable) { + outputSlotDesc.setIsNullable(true); + } + srcTblRefToOutputTupleSmap.put(new SlotRef(leftSlotDesc), new SlotRef(outputSlotDesc)); + } + } + } + if (copyRight) { + List srcTupleIds = getChild(1) instanceof JoinNodeBase ? getChild(1).getOutputTupleIds() + : getChild(1).getOutputTblRefIds(); + for (TupleDescriptor rightTupleDesc : analyzer.getDescTbl().getTupleDesc(srcTupleIds)) { + boolean needSetToNullable = + getChild(1) instanceof JoinNodeBase && analyzer.isOuterJoined(rightTupleDesc.getId()); + for (SlotDescriptor rightSlotDesc : rightTupleDesc.getSlots()) { + if (!isMaterializedByChild(rightSlotDesc, getChild(1).getOutputSmap())) { + continue; + } + SlotDescriptor outputSlotDesc = + analyzer.getDescTbl().copySlotDescriptor(vOutputTupleDesc, rightSlotDesc); + if (rightNullable) { + outputSlotDesc.setIsNullable(true); + rightNullableNumber++; + } + if (needSetToNullable) { + outputSlotDesc.setIsNullable(true); + } + srcTblRefToOutputTupleSmap.put(new SlotRef(rightSlotDesc), new SlotRef(outputSlotDesc)); + } + } + } + // 2. compute srcToOutputMap + vSrcToOutputSMap = ExprSubstitutionMap.subtraction(outputSmap, srcTblRefToOutputTupleSmap, analyzer); + for (int i = 0; i < vSrcToOutputSMap.size(); i++) { + Preconditions.checkState(vSrcToOutputSMap.getRhs().get(i) instanceof SlotRef); + SlotRef rSlotRef = (SlotRef) vSrcToOutputSMap.getRhs().get(i); + if (vSrcToOutputSMap.getLhs().get(i) instanceof SlotRef) { + SlotRef lSlotRef = (SlotRef) vSrcToOutputSMap.getLhs().get(i); + rSlotRef.getDesc().setIsMaterialized(lSlotRef.getDesc().isMaterialized()); + } else { + rSlotRef.getDesc().setIsMaterialized(true); + } + } + vOutputTupleDesc.computeStatAndMemLayout(); + // 3. add tupleisnull in null-side + Preconditions.checkState(srcTblRefToOutputTupleSmap.getLhs().size() == vSrcToOutputSMap.getLhs().size()); + // Condition1: the left child is null-side + // Condition2: the left child is a inline view + // Then: add tuple is null in left child columns + if (leftNullable && getChild(0).tblRefIds.size() == 1 && analyzer.isInlineView(getChild(0).tblRefIds.get(0))) { + List tupleIsNullLhs = TupleIsNullPredicate + .wrapExprs(vSrcToOutputSMap.getLhs().subList(0, leftNullableNumber), new ArrayList<>(), + TNullSide.LEFT, analyzer); + tupleIsNullLhs + .addAll(vSrcToOutputSMap.getLhs().subList(leftNullableNumber, vSrcToOutputSMap.getLhs().size())); + vSrcToOutputSMap.updateLhsExprs(tupleIsNullLhs); + } + // Condition1: the right child is null-side + // Condition2: the right child is a inline view + // Then: add tuple is null in right child columns + if (rightNullable && getChild(1).tblRefIds.size() == 1 && analyzer.isInlineView(getChild(1).tblRefIds.get(0))) { + if (rightNullableNumber != 0) { + int rightBeginIndex = vSrcToOutputSMap.size() - rightNullableNumber; + List tupleIsNullLhs = TupleIsNullPredicate + .wrapExprs(vSrcToOutputSMap.getLhs().subList(rightBeginIndex, vSrcToOutputSMap.size()), + new ArrayList<>(), TNullSide.RIGHT, analyzer); + List newLhsList = Lists.newArrayList(); + if (rightBeginIndex > 0) { + newLhsList.addAll(vSrcToOutputSMap.getLhs().subList(0, rightBeginIndex)); + } + newLhsList.addAll(tupleIsNullLhs); + vSrcToOutputSMap.updateLhsExprs(newLhsList); + } + } + // 4. change the outputSmap + outputSmap = ExprSubstitutionMap.composeAndReplace(outputSmap, srcTblRefToOutputTupleSmap, analyzer); + } + + protected void replaceOutputSmapForOuterJoin() { + if (joinOp.isOuterJoin() && !VectorizedUtil.isVectorized()) { + List lhs = new ArrayList<>(); + List rhs = new ArrayList<>(); + + for (int i = 0; i < outputSmap.size(); i++) { + Expr expr = outputSmap.getLhs().get(i); + boolean isInNullableTuple = false; + for (TupleId tupleId : nullableTupleIds) { + if (expr.isBound(tupleId)) { + isInNullableTuple = true; + break; + } + } + + if (!isInNullableTuple) { + lhs.add(outputSmap.getLhs().get(i)); + rhs.add(outputSmap.getRhs().get(i)); + } + } + outputSmap = new ExprSubstitutionMap(lhs, rhs); + } + } + + @Override + public void initOutputSlotIds(Set requiredSlotIdSet, Analyzer analyzer) { + outputSlotIds = Lists.newArrayList(); + List outputTupleDescList = Lists.newArrayList(); + if (vOutputTupleDesc != null) { + outputTupleDescList.add(vOutputTupleDesc); + } else { + for (TupleId tupleId : tupleIds) { + outputTupleDescList.add(analyzer.getTupleDesc(tupleId)); + } + } + SlotId firstMaterializedSlotId = null; + for (TupleDescriptor tupleDescriptor : outputTupleDescList) { + for (SlotDescriptor slotDescriptor : tupleDescriptor.getSlots()) { + if (slotDescriptor.isMaterialized()) { + if ((requiredSlotIdSet == null || requiredSlotIdSet.contains(slotDescriptor.getId()))) { + outputSlotIds.add(slotDescriptor.getId()); + } + if (firstMaterializedSlotId == null) { + firstMaterializedSlotId = slotDescriptor.getId(); + } + } + } + } + + // be may be possible to output correct row number without any column data in future + // but for now, in order to have correct output row number, should keep at least one slot. + // use first materialized slot if outputSlotIds is empty. + if (outputSlotIds.isEmpty() && firstMaterializedSlotId != null) { + outputSlotIds.add(firstMaterializedSlotId); + } + } + + @Override + public void projectOutputTuple() throws NotImplementedException { + if (vOutputTupleDesc == null) { + return; + } + if (vOutputTupleDesc.getSlots().size() == outputSlotIds.size()) { + return; + } + Iterator iterator = vOutputTupleDesc.getSlots().iterator(); + while (iterator.hasNext()) { + SlotDescriptor slotDescriptor = iterator.next(); + boolean keep = false; + for (SlotId outputSlotId : outputSlotIds) { + if (slotDescriptor.getId().equals(outputSlotId)) { + keep = true; + break; + } + } + if (!keep) { + iterator.remove(); + SlotRef slotRef = new SlotRef(slotDescriptor); + vSrcToOutputSMap.removeByRhsExpr(slotRef); + } + } + vOutputTupleDesc.computeStatAndMemLayout(); + } + + protected abstract Pair needToCopyRightAndLeft(); + + protected void computeOtherConjuncts(Analyzer analyzer, ExprSubstitutionMap originToIntermediateSmap) {} + + protected void computeIntermediateTuple(Analyzer analyzer) throws AnalysisException { + // 1. create new tuple + TupleDescriptor vIntermediateLeftTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); + TupleDescriptor vIntermediateRightTupleDesc = analyzer.getDescTbl().createTupleDescriptor(); + vIntermediateTupleDescList = new ArrayList<>(); + vIntermediateTupleDescList.add(vIntermediateLeftTupleDesc); + vIntermediateTupleDescList.add(vIntermediateRightTupleDesc); + boolean leftNullable = false; + boolean rightNullable = false; + + switch (joinOp) { + case LEFT_OUTER_JOIN: + rightNullable = true; + break; + case RIGHT_OUTER_JOIN: + leftNullable = true; + break; + case FULL_OUTER_JOIN: + leftNullable = true; + rightNullable = true; + break; + default: + break; + } + Pair tmpPair = needToCopyRightAndLeft(); + boolean copyleft = tmpPair.first; + boolean copyRight = tmpPair.second; + // 2. exprsmap: + ExprSubstitutionMap originToIntermediateSmap = new ExprSubstitutionMap(); + Map, TupleId> originTidsToIntermediateTidMap = Maps.newHashMap(); + // left + if (copyleft) { + originTidsToIntermediateTidMap.put(getChild(0).getOutputTupleIds(), vIntermediateLeftTupleDesc.getId()); + for (TupleDescriptor tupleDescriptor : analyzer.getDescTbl() + .getTupleDesc(getChild(0).getOutputTupleIds())) { + for (SlotDescriptor slotDescriptor : tupleDescriptor.getMaterializedSlots()) { + SlotDescriptor intermediateSlotDesc = + analyzer.getDescTbl().copySlotDescriptor(vIntermediateLeftTupleDesc, slotDescriptor); + if (leftNullable) { + intermediateSlotDesc.setIsNullable(true); + } + originToIntermediateSmap.put(new SlotRef(slotDescriptor), new SlotRef(intermediateSlotDesc)); + } + } + } + vIntermediateLeftTupleDesc.computeMemLayout(); + // right + if (copyRight) { + originTidsToIntermediateTidMap.put(getChild(1).getOutputTupleIds(), vIntermediateRightTupleDesc.getId()); + for (TupleDescriptor tupleDescriptor : analyzer.getDescTbl() + .getTupleDesc(getChild(1).getOutputTupleIds())) { + for (SlotDescriptor slotDescriptor : tupleDescriptor.getMaterializedSlots()) { + SlotDescriptor intermediateSlotDesc = + analyzer.getDescTbl().copySlotDescriptor(vIntermediateRightTupleDesc, slotDescriptor); + if (rightNullable) { + intermediateSlotDesc.setIsNullable(true); + } + originToIntermediateSmap.put(new SlotRef(slotDescriptor), new SlotRef(intermediateSlotDesc)); + } + } + } + vIntermediateRightTupleDesc.computeMemLayout(); + // 3. replace srcExpr by intermediate tuple + Preconditions.checkState(vSrcToOutputSMap != null); + // Set `preserveRootTypes` to true because we should keep the consistent for types. See Issue-11314. + vSrcToOutputSMap.substituteLhs(originToIntermediateSmap, analyzer, true); + // 4. replace other conjuncts and conjuncts + computeOtherConjuncts(analyzer, originToIntermediateSmap); + conjuncts = Expr.substituteList(conjuncts, originToIntermediateSmap, analyzer, false); + if (vconjunct != null) { + vconjunct = Expr.substituteList(Arrays.asList(vconjunct), originToIntermediateSmap, analyzer, false).get(0); + } + // 5. replace tuple is null expr + TupleIsNullPredicate.substitueListForTupleIsNull(vSrcToOutputSMap.getLhs(), originTidsToIntermediateTidMap); + } + + @Override + public void finalize(Analyzer analyzer) throws UserException { + super.finalize(analyzer); + if (VectorizedUtil.isVectorized()) { + computeIntermediateTuple(analyzer); + } + } + + /** + * Only for Nereids. + */ + public JoinNodeBase(PlanNodeId id, String planNodeName, StatisticalType statisticalType, JoinOperator joinOp) { + super(id, planNodeName, statisticalType); + this.innerRef = null; + this.joinOp = joinOp; + } + + public TableRef getInnerRef() { + return innerRef; + } + + @Override + public void init(Analyzer analyzer) throws UserException { + super.init(analyzer); + assignedConjuncts = analyzer.getAssignedConjuncts(); + // outSmap replace in outer join may cause NULL be replace by literal + // so need replace the outsmap in nullableTupleID + replaceOutputSmapForOuterJoin(); + computeStats(analyzer); + } + + /** + * If parent wants to get join node tupleids, + * it will call this function instead of read properties directly. + * The reason is that the tuple id of vOutputTupleDesc the real output tuple id for join node. + *

+ * If you read the properties of @tupleids directly instead of this function, + * it reads the input id of the current node. + */ + @Override + public ArrayList getTupleIds() { + Preconditions.checkState(tupleIds != null); + if (vOutputTupleDesc != null) { + return Lists.newArrayList(vOutputTupleDesc.getId()); + } + return tupleIds; + } + + @Override + public ArrayList getOutputTblRefIds() { + if (vOutputTupleDesc != null) { + return Lists.newArrayList(vOutputTupleDesc.getId()); + } + switch (joinOp) { + case LEFT_SEMI_JOIN: + case LEFT_ANTI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + return getChild(0).getOutputTblRefIds(); + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + return getChild(1).getOutputTblRefIds(); + default: + return getTblRefIds(); + } + } + + @Override + public List getOutputTupleIds() { + if (vOutputTupleDesc != null) { + return Lists.newArrayList(vOutputTupleDesc.getId()); + } + switch (joinOp) { + case LEFT_SEMI_JOIN: + case LEFT_ANTI_JOIN: + case NULL_AWARE_LEFT_ANTI_JOIN: + return getChild(0).getOutputTupleIds(); + case RIGHT_SEMI_JOIN: + case RIGHT_ANTI_JOIN: + return getChild(1).getOutputTupleIds(); + default: + return tupleIds; + } + } + + @Override + public void computeStats(Analyzer analyzer) throws UserException { + super.computeStats(analyzer); + if (!analyzer.safeIsEnableJoinReorderBasedCost()) { + return; + } + StatsRecursiveDerive.getStatsRecursiveDerive().statsRecursiveDerive(this); + cardinality = (long) statsDeriveResult.getRowCount(); + } + + @Override + public int getNumInstances() { + return Math.max(children.get(0).getNumInstances(), children.get(1).getNumInstances()); + } + + /** + * Used by nereids. + */ + public void setvOutputTupleDesc(TupleDescriptor vOutputTupleDesc) { + this.vOutputTupleDesc = vOutputTupleDesc; + } + + /** + * Used by nereids. + */ + public void setvIntermediateTupleDescList(List vIntermediateTupleDescList) { + this.vIntermediateTupleDescList = vIntermediateTupleDescList; + } + + /** + * Used by nereids. + */ + public void setvSrcToOutputSMap(List lhs) { + this.vSrcToOutputSMap = new ExprSubstitutionMap(lhs, Collections.emptyList()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java index 73a6b1ae25..24d07d98e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java @@ -2040,24 +2040,6 @@ public class SingleNodePlanner { // are materialized) getHashLookupJoinConjuncts(analyzer, outer, inner, eqJoinConjuncts, errMsg, innerRef.getJoinOp()); - if (eqJoinConjuncts.isEmpty()) { - - // only inner join can change to cross join - if (innerRef.getJoinOp().isOuterJoin() || innerRef.getJoinOp().isSemiAntiJoin()) { - throw new AnalysisException("non-equal " + innerRef.getJoinOp().toString() - + " is not supported"); - } - - // construct cross join node - // LOG.debug("Join between {} and {} requires at least one conjunctive" - // + " equality predicate between the two tables", - // outerRef.getAliasAsName(), innerRef.getAliasAsName()); - // TODO If there are eq join predicates then we should construct a hash join - CrossJoinNode result = - new CrossJoinNode(ctx.getNextNodeId(), outer, inner, innerRef); - result.init(analyzer); - return result; - } analyzer.markConjunctsAssigned(eqJoinConjuncts); List ojConjuncts = Lists.newArrayList(); @@ -2072,6 +2054,25 @@ public class SingleNodePlanner { ojConjuncts = analyzer.getUnassignedConjuncts(tupleIds, false); } analyzer.markConjunctsAssigned(ojConjuncts); + if (eqJoinConjuncts.isEmpty()) { + + // only inner join can change to cross join + if (innerRef.getJoinOp().isSemiAntiJoin()) { + throw new AnalysisException("non-equal " + innerRef.getJoinOp().toString() + + " is not supported"); + } + + // construct cross join node + // LOG.debug("Join between {} and {} requires at least one conjunctive" + // + " equality predicate between the two tables", + // outerRef.getAliasAsName(), innerRef.getAliasAsName()); + // TODO If there are eq join predicates then we should construct a hash join + CrossJoinNode result = + new CrossJoinNode(ctx.getNextNodeId(), outer, inner, innerRef); + result.addConjuncts(ojConjuncts); + result.init(analyzer); + return result; + } HashJoinNode result = new HashJoinNode(ctx.getNextNodeId(), outer, inner, innerRef, eqJoinConjuncts, diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java index 73c18db8d0..5ff92f5441 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java @@ -153,8 +153,7 @@ public class StmtRewriterTest { String query = "select empid, sum(salary) from " + TABLE_NAME + " group by empid having sum(salary) > (" + subquery + ");"; LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); - dorisAssert.query(query).explainContains("CROSS JOIN", - "predicates: sum(`salary`) > avg(`salary`)"); + dorisAssert.query(query).explainContains("CROSS JOIN"); } /** @@ -265,7 +264,6 @@ public class StmtRewriterTest { + subquery + ") order by a;"; LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); dorisAssert.query(query).explainContains("CROSS JOIN", - "predicates: sum(`salary`) > avg(`salary`)", "order by: `$a$1`.`$c$1` ASC"); } @@ -378,7 +376,6 @@ public class StmtRewriterTest { LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); dorisAssert.query(query).explainContains("group by: `empid`", "CROSS JOIN", - "predicates: sum(`salary`) > avg(`salary`)", "order by: `$a$1`.`$c$2` ASC", "OUTPUT EXPRS:\n `$a$1`.`$c$1`"); } @@ -493,7 +490,6 @@ public class StmtRewriterTest { LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); dorisAssert.query(query).explainContains("group by: `empid`", "CROSS JOIN", - "predicates: sum(`salary`) > avg(`salary`)", "order by: `$a$1`.`$c$2` ASC", "OUTPUT EXPRS:\n `$a$1`.`$c$1`\n `$a$1`.`$c$2`"); } @@ -607,7 +603,6 @@ public class StmtRewriterTest { LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); dorisAssert.query(query).explainContains("group by: `empid`", "CROSS JOIN", - "predicates: sum(`salary`) > avg(`salary`)", "order by: `$a$1`.`$c$2` ASC", "OUTPUT EXPRS:\n `$a$1`.`$c$1`\n `$a$1`.`$c$2`"); } @@ -622,8 +617,7 @@ public class StmtRewriterTest { "select empid a, sum(salary) b from " + TABLE_NAME + " group by a having b > (" + subquery + ");"; LOG.info("EXPLAIN:{}", dorisAssert.query(query).explainQuery()); dorisAssert.query(query) - .explainContains("CROSS JOIN", "predicates: sum(`salary`) > avg(`salary`)", - "PREDICATES: `empid` >= 1, `empid` <= 2"); + .explainContains("CROSS JOIN"); } @AfterClass diff --git a/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java index ac089c7423..51ac8dc04d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/rewrite/ExtractCommonFactorsRuleFunctionTest.java @@ -113,7 +113,6 @@ public class ExtractCommonFactorsRuleFunctionTest { String query = "select * from tb1, tb2 where (tb1.k1 >1 and tb1.k1 <3 and tb1.k1 <5 and tb2.k1=1) " + "or (tb1.k1 <2 and tb2.k2=2)"; String planString = dorisAssert.query(query).explainQuery(); - Assert.assertTrue(planString.contains("`tb1`.`k1` < 5")); Assert.assertTrue(planString.contains("CROSS JOIN")); } diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 7e41a54d6f..be227dd7cd 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -583,6 +583,16 @@ struct THashJoinNode { 10: optional bool is_broadcast_join; } +struct TNestedLoopJoinNode { + 1: required TJoinOp join_op + // TODO: remove 2 and 3 in the version after the version include projection on ExecNode + 2: optional list srcExprList + + 3: optional Types.TTupleId voutput_tuple_id + + 4: optional list vintermediate_tuple_id_list +} + struct TMergeJoinNode { // anything from the ON, USING or WHERE clauses that's an equi-join predicate 1: required list cmp_conjuncts @@ -974,6 +984,7 @@ struct TPlanNode { // file scan node 44: optional TFileScanNode file_scan_node 45: optional TJdbcScanNode jdbc_scan_node + 46: optional TNestedLoopJoinNode nested_loop_join_node 101: optional list projections 102: optional Types.TTupleId output_tuple_id diff --git a/regression-test/data/query_p0/join/test_nestedloop_outer_join.out b/regression-test/data/query_p0/join/test_nestedloop_outer_join.out new file mode 100644 index 0000000000..5cde383a94 --- /dev/null +++ b/regression-test/data/query_p0/join/test_nestedloop_outer_join.out @@ -0,0 +1,77 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !join -- + +-- !join -- + +-- !join -- + +-- !join -- + +-- !join -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +10 10 \N \N + +-- !join -- + +-- !join -- +1 1 \N \N +2 2 \N \N +3 3 \N \N +10 10 \N \N + +-- !join -- + +-- !join -- +\N \N 1 1 +\N \N 2 2 +\N \N 3 3 +\N \N 10 10 + +-- !join -- +\N \N 1 1 +\N \N 2 2 +\N \N 3 3 +\N \N 10 10 + +-- !join -- + +-- !join -- + +-- !join -- +\N \N 0 0 +1 1 2 2 +1 1 3 3 +1 1 4 4 +2 2 3 3 +2 2 4 4 +3 3 4 4 +10 10 \N \N + +-- !join -- +\N \N 0 0 +1 1 2 2 +1 1 3 3 +1 1 4 4 +2 2 3 3 +2 2 4 4 +3 3 4 4 + +-- !join -- +1 1 2 2 +1 1 3 3 +1 1 4 4 +2 2 3 3 +2 2 4 4 +3 3 4 4 +10 10 \N \N + +-- !join -- +1 1 2 2 +1 1 3 3 +1 1 4 4 +2 2 3 3 +2 2 4 4 +3 3 4 4 + diff --git a/regression-test/suites/query/join/test_join.groovy b/regression-test/suites/query/join/test_join.groovy index c51d65826f..742ded403c 100644 --- a/regression-test/suites/query/join/test_join.groovy +++ b/regression-test/suites/query/join/test_join.groovy @@ -180,22 +180,10 @@ suite("test_join", "query,p0") { for (s in selected) { qt_left_join4"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 > b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 > 0 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } test { sql"""select ${s} from ${tbName1} a left join ${tbName2} b order by 1, 2, 3, 4, 5 limit 65535""" @@ -204,47 +192,17 @@ suite("test_join", "query,p0") { logger.info(exception.message) } } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 or a.k2 = b.k2 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 < b.k1 or a.k2 > b.k2 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 or a.k2 > b.k2 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 or a.k2 > 0 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left join ${tbName2} b + sql"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 < b.k1 or a.k2 > 0 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_left_join5"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 + qt_left_join5"""select ${s} from ${tbName1} a left join ${tbName2} b on a.k1 = b.k1 left join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" } @@ -260,22 +218,10 @@ suite("test_join", "query,p0") { for (s in selected) { qt_left_outer_join4"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 > b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 > 0 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } test { sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" @@ -284,52 +230,22 @@ suite("test_join", "query,p0") { logger.info(exception.message) } } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 or a.k2 = b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_left_outer_join5"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 + qt_left_outer_join5"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 left outer join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" } @@ -341,24 +257,12 @@ suite("test_join", "query,p0") { for (s in selected) { qt_right_join2"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 > b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_right_join3"""select ${s} from ${tbName1} a right join ${tbName2} b + qt_right_join3"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 and a.k2 > 0 order by 1, 2, 3, 4, 5 limit 65535""" test { sql"""select ${s} from ${tbName1} a right join ${tbName2} b @@ -368,52 +272,22 @@ suite("test_join", "query,p0") { assertTrue(exception != null) } } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 or a.k2 = b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 < b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right join ${tbName2} b + sql"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 < b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_right_join4"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 + qt_right_join4"""select ${s} from ${tbName1} a right join ${tbName2} b on a.k1 = b.k1 right join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" } qt_right_join5"""select ${i} from ${tbName1} a right join ${tbName2} b on a.k2 = b.k2 and a.k1 > 0 @@ -427,25 +301,13 @@ suite("test_join", "query,p0") { for (s in selected) { qt_right_outer_join2"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 > b.k1 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_right_outer_join3"""select ${s} from ${tbName1} a right outer join ${tbName2} b + qt_right_outer_join3"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 and a.k2 > 0 order by 1, 2, 3, 4, 5 limit 65535""" test { sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b @@ -455,52 +317,22 @@ suite("test_join", "query,p0") { logger.info(exception.message) } } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 or a.k2 = b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - qt_right_outer_join4"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 + qt_right_outer_join4"""select ${s} from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 right outer join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" } @@ -519,22 +351,10 @@ suite("test_join", "query,p0") { order by 1, 2, 3, 4, 5 limit 65535""" sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 order by 1, 2, 3, 4, 5 limit 65535""" - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 > b.k1 + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 > b.k1 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 > 0 + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 > 0 order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } test { sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" @@ -543,52 +363,22 @@ suite("test_join", "query,p0") { logger.info(exception.message) } } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 or a.k2 = b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > b.k2 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - test { - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 < b.k1 or a.k2 > 0 where a.k2 > 0 and b.k3 != 0 and a.k6 > "000" order by 1, 2, 3, 4, 5 limit 65535""" - check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) - } - } - sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 + sql"""select ${s} from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 full outer join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" sql"""select ${s} from ${tbName1} a left outer join ${tbName2} b on a.k1 = b.k1 left outer join ${tbName3} c on a.k2 = c.k2 order by 1, 2, 3, 4, 5 limit 65535""" diff --git a/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy new file mode 100644 index 0000000000..5ed53d0402 --- /dev/null +++ b/regression-test/suites/query_p0/join/test_nestedloop_outer_join.groovy @@ -0,0 +1,110 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nestedloop_outer_join", "query_p0") { + def tbl1 = "test_nestedloop_outer_join1" + def tbl2 = "test_nestedloop_outer_join2" + + sql "DROP TABLE IF EXISTS ${tbl1}" + sql """ + CREATE TABLE IF NOT EXISTS ${tbl1} ( + `user_id` LARGEINT NOT NULL COMMENT "", + `user_id2` LARGEINT NOT NULL COMMENT "" + ) + DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); + """ + + sql "DROP TABLE IF EXISTS ${tbl2}" + sql """ + CREATE TABLE IF NOT EXISTS ${tbl2} ( + `user_id` LARGEINT NOT NULL COMMENT "", + `user_id2` LARGEINT NOT NULL COMMENT "" + ) + DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); + """ + + qt_join """ + select * from ${tbl1} full outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} right outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} left outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} inner join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + sql """ INSERT INTO ${tbl1} VALUES (1, 1), (2, 2), (3, 3), (10, 10); """ + qt_join """ + select * from ${tbl1} full outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} right outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} left outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} inner join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl2} full outer join ${tbl1} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl2} right outer join ${tbl1} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl2} left outer join ${tbl1} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl2} inner join ${tbl1} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + + sql """ INSERT INTO ${tbl2} VALUES (2, 2), (3, 3), (4, 4), (0, 0); """ + qt_join """ + select * from ${tbl1} full outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} right outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} left outer join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + qt_join """ + select * from ${tbl1} inner join ${tbl2} on ${tbl1}.user_id < ${tbl2}.user_id order by ${tbl1}.user_id, ${tbl2}.user_id; + """ + + + + sql "DROP TABLE IF EXISTS ${tbl1}" + sql "DROP TABLE IF EXISTS ${tbl2}" +} diff --git a/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q11.groovy b/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q11.groovy index dc05e2e347..d8e5bc2dae 100644 --- a/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q11.groovy +++ b/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q11.groovy @@ -55,10 +55,11 @@ suite("test_explain_tpch_sf_1_q11") { """ check { explainStr -> - explainStr.contains("VTOP-N\n" + - " | order by: `\$a\$1`.`\$c\$2` DESC") && - explainStr.contains("cross join:\n" + - " | predicates: sum( * ) > sum( * ) * 0.0001") && + explainStr.contains("VTOP-N\n" + + " | order by: `\$a\$1`.`\$c\$2` DESC") && + explainStr.contains("CROSS JOIN\n" + + " | join op: CROSS JOIN()\n" + + " | other predicates: > * 0.0001") && explainStr.contains("VAGGREGATE (merge finalize)\n" + " | output: sum( sum( * ))\n" + " | group by: `ps_partkey`") && diff --git a/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q22.groovy b/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q22.groovy index 29b4d6e9a8..e572aee35a 100644 --- a/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q22.groovy +++ b/regression-test/suites/tpch_sf1_p1/tpch_sf1/explain/test_q22.groovy @@ -64,23 +64,22 @@ suite("test_explain_tpch_sf_1_q22") { explainStr -> explainStr.contains("VTOP-N\n" + " | order by: `cntrycode` ASC") && - explainStr.contains("VAGGREGATE (merge finalize)\n" + - " | output: count( count(*)), sum( sum())\n" + - " | group by: `cntrycode`") && - explainStr.contains("VAGGREGATE (update serialize)\n" + - " | STREAMING\n" + - " | output: count(*), sum()\n" + - " | group by: substr(, 1, 2)") && - explainStr.contains("join op: LEFT ANTI JOIN(BROADCAST)[Tables are not in the same group]\n" + - " | equal join conjunct: `c_custkey` = `o_custkey`") && - explainStr.contains("vec output tuple id: 10") && - explainStr.contains("output slot ids: 36 37 \n" + - " | hash output slot ids: 25 26 ") && - explainStr.contains("cross join:\n" + - " | predicates: `c_acctbal` > avg(`c_acctbal`)") && - explainStr.contains("TABLE: default_cluster:regression_test_tpch_sf1_p1_tpch_sf1.customer(customer), PREAGGREGATION: ON\n" + - " PREDICATES: substr(`c_phone`, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')") && - explainStr.contains("TABLE: default_cluster:regression_test_tpch_sf1_p1_tpch_sf1.orders(orders), PREAGGREGATION: ON") && + explainStr.contains("VAGGREGATE (merge finalize)\n" + + " | output: count( count(*)), sum( sum())\n" + + " | group by: `cntrycode`") && + explainStr.contains("VAGGREGATE (update serialize)\n" + + " | STREAMING\n" + + " | output: count(*), sum()\n" + + " | group by: substr(, 1, 2)") && + explainStr.contains("vec output tuple id: 11") && + explainStr.contains("output slot ids: 40 41 \n" + + " | hash output slot ids: 36 37 ") && + explainStr.contains("VCROSS JOIN\n" + + " | join op: CROSS JOIN()\n" + + " | other predicates: > ") && + explainStr.contains("TABLE: default_cluster:regression_test_tpch_sf1_p1_tpch_sf1.customer(customer), PREAGGREGATION: ON\n" + + " PREDICATES: substr(`c_phone`, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')") && + explainStr.contains("TABLE: default_cluster:regression_test_tpch_sf1_p1_tpch_sf1.orders(orders), PREAGGREGATION: ON") && explainStr.contains("VAGGREGATE (merge finalize)\n" + " | output: avg( avg(`c_acctbal`))\n" + " | group by: ") &&