[Bug](pipeline) Fix bug in non-local exchange on pipeline engine (#16463)

Currently, for broadcast shuffle, we serialize a block once and then send it by RPC through multiple channel. After this, we will serialize next block in the same memory for consideration of memory reuse. However, since the RPC is asynchronized, maybe the next block serialization will happen before sending the previous block.

So, in this PR, I use a ref count to identify if the serialized block can be reuse in broadcast shuffle.
This commit is contained in:
Gabriel
2023-02-09 19:22:40 +08:00
committed by GitHub
parent 539fd684e9
commit a038fdaec6
5 changed files with 207 additions and 49 deletions

View File

@ -32,7 +32,8 @@ namespace doris::pipeline {
template <typename T>
class SelfDeleteClosure : public google::protobuf::Closure {
public:
SelfDeleteClosure(InstanceLoId id, bool eos) : _id(id), _eos(eos) {}
SelfDeleteClosure(InstanceLoId id, bool eos, vectorized::BroadcastPBlockHolder* data = nullptr)
: _id(id), _eos(eos), _data(data) {}
~SelfDeleteClosure() override = default;
SelfDeleteClosure(const SelfDeleteClosure& other) = delete;
SelfDeleteClosure& operator=(const SelfDeleteClosure& other) = delete;
@ -56,6 +57,9 @@ public:
} else {
_suc_fn(_id, _eos, result);
}
if (_data) {
_data->unref();
}
} catch (const std::exception& exp) {
LOG(FATAL) << "brpc callback error: " << exp.what();
} catch (...) {
@ -71,6 +75,7 @@ private:
std::function<void(const InstanceLoId&, const bool&, const T&)> _suc_fn;
InstanceLoId _id;
bool _eos;
vectorized::BroadcastPBlockHolder* _data;
};
ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, int send_id,
@ -126,6 +131,8 @@ void ExchangeSinkBuffer::register_sink(TUniqueId fragment_instance_id) {
_instance_to_package_queue_mutex[low_id] = std::make_unique<std::mutex>();
_instance_to_seq[low_id] = 0;
_instance_to_package_queue[low_id] = std::queue<TransmitInfo, std::list<TransmitInfo>>();
_instance_to_broadcast_package_queue[low_id] =
std::queue<BroadcastTransmitInfo, std::list<BroadcastTransmitInfo>>();
PUniqueId finst_id;
finst_id.set_hi(fragment_instance_id.hi);
finst_id.set_lo(fragment_instance_id.lo);
@ -155,59 +162,93 @@ Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) {
return Status::OK();
}
Status ExchangeSinkBuffer::add_block(BroadcastTransmitInfo&& request) {
if (_is_finishing) {
return Status::OK();
}
TUniqueId ins_id = request.channel->_fragment_instance_id;
bool send_now = false;
request.block_holder->ref();
{
std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[ins_id.lo]);
// Do not have in process rpc, directly send
if (_instance_to_sending_by_pipeline[ins_id.lo]) {
send_now = true;
_instance_to_sending_by_pipeline[ins_id.lo] = false;
}
_instance_to_broadcast_package_queue[ins_id.lo].emplace(std::move(request));
}
if (send_now) {
RETURN_IF_ERROR(_send_rpc(ins_id.lo));
}
return Status::OK();
}
Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) {
std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[id]);
std::queue<TransmitInfo, std::list<TransmitInfo>>& q = _instance_to_package_queue[id];
if (q.empty() || _is_finishing) {
std::queue<BroadcastTransmitInfo, std::list<BroadcastTransmitInfo>>& broadcast_q =
_instance_to_broadcast_package_queue[id];
if (_is_finishing) {
_instance_to_sending_by_pipeline[id] = true;
return Status::OK();
}
TransmitInfo& request = q.front();
#define DO_RPC(QUEUE, BLOCK, HOLDER) \
auto& request = QUEUE.front(); \
if (!_instance_to_request[id]) { \
_construct_request(id); \
} \
auto brpc_request = _instance_to_request[id]; \
brpc_request->set_eos(request.eos); \
brpc_request->set_packet_seq(_instance_to_seq[id]++); \
if (request.BLOCK) { \
brpc_request->set_allocated_block(request.BLOCK); \
} \
auto* _closure = new SelfDeleteClosure<PTransmitDataResult>(id, request.eos, HOLDER); \
_closure->cntl.set_timeout_ms(request.channel->_brpc_timeout_ms); \
_closure->addFailedHandler( \
[&](const InstanceLoId& id, const std::string& err) { _failed(id, err); }); \
_closure->addSuccessHandler([&](const InstanceLoId& id, const bool& eos, \
const PTransmitDataResult& result) { \
Status s = Status(result.status()); \
if (!s.ok()) { \
_failed(id, \
fmt::format("exchange req success but status isn't ok: {}", s.to_string())); \
} else if (eos) { \
_ended(id); \
} else { \
_send_rpc(id); \
} \
}); \
{ \
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker()); \
if (enable_http_send_block(*brpc_request)) { \
RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), _closure, \
*brpc_request, request.channel->_brpc_dest_addr)); \
} else { \
transmit_block(*request.channel->_brpc_stub, _closure, *brpc_request); \
} \
} \
if (request.BLOCK) { \
brpc_request->release_block(); \
} \
QUEUE.pop();
if (!_instance_to_request[id]) {
_construct_request(id);
if (!q.empty()) {
// If we have data to shuffle which is not broadcasted
DO_RPC(q, block, nullptr)
} else if (!broadcast_q.empty()) {
// If we have data to shuffle which is broadcasted
DO_RPC(broadcast_q, block_holder->get_block(), request.block_holder)
} else {
_instance_to_sending_by_pipeline[id] = true;
return Status::OK();
}
auto brpc_request = _instance_to_request[id];
brpc_request->set_eos(request.eos);
brpc_request->set_packet_seq(_instance_to_seq[id]++);
if (request.block) {
brpc_request->set_allocated_block(request.block.get());
}
auto* _closure = new SelfDeleteClosure<PTransmitDataResult>(id, request.eos);
_closure->cntl.set_timeout_ms(request.channel->_brpc_timeout_ms);
_closure->addFailedHandler(
[&](const InstanceLoId& id, const std::string& err) { _failed(id, err); });
_closure->addSuccessHandler([&](const InstanceLoId& id, const bool& eos,
const PTransmitDataResult& result) {
Status s = Status(result.status());
if (!s.ok()) {
_failed(id, fmt::format("exchange req success but status isn't ok: {}", s.to_string()));
} else if (eos) {
_ended(id);
} else {
_send_rpc(id);
}
});
{
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
if (enable_http_send_block(*brpc_request)) {
RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), _closure,
*brpc_request, request.channel->_brpc_dest_addr));
} else {
transmit_block(*request.channel->_brpc_stub, _closure, *brpc_request);
}
}
if (request.block) {
brpc_request->release_block();
}
q.pop();
return Status::OK();
}