[Improvement](join) Support nested loop outer join (#13965)

This commit is contained in:
Gabriel
2022-11-10 19:50:46 +08:00
committed by GitHub
parent 6c13126e5c
commit 1ef85ae1f2
25 changed files with 1903 additions and 1351 deletions

View File

@ -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));

View File

@ -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; }

View File

@ -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

View File

@ -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<JoinOpType>::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<SlotId> {}),
_intermediate_row_desc(
descs, tnode.hash_join_node.vintermediate_tuple_id_list,
std::vector<bool>(tnode.hash_join_node.vintermediate_tuple_id_list.size())),
_output_row_desc(descs, {tnode.hash_join_node.voutput_tuple_id}, {false}) {
: std::vector<SlotId> {}) {
_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<std::integral_constant<TJoinOp::type, TJoinOp::NAME>>(); \
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<int64_t>(&RuntimeProfile::units_per_second, _rows_returned_counter,
runtime_profile()->total_time_counter()),
"");
_mem_tracker = std::make_unique<MemTracker>("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<Status> 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>* 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<uint16_t> 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<ColumnNullable&>(*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();

View File

@ -20,17 +20,18 @@
#include <variant>
#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 <typename RowRefListType>
@ -160,30 +161,6 @@ using HashTableVariants = std::variant<
I256FixedKeyHashTableContext<true, RowRefListWithFlags>,
I256FixedKeyHashTableContext<false, RowRefListWithFlags>>;
using JoinOpVariants =
std::variant<std::integral_constant<TJoinOp::type, TJoinOp::INNER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_SEMI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_ANTI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::FULL_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::CROSS_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_SEMI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_ANTI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN>>;
#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<TJoinOp::type, TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN>>>;
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<VExprContext*>;
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<VExprContext*> _vother_join_conjunct_ptr;
// output expr
VExprContexts _output_expr_ctxs;
// mark the join column whether support null eq
std::vector<bool> _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<SlotId> _hash_output_slot_ids;
std::vector<bool> _left_output_slot_flags;
std::vector<bool> _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);
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);

View File

@ -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 <sstream>
#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<bool>(
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<ColumnNullable&>(*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<Status> 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>* 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<std::integral_constant<TJoinOp::type, TJoinOp::NAME>>(); \
break;
APPLY_FOR_JOINOP_VARIANTS(M);
#undef M
default:
//do nothing
break;
}
}
} // namespace doris::vectorized

View File

@ -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 <future>
#include <string>
#include <variant>
#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<TJoinOp::type, TJoinOp::INNER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_SEMI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_ANTI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::LEFT_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::FULL_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_OUTER_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::CROSS_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_SEMI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::RIGHT_ANTI_JOIN>,
std::integral_constant<TJoinOp::type, TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN>>;
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>* 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<VExprContext*> _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

View File

@ -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 <sstream>
#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<uint16_t> 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<set_build_side_flag,
set_probe_side_flag>(
&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<false>(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<set_build_side_flag,
set_probe_side_flag>(
&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<true>(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<ColumnNullable*>(dst_columns[i].get())
->get_nested_column_ptr()
->insert_many_from(*src_column.column, _left_block_pos, max_added_rows);
assert_cast<ColumnNullable*>(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<ColumnNullable*>(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<ColumnNullable*>(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 <bool BuildSide>
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<ColumnUInt8*>(_build_side_visited_flags[i].get())
->get_data()
.data();
const auto num_rows = cur_block.rows();
std::vector<int> 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<ColumnNullable*>(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<ColumnNullable*>(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<ColumnNullable*>(dst_columns[i].get())
->get_nested_column_ptr()
->insert_many_from(*src_column.column, _left_block_pos, 1);
assert_cast<ColumnNullable*>(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 <bool SetBuildSideFlag, bool SetProbeSideFlag>
Status VNestedLoopJoinNode::_do_filtering_and_update_visited_flags(
Block* block, std::stack<uint16_t>& 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<ColumnNullable>(*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<ColumnUInt8*>(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<ColumnUInt8*>(
_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<uint8>(filter_data, size, 1);
}
Block::filter_block_internal(block, filter, column_to_keep);
} else if (auto* const_column = check_and_get_column<ColumnConst>(*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<ColumnUInt8*>(
_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<void*>(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<const doris::vectorized::ColumnVector<UInt8>&>(*filter_column)
.get_data();
if constexpr (SetBuildSideFlag) {
for (size_t i = 0; i < processed_blocks_num; i++) {
auto& build_side_flag =
assert_cast<ColumnUInt8*>(
_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<uint8>(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

View File

@ -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 <boost/thread.hpp>
#include <future>
#include <stack>
#include <string>
#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>* 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 <bool SetBuildSideFlag, bool SetProbeSideFlag>
Status _do_filtering_and_update_visited_flags(Block* block, std::stack<uint16_t>& offset_stack);
template <bool BuildSide>
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

View File

@ -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 <sstream>
#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<MonotonicStopWatch> 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>* 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<Status> 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

View File

@ -691,10 +691,87 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
.collect(Collectors.toList());
CrossJoinNode crossJoinNode = new CrossJoinNode(context.nextPlanNodeId(),
leftFragmentPlanRoot, rightFragmentPlanRoot, tupleIds);
leftFragmentPlanRoot, rightFragmentPlanRoot, tupleIds, null, null, null);
if (nestedLoopJoin.getStats() != null) {
crossJoinNode.setCardinality((long) nestedLoopJoin.getStats().getRowCount());
}
Map<ExprId, SlotReference> leftChildOutputMap = Maps.newHashMap();
nestedLoopJoin.child(0).getOutput().stream()
.map(SlotReference.class::cast)
.forEach(s -> leftChildOutputMap.put(s.getExprId(), s));
Map<ExprId, SlotReference> rightChildOutputMap = Maps.newHashMap();
nestedLoopJoin.child(1).getOutput().stream()
.map(SlotReference.class::cast)
.forEach(s -> rightChildOutputMap.put(s.getExprId(), s));
// make intermediate tuple
List<SlotDescriptor> leftIntermediateSlotDescriptor = Lists.newArrayList();
List<SlotDescriptor> 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<SlotDescriptor> leftSlotDescriptors = leftTuples.stream()
.map(TupleDescriptor::getSlots)
.flatMap(Collection::stream)
.collect(Collectors.toList());
List<SlotDescriptor> rightSlotDescriptors = rightTuples.stream()
.map(TupleDescriptor::getSlots)
.flatMap(Collection::stream)
.collect(Collectors.toList());
Map<ExprId, SlotReference> outputSlotReferenceMap = Maps.newHashMap();
nestedLoopJoin.getOutput().stream()
.map(SlotReference.class::cast)
.forEach(s -> outputSlotReferenceMap.put(s.getExprId(), s));
List<SlotReference> 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<PlanFragment, Pla
nestedLoopJoin.getOtherJoinConjuncts().stream()
.map(e -> ExpressionTranslator.translate(e, context)).forEach(crossJoinNode::addConjunct);
if (nestedLoopJoin.isShouldTranslateOutput()) {
// translate output expr on intermediate tuple
List<Expr> 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<PlanFragment, Pla
if (project.child(0) instanceof PhysicalHashJoin) {
((PhysicalHashJoin<?, ?>) 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<PlanFragment, Pla
hashJoinNode.setvSrcToOutputSMap(execExprList);
return inputFragment;
}
if (inputPlanNode instanceof CrossJoinNode) {
CrossJoinNode crossJoinNode = (CrossJoinNode) inputPlanNode;
crossJoinNode.setvOutputTupleDesc(tupleDescriptor);
crossJoinNode.setvSrcToOutputSMap(execExprList);
return inputFragment;
}
inputPlanNode.setProjectList(execExprList);
inputPlanNode.setOutputTupleDesc(tupleDescriptor);

View File

@ -48,6 +48,8 @@ public abstract class AbstractPhysicalJoin<
protected final ImmutableList<Expression> 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;
}

View File

@ -42,8 +42,6 @@ public class PhysicalHashJoin<
RIGHT_CHILD_TYPE extends Plan>
extends AbstractPhysicalJoin<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> {
private boolean shouldTranslateOutput = true;
// TODO: What's purpose? it's alway empty.
private final List<Expression> 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<Expression> getFilterConjuncts() {
return filterConjuncts;
}

View File

@ -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<SlotId> computeInputSlotIds(Analyzer analyzer) throws NotImplementedException {
Set<SlotId> 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<SlotRef> srcSlotRefList = Lists.newArrayList();
srcExpr.collect(SlotRef.class, srcSlotRefList);
result.addAll(srcSlotRefList.stream().map(e -> e.getSlotId()).collect(Collectors.toList()));
}
}
}
// conjunct
List<SlotId> conjunctSlotIds = Lists.newArrayList();
Expr.getIds(conjuncts, null, conjunctSlotIds);
result.addAll(conjunctSlotIds);
return result;
}
@Override
protected Pair<Boolean, Boolean> 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<TupleId> tupleIds) {
super(id, "CROSS JOIN", StatisticalType.CROSS_JOIN_NODE);
this.innerRef = null;
public CrossJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, List<TupleId> tupleIds,
List<Expr> 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());
}
}

View File

@ -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<BinaryPredicate> 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<SlotId> hashOutputSlotIds = new ArrayList<>(); //init for nereids
private TupleDescriptor vOutputTupleDesc;
private ExprSubstitutionMap vSrcToOutputSMap;
private List<TupleDescriptor> vIntermediateTupleDescList;
/**
* Constructor of HashJoinNode.
*/
public HashJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, TableRef innerRef, List<Expr> eqJoinConjuncts,
List<Expr> 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<Expr> eqJoinConjuncts, List<Expr> otherJoinConjuncts, List<Expr> 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<SlotId> requiredSlotIdSet, Analyzer analyzer) {
outputSlotIds = Lists.newArrayList();
List<TupleDescriptor> 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<SlotDescriptor> 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<Expr> 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<TupleId> 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<TupleId> 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<Expr> 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<Expr> tupleIsNullLhs = TupleIsNullPredicate
.wrapExprs(vSrcToOutputSMap.getLhs().subList(rightBeginIndex, vSrcToOutputSMap.size()),
new ArrayList<>(), TNullSide.RIGHT, analyzer);
List<Expr> 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<Expr> lhs = new ArrayList<>();
List<Expr> 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<Boolean, Boolean> 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: <originslot, intermediateslot>
ExprSubstitutionMap originToIntermediateSmap = new ExprSubstitutionMap();
Map<List<TupleId>, 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.
* <p>
* If you read the properties of @tupleids directly instead of this function,
* it reads the input id of the current node.
*/
@Override
public ArrayList<TupleId> getTupleIds() {
Preconditions.checkState(tupleIds != null);
if (vOutputTupleDesc != null) {
return Lists.newArrayList(vOutputTupleDesc.getId());
}
return tupleIds;
}
@Override
public ArrayList<TupleId> 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<TupleId> 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<SlotRef> 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<Expr> otherJoinConjuncts) {
this.otherJoinConjuncts = otherJoinConjuncts;
}
/**
* Used by nereids.
*/
public void setvOutputTupleDesc(TupleDescriptor vOutputTupleDesc) {
this.vOutputTupleDesc = vOutputTupleDesc;
}
/**
* Used by nereids.
*/
public void setvIntermediateTupleDescList(List<TupleDescriptor> vIntermediateTupleDescList) {
this.vIntermediateTupleDescList = vIntermediateTupleDescList;
}
/**
* Used by nereids.
*/
public void setvSrcToOutputSMap(List<Expr> lhs) {
this.vSrcToOutputSMap = new ExprSubstitutionMap(lhs, Collections.emptyList());
}
}

View File

@ -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<TupleDescriptor> 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<SlotRef> 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<TupleId> 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<TupleId> 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<Expr> 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<Expr> tupleIsNullLhs = TupleIsNullPredicate
.wrapExprs(vSrcToOutputSMap.getLhs().subList(rightBeginIndex, vSrcToOutputSMap.size()),
new ArrayList<>(), TNullSide.RIGHT, analyzer);
List<Expr> 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<Expr> lhs = new ArrayList<>();
List<Expr> 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<SlotId> requiredSlotIdSet, Analyzer analyzer) {
outputSlotIds = Lists.newArrayList();
List<TupleDescriptor> 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<SlotDescriptor> 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<Boolean, Boolean> 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<Boolean, Boolean> tmpPair = needToCopyRightAndLeft();
boolean copyleft = tmpPair.first;
boolean copyRight = tmpPair.second;
// 2. exprsmap: <originslot, intermediateslot>
ExprSubstitutionMap originToIntermediateSmap = new ExprSubstitutionMap();
Map<List<TupleId>, 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.
* <p>
* If you read the properties of @tupleids directly instead of this function,
* it reads the input id of the current node.
*/
@Override
public ArrayList<TupleId> getTupleIds() {
Preconditions.checkState(tupleIds != null);
if (vOutputTupleDesc != null) {
return Lists.newArrayList(vOutputTupleDesc.getId());
}
return tupleIds;
}
@Override
public ArrayList<TupleId> 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<TupleId> 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<TupleDescriptor> vIntermediateTupleDescList) {
this.vIntermediateTupleDescList = vIntermediateTupleDescList;
}
/**
* Used by nereids.
*/
public void setvSrcToOutputSMap(List<Expr> lhs) {
this.vSrcToOutputSMap = new ExprSubstitutionMap(lhs, Collections.emptyList());
}
}

View File

@ -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<Expr> 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,

View File

@ -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: <slot 3> sum(`salary`) > <slot 8> 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: <slot 3> sum(`salary`) > <slot 8> avg(`salary`)",
"order by: <slot 10> `$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: <slot 3> sum(`salary`) > <slot 8> avg(`salary`)",
"order by: <slot 10> `$a$1`.`$c$2` ASC",
"OUTPUT EXPRS:\n <slot 11> `$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: <slot 3> sum(`salary`) > <slot 8> avg(`salary`)",
"order by: <slot 10> `$a$1`.`$c$2` ASC",
"OUTPUT EXPRS:\n <slot 11> `$a$1`.`$c$1`\n <slot 10> `$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: <slot 3> sum(`salary`) > <slot 8> avg(`salary`)",
"order by: <slot 10> `$a$1`.`$c$2` ASC",
"OUTPUT EXPRS:\n <slot 11> `$a$1`.`$c$1`\n <slot 10> `$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: <slot 3> sum(`salary`) > <slot 9> avg(`salary`)",
"PREDICATES: `empid` >= 1, `empid` <= 2");
.explainContains("CROSS JOIN");
}
@AfterClass

View File

@ -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"));
}

View File

@ -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<Exprs.TExpr> srcExprList
3: optional Types.TTupleId voutput_tuple_id
4: optional list<Types.TTupleId> vintermediate_tuple_id_list
}
struct TMergeJoinNode {
// anything from the ON, USING or WHERE clauses that's an equi-join predicate
1: required list<TEqJoinCondition> 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<Exprs.TExpr> projections
102: optional Types.TTupleId output_tuple_id

View File

@ -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

View File

@ -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"""

View File

@ -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}"
}

View File

@ -55,10 +55,11 @@ suite("test_explain_tpch_sf_1_q11") {
"""
check {
explainStr ->
explainStr.contains("VTOP-N\n" +
" | order by: <slot 22> `\$a\$1`.`\$c\$2` DESC") &&
explainStr.contains("cross join:\n" +
" | predicates: <slot 9> sum(<slot 31> * <slot 32>) > <slot 20> sum(<slot 43> * <slot 44>) * 0.0001") &&
explainStr.contains("VTOP-N\n" +
" | order by: <slot 22> `\$a\$1`.`\$c\$2` DESC") &&
explainStr.contains("CROSS JOIN\n" +
" | join op: CROSS JOIN()\n" +
" | other predicates: <slot 76> > <slot 77> * 0.0001") &&
explainStr.contains("VAGGREGATE (merge finalize)\n" +
" | output: sum(<slot 9> sum(<slot 31> * <slot 32>))\n" +
" | group by: <slot 8> `ps_partkey`") &&

View File

@ -64,23 +64,22 @@ suite("test_explain_tpch_sf_1_q22") {
explainStr ->
explainStr.contains("VTOP-N\n" +
" | order by: <slot 32> <slot 29> `cntrycode` ASC") &&
explainStr.contains("VAGGREGATE (merge finalize)\n" +
" | output: count(<slot 30> count(*)), sum(<slot 31> sum(<slot 37>))\n" +
" | group by: <slot 29> `cntrycode`") &&
explainStr.contains("VAGGREGATE (update serialize)\n" +
" | STREAMING\n" +
" | output: count(*), sum(<slot 37>)\n" +
" | group by: substr(<slot 36>, 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` > <slot 3> 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(<slot 30> count(*)), sum(<slot 31> sum(<slot 41>))\n" +
" | group by: <slot 29> `cntrycode`") &&
explainStr.contains("VAGGREGATE (update serialize)\n" +
" | STREAMING\n" +
" | output: count(*), sum(<slot 41>)\n" +
" | group by: substr(<slot 40>, 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: <slot 45> > <slot 46>") &&
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(<slot 2> avg(`c_acctbal`))\n" +
" | group by: ") &&