[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

@ -242,7 +242,7 @@ Status Channel::close_internal() {
RETURN_IF_ERROR(send_current_block(true));
} else {
SCOPED_CONSUME_MEM_TRACKER(_parent->_mem_tracker.get());
RETURN_IF_ERROR(send_block(nullptr, true));
RETURN_IF_ERROR(send_block((PBlock*)nullptr, true));
}
// Don't wait for the last packet to finish, left it to close_wait.
return Status::OK();
@ -287,7 +287,6 @@ VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int
sink.output_partition.type == TPartitionType::RANDOM ||
sink.output_partition.type == TPartitionType::RANGE_PARTITIONED ||
sink.output_partition.type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED);
_cur_pb_block = &_pb_block1;
std::map<int64_t, int64_t> fragment_id_to_channel_index;
@ -317,6 +316,12 @@ VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int
}
}
_name = "VDataStreamSender";
if (state->enable_pipeline_exec()) {
_broadcast_pb_blocks.resize(config::num_broadcast_buffer);
_broadcast_pb_block_idx = 0;
} else {
_cur_pb_block = &_pb_block1;
}
}
VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc,
@ -470,6 +475,23 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block, bool eos) {
for (auto channel : _channels) {
RETURN_IF_ERROR(channel->send_local_block(block));
}
} else if (state->enable_pipeline_exec()) {
BroadcastPBlockHolder* block_holder = nullptr;
RETURN_IF_ERROR(_get_next_available_buffer(&block_holder));
{
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
RETURN_IF_ERROR(
serialize_block(block, block_holder->get_block(), _channels.size()));
}
for (auto channel : _channels) {
if (channel->is_local()) {
RETURN_IF_ERROR(channel->send_local_block(block));
} else {
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
RETURN_IF_ERROR(channel->send_block(block_holder, eos));
}
}
} else {
{
SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
@ -620,6 +642,28 @@ void VDataStreamSender::_roll_pb_block() {
_cur_pb_block = (_cur_pb_block == &_pb_block1 ? &_pb_block2 : &_pb_block1);
}
Status VDataStreamSender::_get_next_available_buffer(BroadcastPBlockHolder** holder) {
constexpr int MAX_LOOP = 1000;
size_t it = 0;
while (it < MAX_LOOP) {
if (_broadcast_pb_block_idx == _broadcast_pb_blocks.size()) {
_broadcast_pb_block_idx = 0;
}
for (; _broadcast_pb_block_idx < _broadcast_pb_blocks.size(); _broadcast_pb_block_idx++) {
if (_broadcast_pb_blocks[_broadcast_pb_block_idx].available()) {
_broadcast_pb_block_idx++;
*holder = &_broadcast_pb_blocks[_broadcast_pb_block_idx - 1];
return Status::OK();
}
}
it++;
}
return Status::InternalError(
"Exceed the max loop limit when acquire the next available buffer!");
}
void VDataStreamSender::registe_channels(pipeline::ExchangeSinkBuffer* buffer) {
for (auto channel : _channels) {
((PipChannel*)channel)->registe(buffer);

View File

@ -49,6 +49,42 @@ namespace vectorized {
class VExprContext;
class Channel;
template <typename T>
struct AtomicWrapper {
std::atomic<T> _value;
AtomicWrapper() : _value() {}
AtomicWrapper(const std::atomic<T>& a) : _value(a.load()) {}
AtomicWrapper(const AtomicWrapper& other) : _value(other._value.load()) {}
AtomicWrapper& operator=(const AtomicWrapper& other) { _value.store(other._a.load()); }
};
// We use BroadcastPBlockHolder to hold a broadcasted PBlock. For broadcast shuffle, one PBlock
// will be shared between different channel, so we have to use a ref count to mark if this
// PBlock is available for next serialization.
class BroadcastPBlockHolder {
public:
BroadcastPBlockHolder() : _ref_count(0) {}
~BroadcastPBlockHolder() noexcept = default;
void unref() noexcept {
DCHECK_GT(_ref_count._value, 0);
_ref_count._value.fetch_sub(1);
}
void ref() noexcept { _ref_count._value.fetch_add(1); }
bool available() { return _ref_count._value == 0; }
PBlock* get_block() { return &pblock; }
private:
AtomicWrapper<uint32_t> _ref_count;
PBlock pblock;
};
class VDataStreamSender : public DataSink {
public:
friend class pipeline::ExchangeSinkOperator;
@ -91,6 +127,7 @@ protected:
friend class pipeline::ExchangeSinkBuffer;
void _roll_pb_block();
Status _get_next_available_buffer(BroadcastPBlockHolder** holder);
Status get_partition_column_result(Block* block, int* result) const {
int counter = 0;
@ -131,6 +168,10 @@ protected:
PBlock _pb_block2;
PBlock* _cur_pb_block;
// used by pipeline engine
std::vector<BroadcastPBlockHolder> _broadcast_pb_blocks;
int _broadcast_pb_block_idx;
// compute per-row partition values
std::vector<VExprContext*> _partition_expr_ctxs;
@ -219,6 +260,10 @@ public:
// if batch is nullptr, send the eof packet
virtual Status send_block(PBlock* block, bool eos = false);
virtual Status send_block(BroadcastPBlockHolder* block, bool eos = false) {
return Status::InternalError("Send BroadcastPBlockHolder is not allowed!");
}
Status add_rows(Block* block, const std::vector<int>& row);
virtual Status send_current_block(bool eos);
@ -369,8 +414,21 @@ public:
}
}
if (eos || block->column_metas_size()) {
RETURN_IF_ERROR(_buffer->add_block(
{this, block ? std::make_unique<PBlock>(*block) : nullptr, eos}));
RETURN_IF_ERROR(_buffer->add_block({this, block, eos}));
}
return Status::OK();
}
Status send_block(BroadcastPBlockHolder* block, bool eos = false) override {
if (eos) {
if (_eos_send) {
return Status::OK();
} else {
_eos_send = true;
}
}
if (eos || block->get_block()->column_metas_size()) {
RETURN_IF_ERROR(_buffer->add_block({this, block, eos}));
}
return Status::OK();
}