Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix](exchange)fix exchange sink buffer does not update total_queue_size when EOF. #47312

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 20 additions & 1 deletion be/src/pipeline/exec/exchange_sink_buffer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
#include "runtime/exec_env.h"
#include "runtime/thread_context.h"
#include "service/backend_options.h"
#include "util/defer_op.h"
#include "util/proto_util.h"
#include "util/time.h"
#include "vec/sink/vdata_stream_sender.h"
Expand Down Expand Up @@ -442,7 +443,8 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) {
// When the receiving side reaches eof, it means the receiver has finished early.
// The remaining data in the current rpc_channel does not need to be sent,
// and the rpc_channel should be turned off immediately.
_turn_off_channel(id, lock);
Defer turn_off([&]() { _turn_off_channel(id, lock); });

std::queue<BroadcastTransmitInfo, std::list<BroadcastTransmitInfo>>& broadcast_q =
_instance_to_broadcast_package_queue[id];
for (; !broadcast_q.empty(); broadcast_q.pop()) {
Expand All @@ -458,12 +460,19 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) {

std::queue<TransmitInfo, std::list<TransmitInfo>>& q = _instance_to_package_queue[id];
for (; !q.empty(); q.pop()) {
_total_queue_size--;
if (q.front().block) {
COUNTER_UPDATE(q.front().channel->_parent->memory_used_counter(),
-q.front().block->ByteSizeLong());
}
}

if (_total_queue_size <= _queue_capacity) {
for (auto& [_, dep] : _queue_deps) {
dep->set_ready();
}
}

{
std::queue<TransmitInfo, std::list<TransmitInfo>> empty;
swap(empty, q);
Expand Down Expand Up @@ -575,6 +584,16 @@ void ExchangeSinkBuffer::update_profile(RuntimeProfile* profile) {
}
}

std::string ExchangeSinkBuffer::debug_each_instance_queue_size() {
fmt::memory_buffer debug_string_buffer;
for (auto& [id, m] : _instance_to_package_queue_mutex) {
std::unique_lock<std::mutex> lock(*m);
fmt::format_to(debug_string_buffer, "Instance: {}, queue size: {}\n", id,
_instance_to_package_queue[id].size());
}
return fmt::to_string(debug_string_buffer);
}

} // namespace pipeline
#include "common/compile_check_end.h"
} // namespace doris
2 changes: 2 additions & 0 deletions be/src/pipeline/exec/exchange_sink_buffer.h
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,8 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx {
_queue_deps[sender_ins_id] = queue_dependency;
_parents[sender_ins_id] = local_state;
}

std::string debug_each_instance_queue_size();
#ifdef BE_TEST
public:
#else
Expand Down
13 changes: 7 additions & 6 deletions be/src/pipeline/exec/exchange_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -499,12 +499,13 @@ std::string ExchangeSinkLocalState::debug_string(int indentation_level) const {
fmt::memory_buffer debug_string_buffer;
fmt::format_to(debug_string_buffer, "{}", Base::debug_string(indentation_level));
if (_sink_buffer) {
fmt::format_to(debug_string_buffer,
", Sink Buffer: (_is_finishing = {}, blocks in queue: {}, queue capacity: "
"{}, queue dep: {}), _reach_limit: {}, working channels: {}",
_sink_buffer->_is_failed.load(), _sink_buffer->_total_queue_size,
_sink_buffer->_queue_capacity, (void*)_queue_dependency.get(),
_reach_limit.load(), _working_channels_count.load());
fmt::format_to(
debug_string_buffer,
", Sink Buffer: (_is_finishing = {}, blocks in queue: {}, queue capacity: "
"{}, queue dep: {}), _reach_limit: {}, working channels: {} , each queue size: {}",
_sink_buffer->_is_failed.load(), _sink_buffer->_total_queue_size,
_sink_buffer->_queue_capacity, (void*)_queue_dependency.get(), _reach_limit.load(),
_working_channels_count.load(), _sink_buffer->debug_each_instance_queue_size());
}
return fmt::to_string(debug_string_buffer);
}
Expand Down
37 changes: 37 additions & 0 deletions be/test/vec/exec/exchange_sink_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -193,4 +193,41 @@ TEST_F(ExchangeSInkTest, test_error_end) {
}
}

TEST_F(ExchangeSInkTest, test_queue_size) {
{
auto state = create_runtime_state();
auto buffer = create_buffer(state);

auto sink1 = create_sink(state, buffer);

EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK());

EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK());

EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK());
EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK());

std::cout << "queue size : " << buffer->_total_queue_size << "\n";

EXPECT_EQ(buffer->_total_queue_size, 6);

std::cout << "each queue size : \n" << buffer->debug_each_instance_queue_size() << "\n";

pop_block(dest_ins_id_2, PopState::eof);

std::cout << "queue size : " << buffer->_total_queue_size << "\n";

EXPECT_EQ(buffer->_total_queue_size, 4);

std::cout << "each queue size : \n" << buffer->debug_each_instance_queue_size() << "\n";

clear_all_done();
}
}

} // namespace doris::vectorized
Loading