[enhancement](group commit)Add group commit block queues memory back pressure (#26045)

This commit is contained in:
abmdocrt
2023-11-01 16:29:45 +08:00
committed by GitHub
parent 683832230c
commit 387e33fa34
4 changed files with 58 additions and 13 deletions

View File

@ -23,7 +23,11 @@
#include <gen_cpp/PaloInternalService_types.h>
#include <gen_cpp/Types_types.h>
#include <memory>
#include <numeric>
#include "client_cache.h"
#include "common/config.h"
#include "common/object_pool.h"
#include "exec/data_sink.h"
#include "io/fs/stream_load_pipe.h"
@ -34,6 +38,7 @@
#include "runtime/stream_load/new_load_stream_mgr.h"
#include "runtime/stream_load/stream_load_context.h"
#include "util/thrift_rpc_helper.h"
#include "vec/core/future_block.h"
#include "vec/exec/scan/new_file_scan_node.h"
#include "vec/sink/group_commit_block_sink.h"
@ -45,10 +50,16 @@ Status LoadBlockQueue::add_block(std::shared_ptr<vectorized::FutureBlock> block)
DCHECK(block->get_schema_version() == schema_version);
std::unique_lock l(*_mutex);
RETURN_IF_ERROR(_status);
while (*_all_block_queues_bytes > config::group_commit_max_queue_size) {
_put_cond.wait_for(
l, std::chrono::milliseconds(LoadBlockQueue::MAX_BLOCK_QUEUE_ADD_WAIT_TIME));
}
if (block->rows() > 0) {
_block_queue.push_back(block);
*_all_block_queues_bytes += block->bytes();
*_single_block_queue_bytes += block->bytes();
}
_cv->notify_one();
_get_cond.notify_all();
return Status::OK();
}
@ -67,6 +78,7 @@ Status LoadBlockQueue::get_block(vectorized::Block* block, bool* find_block, boo
}
while (_status.ok() && _block_queue.empty() &&
(!need_commit || (need_commit && !_load_ids.empty()))) {
CHECK(*_single_block_queue_bytes == 0);
auto left_milliseconds = config::group_commit_interval_ms;
if (!need_commit) {
left_milliseconds = config::group_commit_interval_ms -
@ -79,9 +91,9 @@ Status LoadBlockQueue::get_block(vectorized::Block* block, bool* find_block, boo
}
}
#if !defined(USE_BTHREAD_SCANNER)
_cv->wait_for(l, std::chrono::milliseconds(left_milliseconds));
_get_cond.wait_for(l, std::chrono::milliseconds(left_milliseconds));
#else
_cv->wait_for(l, left_milliseconds * 1000);
_get_cond.wait_for(l, left_milliseconds * 1000);
#endif
}
if (!_block_queue.empty()) {
@ -90,12 +102,16 @@ Status LoadBlockQueue::get_block(vectorized::Block* block, bool* find_block, boo
fblock->swap_future_block(future_block);
*find_block = true;
_block_queue.pop_front();
*_all_block_queues_bytes -= fblock->bytes();
*_single_block_queue_bytes -= block->bytes();
}
if (_block_queue.empty() && need_commit && _load_ids.empty()) {
CHECK(*_single_block_queue_bytes == 0);
*eos = true;
} else {
*eos = false;
}
_put_cond.notify_all();
return Status::OK();
}
@ -103,7 +119,7 @@ void LoadBlockQueue::remove_load_id(const UniqueId& load_id) {
std::unique_lock l(*_mutex);
if (_load_ids.find(load_id) != _load_ids.end()) {
_load_ids.erase(load_id);
_cv->notify_one();
_get_cond.notify_all();
}
}
@ -126,6 +142,8 @@ void LoadBlockQueue::cancel(const Status& st) {
auto& future_block = _block_queue.front();
std::unique_lock<doris::Mutex> l0(*(future_block->lock));
future_block->set_result(st, future_block->rows(), 0);
*_all_block_queues_bytes -= future_block->bytes();
*_single_block_queue_bytes -= future_block->bytes();
future_block->cv->notify_all();
}
_block_queue.pop_front();
@ -248,8 +266,8 @@ Status GroupCommitTable::_create_group_commit_load(
<< ", txn_id=" << txn_id << ", instance_id=" << print_id(instance_id)
<< ", is_pipeline=" << is_pipeline;
{
load_block_queue =
std::make_shared<LoadBlockQueue>(instance_id, label, txn_id, schema_version);
load_block_queue = std::make_shared<LoadBlockQueue>(
instance_id, label, txn_id, schema_version, _all_block_queues_bytes);
std::unique_lock l(_lock);
_load_block_queues.emplace(instance_id, load_block_queue);
_need_plan_fragment = false;
@ -398,6 +416,7 @@ GroupCommitMgr::GroupCommitMgr(ExecEnv* exec_env) : _exec_env(exec_env) {
.set_min_threads(1)
.set_max_threads(config::group_commit_insert_threads)
.build(&_thread_pool));
_all_block_queues_bytes = std::make_shared<std::atomic_size_t>(0);
}
GroupCommitMgr::~GroupCommitMgr() {
@ -536,7 +555,8 @@ Status GroupCommitMgr::get_first_block_load_queue(
std::lock_guard wlock(_lock);
if (_table_map.find(table_id) == _table_map.end()) {
_table_map.emplace(table_id, std::make_shared<GroupCommitTable>(
_exec_env, _thread_pool.get(), db_id, table_id));
_exec_env, _thread_pool.get(), db_id, table_id,
_all_block_queues_bytes));
}
group_commit_table = _table_map[table_id];
}