diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index b48b9f780b8754e..c8aed6220c83e21 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -27,6 +27,7 @@ #include #include "common/logging.h" +#include "common/status.h" #include "pipeline/exec/exchange_sink_operator.h" #include "pipeline/exec/exchange_source_operator.h" #include "runtime/memory/mem_tracker.h" @@ -95,7 +96,9 @@ Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block } DCHECK(!_block_queue.empty()); - auto [next_block, block_byte_size] = std::move(_block_queue.front()); + BlockUPtr next_block; + RETURN_IF_ERROR(_block_queue.front().get_block(next_block)); + size_t block_byte_size = _block_queue.front().block_byte_size(); _block_queue.pop_front(); _recvr->_parent->memory_used_counter()->update(-(int64_t)block_byte_size); sub_blocks_memory_usage(block_byte_size); @@ -163,30 +166,12 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num } } - BlockUPtr block = nullptr; - int64_t deserialize_time = 0; - { - SCOPED_RAW_TIMER(&deserialize_time); - block = Block::create_unique(); - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(block->deserialize(pblock)); - } - - const auto rows = block->rows(); - if (rows == 0) { - return Status::OK(); - } - auto block_byte_size = block->allocated_bytes(); - VLOG_ROW << "added #rows=" << rows << " batch_size=" << block_byte_size << "\n"; - std::lock_guard l(_lock); if (_is_cancelled) { return Status::OK(); } - COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, deserialize_time); - COUNTER_UPDATE(_recvr->_decompress_timer, block->get_decompress_time()); - COUNTER_UPDATE(_recvr->_decompress_bytes, block->get_decompressed_bytes()); - COUNTER_UPDATE(_recvr->_rows_produced_counter, rows); + const auto block_byte_size = pblock.ByteSizeLong(); COUNTER_UPDATE(_recvr->_blocks_produced_counter, 1); if (_recvr->_max_wait_worker_time->value() < wait_for_worker) { _recvr->_max_wait_worker_time->set(wait_for_worker); @@ -196,7 +181,7 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num _recvr->_max_find_recvr_time->set((int64_t)time_to_find_recvr); } - _block_queue.emplace_back(std::move(block), block_byte_size); + _block_queue.emplace_back(pblock, block_byte_size); COUNTER_UPDATE(_recvr->_remote_bytes_received_counter, block_byte_size); _record_debug_info(); try_set_dep_ready_without_lock(); diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 08fb004f3b1a8ea..260d9f75d531534 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include @@ -260,7 +261,36 @@ class VDataStreamRecvr::SenderQueue { Status _cancel_status; int _num_remaining_senders; std::unique_ptr _queue_mem_tracker; - std::list> _block_queue; + + struct BlockItem { + Status get_block(BlockUPtr& block) { + if (!_block) { + SCOPED_RAW_TIMER(&_deserialize_time); + _block = Block::create_unique(); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_block->deserialize(_pblock)); + } + block.swap(_block); + _block.reset(); + return Status::OK(); + } + + size_t block_byte_size() const { return _block_byte_size; } + int64_t deserialize_time() const { return _deserialize_time; } + + BlockItem(BlockUPtr&& block, size_t block_byte_size) + : _block(std::move(block)), _block_byte_size(block_byte_size) {} + + BlockItem(const PBlock& pblock, size_t block_byte_size) + : _block(nullptr), _pblock(pblock), _block_byte_size(block_byte_size) {} + + private: + BlockUPtr _block; + PBlock _pblock; + const size_t _block_byte_size; + int64_t _deserialize_time = 0; + }; + + std::list _block_queue; // sender_id std::unordered_set _sender_eos_set;