Skip to content

[feat](load) quorum success write #51567

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

Draft
wants to merge 5 commits into
base: master
Choose a base branch
from
Draft
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
3 changes: 3 additions & 0 deletions be/src/common/config.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1496,6 +1496,9 @@ DEFINE_mInt32(load_trigger_compaction_version_percent, "66");
DEFINE_mInt64(base_compaction_interval_seconds_since_last_operation, "86400");
DEFINE_mBool(enable_compaction_pause_on_high_memory, "true");

DEFINE_mDouble(max_wait_time_multiplier, "0.5");
DEFINE_mInt32(load_timeout_remaining_seconds, "30");

DEFINE_mBool(enable_calc_delete_bitmap_between_segments_concurrently, "false");

DEFINE_mBool(enable_update_delete_bitmap_kv_check_core, "false");
Expand Down
3 changes: 3 additions & 0 deletions be/src/common/config.h
Original file line number Diff line number Diff line change
Expand Up @@ -1570,6 +1570,9 @@ DECLARE_mInt32(load_trigger_compaction_version_percent);
DECLARE_mInt64(base_compaction_interval_seconds_since_last_operation);
DECLARE_mBool(enable_compaction_pause_on_high_memory);

DECLARE_mDouble(max_wait_time_multiplier);
DECLARE_mInt32(load_timeout_remaining_seconds);

DECLARE_mBool(enable_calc_delete_bitmap_between_segments_concurrently);

DECLARE_mBool(enable_update_delete_bitmap_kv_check_core);
Expand Down
3 changes: 2 additions & 1 deletion be/src/exec/tablet_info.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -498,7 +498,8 @@ Status VOlapTablePartitionParam::init() {
RETURN_IF_ERROR(generate_partition_from(t_part, part));
_partitions.emplace_back(part);

if (!_t_param.partitions_is_fake) {
bool partition_is_fake = t_part.__isset.partition_is_fake && t_part.partition_is_fake;
if (!_t_param.partitions_is_fake && !partition_is_fake) {
if (_is_in_partition) {
for (auto& in_key : part->in_keys) {
_partitions_map->emplace(std::tuple {in_key.first, in_key.second, false}, part);
Expand Down
7 changes: 6 additions & 1 deletion be/src/runtime/load_stream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -453,7 +453,12 @@ LoadStream::~LoadStream() {
Status LoadStream::init(const POpenLoadStreamRequest* request) {
_txn_id = request->txn_id();
_total_streams = static_cast<int32_t>(request->total_streams());
_is_incremental = (_total_streams == 0);
bool auto_partition_one_step_close = request->has_auto_partition_one_step_close()
? request->auto_partition_one_step_close()
: false;
if (!auto_partition_one_step_close) {
_is_incremental = (_total_streams == 0);
}

_schema = std::make_shared<OlapTableSchemaParam>();
RETURN_IF_ERROR(_schema->init(request->schema()));
Expand Down
5 changes: 4 additions & 1 deletion be/src/runtime/tablets_channel.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,10 @@ Status BaseTabletsChannel::open(const PTabletWriterOpenRequest& request) {
* then _num_remaining_senders will not be set here. but inc every time when incremental_open() called. so it's dynamic
* and also need same number of senders' close to close. but will not hang.
*/
if (_open_by_incremental) {
bool auto_partition_one_step_close = request.has_auto_partition_one_step_close()
? request.auto_partition_one_step_close()
: false;
if (!auto_partition_one_step_close && _open_by_incremental) {
DCHECK(_num_remaining_senders == 0) << _num_remaining_senders;
} else {
_num_remaining_senders = max_sender;
Expand Down
16 changes: 16 additions & 0 deletions be/src/vec/sink/load_stream_map_pool.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,22 @@ void LoadStreamMap::close_load(bool incremental) {
}
}

void LoadStreamMap::close_load_all_streams() {
for (auto& [dst_id, streams] : _streams_for_node) {
std::vector<PTabletID> tablets_to_commit;
const auto& tablets = _tablets_to_commit[dst_id];
tablets_to_commit.reserve(tablets.size());
for (const auto& [tablet_id, tablet] : tablets) {
tablets_to_commit.push_back(tablet);
tablets_to_commit.back().set_num_segments(_segments_for_tablet[tablet_id]);
}
auto st = streams->close_load(tablets_to_commit);
if (!st.ok()) {
LOG(WARNING) << "close_load for all streams failed: " << st << ", load_id=" << _load_id;
}
}
}

LoadStreamMapPool::LoadStreamMapPool() = default;

LoadStreamMapPool::~LoadStreamMapPool() = default;
Expand Down
13 changes: 13 additions & 0 deletions be/src/vec/sink/load_stream_map_pool.h
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,19 @@ class LoadStreamMap {
// only call this method after release() returns true.
void close_load(bool incremental);

// send CLOSE_LOAD to all streams, return ERROR if any.
// only call this method after release() returns true.
void close_load_all_streams();

std::unordered_map<int64_t, std::shared_ptr<LoadStreamStubs>> get_streams_for_node() {
decltype(_streams_for_node) snapshot;
{
std::lock_guard<std::mutex> lock(_mutex);
snapshot = _streams_for_node;
}
return snapshot;
}

private:
const UniqueId _load_id;
const int64_t _src_id;
Expand Down
68 changes: 27 additions & 41 deletions be/src/vec/sink/load_stream_stub.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -114,9 +114,7 @@ void LoadStreamReplyHandler::on_closed(brpc::StreamId id) {
LOG(WARNING) << "stub is not exist when on_closed, " << *this;
return;
}
std::lock_guard<bthread::Mutex> lock(stub->_close_mutex);
stub->_is_closed.store(true);
stub->_close_cv.notify_all();
}

inline std::ostream& operator<<(std::ostream& ostr, const LoadStreamReplyHandler& handler) {
Expand Down Expand Up @@ -146,7 +144,8 @@ Status LoadStreamStub::open(BrpcClientCache<PBackendService_Stub>* client_cache,
const NodeInfo& node_info, int64_t txn_id,
const OlapTableSchemaParam& schema,
const std::vector<PTabletID>& tablets_for_schema, int total_streams,
int64_t idle_timeout_ms, bool enable_profile) {
int64_t idle_timeout_ms, bool enable_profile,
bool auto_partition_one_step_close) {
std::unique_lock<bthread::Mutex> lock(_open_mutex);
if (_is_init.load()) {
return _status;
Expand All @@ -170,14 +169,17 @@ Status LoadStreamStub::open(BrpcClientCache<PBackendService_Stub>* client_cache,
request.set_src_id(_src_id);
request.set_txn_id(txn_id);
request.set_enable_profile(enable_profile);
if (_is_incremental) {
if (_is_incremental && !auto_partition_one_step_close) {
request.set_total_streams(0);
} else if (total_streams > 0) {
request.set_total_streams(total_streams);
} else {
_status = Status::InternalError("total_streams should be greator than 0");
return _status;
}
if (auto_partition_one_step_close) {
request.set_auto_partition_one_step_close(auto_partition_one_step_close);
}
request.set_idle_timeout_ms(idle_timeout_ms);
schema.to_protobuf(request.mutable_schema());
for (auto& tablet : tablets_for_schema) {
Expand Down Expand Up @@ -330,37 +332,28 @@ Status LoadStreamStub::wait_for_schema(int64_t partition_id, int64_t index_id, i
return Status::OK();
}

Status LoadStreamStub::close_wait(RuntimeState* state, int64_t timeout_ms) {
Status LoadStreamStub::close_finish_check(RuntimeState* state, bool* is_closed) {
DBUG_EXECUTE_IF("LoadStreamStub::close_wait.long_wait", DBUG_BLOCK);
*is_closed = true;
if (!_is_open.load()) {
// we don't need to close wait on non-open streams
return Status::OK();
}
if (!_is_closing.load()) {
*is_closed = false;
return _status;
}
if (state->get_query_ctx()->is_cancelled()) {
return state->get_query_ctx()->exec_status();
}
if (_is_closed.load()) {
return _check_cancel();
}
DCHECK(timeout_ms > 0) << "timeout_ms should be greator than 0";
std::unique_lock<bthread::Mutex> lock(_close_mutex);
auto timeout_sec = timeout_ms / 1000;
while (!_is_closed.load() && !state->get_query_ctx()->is_cancelled()) {
//the query maybe cancel, so need check after wait 1s
timeout_sec = timeout_sec - 1;
LOG(INFO) << "close waiting, " << *this << ", timeout_sec=" << timeout_sec
<< ", is_closed=" << _is_closed.load()
<< ", is_cancelled=" << state->get_query_ctx()->is_cancelled();
int ret = _close_cv.wait_for(lock, 1000000);
if (ret != 0 && timeout_sec <= 0) {
return Status::InternalError("stream close_wait timeout, error={}, timeout_ms={}, {}",
ret, timeout_ms, to_string());
RETURN_IF_ERROR(_check_cancel());
if (!_is_eos.load()) {
return Status::InternalError("Stream closed without EOS, {}", to_string());
}
return Status::OK();
}
RETURN_IF_ERROR(_check_cancel());
if (!_is_eos.load()) {
return Status::InternalError("stream closed without eos, {}", to_string());
}
*is_closed = false;
return Status::OK();
}

Expand All @@ -374,15 +367,12 @@ void LoadStreamStub::cancel(Status reason) {
_cancel_st = reason;
_is_cancelled.store(true);
}
{
std::lock_guard<bthread::Mutex> lock(_close_mutex);
_is_closed.store(true);
_close_cv.notify_all();
}
_is_closed.store(true);
}

Status LoadStreamStub::_encode_and_send(PStreamHeader& header, std::span<const Slice> data) {
butil::IOBuf buf;
// append data to buffer
size_t header_len = header.ByteSizeLong();
buf.append(reinterpret_cast<uint8_t*>(&header_len), sizeof(header_len));
buf.append(header.SerializeAsString());
Expand All @@ -394,6 +384,9 @@ Status LoadStreamStub::_encode_and_send(PStreamHeader& header, std::span<const S
}
bool eos = header.opcode() == doris::PStreamHeader::CLOSE_LOAD;
bool get_schema = header.opcode() == doris::PStreamHeader::GET_SCHEMA;
// update bytes written
_bytes_written += buf.size();
// send buffer
return _send_with_buffer(buf, eos || get_schema);
}

Expand Down Expand Up @@ -506,17 +499,19 @@ Status LoadStreamStubs::open(BrpcClientCache<PBackendService_Stub>* client_cache
const NodeInfo& node_info, int64_t txn_id,
const OlapTableSchemaParam& schema,
const std::vector<PTabletID>& tablets_for_schema, int total_streams,
int64_t idle_timeout_ms, bool enable_profile) {
int64_t idle_timeout_ms, bool enable_profile,
bool auto_partition_one_step_close) {
bool get_schema = true;
auto status = Status::OK();
for (auto& stream : _streams) {
Status st;
if (get_schema) {
st = stream->open(client_cache, node_info, txn_id, schema, tablets_for_schema,
total_streams, idle_timeout_ms, enable_profile);
total_streams, idle_timeout_ms, enable_profile,
auto_partition_one_step_close);
} else {
st = stream->open(client_cache, node_info, txn_id, schema, {}, total_streams,
idle_timeout_ms, enable_profile);
idle_timeout_ms, enable_profile, auto_partition_one_step_close);
}
if (st.ok()) {
get_schema = false;
Expand Down Expand Up @@ -556,13 +551,4 @@ Status LoadStreamStubs::close_load(const std::vector<PTabletID>& tablets_to_comm
return status;
}

Status LoadStreamStubs::close_wait(RuntimeState* state, int64_t timeout_ms) {
MonotonicStopWatch watch;
watch.start();
for (auto& stream : _streams) {
RETURN_IF_ERROR(stream->close_wait(state, timeout_ms - watch.elapsed_time() / 1000 / 1000));
}
return Status::OK();
}

} // namespace doris
17 changes: 10 additions & 7 deletions be/src/vec/sink/load_stream_stub.h
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
#include <atomic>
// IWYU pragma: no_include <bits/chrono.h>
#include <chrono> // IWYU pragma: keep
#include <cstdint>
#include <functional>
#include <initializer_list>
#include <map>
Expand Down Expand Up @@ -127,7 +128,7 @@ class LoadStreamStub : public std::enable_shared_from_this<LoadStreamStub> {
Status open(BrpcClientCache<PBackendService_Stub>* client_cache, const NodeInfo& node_info,
int64_t txn_id, const OlapTableSchemaParam& schema,
const std::vector<PTabletID>& tablets_for_schema, int total_streams,
int64_t idle_timeout_ms, bool enable_profile);
int64_t idle_timeout_ms, bool enable_profile, bool auto_partition_one_step_close);

// for mock this class in UT
#ifdef BE_TEST
Expand Down Expand Up @@ -155,7 +156,7 @@ class LoadStreamStub : public std::enable_shared_from_this<LoadStreamStub> {

// wait remote to close stream,
// remote will close stream when it receives CLOSE_LOAD
Status close_wait(RuntimeState* state, int64_t timeout_ms = 0);
Status close_finish_check(RuntimeState* state, bool* is_closed);

// cancel the stream, abort close_wait, mark _is_closed and _is_cancelled
void cancel(Status reason);
Expand Down Expand Up @@ -216,6 +217,8 @@ class LoadStreamStub : public std::enable_shared_from_this<LoadStreamStub> {
_failed_tablets[tablet_id] = reason;
}

int64_t bytes_written() const { return _bytes_written; }

private:
Status _encode_and_send(PStreamHeader& header, std::span<const Slice> data = {});
Status _send_with_buffer(butil::IOBuf& buf, bool sync = false);
Expand Down Expand Up @@ -247,9 +250,7 @@ class LoadStreamStub : public std::enable_shared_from_this<LoadStreamStub> {
Status _cancel_st;

bthread::Mutex _open_mutex;
bthread::Mutex _close_mutex;
bthread::Mutex _cancel_mutex;
bthread::ConditionVariable _close_cv;

std::mutex _buffer_mutex;
std::mutex _send_mutex;
Expand All @@ -266,6 +267,8 @@ class LoadStreamStub : public std::enable_shared_from_this<LoadStreamStub> {
std::unordered_map<int64_t, Status> _failed_tablets;

bool _is_incremental = false;
bool _auto_partition_one_step_close = false;
size_t _bytes_written = 0;
};

// a collection of LoadStreams connect to the same node
Expand All @@ -285,7 +288,7 @@ class LoadStreamStubs {
Status open(BrpcClientCache<PBackendService_Stub>* client_cache, const NodeInfo& node_info,
int64_t txn_id, const OlapTableSchemaParam& schema,
const std::vector<PTabletID>& tablets_for_schema, int total_streams,
int64_t idle_timeout_ms, bool enable_profile);
int64_t idle_timeout_ms, bool enable_profile, bool auto_partition_one_step_close);

bool is_incremental() const { return _is_incremental; }

Expand All @@ -310,8 +313,6 @@ class LoadStreamStubs {

Status close_load(const std::vector<PTabletID>& tablets_to_commit);

Status close_wait(RuntimeState* state, int64_t timeout_ms = 0);

std::unordered_set<int64_t> success_tablets() {
std::unordered_set<int64_t> s;
for (auto& stream : _streams) {
Expand All @@ -330,6 +331,8 @@ class LoadStreamStubs {
return m;
}

std::vector<std::shared_ptr<LoadStreamStub>> streams() { return _streams; }

private:
std::vector<std::shared_ptr<LoadStreamStub>> _streams;
std::atomic<bool> _open_success = false;
Expand Down
Loading