Skip to content

Commit 799f68c

Browse files
[BugFix] Revert 'Avoid brpc communication when using local pass through' (#61709)
1 parent c09ac51 commit 799f68c

21 files changed

+458
-297
lines changed

be/src/exec/pipeline/exchange/exchange_sink_operator.cpp

Lines changed: 27 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
#include "runtime/data_stream_mgr.h"
3030
#include "runtime/descriptors.h"
3131
#include "runtime/exec_env.h"
32+
#include "runtime/local_pass_through_buffer.h"
3233
#include "runtime/runtime_state.h"
3334
#include "serde/compress_strategy.h"
3435
#include "serde/protobuf_serde.h"
@@ -46,13 +47,15 @@ class ExchangeSinkOperator::Channel {
4647
// how much tuple data is getting accumulated before being sent; it only applies
4748
// when data is added via add_row() and not sent directly via send_batch().
4849
Channel(ExchangeSinkOperator* parent, const TNetworkAddress& brpc_dest, const TUniqueId& fragment_instance_id,
49-
PlanNodeId dest_node_id, int32_t num_shuffles, bool enable_exchange_pass_through, bool enable_exchange_perf)
50+
PlanNodeId dest_node_id, int32_t num_shuffles, bool enable_exchange_pass_through, bool enable_exchange_perf,
51+
PassThroughChunkBuffer* pass_through_chunk_buffer)
5052
: _parent(parent),
5153
_brpc_dest_addr(brpc_dest),
5254
_fragment_instance_id(fragment_instance_id),
5355
_dest_node_id(dest_node_id),
5456
_enable_exchange_pass_through(enable_exchange_pass_through),
5557
_enable_exchange_perf(enable_exchange_perf),
58+
_pass_through_context(pass_through_chunk_buffer, fragment_instance_id, dest_node_id),
5659
_chunks(num_shuffles) {}
5760

5861
// Initialize channel.
@@ -114,6 +117,7 @@ class ExchangeSinkOperator::Channel {
114117
// enable it to profile exchange's performance, which ignores computing local data for exchange_speed/_bytes,
115118
// because local data isn't accessed by remote network.
116119
const bool _enable_exchange_perf;
120+
PassThroughContext _pass_through_context;
117121

118122
bool _is_first_chunk = true;
119123
std::shared_ptr<PInternalService_RecoverableStub> _brpc_stub = nullptr;
@@ -123,8 +127,6 @@ class ExchangeSinkOperator::Channel {
123127
// If pipeline level shuffle is disable, the size of _chunks
124128
// always be 1
125129
std::vector<std::unique_ptr<Chunk>> _chunks;
126-
ChunkPassThroughVectorPtr _pass_through_chunks;
127-
int64_t _pass_through_physical_bytes = 0;
128130
PTransmitChunkParamsPtr _chunk_request;
129131
size_t _current_request_bytes = 0;
130132

@@ -152,6 +154,7 @@ bool ExchangeSinkOperator::Channel::_check_use_pass_through() {
152154
}
153155

154156
void ExchangeSinkOperator::Channel::_prepare_pass_through() {
157+
_pass_through_context.init();
155158
_use_pass_through = _check_use_pass_through();
156159
}
157160

@@ -226,8 +229,6 @@ Status ExchangeSinkOperator::Channel::send_one_chunk(RuntimeState* state, const
226229
_chunk_request->set_node_id(_dest_node_id);
227230
_chunk_request->set_sender_id(_parent->_sender_id);
228231
_chunk_request->set_be_number(_parent->_be_number);
229-
_pass_through_chunks = std::make_unique<ChunkPassThroughVector>();
230-
_pass_through_physical_bytes = 0;
231232
if (_parent->_is_pipeline_level_shuffle) {
232233
_chunk_request->set_is_pipeline_level_shuffle(true);
233234
}
@@ -236,19 +237,18 @@ Status ExchangeSinkOperator::Channel::send_one_chunk(RuntimeState* state, const
236237
// If chunk is not null, append it to request
237238
if (chunk != nullptr) {
238239
if (_use_pass_through) {
239-
int64_t before_bytes = CurrentThread::current().get_consumed_bytes();
240-
auto clone = chunk->clone_unique();
241-
int64_t physical_bytes = CurrentThread::current().get_consumed_bytes() - before_bytes;
242-
_pass_through_physical_bytes += physical_bytes;
243240
size_t chunk_size = serde::ProtobufChunkSerde::max_serialized_size(*chunk);
244-
_pass_through_chunks->emplace_back(std::move(clone), driver_sequence, chunk_size, physical_bytes);
245-
COUNTER_UPDATE(_parent->_bytes_pass_through_counter, chunk_size);
241+
// -1 means disable pipeline level shuffle
242+
TRY_CATCH_BAD_ALLOC(
243+
_pass_through_context.append_chunk(_parent->_sender_id, chunk, chunk_size,
244+
_parent->_is_pipeline_level_shuffle ? driver_sequence : -1));
246245
_current_request_bytes += chunk_size;
246+
COUNTER_UPDATE(_parent->_bytes_pass_through_counter, chunk_size);
247+
COUNTER_SET(_parent->_pass_through_buffer_peak_mem_usage, _pass_through_context.total_bytes());
247248
} else {
248249
if (_parent->_is_pipeline_level_shuffle) {
249250
_chunk_request->add_driver_sequences(driver_sequence);
250251
}
251-
252252
auto pchunk = _chunk_request->add_chunks();
253253
TRY_CATCH_BAD_ALLOC(RETURN_IF_ERROR(_parent->serialize_chunk(chunk, pchunk, &_is_first_chunk)));
254254
_current_request_bytes += pchunk->data().size();
@@ -261,21 +261,12 @@ Status ExchangeSinkOperator::Channel::send_one_chunk(RuntimeState* state, const
261261
_chunk_request->set_eos(eos);
262262
_chunk_request->set_use_pass_through(_use_pass_through);
263263
butil::IOBuf attachment;
264-
int64_t physical_bytes = _use_pass_through ? _pass_through_physical_bytes
265-
: _parent->construct_brpc_attachment(_chunk_request, attachment);
266-
TransmitChunkInfo info = {this->_fragment_instance_id,
267-
_brpc_stub,
268-
std::move(_chunk_request),
269-
std::move(_pass_through_chunks),
270-
state->exec_env()->stream_mgr(),
271-
attachment,
272-
physical_bytes,
273-
_brpc_dest_addr};
264+
int64_t attachment_physical_bytes = _parent->construct_brpc_attachment(_chunk_request, attachment);
265+
TransmitChunkInfo info = {this->_fragment_instance_id, _brpc_stub, std::move(_chunk_request), attachment,
266+
attachment_physical_bytes, _brpc_dest_addr};
274267
RETURN_IF_ERROR(_parent->_buffer->add_request(info));
275268
_current_request_bytes = 0;
276269
_chunk_request.reset();
277-
_pass_through_chunks = std::make_unique<ChunkPassThroughVector>();
278-
_pass_through_physical_bytes = 0;
279270
*is_real_sent = true;
280271
}
281272

@@ -293,8 +284,8 @@ Status ExchangeSinkOperator::Channel::send_chunk_request(RuntimeState* state, PT
293284
chunk_request->set_be_number(_parent->_be_number);
294285
chunk_request->set_eos(false);
295286
chunk_request->set_use_pass_through(_use_pass_through);
296-
TransmitChunkInfo info = {this->_fragment_instance_id, _brpc_stub, std::move(chunk_request), nullptr,
297-
state->exec_env()->stream_mgr(), attachment, attachment_physical_bytes, _brpc_dest_addr};
287+
TransmitChunkInfo info = {this->_fragment_instance_id, _brpc_stub, std::move(chunk_request), attachment,
288+
attachment_physical_bytes, _brpc_dest_addr};
298289
RETURN_IF_ERROR(_parent->_buffer->add_request(info));
299290

300291
return Status::OK();
@@ -351,6 +342,10 @@ ExchangeSinkOperator::ExchangeSinkOperator(
351342
_output_columns(output_columns),
352343
_num_sinkers(num_sinkers) {
353344
std::map<int64_t, int64_t> fragment_id_to_channel_index;
345+
RuntimeState* state = fragment_ctx->runtime_state();
346+
347+
PassThroughChunkBuffer* pass_through_chunk_buffer =
348+
state->exec_env()->stream_mgr()->get_pass_through_chunk_buffer(state->query_id());
354349

355350
_channels.reserve(destinations.size());
356351
std::vector<int> driver_sequence_per_channel(destinations.size(), 0);
@@ -364,7 +359,7 @@ ExchangeSinkOperator::ExchangeSinkOperator(
364359
} else {
365360
std::unique_ptr<Channel> channel = std::make_unique<Channel>(
366361
this, destination.brpc_server, fragment_instance_id, dest_node_id, _num_shuffles_per_channel,
367-
enable_exchange_pass_through, enable_exchange_perf);
362+
enable_exchange_pass_through, enable_exchange_perf, pass_through_chunk_buffer);
368363
_channels.emplace_back(channel.get());
369364
_instance_id2channel.emplace(fragment_instance_id.lo, std::move(channel));
370365
}
@@ -460,6 +455,9 @@ Status ExchangeSinkOperator::prepare(RuntimeState* state) {
460455
_shuffle_chunk_append_counter = ADD_COUNTER(_unique_metrics, "ShuffleChunkAppendCounter", TUnit::UNIT);
461456
_shuffle_chunk_append_timer = ADD_TIMER(_unique_metrics, "ShuffleChunkAppendTime");
462457
_compress_timer = ADD_TIMER(_unique_metrics, "CompressTime");
458+
_pass_through_buffer_peak_mem_usage = _unique_metrics->AddHighWaterMarkCounter(
459+
"PassThroughBufferPeakMemoryUsage", TUnit::BYTES,
460+
RuntimeProfile::Counter::create_strategy(TUnit::BYTES, TCounterMergeType::SKIP_FIRST_MERGE));
463461

464462
for (auto& [_, channel] : _instance_id2channel) {
465463
RETURN_IF_ERROR(channel->init(state));
@@ -651,10 +649,8 @@ Status ExchangeSinkOperator::set_finishing(RuntimeState* state) {
651649
butil::IOBuf attachment;
652650
const int64_t attachment_physical_bytes = construct_brpc_attachment(_chunk_request, attachment);
653651
for (const auto& [_, channel] : _instance_id2channel) {
654-
if (!channel->use_pass_through()) {
655-
PTransmitChunkParamsPtr copy = std::make_shared<PTransmitChunkParams>(*_chunk_request);
656-
RETURN_IF_ERROR(channel->send_chunk_request(state, copy, attachment, attachment_physical_bytes));
657-
}
652+
PTransmitChunkParamsPtr copy = std::make_shared<PTransmitChunkParams>(*_chunk_request);
653+
RETURN_IF_ERROR(channel->send_chunk_request(state, copy, attachment, attachment_physical_bytes));
658654
}
659655
_current_request_bytes = 0;
660656
_chunk_request.reset();

be/src/exec/pipeline/exchange/exchange_sink_operator.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -186,6 +186,7 @@ class ExchangeSinkOperator final : public Operator {
186186
RuntimeProfile::Counter* _raw_input_bytes_counter = nullptr;
187187
RuntimeProfile::Counter* _serialized_bytes_counter = nullptr;
188188
RuntimeProfile::Counter* _compressed_bytes_counter = nullptr;
189+
RuntimeProfile::HighWaterMarkCounter* _pass_through_buffer_peak_mem_usage = nullptr;
189190

190191
std::atomic<bool> _is_finished = false;
191192
std::atomic<bool> _is_cancelled = false;

be/src/exec/pipeline/exchange/sink_buffer.cpp

Lines changed: 9 additions & 121 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,9 @@
2020
#include <memory>
2121
#include <mutex>
2222
#include <string_view>
23-
#include <thread>
2423

2524
#include "exec/pipeline/schedule/utils.h"
2625
#include "fmt/core.h"
27-
#include "runtime/data_stream_mgr.h"
2826
#include "util/defer_op.h"
2927
#include "util/time.h"
3028
#include "util/uid_util.h"
@@ -56,7 +54,6 @@ SinkBuffer::SinkBuffer(FragmentContext* fragment_ctx, const std::vector<TPlanFra
5654
ctx.num_finished_rpcs = 0;
5755
ctx.num_in_flight_rpcs = 0;
5856
ctx.dest_addrs = dest.brpc_server;
59-
ctx.pass_through_blocked = false;
6057

6158
PUniqueId finst_id;
6259
finst_id.set_hi(instance_id.hi);
@@ -112,16 +109,7 @@ Status SinkBuffer::add_request(TransmitChunkInfo& request) {
112109
auto& instance_id = request.fragment_instance_id;
113110
auto& context = sink_ctx(instance_id.lo);
114111

115-
if (request.params->has_use_pass_through() && request.params->use_pass_through()) {
116-
RETURN_IF_ERROR(_try_to_send_local(instance_id, [&, tracker = CurrentThread::current().mem_tracker()]() {
117-
// Release allocated bytes in current MemTracker, since it would not be released at current MemTracker
118-
tracker->release(request.physical_bytes);
119-
GlobalEnv::GetInstance()->passthrough_mem_tracker()->consume(request.physical_bytes);
120-
context.buffer.push(std::move(request));
121-
}));
122-
} else {
123-
RETURN_IF_ERROR(_try_to_send_rpc(instance_id, [&]() { context.buffer.push(std::move(request)); }));
124-
}
112+
RETURN_IF_ERROR(_try_to_send_rpc(instance_id, [&]() { context.buffer.push(request); }));
125113
}
126114

127115
return Status::OK();
@@ -163,7 +151,7 @@ bool SinkBuffer::is_finished() const {
163151
return false;
164152
}
165153

166-
return _num_sending == 0 && _total_in_flight_rpc == 0;
154+
return _num_sending_rpc == 0 && _total_in_flight_rpc == 0;
167155
}
168156

169157
void SinkBuffer::update_profile(RuntimeProfile* profile) {
@@ -233,9 +221,9 @@ void SinkBuffer::cancel_one_sinker(RuntimeState* const state) {
233221
// check how many cancel operations are issued, and show the state of that time.
234222
VLOG_OPERATOR << fmt::format(
235223
"fragment_instance_id {}, _num_uncancelled_sinkers {}, _is_finishing {}, _num_remaining_eos {}, "
236-
"_num_sending {}, chunk is full {}",
224+
"_num_sending_rpc {}, chunk is full {}",
237225
print_id(_fragment_ctx->fragment_instance_id()), _num_uncancelled_sinkers, _is_finishing,
238-
_num_remaining_eos, _num_sending, is_full());
226+
_num_remaining_eos, _num_sending_rpc, is_full());
239227
}
240228
}
241229

@@ -268,114 +256,13 @@ void SinkBuffer::_process_send_window(const TUniqueId& instance_id, const int64_
268256
}
269257
}
270258

271-
Status SinkBuffer::_try_to_send_local(const TUniqueId& instance_id, const std::function<void()>& pre_works) {
272-
// switch to process tracker
273-
SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(nullptr);
274-
auto& context = sink_ctx(instance_id.lo);
275-
auto notify = this->finishing_defer();
276-
std::lock_guard guard(context.mutex);
277-
DeferOp decrease_defer([this]() { --_num_sending; });
278-
++_num_sending;
279-
280-
pre_works();
281-
282-
for (;;) {
283-
DeferOp reset_owner_id([&context]() { context.owner_id = {}; });
284-
285-
if (_is_finishing) {
286-
return Status::OK();
287-
}
288-
289-
auto& buffer = context.buffer;
290-
if (buffer.empty() || context.pass_through_blocked) {
291-
return Status::OK();
292-
}
293-
294-
TransmitChunkInfo& request = buffer.front();
295-
DeferOp pop_defer([&buffer, mem_tracker = _mem_tracker]() {
296-
// The request memory is acquired by ExchangeSinkOperator,
297-
// so use the instance_mem_tracker passed from ExchangeSinkOperator to release memory.
298-
// This must be invoked before decrease_defer desctructed to avoid sink_buffer and fragment_ctx released.
299-
SCOPED_THREAD_LOCAL_MEM_TRACKER_SETTER(mem_tracker);
300-
buffer.pop();
301-
});
302-
303-
if (request.params->eos()) {
304-
DeferOp eos_defer([this, &instance_id]() {
305-
if (--_num_remaining_eos == 0) {
306-
_is_finishing = true;
307-
}
308-
sink_ctx(instance_id.lo).num_sinker--;
309-
});
310-
// Only the last eos is sent to ExchangeSourceOperator.
311-
if (context.num_sinker > 1) {
312-
if (request.pass_through_chunks->size() == 0) {
313-
continue;
314-
}
315-
request.params->set_eos(false);
316-
} else {
317-
// this is the last eos query, set query stats
318-
if (auto final_stats = _fragment_ctx->runtime_state()->query_ctx()->intermediate_query_statistic(
319-
_delta_bytes_sent.exchange(0))) {
320-
final_stats->to_pb(request.params->mutable_query_statistics());
321-
}
322-
}
323-
}
324-
325-
auto query_ctx = std::weak_ptr(_fragment_ctx->runtime_state()->query_ctx()->shared_from_this());
326-
context.owner_id = std::this_thread::get_id();
327-
auto* closure = new DisposablePassThroughClosure([query_ctx, this, instance_id]() noexcept {
328-
auto guard = query_ctx.lock();
329-
RETURN_IF(guard == nullptr, (void)0);
330-
auto& context = sink_ctx(instance_id.lo);
331-
332-
// Avoid local-passthrough recursive calls.
333-
if (context.owner_id == std::this_thread::get_id()) {
334-
--_total_in_flight_rpc;
335-
context.pass_through_blocked = false;
336-
return;
337-
}
338-
339-
auto notify = this->defer_notify();
340-
auto defer = DeferOp([this]() { --_total_in_flight_rpc; });
341-
context.pass_through_blocked = false;
342-
static_cast<void>(_try_to_send_local(instance_id, []() {}));
343-
});
344-
345-
context.pass_through_blocked = true;
346-
if (_first_send_time == -1) {
347-
_first_send_time = MonotonicNanos();
348-
}
349-
350-
// Decrease memory from pass through before moving chunks to the reciever fragment.
351-
GlobalEnv::GetInstance()->passthrough_mem_tracker()->release(request.physical_bytes);
352-
353-
DCHECK_EQ(request.params->use_pass_through(), true);
354-
::google::protobuf::Closure* done = closure;
355-
356-
_total_in_flight_rpc++;
357-
auto defer = CancelableDefer([this]() { --_total_in_flight_rpc; });
358-
Status st = request.stream_mgr->transmit_chunk(instance_id, *request.params,
359-
std::move(request.pass_through_chunks), &done);
360-
if (st.ok()) defer.cancel();
361-
if (st.ok() && done != nullptr) {
362-
// if the closure was not removed delete it and continue transmitting chunks.
363-
done->Run();
364-
continue;
365-
}
366-
return st;
367-
}
368-
return Status::OK();
369-
}
370-
371259
Status SinkBuffer::_try_to_send_rpc(const TUniqueId& instance_id, const std::function<void()>& pre_works) {
372260
auto& context = sink_ctx(instance_id.lo);
373-
auto notify = this->finishing_defer();
374261
std::lock_guard guard(context.mutex);
375262
pre_works();
376263

377-
DeferOp decrease_defer([this]() { --_num_sending; });
378-
++_num_sending;
264+
DeferOp decrease_defer([this]() { --_num_sending_rpc; });
265+
++_num_sending_rpc;
379266

380267
for (;;) {
381268
if (_is_finishing) {
@@ -426,6 +313,7 @@ Status SinkBuffer::_try_to_send_rpc(const TUniqueId& instance_id, const std::fun
426313
return;
427314
}
428315
if (--_num_remaining_eos == 0) {
316+
auto notify = this->defer_notify();
429317
_is_finishing = true;
430318
}
431319
sink_ctx(instance_id.lo).num_sinker--;
@@ -527,8 +415,8 @@ Status SinkBuffer::_try_to_send_rpc(const TUniqueId& instance_id, const std::fun
527415

528416
// Attachment will be released by process_mem_tracker in closure->Run() in bthread, when receiving the response,
529417
// so decrease the memory usage of attachment from instance_mem_tracker immediately before sending the request.
530-
_mem_tracker->release(request.physical_bytes);
531-
GlobalEnv::GetInstance()->process_mem_tracker()->consume(request.physical_bytes);
418+
_mem_tracker->release(request.attachment_physical_bytes);
419+
GlobalEnv::GetInstance()->process_mem_tracker()->consume(request.attachment_physical_bytes);
532420

533421
closure->cntl.Reset();
534422
closure->cntl.set_timeout_ms(_brpc_timeout_ms);

0 commit comments

Comments
 (0)