diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 9161c9e6f661b9..86de246a6031b1 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -22,6 +22,8 @@ #include #include +#include +#include #include #include #include @@ -34,10 +36,12 @@ #include "pipeline/exec/sort_source_operator.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" #include "pipeline/pipeline_fragment_context.h" +#include "pipeline/shuffle/exchange_writer.h" #include "util/runtime_profile.h" #include "util/uid_util.h" #include "vec/columns/column_const.h" #include "vec/exprs/vexpr.h" +#include "vec/sink/scale_writer_partitioning_exchanger.hpp" #include "vec/sink/tablet_sink_hash_partitioner.h" namespace doris::pipeline { @@ -65,6 +69,8 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _distribute_rows_into_channels_timer = ADD_TIMER(custom_profile(), "DistributeRowsIntoChannelsTime"); _send_new_partition_timer = ADD_TIMER(custom_profile(), "SendNewPartitionTime"); + _add_partition_request_timer = + ADD_CHILD_TIMER(custom_profile(), "AddPartitionRequestTime", "SendNewPartitionTime"); _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(custom_profile(), "BlocksProduced", TUnit::UNIT, 1); _overall_throughput = custom_profile()->add_derived_counter( @@ -141,11 +147,12 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf custom_profile()->add_info_string( "Partitioner", fmt::format("Crc32HashPartitioner({})", _partition_count)); } else if (_part_type == TPartitionType::OLAP_TABLE_SINK_HASH_PARTITIONED) { + // in ExchangeOlapWriter we rely on type of _partitioner here _partition_count = channels.size(); custom_profile()->add_info_string( "Partitioner", fmt::format("TabletSinkHashPartitioner({})", _partition_count)); _partitioner = std::make_unique( - _partition_count, p._tablet_sink_txn_id, p._tablet_sink_schema, + cast_set(_partition_count), p._tablet_sink_txn_id, p._tablet_sink_schema, p._tablet_sink_partition, p._tablet_sink_location, p._tablet_sink_tuple_id, this); RETURN_IF_ERROR(_partitioner->init({})); RETURN_IF_ERROR(_partitioner->prepare(state, {})); @@ -217,7 +224,11 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); RETURN_IF_ERROR(Base::open(state)); - _writer = std::make_unique(); + if (_part_type == TPartitionType::OLAP_TABLE_SINK_HASH_PARTITIONED) { + _writer = std::make_unique(*this); + } else { + _writer = std::make_unique(*this); + } for (auto& channel : channels) { RETURN_IF_ERROR(channel->open(state)); @@ -365,7 +376,8 @@ Status ExchangeSinkOperatorX::_handle_eof_channel(RuntimeState* state, ChannelPt Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); - COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); + COUNTER_UPDATE(local_state.rows_input_counter(), + (int64_t)block->rows()); // for auto-partition, may decease when do_partitioning SCOPED_TIMER(local_state.exec_time_counter()); bool all_receiver_eof = true; for (auto& channel : local_state.channels) { @@ -488,7 +500,7 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block if (!local_state.local_channel_ids.empty()) { const auto& ids = local_state.local_channel_ids; // Find the first channel ID >= current_channel_idx - auto it = std::lower_bound(ids.begin(), ids.end(), local_state.current_channel_idx); + auto it = std::ranges::lower_bound(ids, local_state.current_channel_idx); if (it != ids.end()) { local_state.current_channel_idx = *it; } else { @@ -501,9 +513,9 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block (local_state.current_channel_idx + 1) % local_state.channels.size(); } else if (_part_type == TPartitionType::HASH_PARTITIONED || _part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED || - _part_type == TPartitionType::OLAP_TABLE_SINK_HASH_PARTITIONED || - _part_type == TPartitionType::HIVE_TABLE_SINK_HASH_PARTITIONED) { - RETURN_IF_ERROR(local_state._writer->write(&local_state, state, block, eos)); + _part_type == TPartitionType::HIVE_TABLE_SINK_HASH_PARTITIONED || + _part_type == TPartitionType::OLAP_TABLE_SINK_HASH_PARTITIONED) { + RETURN_IF_ERROR(local_state._writer->write(state, block, eos)); } else if (_part_type == TPartitionType::HIVE_TABLE_SINK_UNPARTITIONED) { // Control the number of channels according to the flow, thereby controlling the number of table sink writers. RETURN_IF_ERROR(send_to_current_channel()); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 1f8581923ea939..8b47f1dfd526b3 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -19,6 +19,7 @@ #include +#include #include #include #include @@ -26,8 +27,7 @@ #include "common/status.h" #include "exchange_sink_buffer.h" #include "operator.h" -#include "pipeline/shuffle/writer.h" -#include "vec/sink/scale_writer_partitioning_exchanger.hpp" +#include "pipeline/shuffle/exchange_writer.h" #include "vec/sink/vdata_stream_sender.h" namespace doris { @@ -66,8 +66,9 @@ class ExchangeSinkLocalState MOCK_REMOVE(final) : public PipelineXSinkLocalState if (_queue_dependency) { dep_vec.push_back(_queue_dependency.get()); } - std::for_each(_local_channels_dependency.begin(), _local_channels_dependency.end(), - [&](std::shared_ptr dep) { dep_vec.push_back(dep.get()); }); + std::ranges::for_each(_local_channels_dependency, [&](std::shared_ptr dep) { + dep_vec.push_back(dep.get()); + }); return dep_vec; } Status init(RuntimeState* state, LocalSinkStateInfo& info) override; @@ -168,7 +169,7 @@ class ExchangeSinkLocalState MOCK_REMOVE(final) : public PipelineXSinkLocalState */ std::vector> _local_channels_dependency; std::unique_ptr _partitioner; - std::unique_ptr _writer; + std::unique_ptr _writer; size_t _partition_count; std::shared_ptr _finish_dependency; diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp index dd244fedf57160..3f28bc6179363b 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -551,7 +551,7 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, vectorized:: } std::vector> partition_indexes(_partition_count); - const auto* channel_ids = local_state._partitioner->get_channel_ids().get(); + const auto& channel_ids = local_state._partitioner->get_channel_ids(); for (uint32_t i = 0; i != rows; ++i) { partition_indexes[channel_ids[i]].emplace_back(i); } diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index fb05e6bc8b3c00..b0595952a49636 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -199,7 +199,7 @@ Status PartitionedHashJoinSinkLocalState::_execute_spill_unpartitioned_block( (void)_partitioner->do_partitioning(state, &sub_block); } - const auto* channel_ids = _partitioner->get_channel_ids().get(); + const auto& channel_ids = _partitioner->get_channel_ids(); for (size_t i = 0; i != sub_block.rows(); ++i) { partitions_indexes[channel_ids[i]].emplace_back(i); } @@ -435,7 +435,7 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, auto& p = _parent->cast(); SCOPED_TIMER(_partition_shuffle_timer); - const auto* channel_ids = _partitioner->get_channel_ids().get(); + const auto& channel_ids = _partitioner->get_channel_ids(); std::vector> partition_indexes(p._partition_count); DCHECK_LT(begin, end); for (size_t i = begin; i != end; ++i) { diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp index 8b76900bfdb519..1937b111975cf2 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp @@ -153,11 +153,14 @@ Status LocalExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* if (state->low_memory_mode()) { set_low_memory_mode(state); } + SinkInfo sink_info = {.channel_id = &local_state._channel_id, + .partitioner = local_state._partitioner.get(), + .local_state = &local_state, + .shuffle_idx_to_instance_idx = &_shuffle_idx_to_instance_idx}; RETURN_IF_ERROR(local_state._exchanger->sink( state, in_block, eos, {local_state._compute_hash_value_timer, local_state._distribute_timer, nullptr}, - {&local_state._channel_id, local_state._partitioner.get(), &local_state, - &_shuffle_idx_to_instance_idx})); + sink_info)); // If all exchange sources ended due to limit reached, current task should also finish if (local_state._exchanger->_running_source_operators == 0) { diff --git a/be/src/pipeline/local_exchange/local_exchanger.cpp b/be/src/pipeline/local_exchange/local_exchanger.cpp index d1d6b7387a0be8..eb211364705769 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.cpp +++ b/be/src/pipeline/local_exchange/local_exchanger.cpp @@ -113,7 +113,7 @@ bool Exchanger::_dequeue_data(BlockType& block, bool* eos, vectorized } Status ShuffleExchanger::sink(RuntimeState* state, vectorized::Block* in_block, bool eos, - Profile&& profile, SinkInfo&& sink_info) { + Profile&& profile, SinkInfo& sink_info) { if (in_block->empty()) { return Status::OK(); } @@ -123,8 +123,8 @@ Status ShuffleExchanger::sink(RuntimeState* state, vectorized::Block* in_block, } { SCOPED_TIMER(profile.distribute_timer); - RETURN_IF_ERROR(_split_rows(state, sink_info.partitioner->get_channel_ids().get(), - in_block, *sink_info.channel_id, sink_info.local_state, + RETURN_IF_ERROR(_split_rows(state, sink_info.partitioner->get_channel_ids(), in_block, + *sink_info.channel_id, sink_info.local_state, sink_info.shuffle_idx_to_instance_idx)); } @@ -172,7 +172,7 @@ Status ShuffleExchanger::get_block(RuntimeState* state, vectorized::Block* block return Status::OK(); } -Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, +Status ShuffleExchanger::_split_rows(RuntimeState* state, const std::vector& channel_ids, vectorized::Block* block, int channel_id, LocalExchangeSinkLocalState* local_state, std::map* shuffle_idx_to_instance_idx) { @@ -223,8 +223,10 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest uint32_t size = partition_rows_histogram[it.first + 1] - start; if (size > 0) { enqueue_rows += size; - _enqueue_data_and_set_ready(it.second, local_state, - {new_block_wrapper, {row_idx, start, size}}); + _enqueue_data_and_set_ready( + it.second, local_state, + {new_block_wrapper, + {.row_idxs = row_idx, .offset_start = start, .length = size}}); } } if (enqueue_rows != rows) [[unlikely]] { @@ -243,7 +245,7 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest return Status::OK(); } -Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, +Status ShuffleExchanger::_split_rows(RuntimeState* state, const std::vector& channel_ids, vectorized::Block* block, int channel_id) { const auto rows = cast_set(block->rows()); auto row_idx = std::make_shared>(rows); @@ -276,7 +278,9 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest uint32_t start = partition_rows_histogram[i]; uint32_t size = partition_rows_histogram[i + 1] - start; if (size > 0) { - _enqueue_data_and_set_ready(i, {new_block_wrapper, {row_idx, start, size}}); + _enqueue_data_and_set_ready( + i, {new_block_wrapper, + {.row_idxs = row_idx, .offset_start = start, .length = size}}); } } @@ -284,7 +288,7 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest } Status PassthroughExchanger::sink(RuntimeState* state, vectorized::Block* in_block, bool eos, - Profile&& profile, SinkInfo&& sink_info) { + Profile&& profile, SinkInfo& sink_info) { if (in_block->empty()) { return Status::OK(); } @@ -336,7 +340,7 @@ Status PassthroughExchanger::get_block(RuntimeState* state, vectorized::Block* b } Status PassToOneExchanger::sink(RuntimeState* state, vectorized::Block* in_block, bool eos, - Profile&& profile, SinkInfo&& sink_info) { + Profile&& profile, SinkInfo& sink_info) { if (in_block->empty()) { return Status::OK(); } @@ -377,7 +381,7 @@ void ExchangerBase::finalize() { } Status BroadcastExchanger::sink(RuntimeState* state, vectorized::Block* in_block, bool eos, - Profile&& profile, SinkInfo&& sink_info) { + Profile&& profile, SinkInfo& sink_info) { if (in_block->empty()) { return Status::OK(); } @@ -390,8 +394,9 @@ Status BroadcastExchanger::sink(RuntimeState* state, vectorized::Block* in_block std::move(new_block), sink_info.local_state ? sink_info.local_state->_shared_state : nullptr, -1); for (int i = 0; i < _num_partitions; i++) { - _enqueue_data_and_set_ready(i, sink_info.local_state, - {wrapper, {0, wrapper->_data_block.rows()}}); + _enqueue_data_and_set_ready( + i, sink_info.local_state, + {wrapper, {.offset_start = 0, .length = wrapper->_data_block.rows()}}); } return Status::OK(); @@ -429,7 +434,7 @@ Status BroadcastExchanger::get_block(RuntimeState* state, vectorized::Block* blo Status AdaptivePassthroughExchanger::_passthrough_sink(RuntimeState* state, vectorized::Block* in_block, - SinkInfo&& sink_info) { + SinkInfo& sink_info) { vectorized::Block new_block; if (!_free_blocks.try_dequeue(new_block)) { new_block = {in_block->clone_empty()}; @@ -449,7 +454,7 @@ Status AdaptivePassthroughExchanger::_passthrough_sink(RuntimeState* state, } Status AdaptivePassthroughExchanger::_shuffle_sink(RuntimeState* state, vectorized::Block* block, - SinkInfo&& sink_info) { + SinkInfo& sink_info) { std::vector channel_ids; const auto num_rows = block->rows(); channel_ids.resize(num_rows, 0); @@ -467,13 +472,13 @@ Status AdaptivePassthroughExchanger::_shuffle_sink(RuntimeState* state, vectoriz sink_info.local_state->_memory_used_counter->set( sink_info.local_state->_shared_state->mem_usage); - RETURN_IF_ERROR(_split_rows(state, channel_ids.data(), block, std::move(sink_info))); + RETURN_IF_ERROR(_split_rows(state, channel_ids, block, sink_info)); return Status::OK(); } Status AdaptivePassthroughExchanger::_split_rows(RuntimeState* state, - const uint32_t* __restrict channel_ids, - vectorized::Block* block, SinkInfo&& sink_info) { + const std::vector& channel_ids, + vectorized::Block* block, SinkInfo& sink_info) { const auto rows = cast_set(block->rows()); auto row_idx = std::make_shared>(rows); auto& partition_rows_histogram = _partition_rows_histogram[*sink_info.channel_id]; @@ -512,17 +517,17 @@ Status AdaptivePassthroughExchanger::_split_rows(RuntimeState* state, } Status AdaptivePassthroughExchanger::sink(RuntimeState* state, vectorized::Block* in_block, - bool eos, Profile&& profile, SinkInfo&& sink_info) { + bool eos, Profile&& profile, SinkInfo& sink_info) { if (in_block->empty()) { return Status::OK(); } if (_is_pass_through) { - return _passthrough_sink(state, in_block, std::move(sink_info)); + return _passthrough_sink(state, in_block, sink_info); } else { if (++_total_block >= _num_partitions) { _is_pass_through = true; } - return _shuffle_sink(state, in_block, std::move(sink_info)); + return _shuffle_sink(state, in_block, sink_info); } } diff --git a/be/src/pipeline/local_exchange/local_exchanger.h b/be/src/pipeline/local_exchange/local_exchanger.h index e2d9ae6807c28e..08fff542f3be04 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.h +++ b/be/src/pipeline/local_exchange/local_exchanger.h @@ -145,7 +145,7 @@ class ExchangerBase { virtual Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) = 0; virtual Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, - Profile&& profile, SinkInfo&& sink_info) = 0; + Profile&& profile, SinkInfo& sink_info) = 0; virtual ExchangeType get_type() const = 0; // Called if a local exchanger source operator are closed. Free the unused data block in data_queue. virtual void close(SourceInfo&& source_info) = 0; @@ -282,7 +282,7 @@ class ShuffleExchanger : public Exchanger { } ~ShuffleExchanger() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, Profile&& profile, - SinkInfo&& sink_info) override; + SinkInfo& sink_info) override; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) override; @@ -290,11 +290,11 @@ class ShuffleExchanger : public Exchanger { ExchangeType get_type() const override { return ExchangeType::HASH_SHUFFLE; } protected: - Status _split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, + Status _split_rows(RuntimeState* state, const std::vector& channel_ids, vectorized::Block* block, int channel_id, LocalExchangeSinkLocalState* local_state, std::map* shuffle_idx_to_instance_idx); - Status _split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, + Status _split_rows(RuntimeState* state, const std::vector& channel_ids, vectorized::Block* block, int channel_id); std::vector> _partition_rows_histogram; }; @@ -317,7 +317,7 @@ class PassthroughExchanger final : public Exchanger { free_block_limit) {} ~PassthroughExchanger() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, Profile&& profile, - SinkInfo&& sink_info) override; + SinkInfo& sink_info) override; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) override; @@ -333,7 +333,7 @@ class PassToOneExchanger final : public Exchanger { free_block_limit) {} ~PassToOneExchanger() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, Profile&& profile, - SinkInfo&& sink_info) override; + SinkInfo& sink_info) override; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) override; @@ -347,7 +347,7 @@ class BroadcastExchanger final : public Exchanger { : Exchanger(running_sink_operators, num_partitions, free_block_limit) {} ~BroadcastExchanger() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, Profile&& profile, - SinkInfo&& sink_info) override; + SinkInfo& sink_info) override; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) override; @@ -367,7 +367,7 @@ class AdaptivePassthroughExchanger : public Exchanger { _partition_rows_histogram.resize(running_sink_operators); } Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos, Profile&& profile, - SinkInfo&& sink_info) override; + SinkInfo& sink_info) override; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos, Profile&& profile, SourceInfo&& source_info) override; @@ -376,11 +376,10 @@ class AdaptivePassthroughExchanger : public Exchanger { void close(SourceInfo&& source_info) override; private: - Status _passthrough_sink(RuntimeState* state, vectorized::Block* in_block, - SinkInfo&& sink_info); - Status _shuffle_sink(RuntimeState* state, vectorized::Block* in_block, SinkInfo&& sink_info); - Status _split_rows(RuntimeState* state, const uint32_t* __restrict channel_ids, - vectorized::Block* block, SinkInfo&& sink_info); + Status _passthrough_sink(RuntimeState* state, vectorized::Block* in_block, SinkInfo& sink_info); + Status _shuffle_sink(RuntimeState* state, vectorized::Block* in_block, SinkInfo& sink_info); + Status _split_rows(RuntimeState* state, const std::vector& channel_ids, + vectorized::Block* block, SinkInfo& sink_info); std::atomic_bool _is_pass_through = false; std::atomic_int32_t _total_block = 0; diff --git a/be/src/pipeline/shuffle/exchange_writer.cpp b/be/src/pipeline/shuffle/exchange_writer.cpp new file mode 100644 index 00000000000000..e806ff6fcc8985 --- /dev/null +++ b/be/src/pipeline/shuffle/exchange_writer.cpp @@ -0,0 +1,205 @@ +// 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 "exchange_writer.h" + +#include + +#include +#include +#include + +#include "common/logging.h" +#include "common/status.h" +#include "pipeline/exec/exchange_sink_operator.h" +#include "vec/core/block.h" +#include "vec/sink/tablet_sink_hash_partitioner.h" + +namespace doris::pipeline { +#include "common/compile_check_begin.h" + +ExchangeWriterBase::ExchangeWriterBase(ExchangeSinkLocalState& local_state) + : _local_state(local_state), + _partitioner( + static_cast(local_state.partitioner())) {} + +template +Status ExchangeWriterBase::_handle_eof_channel(RuntimeState* state, ChannelPtrType channel, + Status st) const { + channel->set_receiver_eof(st); + // Channel will not send RPC to the downstream when eof, so close channel by OK status. + return channel->close(state); +} + +// NOLINTBEGIN(readability-function-cognitive-complexity) +Status ExchangeWriterBase::_add_rows_impl( + RuntimeState* state, std::vector>& channels, + size_t channel_count, vectorized::Block* block, bool eos) { + Status status = Status::OK(); + uint32_t offset = 0; + for (size_t i = 0; i < channel_count; ++i) { + uint32_t size = _channel_rows_histogram[i]; + if (!channels[i]->is_receiver_eof() && size > 0) { + VLOG_DEBUG << fmt::format("partition {} of {}, block:\n{}, start: {}, size: {}", i, + channel_count, block->dump_data(), offset, size); + status = channels[i]->add_rows(block, _origin_row_idx.data(), offset, size, false); + HANDLE_CHANNEL_STATUS(state, channels[i], status); + } + offset += size; + } + if (eos) { + for (int i = 0; i < channel_count; ++i) { + if (!channels[i]->is_receiver_eof()) { + VLOG_DEBUG << fmt::format("EOS partition {} of {}, block:\n{}", i, channel_count, + block->dump_data()); + status = channels[i]->add_rows(block, _origin_row_idx.data(), 0, 0, true); + HANDLE_CHANNEL_STATUS(state, channels[i], status); + } + } + } + return Status::OK(); +} +// NOLINTEND(readability-function-cognitive-complexity) + +Status ExchangeOlapWriter::write(RuntimeState* state, vectorized::Block* block, bool eos) { + vectorized::Block prior_block; + RETURN_IF_ERROR(_partitioner->try_cut_in_line(prior_block)); + if (!prior_block.empty()) { + // prior_block (batching rows) cuts in line, deal it first. + RETURN_IF_ERROR(_write_impl(state, &prior_block)); + _partitioner->finish_cut_in_line(); + } + + RETURN_IF_ERROR(_write_impl(state, block)); + + // all data wrote. consider batched rows before eos. + if (eos) { + // get all batched rows + _partitioner->mark_last_block(); + vectorized::Block final_batching_block; + RETURN_IF_ERROR(_partitioner->try_cut_in_line(final_batching_block)); + if (!final_batching_block.empty()) { + RETURN_IF_ERROR(_write_impl(state, &final_batching_block, true)); + } else { + // No batched rows, send empty block with eos signal. + vectorized::Block empty_block = block->clone_empty(); + RETURN_IF_ERROR(_write_impl(state, &empty_block, true)); + } + } + return Status::OK(); +} + +Status ExchangeOlapWriter::_write_impl(RuntimeState* state, vectorized::Block* block, bool eos) { + auto rows = block->rows(); + { + SCOPED_TIMER(_local_state.split_block_hash_compute_timer()); + RETURN_IF_ERROR(_partitioner->do_partitioning(state, block)); + } + { + SCOPED_TIMER(_local_state.distribute_rows_into_channels_timer()); + const auto& channel_ids = _partitioner->get_channel_ids(); + const auto invalid_val = _partitioner->invalid_sentinel(); + DCHECK_EQ(channel_ids.size(), rows); + + // decrease not sinked rows this time + COUNTER_UPDATE(_local_state.rows_input_counter(), + -1LL * std::ranges::count(channel_ids, invalid_val)); + + RETURN_IF_ERROR(_channel_add_rows(state, _local_state.channels, + _local_state.channels.size(), channel_ids, rows, block, + eos, invalid_val)); + } + return Status::OK(); +} + +Status ExchangeTrivialWriter::write(RuntimeState* state, vectorized::Block* block, bool eos) { + auto rows = block->rows(); + { + SCOPED_TIMER(_local_state.split_block_hash_compute_timer()); + RETURN_IF_ERROR(_partitioner->do_partitioning(state, block)); + } + { + SCOPED_TIMER(_local_state.distribute_rows_into_channels_timer()); + const auto& channel_ids = _partitioner->get_channel_ids(); + + RETURN_IF_ERROR(_channel_add_rows(state, _local_state.channels, + _local_state.channels.size(), channel_ids, rows, block, + eos)); + } + + return Status::OK(); +} + +Status ExchangeOlapWriter::_channel_add_rows( + RuntimeState* state, std::vector>& channels, + size_t channel_count, const std::vector& channel_ids, size_t rows, + vectorized::Block* block, bool eos, HashValType invalid_val) { + size_t effective_rows = 0; + effective_rows = + std::ranges::count_if(channel_ids, [=](int64_t cid) { return cid != invalid_val; }); + + // row index will skip all skipped rows. + _origin_row_idx.resize(effective_rows); + _channel_rows_histogram.assign(channel_count, 0U); + _channel_pos_offsets.resize(channel_count); + for (size_t i = 0; i < rows; ++i) { + if (channel_ids[i] == invalid_val) { + continue; + } + auto cid = channel_ids[i]; + _channel_rows_histogram[cid]++; + } + _channel_pos_offsets[0] = 0; + for (size_t i = 1; i < channel_count; ++i) { + _channel_pos_offsets[i] = _channel_pos_offsets[i - 1] + _channel_rows_histogram[i - 1]; + } + for (uint32_t i = 0; i < rows; ++i) { + if (channel_ids[i] == invalid_val) { + continue; + } + auto cid = channel_ids[i]; + auto pos = _channel_pos_offsets[cid]++; + _origin_row_idx[pos] = i; + } + + return _add_rows_impl(state, channels, channel_count, block, eos); +} + +Status ExchangeTrivialWriter::_channel_add_rows( + RuntimeState* state, std::vector>& channels, + size_t channel_count, const std::vector& channel_ids, size_t rows, + vectorized::Block* block, bool eos) { + _origin_row_idx.resize(rows); + _channel_rows_histogram.assign(channel_count, 0U); + _channel_pos_offsets.resize(channel_count); + for (size_t i = 0; i < rows; ++i) { + _channel_rows_histogram[channel_ids[i]]++; + } + _channel_pos_offsets[0] = 0; + for (size_t i = 1; i < channel_count; ++i) { + _channel_pos_offsets[i] = _channel_pos_offsets[i - 1] + _channel_rows_histogram[i - 1]; + } + for (uint32_t i = 0; i < rows; i++) { + auto cid = channel_ids[i]; + auto pos = _channel_pos_offsets[cid]++; + _origin_row_idx[pos] = i; + } + + return _add_rows_impl(state, channels, channel_count, block, eos); +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/shuffle/exchange_writer.h b/be/src/pipeline/shuffle/exchange_writer.h new file mode 100644 index 00000000000000..fba55b9f17f7d2 --- /dev/null +++ b/be/src/pipeline/shuffle/exchange_writer.h @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "vec/runtime/partitioner.h" +#include "vec/sink/vdata_stream_sender.h" + +namespace doris { +class RuntimeState; +class Status; +namespace vectorized { +class Block; +class Channel; +class TabletSinkHashPartitioner; +} // namespace vectorized +namespace pipeline { + +#include "common/compile_check_begin.h" +class ExchangeSinkLocalState; + +class ExchangeWriterBase { +public: + using HashValType = vectorized::PartitionerBase::HashValType; + ExchangeWriterBase(ExchangeSinkLocalState& local_state); + + virtual Status write(RuntimeState* state, vectorized::Block* block, bool eos) = 0; + + virtual ~ExchangeWriterBase() = default; + +protected: + template + Status _handle_eof_channel(RuntimeState* state, ChannelPtrType channel, Status st) const; + Status _add_rows_impl(RuntimeState* state, + std::vector>& channels, + size_t channel_count, vectorized::Block* block, bool eos); + + // myself as a visitor of local state + ExchangeSinkLocalState& _local_state; + vectorized::TabletSinkHashPartitioner* _partitioner; + + // _origin_row_idx[i]: row id in original block for the i-th's data we send. + vectorized::PaddedPODArray _origin_row_idx; + // _channel_rows_histogram[i]: number of rows for channel i in current batch + vectorized::PaddedPODArray _channel_rows_histogram; + // _channel_start_offsets[i]: the start offset of channel i in _row_idx + // its value equals to prefix sum of _channel_rows_histogram + // after calculation, it will be end offset for channel i. + vectorized::PaddedPODArray _channel_pos_offsets; +}; + +class ExchangeTrivialWriter final : public ExchangeWriterBase { +public: + ExchangeTrivialWriter(ExchangeSinkLocalState& local_state) : ExchangeWriterBase(local_state) {} + + Status write(RuntimeState* state, vectorized::Block* block, bool eos) override; + +private: + Status _channel_add_rows(RuntimeState* state, + std::vector>& channels, + size_t channel_count, const std::vector& channel_ids, + size_t rows, vectorized::Block* block, bool eos); +}; + +// maybe auto partition +class ExchangeOlapWriter final : public ExchangeWriterBase { +public: + ExchangeOlapWriter(ExchangeSinkLocalState& local_state) : ExchangeWriterBase(local_state) {} + + Status write(RuntimeState* state, vectorized::Block* block, bool eos) override; + +private: + Status _write_impl(RuntimeState* state, vectorized::Block* block, bool eos = false); + Status _channel_add_rows(RuntimeState* state, + std::vector>& channels, + size_t channel_count, const std::vector& channel_ids, + size_t rows, vectorized::Block* block, bool eos, + HashValType invalid_val); +}; +#include "common/compile_check_end.h" +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/shuffle/writer.cpp b/be/src/pipeline/shuffle/writer.cpp deleted file mode 100644 index 09958554cca831..00000000000000 --- a/be/src/pipeline/shuffle/writer.cpp +++ /dev/null @@ -1,116 +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 "writer.h" - -#include - -#include "pipeline/exec/exchange_sink_operator.h" -#include "vec/core/block.h" - -namespace doris::pipeline { -#include "common/compile_check_begin.h" -template -Status Writer::_handle_eof_channel(RuntimeState* state, ChannelPtrType channel, Status st) const { - channel->set_receiver_eof(st); - // Chanel will not send RPC to the downstream when eof, so close chanel by OK status. - return channel->close(state); -} - -Status Writer::write(ExchangeSinkLocalState* local_state, RuntimeState* state, - vectorized::Block* block, bool eos) { - bool already_sent = false; - { - SCOPED_TIMER(local_state->split_block_hash_compute_timer()); - RETURN_IF_ERROR( - local_state->partitioner()->do_partitioning(state, block, eos, &already_sent)); - } - if (already_sent) { - // The same block may be sent twice by TabletSinkHashPartitioner. To get the correct - // result, we should not send any rows the last time. - return Status::OK(); - } - auto rows = block->rows(); - { - SCOPED_TIMER(local_state->distribute_rows_into_channels_timer()); - const auto& channel_filed = local_state->partitioner()->get_channel_ids(); - if (channel_filed.len == sizeof(uint32_t)) { - RETURN_IF_ERROR(_channel_add_rows(state, local_state->channels, - local_state->channels.size(), - channel_filed.get(), rows, block, eos)); - } else { - RETURN_IF_ERROR(_channel_add_rows(state, local_state->channels, - local_state->channels.size(), - channel_filed.get(), rows, block, eos)); - } - } - return Status::OK(); -} - -template -Status Writer::_channel_add_rows(RuntimeState* state, - std::vector>& channels, - size_t partition_count, - const ChannelIdType* __restrict channel_ids, size_t rows, - vectorized::Block* block, bool eos) { - _row_idx.resize(rows); - { - _partition_rows_histogram.resize(partition_count); - _channel_start_offsets.resize(partition_count); - for (size_t i = 0; i < partition_count; ++i) { - _partition_rows_histogram[i] = 0; - } - for (size_t i = 0; i < rows; ++i) { - _partition_rows_histogram[channel_ids[i]]++; - } - _channel_start_offsets[0] = 0; - for (size_t i = 1; i < partition_count; ++i) { - _channel_start_offsets[i] = - _channel_start_offsets[i - 1] + _partition_rows_histogram[i - 1]; - } - for (uint32_t i = 0; i < rows; i++) { - if constexpr (std::is_signed_v) { - // -1 means this row is filtered by table sink hash partitioner - if (channel_ids[i] == -1) { - continue; - } - } - _row_idx[_channel_start_offsets[channel_ids[i]]++] = i; - } - } - Status status = Status::OK(); - uint32_t offset = 0; - for (size_t i = 0; i < partition_count; ++i) { - uint32_t size = _partition_rows_histogram[i]; - if (!channels[i]->is_receiver_eof() && size > 0) { - status = channels[i]->add_rows(block, _row_idx.data(), offset, size, false); - HANDLE_CHANNEL_STATUS(state, channels[i], status); - } - offset += size; - } - if (eos) { - for (int i = 0; i < partition_count; ++i) { - if (!channels[i]->is_receiver_eof()) { - status = channels[i]->add_rows(block, _row_idx.data(), 0, 0, true); - HANDLE_CHANNEL_STATUS(state, channels[i], status); - } - } - } - return Status::OK(); -} - -} // namespace doris::pipeline diff --git a/be/src/pipeline/shuffle/writer.h b/be/src/pipeline/shuffle/writer.h deleted file mode 100644 index 5549df91f9a73e..00000000000000 --- a/be/src/pipeline/shuffle/writer.h +++ /dev/null @@ -1,57 +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. - -#pragma once - -#include "vec/sink/vdata_stream_sender.h" - -namespace doris { -class RuntimeState; -class Status; -namespace vectorized { -class Block; -class Channel; -} // namespace vectorized -namespace pipeline { - -#include "common/compile_check_begin.h" -class ExchangeSinkLocalState; - -class Writer { -public: - Writer() = default; - - Status write(ExchangeSinkLocalState* local_state, RuntimeState* state, vectorized::Block* block, - bool eos); - -private: - template - Status _channel_add_rows(RuntimeState* state, - std::vector>& channels, - size_t partition_count, const ChannelIdType* __restrict channel_ids, - size_t rows, vectorized::Block* block, bool eos); - - template - Status _handle_eof_channel(RuntimeState* state, ChannelPtrType channel, Status st) const; - - vectorized::PaddedPODArray _row_idx; - vectorized::PaddedPODArray _partition_rows_histogram; - vectorized::PaddedPODArray _channel_start_offsets; -}; -#include "common/compile_check_end.h" -} // namespace pipeline -} // namespace doris \ No newline at end of file diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 33504640b60d5d..85a374518a3485 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -984,25 +984,6 @@ Status MutableBlock::add_rows(const Block* block, size_t row_begin, size_t lengt return Status::OK(); } -Status MutableBlock::add_rows(const Block* block, const std::vector& rows) { - RETURN_IF_CATCH_EXCEPTION({ - DCHECK_LE(columns(), block->columns()); - const auto& block_data = block->get_columns_with_type_and_name(); - const size_t length = std::ranges::distance(rows); - for (size_t i = 0; i < _columns.size(); ++i) { - DCHECK_EQ(_data_types[i]->get_name(), block_data[i].type->get_name()); - auto& dst = _columns[i]; - const auto& src = *block_data[i].column.get(); - dst->reserve(dst->size() + length); - for (auto row : rows) { - // we can introduce a new function like `insert_assume_reserved` for IColumn. - dst->insert_from(src, row); - } - } - }); - return Status::OK(); -} - Block MutableBlock::to_block(int start_column) { return to_block(start_column, (int)_columns.size()); } diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index 61bbd0303fd128..61da1a6a843ca8 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -565,7 +565,6 @@ class MutableBlock { Status add_rows(const Block* block, const uint32_t* row_begin, const uint32_t* row_end, const std::vector* column_offset = nullptr); Status add_rows(const Block* block, size_t row_begin, size_t length); - Status add_rows(const Block* block, const std::vector& rows); std::string dump_data(size_t row_limit = 100) const; std::string dump_data_json(size_t row_limit = 100) const; diff --git a/be/src/vec/exec/skewed_partition_rebalancer.cpp b/be/src/vec/exec/skewed_partition_rebalancer.cpp index 9549ccc9192780..ce85d271e9acbc 100644 --- a/be/src/vec/exec/skewed_partition_rebalancer.cpp +++ b/be/src/vec/exec/skewed_partition_rebalancer.cpp @@ -21,7 +21,6 @@ #include "vec/exec/skewed_partition_rebalancer.h" #include -#include namespace doris::vectorized { #include "common/compile_check_avoid_begin.h" @@ -56,7 +55,7 @@ SkewedPartitionRebalancer::SkewedPartitionRebalancer( } } -int SkewedPartitionRebalancer::get_task_id(int partition_id, int64_t index) { +int SkewedPartitionRebalancer::get_task_id(uint32_t partition_id, int64_t index) { const std::vector& task_ids = _partition_assignments[partition_id]; return task_ids[index % task_ids.size()].task_id; } @@ -78,11 +77,11 @@ void SkewedPartitionRebalancer::rebalance() { void SkewedPartitionRebalancer::_calculate_partition_data_size(long data_processed) { long total_partition_row_count = 0; - for (int partition = 0; partition < _partition_count; partition++) { + for (uint32_t partition = 0; partition < _partition_count; partition++) { total_partition_row_count += _partition_row_count[partition]; } - for (int partition = 0; partition < _partition_count; partition++) { + for (uint32_t partition = 0; partition < _partition_count; partition++) { _partition_data_size[partition] = std::max( (_partition_row_count[partition] * data_processed) / total_partition_row_count, _partition_data_size[partition]); @@ -239,11 +238,10 @@ void SkewedPartitionRebalancer::_rebalance_partitions(long data_processed) { std::vector> task_bucket_max_partitions; for (int i = 0; i < _task_count * _task_bucket_count; ++i) { - task_bucket_max_partitions.push_back( - IndexedPriorityQueue()); + task_bucket_max_partitions.emplace_back(); } - for (int partition = 0; partition < _partition_count; partition++) { + for (uint32_t partition = 0; partition < _partition_count; partition++) { auto& task_assignments = _partition_assignments[partition]; for (const auto& task_bucket : task_assignments) { auto& queue = task_bucket_max_partitions[task_bucket.id]; diff --git a/be/src/vec/exec/skewed_partition_rebalancer.h b/be/src/vec/exec/skewed_partition_rebalancer.h index c6e8ae11f849b5..0490b236d3891e 100644 --- a/be/src/vec/exec/skewed_partition_rebalancer.h +++ b/be/src/vec/exec/skewed_partition_rebalancer.h @@ -47,10 +47,6 @@ #include -#include -#include -#include -#include #include #include "util/indexed_priority_queue.hpp" @@ -80,7 +76,7 @@ class SkewedPartitionRebalancer { long min_partition_data_processed_rebalance_threshold, long min_data_processed_rebalance_threshold); - int get_task_id(int partition_id, int64_t index); + int get_task_id(uint32_t partition_id, int64_t index); void add_data_processed(long data_size); void add_partition_row_count(int partition, long row_count); void rebalance(); @@ -116,7 +112,7 @@ class SkewedPartitionRebalancer { static constexpr double TASK_BUCKET_SKEWNESS_THRESHOLD = 0.7; // One or more tasks in one partition. `_task_count` equals to the number of channels and `_task_bucket_count` is always 1. - const int _partition_count; + const uint32_t _partition_count; const int _task_count; const int _task_bucket_count; long _min_partition_data_processed_rebalance_threshold; diff --git a/be/src/vec/runtime/partitioner.cpp b/be/src/vec/runtime/partitioner.cpp index 7e9410fd29f47b..5095c7a7dbbd37 100644 --- a/be/src/vec/runtime/partitioner.cpp +++ b/be/src/vec/runtime/partitioner.cpp @@ -28,8 +28,7 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" template -Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Block* block, - bool eos, bool* already_sent) const { +Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Block* block) const { size_t rows = block->rows(); if (rows > 0) { @@ -60,28 +59,28 @@ Status Crc32HashPartitioner::do_partitioning(RuntimeState* state, Bl template void Crc32HashPartitioner::_do_hash(const ColumnPtr& column, - uint32_t* __restrict result, int idx) const { + HashValType* __restrict result, int idx) const { column->update_crcs_with_value( result, _partition_expr_ctxs[idx]->root()->data_type()->get_primitive_type(), - cast_set(column->size())); + cast_set(column->size())); } template Status Crc32HashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { - auto* new_partitioner = new Crc32HashPartitioner(cast_set(_partition_count)); + auto* new_partitioner = new Crc32HashPartitioner(_partition_count); partitioner.reset(new_partitioner); return _clone_expr_ctxs(state, new_partitioner->_partition_expr_ctxs); } -void Crc32CHashPartitioner::_do_hash(const ColumnPtr& column, uint32_t* __restrict result, +void Crc32CHashPartitioner::_do_hash(const ColumnPtr& column, HashValType* __restrict result, int idx) const { column->update_crc32c_batch(result, nullptr); } Status Crc32CHashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { - auto* new_partitioner = new Crc32CHashPartitioner(cast_set(_partition_count)); + auto* new_partitioner = new Crc32CHashPartitioner(_partition_count); partitioner.reset(new_partitioner); return _clone_expr_ctxs(state, new_partitioner->_partition_expr_ctxs); } diff --git a/be/src/vec/runtime/partitioner.h b/be/src/vec/runtime/partitioner.h index 031a97dc2bd9e2..dfe0e79f988e01 100644 --- a/be/src/vec/runtime/partitioner.h +++ b/be/src/vec/runtime/partitioner.h @@ -19,25 +19,18 @@ #include +#include "vec/core/block.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" namespace doris::vectorized { #include "common/compile_check_begin.h" -struct ChannelField { - const void* channel_id; - const uint32_t len; - - template - const T* get() const { - CHECK_EQ(sizeof(T), len) << " sizeof(T): " << sizeof(T) << " len: " << len; - return reinterpret_cast(channel_id); - } -}; class PartitionerBase { public: - PartitionerBase(size_t partition_count) : _partition_count(partition_count) {} + using HashValType = uint32_t; + + PartitionerBase(HashValType partition_count) : _partition_count(partition_count) {} virtual ~PartitionerBase() = default; virtual Status init(const std::vector& texprs) = 0; @@ -48,17 +41,19 @@ class PartitionerBase { virtual Status close(RuntimeState* state) = 0; - virtual Status do_partitioning(RuntimeState* state, Block* block, bool eos = false, - bool* already_sent = nullptr) const = 0; + virtual Status do_partitioning(RuntimeState* state, Block* block) const = 0; - virtual ChannelField get_channel_ids() const = 0; + virtual const std::vector& get_channel_ids() const = 0; virtual Status clone(RuntimeState* state, std::unique_ptr& partitioner) = 0; - size_t partition_count() const { return _partition_count; } + // use _partition_count as invalid sentinel value. since modulo operation result is [0, partition_count-1] + HashValType partition_count() const { return _partition_count; } + // use a individual function to highlight its special meaning + HashValType invalid_sentinel() const { return partition_count(); } protected: - const size_t _partition_count; + const HashValType _partition_count; }; template @@ -79,12 +74,9 @@ class Crc32HashPartitioner : public PartitionerBase { Status close(RuntimeState* state) override { return Status::OK(); } - Status do_partitioning(RuntimeState* state, Block* block, bool eos, - bool* already_sent) const override; + Status do_partitioning(RuntimeState* state, Block* block) const override; - ChannelField get_channel_ids() const override { - return {.channel_id = _hash_vals.data(), .len = sizeof(uint32_t)}; - } + const std::vector& get_channel_ids() const override { return _hash_vals; } Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; @@ -105,31 +97,27 @@ class Crc32HashPartitioner : public PartitionerBase { return Status::OK(); } - virtual void _do_hash(const ColumnPtr& column, uint32_t* __restrict result, int idx) const; + virtual void _do_hash(const ColumnPtr& column, HashValType* __restrict result, int idx) const; virtual void _initialize_hash_vals(size_t rows) const { _hash_vals.resize(rows); std::ranges::fill(_hash_vals, 0); } VExprContextSPtrs _partition_expr_ctxs; - mutable std::vector _hash_vals; + mutable std::vector _hash_vals; }; struct ShuffleChannelIds { - template - HashValueType operator()(HashValueType l, size_t r) { - return l % r; - } + using HashValType = PartitionerBase::HashValType; + HashValType operator()(HashValType l, size_t r) { return l % r; } }; struct SpillPartitionChannelIds { - template - HashValueType operator()(HashValueType l, size_t r) { - return ((l >> 16) | (l << 16)) % r; - } + using HashValType = PartitionerBase::HashValType; + HashValType operator()(HashValType l, size_t r) { return ((l >> 16) | (l << 16)) % r; } }; -static inline uint32_t crc32c_shuffle_mix(uint32_t h) { +static inline PartitionerBase::HashValType crc32c_shuffle_mix(PartitionerBase::HashValType h) { // Step 1: fold high entropy into low bits h ^= h >> 16; // Step 2: odd multiplicative scramble (cheap avalanche) @@ -143,10 +131,8 @@ static inline uint32_t crc32c_shuffle_mix(uint32_t h) { // shuffle hash function same with crc32c hash table(eg join hash table) will lead bad performance // hash table offten use low 16 bits as bucket index, so we shift 16 bits to high bits to avoid conflict struct ShiftChannelIds { - template - HashValueType operator()(HashValueType l, size_t r) { - return crc32c_shuffle_mix(l) % r; - } + using HashValType = PartitionerBase::HashValType; + HashValType operator()(HashValType l, size_t r) { return crc32c_shuffle_mix(l) % r; } }; class Crc32CHashPartitioner : public Crc32HashPartitioner { @@ -157,12 +143,12 @@ class Crc32CHashPartitioner : public Crc32HashPartitioner { Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; private: - void _do_hash(const ColumnPtr& column, uint32_t* __restrict result, int idx) const override; + void _do_hash(const ColumnPtr& column, HashValType* __restrict result, int idx) const override; void _initialize_hash_vals(size_t rows) const override { _hash_vals.resize(rows); // use golden ratio to initialize hash values to avoid collision with hash table's hash function - constexpr uint32_t CRC32C_SHUFFLE_SEED = 0x9E3779B9U; + constexpr HashValType CRC32C_SHUFFLE_SEED = 0x9E3779B9U; std::ranges::fill(_hash_vals, CRC32C_SHUFFLE_SEED); } }; diff --git a/be/src/vec/sink/scale_writer_partitioning_exchanger.hpp b/be/src/vec/sink/scale_writer_partitioning_exchanger.hpp index 76da26dcaf82ad..405ac2ec552c6c 100644 --- a/be/src/vec/sink/scale_writer_partitioning_exchanger.hpp +++ b/be/src/vec/sink/scale_writer_partitioning_exchanger.hpp @@ -17,9 +17,7 @@ #pragma once -#include -#include -#include +#include #include #include "vec/core/block.h" @@ -30,7 +28,6 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" class ScaleWriterPartitioner final : public PartitionerBase { public: - using HashValType = uint32_t; ScaleWriterPartitioner(int channel_size, int partition_count, int task_count, int task_bucket_count, long min_partition_data_processed_rebalance_threshold, @@ -67,8 +64,7 @@ class ScaleWriterPartitioner final : public PartitionerBase { Status close(RuntimeState* state) override { return _crc_partitioner->close(state); } - Status do_partitioning(RuntimeState* state, Block* block, bool eos, - bool* already_sent) const override { + Status do_partitioning(RuntimeState* state, Block* block) const override { _hash_vals.resize(block->rows()); for (int partition_id = 0; partition_id < _partition_row_counts.size(); partition_id++) { _partition_row_counts[partition_id] = 0; @@ -78,9 +74,9 @@ class ScaleWriterPartitioner final : public PartitionerBase { _partition_rebalancer.rebalance(); RETURN_IF_ERROR(_crc_partitioner->do_partitioning(state, block)); - const auto* crc_values = _crc_partitioner->get_channel_ids().get(); + const auto& channel_ids = _crc_partitioner->get_channel_ids(); for (size_t position = 0; position < block->rows(); position++) { - int partition_id = crc_values[position]; + auto partition_id = channel_ids[position]; _partition_row_counts[partition_id] += 1; // Get writer id for this partition by looking at the scaling state @@ -101,20 +97,18 @@ class ScaleWriterPartitioner final : public PartitionerBase { return Status::OK(); } - ChannelField get_channel_ids() const override { - return {_hash_vals.data(), sizeof(HashValType)}; - } + const std::vector& get_channel_ids() const override { return _hash_vals; } Status clone(RuntimeState* state, std::unique_ptr& partitioner) override { - partitioner.reset(new ScaleWriterPartitioner( + partitioner = std::make_unique( _channel_size, (int)_partition_count, _task_count, _task_bucket_count, _min_partition_data_processed_rebalance_threshold, - _min_data_processed_rebalance_threshold)); + _min_data_processed_rebalance_threshold); return Status::OK(); } private: - int _get_next_writer_id(int partition_id) const { + int _get_next_writer_id(HashValType partition_id) const { return _partition_rebalancer.get_task_id(partition_id, _partition_writer_indexes[partition_id]++); } @@ -132,4 +126,4 @@ class ScaleWriterPartitioner final : public PartitionerBase { const long _min_data_processed_rebalance_threshold; }; #include "common/compile_check_end.h" -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized diff --git a/be/src/vec/sink/tablet_sink_hash_partitioner.cpp b/be/src/vec/sink/tablet_sink_hash_partitioner.cpp index 995ffa2f031b08..f27a019f025a54 100644 --- a/be/src/vec/sink/tablet_sink_hash_partitioner.cpp +++ b/be/src/vec/sink/tablet_sink_hash_partitioner.cpp @@ -17,20 +17,25 @@ #include "vec/sink/tablet_sink_hash_partitioner.h" +#include +#include +#include + #include "pipeline/exec/operator.h" namespace doris::vectorized { #include "common/compile_check_begin.h" -TabletSinkHashPartitioner::TabletSinkHashPartitioner( - size_t partition_count, int64_t txn_id, const TOlapTableSchemaParam& tablet_sink_schema, - const TOlapTablePartitionParam& tablet_sink_partition, - const TOlapTableLocationParam& tablet_sink_location, const TTupleId& tablet_sink_tuple_id, - pipeline::ExchangeSinkLocalState* local_state) +TabletSinkHashPartitioner::TabletSinkHashPartitioner(uint32_t partition_count, int64_t txn_id, + TOlapTableSchemaParam tablet_sink_schema, + TOlapTablePartitionParam tablet_sink_partition, + TOlapTableLocationParam tablet_sink_location, + const TTupleId& tablet_sink_tuple_id, + pipeline::ExchangeSinkLocalState* local_state) : PartitionerBase(partition_count), _txn_id(txn_id), - _tablet_sink_schema(tablet_sink_schema), - _tablet_sink_partition(tablet_sink_partition), - _tablet_sink_location(tablet_sink_location), + _tablet_sink_schema(std::move(tablet_sink_schema)), + _tablet_sink_partition(std::move(tablet_sink_partition)), + _tablet_sink_location(std::move(tablet_sink_location)), _tablet_sink_tuple_id(tablet_sink_tuple_id), _local_state(local_state) {} @@ -81,45 +86,68 @@ Status TabletSinkHashPartitioner::open(RuntimeState* state) { return Status::OK(); } -Status TabletSinkHashPartitioner::do_partitioning(RuntimeState* state, Block* block, bool eos, - bool* already_sent) const { +Status TabletSinkHashPartitioner::do_partitioning(RuntimeState* state, Block* block) const { _hash_vals.resize(block->rows()); if (block->empty()) { return Status::OK(); } - std::fill(_hash_vals.begin(), _hash_vals.end(), -1); - int64_t filtered_rows = 0; - int64_t number_input_rows = _local_state->rows_input_counter()->value(); + + // tablet_id_hash % invalid_val never get invalid_val, so we use invalid_val as sentinel value + DCHECK_EQ(invalid_sentinel(), partition_count()); + const auto& invalid_val = invalid_sentinel(); + std::ranges::fill(_hash_vals, invalid_val); + + int64_t dummy_stats = 0; // _local_state->rows_input_counter() updated in sink and write. std::shared_ptr convert_block = std::make_shared(); + RETURN_IF_ERROR(_row_distribution.generate_rows_distribution( - *block, convert_block, filtered_rows, _row_part_tablet_ids, number_input_rows)); - if (_row_distribution.batching_rows() > 0) { - SCOPED_TIMER(_local_state->send_new_partition_timer()); - RETURN_IF_ERROR(_send_new_partition_batch(state, block, eos)); - *already_sent = true; - } else { - const auto& row_ids = _row_part_tablet_ids[0].row_ids; - const auto& tablet_ids = _row_part_tablet_ids[0].tablet_ids; - for (int idx = 0; idx < row_ids.size(); ++idx) { - const auto& row = row_ids[idx]; - const auto& tablet_id_hash = - HashUtil::zlib_crc_hash(&tablet_ids[idx], sizeof(HashValType), 0); - _hash_vals[row] = tablet_id_hash % _partition_count; + *block, convert_block, _row_part_tablet_ids, dummy_stats)); + _skipped = _row_distribution.get_skipped(); + const auto& row_ids = _row_part_tablet_ids[0].row_ids; + const auto& tablet_ids = _row_part_tablet_ids[0].tablet_ids; + + for (int idx = 0; idx < row_ids.size(); ++idx) { + const auto& row = row_ids[idx]; + const auto& tablet_id_hash = + HashUtil::zlib_crc_hash(&tablet_ids[idx], sizeof(HashValType), 0); + _hash_vals[row] = tablet_id_hash % invalid_val; + } + + // _hash_vals[i] == invalid_val => row i is skipped or filtered +#ifndef NDEBUG + for (size_t i = 0; i < _skipped.size(); ++i) { + if (_skipped[i]) { + CHECK_EQ(_hash_vals[i], invalid_val); } } + CHECK_LE(std::ranges::count_if(_skipped, [](bool v) { return v; }), + std::ranges::count_if(_hash_vals, [=](HashValType v) { return v == invalid_val; })); +#endif return Status::OK(); } -ChannelField TabletSinkHashPartitioner::get_channel_ids() const { - return {_hash_vals.data(), sizeof(HashValType)}; +Status TabletSinkHashPartitioner::try_cut_in_line(Block& prior_block) const { + // check if we need send batching block first + if (_row_distribution.need_deal_batching()) { + { + SCOPED_TIMER(_local_state->send_new_partition_timer()); + RETURN_IF_ERROR(_row_distribution.automatic_create_partition()); + } + + prior_block = _row_distribution._batching_block->to_block(); // Borrow out, for lval ref + _row_distribution._batching_block.reset(); // clear. vrow_distribution will re-construct it + _row_distribution.clear_batching_stats(); + VLOG_DEBUG << "sinking batched block:\n" << prior_block.dump_data(); + } + return Status::OK(); } Status TabletSinkHashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { - partitioner.reset(new TabletSinkHashPartitioner(_partition_count, _txn_id, _tablet_sink_schema, - _tablet_sink_partition, _tablet_sink_location, - _tablet_sink_tuple_id, _local_state)); + partitioner = std::make_unique( + _partition_count, _txn_id, _tablet_sink_schema, _tablet_sink_partition, + _tablet_sink_location, _tablet_sink_tuple_id, _local_state); return Status::OK(); } @@ -135,18 +163,4 @@ Status TabletSinkHashPartitioner::close(RuntimeState* state) { } return Status::OK(); } - -Status TabletSinkHashPartitioner::_send_new_partition_batch(RuntimeState* state, - vectorized::Block* input_block, - bool eos) const { - RETURN_IF_ERROR(_row_distribution.automatic_create_partition()); - auto& p = _local_state->parent()->cast(); - // Recovery back - _row_distribution.clear_batching_stats(); - _row_distribution._batching_block->clear_column_data(); - _row_distribution._deal_batched = false; - RETURN_IF_ERROR(p.sink(state, input_block, eos)); - return Status::OK(); -} - -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized diff --git a/be/src/vec/sink/tablet_sink_hash_partitioner.h b/be/src/vec/sink/tablet_sink_hash_partitioner.h index 0b61aad9aad751..9f84fd429fa211 100644 --- a/be/src/vec/sink/tablet_sink_hash_partitioner.h +++ b/be/src/vec/sink/tablet_sink_hash_partitioner.h @@ -30,11 +30,10 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" class TabletSinkHashPartitioner final : public PartitionerBase { public: - using HashValType = int64_t; - TabletSinkHashPartitioner(size_t partition_count, int64_t txn_id, - const TOlapTableSchemaParam& tablet_sink_schema, - const TOlapTablePartitionParam& tablet_sink_partition, - const TOlapTableLocationParam& tablet_sink_location, + TabletSinkHashPartitioner(uint32_t partition_count, int64_t txn_id, + TOlapTableSchemaParam tablet_sink_schema, + TOlapTablePartitionParam tablet_sink_partition, + TOlapTableLocationParam tablet_sink_location, const TTupleId& tablet_sink_tuple_id, pipeline::ExchangeSinkLocalState* local_state); @@ -46,10 +45,15 @@ class TabletSinkHashPartitioner final : public PartitionerBase { Status open(RuntimeState* state) override; - Status do_partitioning(RuntimeState* state, Block* block, bool eos, - bool* already_sent) const override; + Status do_partitioning(RuntimeState* state, Block* block) const override; + // block to create new partition by RPC. return batched data to create. + Status try_cut_in_line(Block& prior_block) const; + void finish_cut_in_line() const { _row_distribution._deal_batched = false; } + void mark_last_block() const { _row_distribution._deal_batched = true; } + + const std::vector& get_channel_ids() const override { return _hash_vals; } + const std::vector& get_skipped() const { return _skipped; } - ChannelField get_channel_ids() const override; Status clone(RuntimeState* state, std::unique_ptr& partitioner) override; Status close(RuntimeState* state) override; @@ -59,9 +63,6 @@ class TabletSinkHashPartitioner final : public PartitionerBase { return Status::OK(); } - Status _send_new_partition_batch(RuntimeState* state, vectorized::Block* input_block, - bool eos) const; - const int64_t _txn_id = -1; const TOlapTableSchemaParam _tablet_sink_schema; const TOlapTablePartitionParam _tablet_sink_partition; @@ -79,6 +80,7 @@ class TabletSinkHashPartitioner final : public PartitionerBase { mutable RowDescriptor* _tablet_sink_row_desc = nullptr; mutable std::vector _row_part_tablet_ids; mutable std::vector _hash_vals; + mutable std::vector _skipped; }; #include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index d02519831d043c..13102c2556373c 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -203,7 +203,7 @@ Status Channel::_send_local_block(bool eos) { _serializer.get_block()->set_mutable_columns(block.clone_empty_columns()); } - if (!block.empty() || eos) { + if (!block.empty() || eos) { // if eos is true, we MUST to send an empty block RETURN_IF_ERROR(send_local_block(&block, eos, true)); } return Status::OK(); diff --git a/be/src/vec/sink/vrow_distribution.cpp b/be/src/vec/sink/vrow_distribution.cpp index 7ed96000131141..cc13309b7a6f55 100644 --- a/be/src/vec/sink/vrow_distribution.cpp +++ b/be/src/vec/sink/vrow_distribution.cpp @@ -33,6 +33,7 @@ #include "runtime/query_context.h" #include "runtime/runtime_state.h" #include "service/backend_options.h" +#include "util/debug_points.h" #include "util/doris_metrics.h" #include "util/thrift_rpc_helper.h" #include "vec/columns/column.h" @@ -46,17 +47,18 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" -std::pair -VRowDistribution::_get_partition_function() { +std::pair VRowDistribution::_get_partition_function() { return {_vpartition->get_part_func_ctx(), _vpartition->get_partition_function()}; } Status VRowDistribution::_save_missing_values( + const Block& input_block, std::vector>& col_strs, // non-const ref for move - int col_size, Block* block, const std::vector& filter, + int col_size, Block* block, const std::vector& filter, const std::vector& col_null_maps) { // de-duplication for new partitions but save all rows. - RETURN_IF_ERROR(_batching_block->add_rows(block, filter)); + RETURN_IF_ERROR( + _batching_block->add_rows(&input_block, filter.data(), filter.data() + filter.size())); std::vector cur_row_values; for (int row = 0; row < col_strs[0].size(); ++row) { cur_row_values.clear(); @@ -81,7 +83,7 @@ Status VRowDistribution::_save_missing_values( if (_batching_block->rows() > _batch_size) { _deal_batched = true; } - + _batching_rows = _batching_block->rows(); VLOG_NOTICE << "pushed some batching lines, now numbers = " << _batching_rows; return Status::OK(); @@ -94,9 +96,15 @@ void VRowDistribution::clear_batching_stats() { } Status VRowDistribution::automatic_create_partition() { + MonotonicStopWatch timer; + if (_state->enable_profile() && _state->profile_level() >= 2) { + timer.start(); + } + SCOPED_TIMER(_add_partition_request_timer); TCreatePartitionRequest request; TCreatePartitionResult result; + bool injected = false; std::string be_endpoint = BackendOptions::get_be_endpoint(); request.__set_txn_id(_txn_id); request.__set_db_id(_vpartition->db_id()); @@ -109,15 +117,26 @@ Status VRowDistribution::automatic_create_partition() { request.__set_query_id(_state->get_query_ctx()->query_id()); } + DBUG_EXECUTE_IF("VRowDistribution.automatic_create_partition.inject_result", { + DBUG_RUN_CALLBACK(&request, &result); + injected = true; + }); + VLOG_NOTICE << "automatic partition rpc begin request " << request; - TNetworkAddress master_addr = ExecEnv::GetInstance()->cluster_info()->master_fe_addr; - int time_out = _state->execution_timeout() * 1000; - RETURN_IF_ERROR(ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result](FrontendServiceConnection& client) { - client->createPartition(result, request); - }, - time_out)); + if (!injected) { + auto* cluster_info = ExecEnv::GetInstance()->cluster_info(); + if (cluster_info == nullptr) { + return Status::InternalError("cluster_info is null"); + } + TNetworkAddress master_addr = cluster_info->master_fe_addr; + int time_out = _state->execution_timeout() * 1000; + RETURN_IF_ERROR(ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result](FrontendServiceConnection& client) { + client->createPartition(result, request); + }, + time_out)); + } Status status(Status::create(result.status)); VLOG_NOTICE << "automatic partition rpc end response " << result; @@ -131,6 +150,11 @@ Status VRowDistribution::automatic_create_partition() { RETURN_IF_ERROR(_create_partition_callback(_caller, &result)); } + // Record this request's elapsed time + if (_state->enable_profile() && _state->profile_level() >= 2) { + int64_t elapsed_ns = timer.elapsed_time(); + _add_partition_request_times.push_back(elapsed_ns); + } return status; } @@ -150,6 +174,7 @@ Status VRowDistribution::_replace_overwriting_partition() { SCOPED_TIMER(_add_partition_request_timer); // also for replace_partition TReplacePartitionRequest request; TReplacePartitionResult result; + bool injected = false; request.__set_overwrite_group_id(_vpartition->get_overwrite_group_id()); request.__set_db_id(_vpartition->db_id()); request.__set_table_id(_vpartition->table_id()); @@ -184,15 +209,26 @@ Status VRowDistribution::_replace_overwriting_partition() { std::string be_endpoint = BackendOptions::get_be_endpoint(); request.__set_be_endpoint(be_endpoint); + DBUG_EXECUTE_IF("VRowDistribution.replace_overwriting_partition.inject_result", { + DBUG_RUN_CALLBACK(&request, &result); + injected = true; + }); + VLOG_NOTICE << "auto detect replace partition request: " << request; - TNetworkAddress master_addr = ExecEnv::GetInstance()->cluster_info()->master_fe_addr; - int time_out = _state->execution_timeout() * 1000; - RETURN_IF_ERROR(ThriftRpcHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result](FrontendServiceConnection& client) { - client->replacePartition(result, request); - }, - time_out)); + if (!injected) { + auto* cluster_info = ExecEnv::GetInstance()->cluster_info(); + if (cluster_info == nullptr) { + return Status::InternalError("cluster_info is null"); + } + TNetworkAddress master_addr = cluster_info->master_fe_addr; + int time_out = _state->execution_timeout() * 1000; + RETURN_IF_ERROR(ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result](FrontendServiceConnection& client) { + client->replacePartition(result, request); + }, + time_out)); + } Status status(Status::create(result.status)); VLOG_NOTICE << "auto detect replace partition result: " << result; @@ -212,7 +248,7 @@ Status VRowDistribution::_replace_overwriting_partition() { return status; } -void VRowDistribution::_get_tablet_ids(vectorized::Block* block, int32_t index_idx, +void VRowDistribution::_get_tablet_ids(Block* block, int32_t index_idx, std::vector& tablet_ids) { tablet_ids.resize(block->rows()); for (int row_idx = 0; row_idx < block->rows(); row_idx++) { @@ -228,8 +264,7 @@ void VRowDistribution::_get_tablet_ids(vectorized::Block* block, int32_t index_i } } -void VRowDistribution::_filter_block_by_skip(vectorized::Block* block, - RowPartTabletIds& row_part_tablet_id) { +void VRowDistribution::_filter_block_by_skip(Block* block, RowPartTabletIds& row_part_tablet_id) { auto& row_ids = row_part_tablet_id.row_ids; auto& partition_ids = row_part_tablet_id.partition_ids; auto& tablet_ids = row_part_tablet_id.tablet_ids; @@ -247,8 +282,7 @@ void VRowDistribution::_filter_block_by_skip(vectorized::Block* block, } Status VRowDistribution::_filter_block_by_skip_and_where_clause( - vectorized::Block* block, const vectorized::VExprContextSPtr& where_clause, - RowPartTabletIds& row_part_tablet_id) { + Block* block, const VExprContextSPtr& where_clause, RowPartTabletIds& row_part_tablet_id) { // TODO //SCOPED_RAW_TIMER(&_stat.where_clause_ns); int result_index = -1; @@ -260,8 +294,7 @@ Status VRowDistribution::_filter_block_by_skip_and_where_clause( auto& row_ids = row_part_tablet_id.row_ids; auto& partition_ids = row_part_tablet_id.partition_ids; auto& tablet_ids = row_part_tablet_id.tablet_ids; - if (const auto* nullable_column = - vectorized::check_and_get_column(*filter_column)) { + if (const auto* nullable_column = check_and_get_column(*filter_column)) { auto rows = block->rows(); // row count of a block should not exceed UINT32_MAX auto rows_uint32 = cast_set(rows); @@ -272,8 +305,7 @@ Status VRowDistribution::_filter_block_by_skip_and_where_clause( tablet_ids.emplace_back(_tablet_ids[i]); } } - } else if (const auto* const_column = - vectorized::check_and_get_column(*filter_column)) { + } else if (const auto* const_column = check_and_get_column(*filter_column)) { bool ret = const_column->get_bool(0); if (!ret) { return Status::OK(); @@ -281,7 +313,7 @@ Status VRowDistribution::_filter_block_by_skip_and_where_clause( // should we optimize? _filter_block_by_skip(block, row_part_tablet_id); } else { - const auto& filter = assert_cast(*filter_column).get_data(); + const auto& filter = assert_cast(*filter_column).get_data(); auto rows = block->rows(); // row count of a block should not exceed UINT32_MAX auto rows_uint32 = cast_set(rows); @@ -300,7 +332,7 @@ Status VRowDistribution::_filter_block_by_skip_and_where_clause( return Status::OK(); } -Status VRowDistribution::_filter_block(vectorized::Block* block, +Status VRowDistribution::_filter_block(Block* block, std::vector& row_part_tablet_ids) { for (int i = 0; i < _schema->indexes().size(); i++) { _get_tablet_ids(block, i, _tablet_ids); @@ -316,8 +348,7 @@ Status VRowDistribution::_filter_block(vectorized::Block* block, } Status VRowDistribution::_generate_rows_distribution_for_non_auto_partition( - vectorized::Block* block, bool has_filtered_rows, - std::vector& row_part_tablet_ids) { + Block* block, bool has_filtered_rows, std::vector& row_part_tablet_ids) { int num_rows = cast_set(block->rows()); RETURN_IF_ERROR(_tablet_finder->find_tablets(_state, block, num_rows, _partitions, @@ -331,7 +362,7 @@ Status VRowDistribution::_generate_rows_distribution_for_non_auto_partition( return Status::OK(); } -Status VRowDistribution::_deal_missing_map(vectorized::Block* block, +Status VRowDistribution::_deal_missing_map(const Block& input_block, Block* block, const std::vector& partition_cols_idx, int64_t& rows_stat_val) { // for missing partition keys, calc the missing partition and save in _partitions_need_create @@ -364,24 +395,27 @@ Status VRowDistribution::_deal_missing_map(vectorized::Block* block, } // calc the end value and save them. in the end of sending, we will create partitions for them and deal them. - RETURN_IF_ERROR( - _save_missing_values(col_strs, part_col_num, block, _missing_map, col_null_maps)); + // NOTE: must save old batching stats before calling _save_missing_values(), + // because _save_missing_values() will update _batching_rows internally. + size_t old_bt_rows = _batching_rows; + size_t old_bt_bytes = _batching_bytes; + + RETURN_IF_ERROR(_save_missing_values(input_block, col_strs, part_col_num, block, _missing_map, + col_null_maps)); size_t new_bt_rows = _batching_block->rows(); size_t new_bt_bytes = _batching_block->bytes(); - rows_stat_val -= new_bt_rows - _batching_rows; - _state->update_num_rows_load_total(_batching_rows - new_bt_rows); - _state->update_num_bytes_load_total(_batching_bytes - new_bt_bytes); - DorisMetrics::instance()->load_rows->increment(_batching_rows - new_bt_rows); - DorisMetrics::instance()->load_bytes->increment(_batching_bytes - new_bt_bytes); - _batching_rows = new_bt_rows; - _batching_bytes = new_bt_bytes; + rows_stat_val -= new_bt_rows - old_bt_rows; + _state->update_num_rows_load_total(old_bt_rows - new_bt_rows); + _state->update_num_bytes_load_total(old_bt_bytes - new_bt_bytes); + DorisMetrics::instance()->load_rows->increment(old_bt_rows - new_bt_rows); + DorisMetrics::instance()->load_bytes->increment(old_bt_bytes - new_bt_bytes); return Status::OK(); } Status VRowDistribution::_generate_rows_distribution_for_auto_partition( - vectorized::Block* block, const std::vector& partition_cols_idx, + const Block& input_block, Block* block, const std::vector& partition_cols_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val) { int num_rows = cast_set(block->rows()); @@ -403,13 +437,14 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_partition( RETURN_IF_ERROR(_filter_block(block, row_part_tablet_ids)); if (!_missing_map.empty()) { - RETURN_IF_ERROR(_deal_missing_map(block, partition_cols_idx, rows_stat_val)); + RETURN_IF_ERROR(_deal_missing_map(input_block, block, partition_cols_idx, + rows_stat_val)); // send input block to save } return Status::OK(); } Status VRowDistribution::_generate_rows_distribution_for_auto_overwrite( - vectorized::Block* block, const std::vector& partition_cols_idx, + const Block& input_block, Block* block, const std::vector& partition_cols_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val) { int num_rows = cast_set(block->rows()); @@ -432,7 +467,8 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_overwrite( // allow and really need to create during auto-detect-overwriting. if (!_missing_map.empty()) { - RETURN_IF_ERROR(_deal_missing_map(block, partition_cols_idx, rows_stat_val)); + RETURN_IF_ERROR( + _deal_missing_map(input_block, block, partition_cols_idx, rows_stat_val)); } } else { RETURN_IF_ERROR(_tablet_finder->find_tablets(_state, block, num_rows, _partitions, @@ -486,21 +522,19 @@ void VRowDistribution::_reset_row_part_tablet_ids( } Status VRowDistribution::generate_rows_distribution( - vectorized::Block& input_block, std::shared_ptr& block, - int64_t& filtered_rows, std::vector& row_part_tablet_ids, - int64_t& rows_stat_val) { + Block& input_block, std::shared_ptr& block, + std::vector& row_part_tablet_ids, int64_t& rows_stat_val) { auto input_rows = input_block.rows(); _reset_row_part_tablet_ids(row_part_tablet_ids, input_rows); - int64_t prev_filtered_rows = - _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows(); + // we store the batching block with value of `input_block`. so just do all of these again. bool has_filtered_rows = false; RETURN_IF_ERROR(_block_convertor->validate_and_convert_block( _state, &input_block, block, *_vec_output_expr_ctxs, input_rows, has_filtered_rows)); // batching block rows which need new partitions. deal together at finish. if (!_batching_block) [[unlikely]] { - std::unique_ptr tmp_block = block->create_same_struct_block(0); + std::unique_ptr tmp_block = input_block.create_same_struct_block(0); _batching_block = MutableBlock::create_unique(std::move(*tmp_block)); } @@ -518,7 +552,7 @@ Status VRowDistribution::generate_rows_distribution( // we just calc left range here. leave right to FE to avoid dup calc. RETURN_IF_ERROR(part_funcs[i]->execute(part_ctxs[i].get(), block.get(), &result_idx)); - VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(0, 1); + VLOG_DEBUG << "Partition-calculated block:\n" << block->dump_data(0, 1); DCHECK(result_idx != -1); partition_cols_idx.push_back(cast_set(result_idx)); @@ -531,20 +565,18 @@ Status VRowDistribution::generate_rows_distribution( Status st = Status::OK(); if (_vpartition->is_auto_detect_overwrite() && !_deal_batched) { // when overwrite, no auto create partition allowed. - st = _generate_rows_distribution_for_auto_overwrite(block.get(), partition_cols_idx, - has_filtered_rows, row_part_tablet_ids, - rows_stat_val); + st = _generate_rows_distribution_for_auto_overwrite(input_block, block.get(), + partition_cols_idx, has_filtered_rows, + row_part_tablet_ids, rows_stat_val); } else if (_vpartition->is_auto_partition() && !_deal_batched) { - st = _generate_rows_distribution_for_auto_partition(block.get(), partition_cols_idx, - has_filtered_rows, row_part_tablet_ids, - rows_stat_val); + st = _generate_rows_distribution_for_auto_partition(input_block, block.get(), + partition_cols_idx, has_filtered_rows, + row_part_tablet_ids, rows_stat_val); } else { // not auto partition st = _generate_rows_distribution_for_non_auto_partition(block.get(), has_filtered_rows, row_part_tablet_ids); } - filtered_rows = _block_convertor->num_filtered_rows() + _tablet_finder->num_filtered_rows() - - prev_filtered_rows; return st; } diff --git a/be/src/vec/sink/vrow_distribution.h b/be/src/vec/sink/vrow_distribution.h index 1615da5ffa8e28..b0161bdd13a95a 100644 --- a/be/src/vec/sink/vrow_distribution.h +++ b/be/src/vec/sink/vrow_distribution.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -107,6 +108,21 @@ class VRowDistribution { _create_partition_callback = ctx.create_partition_callback; } + void output_profile_info(RuntimeProfile* profile) { + if (!_add_partition_request_times.empty()) { + std::stringstream ss; + ss << "["; + for (size_t i = 0; i < _add_partition_request_times.size(); ++i) { + if (i > 0) { + ss << ", "; + } + ss << PrettyPrinter::print(_add_partition_request_times[i], TUnit::TIME_NS); + } + ss << "]"; + profile->add_info_string("AddPartitionRequestTimeList", ss.str()); + } + } + Status open(RowDescriptor* output_row_desc) { if (_vpartition->is_auto_partition()) { auto [part_ctxs, part_funcs] = _get_partition_function(); @@ -129,55 +145,55 @@ class VRowDistribution { // mv where clause // v1 needs index->node->row_ids - tabletids // v2 needs index,tablet->rowids - Status generate_rows_distribution(vectorized::Block& input_block, - std::shared_ptr& block, - int64_t& filtered_rows, + Status generate_rows_distribution(Block& input_block, std::shared_ptr& block, std::vector& row_part_tablet_ids, int64_t& rows_stat_val); + // have 2 ways remind to deal batching block: + // 1. in row_distribution, _batching_rows reaches the threshold, this class set _deal_batched = true. + // 2. in caller, after last block and before close, set _deal_batched = true. bool need_deal_batching() const { return _deal_batched && _batching_rows > 0; } - size_t batching_rows() const { return _batching_rows; } // create partitions when need for auto-partition table using #_partitions_need_create. Status automatic_create_partition(); void clear_batching_stats(); + const std::vector& get_skipped() const { return _skip; } // skipped in last round // for auto partition - std::unique_ptr _batching_block; + std::unique_ptr _batching_block; // same structure with input_block bool _deal_batched = false; // If true, send batched block before any block's append. private: - std::pair _get_partition_function(); + std::pair _get_partition_function(); - Status _save_missing_values(std::vector>& col_strs, int col_size, - Block* block, const std::vector& filter, + Status _save_missing_values(const Block& input_block, + std::vector>& col_strs, int col_size, + Block* block, const std::vector& filter, const std::vector& col_null_maps); - void _get_tablet_ids(vectorized::Block* block, int32_t index_idx, - std::vector& tablet_ids); + void _get_tablet_ids(Block* block, int32_t index_idx, std::vector& tablet_ids); - void _filter_block_by_skip(vectorized::Block* block, RowPartTabletIds& row_part_tablet_id); + void _filter_block_by_skip(Block* block, RowPartTabletIds& row_part_tablet_id); - Status _filter_block_by_skip_and_where_clause(vectorized::Block* block, - const vectorized::VExprContextSPtr& where_clause, + Status _filter_block_by_skip_and_where_clause(Block* block, + const VExprContextSPtr& where_clause, RowPartTabletIds& row_part_tablet_id); - Status _filter_block(vectorized::Block* block, - std::vector& row_part_tablet_ids); + Status _filter_block(Block* block, std::vector& row_part_tablet_ids); Status _generate_rows_distribution_for_auto_partition( - vectorized::Block* block, const std::vector& partition_col_idx, + const Block& input_block, Block* block, const std::vector& partition_col_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val); // the whole process to deal missing rows. will call _save_missing_values - Status _deal_missing_map(vectorized::Block* block, + Status _deal_missing_map(const Block& input_block, Block* block, const std::vector& partition_cols_idx, int64_t& rows_stat_val); Status _generate_rows_distribution_for_non_auto_partition( - vectorized::Block* block, bool has_filtered_rows, + Block* block, bool has_filtered_rows, std::vector& row_part_tablet_ids); Status _generate_rows_distribution_for_auto_overwrite( - vectorized::Block* block, const std::vector& partition_cols_idx, + const Block& input_block, Block* block, const std::vector& partition_cols_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val); Status _replace_overwriting_partition(); @@ -217,6 +233,9 @@ class VRowDistribution { int64_t _txn_id = -1; ObjectPool* _pool = nullptr; OlapTableLocationParam* _location = nullptr; + + // Record each auto-partition request time for detailed profiling + std::vector _add_partition_request_times; // int64_t _number_output_rows = 0; const VExprContextSPtrs* _vec_output_expr_ctxs = nullptr; // generally it's writer's on_partitions_created @@ -230,7 +249,7 @@ class VRowDistribution { std::vector _skip; std::vector _tablet_indexes; std::vector _tablet_ids; - std::vector _missing_map; // indice of missing values in partition_col + std::vector _missing_map; // indice of missing values in partition_col // for auto detect overwrite partition std::set _new_partition_ids; // if contains, not to replace it again. }; diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index 834d1f31d7490d..580becb77eae85 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -36,7 +36,7 @@ namespace doris::vectorized { Status OlapTabletFinder::find_tablets(RuntimeState* state, Block* block, int rows, std::vector& partitions, std::vector& tablet_index, std::vector& skip, - std::vector* miss_rows) { + std::vector* miss_rows) { for (int index = 0; index < rows; index++) { _vpartition->find_partition(block, index, partitions[index]); } diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 129697bb9c5bf2..3adece57ce76e6 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -46,7 +46,7 @@ class OlapTabletFinder { Status find_tablets(RuntimeState* state, vectorized::Block* block, int rows, std::vector& partitions, std::vector& tablet_index, std::vector& skip, - std::vector* miss_rows = nullptr); + std::vector* miss_rows = nullptr); bool is_find_tablet_every_sink() { return _find_tablet_mode == FindTabletMode::FIND_TABLET_EVERY_SINK; diff --git a/be/src/vec/sink/writer/vtablet_writer.cpp b/be/src/vec/sink/writer/vtablet_writer.cpp index 74ee16e97a8056..868607cb18129a 100644 --- a/be/src/vec/sink/writer/vtablet_writer.cpp +++ b/be/src/vec/sink/writer/vtablet_writer.cpp @@ -1901,12 +1901,10 @@ Status VTabletWriter::close(Status exec_status) { } writer_stats.num_node_channels += index_channel->num_node_channels(); - if (add_batch_exec_time > writer_stats.max_add_batch_exec_time_ns) { - writer_stats.max_add_batch_exec_time_ns = add_batch_exec_time; - } - if (wait_exec_time > writer_stats.max_wait_exec_time_ns) { - writer_stats.max_wait_exec_time_ns = wait_exec_time; - } + writer_stats.max_add_batch_exec_time_ns = + std::max(add_batch_exec_time, writer_stats.max_add_batch_exec_time_ns); + writer_stats.max_wait_exec_time_ns = + std::max(wait_exec_time, writer_stats.max_wait_exec_time_ns); } // end for index channels if (status.ok()) { @@ -1954,6 +1952,11 @@ Status VTabletWriter::close(Status exec_status) { _state->update_num_rows_load_unselected( _tablet_finder->num_immutable_partition_filtered_rows()); + if (_state->enable_profile() && _state->profile_level() >= 2) { + // Output detailed profiling info for auto-partition requests + _row_distribution.output_profile_info(_operator_profile); + } + // print log of add batch time of all node, for tracing load performance easily std::stringstream ss; ss << "finished to close olap table sink. load_id=" << print_id(_load_id) @@ -2053,7 +2056,6 @@ Status VTabletWriter::write(RuntimeState* state, doris::vectorized::Block& input SCOPED_RAW_TIMER(&_send_data_ns); std::shared_ptr block; - int64_t filtered_rows = 0; _number_input_rows += rows; // update incrementally so that FE can get the progress. // the real 'num_rows_load_total' will be set when sink being closed. @@ -2064,7 +2066,7 @@ Status VTabletWriter::write(RuntimeState* state, doris::vectorized::Block& input _row_distribution_watch.start(); RETURN_IF_ERROR(_row_distribution.generate_rows_distribution( - input_block, block, filtered_rows, _row_part_tablet_ids, _number_input_rows)); + input_block, block, _row_part_tablet_ids, _number_input_rows)); ChannelDistributionPayloadVec channel_to_payload; diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index 6bc7bd6895cff7..dad781059e5a92 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -238,6 +238,8 @@ Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) { ADD_CHILD_TIMER_WITH_LEVEL(_operator_profile, "RowDistributionTime", "SendDataTime", 1); _write_memtable_timer = ADD_CHILD_TIMER_WITH_LEVEL(_operator_profile, "WriteMemTableTime", "SendDataTime", 1); + _add_partition_request_timer = ADD_CHILD_TIMER_WITH_LEVEL( + _operator_profile, "AddPartitionRequestTime", "SendDataTime", 1); _validate_data_timer = ADD_TIMER_WITH_LEVEL(_operator_profile, "ValidateDataTime", 1); _open_timer = ADD_TIMER(_operator_profile, "OpenTime"); _close_timer = ADD_TIMER(_operator_profile, "CloseWaitTime"); @@ -501,15 +503,13 @@ Status VTabletWriterV2::write(RuntimeState* state, Block& input_block) { DorisMetrics::instance()->load_rows->increment(input_rows); DorisMetrics::instance()->load_bytes->increment(input_bytes); - int64_t filtered_rows = 0; - SCOPED_RAW_TIMER(&_send_data_ns); // This is just for passing compilation. _row_distribution_watch.start(); std::shared_ptr block; RETURN_IF_ERROR(_row_distribution.generate_rows_distribution( - input_block, block, filtered_rows, _row_part_tablet_ids, _number_input_rows)); + input_block, block, _row_part_tablet_ids, _number_input_rows)); RowsForTablet rows_for_tablet; _generate_rows_for_tablet(_row_part_tablet_ids, rows_for_tablet); @@ -741,6 +741,11 @@ Status VTabletWriterV2::close(Status exec_status) { _state->update_num_rows_load_unselected( _tablet_finder->num_immutable_partition_filtered_rows()); + if (_state->enable_profile() && _state->profile_level() >= 2) { + // Output detailed profiling info for auto-partition requests + _row_distribution.output_profile_info(_operator_profile); + } + LOG(INFO) << "finished to close olap table sink. load_id=" << print_id(_load_id) << ", txn_id=" << _txn_id; } else { diff --git a/be/test/pipeline/local_exchanger_test.cpp b/be/test/pipeline/local_exchanger_test.cpp index d3a9b0e2d5d350..c87712caf5e480 100644 --- a/be/test/pipeline/local_exchanger_test.cpp +++ b/be/test/pipeline/local_exchanger_test.cpp @@ -18,17 +18,16 @@ #include #include +#include + #include "common/status.h" #include "pipeline/dependency.h" -#include "pipeline/exec/exchange_source_operator.h" -#include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" #include "pipeline/local_exchange/local_exchange_source_operator.h" #include "thrift_builder.h" #include "vec/columns/column.h" #include "vec/columns/column_vector.h" #include "vec/data_types/data_type.h" -#include "vec/data_types/data_type_number.h" #include "vec/exprs/vslot_ref.h" namespace doris::pipeline { @@ -59,7 +58,7 @@ class LocalExchangerTest : public testing::Test { private: std::unique_ptr _runtime_state; - TUniqueId _query_id = TUniqueId(); + TUniqueId _query_id; int _fragment_id = 0; TQueryOptions _query_options; std::shared_ptr _query_ctx; @@ -99,16 +98,17 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { auto* exchanger = (ShuffleExchanger*)shared_state->exchanger.get(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; - _sink_local_states[i]->_partitioner.reset( - new vectorized::Crc32HashPartitioner( - num_partitions)); + _sink_local_states[i]->_partitioner = + std::make_unique>( + + num_partitions); auto texpr = TExprNodeBuilder(TExprNodeType::SLOT_REF, TTypeDescBuilder() @@ -133,10 +133,10 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -151,7 +151,7 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { { // Enqueue 2 blocks with 10 rows for each data queue. for (size_t i = 0; i < num_partitions; i++) { - hash_vals_and_value.push_back({std::vector {}, i}); + hash_vals_and_value.emplace_back(std::vector {}, i); for (size_t j = 0; j < num_blocks; j++) { vectorized::Block in_block; vectorized::DataTypePtr int_type = std::make_shared(); @@ -168,13 +168,14 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; - EXPECT_EQ(exchanger->sink( - _runtime_state.get(), &in_block, in_eos, - {_sink_local_states[i]->_compute_hash_value_timer, - _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), &shuffle_idx_to_instance_idx}), + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = &shuffle_idx_to_instance_idx}; + EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, + {_sink_local_states[i]->_compute_hash_value_timer, + _sink_local_states[i]->_distribute_timer, nullptr}, + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); EXPECT_EQ(_sink_local_states[i]->_dependency->ready(), i < num_partitions - 1); @@ -228,12 +229,14 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = &shuffle_idx_to_instance_idx}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), &shuffle_idx_to_instance_idx}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -273,7 +276,7 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { EXPECT_EQ(_local_states[i]->_dependency->ready(), true); } for (size_t i = 0; i < num_sources; i++) { - exchanger->close({cast_set(i), nullptr}); + exchanger->close({.channel_id = cast_set(i), .local_state = nullptr}); } for (size_t i = 0; i < num_sources; i++) { shared_state->sub_running_source_operators(); @@ -287,7 +290,7 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { // After exchanger closed, data will never push into data queue again. hash_vals_and_value.clear(); for (size_t i = 0; i < num_partitions; i++) { - hash_vals_and_value.push_back({std::vector {}, i}); + hash_vals_and_value.emplace_back(std::vector {}, i); vectorized::Block in_block; vectorized::DataTypePtr int_type = std::make_shared(); auto int_col0 = vectorized::ColumnInt32::create(); @@ -302,12 +305,14 @@ TEST_F(LocalExchangerTest, ShuffleExchanger) { cast_set(int_col0->size()), 0, nullptr); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = &shuffle_idx_to_instance_idx}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), &shuffle_idx_to_instance_idx}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -342,10 +347,10 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { auto* exchanger = (PassthroughExchanger*)shared_state->exchanger.get(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; @@ -356,10 +361,10 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -382,12 +387,14 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_dependency->ready(), i < num_sources - 1); EXPECT_EQ(_sink_local_states[i]->_channel_id, i + 1 + j); @@ -431,12 +438,14 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i + 1 + num_blocks); } @@ -477,7 +486,7 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { EXPECT_EQ(_local_states[i]->_dependency->ready(), true); } for (size_t i = 0; i < num_sources; i++) { - exchanger->close({cast_set(i), nullptr}); + exchanger->close({.channel_id = cast_set(i), .local_state = nullptr}); } for (size_t i = 0; i < num_sources; i++) { shared_state->sub_running_source_operators(); @@ -496,12 +505,14 @@ TEST_F(LocalExchangerTest, PassthroughExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i + 2 + num_blocks); } @@ -536,10 +547,10 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { auto* exchanger = (PassToOneExchanger*)shared_state->exchanger.get(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; @@ -550,10 +561,10 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -576,12 +587,14 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_dependency->ready(), i < num_sources - 1); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); @@ -633,12 +646,14 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -679,7 +694,7 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { EXPECT_EQ(_local_states[i]->_dependency->ready(), true); } for (size_t i = 0; i < num_sources; i++) { - exchanger->close({cast_set(i), nullptr}); + exchanger->close({.channel_id = cast_set(i), .local_state = nullptr}); } for (size_t i = 0; i < num_sources; i++) { shared_state->sub_running_source_operators(); @@ -698,12 +713,14 @@ TEST_F(LocalExchangerTest, PassToOneExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -738,10 +755,10 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { auto* exchanger = (BroadcastExchanger*)shared_state->exchanger.get(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; @@ -752,10 +769,10 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -778,12 +795,14 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_dependency->ready(), i < num_sources - 1); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); @@ -827,12 +846,14 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -873,7 +894,7 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { EXPECT_EQ(_local_states[i]->_dependency->ready(), true); } for (size_t i = 0; i < num_sources; i++) { - exchanger->close({cast_set(i), nullptr}); + exchanger->close({.channel_id = cast_set(i), .local_state = nullptr}); } for (size_t i = 0; i < num_sources; i++) { shared_state->sub_running_source_operators(); @@ -892,12 +913,14 @@ TEST_F(LocalExchangerTest, BroadcastExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i); } @@ -935,10 +958,10 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { auto* exchanger = (AdaptivePassthroughExchanger*)shared_state->exchanger.get(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; @@ -949,10 +972,10 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -977,12 +1000,14 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); EXPECT_EQ(expect_block_bytes, in_block.allocated_bytes()); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_dependency->ready(), i < num_sources - 1) << i << " " << j << " " << shared_state->mem_usage; @@ -1032,12 +1057,14 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { int_col0->insert_many_vals(i, num_rows_per_block); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i == 0 ? i + 1 : i + 1 + num_blocks); } @@ -1078,7 +1105,7 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { EXPECT_EQ(_local_states[i]->_dependency->ready(), true); } for (size_t i = 0; i < num_sources; i++) { - exchanger->close({cast_set(i), nullptr}); + exchanger->close({.channel_id = cast_set(i), .local_state = nullptr}); } for (size_t i = 0; i < num_sources; i++) { shared_state->sub_running_source_operators(); @@ -1097,12 +1124,14 @@ TEST_F(LocalExchangerTest, AdaptivePassthroughExchanger) { int_col0->insert_many_vals(i, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; + SinkInfo sink_info = {.channel_id = &_sink_local_states[i]->_channel_id, + .partitioner = _sink_local_states[i]->_partitioner.get(), + .local_state = _sink_local_states[i].get(), + .shuffle_idx_to_instance_idx = nullptr}; EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, {_sink_local_states[i]->_compute_hash_value_timer, _sink_local_states[i]->_distribute_timer, nullptr}, - {&_sink_local_states[i]->_channel_id, - _sink_local_states[i]->_partitioner.get(), - _sink_local_states[i].get(), nullptr}), + sink_info), Status::OK()); EXPECT_EQ(_sink_local_states[i]->_channel_id, i == 0 ? i + 2 : i + 2 + num_blocks); } @@ -1154,18 +1183,19 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { .set_slot_ref(TSlotRefBuilder(0, 0).build()) .build(); std::vector texprs; - texprs.push_back(TExpr {}); + texprs.emplace_back(); for (size_t i = 0; i < num_sink; i++) { - auto compute_hash_value_timer = + auto* compute_hash_value_timer = ADD_TIMER(profile, "ComputeHashValueTime" + std::to_string(i)); - auto distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); - _sink_local_states[i].reset(new LocalExchangeSinkLocalState(nullptr, nullptr)); + auto* distribute_timer = ADD_TIMER(profile, "distribute_timer" + std::to_string(i)); + _sink_local_states[i] = std::make_unique(nullptr, nullptr); _sink_local_states[i]->_exchanger = shared_state->exchanger.get(); _sink_local_states[i]->_compute_hash_value_timer = compute_hash_value_timer; _sink_local_states[i]->_distribute_timer = distribute_timer; - _sink_local_states[i]->_partitioner.reset( - new vectorized::Crc32HashPartitioner( - num_partitions)); + _sink_local_states[i]->_partitioner = + std::make_unique>( + + num_partitions); auto slot = doris::vectorized::VSlotRef::create_shared(texpr); slot->_column_id = 0; ((vectorized::Crc32HashPartitioner*)_sink_local_states[i] @@ -1179,10 +1209,10 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { "SinkMemoryUsage" + std::to_string(i), TUnit::BYTES, "", 1); } for (size_t i = 0; i < num_sources; i++) { - auto get_block_failed_counter = + auto* get_block_failed_counter = ADD_TIMER(profile, "_get_block_failed_counter" + std::to_string(i)); - auto copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); - _local_states[i].reset(new LocalExchangeSourceLocalState(nullptr, nullptr)); + auto* copy_data_timer = ADD_TIMER(profile, "_copy_data_timer" + std::to_string(i)); + _local_states[i] = std::make_unique(nullptr, nullptr); _local_states[i]->_exchanger = shared_state->exchanger.get(); _local_states[i]->_get_block_failed_counter = get_block_failed_counter; _local_states[i]->_copy_data_timer = copy_data_timer; @@ -1196,7 +1226,7 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { const auto num_blocks = 1; { for (size_t i = 0; i < num_partitions; i++) { - hash_vals_and_value.push_back({std::vector {}, i}); + hash_vals_and_value.emplace_back(std::vector {}, i); for (size_t j = 0; j < num_blocks; j++) { vectorized::Block in_block; vectorized::DataTypePtr int_type = std::make_shared(); @@ -1217,7 +1247,7 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { { // Enqueue 2 blocks with 10 rows for each data queue. for (size_t i = 0; i < num_partitions; i++) { - hash_vals_and_value.push_back({std::vector {}, i}); + hash_vals_and_value.emplace_back(std::vector {}, i); for (size_t j = 0; j < num_blocks; j++) { vectorized::Block in_block; vectorized::DataTypePtr int_type = std::make_shared(); @@ -1225,13 +1255,14 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { int_col0->insert_many_vals(hash_vals_and_value[i].second, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; - EXPECT_EQ(exchanger->sink( - _runtime_state.get(), &in_block, in_eos, - {_sink_local_states[0]->_compute_hash_value_timer, - _sink_local_states[0]->_distribute_timer, nullptr}, - {&_sink_local_states[0]->_channel_id, - _sink_local_states[0]->_partitioner.get(), - _sink_local_states[0].get(), &shuffle_idx_to_instance_idx}), + SinkInfo sink_info = {.channel_id = &_sink_local_states[0]->_channel_id, + .partitioner = _sink_local_states[0]->_partitioner.get(), + .local_state = _sink_local_states[0].get(), + .shuffle_idx_to_instance_idx = &shuffle_idx_to_instance_idx}; + EXPECT_EQ(exchanger->sink(_runtime_state.get(), &in_block, in_eos, + {_sink_local_states[0]->_compute_hash_value_timer, + _sink_local_states[0]->_distribute_timer, nullptr}, + sink_info), Status::OK()); } } @@ -1252,15 +1283,16 @@ TEST_F(LocalExchangerTest, TestShuffleExchangerWrongMap) { int_col0->insert_many_vals(hash_vals_and_value[0].second, 10); in_block.insert({std::move(int_col0), int_type, "test_int_col0"}); bool in_eos = false; - EXPECT_TRUE( - exchanger - ->sink(_runtime_state.get(), &in_block, in_eos, - {_sink_local_states[0]->_compute_hash_value_timer, - _sink_local_states[0]->_distribute_timer, nullptr}, - {&_sink_local_states[0]->_channel_id, - _sink_local_states[0]->_partitioner.get(), - _sink_local_states[0].get(), &wrong_shuffle_idx_to_instance_idx}) - .is()); + SinkInfo sink_info = {.channel_id = &_sink_local_states[0]->_channel_id, + .partitioner = _sink_local_states[0]->_partitioner.get(), + .local_state = _sink_local_states[0].get(), + .shuffle_idx_to_instance_idx = &wrong_shuffle_idx_to_instance_idx}; + EXPECT_TRUE(exchanger + ->sink(_runtime_state.get(), &in_block, in_eos, + {_sink_local_states[0]->_compute_hash_value_timer, + _sink_local_states[0]->_distribute_timer, nullptr}, + sink_info) + .is()); } } } // namespace doris::pipeline diff --git a/be/test/pipeline/operator/partitioned_hash_join_test_helper.h b/be/test/pipeline/operator/partitioned_hash_join_test_helper.h index 47cac3f0921acd..f79781e0a0f891 100644 --- a/be/test/pipeline/operator/partitioned_hash_join_test_helper.h +++ b/be/test/pipeline/operator/partitioned_hash_join_test_helper.h @@ -49,7 +49,6 @@ class MockPartitionedHashJoinSharedState : public PartitionedHashJoinSharedState partitioned_build_blocks.clear(); } - // 添加必要的初始化方法 void init(size_t partition_count) { spilled_streams.resize(partition_count); partitioned_build_blocks.resize(partition_count); diff --git a/be/test/pipeline/operator/spillable_operator_test_helper.h b/be/test/pipeline/operator/spillable_operator_test_helper.h index 79f03268e245fd..bbd6838611df80 100644 --- a/be/test/pipeline/operator/spillable_operator_test_helper.h +++ b/be/test/pipeline/operator/spillable_operator_test_helper.h @@ -24,11 +24,13 @@ #include #include +#include #include "common/object_pool.h" #include "pipeline/pipeline_task.h" #include "testutil/mock/mock_runtime_state.h" #include "util/runtime_profile.h" +#include "vec/core/block.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -46,11 +48,7 @@ class MockPartitioner : public vectorized::PartitionerBase { Status close(RuntimeState* state) override { return Status::OK(); } - Status do_partitioning(RuntimeState* state, vectorized::Block* block, bool eos, - bool* already_sent) const override { - if (already_sent) { - *already_sent = false; - } + Status do_partitioning(RuntimeState* state, vectorized::Block* block) const override { return Status::OK(); } @@ -59,7 +57,10 @@ class MockPartitioner : public vectorized::PartitionerBase { return Status::OK(); } - vectorized::ChannelField get_channel_ids() const override { return {}; } + const std::vector& get_channel_ids() const override { return _mocked_hash_vals; } + +private: + std::vector _mocked_hash_vals; }; class MockExpr : public vectorized::VExpr { diff --git a/be/test/pipeline/shuffle/exchange_writer_test.cpp b/be/test/pipeline/shuffle/exchange_writer_test.cpp new file mode 100644 index 00000000000000..1a39d5918838fd --- /dev/null +++ b/be/test/pipeline/shuffle/exchange_writer_test.cpp @@ -0,0 +1,233 @@ +// 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 "pipeline/shuffle/exchange_writer.h" + +#include + +#include +#include + +#include "gen_cpp/Types_types.h" +#include "pipeline/exec/exchange_sink_operator.h" +#include "testutil/column_helper.h" +#include "testutil/mock/mock_data_stream_sender.h" +#include "testutil/mock/mock_runtime_state.h" + +using doris::MockRuntimeState; +using doris::Status; +using doris::vectorized::Block; +using doris::vectorized::ColumnHelper; +using doris::vectorized::DataTypeInt32; +using doris::vectorized::Channel; +using doris::vectorized::MockChannel; +using doris::pipeline::ExchangeSinkLocalState; + +namespace doris::pipeline { + +// Helper: create channels that will never actually send rows (is_receiver_eof == true), +// so writer logic can be tested without exercising Channel::add_rows / BlockSerializer. +static std::shared_ptr make_disabled_channel(ExchangeSinkLocalState* local_state) { + TUniqueId id; + id.hi = 0; + id.lo = 0; + auto ch = std::make_shared(local_state, id, /*is_local=*/true); + ch->set_receiver_eof(Status::EndOfFile("test eof")); + return ch; +} + +static std::vector> make_disabled_channels( + ExchangeSinkLocalState* local_state, size_t n) { + std::vector> channels; + channels.reserve(n); + for (size_t i = 0; i < n; ++i) { + channels.emplace_back(make_disabled_channel(local_state)); + } + return channels; +} + +TEST(TrivialExchangeWriterTest, BasicDistribution) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeTrivialWriter writer {local_state}; + + const size_t channel_count = 2; + auto channels = make_disabled_channels(&local_state, channel_count); + + // rows: [1,2,3,4,5], channel_ids: [0,1,0,1,1] + Block block = ColumnHelper::create_block({1, 2, 3, 4, 5}); + std::vector channel_ids = {0, 1, 0, 1, 1}; + const size_t rows = channel_ids.size(); + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false); + ASSERT_TRUE(st.ok()) << st.to_string(); + + // Expect histogram: [2,3] + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + EXPECT_EQ(writer._channel_rows_histogram[0], 2U); + EXPECT_EQ(writer._channel_rows_histogram[1], 3U); + + // Expect row index order: [0,2,1,3,4] + ASSERT_EQ(writer._origin_row_idx.size(), rows); + std::vector got(rows); + for (size_t i = 0; i < rows; ++i) { + got[i] = writer._origin_row_idx[i]; + } + std::vector expected {0, 2, 1, 3, 4}; + EXPECT_EQ(got, expected); +} + +TEST(TrivialExchangeWriterTest, AllRowsToSingleChannel) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeTrivialWriter writer {local_state}; + + const size_t channel_count = 3; + auto channels = make_disabled_channels(&local_state, channel_count); + + Block block = ColumnHelper::create_block({10, 20, 30, 40}); + std::vector channel_ids = {2, 2, 2, 2}; + const size_t rows = channel_ids.size(); + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + EXPECT_EQ(writer._channel_rows_histogram[0], 0U); + EXPECT_EQ(writer._channel_rows_histogram[1], 0U); + EXPECT_EQ(writer._channel_rows_histogram[2], 4U); + + ASSERT_EQ(writer._origin_row_idx.size(), rows); + std::vector got(rows); + for (size_t i = 0; i < rows; ++i) { + got[i] = writer._origin_row_idx[i]; + } + std::vector expected {0, 1, 2, 3}; + EXPECT_EQ(got, expected); +} + +TEST(TrivialExchangeWriterTest, EmptyInput) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeTrivialWriter writer {local_state}; + + const size_t channel_count = 4; + auto channels = make_disabled_channels(&local_state, channel_count); + + Block block = ColumnHelper::create_block({}); + std::vector channel_ids {}; + const size_t rows = 0; + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + for (size_t i = 0; i < channel_count; ++i) { + EXPECT_EQ(writer._channel_rows_histogram[i], 0U); + } + EXPECT_EQ(writer._origin_row_idx.size(), 0U); +} + +TEST(OlapExchangeWriterTest, NeedCheckSkipsInvalidChannelIds) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeOlapWriter writer {local_state}; + + const size_t channel_count = 3; + auto channels = make_disabled_channels(&local_state, channel_count); + + // channel_ids: [0, x, 2, x, 2] + Block block = ColumnHelper::create_block({10, 20, 30, 40, 50}); + std::vector channel_ids = {0, 10, 2, 10, 2}; + const size_t rows = channel_ids.size(); + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false, 10); + ASSERT_TRUE(st.ok()) << st.to_string(); + + // Only valid ids(less than _partition_count) should be counted: hist = [1,0,2] + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + EXPECT_EQ(writer._channel_rows_histogram[0], 1U); + EXPECT_EQ(writer._channel_rows_histogram[1], 0U); + EXPECT_EQ(writer._channel_rows_histogram[2], 2U); + + // row_idx should contain rows [0,2,4] grouped by channel + ASSERT_EQ(writer._origin_row_idx.size(), 3U); + std::vector got(3); + for (size_t i = 0; i < 3; ++i) { + got[i] = writer._origin_row_idx[i]; + } + std::vector expected {0, 2, 4}; + EXPECT_EQ(got, expected); +} + +TEST(OlapExchangeWriterTest, NoCheckUsesAllRows) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeOlapWriter writer {local_state}; + + const size_t channel_count = 2; + auto channels = make_disabled_channels(&local_state, channel_count); + + Block block = ColumnHelper::create_block({1, 2, 3}); + std::vector channel_ids = {0, 1, 0}; + const size_t rows = channel_ids.size(); + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false, 10); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + EXPECT_EQ(writer._channel_rows_histogram[0], 2U); + EXPECT_EQ(writer._channel_rows_histogram[1], 1U); + + ASSERT_EQ(writer._origin_row_idx.size(), rows); + std::vector got(rows); + for (size_t i = 0; i < rows; ++i) { + got[i] = writer._origin_row_idx[i]; + } + std::vector expected {0, 2, 1}; + EXPECT_EQ(got, expected); +} + +TEST(OlapExchangeWriterTest, EmptyInput) { + MockRuntimeState state; + ExchangeSinkLocalState local_state(&state); + ExchangeOlapWriter writer {local_state}; + + const size_t channel_count = 3; + auto channels = make_disabled_channels(&local_state, channel_count); + + Block block = ColumnHelper::create_block({}); + std::vector channel_ids {}; + const size_t rows = 0; + + Status st = writer._channel_add_rows(&state, channels, channel_count, channel_ids, rows, &block, + /*eos=*/false, 1); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(writer._channel_rows_histogram.size(), channel_count); + for (size_t i = 0; i < channel_count; ++i) { + EXPECT_EQ(writer._channel_rows_histogram[i], 0U); + } + EXPECT_EQ(writer._origin_row_idx.size(), 0U); +} + +} // namespace doris::pipeline diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp index 86902a94bec588..dd09133e2fb99f 100644 --- a/be/test/vec/core/block_test.cpp +++ b/be/test/vec/core/block_test.cpp @@ -1286,26 +1286,6 @@ TEST(BlockTest, filter) { } } -TEST(BlockTest, add_rows) { - auto block = vectorized::ColumnHelper::create_block({1, 2, 3}); - block.insert(vectorized::ColumnHelper::create_column_with_name( - {"abc", "efg", "hij"})); - - auto block2 = vectorized::ColumnHelper::create_block({4}); - block2.insert( - vectorized::ColumnHelper::create_column_with_name({"lmn"})); - - vectorized::MutableBlock mutable_block(&block); - mutable_block.add_row(&block2, 0); - ASSERT_EQ(mutable_block.rows(), 4); - - vectorized::MutableBlock mutable_block2(&block2); - auto st = mutable_block2.add_rows(&block, {0, 2}); - ASSERT_TRUE(st.ok()) << st.to_string(); - - ASSERT_EQ(mutable_block2.rows(), 3); -} - TEST(BlockTest, others) { auto block = vectorized::ColumnHelper::create_block({1, 2, 3}); block.insert(vectorized::ColumnHelper::create_column_with_name( diff --git a/be/test/vec/exec/exchange_sink_test.cpp b/be/test/vec/exec/exchange_sink_test.cpp index 0643d3c67b8aa7..3eaf82de67036b 100644 --- a/be/test/vec/exec/exchange_sink_test.cpp +++ b/be/test/vec/exec/exchange_sink_test.cpp @@ -20,13 +20,12 @@ #include #include -#include #include "pipeline/exec/exchange_sink_buffer.h" namespace doris::vectorized { using namespace pipeline; -TEST_F(ExchangeSInkTest, test_normal_end) { +TEST_F(ExchangeSinkTest, test_normal_end) { { auto state = create_runtime_state(); auto buffer = create_buffer(state); @@ -78,7 +77,7 @@ TEST_F(ExchangeSInkTest, test_normal_end) { } } -TEST_F(ExchangeSInkTest, test_eof_end) { +TEST_F(ExchangeSinkTest, test_eof_end) { { auto state = create_runtime_state(); auto buffer = create_buffer(state); @@ -140,7 +139,7 @@ TEST_F(ExchangeSInkTest, test_eof_end) { } } -TEST_F(ExchangeSInkTest, test_error_end) { +TEST_F(ExchangeSinkTest, test_error_end) { { auto state = create_runtime_state(); auto buffer = create_buffer(state); @@ -195,7 +194,7 @@ TEST_F(ExchangeSInkTest, test_error_end) { } } -TEST_F(ExchangeSInkTest, test_queue_size) { +TEST_F(ExchangeSinkTest, test_queue_size) { { auto state = create_runtime_state(); auto buffer = create_buffer(state); diff --git a/be/test/vec/exec/exchange_sink_test.h b/be/test/vec/exec/exchange_sink_test.h index b121571e0b5bcb..3adab7fe034f8a 100644 --- a/be/test/vec/exec/exchange_sink_test.h +++ b/be/test/vec/exec/exchange_sink_test.h @@ -90,10 +90,10 @@ void transmit_blockv2(PBackendService_Stub* stub, namespace doris::vectorized { using namespace pipeline; -class ExchangeSInkTest : public testing::Test { +class ExchangeSinkTest : public testing::Test { public: - ExchangeSInkTest() = default; - ~ExchangeSInkTest() override = default; + ExchangeSinkTest() = default; + ~ExchangeSinkTest() override = default; }; class MockContext : public TaskExecutionContext {}; diff --git a/be/test/vec/sink/sink_test_utils.h b/be/test/vec/sink/sink_test_utils.h new file mode 100644 index 00000000000000..0b9864439c1eeb --- /dev/null +++ b/be/test/vec/sink/sink_test_utils.h @@ -0,0 +1,229 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include + +#include + +#include "exec/tablet_info.h" +#include "pipeline/operator/operator_helper.h" +#include "runtime/descriptor_helper.h" +#include "runtime/descriptors.h" +#include "runtime/types.h" + +namespace doris::vectorized { + +namespace sink_test_utils { + +inline TExprNode make_int_literal(int32_t v) { + TExprNode node; + node.__set_node_type(TExprNodeType::INT_LITERAL); + node.__set_num_children(0); + node.__set_output_scale(0); + + TIntLiteral int_lit; + int_lit.__set_value(v); + node.__set_int_literal(int_lit); + + TTypeDesc type_desc = create_type_desc(PrimitiveType::TYPE_INT); + type_desc.__set_is_nullable(false); + node.__set_type(type_desc); + node.__set_is_nullable(false); + + return node; +} + +inline TExprNode make_bool_literal(bool v) { + TExprNode node; + node.__set_node_type(TExprNodeType::BOOL_LITERAL); + node.__set_num_children(0); + node.__set_output_scale(0); + + TBoolLiteral bool_lit; + bool_lit.__set_value(v); + node.__set_bool_literal(bool_lit); + + TTypeDesc type_desc = create_type_desc(PrimitiveType::TYPE_BOOLEAN); + type_desc.__set_is_nullable(false); + node.__set_type(type_desc); + node.__set_is_nullable(false); + + return node; +} + +inline TExpr make_slot_ref_expr(TSlotId slot_id, TTupleId tuple_id) { + TExprNode node; + node.__set_node_type(TExprNodeType::SLOT_REF); + node.__set_num_children(0); + + TSlotRef slot_ref; + slot_ref.__set_slot_id(slot_id); + slot_ref.__set_tuple_id(tuple_id); + node.__set_slot_ref(slot_ref); + + TTypeDesc type_desc = create_type_desc(PrimitiveType::TYPE_INT); + type_desc.__set_is_nullable(false); + node.__set_type(type_desc); + node.__set_is_nullable(false); + + TExpr expr; + expr.nodes.emplace_back(node); + return expr; +} + +inline void build_desc_tbl_and_schema(doris::pipeline::OperatorContext& ctx, + TOlapTableSchemaParam& tschema, + TTupleId& tablet_sink_tuple_id, int64_t& schema_index_id, + bool is_nullable = true) { + TDescriptorTableBuilder dtb; + { + TTupleDescriptorBuilder tuple_builder; + tuple_builder.add_slot(TSlotDescriptorBuilder() + .type(TYPE_INT) + .nullable(is_nullable) + .column_name("c1") + .column_pos(1) + .build()); + tuple_builder.build(&dtb); + } + + auto thrift_desc_tbl = dtb.desc_tbl(); + DescriptorTbl* desc_tbl = nullptr; + auto st = DescriptorTbl::create(ctx.state.obj_pool(), thrift_desc_tbl, &desc_tbl); + ASSERT_TRUE(st.ok()) << st.to_string(); + ctx.state.set_desc_tbl(desc_tbl); + + tschema.db_id = 1; + tschema.table_id = 2; + tschema.version = 0; + tschema.slot_descs = thrift_desc_tbl.slotDescriptors; + tschema.tuple_desc = thrift_desc_tbl.tupleDescriptors[0]; + + TOlapTableIndexSchema index_schema; + index_schema.id = 10; + index_schema.columns = {"c1"}; + index_schema.schema_hash = 123; + tschema.indexes = {index_schema}; + + tablet_sink_tuple_id = tschema.tuple_desc.id; + schema_index_id = index_schema.id; +} + +inline TOlapTablePartitionParam build_partition_param(int64_t schema_index_id) { + TOlapTablePartitionParam param; + param.db_id = 1; + param.table_id = 2; + param.version = 0; + + param.__set_partition_type(TPartitionType::RANGE_PARTITIONED); + param.__set_partition_columns({"c1"}); + param.__set_distributed_columns({"c1"}); + + TOlapTablePartition p1; + p1.id = 1; + p1.num_buckets = 1; + p1.__set_is_mutable(true); + { + TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {100}; + p1.indexes = {index_tablets}; + } + p1.__set_start_keys({make_int_literal(0)}); + p1.__set_end_keys({make_int_literal(10)}); + + TOlapTablePartition p2; + p2.id = 2; + p2.num_buckets = 1; + p2.__set_is_mutable(true); + { + TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {200}; + p2.indexes = {index_tablets}; + } + p2.__set_start_keys({make_int_literal(20)}); + p2.__set_end_keys({make_int_literal(1000)}); + + param.partitions = {p1, p2}; + return param; +} + +inline TOlapTablePartitionParam build_auto_partition_param(int64_t schema_index_id, + TTupleId tuple_id, TSlotId slot_id) { + auto param = build_partition_param(schema_index_id); + param.__set_enable_automatic_partition(true); + param.__set_partition_function_exprs({ + make_slot_ref_expr(slot_id, tuple_id), + }); + return param; +} + +inline TOlapTablePartitionParam build_partition_param_with_load_tablet_idx( + int64_t schema_index_id, int64_t load_tablet_idx) { + TOlapTablePartitionParam param; + param.db_id = 1; + param.table_id = 2; + param.version = 0; + + param.__set_partition_type(TPartitionType::RANGE_PARTITIONED); + param.__set_partition_columns({"c1"}); + + TOlapTablePartition p1; + p1.id = 1; + p1.num_buckets = 2; + p1.__set_is_mutable(true); + p1.__set_load_tablet_idx(load_tablet_idx); + { + TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {100, 101}; + p1.indexes = {index_tablets}; + } + p1.__set_start_keys({make_int_literal(0)}); + p1.__set_end_keys({make_int_literal(1000)}); + + param.partitions = {p1}; + return param; +} + +inline TOlapTableLocationParam build_location_param() { + TOlapTableLocationParam location; + location.db_id = 1; + location.table_id = 2; + location.version = 0; + + TTabletLocation t1; + t1.tablet_id = 100; + t1.node_ids = {1}; + + TTabletLocation t2; + t2.tablet_id = 200; + t2.node_ids = {1}; + + location.tablets = {t1, t2}; + return location; +} + +} // namespace sink_test_utils + +} // namespace doris::vectorized diff --git a/be/test/vec/sink/tablet_sink_hash_partitioner_test.cpp b/be/test/vec/sink/tablet_sink_hash_partitioner_test.cpp new file mode 100644 index 00000000000000..d52412e324d412 --- /dev/null +++ b/be/test/vec/sink/tablet_sink_hash_partitioner_test.cpp @@ -0,0 +1,280 @@ +// 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/sink/tablet_sink_hash_partitioner.h" + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "common/cast_set.h" +#include "common/config.h" +#include "exec/tablet_info.h" +#include "pipeline/exec/exchange_sink_operator.h" +#include "pipeline/operator/operator_helper.h" +#include "runtime/descriptor_helper.h" +#include "runtime/descriptors.h" +#include "runtime/types.h" +#include "testutil/column_helper.h" +#include "testutil/mock/mock_descriptors.h" +#include "util/debug_points.h" +#include "util/hash_util.hpp" +#include "util/runtime_profile.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" +#include "vec/sink/sink_test_utils.h" +#include "vec/sink/vtablet_finder.h" + +namespace doris::vectorized { + +namespace { + +using doris::pipeline::ExchangeSinkLocalState; +using doris::pipeline::ExchangeSinkOperatorX; +using doris::pipeline::OperatorContext; + +std::shared_ptr _create_parent_operator( + OperatorContext& ctx, const std::shared_ptr& row_desc_holder) { + TDataStreamSink sink; + sink.dest_node_id = 0; + sink.output_partition.type = TPartitionType::UNPARTITIONED; + + return std::make_shared(&ctx.state, *row_desc_holder, 0, sink, + std::vector {}, + std::vector {}); +} + +std::unique_ptr _create_partitioner( + OperatorContext& ctx, ExchangeSinkLocalState* local_state, size_t partition_count, + int64_t txn_id, const TOlapTableSchemaParam& schema, + const TOlapTablePartitionParam& partition, const TOlapTableLocationParam& location, + TTupleId tablet_sink_tuple_id) { + auto partitioner = std::make_unique( + partition_count, txn_id, schema, partition, location, tablet_sink_tuple_id, + local_state); + auto st = partitioner->open(&ctx.state); + EXPECT_TRUE(st.ok()) << st.to_string(); + return partitioner; +} + +TEST(TabletSinkHashPartitionerTest, DoPartitioningSkipsImmutablePartitionAndHashesOthers) { + OperatorContext ctx; + constexpr size_t partition_count = 8; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto row_desc_holder = std::make_shared( + std::vector {std::make_shared()}, &ctx.pool); + auto parent_op = _create_parent_operator(ctx, row_desc_holder); + ExchangeSinkLocalState local_state(parent_op.get(), &ctx.state); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + ASSERT_EQ(tpartition.partitions.size(), 2); + // 1: [0, 10), 2: [20, 1000) + tpartition.partitions[0].__set_is_mutable(false); + auto tlocation = sink_test_utils::build_location_param(); + + auto partitioner = _create_partitioner(ctx, &local_state, partition_count, txn_id, tschema, + tpartition, tlocation, tablet_sink_tuple_id); + + // 1 -> no partition, 25 -> p1 + auto block = ColumnHelper::create_block({1, 25}); + auto st = partitioner->do_partitioning(&ctx.state, &block); + ASSERT_TRUE(st.ok()) << st.to_string(); + + const auto& skipped = partitioner->get_skipped(); + ASSERT_EQ(skipped.size(), block.rows()); + EXPECT_TRUE(skipped[0]); + EXPECT_FALSE(skipped[1]); + + auto channel_ids = partitioner->get_channel_ids(); + ASSERT_EQ(channel_ids.size(), 2); + EXPECT_EQ(channel_ids[0], partition_count); // skipped partition + + uint32_t tablet_id = 200; + auto hash = HashUtil::zlib_crc_hash(&tablet_id, sizeof(uint32_t), 0); + EXPECT_EQ(channel_ids[1], static_cast(hash % partition_count)); +} + +TEST(TabletSinkHashPartitionerTest, TryCutInLineCreatesPartitionAndReturnsBatchedBlock) { + OperatorContext ctx; + constexpr size_t partition_count = 8; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + TSlotId partition_slot_id = tschema.slot_descs[0].id; + + auto row_desc_holder = std::make_shared( + std::vector {std::make_shared()}, &ctx.pool); + auto parent_op = _create_parent_operator(ctx, row_desc_holder); + ExchangeSinkLocalState local_state(parent_op.get(), &ctx.state); + + auto tpartition = sink_test_utils::build_auto_partition_param( + schema_index_id, tablet_sink_tuple_id, partition_slot_id); + auto tlocation = sink_test_utils::build_location_param(); + + auto partitioner = _create_partitioner(ctx, &local_state, partition_count, txn_id, tschema, + tpartition, tlocation, tablet_sink_tuple_id); + + doris::config::enable_debug_points = true; + doris::DebugPoints::instance()->clear(); + + bool injected = false; + std::function handler = + [&](doris::TCreatePartitionRequest* req, doris::TCreatePartitionResult* res) { + injected = true; + ASSERT_TRUE(req->__isset.partitionValues); + ASSERT_EQ(req->partitionValues.size(), 1); + ASSERT_EQ(req->partitionValues[0].size(), 1); + ASSERT_TRUE(req->partitionValues[0][0].__isset.value); + EXPECT_EQ(req->partitionValues[0][0].value, "15"); + + doris::TStatus tstatus; + tstatus.__set_status_code(doris::TStatusCode::OK); + res->__set_status(tstatus); + + doris::TOlapTablePartition new_part; + new_part.id = 3; + new_part.num_buckets = 1; + new_part.__set_is_mutable(true); + { + doris::TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {300}; + new_part.indexes = {index_tablets}; + } + new_part.__set_start_keys({sink_test_utils::make_int_literal(10)}); + new_part.__set_end_keys({sink_test_utils::make_int_literal(20)}); + res->__set_partitions({new_part}); + + doris::TTabletLocation new_location; + new_location.__set_tablet_id(300); + new_location.__set_node_ids({1}); + res->__set_tablets({new_location}); + }; + doris::DebugPoints::instance()->add_with_callback( + "VRowDistribution.automatic_create_partition.inject_result", handler); + + { + auto block = ColumnHelper::create_block({15, 15}); + auto st = partitioner->do_partitioning(&ctx.state, &block); + ASSERT_TRUE(st.ok()) << st.to_string(); + + // Flush batching data at end-of-stream. + partitioner->mark_last_block(); + Block batched; + st = partitioner->try_cut_in_line(batched); + ASSERT_TRUE(st.ok()) << st.to_string(); + EXPECT_TRUE(injected); + + ASSERT_EQ(batched.rows(), 2); + ASSERT_EQ(batched.columns(), 1); + const auto& col = batched.get_by_position(0).column; + ASSERT_EQ(col->size(), 2); + EXPECT_EQ(assert_cast(*col).get_data()[0], 15); + EXPECT_EQ(assert_cast(*col).get_data()[1], 15); + } + + doris::DebugPoints::instance()->clear(); + doris::config::enable_debug_points = false; +} + +TEST(TabletSinkHashPartitionerTest, OlapTabletFinderRoundRobinEveryBatch) { + OperatorContext ctx; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto schema = std::make_shared(); + auto st = schema->init(tschema); + ASSERT_TRUE(st.ok()) << st.to_string(); + + auto tpartition = + sink_test_utils::build_partition_param_with_load_tablet_idx(schema_index_id, 0); + auto vpartition = std::make_unique(schema, tpartition); + st = vpartition->init(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + OlapTabletFinder finder(vpartition.get(), + OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH); + + { + auto block = ColumnHelper::create_block({1, 2, 3}); + std::vector partitions(block.rows(), nullptr); + std::vector tablet_index(block.rows(), 0); + std::vector skip(block.rows(), false); + + st = finder.find_tablets(&ctx.state, &block, cast_set(block.rows()), partitions, + tablet_index, skip, nullptr); + ASSERT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(tablet_index[0], 0); + EXPECT_EQ(tablet_index[1], 0); + EXPECT_EQ(tablet_index[2], 0); + } + + { + auto block = ColumnHelper::create_block({1, 2}); + std::vector partitions(block.rows(), nullptr); + std::vector tablet_index(block.rows(), 0); + std::vector skip(block.rows(), false); + + st = finder.find_tablets(&ctx.state, &block, cast_set(block.rows()), partitions, + tablet_index, skip, nullptr); + ASSERT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(tablet_index[0], 1); + EXPECT_EQ(tablet_index[1], 1); + } + + { + auto block = ColumnHelper::create_block({1}); + std::vector partitions(block.rows(), nullptr); + std::vector tablet_index(block.rows(), 0); + std::vector skip(block.rows(), false); + + st = finder.find_tablets(&ctx.state, &block, cast_set(block.rows()), partitions, + tablet_index, skip, nullptr); + ASSERT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(tablet_index[0], 0); + } +} +} // anonymous namespace +} // namespace doris::vectorized diff --git a/be/test/vec/sink/vrow_distribution_test.cpp b/be/test/vec/sink/vrow_distribution_test.cpp new file mode 100644 index 00000000000000..41b4d83769b25e --- /dev/null +++ b/be/test/vec/sink/vrow_distribution_test.cpp @@ -0,0 +1,453 @@ +// 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/sink/vrow_distribution.h" + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "common/config.h" +#include "pipeline/operator/operator_helper.h" +#include "testutil/column_helper.h" +#include "testutil/mock/mock_slot_ref.h" +#include "util/debug_points.h" +#include "vec/data_types/data_type_number.h" +#include "vec/sink/sink_test_utils.h" +#include "vec/sink/vtablet_block_convertor.h" +#include "vec/sink/vtablet_finder.h" + +namespace doris::vectorized { + +namespace { + +using doris::pipeline::OperatorContext; + +struct VRowDistributionHarness { + std::shared_ptr schema; + std::unique_ptr vpartition; + std::unique_ptr location; + std::unique_ptr tablet_finder; + std::unique_ptr block_convertor; + VExprContextSPtrs output_expr_ctxs; + std::unique_ptr output_row_desc; + VRowDistribution row_distribution; +}; + +Status _noop_create_partition_callback(void*, TCreatePartitionResult*) { + return Status::OK(); +} + +std::unique_ptr _build_vrow_distribution_harness( + OperatorContext& ctx, const TOlapTableSchemaParam& tschema, + const TOlapTablePartitionParam& tpartition, const TOlapTableLocationParam& tlocation, + TTupleId tablet_sink_tuple_id, int64_t txn_id) { + auto h = std::make_unique(); + + h->schema = std::make_shared(); + auto st = h->schema->init(tschema); + EXPECT_TRUE(st.ok()) << st.to_string(); + + h->vpartition = std::make_unique(h->schema, tpartition); + st = h->vpartition->init(); + EXPECT_TRUE(st.ok()) << st.to_string(); + + h->location = std::make_unique(tlocation); + h->tablet_finder = std::make_unique(h->vpartition.get(), + OlapTabletFinder::FIND_TABLET_EVERY_ROW); + h->block_convertor = std::make_unique(h->schema->tuple_desc()); + + h->output_row_desc = std::make_unique( + ctx.state.desc_tbl(), std::vector {tablet_sink_tuple_id}); + + VRowDistribution::VRowDistributionContext rctx; + rctx.state = &ctx.state; + rctx.block_convertor = h->block_convertor.get(); + rctx.tablet_finder = h->tablet_finder.get(); + rctx.vpartition = h->vpartition.get(); + rctx.add_partition_request_timer = nullptr; + rctx.txn_id = txn_id; + rctx.pool = &ctx.pool; + rctx.location = h->location.get(); + rctx.vec_output_expr_ctxs = &h->output_expr_ctxs; + rctx.schema = h->schema; + rctx.caller = nullptr; + rctx.write_single_replica = false; + rctx.create_partition_callback = &_noop_create_partition_callback; + h->row_distribution.init(rctx); + + st = h->row_distribution.open(h->output_row_desc.get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + + return h; +} + +TEST(VRowDistributionTest, GenerateRowsDistributionNonAutoPartitionBasic) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + auto input_block = ColumnHelper::create_block({1, 25}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + auto st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_NE(converted_block, nullptr); + + ASSERT_EQ(row_part_tablet_ids.size(), 1); + ASSERT_EQ(row_part_tablet_ids[0].row_ids.size(), 2); + EXPECT_EQ(row_part_tablet_ids[0].row_ids[0], 0); + EXPECT_EQ(row_part_tablet_ids[0].row_ids[1], 1); + EXPECT_EQ(row_part_tablet_ids[0].partition_ids[0], 1); + EXPECT_EQ(row_part_tablet_ids[0].partition_ids[1], 2); +} + +TEST(VRowDistributionTest, GenerateRowsDistributionSkipsImmutablePartition) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + ASSERT_EQ(tpartition.partitions.size(), 2); + tpartition.partitions[0].__set_is_mutable(false); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + auto input_block = ColumnHelper::create_block({1}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + auto st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(row_part_tablet_ids.size(), 1); + EXPECT_TRUE(row_part_tablet_ids[0].row_ids.empty()); + + auto skipped = h->row_distribution.get_skipped(); + ASSERT_EQ(skipped.size(), 1); + EXPECT_TRUE(skipped[0]); +} + +TEST(VRowDistributionTest, GenerateRowsDistributionWhereClauseConstFalseFiltersAllRows) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + TExpr texpr; + texpr.nodes.emplace_back(sink_test_utils::make_bool_literal(false)); + VExprContextSPtr where_ctx; + auto st = VExpr::create_expr_tree(texpr, where_ctx); + ASSERT_TRUE(st.ok()) << st.to_string(); + st = where_ctx->prepare(&ctx.state, *h->output_row_desc); + ASSERT_TRUE(st.ok()) << st.to_string(); + st = where_ctx->open(&ctx.state); + ASSERT_TRUE(st.ok()) << st.to_string(); + h->schema->indexes()[0]->where_clause = where_ctx; + + auto input_block = ColumnHelper::create_block({1, 25}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_EQ(row_part_tablet_ids.size(), 1); + EXPECT_TRUE(row_part_tablet_ids[0].row_ids.empty()); + EXPECT_TRUE(row_part_tablet_ids[0].partition_ids.empty()); + EXPECT_TRUE(row_part_tablet_ids[0].tablet_ids.empty()); +} + +TEST(VRowDistributionTest, GenerateRowsDistributionWhereClauseUInt8ColumnFiltersSomeRows) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + auto where_ctx = VExprContext::create_shared( + std::make_shared(1, std::make_shared())); + auto st = where_ctx->prepare(&ctx.state, *h->output_row_desc); + ASSERT_TRUE(st.ok()) << st.to_string(); + st = where_ctx->open(&ctx.state); + ASSERT_TRUE(st.ok()) << st.to_string(); + h->schema->indexes()[0]->where_clause = where_ctx; + + auto input_block = ColumnHelper::create_block({1, 25, 2}); + auto filter_col_mut = ColumnUInt8::create(); + filter_col_mut->get_data().push_back(1); + filter_col_mut->get_data().push_back(0); + filter_col_mut->get_data().push_back(1); + ColumnPtr filter_col = std::move(filter_col_mut); + input_block.insert({filter_col, std::make_shared(), "f"}); + + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_EQ(row_part_tablet_ids.size(), 1); + ASSERT_EQ(row_part_tablet_ids[0].row_ids.size(), 2); + EXPECT_EQ(row_part_tablet_ids[0].row_ids[0], 0); + EXPECT_EQ(row_part_tablet_ids[0].row_ids[1], 2); +} + +TEST(VRowDistributionTest, AutoPartitionMissingValuesBatchingDedupAndCreatePartition) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + TSlotId partition_slot_id = tschema.slot_descs[0].id; + auto tpartition = sink_test_utils::build_auto_partition_param( + schema_index_id, tablet_sink_tuple_id, partition_slot_id); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + auto input_block = ColumnHelper::create_block({15, 15}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + auto st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_TRUE(h->row_distribution._batching_block); + EXPECT_EQ(h->row_distribution._batching_block->rows(), 2); + + h->row_distribution._deal_batched = true; + EXPECT_TRUE(h->row_distribution.need_deal_batching()); + + doris::config::enable_debug_points = true; + doris::DebugPoints::instance()->clear(); + + bool injected = false; + std::function handler = + [&](doris::TCreatePartitionRequest* req, doris::TCreatePartitionResult* res) { + injected = true; + ASSERT_TRUE(req->__isset.partitionValues); + ASSERT_EQ(req->partitionValues.size(), 1); + ASSERT_EQ(req->partitionValues[0].size(), 1); + ASSERT_TRUE(req->partitionValues[0][0].__isset.value); + EXPECT_EQ(req->partitionValues[0][0].value, "15"); + + doris::TStatus tstatus; + tstatus.__set_status_code(doris::TStatusCode::OK); + res->__set_status(tstatus); + + doris::TOlapTablePartition new_part; + new_part.id = 3; + new_part.num_buckets = 1; + new_part.__set_is_mutable(true); + { + doris::TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {300}; + new_part.indexes = {index_tablets}; + } + new_part.__set_start_keys({sink_test_utils::make_int_literal(10)}); + new_part.__set_end_keys({sink_test_utils::make_int_literal(20)}); + res->__set_partitions({new_part}); + + doris::TTabletLocation new_location; + new_location.__set_tablet_id(300); + new_location.__set_node_ids({1}); + res->__set_tablets({new_location}); + }; + doris::DebugPoints::instance()->add_with_callback( + "VRowDistribution.automatic_create_partition.inject_result", handler); + + st = h->row_distribution.automatic_create_partition(); + EXPECT_TRUE(st.ok()) << st.to_string(); + EXPECT_TRUE(injected); + + auto check_block = ColumnHelper::create_block({15}); + std::vector parts(1, nullptr); + h->vpartition->find_partition(&check_block, 0, parts[0]); + ASSERT_NE(parts[0], nullptr); + EXPECT_EQ(parts[0]->id, 3); + + h->row_distribution.clear_batching_stats(); + EXPECT_FALSE(h->row_distribution.need_deal_batching()); + + doris::DebugPoints::instance()->clear(); + doris::config::enable_debug_points = false; +} + +TEST(VRowDistributionTest, ReplaceOverwritingPartitionInjectedRequestDedupAndReplace) { + OperatorContext ctx; + constexpr int64_t txn_id = 1; + + TOlapTableSchemaParam tschema; + TTupleId tablet_sink_tuple_id = 0; + int64_t schema_index_id = 0; + sink_test_utils::build_desc_tbl_and_schema(ctx, tschema, tablet_sink_tuple_id, schema_index_id, + false); + + auto tpartition = sink_test_utils::build_partition_param(schema_index_id); + tpartition.__set_enable_auto_detect_overwrite(true); + tpartition.__set_overwrite_group_id(123); + auto tlocation = sink_test_utils::build_location_param(); + + auto h = _build_vrow_distribution_harness(ctx, tschema, tpartition, tlocation, + tablet_sink_tuple_id, txn_id); + + doris::config::enable_debug_points = true; + doris::DebugPoints::instance()->clear(); + + int injected_times = 0; + std::function handler = + [&](doris::TReplacePartitionRequest* req, doris::TReplacePartitionResult* res) { + injected_times++; + ASSERT_TRUE(req->__isset.partition_ids); + ASSERT_EQ(req->partition_ids.size(), 2); + EXPECT_EQ(req->partition_ids[0], 1); + EXPECT_EQ(req->partition_ids[1], 2); + ASSERT_TRUE(req->__isset.overwrite_group_id); + EXPECT_EQ(req->overwrite_group_id, 123); + + doris::TStatus tstatus; + tstatus.__set_status_code(doris::TStatusCode::OK); + res->__set_status(tstatus); + + doris::TOlapTablePartition new_p1; + new_p1.id = 11; + new_p1.num_buckets = 1; + new_p1.__set_is_mutable(true); + { + doris::TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {1100}; + new_p1.indexes = {index_tablets}; + } + + doris::TOlapTablePartition new_p2; + new_p2.id = 12; + new_p2.num_buckets = 1; + new_p2.__set_is_mutable(true); + { + doris::TOlapTableIndexTablets index_tablets; + index_tablets.index_id = schema_index_id; + index_tablets.tablets = {1200}; + new_p2.indexes = {index_tablets}; + } + + res->__set_partitions({new_p1, new_p2}); + + doris::TTabletLocation loc1; + loc1.__set_tablet_id(1100); + loc1.__set_node_ids({1}); + doris::TTabletLocation loc2; + loc2.__set_tablet_id(1200); + loc2.__set_node_ids({1}); + res->__set_tablets({loc1, loc2}); + }; + doris::DebugPoints::instance()->add_with_callback( + "VRowDistribution.replace_overwriting_partition.inject_result", handler); + + Status st; + { + auto input_block = ColumnHelper::create_block({1, 25}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + EXPECT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(injected_times, 1); + + ASSERT_EQ(row_part_tablet_ids.size(), 1); + ASSERT_EQ(row_part_tablet_ids[0].partition_ids.size(), 2); + EXPECT_EQ(row_part_tablet_ids[0].partition_ids[0], 11); + EXPECT_EQ(row_part_tablet_ids[0].partition_ids[1], 12); + ASSERT_EQ(row_part_tablet_ids[0].tablet_ids.size(), 2); + EXPECT_EQ(row_part_tablet_ids[0].tablet_ids[0], 1100); + EXPECT_EQ(row_part_tablet_ids[0].tablet_ids[1], 1200); + } + + { + auto input_block = ColumnHelper::create_block({1}); + std::shared_ptr converted_block; + std::vector row_part_tablet_ids; + int64_t rows_stat_val = input_block.rows(); + st = h->row_distribution.generate_rows_distribution(input_block, converted_block, + row_part_tablet_ids, rows_stat_val); + EXPECT_TRUE(st.ok()) << st.to_string(); + EXPECT_EQ(injected_times, 1); + } + + doris::DebugPoints::instance()->clear(); + doris::config::enable_debug_points = false; +} + +} // namespace + +} // namespace doris::vectorized diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index 897dc22283315d..d5fddd65c387e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -436,7 +436,6 @@ public long getMaxVersion() { return maxVersion; } - // for non-cloud mode. for cloud mode see `updateScanRangeVersions` private void addScanRangeLocations(Partition partition, List tablets, Map> backendAlivePathHashs) throws UserException { long visibleVersion = Partition.PARTITION_INIT_VERSION; @@ -446,6 +445,7 @@ private void addScanRangeLocations(Partition partition, if (!(Config.isCloudMode() && Config.enable_cloud_snapshot_version)) { visibleVersion = partition.getVisibleVersion(); } + // for non-cloud mode. for cloud mode see `updateScanRangeVersions` maxVersion = Math.max(maxVersion, visibleVersion); int useFixReplica = -1; diff --git a/regression-test/data/nereids_p0/insert_into_table/random.out b/regression-test/data/nereids_p0/insert_into_table/nereids_insert_random.out similarity index 100% rename from regression-test/data/nereids_p0/insert_into_table/random.out rename to regression-test/data/nereids_p0/insert_into_table/nereids_insert_random.out diff --git a/regression-test/suites/insert_p0/insert.groovy b/regression-test/suites/insert_p0/insert.groovy index 84696941da5e2a..78a648dd1544af 100644 --- a/regression-test/suites/insert_p0/insert.groovy +++ b/regression-test/suites/insert_p0/insert.groovy @@ -112,6 +112,7 @@ suite("insert") { b as (select * from a) select id from a; """ + sql """ DROP TABLE IF EXISTS source; DROP TABLE IF EXISTS dest; @@ -126,7 +127,6 @@ suite("insert") { PROPERTIES ( "replication_num" = "1" ); - CREATE TABLE dest ( l_shipdate DATE NOT NULL, l_orderkey bigint NOT NULL, @@ -140,15 +140,13 @@ suite("insert") { "replication_num" = "1" ); insert into source values('1994-12-08', 1,1) , ('1994-12-14',1,1), ('1994-12-14',2,1); - - insert into dest select * from source where l_shipdate = '1994-12-08'; insert into dest select * from source where l_shipdate = '1994-12-14'; """ def rows1 = sql """select count() from source;""" def rows2 = sql """select count() from dest;""" - assertTrue(rows1 == rows2); + assertEquals(rows1, rows2); test { sql("insert into dest values(now(), 0xff, 0xaa)") diff --git a/regression-test/suites/nereids_p0/insert_into_table/random.groovy b/regression-test/suites/nereids_p0/insert_into_table/nereids_insert_random.groovy similarity index 100% rename from regression-test/suites/nereids_p0/insert_into_table/random.groovy rename to regression-test/suites/nereids_p0/insert_into_table/nereids_insert_random.groovy diff --git a/regression-test/suites/query_p0/load.groovy b/regression-test/suites/query_p0/load.groovy index f6b2fa33507534..e389abb6129132 100644 --- a/regression-test/suites/query_p0/load.groovy +++ b/regression-test/suites/query_p0/load.groovy @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. -suite("load_test_query_db") { +suite("load") { // init query case data - def dbName = "test_query_db" - sql "DROP DATABASE IF EXISTS ${dbName}" - sql "CREATE DATABASE ${dbName}" - sql "USE $dbName" + sql "DROP DATABASE IF EXISTS test_query_db" + sql "CREATE DATABASE test_query_db" + sql "USE test_query_db" sql """ CREATE TABLE IF NOT EXISTS `baseall` ( `k0` boolean null comment "", @@ -80,12 +79,12 @@ suite("load_test_query_db") { """ streamLoad { table "baseall" - db dbName + db "test_query_db" set 'column_separator', ',' file "baseall.txt" } - sql "insert into ${dbName}.test select * from ${dbName}.baseall where k1 <= 3" - sql "insert into ${dbName}.bigtable select * from ${dbName}.baseall" + sql "insert into test_query_db.test select * from test_query_db.baseall where k1 <= 3" + sql "insert into test_query_db.bigtable select * from test_query_db.baseall" // table for compaction sql """ @@ -109,9 +108,9 @@ suite("load_test_query_db") { sql """insert into compaction_tbl values(123,"1999-10-10",'aaa',123,123,"1970-01-01 00:00:00","1970-01-01 00:00:00","1970-01-01 00:00:00",123,123,123,hll_hash(""),bitmap_from_string(""));""" - def baseall_count = sql "select count(*) from ${dbName}.baseall" + def baseall_count = sql "select count(*) from test_query_db.baseall" assertEquals(16, baseall_count[0][0]) - def test_count = sql "select count(*) from ${dbName}.test" + def test_count = sql "select count(*) from test_query_db.test" assertEquals(3, test_count[0][0]) }