[FEAT MERGE] Optimize OBCDC handle misslog

Co-authored-by: zxlzxlzxlzxlzxl <zxlzxlzxlzxlzxl@outlook.com>
This commit is contained in:
SanmuWangZJU 2024-09-26 12:47:03 +00:00 committed by ob-robot
parent 1df5c3224b
commit 1a96e0a6fa
37 changed files with 3271 additions and 897 deletions

View File

@ -113,12 +113,12 @@ public:
return ret;
}
OB_INLINE bool is_empty()
OB_INLINE bool is_empty() const
{
return (NULL == root_);
}
OB_INLINE T *get_first()
OB_INLINE T *get_first() const
{
T *tmp_node = NULL;
if (OB_NOT_NULL(root_)) {
@ -127,7 +127,7 @@ public:
return tmp_node;
}
OB_INLINE T *get_last()
OB_INLINE T *get_last() const
{
T *tmp_node = NULL;
if (OB_NOT_NULL(root_)) {
@ -136,7 +136,7 @@ public:
return tmp_node;
}
OB_INLINE int get_next(const T *node, T *&return_node)
OB_INLINE int get_next(const T *node, T *&return_node) const
{
int ret = common::OB_SUCCESS;
T *tmp_node = NULL;
@ -166,7 +166,7 @@ public:
return ret;
}
OB_INLINE int get_prev(const T *node, T *&return_node)
OB_INLINE int get_prev(const T *node, T *&return_node) const
{
int ret = common::OB_SUCCESS;
T *tmp_node = NULL;
@ -197,7 +197,7 @@ public:
}
//Node in tree that matches key, or NULL if no match
OB_INLINE int search(const T *key, T *&return_node)
OB_INLINE int search(const T *key, T *&return_node) const
{
int ret = common::OB_SUCCESS;
int cmp = 0;
@ -291,62 +291,64 @@ public:
} else {
init_node(node);
path[0].node_ = root_;
for (pathp = path; OB_NOT_NULL(pathp->node_); pathp++) {
for (pathp = path; OB_NOT_NULL(pathp->node_) && OB_SUCC(ret); pathp++) {
cmp = pathp->cmp_ = compare_.compare(node, pathp->node_);
abort_unless(0 != cmp);
if (cmp < 0) {
pathp[1].node_ = get_left(pathp->node_);
} else if (cmp > 0) {
pathp[1].node_ = get_right(pathp->node_);
}
}
pathp->node_ = node;
for (pathp--; (reinterpret_cast<uint64_t>(pathp) >= reinterpret_cast<uint64_t>(path)); pathp--) {
cur_node = pathp->node_;
if (pathp->cmp_ < 0) {
left_node = pathp[1].node_;
set_left(cur_node, left_node);
if (get_red(left_node)) {
left_node_left = get_left(left_node);
if (OB_NOT_NULL(left_node_left) && get_red(left_node_left)) {
//Fix up 4-node.
set_black(left_node_left);
rotate_right(cur_node, tmp_node);
cur_node = tmp_node;
}
} else {
break;
}
} else {
right_node = pathp[1].node_;
set_right(cur_node, right_node);
if (!get_red(right_node)) {
break;
} else {
left_node = get_left(cur_node);
if (OB_NOT_NULL(left_node) && get_red(left_node)) {
//Split 4-node.
set_black(left_node);
set_black(right_node);
set_red(cur_node);
} else {
//Lean left.
bool tred = get_red(cur_node);
rotate_left(cur_node, tmp_node);
set_color(tmp_node, tred);
set_red(cur_node);
cur_node = tmp_node;
}
}
ret = common::OB_ENTRY_EXIST;
}
pathp->node_ = cur_node;
}
//Set root, and make it black.
root_ = path->node_;
set_black(root_);
if (OB_SUCC(ret)) {
pathp->node_ = node;
for (pathp--; (reinterpret_cast<uint64_t>(pathp) >= reinterpret_cast<uint64_t>(path)); pathp--) {
cur_node = pathp->node_;
if (pathp->cmp_ < 0) {
left_node = pathp[1].node_;
set_left(cur_node, left_node);
if (get_red(left_node)) {
left_node_left = get_left(left_node);
if (OB_NOT_NULL(left_node_left) && get_red(left_node_left)) {
//Fix up 4-node.
set_black(left_node_left);
rotate_right(cur_node, tmp_node);
cur_node = tmp_node;
}
} else {
break;
}
} else {
right_node = pathp[1].node_;
set_right(cur_node, right_node);
if (!get_red(right_node)) {
break;
} else {
left_node = get_left(cur_node);
if (OB_NOT_NULL(left_node) && get_red(left_node)) {
//Split 4-node.
set_black(left_node);
set_black(right_node);
set_red(cur_node);
} else {
//Lean left.
bool tred = get_red(cur_node);
rotate_left(cur_node, tmp_node);
set_color(tmp_node, tred);
set_red(cur_node);
cur_node = tmp_node;
}
}
}
pathp->node_ = cur_node;
}
//Set root, and make it black.
root_ = path->node_;
set_black(root_);
}
}
return ret;
}
@ -884,7 +886,7 @@ private:
}
//Internal utility macros.
OB_INLINE T *get_first(T *root)
OB_INLINE T *get_first(T *root) const
{
T *tmp_node = NULL;
T * return_node = NULL;
@ -897,7 +899,7 @@ private:
return return_node;
}
OB_INLINE T *get_last(T *root)
OB_INLINE T *get_last(T *root) const
{
T *tmp_node = NULL;
T *return_node = NULL;

View File

@ -16,6 +16,7 @@
#include "ob_cdc_fetcher.h"
#include "ob_cdc_define.h"
#include "storage/tx_storage/ob_ls_handle.h"
#include "logservice/restoreservice/ob_remote_log_raw_reader.h"
#include "logservice/restoreservice/ob_remote_log_source_allocator.h"
#include "logservice/restoreservice/ob_remote_log_raw_reader.h"
@ -104,6 +105,7 @@ int ObCdcFetcher::fetch_log(const ObCdcLSFetchLogReq &req,
} else {
const ObLSID &ls_id = req.get_ls_id();
const LSN &start_lsn = req.get_start_lsn();
PalfHandleGuard palf_handle_guard;
const ObCdcRpcId &rpc_id = req.get_client_id();
ClientLSCtx *ls_ctx = NULL;
@ -211,19 +213,29 @@ int ObCdcFetcher::fetch_missing_log(const obrpc::ObCdcLSFetchMissLogReq &req,
const ObCdcRpcId &rpc_id = req.get_client_id();
ClientLSKey ls_key(rpc_id.get_addr(), rpc_id.get_pid(), req.get_tenant_id(), ls_id);
ClientLSCtxMap &ctx_map = MTL(ObLogService*)->get_cdc_service()->get_ls_ctx_map();
ClientLSCtx *ls_ctx = NULL;
ClientLSCtx *ls_ctx = nullptr;
if (OB_FAIL(ctx_map.get(ls_key, ls_ctx))) {
LOG_WARN("get client ls ctx from ctx map failed", KR(ret));
if (OB_FAIL(host_->get_or_create_client_ls_ctx(req.get_client_id(),
req.get_tenant_id(), ls_id, req.get_flag(),
req.get_progress(), FetchLogProtocolType::Unknown, ls_ctx))) {
LOG_ERROR("get_or_create_client_ls_ctx failed", KR(ret), K(req));
} else if (OB_ISNULL(ls_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls ctx is null, unexpected", KR(ret), K(ls_key));
LOG_WARN("get null ls_ctx, unexpected", K(ls_ctx));
} else {
ls_ctx->update_touch_ts();
if (OB_FAIL(do_fetch_missing_log_(req, frt, resp, *ls_ctx))) {
LOG_WARN("do fetch log error", KR(ret), K(req));
} else {}
ctx_map.revert(ls_ctx);
}
if (OB_NOT_NULL(ls_ctx)) {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(host_->revert_client_ls_ctx(ls_ctx))) {
LOG_WARN_RET(tmp_ret, "failed to revert client ls ctx", K(req));
} else {
ls_ctx = nullptr;
}
}
}
}
@ -883,6 +895,237 @@ int ObCdcFetcher::check_ls_sync_status_(const ObLSID &ls_id,
return ret;
}
int ObCdcFetcher::fetch_missing_logs_in_palf_(const ObLSID &ls_id,
palf::PalfHandleGuard &palf_handle_guard,
const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
int64_t &cur_idx,
obrpc::ObCdcLSFetchLogResp &resp,
FetchRunTime &frt)
{
int ret = OB_SUCCESS;
bool need_seek = true;
PalfBufferIterator log_entry_iter;
const int64_t SINGLE_READ_SIZE = 16 * 1024 * 1024L;
while (! frt.is_stopped() && OB_SUCC(ret) && cur_idx < miss_log_array.count()) {
const palf::LSN &curr_lsn = miss_log_array.at(cur_idx).miss_lsn_;
resp.set_next_miss_lsn(curr_lsn);
if (ObTimeUtility::current_time() > frt.rpc_deadline_) {
frt.stop("ReachRpcDeadline");
} else if (need_seek && OB_FAIL(seek_log_iterator_for_cdc(ls_id, curr_lsn, SINGLE_READ_SIZE, log_entry_iter))) {
if (OB_ERR_OUT_OF_LOWER_BOUND != ret) {
LOG_WARN("failed to seek log entry iterator", K(curr_lsn), K(ls_id));
}
} else if (OB_FAIL(log_entry_iter.set_io_context(palf::LogIOContext(tenant_id_, ls_id.id(), palf::LogIOUser::CDC)))) {
LOG_WARN("set_io_context fail", KR(ret), K(ls_id), K(curr_lsn), K(cur_idx));
} else {
LogEntry log_entry;
palf::LSN log_entry_lsn;
need_seek = false;
if (OB_FAIL(log_entry_iter.next())) {
LOG_WARN("log entry iterator failed to next", K(curr_lsn), K(ls_id));
} else if (OB_FAIL(log_entry_iter.get_entry(log_entry, log_entry_lsn))) {
LOG_WARN("log entry iterator failed to get entry", K(curr_lsn), K(ls_id));
} else if (log_entry_lsn < curr_lsn) {
// do nothing
} else if (log_entry_lsn == curr_lsn) {
if (OB_FAIL(prefill_resp_with_log_entry_(ls_id, log_entry_lsn, log_entry, resp))) {
if (OB_BUF_NOT_ENOUGH == ret) {
handle_when_buffer_full_(frt); // stop
ret = OB_SUCCESS;
} else {
LOG_WARN("prefill_resp_with_log_entry fail", KR(ret), K(frt), K(resp));
}
} else {
++cur_idx;
if (cur_idx < miss_log_array.count()) {
if (miss_log_array.at(cur_idx).miss_lsn_ <= curr_lsn) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the order of miss_log_array is unexpected", "next_miss_lsn",
miss_log_array.at(cur_idx).miss_lsn_, K(curr_lsn));
} else if (miss_log_array.at(cur_idx).miss_lsn_ - curr_lsn > MAX_LOG_BUFFER_SIZE) {
need_seek = true;
}
}
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get log_entry_lsn which is larger than curr_lsn, maybe fetch log from wrong cluster",
K(log_entry_lsn), K(curr_lsn));
}
}
}
return ret;
}
int ObCdcFetcher::fetch_missing_logs_in_archive_(const ObLSID &ls_id,
ClientLSCtx &ctx,
const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
int64_t &cur_idx,
obrpc::ObCdcLSFetchLogResp &resp,
FetchRunTime &frt)
{
int ret = OB_SUCCESS;
int64_t version = 0;
ObCdcGetSourceFunctor get_source_func(ctx, version);
ObCdcUpdateSourceFunctor update_source_func(ctx, version);
ObRemoteLogRawReader raw_reader(get_source_func, update_source_func);
share::SCN pre_scn;
const int64_t end_tstamp = frt.rpc_deadline_ - RPC_QIT_RESERVED_TIME;
const int64_t MAX_RETRY_COUNT = 4;
int64_t retry_count = 0;
if (OB_FAIL(pre_scn.convert_from_ts(ctx.get_progress()/1000L))) {
LOG_WARN("failed to convert progress to scn", "progress", ctx.get_progress());
} else if (OB_FAIL(raw_reader.init(tenant_id_, ls_id, pre_scn, log_ext_handler_))) {
LOG_WARN("raw reader failed to init", K(tenant_id_), K(ls_id), K(pre_scn), KP(log_ext_handler_));
} else {
const int64_t MAX_READ_SIZE = 16L << 20;
char *tmp_buf = large_buffer_pool_->acquire(MAX_READ_SIZE);
int64_t read_size = 0;
struct {
palf::LSN operator()() {
return palf::LSN(palf::LOG_MAX_LSN_VAL);
}
} fetch_missing_functor;
if (nullptr == tmp_buf) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to allocate memory for raw_read buffer", K(MAX_READ_SIZE));
}
while (OB_SUCC(ret) && cur_idx < miss_log_array.count() && !frt.is_stopped()) {
const palf::LSN &read_start_lsn = miss_log_array.at(cur_idx).miss_lsn_;
int64_t real_read_size = 0;
MemPalfBufferIterator iter;
MemoryStorage mem_storage;
int64_t target_idx = 0;
resp.set_next_miss_lsn(read_start_lsn);
if (ObTimeUtility::current_time() > end_tstamp) {
frt.stop("OuterTimeUp");
} else if (OB_FAIL(calc_raw_read_size_(miss_log_array, cur_idx, MAX_READ_SIZE, read_size, target_idx))) {
LOG_WARN("failed to calc raw read size", K(miss_log_array), K(cur_idx), K(MAX_READ_SIZE));
} else if (OB_FAIL(raw_reader.raw_read(read_start_lsn, tmp_buf, read_size, real_read_size))) {
if (OB_ERR_OUT_OF_UPPER_BOUND != ret) {
LOG_WARN("raw_reader failed to read raw log", K(read_start_lsn), K(read_size));
}
} else if (OB_FAIL(mem_storage.init(read_start_lsn))) {
LOG_WARN("memory storage failed to init", K(read_start_lsn));
} else if (OB_FAIL(mem_storage.append(tmp_buf, real_read_size))) {
LOG_WARN("memory storage failed to append buffer", K(real_read_size));
} else if (OB_FAIL(iter.init(read_start_lsn, fetch_missing_functor, &mem_storage))) {
LOG_WARN("iterator failed to init", K(read_start_lsn), K(mem_storage));
} else {
LOG_TRACE("raw read finish", K(read_start_lsn), K(read_size), K(real_read_size), K(mem_storage));
raw_reader.update_source_cb();
while (OB_SUCC(ret) && cur_idx < miss_log_array.count() && cur_idx < target_idx && !frt.is_stopped()) {
const palf::LSN &cur_wanted_lsn = miss_log_array.at(cur_idx).miss_lsn_;
LogEntry log_entry;
palf::LSN log_entry_lsn;
resp.set_next_miss_lsn(cur_wanted_lsn);
if (ObTimeUtility::current_time() > end_tstamp) {
frt.stop("InnerTimeUp");
} else if (OB_FAIL(iter.next())) {
LOG_WARN("failed to iterate log", K(cur_idx), K(cur_wanted_lsn), K(real_read_size), K(mem_storage));
} else if (OB_FAIL(iter.get_entry(log_entry, log_entry_lsn))) {
LOG_WARN("iterator failed to get entry", K(mem_storage));
} else if (cur_wanted_lsn == log_entry_lsn) {
if (OB_FAIL(prefill_resp_with_log_entry_(ls_id, log_entry_lsn, log_entry, resp))) {
if (OB_BUF_NOT_ENOUGH == ret) {
handle_when_buffer_full_(frt); // stop
ret = OB_SUCCESS;
} else {
LOG_WARN("prefill_resp_with_log_entry fail", KR(ret), K(frt), K(resp));
}
} else {
cur_idx++;
ctx.set_progress(log_entry.get_scn().get_val_for_logservice());
if (cur_idx < miss_log_array.count()) {
const palf::LSN next_missing_lsn = miss_log_array.at(cur_idx).miss_lsn_;
if (next_missing_lsn < cur_wanted_lsn) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("next_missing_lsn is smaller, unexpected", K(next_missing_lsn), K(read_start_lsn));
}
}
}
} else if (cur_wanted_lsn > log_entry_lsn) {
// do nothing
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get log_entry_lsn which is larger than curr_lsn, maybe fetch log from wrong cluster",
K(log_entry_lsn), K(read_start_lsn));
}
}
if ((OB_NEED_RETRY == ret || OB_INVALID_DATA == ret || OB_CHECKSUM_ERROR == ret || OB_ITER_END == ret) &&
retry_count < MAX_RETRY_COUNT) {
retry_count++;
ret = OB_SUCCESS;
}
}
}
if (nullptr != tmp_buf) {
large_buffer_pool_->reclaim(tmp_buf);
tmp_buf = nullptr;
}
}
return ret;
}
int ObCdcFetcher::calc_raw_read_size_(const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
const int64_t cur_idx,
const int64_t read_buf_len,
int64_t &read_size,
int64_t &target_idx)
{
int ret = OB_SUCCESS;
if (cur_idx >= miss_log_array.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cur_idx is greater than miss_log_array count", K(cur_idx), "count", miss_log_array.count());
} else {
// target_idx and read_size would be calculated here, the logs in [cur_idx, target_idx) would be read;
// To minimize read size of archive log, here is the principle:
// 1. The read size should be less than read_buf_len;
// 2. The mininal size of one read is MAX_LOG_BUFFER_SIZE to make sure the complete LogEntry has been read;
// 3. If gap between two adjacent missing_lsn is less than MAX_LOG_BUFFER_SIZE, try to read them in one read;
const palf::LSN &start_lsn = miss_log_array.at(cur_idx).miss_lsn_;
const int64_t arr_cnt = miss_log_array.count();
bool find_end = false;
read_size = MAX_LOG_BUFFER_SIZE;
for (target_idx = cur_idx + 1; !find_end && target_idx < arr_cnt && OB_SUCC(ret);) {
const palf::LSN curr_lsn = miss_log_array.at(target_idx).miss_lsn_;
if (curr_lsn <= start_lsn) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("curr_lsn is less equal than start_lsn, unexpected", K(curr_lsn), K(start_lsn), K(miss_log_array));
} else if (curr_lsn < start_lsn + read_size) {
const int64_t next_read_size = read_size + static_cast<int64_t>(curr_lsn - start_lsn);
if (next_read_size <= read_buf_len) {
// target_idx should be updated only when we can make sure the corresponding logentry could
// be read out of archive log;
read_size = max(read_size, next_read_size);
target_idx++;
} else {
find_end = true;
}
} else {
find_end = true;
}
}
LOG_TRACE("finish calc raw read size", K(cur_idx), K(read_buf_len), K(read_size), K(target_idx));
}
return ret;
}
int ObCdcFetcher::do_fetch_missing_log_(const obrpc::ObCdcLSFetchMissLogReq &req,
FetchRunTime &frt,
obrpc::ObCdcLSFetchLogResp &resp,
@ -894,7 +1137,6 @@ int ObCdcFetcher::do_fetch_missing_log_(const obrpc::ObCdcLSFetchMissLogReq &req
const int64_t end_tstamp = frt.rpc_deadline_ - RPC_QIT_RESERVED_TIME;
int64_t scan_round_count = 0; // epoch of fetching
int64_t fetched_log_count = 0; // count of log fetched
const FetchMode ctx_fetch_mode = ctx.get_fetch_mode();
int8_t req_flag = req.get_flag();
bool fetch_archive_only = is_sys_tenant(tenant_id_) ? false : ObCdcRpcTestFlag::is_fetch_archive_only(req_flag);
@ -921,80 +1163,53 @@ int ObCdcFetcher::do_fetch_missing_log_(const obrpc::ObCdcLSFetchMissLogReq &req
} else if (OB_FAIL(prepare_berfore_fetch_missing_(ls_id, ctx, palf_guard, ls_exist_in_palf, archive_is_on))) {
LOG_WARN("failed to prepare before fetching missing log", KR(ret), K(ls_id), K(tenant_id_));
} else {
for (int64_t idx = 0; OB_SUCC(ret) && ! frt.is_stopped() && idx < miss_log_array.count(); idx++) {
for (int64_t idx = 0; OB_SUCC(ret) && ! frt.is_stopped() && idx < miss_log_array.count(); ) {
// need_init_iter should always be true, declared here to ensure need init iter be true in each loop
PalfBufferIterator palf_iter;
ObRemoteLogpEntryIterator remote_iter(get_source_func, update_source_func);
const obrpc::ObCdcLSFetchMissLogReq::MissLogParam &miss_log_info = miss_log_array[idx];
const LSN &missing_lsn = miss_log_info.miss_lsn_;
LogEntry log_entry;
LSN lsn;
resp.set_next_miss_lsn(missing_lsn);
int64_t start_fetch_ts = ObTimeUtility::current_time();
bool log_fetched_in_palf = false;
bool log_fetched_in_archive = false;
if (is_time_up_(fetched_log_count, end_tstamp)) { // time up, stop fetching logs globally
frt.stop("TimeUP");
LOG_INFO("fetch log quit in time", K(end_tstamp), K(frt), K(fetched_log_count));
} else {
// first, try to fetch logs in palf
const int64_t start_idx = idx;
if (!fetch_archive_only && ls_exist_in_palf) {
if (OB_FAIL(fetch_log_in_palf_(ls_id, palf_iter,
missing_lsn, need_init_iter, replayable_point_scn,
log_entry, lsn))) {
if (OB_FAIL(fetch_missing_logs_in_palf_(ls_id, palf_guard, miss_log_array, idx, resp, frt))) {
if (OB_ERR_OUT_OF_LOWER_BOUND == ret) {
// block OB_ERR_OUT_OF_LOWER_BOUND
ret = OB_SUCCESS;
} else {
LOG_WARN("fetch missing log in palf failed", KR(ret), K(missing_lsn));
LOG_WARN("fetch missing log in palf failed", KR(ret), K(idx), K(miss_log_array));
}
} else {
log_fetched_in_palf = true;
if (FetchMode::FETCHMODE_ONLINE != ctx.get_fetch_mode()) {
// set fetch_mode to online to resize log_ext_handler threads
ctx.set_fetch_mode(FetchMode::FETCHMODE_ONLINE, "FetchMissInPalf");
}
}
}
// if no log fetched in palf, try to fetch log in archive
if (OB_SUCC(ret) && !log_fetched_in_palf && archive_is_on) {
if (OB_FAIL(fetch_log_in_archive_(ls_id, remote_iter, missing_lsn,
need_init_iter, log_entry, lsn, ctx))) {
LOG_WARN("fetch missng log in archive failed", KR(ret), K(missing_lsn));
} else {
log_fetched_in_archive = true;
if (OB_FAIL(fetch_missing_logs_in_archive_(ls_id, ctx, miss_log_array, idx, resp, frt))) {
LOG_WARN("fetch missng log in archive failed", KR(ret), K(idx),
"missing_log", miss_log_array.at(idx));
} else if (FetchMode::FETCHMODE_ARCHIVE != ctx.get_fetch_mode()) {
// set fetch_mode to archive to enable parallel fetch from archive
ctx.set_fetch_mode(FetchMode::FETCHMODE_ARCHIVE, "FetchMissInArchive");
}
}
if (OB_SUCC(ret) && (log_fetched_in_palf || log_fetched_in_archive)) {
if (OB_UNLIKELY(missing_lsn != lsn)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("do_fetch_missing_log missing_lsn not match", KR(ret), K(tenant_id_), K(ls_id),
K(missing_lsn), K(lsn));
} else {
resp.inc_log_fetch_time(ObTimeUtility::current_time() - start_fetch_ts);
check_next_entry_(lsn, log_entry, resp, frt);
if (frt.is_stopped()) {
// Stop fetching log
} else if (OB_FAIL(prefill_resp_with_log_entry_(ls_id, lsn, log_entry, resp))) {
if (OB_BUF_NOT_ENOUGH == ret) {
handle_when_buffer_full_(frt); // stop
ret = OB_SUCCESS;
} else {
LOG_WARN("prefill_resp_with_log_entry fail", KR(ret), K(frt), K(end_tstamp), K(resp));
}
} else {
// log fetched successfully
fetched_log_count++;
LOG_TRACE("LS fetch a missing log", K(tenant_id_), K(ls_id), K(fetched_log_count), K(frt));
}
}
} else if (! (log_fetched_in_palf || log_fetched_in_archive)) {
if (OB_SUCC(ret) && start_idx == idx) {
ret = OB_ERR_OUT_OF_LOWER_BOUND;
LOG_WARN("no log fetched from palf or archive, lower bound", K(log_fetched_in_palf),
K(log_fetched_in_archive), K(missing_lsn), K(idx));
} else {
// failed
LOG_WARN("no log fetched, log not exist in server", K(ls_id), K(idx), "missing_log",
miss_log_array.at(idx));
}
}
} // for
} // else
@ -1064,24 +1279,25 @@ int ObCdcFetcher::prepare_berfore_fetch_missing_(const ObLSID &ls_id,
{
int ret = OB_SUCCESS;
if (OB_SUCC(ret) && OB_FAIL(init_palf_handle_guard_(ls_id, palf_handle_guard))) {
if (OB_LS_NOT_EXIST != ret) {
LOG_WARN("ObLogService open_palf fail", KR(ret), K(tenant_id_), K(ls_id));
} else {
ret = OB_SUCCESS;
ls_exist_in_palf = false;
}
if (OB_LS_NOT_EXIST != ret) {
LOG_WARN("ObLogService open_palf fail", KR(ret), K(tenant_id_), K(ls_id));
} else {
ret = OB_SUCCESS;
ls_exist_in_palf = false;
}
}
if (OB_SUCC(ret) && OB_FAIL(host_->init_archive_source_if_needed(ls_id, ctx))) {
if (OB_ALREADY_IN_NOARCHIVE_MODE == ret) {
ret = OB_SUCCESS;
archive_is_on = false;
}
}
if (OB_SUCC(ret) && OB_UNLIKELY(!ls_exist_in_palf && !archive_is_on)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls not exist in palf and archive is not on, not able to fetch missing log", KR(ret), K(ls_id));
if (OB_SUCC(ret) && OB_FAIL(host_->init_archive_source_if_needed(ls_id, ctx))) {
if (OB_ALREADY_IN_NOARCHIVE_MODE == ret) {
ret = OB_SUCCESS;
archive_is_on = false;
}
}
if (OB_SUCC(ret) && OB_UNLIKELY(!ls_exist_in_palf && !archive_is_on)) {
ret = OB_ERR_OUT_OF_LOWER_BOUND;
LOG_WARN("ls not exist in palf and archive is not on, not able to fetch missing log", KR(ret), K(ls_id));
}
return ret;
}

View File

@ -136,6 +136,28 @@ private:
LogEntryType &log_group_entry,
LSN &lsn,
ClientLSCtx &ctx);
int fetch_missing_logs_in_palf_(const ObLSID &ls_id,
palf::PalfHandleGuard &palf_handle_guard,
const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
int64_t &cur_idx,
obrpc::ObCdcLSFetchLogResp &resp,
FetchRunTime &frt);
int fetch_missing_logs_in_archive_(const ObLSID &ls_id,
ClientLSCtx &ctx,
const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
int64_t &cur_idx,
obrpc::ObCdcLSFetchLogResp &resp,
FetchRunTime &frt);
int calc_raw_read_size_(const obrpc::ObCdcLSFetchMissLogReq::MissLogParamArray &miss_log_array,
const int64_t cur_idx,
const int64_t read_buf_len,
int64_t &read_size,
int64_t &target_idx);
int init_archive_source_(ClientLSCtx &ctx, ObLSID ls_id);
// Check whether has reached time limit
inline bool is_time_up_(const int64_t scan_round, const int64_t end_tstamp)
{

View File

@ -439,6 +439,9 @@ public:
void set_tenant_id(const uint64_t tenant_id) { tenant_id_ = tenant_id; }
uint64_t get_tenant_id() const { return tenant_id_; }
void set_progress(const int64_t progress) { progress_ = progress; }
int64_t get_progress() const { return progress_; }
void set_compressor_type(const common::ObCompressorType &compressor_type) { compressor_type_ = compressor_type; }
common::ObCompressorType get_compressor_type() const { return compressor_type_; }

View File

@ -448,9 +448,6 @@ int ObCdcService::get_or_create_client_ls_ctx(const obrpc::ObCdcRpcId &client_id
}
} else if (FetchLogProtocolType::RawLogDataProto == proto_type) {
ctx->set_proto_type(proto_type);
} else {
ret = OB_INVALID_ARGUMENT;
EXTLOG_LOG(WARN, "get invalid proto_type", K(proto_type), K(ls_id), K(client_id));
}
} else {
EXTLOG_LOG(ERROR, "get client ls ctx from ctx map failed", KR(ret));

View File

@ -32,6 +32,7 @@ ob_set_subtarget(obcdc_object_list common
ob_cdc_lob_aux_table_schema_info.cpp
ob_cdc_lob_aux_table_parse.cpp
ob_cdc_malloc_sample_info.cpp
ob_cdc_miss_log_handler.cpp
ob_cdc_server_endpoint_access_info.cpp
ob_cdc_tenant_endpoint_provider.cpp
ob_cdc_udt.cpp

View File

@ -36,7 +36,7 @@ TenantTransID::~TenantTransID()
void TenantTransID::reset()
{
tenant_id_ = common::OB_INVALID_TENANT_ID;
trans_id_ = -1;
trans_id_.reset();
}
int TenantTransID::compare(const TenantTransID &other) const
@ -88,7 +88,7 @@ PartTransID::~PartTransID()
void PartTransID::reset()
{
tls_id_.reset();
trans_id_ = -1;
trans_id_.reset();
}
bool PartTransID::operator==(const PartTransID &part_trans_id) const
@ -96,6 +96,15 @@ bool PartTransID::operator==(const PartTransID &part_trans_id) const
return trans_id_ == part_trans_id.trans_id_ && tls_id_ == part_trans_id.get_tls_id();
}
PartTransID &PartTransID::operator=(const PartTransID &other)
{
if (this != &other) {
this->tls_id_ = other.get_tls_id();
this->trans_id_ = other.get_tx_id();
}
return *this;
}
uint64_t PartTransID::hash() const
{
uint64_t hash_val = 0;

View File

@ -52,9 +52,11 @@ struct PartTransID
// transaction::ObTransID(int64_t)
transaction::ObTransID trans_id_;
PartTransID() : tls_id_(), trans_id_() {}
PartTransID(const logservice::TenantLSID &tls_id, const transaction::ObTransID &trans_id);
~PartTransID();
bool operator==(const PartTransID &part_trans_id) const;
PartTransID &operator=(const PartTransID &other);
uint64_t hash() const;
int hash(uint64_t &hash_val) const;
void reset();

View File

@ -0,0 +1,829 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* OBCDC header file
* This file defines interface of OBCDC
*/
#define USING_LOG_PREFIX OBLOG_FETCHER
#include "logservice/cdcservice/ob_cdc_req.h"
#include "ob_cdc_miss_log_handler.h"
#include "ob_log_config.h"
#include "ob_log_utils.h"
namespace oceanbase
{
namespace libobcdc
{
int64_t ObCDCMissLogHandler::g_rpc_timeout = TCONF.fetch_log_rpc_timeout_sec * _SEC_;
const int64_t ObCDCMissLogHandler::RETRY_LOG_PRINT_INTERVAL = 30 * _SEC_;
const int64_t ObCDCMissLogHandler::RETRY_TIMEOUT = 1 * _HOUR_;
const int64_t ObCDCMissLogHandler::MAX_RPC_TIMEOUT = 5 * _MIN_;
MissLogTask::MissLogTask(
const common::ObAddr &orig_svr,
IObLogRpc &rpc,
LSFetchCtx &ls_fetch_ctx,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi)
: svr_(), rpc_(rpc), ls_fetch_ctx_(ls_fetch_ctx), missing_info_(missing_info), need_change_server_(false), tsi_(tsi)
{
svr_ = orig_svr;
}
void MissLogTask::reset()
{
// only server is init by MissLogTask
svr_.reset();
need_change_server_ = false;
}
int MissLogTask::try_change_server(const int64_t timeout, volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
const int64_t end_time = get_timestamp() + timeout;
if (OB_UNLIKELY(need_change_server_)) {
common::ObAddr req_svr;
int64_t retry_times = 0;
bool is_timeout = false;
const static int64_t RETRY_LOG_PRINT_INTERVAL = 30 * _SEC_;
do {
if (OB_FAIL(ls_fetch_ctx_.next_server(req_svr))) {
LOG_WARN("get next_server failed", KR(ret), K_(ls_fetch_ctx));
} else if (OB_UNLIKELY(!req_svr.is_valid())) {
ret = OB_NEED_RETRY;
LOG_WARN("next_server is not valid, need_retry", KR(ret), K(req_svr), K_(ls_fetch_ctx));
} else {
_LOG_INFO("[MISS_LOG][CHANGE_SVR][FROM=%s][TO=%s][PART_TRANS_ID=%s]",
to_cstring(svr_), to_cstring(req_svr), to_cstring(get_part_trans_id()));
svr_ = req_svr;
need_change_server_ = false;
}
if (OB_FAIL(ret)) {
is_timeout = get_timestamp() >= end_time;
if (is_timeout) {
LOG_ERROR("[MISS_LOG][NEXT_SERVER]RETRY_GET_NEXT_SERVER TIMEOUT", KR(ret), K(timeout), KPC(this));
ret = OB_TIMEOUT;
} else {
++retry_times;
if (TC_REACH_TIME_INTERVAL(RETRY_LOG_PRINT_INTERVAL)) {
LOG_WARN("[MISS_LOG][NEXT_SERVER]RETRY_GET_NEXT_SERVER",
"tls_id", ls_fetch_ctx_.get_tls_id(), K(retry_times),
"end_time", TS_TO_STR(end_time));
}
ob_usleep(10 * _MSEC_);
}
}
} while (OB_FAIL(ret) && ! is_timeout && ! stop_flag);
if (OB_UNLIKELY(stop_flag)) {
ret = OB_IN_STOP_STATE;
LOG_WARN("handle miss log task stop", KR(ret), KPC(this));
}
}
return ret;
}
ObCDCMissLogHandler& ObCDCMissLogHandler::get_instance()
{
static ObCDCMissLogHandler instance;
return instance;
}
void ObCDCMissLogHandler::configure(const ObLogConfig &config)
{
g_rpc_timeout = config.fetch_log_rpc_timeout_sec * _SEC_;
LOG_INFO("[MISS_LOG][CONFIG]", "rpc_timeout", TVAL_TO_STR(g_rpc_timeout));
}
// TODO add metrics
int ObCDCMissLogHandler::handle_log_miss(
const common::ObAddr &cur_svr,
IObLogRpc *rpc,
LSFetchCtx &ls_fetch_ctx,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(rpc)
|| OB_UNLIKELY(missing_info.is_empty()
|| ! cur_svr.is_valid()
|| missing_info.get_part_trans_id().get_tls_id() != ls_fetch_ctx.get_tls_id())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("invalid args to handle miss log", KR(ret), K(cur_svr), K(missing_info), K(rpc));
} else {
MissLogTask misslog_task(cur_svr, *rpc, ls_fetch_ctx, missing_info, tsi);
if (OB_FAIL(handle_miss_log_task_(misslog_task, stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
int debug_err = ret;
// overwrite ret
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("handle_miss_log_task_ failed", KR(ret), KR(debug_err), K(misslog_task), K(stop_flag));
}
}
}
return ret;
}
int ObCDCMissLogHandler::handle_miss_log_task_(MissLogTask &misslog_task, volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
ObTraceIdGuard guard(*common::ObCurTraceId::get_trace_id());
const int64_t start_ts = get_timestamp();
misslog_task.missing_info_.set_resolving_miss_log();
FetchLogSRpc *fetch_log_srpc = nullptr;
if (OB_FAIL(alloc_fetch_log_srpc_(misslog_task.get_tenant_id(), fetch_log_srpc))) {
LOG_ERROR("alloc fetch_log_srpc fail", KR(ret));
} else if (OB_ISNULL(fetch_log_srpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid fetch_log_srpc", KR(ret), K(misslog_task));
} else if (OB_FAIL(handle_miss_record_or_state_log_(
misslog_task,
*fetch_log_srpc,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("handle_miss_record_or_state_log_ failed", KR(ret), K(misslog_task));
}
} else if (OB_FAIL(handle_miss_redo_log_(
misslog_task,
*fetch_log_srpc,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("handle_miss_redo_log_ failed", KR(ret), K(misslog_task));
}
}
if (stop_flag) {
ret = OB_IN_STOP_STATE;
} else {
const int64_t cost_time = get_timestamp() - start_ts;
LOG_INFO("[MISS_LOG][HANDLE_DONE]", KR(ret),
K(cost_time), "cost_time", TVAL_TO_STR(cost_time),
K(misslog_task));
}
if (OB_NOT_NULL(fetch_log_srpc)) {
free_fetch_log_srpc_(fetch_log_srpc);
fetch_log_srpc = nullptr;
}
return ret;
}
int ObCDCMissLogHandler::handle_miss_record_or_state_log_(
MissLogTask &misslog_task,
FetchLogSRpc &fetch_log_srpc,
volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
if (misslog_task.missing_info_.has_miss_record_or_state_log()) {
int64_t miss_record_cnt = 1;
ObArrayImpl<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> batched_misslog_lsn_arr;
palf::LSN misslog_lsn;
const int64_t start_ts = get_timestamp();
LOG_INFO("[MISS_LOG][FETCH_RECORD_OR_STATE][BEGIN]", KR(ret), "part_trans_id", misslog_task.get_part_trans_id());
while (OB_SUCC(ret) && ! stop_flag && misslog_task.missing_info_.has_miss_record_or_state_log()) {
misslog_lsn.reset();
batched_misslog_lsn_arr.reset();
ObCdcLSFetchMissLogReq::MissLogParam param;
if (OB_FAIL(misslog_task.missing_info_.get_miss_record_or_state_log_lsn(misslog_lsn))) {
LOG_ERROR("get_miss_record_or_state_log_lsn failed", KR(ret), K(misslog_task), K(misslog_lsn));
} else {
param.miss_lsn_ = misslog_lsn;
if (OB_FAIL(batched_misslog_lsn_arr.push_back(param))) {
LOG_ERROR("push_back miss_record_or_state_log_lsn into batched_misslog_lsn_arr failed", KR(ret), K(param));
} else if (OB_FAIL(fetch_miss_log_with_retry_(
misslog_task,
batched_misslog_lsn_arr,
fetch_log_srpc,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("fetch_miss_log_with_retry_ failed", KR(ret), K(batched_misslog_lsn_arr), K(misslog_task));
}
} else {
const obrpc::ObRpcResultCode &rcode = fetch_log_srpc.get_result_code();
const obrpc::ObCdcLSFetchLogResp &resp = fetch_log_srpc.get_resp();
if (OB_FAIL(rcode.rcode_) || OB_FAIL(resp.get_err())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetch log fail on rpc", KR(ret), K(rcode), K(resp), K(batched_misslog_lsn_arr));
} else if (resp.get_log_num() < 1) {
LOG_INFO("fetch_miss_log_rpc doesn't fetch log, retry", K(misslog_lsn));
} else if (OB_UNLIKELY(resp.get_log_num() > 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("expect only one misslog while fetching miss_record_or_state_log", K(resp));
} else if (OB_UNLIKELY(resp.get_next_miss_lsn() != misslog_lsn)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetched log not match miss_log_lsn", KR(ret), K(misslog_lsn), K(resp));
} else {
misslog_task.missing_info_.reset_miss_record_or_state_log_lsn();
palf::LogEntry miss_log_entry;
miss_log_entry.reset();
const char *buf = resp.get_log_entry_buf();
const int64_t len = resp.get_pos();
int64_t pos = 0;
if (OB_FAIL(miss_log_entry.deserialize(buf, len, pos))) {
LOG_ERROR("deserialize log_entry of miss_record_or_state_log failed", KR(ret), K(misslog_lsn), KP(buf), K(len), K(pos));
} else if (OB_FAIL(misslog_task.ls_fetch_ctx_.read_miss_tx_log(miss_log_entry, misslog_lsn, misslog_task.tsi_, misslog_task.missing_info_))) {
if (OB_ITEM_NOT_SETTED == ret) {
miss_record_cnt ++;
ret = OB_SUCCESS;
LOG_INFO("[MISS_LOG][FETCH_RECORD_OR_STATE] found new miss_record_or_state_log",
"tls_id", misslog_task.ls_fetch_ctx_.get_tls_id(), K(misslog_lsn), K(miss_record_cnt), K(misslog_task));
} else {
LOG_ERROR("[MISS_LOG][FETCH_RECORD_OR_STATE] read miss_log failed", KR(ret), K(miss_log_entry), K(misslog_lsn), K(misslog_task));
}
}
}
}
}
}
if (stop_flag) {
ret = OB_IN_STOP_STATE;
}
if (OB_SUCC(ret)) {
LOG_INFO("[MISS_LOG][FETCH_RECORD_OR_STATE][END]",
"part_trans_id", misslog_task.missing_info_.get_part_trans_id(),
K(miss_record_cnt),
"miss_redo_cnt", misslog_task.missing_info_.get_miss_redo_lsn_arr().count(),
"cost", get_timestamp() - start_ts);
}
}
return ret;
}
int ObCDCMissLogHandler::handle_miss_redo_log_(
MissLogTask &misslog_task,
FetchLogSRpc &fetch_log_srpc,
volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
if (OB_FAIL(misslog_task.missing_info_.sort_and_unique_missing_log_lsn())) {
LOG_ERROR("sort_and_unique_missing_log_lsn failed", KR(ret), K(misslog_task));
} else {
const int64_t total_misslog_cnt = misslog_task.missing_info_.get_total_misslog_cnt();
int64_t fetched_missing_log_cnt = 0;
ObArrayImpl<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> batched_misslog_lsn_arr;
const int64_t start_ts = get_timestamp();
LOG_INFO("[MISS_LOG][FETCH_REDO][BEGIN]", KR(ret),
"part_trans_id", misslog_task.get_part_trans_id(), K(total_misslog_cnt));
while (OB_SUCC(ret) && ! stop_flag && fetched_missing_log_cnt < total_misslog_cnt) {
batched_misslog_lsn_arr.reset();
if (OB_FAIL(build_batch_misslog_lsn_arr_(
fetched_missing_log_cnt,
misslog_task.missing_info_,
batched_misslog_lsn_arr))) {
LOG_ERROR("build_batch_misslog_lsn_arr_ failed", KR(ret),
K(fetched_missing_log_cnt), K(misslog_task));
} else if (OB_FAIL(fetch_miss_log_with_retry_(
misslog_task,
batched_misslog_lsn_arr,
fetch_log_srpc,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("fetch_miss_log_with_retry_ failed", KR(ret), K(batched_misslog_lsn_arr), K(misslog_task));
}
} else {
const obrpc::ObRpcResultCode &rcode = fetch_log_srpc.get_result_code();
const obrpc::ObCdcLSFetchLogResp &resp = fetch_log_srpc.get_resp();
if (OB_FAIL(rcode.rcode_) || OB_FAIL(resp.get_err())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetch log fail on rpc", KR(ret), K(rcode), K(resp), K(batched_misslog_lsn_arr));
} else {
// check next_miss_lsn
bool is_next_miss_lsn_match = false;
palf::LSN next_miss_lsn = resp.get_next_miss_lsn();
const int64_t batch_cnt = batched_misslog_lsn_arr.count();
const int64_t resp_log_cnt = resp.get_log_num();
if (batch_cnt == resp_log_cnt) {
is_next_miss_lsn_match = (batched_misslog_lsn_arr.at(batch_cnt - 1).miss_lsn_ == next_miss_lsn);
} else if (batch_cnt > resp_log_cnt) {
is_next_miss_lsn_match = (batched_misslog_lsn_arr.at(resp_log_cnt).miss_lsn_ == next_miss_lsn);
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("too many misslog fetched", KR(ret), K(next_miss_lsn), K(batch_cnt),
K(resp_log_cnt),K(resp), K(misslog_task));
}
if (OB_SUCC(ret)) {
if (!is_next_miss_lsn_match) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("misslog fetched is not match batched_misslog_lsn_arr requested", KR(ret),
K(next_miss_lsn), K(batch_cnt), K(resp_log_cnt), K(batched_misslog_lsn_arr), K(resp), K(misslog_task));
} else if (OB_FAIL(read_batch_misslog_(
misslog_task.ls_fetch_ctx_,
resp,
fetched_missing_log_cnt,
misslog_task.tsi_,
misslog_task.missing_info_))) {
// expected no misslog found while resolving normal log.
LOG_ERROR("read_batch_misslog failed", KR(ret), K(fetched_missing_log_cnt), K(misslog_task));
}
}
}
}
}
LOG_INFO("[MISS_LOG][FETCH_REDO][END]", KR(ret),
"part_trans_id", misslog_task.missing_info_.get_part_trans_id(),
K(total_misslog_cnt),
"cost", get_timestamp() - start_ts);
}
return ret;
}
// split all miss_logs by batch
int ObCDCMissLogHandler::build_batch_misslog_lsn_arr_(
const int64_t fetched_log_idx,
IObCDCPartTransResolver::MissingLogInfo &missing_log_info,
ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &batched_misslog_lsn_arr)
{
int ret = OB_SUCCESS;
int64_t batched_cnt = 0;
static int64_t MAX_MISSLOG_CNT_PER_RPC= 1024;
const ObLogLSNArray &miss_redo_or_state_log_arr = missing_log_info.get_miss_redo_lsn_arr();
const int64_t miss_log_cnt = miss_redo_or_state_log_arr.count();
if (OB_UNLIKELY(0 < batched_misslog_lsn_arr.count())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("invalid batched_misslog_lsn_arr", KR(ret), K(batched_misslog_lsn_arr));
} else {
batched_misslog_lsn_arr.reset();
// fetched_log_idx is log_count that already fetched after last batch rpc
// for miss_redo_or_state_log_arr with 100 miss_log and MAX_MISSLOG_CNT_PER_RPC = 10
// fetched_log_idx start from 0, if fetched 8 miss_log in one rpc, then fetched_log_idx is 8,
// and for next batch, miss_redo_or_state_log_arr.at(8) is the 9th miss_log as expected.
for (int idx = fetched_log_idx; OB_SUCC(ret) && batched_cnt < MAX_MISSLOG_CNT_PER_RPC && idx < miss_log_cnt; idx++) {
const palf::LSN &lsn = miss_redo_or_state_log_arr.at(idx);
ObCdcLSFetchMissLogReq::MissLogParam param;
param.miss_lsn_ = lsn;
if (OB_FAIL(batched_misslog_lsn_arr.push_back(param))) {
LOG_ERROR("push_back missing_log lsn into batched_misslog_lsn_arr failed", KR(ret), K(idx),
K(fetched_log_idx), K(miss_redo_or_state_log_arr), K(batched_misslog_lsn_arr), K(param));
} else {
batched_cnt++;
}
}
}
_LOG_INFO("[MISS_LOG][BATCH_MISSLOG][PART_TRANS_ID=%s][PROGRESS=%ld/%ld][BATCH_SIZE=%ld]",
to_cstring(missing_log_info.get_part_trans_id()),
fetched_log_idx,
miss_log_cnt,
batched_misslog_lsn_arr.count());
return ret;
}
// read batched misslog
int ObCDCMissLogHandler::read_batch_misslog_(
LSFetchCtx &ls_fetch_ctx,
const obrpc::ObCdcLSFetchLogResp &resp,
int64_t &fetched_missing_log_cnt,
logfetcher::TransStatInfo &tsi,
IObCDCPartTransResolver::MissingLogInfo &missing_info)
{
int ret = OB_SUCCESS;
LOG_INFO("read_batch_misslog_ begin", "tls_id", ls_fetch_ctx.get_tls_id(), K(resp), K(fetched_missing_log_cnt));
const int64_t total_misslog_cnt = missing_info.get_total_misslog_cnt();
const char *buf = resp.get_log_entry_buf();
const int64_t len = resp.get_pos();
int64_t pos = 0;
const int64_t log_cnt = resp.get_log_num();
const ObLogLSNArray &org_misslog_arr = missing_info.get_miss_redo_lsn_arr();
int64_t start_ts = get_timestamp();
if (OB_UNLIKELY(log_cnt <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("expected valid log count from FetchLogSRpc for misslog", KR(ret), K(resp));
} else {
for (int64_t idx = 0; OB_SUCC(ret) && idx < log_cnt; idx++) {
if (fetched_missing_log_cnt >= total_misslog_cnt) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetched_missing_log_cnt is more than total_misslog_cnt", KR(ret),
K(fetched_missing_log_cnt), K(missing_info), K(idx), K(resp));
} else {
palf::LSN misslog_lsn;
palf::LogEntry miss_log_entry;
misslog_lsn.reset();
miss_log_entry.reset();
IObCDCPartTransResolver::MissingLogInfo tmp_miss_info;
tmp_miss_info.set_resolving_miss_log();
if (org_misslog_arr.count() == fetched_missing_log_cnt) {
// TODO check it!
// already consume the all miss_redo_log, but still exist one miss_record_log.
// lsn record_log is the last miss_log to fetch.
if (OB_FAIL(missing_info.get_miss_record_or_state_log_lsn(misslog_lsn))) {
if (OB_ENTRY_NOT_EXIST == ret) {
LOG_ERROR("expect valid miss-record_log_lsn", KR(ret), K(missing_info), K(fetched_missing_log_cnt), K(ls_fetch_ctx));
} else {
LOG_ERROR("get_miss_record_or_state_log_lsn failed", KR(ret), K(missing_info), K(fetched_missing_log_cnt), K(ls_fetch_ctx));
}
}
} else if (OB_FAIL(org_misslog_arr.at(fetched_missing_log_cnt, misslog_lsn))) {
LOG_ERROR("get misslog_lsn fail", KR(ret), K(fetched_missing_log_cnt),
K(idx), K(org_misslog_arr), K(resp));
}
if (FAILEDx(miss_log_entry.deserialize(buf, len, pos))) {
LOG_ERROR("deserialize miss_log_entry fail", KR(ret), K(len), K(pos));
} else if (OB_FAIL(ls_fetch_ctx.read_miss_tx_log(miss_log_entry, misslog_lsn, tsi, tmp_miss_info))) {
LOG_ERROR("read_miss_log fail", KR(ret), K(miss_log_entry),
K(misslog_lsn), K(fetched_missing_log_cnt), K(idx), K(tmp_miss_info));
} else {
fetched_missing_log_cnt++;
// update last misslog submit ts to orig missing_info
missing_info.set_last_misslog_progress(tmp_miss_info.get_last_misslog_progress());
}
}
}
}
int64_t read_batch_missing_cost = get_timestamp() - start_ts;
const int64_t handle_miss_progress = missing_info.get_last_misslog_progress();
_LOG_INFO("[MISS_LOG][READ_MISSLOG][PART_TRANS_ID=%s][COST=%ld][PROGRESS_CNT=%ld/%ld][PROGRESS_SCN=%ld(%s)]",
to_cstring(missing_info.get_part_trans_id()),
read_batch_missing_cost,
fetched_missing_log_cnt,
org_misslog_arr.count(),
handle_miss_progress,
NTS_TO_STR(handle_miss_progress));
return ret;
}
int ObCDCMissLogHandler::alloc_fetch_log_srpc_(const uint64_t tenant_id, FetchLogSRpc *&fetch_log_srpc)
{
int ret = OB_SUCCESS;
void *buf = ob_cdc_malloc(sizeof(FetchLogSRpc), ObModIds::OB_LOG_FETCH_LOG_SRPC, tenant_id);
if (OB_ISNULL(buf)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc memory for FetchLogSRpc fail", KR(ret), K(tenant_id), K(sizeof(FetchLogSRpc)));
} else if (OB_ISNULL(fetch_log_srpc = new(buf) FetchLogSRpc())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("construct fetch miss log srpc fail", KR(ret), K(tenant_id), K(buf));
} else {
// success
}
return ret;
}
void ObCDCMissLogHandler::free_fetch_log_srpc_(FetchLogSRpc *fetch_log_srpc)
{
if (OB_NOT_NULL(fetch_log_srpc)) {
fetch_log_srpc->~FetchLogSRpc();
ob_cdc_free(fetch_log_srpc);
fetch_log_srpc = nullptr;
}
}
int ObCDCMissLogHandler::fetch_miss_log_with_retry_(
MissLogTask &misslog_task,
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
FetchLogSRpc &fetch_srpc,
volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
int64_t fetch_log_timeout = g_rpc_timeout; // default fetch_log_timeoout to user_config;
const int64_t start_ts = get_timestamp();
const int64_t end_ts = start_ts + RETRY_TIMEOUT;
int64_t cur_ts = start_ts;
int64_t try_cnt = 0;
bool rpc_succ = false;
bool rpc_fetch_no_log = false;
const bool test_mode_misslog_errsim = (1 == TCONF.test_mode_on && 1 == TCONF.test_fetch_missing_errsim);
int test_mode_fail_max_cnt = test_mode_misslog_errsim ? 2 : 0;
int test_mode_fail_cnt = 0;
while (! stop_flag && ! rpc_succ && cur_ts < end_ts) {
bool has_valid_feedback = false;
try_cnt ++;
if (OB_FAIL(fetch_miss_log_(
miss_log_array,
fetch_log_timeout,
misslog_task,
fetch_srpc,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("fetch_miss_log failed", KR(ret), K(misslog_task));
}
} else {
const obrpc::ObRpcResultCode &rcode = fetch_srpc.get_result_code();
const obrpc::ObCdcLSFetchLogResp &resp = fetch_srpc.get_resp();
if (OB_SUCCESS != rcode.rcode_ || OB_SUCCESS != resp.get_err()) {
if (OB_TIMEOUT == rcode.rcode_ || OB_TIMEOUT == resp.get_err() || OB_TIMEOUT == resp.get_debug_err()) {
ret = OB_TIMEOUT;
} else {
if (OB_SUCCESS != rcode.rcode_) {
ret = rcode.rcode_;
} else if (OB_SUCCESS != resp.get_err()) {
ret = resp.get_err();
} else if (OB_SUCCESS != resp.get_debug_err()) {
ret = resp.get_debug_err();
}
if (OB_ERR_OUT_OF_LOWER_BOUND == ret) {
rpc_fetch_no_log = true;
// exit if fatal error
LOG_ERROR("[MISS_LOG] fetch log fail on rpc, log has been recycled", KR(ret), K(rpc_fetch_no_log), K(rcode), K(resp), K(misslog_task));
} else {
LOG_WARN("[MISS_LOG] fetch log fail on rpc", KR(ret), K(rpc_fetch_no_log), K(rcode), K(resp), K(misslog_task));
}
}
} else if (OB_UNLIKELY(test_mode_fail_cnt++ < test_mode_fail_max_cnt)) {
ret = OB_TIMEOUT;
LOG_INFO("mock fetch log fail in test mode", KR(ret), K(test_mode_fail_cnt), K(test_mode_fail_max_cnt), "end_time", TS_TO_STR(end_ts));
} else if (FALSE_IT(check_feedback_(resp, has_valid_feedback, rpc_fetch_no_log))) {
} else if (OB_UNLIKELY(resp.get_log_num() <= 0)) {
LOG_WARN("[MISS_LOG][EMPTY_RPC_RESPONSE], need retry", K(resp), K(misslog_task));
} else {
rpc_succ = true;
}
if (OB_TIMEOUT == ret) {
// allow adjust max_rpc_timeout by change config in libobcdc.conf while fetch_log_timeout * 2 is larger than MAX_RPC_TIMEOUT
const int64_t max_rpc_timeout = std::max(MAX_RPC_TIMEOUT, g_rpc_timeout);
const int64_t new_fetch_log_timeout = std::min(max_rpc_timeout, fetch_log_timeout * 2);
_LOG_INFO("[MISS_LOG][FETCH_TIMEOUT][ADJUST_FETCH_MISSLOG_TIMEOUT][FROM=%s][TO=%s][rpc_rcode=%s][rpc_response=%s]",
TVAL_TO_STR(fetch_log_timeout), TVAL_TO_STR(new_fetch_log_timeout), to_cstring(rcode), to_cstring(resp));
fetch_log_timeout = new_fetch_log_timeout;
}
}
if (OB_UNLIKELY(! rpc_succ || has_valid_feedback)) {
const uint64_t cluster_version = GET_MIN_CLUSTER_VERSION();
// cdcservice not support fetch_miss_log on server didn't fetch_log(normal mode) for the ls(in current cdc progress) before.
misslog_task.need_change_server_ = (cluster_version >= MOCK_CLUSTER_VERSION_4_2_1_7 && cluster_version < CLUSTER_VERSION_4_2_2_0)
|| (cluster_version >= MOCK_CLUSTER_VERSION_4_2_5_0 && cluster_version < CLUSTER_VERSION_4_3_0_0)
|| cluster_version >= CLUSTER_VERSION_4_3_4_0;
if (rpc_fetch_no_log) {
LOG_WARN("[MISS_LOG][FETCH_NO_LOG]", KR(ret), K(misslog_task), "rpc_response", fetch_srpc.get_resp(), "rpc_request", fetch_srpc.get_req());
}
}
cur_ts = get_timestamp();
} // end while
if (stop_flag) {
ret = OB_IN_STOP_STATE;
}
_LOG_INFO("[MISS_LOG][FETCH_MISSLOG][ret=%d][PART_TRANS_ID=%s][COST=%ld][LOG_CNT=%ld][FETCH_STATUS=%s][FETCH_ROUND=%ld]", ret,
to_cstring(misslog_task.get_part_trans_id()),
cur_ts - start_ts,
miss_log_array.count(),
to_cstring(fetch_srpc.get_resp().get_fetch_status()),
try_cnt);
return ret;
}
int ObCDCMissLogHandler::fetch_miss_log_(
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
MissLogTask &misslog_task,
FetchLogSRpc &fetch_srpc,
volatile bool &stop_flag)
{
int ret = OB_SUCCESS;
const ClientFetchingMode fetching_mode = misslog_task.ls_fetch_ctx_.get_fetching_mode();
const int64_t last_handle_progress = misslog_task.get_handle_progress();
LSFetchCtx &ls_fetch_ctx = misslog_task.ls_fetch_ctx_;
if (OB_UNLIKELY(! is_fetching_mode_valid(fetching_mode))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetching mode is not valid", KR(ret), K(fetching_mode));
} else if (is_integrated_fetching_mode(fetching_mode)) {
if (OB_FAIL(misslog_task.try_change_server(timeout, stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("get fetch_miss_log server failed", KR(ret), K(ls_fetch_ctx));
}
} else if (OB_FAIL(fetch_srpc.fetch_log(
misslog_task.rpc_,
ls_fetch_ctx.get_tls_id().get_tenant_id(),
ls_fetch_ctx.get_tls_id().get_ls_id(),
miss_log_array,
misslog_task.svr_,
timeout,
last_handle_progress))) {
LOG_ERROR("fetch_misslog_rpc exec failed", KR(ret), K(miss_log_array), K(last_handle_progress), K(misslog_task));
ret = OB_NEED_RETRY;
} else {
// succ
}
} else if (is_direct_fetching_mode(fetching_mode)) {
// mock FetchLogSRpc here
if (OB_FAIL(fetch_miss_log_direct_(miss_log_array, timeout, fetch_srpc, ls_fetch_ctx))) {
LOG_ERROR("fetch missing log direct failed", KR(ret), K(miss_log_array), K(misslog_task));
// rewrite ret code to make sure that cdc wouldn't exit because fetch_missing_log_direct_ failed.
ret = OB_NEED_RETRY;
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid fetching mode", KR(ret), K(fetching_mode), K(misslog_task));
}
return ret;
}
int ObCDCMissLogHandler::fetch_miss_log_direct_(
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
FetchLogSRpc &fetch_log_srpc,
LSFetchCtx &ls_fetch_ctx)
{
int ret = OB_SUCCESS;
ObCdcLSFetchLogResp *resp = NULL;
LSFetchCtxGetSourceFunctor get_source_func(ls_fetch_ctx);
LSFetchCtxUpdateSourceFunctor update_source_func(ls_fetch_ctx);
const int64_t current_progress = ls_fetch_ctx.get_progress();
const int64_t tenant_id = ls_fetch_ctx.get_tls_id().get_tenant_id();
const ObLSID &ls_id = ls_fetch_ctx.get_tls_id().get_ls_id();
archive::LargeBufferPool *buffer_pool = NULL;
logservice::ObLogExternalStorageHandler *log_ext_handler = NULL;
ObRpcResultCode rcode;
SCN cur_scn;
const int64_t start_fetch_ts = get_timestamp();
const int64_t time_upper_limit = start_fetch_ts + timeout;
bool stop_fetch = false;
bool is_timeout = false;
if (OB_ISNULL(resp = OB_NEW(ObCdcLSFetchLogResp, "FetchMissResp"))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc ObCdcLSFetchLogResp failed", KR(ret));
} else if (OB_FAIL(cur_scn.convert_from_ts(current_progress/1000L))) {
LOG_ERROR("convert log progress to scn failed", KR(ret), K(current_progress));
} else if (OB_FAIL(ls_fetch_ctx.get_large_buffer_pool(buffer_pool))) {
LOG_ERROR("get large buffer pool when fetching missing log failed", KR(ret), K(ls_fetch_ctx));
} else if (OB_FAIL(ls_fetch_ctx.get_log_ext_handler(log_ext_handler))) {
LOG_ERROR("get log ext handler when fetching missing log failed", KR(ret), K(ls_fetch_ctx));
} else {
int64_t fetched_cnt = 0;
const int64_t arr_cnt = miss_log_array.count();
resp->set_next_miss_lsn(miss_log_array.at(0).miss_lsn_);
while (OB_SUCC(ret) && !stop_fetch) {
bool retry_on_err = false;
while (OB_SUCC(ret) && fetched_cnt < arr_cnt && !is_timeout) {
const int64_t start_fetch_entry_ts = get_timestamp();
const ObCdcLSFetchMissLogReq::MissLogParam &param = miss_log_array.at(fetched_cnt);
const LSN &missing_lsn = param.miss_lsn_;
const char *buf;
int64_t buf_size = 0;
palf::LogEntry log_entry;
palf::LSN lsn;
logservice::ObRemoteLogpEntryIterator entry_iter(get_source_func, update_source_func);
resp->set_next_miss_lsn(missing_lsn);
if (get_timestamp() > time_upper_limit) {
is_timeout = true;
} else if (OB_FAIL(entry_iter.init(tenant_id, ls_id, cur_scn, missing_lsn,
LSN(palf::LOG_MAX_LSN_VAL), buffer_pool, log_ext_handler))) {
LOG_WARN("remote entry iter init failed", KR(ret));
} else if (OB_FAIL(entry_iter.next(log_entry, lsn, buf, buf_size))) {
retry_on_err =true;
LOG_WARN("log entry iter failed to iterate", KR(ret));
} else {
if (OB_UNLIKELY(missing_lsn != lsn)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("direct fetch missing_lsn not match", KR(ret), K(tenant_id), K(ls_id),
K(missing_lsn), K(lsn));
} else {
const int64_t entry_size = log_entry.get_serialize_size();
int64_t pos = 0;
resp->inc_log_fetch_time(get_timestamp() - start_fetch_entry_ts);
if (! resp->has_enough_buffer(entry_size)) {
ret = OB_BUF_NOT_ENOUGH;
} else {
int64_t remain_size = 0;
char *remain_buf = resp->get_remain_buf(remain_size);
if (OB_ISNULL(remain_buf)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("remain buffer is null", KR(ret));
}
else if (OB_FAIL(log_entry.serialize(remain_buf, remain_size, pos))) {
LOG_WARN("missing LogEntry serialize failed", KR(ret), K(remain_size),
K(pos), K(missing_lsn), K(log_entry));
}
else {
// TODO: there is an issue in entry_iter.next(), the returned buffer is not as expected
// MEMCPY(remain_buf, buf, buf_size);
resp->log_entry_filled(entry_size);
fetched_cnt++;
}
}
}
}
}// while
if (OB_SUCC(ret)) {
stop_fetch = true;
} else if (OB_BUF_NOT_ENOUGH == ret) {
stop_fetch = true;
ret = OB_SUCCESS;
} else if (retry_on_err) {
ret = OB_SUCCESS;
}
} // while
resp->set_l2s_net_time(0);
resp->set_svr_queue_time(0);
resp->set_process_time(get_timestamp() - start_fetch_ts);
}
// regard resp not null as sending rpc successfully
if (OB_NOT_NULL(resp)) {
resp->set_debug_err(ret);
if (OB_FAIL(ret)) {
resp->set_err(OB_ERR_SYS);
} else {
resp->set_err(OB_SUCCESS);
}
ret = OB_SUCCESS;
rcode.rcode_ = OB_SUCCESS;
} else {
rcode.rcode_ = ret;
sprintf(rcode.msg_, "failed to allocate fetchlogresp");
}
fetch_log_srpc.set_resp(rcode, resp);
if (OB_NOT_NULL(resp)) {
OB_DELETE(ObCdcLSFetchLogResp, "FetchMissResp", resp);
resp = NULL;
}
return ret;
}
void ObCDCMissLogHandler::check_feedback_(
const obrpc::ObCdcLSFetchLogResp &resp,
bool &has_valid_feedback,
bool &rpc_fetch_no_log)
{
const obrpc::FeedbackType& feedback = resp.get_feedback_type();
has_valid_feedback = (feedback != obrpc::FeedbackType::INVALID_FEEDBACK);
rpc_fetch_no_log = (obrpc::FeedbackType::LOG_NOT_IN_THIS_SERVER == feedback);
if (has_valid_feedback) {
const char *rpc_feedback_info = nullptr;
switch (feedback) {
case obrpc::FeedbackType::LAGGED_FOLLOWER:
rpc_feedback_info = "fetching log on lagged follower";
break;
case obrpc::FeedbackType::LOG_NOT_IN_THIS_SERVER:
rpc_feedback_info = "log not in this server, may be recycled";
break;
case obrpc::FeedbackType::LS_OFFLINED:
rpc_feedback_info = "fetching log on offline logstream";
break;
case obrpc::FeedbackType::ARCHIVE_ITER_END_BUT_LS_NOT_EXIST_IN_PALF:
rpc_feedback_info = "archive log iter end but no valid palf on this server";
break;
default:
rpc_feedback_info = "unknown feedback type";
break;
}
LOG_INFO("[MISS_LOG][RPC_FEEDBACK]", K(rpc_fetch_no_log), K(resp), KCSTRING(rpc_feedback_info));
}
}
} // namespace libobcdc
} // namespace oceanbase

View File

@ -0,0 +1,143 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* OBCDC header file
* This file defines interface of OBCDC
*/
#ifndef OCEANBASE_LIBOBCDC_MISS_LOG_HANDLER_H_
#define OCEANBASE_LIBOBCDC_MISS_LOG_HANDLER_H_
#include "lib/net/ob_addr.h"
#include "logservice/palf/log_entry.h"
#include "logservice/logfetcher/ob_log_fetch_stat_info.h" // TransStatInfo
#include "ob_log_ls_fetch_ctx.h" // LSFetchCtx
#include "ob_cdc_part_trans_resolver.h" // MissingLogInfo
#include "ob_log_fetch_log_rpc.h"
namespace oceanbase
{
using namespace palf;
using namespace common;
using namespace logfetcher;
using namespace obrpc;
namespace libobcdc
{
struct MissLogTask
{
public:
MissLogTask(
const common::ObAddr &orig_svr,
IObLogRpc &rpc,
LSFetchCtx &ls_fetch_ctx,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi);
~MissLogTask() { reset(); }
void reset();
public:
uint64_t get_tenant_id() const
{ return ls_fetch_ctx_.get_tls_id().get_tenant_id(); }
const PartTransID &get_part_trans_id() const
{ return missing_info_.get_part_trans_id(); }
int64_t get_handle_progress() const { return missing_info_.get_last_misslog_progress(); }
int try_change_server(const int64_t timeout, volatile bool &stop_flag);
TO_STRING_KV(K_(missing_info), K_(svr), K_(ls_fetch_ctx));
public:
common::ObAddr svr_;
IObLogRpc &rpc_; // rpc client
LSFetchCtx &ls_fetch_ctx_;
IObCDCPartTransResolver::MissingLogInfo &missing_info_;
bool need_change_server_;
logfetcher::TransStatInfo &tsi_;
};
// NOTICE: SINGLETON TOOL TO HANDLE MISSLOG. NOT THREAD-SAFE. DO NOT ADD FIELD IN ObCDCMissLogHandler
class ObCDCMissLogHandler
{
public:
static ObCDCMissLogHandler &get_instance();
~ObCDCMissLogHandler() {}
public:
int handle_log_miss(
const common::ObAddr &cur_svr,
IObLogRpc *rpc,
LSFetchCtx &ls_fetch_ctx,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag);
static void configure(const ObLogConfig &config);
private:
static int64_t g_rpc_timeout;
static const int64_t RETRY_LOG_PRINT_INTERVAL;
static const int64_t RETRY_TIMEOUT;
static const int64_t MAX_RPC_TIMEOUT;
private:
int handle_miss_log_task_(MissLogTask &misslog_task, volatile bool &stop_flag);
int handle_miss_record_or_state_log_(
MissLogTask &misslog_task,
FetchLogSRpc &fetch_log_srpc,
volatile bool &stop_flag);
int handle_miss_redo_log_(
MissLogTask &misslog_task,
FetchLogSRpc &fetch_log_srpc,
volatile bool &stop_flag);
// split all miss_logs by batch
int build_batch_misslog_lsn_arr_(
const int64_t fetched_log_idx,
IObCDCPartTransResolver::MissingLogInfo &missing_log_info,
ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &batched_misslog_lsn_arr);
// read batched misslog
int read_batch_misslog_(
LSFetchCtx &ls_fetch_ctx,
const obrpc::ObCdcLSFetchLogResp &resp,
int64_t &fetched_missing_log_cnt,
logfetcher::TransStatInfo &tsi,
IObCDCPartTransResolver::MissingLogInfo &missing_info);
int alloc_fetch_log_srpc_(const uint64_t tenant_id, FetchLogSRpc *&fetch_log_srpc);
void free_fetch_log_srpc_(FetchLogSRpc *fetch_log_srpc);
int fetch_miss_log_with_retry_(
MissLogTask &misslog_task,
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
FetchLogSRpc &fetch_srpc,
volatile bool &stop_flag);
int fetch_miss_log_(
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
MissLogTask &misslog_task,
FetchLogSRpc &fetch_srpc,
volatile bool &stop_flag);
int fetch_miss_log_direct_(
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
FetchLogSRpc &fetch_log_srpc,
LSFetchCtx &ls_fetch_ctx);
void check_feedback_(
const obrpc::ObCdcLSFetchLogResp &resp,
bool &has_valid_feedback,
bool &rpc_fatal_err);
private:
ObCDCMissLogHandler() {}
DISABLE_COPY_ASSIGN(ObCDCMissLogHandler);
};
} // namespace libobcdc
} // namespace oceanbase
#endif

View File

@ -44,11 +44,15 @@ IObCDCPartTransResolver::MissingLogInfo::~MissingLogInfo()
IObCDCPartTransResolver::MissingLogInfo
&IObCDCPartTransResolver::MissingLogInfo::operator=(const IObCDCPartTransResolver::MissingLogInfo &miss_log_info)
{
this->miss_redo_lsn_arr_ = miss_log_info.miss_redo_lsn_arr_;
this->miss_record_or_state_log_lsn_ = miss_log_info.miss_record_or_state_log_lsn_;
this->need_reconsume_commit_log_entry_ = miss_log_info.need_reconsume_commit_log_entry_;
this->is_resolving_miss_log_ = miss_log_info.is_resolving_miss_log_;
this->is_reconsuming_ = miss_log_info.is_reconsuming_;
if (this != &miss_log_info) {
this->part_trans_id_ = miss_log_info.part_trans_id_;
this->miss_redo_lsn_arr_ = miss_log_info.miss_redo_lsn_arr_;
this->miss_record_or_state_log_lsn_ = miss_log_info.miss_record_or_state_log_lsn_;
this->need_reconsume_commit_log_entry_ = miss_log_info.need_reconsume_commit_log_entry_;
this->is_resolving_miss_log_ = miss_log_info.is_resolving_miss_log_;
this->is_reconsuming_ = miss_log_info.is_reconsuming_;
this->last_misslog_process_ = miss_log_info.last_misslog_process_;
}
return *this;
}
@ -239,12 +243,17 @@ int ObCDCPartTransResolver::read(
if (OB_ITER_END == ret) {
if (OB_UNLIKELY(! missing_info.is_empty())) {
missing_info.set_trans_id(tx_log_block_header->get_tx_id());
// miss_log can only find while resolving record/commit_info/prepare/commit
ret = OB_ITEM_NOT_SETTED;
} else {
ret = OB_SUCCESS;
}
}
if (OB_ITEM_NOT_SETTED == ret || missing_info.is_resolving_miss_log()) {
missing_info.set_last_misslog_progress(submit_ts);
}
}
return ret;
@ -533,6 +542,7 @@ int ObCDCPartTransResolver::handle_redo_(
transaction::ObTxRedoLogTempRef tmp_ref;
transaction::ObTxRedoLog redo_log(tmp_ref);
PartTransTask *task = NULL;
const int64_t start_ts = get_timestamp();
if (OB_FAIL(tx_log_block.deserialize_log_body(redo_log))) {
LOG_ERROR("deserialize_redo_log_body failed", KR(ret), K_(tls_id), K(tx_id), K(lsn));
@ -565,8 +575,8 @@ int ObCDCPartTransResolver::handle_redo_(
K(handling_miss_log), K(task), K(redo_log));
}
} else {
LOG_DEBUG("handle_trans_redo", K_(tls_id), K(tx_id), K(lsn), K(submit_ts), K(redo_log),
K(handling_miss_log), KPC(task));
LOG_DEBUG("handle_trans_redo", K_(tls_id), K(tx_id), K(lsn), K(submit_ts), "cost", get_timestamp() - start_ts,
K(redo_log), K(handling_miss_log), KPC(task));
}
return ret;
@ -925,7 +935,7 @@ int ObCDCPartTransResolver::handle_commit_(
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid trans_commit_version", KR(ret), K_(tls_id), K(tx_id), K(lsn), K(submit_ts), K(commit_log));
} else if (!serve_info.is_served(trans_commit_version)) {
LOG_WARN("found trans not served", K_(tls_id), K(tx_id), K(lsn),
LOG_WARN("[IGNORE_TRANS] found trans not served", K_(tls_id), K(tx_id), K(lsn),
K(commit_log), K(serve_info));
if (OB_FAIL(part_trans_dispatcher_.remove_task(tls_id_.is_sys_log_stream(), tx_id))) {
LOG_ERROR("handle unserverd PartTransTask failed", KR(ret), K_(tls_id), K(tx_id));
@ -933,6 +943,8 @@ int ObCDCPartTransResolver::handle_commit_(
} else if (OB_FAIL(obtain_task_(tx_id, part_trans_task, is_reconsuming))) {
LOG_ERROR("obtain_part_trans_task fail while reading commit log", KR(ret), K_(tls_id), K(tx_id), K(lsn),
K(commit_log), K(missing_info));
} else if (OB_UNLIKELY(part_trans_task->is_trans_committed())) {
LOG_WARN("[IGNORE_COMMIT] already handle commit_log of part_trans", K_(tls_id), K(tx_id), K(lsn), K(submit_ts), K(commit_log));
} else if (OB_FAIL(part_trans_task->push_multi_data_source_data(lsn, commit_log.get_multi_source_data(), true/*is_commit_log*/))) {
LOG_ERROR("push_multi_data_source_data failed", KR(ret), K_(tls_id), K(tx_id), K(lsn), K(commit_log), KPC(part_trans_task));
} else if (!part_trans_task->has_read_commit_info()) {
@ -1118,7 +1130,7 @@ int ObCDCPartTransResolver::check_redo_log_list_(
{
int ret = OB_SUCCESS;
ObLogLSNArray sorted_redo_lsn_arr_in_trans_log;
const SortedLogEntryArray &fetched_lsn_arr =
const SortedLogEntryList &fetched_lsn_arr =
part_trans_task.get_sorted_log_entry_info().get_fetched_log_entry_node_arr();
for (int64_t idx = 0; OB_SUCC(ret) && idx < prev_redo_lsn_arr.count(); ++idx) {
@ -1134,7 +1146,7 @@ int ObCDCPartTransResolver::check_redo_log_list_(
sorted_redo_lsn_arr_in_trans_log.begin(),
sorted_redo_lsn_arr_in_trans_log.end(),
CDCLSNComparator());
LogEntryNode *first_fetched_log_entry_node = fetched_lsn_arr.get_first_node();
LogEntryNode *first_fetched_log_entry_node = fetched_lsn_arr.top();
if (OB_ISNULL(first_fetched_log_entry_node)) {
// doesn't fetch any log.
if (OB_FAIL(missing_info.push_back_missing_log_lsn_arr(prev_redo_lsn_arr))) {

View File

@ -81,13 +81,18 @@ public:
void reset()
{
part_trans_id_.reset();
miss_redo_lsn_arr_.reset();
miss_record_or_state_log_lsn_.reset();
need_reconsume_commit_log_entry_ = false;
is_resolving_miss_log_ = false;
is_reconsuming_ = false;
last_misslog_process_ = 0;
}
public:
void set_tls_id(const logservice::TenantLSID &tls_id) { part_trans_id_.tls_id_ = tls_id; }
void set_trans_id(const transaction::ObTransID &trans_id) { part_trans_id_.trans_id_ = trans_id; }
const PartTransID &get_part_trans_id() const { return part_trans_id_; }
/// has misslog or not
/// @retval bool ture if has miss_log(including redo/commit_info/prepare/commit and record_log)
bool is_empty() const { return miss_redo_lsn_arr_.count() <= 0 && !miss_record_or_state_log_lsn_.is_valid(); }
@ -112,8 +117,12 @@ public:
int64_t get_total_misslog_cnt() const;
int sort_and_unique_missing_log_lsn();
int64_t get_last_misslog_progress() const { return last_misslog_process_; }
void set_last_misslog_progress(int64_t last_misslog_process) { last_misslog_process_ = last_misslog_process; }
TO_STRING_KV(
K_(part_trans_id),
K_(last_misslog_process),
"miss_redo_count", miss_redo_lsn_arr_.count(),
K_(miss_redo_lsn_arr),
K_(miss_record_or_state_log_lsn),
@ -122,6 +131,7 @@ public:
K_(is_reconsuming));
private:
PartTransID part_trans_id_;
// miss redo log lsn array
ObLogLSNArray miss_redo_lsn_arr_;
// miss record log or state log(commit_info/prepare) lsn
@ -140,6 +150,7 @@ public:
// will ignore other type log while reconsuming commit_log_entry
bool is_reconsuming_;
// TODO use a int8_t instead above bool variable, may add is_reconsuming var for handle commit_info and commit log
int64_t last_misslog_process_;
};
public:

View File

@ -0,0 +1,255 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* SortedLinkList, NOT THREAD SAFE!
*/
#ifndef OCEANBSE_OBCDC_SORTED_LINKED_LIST_
#define OCEANBSE_OBCDC_SORTED_LINKED_LIST_
#include "ob_cdc_sorted_list.h"
namespace oceanbase
{
namespace libobcdc
{
// Obj T should:
// 0. not final
// 1. set_next() and get_next() to set next obj to build the list;
// 2. operator== to check if already has same obj in list;
// 3. operator< and operator> to find pos to insert obj into the list;
// 4. should already imply TO_STRING_KV;
template <typename T, typename CompareFunc = DefaultValComparator<T>>
class SortedLinkedList : public SortedList<T, CompareFunc>
{
public:
typedef ListNode<T, CompareFunc> ListNodeType;
typedef SortedList<T, CompareFunc> BaseList;
SortedLinkedList(ObIAllocator &allocator, const bool is_unique = true)
: BaseList(),
allocator_(allocator),
l_head_(),
l_tail_(nullptr),
is_unique_(is_unique) {}
virtual ~SortedLinkedList() { reset(); }
void reset() override
{
reset_data_();
is_unique_ = false;
}
public:
int push(T &val) override
{
int ret = OB_SUCCESS;
ListNodeType *node_for_val = nullptr;
if (OB_FAIL(alloc_node_(val, node_for_val))) {
OBLOG_LOG(ERROR, "alloc_node for val failed", KR(ret), K(val));
} else if (OB_FAIL(push_node(node_for_val))) {
if (OB_ENTRY_EXIST != ret) {
OBLOG_LOG(ERROR, "push node into SortedList failed", KR(ret), K(val));
} else {
OBLOG_LOG(INFO, "duplicated node with same value in SortedList, push failed", KR(ret), K(val));
}
}
if (OB_FAIL(ret) && OB_NOT_NULL(node_for_val)) {
node_for_val->~ListNodeType();
allocator_.free(node_for_val);
}
return ret;
}
int pop(T *&val) override
{
int ret = OB_SUCCESS;
ListNodeType *first_node = pop_front_node();
if (OB_NOT_NULL(first_node)) {
val = first_node->get_val();
first_node->~ListNodeType();
allocator_.free(first_node);
first_node = nullptr;
}
return ret;
}
// check if queue contains element that has the same value with val
bool contains(const T &val) override
{
bool b_ret = false;
ListNodeType *tmp = nullptr;
if (!is_list_empty()) {
if ((*l_tail_) < val) {
// quick-path: val is lager than last ListNode: list doesn't contains val
b_ret = false;
} else if ((*l_tail_) == val || (*get_first_node()) == val) {
// quick-path: val is equals to first/last ListNode: list contains val
b_ret = true;
} else {
find_pos_(val, tmp, b_ret);
}
}
return b_ret;
}
ListNodeType *get_first_node() const override { return l_head_.get_next(); }
int get_next_node(const ListNodeType &node, ListNodeType *&next_node) const override
{
next_node = node.get_next();
return OB_SUCCESS;
}
public:
bool is_list_empty() const { return nullptr == l_head_.get_next(); }
// push ListNode(type of T*) into queue
// can't push the same data into list multi times !!!
// retval OB_SUCCESS: push success
// retval OB_INVALID_ARGUMENT ListNode is nullptr
// retval OB_ENTRY_EXIST elements if queue is required unique(is_unique = true) and ListNode is already in queue
int push_node(ListNodeType *node_for_val)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(node_for_val)) {
ret = OB_INVALID_ARGUMENT;
OBLOG_LOG(ERROR, "invalid node to push into SortedList", KR(ret));
} else if (OB_ISNULL(l_tail_)) {
// empty list
ob_assert(is_list_empty());
l_head_.set_next(node_for_val);
l_tail_ = node_for_val;
BaseList::inc_node_num_();
} else if (*l_tail_ < *node_for_val) {
// insert at tail
l_tail_->set_next(node_for_val);
l_tail_ = node_for_val;
BaseList::inc_node_num_();
} else {
// insert between dummy and l_tail_
ListNodeType *prev_node = nullptr;
bool has_duplicate_node = false;
find_pos_(*(node_for_val->get_val()), prev_node, has_duplicate_node);
if (has_duplicate_node && is_unique_) {
ret = OB_ENTRY_EXIST;
OBLOG_LOG(INFO, "found duplicatee ListNode in list", KPC(node_for_val), KP(prev_node), KPC(prev_node), "next_node", prev_node->get_next(), KPC(this));
} else {
node_for_val->set_next(prev_node->get_next());
prev_node->set_next(node_for_val);
BaseList::inc_node_num_();
}
}
OBLOG_LOG(DEBUG, "push_node finish", KR(ret), KPC(this), KPC(node_for_val));
return ret;
}
ListNodeType* pop_front_node()
{
ListNodeType *node = get_first_node();
if (OB_NOT_NULL(node)) {
ListNodeType *next = node->get_next();
l_head_.set_next(next);
node->set_next(nullptr);
BaseList::dec_node_num_();
if (OB_ISNULL(next)) {
ob_assert(is_list_empty());
l_tail_ = nullptr;
}
}
OBLOG_LOG(DEBUG, "pop_front_node", KPC(this), KPC(node));
return node;
}
INHERIT_TO_STRING_KV(
"sorted_linked_list", BaseList,
"is_list_empty", is_list_empty(),
"l_head", l_head_.get_next(),
K_(l_tail),
KPC_(l_tail),
K_(is_unique));
private:
void reset_data_()
{
T *val = nullptr;
while (! is_list_empty()) {
pop(val);
}
l_tail_ = nullptr;
l_head_.set_next(l_tail_);
}
int alloc_node_(T &val, ListNodeType *&node)
{
int ret = OB_SUCCESS;
int64_t alloc_size = sizeof(ListNodeType);
node = static_cast<ListNodeType*>(allocator_.alloc(alloc_size));
if (OB_ISNULL(node)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
OBLOG_LOG(ERROR, "alloc memory for list node failed", KR(ret), K(val), K(alloc_size));
} else {
new(node) ListNodeType(val);
}
return ret;
}
// find pos to insert new obj
// prev_node is the last ListNode in list that value less than the input_ListNode
// O(N)
// @param [in] ListNode ListNode to search
// @param [out] prev_node the last ListNode already in list that has value less than ListNode
// @param [out] has_duplicate has duplicate ListNode(means has the same value with ListNode)
void find_pos_(const T &val, ListNodeType *&prev_node, bool &has_duplicate)
{
prev_node = &l_head_;
ListNodeType *next = l_head_.get_next();
bool found = false;
has_duplicate = false;
// conditions to quit search prev_node
// 1. found duplicate ListNode, ret = OB_ENTRY_EXIST
// 2. not found data equals or greater than ListNode, the last prev_node is returned(expected l_tail_)
while (OB_NOT_NULL(next = prev_node->get_next()) && !found) {
if (*next < val) {
// next ListNode is less than ListNode, push forword prev_node and go on search
prev_node = next;
} else {
found = true;
if (*next == val) {
has_duplicate = true;
}
}
}
}
private:
ObIAllocator &allocator_;
ListNodeType l_head_; // as head of list,always points to dummy_
ListNodeType *l_tail_; // tail of list, change if ListNode appends to last ListNode of lis
bool is_unique_; // if ListNode in list is unique(value but not address)
private:
DISALLOW_COPY_AND_ASSIGN(SortedLinkedList);
};
}
}
#endif

View File

@ -0,0 +1,117 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* Abstract SortedList
*/
#ifndef OCEANBSE_OBCDC_SORTED_LIST_
#define OCEANBSE_OBCDC_SORTED_LIST_
#include "lib/utility/ob_macro_utils.h"
#include "lib/utility/ob_print_utils.h"
#include "lib/ob_errno.h"
#include "lib/oblog/ob_log_module.h"
namespace oceanbase
{
namespace libobcdc
{
template<typename T>
struct DefaultValComparator
{
static int compare(const T &a, const T &b) { return a.compare(b); }
};
template<typename T, typename CompareFunc>
class ListNode
{
public:
typedef ListNode<T, CompareFunc> ListNodeType;
ListNode() : val_(nullptr), next_(nullptr) {}
ListNode(T &val) : val_(&val), next_(nullptr) {}
virtual ~ListNode()
{
val_ = nullptr;
next_ = nullptr;
}
public:
virtual int compare(const ListNodeType &other) const
{ OB_ASSERT(nullptr != val_); return CompareFunc::compare(*val_, *other.val_); }
bool operator<(const ListNodeType &other) { return compare(other) < 0; }
bool operator==(const ListNodeType &other) { return compare(other) == 0; }
bool operator>(const ListNodeType &other) { return compare(other) > 0; }
bool operator!=(const ListNodeType &other) { return compare(other) != 0; }
int compare(const T &other) const
{ OB_ASSERT(nullptr != val_); return CompareFunc::compare(*val_, other); }
bool operator<(const T &other) { return compare(other) < 0; }
bool operator==(const T &other) { return compare(other) == 0; }
bool operator>(const T &other) { return compare(other) > 0; }
public:
OB_INLINE T* get_val() { return val_; }
OB_INLINE void set_next(ListNodeType *next_node) { next_ = next_node; }
OB_INLINE ListNodeType *get_next() const {return next_;}
VIRTUAL_TO_STRING_KV(KPC_(val));
DISABLE_COPY_ASSIGN(ListNode);
protected:
T *val_;
ListNodeType *next_;
};
template <typename T, typename CompareFunc = DefaultValComparator<T>>
class SortedListIterator;
// interface for sorted list
template <typename T, typename CompareFunc = DefaultValComparator<T>>
class SortedList
{
friend class SortedListIterator<T, CompareFunc>;
typedef ListNode<T, CompareFunc> ListNodeType;
public:
SortedList() : count_(0) {}
virtual ~SortedList() { reset(); }
virtual void reset() { ATOMIC_SET(&count_, 0); };
virtual int push(T &val) = 0;
virtual int pop(T *&val) = 0;
virtual bool contains(const T &val) = 0;
public:
typedef SortedListIterator<T, CompareFunc> Iterator;
public:
Iterator begin() const;
Iterator end() const;
T *top() const
{
T *val = nullptr;
ListNodeType *first_node = get_first_node();
if (OB_NOT_NULL(first_node)) {
val = first_node->get_val();
}
return val;
}
int count() const { return ATOMIC_LOAD(&count_); }
bool empty() const { return 0 == count(); }
VIRTUAL_TO_STRING_KV(K_(count));
protected:
virtual ListNodeType *get_first_node() const = 0;
virtual int get_next_node(const ListNodeType &node, ListNodeType *&next_node) const = 0;
void inc_node_num_() { ATOMIC_INC(&count_); }
void dec_node_num_() { ATOMIC_DEC(&count_); }
private:
int64_t count_;
};
} // end of namespace libobcdc
} // end of namespace oceanbase
#endif

View File

@ -0,0 +1,109 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* Iterator of SortedList
*/
#ifndef OCEANBSE_OBCDC_SORTED_LIST_ITERATOR_
#define OCEANBSE_OBCDC_SORTED_LIST_ITERATOR_
#include "ob_cdc_sorted_list.h"
namespace oceanbase
{
namespace libobcdc
{
template <typename T, typename CompareFunc>
typename SortedList<T, CompareFunc>::Iterator SortedList<T, CompareFunc>::begin() const
{
return Iterator(this, get_first_node());
}
template <typename T, typename CompareFunc>
typename SortedList<T, CompareFunc>::Iterator SortedList<T, CompareFunc>::end() const
{
return Iterator(this, 0);
}
template <typename T, typename CompareFunc>
class SortedListIterator
{
typedef ListNode<T, CompareFunc> ListNodeType;
public:
// using standard iterator feature
using iterator_category = std::forward_iterator_tag;
using difference_type = std::ptrdiff_t;
using value_type = T;
using pointer = T*;
using reference = T&;
SortedListIterator(const SortedList<T, CompareFunc> *list, ListNodeType *node_ptr) : list_(list), node_ptr_(node_ptr) {}
virtual ~SortedListIterator() { reset(); }
bool is_valid() const { return nullptr != list_ && nullptr != node_ptr_; }
void reset() { list_ = nullptr; node_ptr_ = nullptr; }
// dereference operation
reference operator*() const {
ob_assert(nullptr != list_);
ob_assert(nullptr != node_ptr_);
ob_assert(nullptr != node_ptr_->get_val());
return *(node_ptr_->get_val());
}
// member pointer access
pointer operator->() const {
ob_assert(nullptr != list_);
ob_assert(nullptr != node_ptr_);
return node_ptr_->get_val(); // return pointer to data pointer
}
// ++iter
SortedListIterator& operator++() {
ob_assert(nullptr != list_);
if (node_ptr_) {
int ret = OB_SUCCESS;
ListNodeType *next = nullptr;
if (OB_FAIL(list_->get_next_node(*node_ptr_, next))) {
OBLOG_LOG(ERROR, "get_next_node failed", KR(ret), KPC_(list));
} else {
node_ptr_ = next;
}
}
return *this;
}
// iter++
SortedListIterator operator++(int) {
ob_assert(nullptr != list_);
SortedListIterator tmp = *this;
++(*this);
return tmp;
}
// NOTICE: not support operator--
bool operator==(const SortedListIterator& other) const {
return node_ptr_ == other.node_ptr_;
}
bool operator!=(const SortedListIterator& other) const {
return node_ptr_ != other.node_ptr_;
}
TO_STRING_KV(KPC_(list), KPC_(node_ptr));
private:
const SortedList<T, CompareFunc>* list_;
ListNode<T, CompareFunc>* node_ptr_;
};
} // end namespace libobcdc
} // end namespace oceanbase
#endif

View File

@ -0,0 +1,368 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*
* SortedTreeifyList, NOT THREAD SAFE!
*/
#ifndef OCEANBSE_OBCDC_SORTED_TREEIFY_LIST_
#define OCEANBSE_OBCDC_SORTED_TREEIFY_LIST_
#include "lib/container/ob_rbtree.h"
#include "ob_cdc_sorted_linked_list.h"
#include "ob_log_config.h"
namespace oceanbase
{
using namespace container;
namespace libobcdc
{
template <typename T, typename CompareFunc>
class TreeNode : public ListNode<T, CompareFunc>
{
public:
typedef TreeNode<T, CompareFunc> NodeType;
explicit TreeNode(T &val) : ListNode<T, CompareFunc>(val) {}
virtual ~TreeNode() {}
int compare(const NodeType *left, const NodeType *right) const
{
int ret = 0;
if (OB_ISNULL(left) || OB_ISNULL(left->val_)) {
ret = -1;
} else if (OB_ISNULL(right) || OB_ISNULL(right->val_)) {
ret = 1;
} else {
ret = CompareFunc::compare(*(left->val_), *(right->val_));
}
return ret;
}
int compare(const NodeType *other) const { return compare(this, other); }
RBNODE(NodeType, rblink);
};
template<typename T, typename CompareFunc = DefaultValComparator<T>>
class SortedTreeifyList : public SortedList<T, CompareFunc>
{
friend class SortedLinkedList<T, CompareFunc>;
public:
typedef TreeNode<T, CompareFunc> NodeType;
typedef ListNode<T, CompareFunc> BaseNodeType;
typedef SortedList<T, CompareFunc> BaseList;
SortedTreeifyList(ObIAllocator &allocator, bool auto_treeify_mode = false)
: BaseList(),
allocator_(allocator),
auto_treeify_mode_(auto_treeify_mode),
is_tree_mode_(false),
list_(allocator, true/*is_unique*/), // treeify list must guaratee val in list is unique
tree_(),
auto_treeify_threshold_(0),
auto_untreeify_threshold_(0)
{
if (auto_treeify_mode_) {
auto_treeify_threshold_ = TCONF.sorted_list_auto_treeify_threshold;
auto_untreeify_threshold_ = TCONF.sorted_list_auto_untreeify_threshold;
_OBLOG_LOG(TRACE, "[TREEIFY_LIST][AUTO_TREEIFY_THRESHOLD:%ld][AUTO_UNTREEIFY_THRESHOLD:%ld]",
auto_treeify_threshold_, auto_untreeify_threshold_);
}
}
virtual ~SortedTreeifyList() {}
void reset() override
{
untreeify();
NodeType *node = nullptr;
while(OB_NOT_NULL(node = pop_from_list_())) {
if (OB_ISNULL(node)) {
OBLOG_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "unexpected null node");
} else {
node->~NodeType();
allocator_.free(node);
node = nullptr;
}
}
ob_assert(list_.is_list_empty());
ob_assert(BaseList::empty());
tree_.init_tree();
}
public:
int push(T &val) override
{
int ret = OB_SUCCESS;
int64_t alloc_size = sizeof(NodeType);
NodeType *node = static_cast<NodeType*>(allocator_.alloc(alloc_size));
if (OB_ISNULL(node)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
OBLOG_LOG(ERROR, "alloc memory for node failed", KR(ret), K(alloc_size));
} else {
new(node) NodeType(val);
if (OB_UNLIKELY(is_tree_mode_)) {
if (OB_FAIL(push_to_tree_(*node))) {
OBLOG_LOG(ERROR, "insert node into tree failed", KR(ret), KPC(this));
}
} else if (OB_FAIL(push_to_list_(*node))) {
OBLOG_LOG(ERROR, "insert node into list failed", KR(ret), KPC(this));
}
if (OB_FAIL(ret) && OB_NOT_NULL(node)) {
node->~NodeType();
allocator_.free(node);
node = nullptr;
}
}
if (OB_SUCC(ret)) {
if (auto_treeify_mode_ && ! is_tree_mode_ && BaseList::count() >= auto_treeify_threshold_) {
if (OB_FAIL(treeify())) {
OBLOG_LOG(WARN, "treeify failed", KR(ret), KPC(this));
}
}
}
return ret;
}
int pop(T *&val) override
{
int ret = OB_SUCCESS;
NodeType *node = nullptr;
val = nullptr;
if (OB_UNLIKELY(is_tree_mode_)) {
node = pop_from_tree_();
} else {
node = pop_from_list_();
}
if (OB_SUCC(ret) && OB_NOT_NULL(node)) {
if (auto_treeify_mode_ && is_tree_mode_ && BaseList::count() < auto_untreeify_threshold_) {
if (OB_FAIL(untreeify())) {
OBLOG_LOG(WARN, "untreeify failed", KR(ret), KPC(this));
}
}
}
if (OB_SUCC(ret) && OB_NOT_NULL(node)) {
val = node->get_val();
node->~NodeType();
allocator_.free(node);
node = nullptr;
}
return ret;
}
bool contains(const T &val) override
{
int ret = OB_SUCCESS;
bool found = false;
if (is_tree_mode_) {
T &non_const_val = const_cast<T&>(val);
NodeType node(non_const_val);
NodeType *found_node = nullptr;
if (OB_FAIL(tree_.search(&node, found_node))) {
OBLOG_LOG(WARN, "search node from tree failed", KR(ret), K(node));
} else {
found = (nullptr != found_node);
}
} else {
found = list_.contains(val);
}
return found;
}
// manage node by rbtree
int treeify()
{
int ret = OB_SUCCESS;
const int64_t node_cnt = BaseList::count();
OBLOG_LOG(DEBUG, "treeify begin", KPC(this));
if (OB_UNLIKELY(is_tree_mode_)) {
// ignore cause already in tree mode
} else if (OB_UNLIKELY(!tree_.is_empty())) {
ret = OB_STATE_NOT_MATCH;
OBLOG_LOG(ERROR, "expect empty tree before treeify", KR(ret), KPC(this));
} else {
int64_t removed_linked_node_cnt = 0;
while (OB_SUCC(ret) && removed_linked_node_cnt++ < node_cnt) {
NodeType *node = nullptr;
if (OB_ISNULL(node = pop_from_list_())) {
ret = OB_ERR_UNEXPECTED;
OBLOG_LOG(ERROR, "invalid node poped from linked_list", KR(ret), KP(node));
} else if (OB_FAIL(push_to_tree_(*node))) {
OBLOG_LOG(ERROR, "insert node into tree failed", KR(ret), KPC(this));
}
}
}
if (OB_SUCC(ret)) {
ob_assert(list_.is_list_empty());
ob_assert(node_cnt == BaseList::count());
is_tree_mode_ = true;
}
OBLOG_LOG(DEBUG, "treeify end", KPC(this));
return ret;
}
// manage node by linked list
int untreeify()
{
int ret = OB_SUCCESS;
const int64_t node_cnt = BaseList::count();
OBLOG_LOG(DEBUG, "untreeify begin", KPC(this));
if (OB_UNLIKELY(!is_tree_mode_)) {
} else if (OB_UNLIKELY(! list_.is_list_empty())) {
ret = OB_STATE_NOT_MATCH;
OBLOG_LOG(ERROR, "expect empty list before untreeify", KR(ret), KPC(this));
} else {
int64_t removed_tree_node_cnt = 0;
while (OB_SUCC(ret) && removed_tree_node_cnt++ < node_cnt) {
NodeType *node = nullptr;
if (OB_ISNULL(node = pop_from_tree_())) {
ret = OB_ERR_UNEXPECTED;
OBLOG_LOG(ERROR, "pop_from_tree_ failed", KR(ret));
} else if (OB_FAIL(push_to_list_(*node))) {
OBLOG_LOG(ERROR, "insert node into list failed", KR(ret), KPC(this));
}
}
}
if (OB_SUCC(ret)) {
ob_assert(tree_.is_empty());
ob_assert(node_cnt == BaseList::count());
is_tree_mode_ = false;
}
OBLOG_LOG(DEBUG, "untreeify end", KPC(this));
return ret;
}
virtual BaseNodeType* get_first_node() const override
{
BaseNodeType *node = nullptr;
if (OB_UNLIKELY(is_tree_mode_)) {
node = static_cast<BaseNodeType*>(tree_.get_first());
} else {
node = list_.get_first_node();
}
return node;
}
int get_next_node(const BaseNodeType &node, BaseNodeType *&next_node) const override
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_tree_mode_)) {
const NodeType *cur_node = static_cast<const NodeType *>(&node);
NodeType *next = nullptr;
if (OB_ISNULL(cur_node)) {
ret = OB_ERR_UNEXPECTED;
OBLOG_LOG(ERROR, "invalid TreeNode converted from ListNodeType", KR(ret), K(node));
} else if (OB_FAIL(tree_.get_next(cur_node, next))) {
OBLOG_LOG(WARN, "fail to get next node from tree", KR(ret), KPC(cur_node));
} else {
next_node = static_cast<BaseNodeType*>(next);
}
} else {
next_node = node.get_next();
}
return ret;
}
INHERIT_TO_STRING_KV(
"sorted_treeify_list", BaseList,
K_(is_tree_mode),
K_(auto_treeify_mode),
"is_tree_empty", tree_.is_empty(),
K_(list),
K_(auto_treeify_threshold),
K_(auto_untreeify_threshold));
private:
int push_to_tree_(NodeType &node)
{
int ret = OB_SUCCESS;
if (OB_FAIL(tree_.insert(&node))) {
OBLOG_LOG(ERROR, "insert node into tree failed", KR(ret), KPC(this));
} else {
BaseList::inc_node_num_();
OBLOG_LOG(DEBUG, "push_to_tree_ succ", KPC(this));
}
return ret;
}
NodeType *pop_from_tree_()
{
int ret = OB_SUCCESS;
NodeType *node = tree_.get_first();
if (OB_NOT_NULL(node)) {
if (OB_FAIL(tree_.remove(node))) {
OBLOG_LOG(ERROR, "remove node from tree failed", KR(ret), KPC(node));
} else {
BaseList::dec_node_num_();
OBLOG_LOG(DEBUG, "pop_from_tree_ succ", KPC(this));
}
}
return node;
}
int push_to_list_(NodeType &node)
{
int ret = OB_SUCCESS;
if (OB_FAIL(list_.push_node(&node))) {
OBLOG_LOG(ERROR, "insert node into list failed", KR(ret), KPC(this));
} else {
BaseList::inc_node_num_(); // should operate the node count cause list is another container which is seperate with treeify_list
OBLOG_LOG(DEBUG, "push_to_list_ succ", KPC(this));
}
return ret;
}
NodeType *pop_from_list_()
{
NodeType *node = static_cast<NodeType*>(list_.pop_front_node());
if (OB_NOT_NULL(node)) {
BaseList::dec_node_num_();
OBLOG_LOG(DEBUG, "pop_from_list_ succ", KPC(this));
}
return node;
}
private:
typedef ObRbTree<NodeType, ObDummyCompHelper<NodeType>> tree_t;
// allocator to alloc memory for NODE
ObIAllocator &allocator_;
bool auto_treeify_mode_;
bool is_tree_mode_;
SortedLinkedList<T, CompareFunc> list_;
tree_t tree_;
// treefiy the list if auto_treeify_mode_ = true and node_num is large than auto_treeify_threshold_
int64_t auto_treeify_threshold_;
// untreeify the list if auto_treeify_mode_ = true and node_num is small than auto_untreeify_threshold_
int64_t auto_untreeify_threshold_;
}; // end of SortedTreeifyList
} // end of namespace libobcdc
} // end of namespace oceanbase
#endif

View File

@ -127,7 +127,7 @@ void ObLogClusterIDFilter::stat_ignored_tps()
if (last_stat_time_ > 0 && delta_time > 0) {
double tps = static_cast<double>(cur_count - last_count) / static_cast<double>(delta_time);
_LOG_INFO("[TPS_STAT] CLUSTER_ID_IGNORED_PART_TPS=%.3lf", tps);
_LOG_INFO("[TPS_STAT] CLUSTER_ID_IGNORED_LOG_ENTRY_TPS=%.3lf", tps);
}
last_cluster_id_ignored_part_trans_count_ = cur_count;

View File

@ -613,6 +613,8 @@ public:
T_DEF_BOOL(enable_direct_load_inc, OB_CLUSTER_PARAMETER, 0, "0:disabled, 1:enabled");
T_DEF_INT_INFT(direct_load_inc_thread_num, OB_CLUSTER_PARAMETER, 0, 0, "thread num of reading and parsing direct load inc log");
T_DEF_INT_INFT(direct_load_inc_queue_backlog_lowest_tolerance, OB_CLUSTER_PARAMETER, 0, 0, "lowest threshold of queue_backlog that will touch parser flow control in direct load inc case");
T_DEF_INT_INFT(sorted_list_auto_treeify_threshold, OB_CLUSTER_PARAMETER, 32, 0, "treeify list auto-treeify mode treeify threshold");
T_DEF_INT_INFT(sorted_list_auto_untreeify_threshold, OB_CLUSTER_PARAMETER, 30, 0, "treeify list auto-treeify mode treeify threshold");
#undef OB_CLUSTER_PARAMETER

View File

@ -66,14 +66,15 @@ int FetchLogSRpc::fetch_log(IObLogRpc &rpc,
const share::ObLSID &ls_id,
const ObIArray<ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const common::ObAddr &svr,
const int64_t timeout)
const int64_t timeout,
const int64_t progress)
{
int ret = OB_SUCCESS;
reset();
// build request
if (OB_FAIL(build_request_(tenant_id, ls_id, miss_log_array))) {
if (OB_FAIL(build_request_(tenant_id, ls_id, miss_log_array, progress))) {
LOG_ERROR("build request fail", KR(ret), K(tenant_id), K(ls_id));
}
// Send asynchronous fetch log RPC
@ -103,7 +104,8 @@ int FetchLogSRpc::fetch_log(IObLogRpc &rpc,
int FetchLogSRpc::build_request_(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const ObIArray<ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array)
const ObIArray<ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t progress)
{
int ret = OB_SUCCESS;
reset();
@ -111,6 +113,7 @@ int FetchLogSRpc::build_request_(
// Set request parameters
req_.set_ls_id(ls_id);
req_.set_client_pid(static_cast<uint64_t>(getpid()));
req_.set_progress(progress);
ARRAY_FOREACH_N(miss_log_array, idx, count) {
const ObCdcLSFetchMissLogReq::MissLogParam &miss_log_param = miss_log_array.at(idx);

View File

@ -55,7 +55,8 @@ public:
const share::ObLSID &ls_id,
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const common::ObAddr &svr,
const int64_t timeout);
const int64_t timeout,
const int64_t progress);
int set_resp(const obrpc::ObRpcResultCode &rcode,
const obrpc::ObCdcLSFetchLogResp *resp);
@ -70,7 +71,8 @@ private:
int build_request_(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array);
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t progress);
private:
////////////////////////////// RpcCB //////////////////////////////

View File

@ -1050,6 +1050,23 @@ int LSFetchCtx::next_server(common::ObAddr &request_svr)
return ret;
}
int LSFetchCtx::get_server_count(int64_t &server_count)
{
int ret = OB_SUCCESS;
logservice::ObLogRouteService *log_route_service = nullptr;
if (OB_FAIL(get_log_route_service_(log_route_service))) {
LOG_ERROR("get_log_route_service_ failed", KR(ret));
} else if (OB_FAIL(log_route_service->get_server_count(
tls_id_.get_tenant_id(),
tls_id_.get_ls_id(),
server_count))) {
LOG_ERROR("get_server_count failed", KR(ret), K_(tls_id));
}
return ret;
}
int LSFetchCtx::init_locate_req_svr_list_(StartLSNLocateReq &req, LocateSvrList &locate_svr_list)
{
int ret = OB_SUCCESS;

View File

@ -209,6 +209,8 @@ public:
/// @retval Other error codes Failed
int next_server(common::ObAddr &request_svr);
int get_server_count(int64_t &server_count);
void mark_svr_list_update_flag(const bool need_update);
uint64_t hash() const;

View File

@ -24,6 +24,7 @@
#include "ob_ls_worker.h" // IObLSWorker
#include "ob_log_part_progress_controller.h" // PartProgressController
#include "ob_log_trace_id.h" // ObLogTraceIdGuard
#include "ob_cdc_miss_log_handler.h" // MissLogHandler
using namespace oceanbase::common;
using namespace oceanbase::obrpc;
@ -274,6 +275,8 @@ void FetchStream::configure(const ObLogConfig &config)
LOG_INFO("[CONFIG]", K(print_rpc_handle_info));
ATOMIC_STORE(&g_print_stream_dispatch_info, print_stream_dispatch_info);
LOG_INFO("[CONFIG]", K(print_stream_dispatch_info));
ObCDCMissLogHandler::configure(config);
}
void FetchStream::do_stat(int64_t &traffic)
@ -821,6 +824,7 @@ int FetchStream::read_group_entry_(palf::LogGroupEntry &group_entry,
IObCDCPartTransResolver::MissingLogInfo missing_info;
log_entry.reset();
missing_info.reset();
missing_info.set_tls_id(ls_fetch_ctx_->get_tls_id());
if (OB_FAIL(entry_iter.next())) {
if (OB_ITER_END != ret) {
@ -838,22 +842,16 @@ int FetchStream::read_group_entry_(palf::LogGroupEntry &group_entry,
stop_flag))) {
if (OB_ITEM_NOT_SETTED == ret) {
// handle missing_log_info
missing_info.set_resolving_miss_log();
const bool need_reconsume = missing_info.need_reconsume_commit_log_entry();
KickOutReason fail_reason = NONE;
if (OB_FAIL(handle_log_miss_(log_entry, missing_info, local_tsi, stop_flag, fail_reason))) {
if (OB_NEED_RETRY == ret) {
int tmp_ret = OB_SUCCESS;
// need switch other server(fetch_stream) to fetch log
if (OB_TMP_FAIL(set_(kick_out_info, ls_fetch_ctx_->get_tls_id(), fail_reason))) {
if (OB_ENTRY_EXIST == tmp_ret) {
tmp_ret = OB_SUCCESS;
} else {
LOG_ERROR("set_kickout_set fail", KR(tmp_ret), K_(ls_fetch_ctx), K(missing_info), K(fail_reason));
}
}
} else if (OB_IN_STOP_STATE != ret) {
if (OB_FAIL(ObCDCMissLogHandler::get_instance().handle_log_miss(
svr_,
rpc_,
*ls_fetch_ctx_,
missing_info,
tsi,
stop_flag))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("handle_missing_log_ fail", KR(ret), K(entry_iter), K(group_entry),
K(group_start_lsn), K_(ls_fetch_ctx));
}
@ -1442,533 +1440,6 @@ int FetchStream::read_log_(
return ret;
}
int FetchStream::fetch_miss_log_direct_(
const ObIArray<ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
FetchLogSRpc &fetch_log_srpc,
LSFetchCtx &ls_fetch_ctx)
{
int ret = OB_SUCCESS;
ObCdcLSFetchLogResp *resp = NULL;
LSFetchCtxGetSourceFunctor get_source_func(ls_fetch_ctx);
const int64_t current_progress = ls_fetch_ctx.get_progress();
const int64_t tenant_id = ls_fetch_ctx.get_tls_id().get_tenant_id();
const ObLSID &ls_id = ls_fetch_ctx.get_tls_id().get_ls_id();
archive::LargeBufferPool *buffer_pool = NULL;
logservice::ObLogExternalStorageHandler *log_ext_handler = NULL;
ObRpcResultCode rcode;
SCN cur_scn;
const int64_t start_fetch_ts = get_timestamp();
const int64_t time_upper_limit = start_fetch_ts + timeout;
bool stop_fetch = false;
bool is_timeout = false;
if (OB_ISNULL(resp = OB_NEW(ObCdcLSFetchLogResp, "FetchMissResp"))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc ObCdcLSFetchLogResp failed", KR(ret));
} else if (OB_FAIL(cur_scn.convert_from_ts(current_progress/1000L))) {
LOG_ERROR("convert log progress to scn failed", KR(ret), K(current_progress));
} else if (OB_FAIL(ls_fetch_ctx.get_large_buffer_pool(buffer_pool))) {
LOG_ERROR("get large buffer pool when fetching missing log failed", KR(ret), K(ls_fetch_ctx));
} else if (OB_FAIL(ls_fetch_ctx.get_log_ext_handler(log_ext_handler))) {
LOG_ERROR("get log ext handler when fetching missing log failed", KR(ret), K(ls_fetch_ctx));
} else {
int64_t fetched_cnt = 0;
const int64_t arr_cnt = miss_log_array.count();
resp->set_next_miss_lsn(miss_log_array.at(0).miss_lsn_);
while (OB_SUCC(ret) && !stop_fetch) {
bool retry_on_err = false;
while (OB_SUCC(ret) && fetched_cnt < arr_cnt && !is_timeout) {
const int64_t start_fetch_entry_ts = get_timestamp();
const ObCdcLSFetchMissLogReq::MissLogParam &param = miss_log_array.at(fetched_cnt);
const LSN &missing_lsn = param.miss_lsn_;
const char *buf;
int64_t buf_size = 0;
palf::LogEntry log_entry;
palf::LSN lsn;
logservice::ObRemoteLogpEntryIterator entry_iter(get_source_func);
resp->set_next_miss_lsn(missing_lsn);
if (get_timestamp() > time_upper_limit) {
is_timeout = true;
} else if (OB_FAIL(entry_iter.init(tenant_id, ls_id, cur_scn, missing_lsn,
LSN(palf::LOG_MAX_LSN_VAL), buffer_pool, log_ext_handler))) {
LOG_WARN("remote entry iter init failed", KR(ret));
} else if (OB_FAIL(entry_iter.set_io_context(palf::LogIOContext(palf::LogIOUser::CDC)))) {
LOG_WARN("remote entry iter set_io_context failed", KR(ret));
} else if (OB_FAIL(entry_iter.next(log_entry, lsn, buf, buf_size))) {
retry_on_err =true;
LOG_WARN("log entry iter failed to iterate", KR(ret));
} else {
if (OB_UNLIKELY(missing_lsn != lsn)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("direct fetch missing_lsn not match", KR(ret), K(tenant_id), K(ls_id),
K(missing_lsn), K(lsn));
} else {
const int64_t entry_size = log_entry.get_serialize_size();
int64_t pos = 0;
resp->inc_log_fetch_time(get_timestamp() - start_fetch_entry_ts);
if (! resp->has_enough_buffer(entry_size)) {
ret = OB_BUF_NOT_ENOUGH;
} else {
int64_t remain_size = 0;
char *remain_buf = resp->get_remain_buf(remain_size);
if (OB_ISNULL(remain_buf)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("remain buffer is null", KR(ret));
}
else if (OB_FAIL(log_entry.serialize(remain_buf, remain_size, pos))) {
LOG_WARN("missing LogEntry serialize failed", KR(ret), K(remain_size),
K(pos), K(missing_lsn), K(log_entry));
}
else {
// TODO: there is an issue in entry_iter.next(), the returned buffer is not as expected
// MEMCPY(remain_buf, buf, buf_size);
resp->log_entry_filled(entry_size);
fetched_cnt++;
}
}
}
}
}// while
if (OB_SUCC(ret)) {
stop_fetch = true;
} else if (OB_BUF_NOT_ENOUGH == ret) {
stop_fetch = true;
ret = OB_SUCCESS;
} else if (retry_on_err) {
ret = OB_SUCCESS;
}
} // while
resp->set_l2s_net_time(0);
resp->set_svr_queue_time(0);
resp->set_process_time(get_timestamp() - start_fetch_ts);
}
// regard resp not null as sending rpc successfully
if (OB_NOT_NULL(resp)) {
resp->set_debug_err(ret);
if (OB_FAIL(ret)) {
resp->set_err(OB_ERR_SYS);
} else {
resp->set_err(OB_SUCCESS);
}
ret = OB_SUCCESS;
rcode.rcode_ = OB_SUCCESS;
} else {
rcode.rcode_ = ret;
sprintf(rcode.msg_, "failed to allocate fetchlogresp");
}
fetch_log_srpc.set_resp(rcode, resp);
if (OB_NOT_NULL(resp)) {
OB_DELETE(ObCdcLSFetchLogResp, "FetchMissResp", resp);
resp = NULL;
}
return ret;
}
int FetchStream::fetch_miss_log_(
FetchLogSRpc &fetch_srpc,
IObLogRpc &rpc,
LSFetchCtx &ls_fetch_ctx,
const ObIArray<ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const common::ObAddr &svr,
const int64_t timeout)
{
int ret = OB_SUCCESS;
const ClientFetchingMode fetching_mode = ls_fetch_ctx.get_fetching_mode();
static bool fetch_missing_fail = true;
if ((1 == TCONF.test_fetch_missing_errsim) && fetch_missing_fail) {
fetch_missing_fail = false;
ret = OB_NEED_RETRY;
LOG_ERROR("errsim fetch missing log fail", K(ls_fetch_ctx));
} else if (! is_fetching_mode_valid(fetching_mode)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetching mode is not valid", KR(ret), K(fetching_mode));
} else if (is_integrated_fetching_mode(fetching_mode)) {
if (OB_FAIL(fetch_srpc.fetch_log(*rpc_,
ls_fetch_ctx_->get_tls_id().get_tenant_id(),
ls_fetch_ctx_->get_tls_id().get_ls_id(),
miss_log_array,
svr_,
timeout))) {
LOG_ERROR("fetch_misslog_rpc exec failed", KR(ret), K_(ls_fetch_ctx), K_(svr), K(miss_log_array));
ret = OB_NEED_RETRY;
} else {
// succ
}
} else if (is_direct_fetching_mode(fetching_mode)) {
// mock FetchLogSRpc here
if (OB_FAIL(fetch_miss_log_direct_(miss_log_array, timeout, fetch_srpc, ls_fetch_ctx))) {
LOG_ERROR("fetch missing log direct failed", KR(ret), K(ls_fetch_ctx), K(miss_log_array));
// rewrite ret code to make sure that cdc wouldn't exit because fetch_missing_log_direct_ failed.
ret = OB_NEED_RETRY;
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid fetching mode", KR(ret), K(fetching_mode), K(ls_fetch_ctx));
}
return ret;
}
int FetchStream::handle_log_miss_(
palf::LogEntry &log_entry,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(missing_info.is_empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("empty missing_info", KR(ret), K(missing_info), K(log_entry));
} else {
FetchLogSRpc *fetch_log_srpc = NULL;
if (OB_FAIL(alloc_fetch_log_srpc_(fetch_log_srpc))) {
LOG_ERROR("alloc fetch_log_srpc fail", KR(ret));
} else if (OB_ISNULL(fetch_log_srpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid fetch_log_srpc", KR(ret));
} else if (OB_FAIL(handle_miss_record_or_state_log_(
*fetch_log_srpc,
missing_info,
tsi,
stop_flag,
fail_reason))) {
LOG_ERROR("handle_miss_record_or_state_log_ failed", KR(ret));
} else if (OB_FAIL(handle_miss_redo_log_(
*fetch_log_srpc,
missing_info,
tsi,
stop_flag,
fail_reason))) {
LOG_ERROR("handle_miss_redo_log_ failed", KR(ret), KR(ret));
}
if (stop_flag) {
ret = OB_IN_STOP_STATE;
} else if (OB_NEED_RETRY == ret) {
fail_reason = KickOutReason::MISSING_LOG_FETCH_FAIL;
} else {
LOG_INFO("handle miss_log done", KR(ret), "tls_id", ls_fetch_ctx_->get_tls_id(), K(missing_info));
}
if (OB_NOT_NULL(fetch_log_srpc)) {
free_fetch_log_srpc_(fetch_log_srpc);
fetch_log_srpc = NULL;
}
}
return ret;
}
int FetchStream::handle_miss_record_or_state_log_(
FetchLogSRpc &fetch_log_srpc,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason)
{
int ret = OB_SUCCESS;
if (missing_info.has_miss_record_or_state_log()) {
int64_t rpc_timeout = ATOMIC_LOAD(&g_rpc_timeout);
ObArrayImpl<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> batched_misslog_lsn_arr;
palf::LSN misslog_lsn;
while (OB_SUCC(ret) && ! stop_flag && missing_info.has_miss_record_or_state_log()) {
misslog_lsn.reset();
batched_misslog_lsn_arr.reset();
ObCdcLSFetchMissLogReq::MissLogParam param;
if (OB_FAIL(missing_info.get_miss_record_or_state_log_lsn(misslog_lsn))) {
LOG_ERROR("get_miss_record_or_state_log_lsn failed", K(missing_info), K(misslog_lsn));
} else {
param.miss_lsn_ = misslog_lsn;
if (OB_FAIL(batched_misslog_lsn_arr.push_back(param))) {
LOG_ERROR("push_back miss_record_or_state_log_lsn into batched_misslog_lsn_arr failed", KR(ret), K(param));
} else if (OB_FAIL(fetch_miss_log_(
fetch_log_srpc,
*rpc_,
*ls_fetch_ctx_,
batched_misslog_lsn_arr,
svr_,
rpc_timeout))) {
LOG_ERROR("fetch_miss_log_ failed", KR(ret), K_(ls_fetch_ctx), K_(svr), K(batched_misslog_lsn_arr));
} else {
const obrpc::ObRpcResultCode &rcode = fetch_log_srpc.get_result_code();
const obrpc::ObCdcLSFetchLogResp &resp = fetch_log_srpc.get_resp();
if (OB_FAIL(rcode.rcode_) || OB_FAIL(resp.get_err())) {
ret = OB_NEED_RETRY;
LOG_ERROR("fetch log fail on rpc", K(rcode), K(resp), K(batched_misslog_lsn_arr));
} else if (resp.get_log_num() < 1) {
LOG_INFO("fetch_miss_log_rpc doesn't fetch log, retry", K(misslog_lsn), K_(svr));
} else if (OB_UNLIKELY(resp.get_log_num() > 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("expect only one misslog while fetching miss_record_or_state_log", K(resp));
} else if (OB_UNLIKELY(resp.get_next_miss_lsn() != misslog_lsn)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetched log not match miss_log_lsn", KR(ret), K(misslog_lsn), K(resp));
} else {
missing_info.reset_miss_record_or_state_log_lsn();
palf::LogEntry miss_log_entry;
miss_log_entry.reset();
const char *buf = resp.get_log_entry_buf();
const int64_t len = resp.get_pos();
int64_t pos = 0;
if (OB_FAIL(miss_log_entry.deserialize(buf, len, pos))) {
LOG_ERROR("deserialize log_entry of miss_record_or_state_log failed", KR(ret), K(misslog_lsn), KP(buf), K(len), K(pos));
} else if (OB_FAIL(ls_fetch_ctx_->read_miss_tx_log(miss_log_entry, misslog_lsn, tsi, missing_info))) {
if (OB_ITEM_NOT_SETTED == ret) {
ret = OB_SUCCESS;
LOG_INFO("found new miss_record_or_state_log while resolving current miss_record_or_state_log",
"tls_id", ls_fetch_ctx_->get_tls_id(), K(misslog_lsn), K(missing_info));
} else if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("read miss_log failed", KR(ret), K(miss_log_entry), K(misslog_lsn), K(missing_info));
}
}
}
}
}
}
if (OB_SUCC(ret)) {
LOG_INFO("fetch record and state misslog done and collect all miss normal misslog",
"tls_id", ls_fetch_ctx_->get_tls_id(), K(missing_info));
}
}
return ret;
}
int FetchStream::handle_miss_redo_log_(
FetchLogSRpc &fetch_log_srpc,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason)
{
int ret = OB_SUCCESS;
if (OB_FAIL(missing_info.sort_and_unique_missing_log_lsn())) {
LOG_ERROR("sort_and_unique_missing_log_lsn failed", KR(ret), K(missing_info), K_(ls_fetch_ctx));
} else {
const int64_t total_misslog_cnt = missing_info.get_total_misslog_cnt();
int64_t fetched_missing_log_cnt = 0;
int64_t rpc_timeout = ATOMIC_LOAD(&g_rpc_timeout);
ObArrayImpl<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> batched_misslog_lsn_arr;
while (OB_SUCC(ret) && ! stop_flag && fetched_missing_log_cnt < total_misslog_cnt) {
batched_misslog_lsn_arr.reset();
if (OB_FAIL(build_batch_misslog_lsn_arr_(
fetched_missing_log_cnt,
missing_info,
batched_misslog_lsn_arr))) {
LOG_ERROR("build_batch_misslog_lsn_arr_ failed", KR(ret),
K(missing_info), K(fetched_missing_log_cnt));
} else if (OB_FAIL(fetch_miss_log_(
fetch_log_srpc,
*rpc_,
*ls_fetch_ctx_,
batched_misslog_lsn_arr,
svr_,
rpc_timeout))) {
LOG_ERROR("fetch_miss_log_ failed", KR(ret), K_(ls_fetch_ctx), K_(svr), K(batched_misslog_lsn_arr));
} else {
const obrpc::ObRpcResultCode &rcode = fetch_log_srpc.get_result_code();
const obrpc::ObCdcLSFetchLogResp &resp = fetch_log_srpc.get_resp();
if (OB_FAIL(rcode.rcode_) || OB_FAIL(resp.get_err())) {
ret = OB_NEED_RETRY;
LOG_ERROR("fetch log fail on rpc", K(rcode), K(resp), K(batched_misslog_lsn_arr));
} else {
// check next_miss_lsn
bool is_next_miss_lsn_match = false;
palf::LSN next_miss_lsn = resp.get_next_miss_lsn();
const int64_t batch_cnt = batched_misslog_lsn_arr.count();
const int64_t resp_log_cnt = resp.get_log_num();
if (batch_cnt == resp_log_cnt) {
is_next_miss_lsn_match = (batched_misslog_lsn_arr.at(batch_cnt - 1).miss_lsn_ == next_miss_lsn);
} else if (batch_cnt > resp_log_cnt) {
is_next_miss_lsn_match = (batched_misslog_lsn_arr.at(resp_log_cnt).miss_lsn_ == next_miss_lsn);
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("too many misslog fetched", KR(ret), K(next_miss_lsn), K(batch_cnt),
K(resp_log_cnt),K(resp), K_(ls_fetch_ctx));
}
if (OB_SUCC(ret)) {
if (!is_next_miss_lsn_match) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("misslog fetched is not match batched_misslog_lsn_arr requested", KR(ret),
K(next_miss_lsn), K(batch_cnt), K(resp_log_cnt), K(batched_misslog_lsn_arr), K(resp), K_(ls_fetch_ctx));
} else if (OB_FAIL(read_batch_misslog_(
resp,
fetched_missing_log_cnt,
tsi,
missing_info))) {
// expected no misslog found while resolving normal log.
LOG_ERROR("read_batch_misslog failed", KR(ret), K_(ls_fetch_ctx),
K(fetched_missing_log_cnt), K(missing_info));
}
}
}
}
}
}
return ret;
}
int FetchStream::build_batch_misslog_lsn_arr_(
const int64_t fetched_log_idx,
IObCDCPartTransResolver::MissingLogInfo &missing_log_info,
ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &batched_misslog_lsn_arr)
{
int ret = OB_SUCCESS;
int64_t batched_cnt = 0;
static int64_t MAX_MISSLOG_CNT_PER_RPC= 1024;
if (OB_UNLIKELY(0 < batched_misslog_lsn_arr.count())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("invalid batched_misslog_lsn_arr", KR(ret), K(batched_misslog_lsn_arr));
} else {
const ObLogLSNArray &miss_redo_or_state_log_arr = missing_log_info.get_miss_redo_lsn_arr();
int miss_log_cnt = miss_redo_or_state_log_arr.count();
batched_misslog_lsn_arr.reset();
// fetched_log_idx is log_count that already fetched after last batch rpc
// for miss_redo_or_state_log_arr with 100 miss_log and MAX_MISSLOG_CNT_PER_RPC = 10
// fetched_log_idx start from 0, if fetched 8 miss_log in one rpc, then fetched_log_idx is 8,
// and for next batch, miss_redo_or_state_log_arr.at(8) is the 9th miss_log as expected.
for (int idx = fetched_log_idx; OB_SUCC(ret) && batched_cnt < MAX_MISSLOG_CNT_PER_RPC && idx < miss_log_cnt; idx++) {
const palf::LSN &lsn = miss_redo_or_state_log_arr.at(idx);
ObCdcLSFetchMissLogReq::MissLogParam param;
param.miss_lsn_ = lsn;
if (OB_FAIL(batched_misslog_lsn_arr.push_back(param))) {
LOG_ERROR("push_back missing_log lsn into batched_misslog_lsn_arr failed", KR(ret), K(idx),
K(fetched_log_idx), K(miss_redo_or_state_log_arr), K(batched_misslog_lsn_arr), K(param));
} else {
batched_cnt++;
}
}
}
LOG_INFO("build_batch_misslog_lsn_arr_", KR(ret),
"tls_id", ls_fetch_ctx_->get_tls_id(),
K(missing_log_info),
"batched_misslog_lsn_count", batched_misslog_lsn_arr.count(),
K(fetched_log_idx));
return ret;
}
int FetchStream::read_batch_misslog_(
const obrpc::ObCdcLSFetchLogResp &resp,
int64_t &fetched_missing_log_cnt,
logfetcher::TransStatInfo &tsi,
IObCDCPartTransResolver::MissingLogInfo &missing_info)
{
int ret = OB_SUCCESS;
LOG_INFO("read_batch_misslog_ begin", "tls_id", ls_fetch_ctx_->get_tls_id(), K(resp), K(fetched_missing_log_cnt));
const int64_t total_misslog_cnt = missing_info.get_total_misslog_cnt();
const char *buf = resp.get_log_entry_buf();
const int64_t len = resp.get_pos();
int64_t pos = 0;
const int64_t log_cnt = resp.get_log_num();
const ObLogLSNArray &org_misslog_arr = missing_info.get_miss_redo_lsn_arr();
int64_t start_ts = get_timestamp();
if (OB_UNLIKELY(log_cnt <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("expected valid log count from FetchLogSRpc for misslog", KR(ret), K(resp));
} else {
for (int64_t idx = 0; OB_SUCC(ret) && idx < log_cnt; idx++) {
if (fetched_missing_log_cnt >= total_misslog_cnt) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fetched_missing_log_cnt is more than total_misslog_cnt", KR(ret),
K(fetched_missing_log_cnt), K(missing_info), K(idx), K(resp));
} else {
palf::LSN misslog_lsn;
palf::LogEntry miss_log_entry;
misslog_lsn.reset();
miss_log_entry.reset();
IObCDCPartTransResolver::MissingLogInfo tmp_miss_info;
tmp_miss_info.set_resolving_miss_log();
if (org_misslog_arr.count() == fetched_missing_log_cnt) {
// already consume the all miss_redo_log, but still exist one miss_record_log.
// lsn record_log is the last miss_log to fetch.
if (OB_FAIL(missing_info.get_miss_record_or_state_log_lsn(misslog_lsn))) {
if (OB_ENTRY_NOT_EXIST == ret) {
LOG_ERROR("expect valid miss-record_log_lsn", KR(ret), K(missing_info), K(fetched_missing_log_cnt), K_(ls_fetch_ctx));
} else {
LOG_ERROR("get_miss_record_or_state_log_lsn failed", KR(ret), K(missing_info), K(fetched_missing_log_cnt), K_(ls_fetch_ctx));
}
}
} else if (OB_FAIL(org_misslog_arr.at(fetched_missing_log_cnt, misslog_lsn))) {
LOG_ERROR("get misslog_lsn fail", KR(ret), K(fetched_missing_log_cnt),
K(idx), K(org_misslog_arr), K(resp));
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(miss_log_entry.deserialize(buf, len, pos))) {
LOG_ERROR("deserialize miss_log_entry fail", KR(ret), K(len), K(pos));
} else if (OB_FAIL(ls_fetch_ctx_->read_miss_tx_log(miss_log_entry, misslog_lsn, tsi, tmp_miss_info))) {
if (OB_IN_STOP_STATE != ret) {
LOG_ERROR("read_miss_log fail", KR(ret), K(miss_log_entry),
K(misslog_lsn), K(fetched_missing_log_cnt), K(idx), K(tmp_miss_info));
}
} else {
fetched_missing_log_cnt++;
}
}
}
}
int64_t read_batch_missing_cost = get_timestamp() - start_ts;
LOG_INFO("read_batch_misslog_ end", KR(ret), "tls_id", ls_fetch_ctx_->get_tls_id(), K(read_batch_missing_cost),
K(fetched_missing_log_cnt), K(resp), K(start_ts));
return ret;
}
int FetchStream::alloc_fetch_log_srpc_(FetchLogSRpc *&fetch_log_srpc)
{
int ret = OB_SUCCESS;
void *buf = ob_malloc(sizeof(FetchLogSRpc), ObModIds::OB_LOG_FETCH_LOG_SRPC);
if (OB_ISNULL(buf)) {
LOG_ERROR("alloc memory for FetchLogSRpc fail", K(sizeof(FetchLogSRpc)));
ret = OB_ALLOCATE_MEMORY_FAILED;
} else if (OB_ISNULL(fetch_log_srpc = new(buf) FetchLogSRpc())) {
LOG_ERROR("construct fetch log srpc fail", K(buf));
ret = OB_ALLOCATE_MEMORY_FAILED;
} else {
// success
}
return ret;
}
void FetchStream::free_fetch_log_srpc_(FetchLogSRpc *fetch_log_srpc)
{
if (NULL != fetch_log_srpc) {
fetch_log_srpc->~FetchLogSRpc();
ob_free(fetch_log_srpc);
fetch_log_srpc = NULL;
}
}
int FetchStream::kick_out_task_(const KickOutInfo &kick_out_info)
{
int ret = OB_SUCCESS;

View File

@ -240,60 +240,6 @@ private:
int64_t &read_log_time,
int64_t &decode_log_entry_time,
logfetcher::TransStatInfo &tsi);
int fetch_miss_log_direct_(
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const int64_t timeout,
FetchLogSRpc &fetch_log_srpc,
LSFetchCtx &ls_fetch_ctx);
int fetch_miss_log_(
FetchLogSRpc &fetch_srpc,
IObLogRpc &rpc,
LSFetchCtx &ctx,
const ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &miss_log_array,
const common::ObAddr &svr,
const int64_t timeout);
// handle if found misslog while read_log_
//
// @param [in] log_entry LogEntry
// @param [in] missing_info MissingLogInfo
// @param [in] tsi TransStatInfo
// @param [out] fail_reason KickOutReason
//
// @retval OB_SUCCESS success
// @retval OB_NEED_RETRY RPC failed, need retry
// @retval other error code fail
int handle_log_miss_(
palf::LogEntry &log_entry,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason);
int handle_miss_record_or_state_log_(
FetchLogSRpc &fetch_log_srpc,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason);
int handle_miss_redo_log_(
FetchLogSRpc &fetch_log_srpc,
IObCDCPartTransResolver::MissingLogInfo &missing_info,
logfetcher::TransStatInfo &tsi,
volatile bool &stop_flag,
KickOutReason &fail_reason);
// split all miss_logs by batch
int build_batch_misslog_lsn_arr_(
const int64_t fetched_log_idx,
IObCDCPartTransResolver::MissingLogInfo &missing_log_info,
ObIArray<obrpc::ObCdcLSFetchMissLogReq::MissLogParam> &batched_misslog_lsn_arr);
// read batched misslog
int read_batch_misslog_(
const obrpc::ObCdcLSFetchLogResp &resp,
int64_t &fetched_missing_log_cnt,
logfetcher::TransStatInfo &tsi,
IObCDCPartTransResolver::MissingLogInfo &missing_info);
int alloc_fetch_log_srpc_(FetchLogSRpc *&fetch_log_srpc);
void free_fetch_log_srpc_(FetchLogSRpc *fetch_log_srpc);
// TODO @bohou handle missing log end
KickOutReason get_feedback_reason_(const Feedback &feedback) const;
int check_feedback_(

View File

@ -126,7 +126,7 @@ int ObLogPartTransParser::parse(PartTransTask &task, const bool is_build_baselin
} else {
const SortedRedoLogList &sorted_redo_list = task.get_sorted_redo_list();
// Parse Redo logs if they exist
if (sorted_redo_list.log_num_ > 0 && OB_FAIL(parse_ddl_redo_log_(task, is_build_baseline, stop_flag))) {
if (sorted_redo_list.get_node_number() > 0 && OB_FAIL(parse_ddl_redo_log_(task, is_build_baseline, stop_flag))) {
LOG_ERROR("parse_ddl_redo_log_ fail", KR(ret), K(task), K(is_build_baseline));
}
}
@ -212,7 +212,7 @@ int ObLogPartTransParser::parse_ddl_redo_log_(PartTransTask &task, const bool is
int ret = OB_SUCCESS;
int64_t redo_num = 0;
SortedRedoLogList &sorted_redo_list = task.get_sorted_redo_list();
DdlRedoLogNode *redo_node = static_cast<DdlRedoLogNode *>(sorted_redo_list.head_);
RedoNodeIterator redo_iter = sorted_redo_list.redo_iter_begin();
const uint64_t tenant_id = task.get_tenant_id();
if (OB_UNLIKELY(! sorted_redo_list.is_valid())) {
@ -241,29 +241,39 @@ int ObLogPartTransParser::parse_ddl_redo_log_(PartTransTask &task, const bool is
}
if (OB_SUCC(ret)) {
while (OB_SUCCESS == ret && NULL != redo_node) {
LOG_DEBUG("parse redo log", "redo_node", *redo_node);
if (OB_UNLIKELY(! redo_node->is_valid())) {
LOG_ERROR("redo_node is invalid", "redo_node", *redo_node, "redo_index", redo_num);
ret = OB_INVALID_DATA;
}
// Calibrate data for completeness
else if (OB_UNLIKELY(! redo_node->check_data_integrity())) {
LOG_ERROR("redo data is not valid", KPC(redo_node));
ret = OB_INVALID_DATA;
} else if (OB_FAIL(parse_stmts_(tenant, *redo_node, is_build_baseline,
invalid_redo_log_entry_task, task, row_index, stop_flag))) {
LOG_ERROR("parse_stmts_ fail", KR(ret), K(tenant), "redo_node", *redo_node,
K(is_build_baseline), K(task), K(row_index));
while (OB_SUCCESS == ret && redo_iter != sorted_redo_list.redo_iter_end() && !stop_flag) {
if (OB_UNLIKELY(!redo_iter.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid redo iterator", KR(ret), K(sorted_redo_list), K(redo_iter));
} else {
redo_num += redo_node->get_log_num();
redo_node = static_cast<DdlRedoLogNode *>(redo_node->get_next());
DdlRedoLogNode *ddl_redo = static_cast<DdlRedoLogNode*>(&(*redo_iter));
LOG_DEBUG("parse redo log", "redo_node", *ddl_redo);
if (OB_UNLIKELY(! ddl_redo->is_valid())) {
ret = OB_INVALID_DATA;
LOG_ERROR("redo_node is invalid", KR(ret), "redo_node", *ddl_redo, "redo_index", redo_num);
}
// Calibrate data for completeness
else if (OB_UNLIKELY(! ddl_redo->check_data_integrity())) {
LOG_ERROR("redo data is not valid", KPC(ddl_redo));
ret = OB_INVALID_DATA;
} else if (OB_FAIL(parse_stmts_(tenant, *ddl_redo, is_build_baseline,
invalid_redo_log_entry_task, task, row_index, stop_flag))) {
LOG_ERROR("parse_stmts_ fail", KR(ret), K(tenant), "redo_node", *ddl_redo,
K(is_build_baseline), K(task), K(row_index));
} else {
redo_num += ddl_redo->get_log_num();
redo_iter++;
}
}
} // while
} // end while
}
}
if (OB_SUCC(ret) && stop_flag) {
ret = OB_IN_STOP_STATE;
}
return ret;
}

View File

@ -1747,10 +1747,10 @@ int DdlStmtTask::parse_ddl_info(
if (OB_SUCCESS == ret) {
_LOG_INFO("[STAT] [DDL] [PARSE] OP_TYPE=%s(%ld) SCHEMA_VERSION=%ld "
"VERSION_DELAY=%.3lf(sec) EXEC_TENANT_ID=%ld TABLE_ID=%ld TENANT_ID=%ld DB_ID=%ld "
"VERSION_DELAY=%s EXEC_TENANT_ID=%ld TABLE_ID=%ld TENANT_ID=%ld DB_ID=%ld "
"TG_ID=%ld DDL_STMT=[%s] CONTAIN_DDL=%d IS_VALID=%d",
ObSchemaOperation::type_str((ObSchemaOperationType)ddl_operation_type_),
ddl_operation_type_, ddl_op_schema_version_, get_delay_sec(ddl_op_schema_version_),
ddl_operation_type_, ddl_op_schema_version_, TS_TO_DELAY(ddl_op_schema_version_),
ddl_exec_tenant_id_, ddl_op_table_id_, ddl_op_tenant_id_,
ddl_op_database_id_, ddl_op_tablegroup_id_,
to_cstring(ddl_stmt_str_), contain_ddl_stmt, is_valid_ddl);
@ -2490,6 +2490,8 @@ void ObLogEntryTask::set_row_ref_cnt(const int64_t row_ref_cnt)
PartTransTask::PartTransTask() :
ObLogResourceRecycleTask(ObLogResourceRecycleTask::PART_TRANS_TASK),
allocator_(),
log_entry_task_base_allocator_(),
serve_state_(SERVED),
cluster_id_(0),
type_(TASK_TYPE_UNKNOWN),
@ -2509,8 +2511,8 @@ PartTransTask::PartTransTask() :
participants_(),
trace_id_(),
trace_info_(),
sorted_log_entry_info_(),
sorted_redo_list_(),
sorted_log_entry_info_(allocator_),
sorted_redo_list_(allocator_),
part_tx_fetch_state_(0),
rollback_list_(),
ref_cnt_(0),
@ -2529,9 +2531,7 @@ PartTransTask::PartTransTask() :
wait_data_ready_cond_(),
wait_formatted_cond_(NULL),
output_br_count_by_turn_(0),
tic_update_infos_(),
allocator_(),
log_entry_task_base_allocator_()
tic_update_infos_()
{
}
@ -3550,6 +3550,8 @@ int PartTransTask::commit(
K(trans_commit_version), K(trans_type), K(ls_info_array), K(commit_log_lsn), KPC(this));
} else if (OB_FAIL(to_string_part_trans_info_())) {
LOG_ERROR("to_string_part_trans_info_str failed", KR(ret), K(trans_commit_version), K(cluster_id), K(commit_log_lsn), KPC(this));
} else if (OB_FAIL(untreeify_redo_list_())) {
LOG_ERROR("untreeify redo_list failed", KR(ret), K(trans_commit_version), K(cluster_id), K(commit_log_lsn), KPC(this));
} else {
// 3. trans_version, cluster_id and commit_log_lsn
commit_ts_ = commit_log_submit_ts;
@ -3575,6 +3577,7 @@ int PartTransTask::commit(
return ret;
}
int PartTransTask::try_to_set_data_ready_status()
{
int ret = OB_SUCCESS;
@ -3790,6 +3793,32 @@ int PartTransTask::parse_tablet_change_mds_(
return ret;
}
int PartTransTask::treeify_redo_list_()
{
int ret = OB_SUCCESS;
if (OB_FAIL(sorted_log_entry_info_.treeify_fetched_log_entry_list())) {
LOG_ERROR("treeify fetched_log_entry_list failed", KR(ret), KPC(this));
} else if (OB_FAIL(sorted_redo_list_.treeify())) {
LOG_ERROR("treeify sorted_redo_list failed", KR(ret), KPC(this));
}
return ret;
}
int PartTransTask::untreeify_redo_list_()
{
int ret = OB_SUCCESS;
if (OB_FAIL(sorted_log_entry_info_.untreeify_fetched_log_entry_list())) {
LOG_ERROR("untreeify fetched_log_entry_list failed", KR(ret), KPC(this));
} else if (OB_FAIL(sorted_redo_list_.untreeify())) {
LOG_ERROR("untreeify sorted_redo_list failed", KR(ret), KPC(this));
}
return ret;
}
int PartTransTask::init_ls_heartbeat_info(const logservice::TenantLSID &tls_id, const int64_t timestamp)
{
int ret = OB_SUCCESS;

View File

@ -1357,7 +1357,18 @@ private:
const MultiDataSourceNode &multi_data_source_node,
ObCDCTabletChangeInfo &tablet_change_info);
int treeify_redo_list_(); // try to convert sorted_redo_list and fetched_log_entry_arr to tree
int untreeify_redo_list_(); // try to convert sorted_redo_list and fetched_log_entry_arr to list
private:
// allocator used to alloc:
// LogEntryNode/RollbackNode
// DdlRedoLogNode/DmlRedoLogNode/mutator_row_data
// trace_id/trace_info/part_trans_info_str_/participant_
// MutatorRow(DDL)/DdlStmtTask
ObSmallArena allocator_;
ObLfFIFOAllocator log_entry_task_base_allocator_;
ServedState serve_state_;
// trans basic info
uint64_t cluster_id_; // cluster ID
@ -1439,14 +1450,6 @@ private:
ObArray<TICUpdateInfo> tic_update_infos_; // table id cache update info
// allocator used to alloc:
// LogEntryNode/RollbackNode
// DdlRedoLogNode/DmlRedoLogNode/mutator_row_data
// trace_id/trace_info/part_trans_info_str_/participant_
// MutatorRow(DDL)/DdlStmtTask
ObSmallArena allocator_;
ObLfFIFOAllocator log_entry_task_base_allocator_;
private:
DISALLOW_COPY_AND_ASSIGN(PartTransTask);
};

View File

@ -591,7 +591,7 @@ int ObLogResourceCollector::handle(void *data,
LOG_ERROR("ObLogBR task is NULL");
ret = OB_ERR_UNEXPECTED;
} else if (OB_FAIL(task->get_record_type(record_type))) {
LOG_ERROR("ObLogBR task get_record_type fail", KR(ret));
LOG_ERROR("ObLogBR task get_record_type fail", KR(ret), KPC(task));
} else {
if (HEARTBEAT == record_type || EBEGIN == record_type || ECOMMIT == record_type) {
br_pool_->free(task);
@ -875,21 +875,30 @@ int ObLogResourceCollector::recycle_stored_redo_(PartTransTask &task)
int ret = OB_SUCCESS;
const logservice::TenantLSID &tenant_ls_id = task.get_tls_id();
SortedRedoLogList &sorted_redo_list = task.get_sorted_redo_list();
DmlRedoLogNode *dml_redo_node = static_cast<DmlRedoLogNode *>(sorted_redo_list.head_);
RedoNodeIterator redo_iter = sorted_redo_list.redo_iter_begin();
while (OB_SUCC(ret) && OB_NOT_NULL(dml_redo_node) && ! RCThread::is_stoped()) {
if (dml_redo_node->is_stored()) {
const palf::LSN &store_log_lsn = dml_redo_node->get_start_log_lsn();
ObLogStoreKey store_key;
std::string key;
while (OB_SUCC(ret) && redo_iter != sorted_redo_list.redo_iter_end() && ! RCThread::is_stoped()) {
if (OB_UNLIKELY(!redo_iter.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("expected valid redo iterator", KR(ret), K(sorted_redo_list), K(redo_iter));
} else {
DmlRedoLogNode *dml_redo_node = static_cast<DmlRedoLogNode *>(&(*redo_iter));
if (OB_ISNULL(dml_redo_node)) {
ret = OB_INVALID_DATA;
LOG_ERROR("invalid dml_redo_node convert from redo_iter", KR(ret), K(redo_iter));
} else if (dml_redo_node->is_stored()) {
const palf::LSN &store_log_lsn = dml_redo_node->get_start_log_lsn();
ObLogStoreKey store_key;
std::string key;
if (OB_FAIL(store_key.init(tenant_ls_id, store_log_lsn))) {
LOG_ERROR("store_key init fail", KR(ret), K(store_key), K(tenant_ls_id), K(store_log_lsn));
} else if (OB_FAIL(store_key.get_key(key))) {
LOG_ERROR("get_storage_key fail", KR(ret), "key", key.c_str());
} else if (OB_FAIL(del_store_service_data_(tenant_ls_id.get_tenant_id(), key))) {
LOG_ERROR("del_store_service_data_ fail", KR(ret), K(task));
} else {}
if (OB_FAIL(store_key.init(tenant_ls_id, store_log_lsn))) {
LOG_ERROR("store_key init fail", KR(ret), K(store_key), K(tenant_ls_id), K(store_log_lsn));
} else if (OB_FAIL(store_key.get_key(key))) {
LOG_ERROR("get_storage_key fail", KR(ret), "key", key.c_str());
} else if (OB_FAIL(del_store_service_data_(tenant_ls_id.get_tenant_id(), key))) {
LOG_ERROR("del_store_service_data_ fail", KR(ret), K(task));
}
}
}
if (RCThread::is_stoped()) {
@ -897,7 +906,7 @@ int ObLogResourceCollector::recycle_stored_redo_(PartTransTask &task)
}
if (OB_SUCC(ret)) {
dml_redo_node = static_cast<DmlRedoLogNode *>(dml_redo_node->get_next());
redo_iter++;
}
}

View File

@ -101,6 +101,7 @@ int get_tenant_tz_map_function(
class IObLogErrHandler;
class IObLogSysTableHelper;
// TODO: replace to linked_hashmap in case of update while using tz_info
typedef common::hash::ObHashMap<uint64_t, ObCDCTenantTimeZoneInfo*> ObLogTZInfoMap;
class ObCDCTimeZoneInfoGetter : public IObCDCTimeZoneInfoGetter
{

View File

@ -65,7 +65,7 @@ int SortedLogEntryInfo::push_fetched_log_entry_node(LogEntryNode *log_entry_node
if (OB_ISNULL(log_entry_node) || OB_UNLIKELY(!log_entry_node->is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("invalid log_entry_node pushed to fetche", KR(ret), KPC(log_entry_node));
} else if (OB_FAIL(fetched_log_entry_arr_.push(log_entry_node))) {
} else if (OB_FAIL(fetched_log_entry_arr_.push(*log_entry_node))) {
LOG_ERROR("push log_entry_node into fetched_log_entry_arr failed", KR(ret), KPC(log_entry_node));
} else {
last_fetched_redo_log_entry_ = log_entry_node;
@ -270,56 +270,16 @@ int SortedRedoLogList::push(const bool is_data_in_memory,
if (OB_ISNULL(node) || OB_UNLIKELY(! node->is_valid(is_data_in_memory))) {
OBLOG_LOG(ERROR, "invalid argument", K(node));
ret = OB_INVALID_ARGUMENT;
} else if (NULL == head_) {
head_ = node;
node->set_next(NULL);
tail_ = node;
node_num_ = 1;
if (is_data_in_memory) {
ready_node_num_ = 1;
}
log_num_ = node->get_log_num();
} else { // NULL != head_
if (OB_ISNULL(tail_)) {
OBLOG_LOG(ERROR, "tail node is NULL, but head node is not NULL", K(head_), K(tail_));
ret = OB_ERR_UNEXPECTED;
} else if (OB_FAIL(redo_node_list_.push(*node))) {
if (OB_ENTRY_EXIST == ret) {
LOG_WARN("duplicate redo node", KR(ret), KPC(node));
} else {
// quick-path
if (tail_->before(*node)) {
tail_->set_next(node);
tail_ = node;
node->set_next(NULL);
} else {
// Iterate through all nodes to find the first redo node that is greater than or equal to the target node
RedoLogMetaNode **next_ptr = &head_;
while ((*next_ptr)->before(*node)) {
next_ptr = &((*next_ptr)->get_next_ptr());
}
// If the node value is duplicated, export error OB_ENTRY_EXIST
// NOTE: if one redo contains multi log_entry(in LOB case), which means start_lsn != log_lsn, should modify code below
if ((*next_ptr)->get_start_log_lsn() == node->get_start_log_lsn()) {
OBLOG_LOG(INFO, "redo log is pushed twice", KPC(node), KPC(*next_ptr), KPC(this));
ret = OB_ENTRY_EXIST;
} else {
node->set_next((*next_ptr));
*next_ptr = node;
}
}
if (OB_SUCCESS == ret) {
log_num_ += node->get_log_num();
ATOMIC_INC(&node_num_);
if (is_data_in_memory) {
ATOMIC_INC(&ready_node_num_);
}
}
LOG_ERROR("push redo node failed", KR(ret), KPC(node));
}
} else {
if (is_data_in_memory) {
inc_ready_node_num();
}
}
if (OB_SUCCESS == ret) {
last_push_node_ = node;
}
return ret;
@ -327,8 +287,8 @@ int SortedRedoLogList::push(const bool is_data_in_memory,
void SortedRedoLogList::init_iterator()
{
cur_dispatch_redo_ = head_;
cur_sort_redo_ = head_;
cur_dispatch_redo_ = redo_node_list_.begin();
cur_sort_redo_ = redo_node_list_.begin();
cur_sort_stmt_ = NULL; // row not format and stmt should be null
sorted_progress_.reset();
}
@ -337,7 +297,7 @@ int SortedRedoLogList::next_dml_redo(RedoLogMetaNode *&dml_redo_meta, bool &is_l
{
int ret = OB_SUCCESS;
if (OB_ISNULL(cur_dispatch_redo_)) {
if (OB_UNLIKELY(!cur_dispatch_redo_.is_valid())) {
if (is_dispatch_finish()) {
ret = OB_EMPTY_RESULT;
} else {
@ -345,9 +305,8 @@ int SortedRedoLogList::next_dml_redo(RedoLogMetaNode *&dml_redo_meta, bool &is_l
LOG_ERROR("can't get redo to dispatch but part_trans not dispatch finished", KR(ret), KPC(this));
}
} else {
RedoLogMetaNode *next_redo = cur_dispatch_redo_->get_next();
dml_redo_meta = cur_dispatch_redo_;
cur_dispatch_redo_ = next_redo;
dml_redo_meta = &(*cur_dispatch_redo_);
cur_dispatch_redo_++;
// Theoretically no concurrent call of this function
sorted_progress_.inc_dispatched_redo_count();
is_last_redo = is_dispatch_finish();
@ -363,7 +322,7 @@ int SortedRedoLogList::next_dml_stmt(ObLink *&dml_stmt_task)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(cur_sort_redo_)) {
if (OB_UNLIKELY(!cur_sort_redo_.is_valid())) {
if (OB_ISNULL(cur_sort_stmt_)) {
ret = OB_ITER_END;
} else {
@ -374,12 +333,12 @@ int SortedRedoLogList::next_dml_stmt(ObLink *&dml_stmt_task)
bool found = false;
while(OB_SUCC(ret) && !found) {
if (OB_ISNULL(cur_sort_redo_)) {
if (OB_UNLIKELY(!cur_sort_redo_.is_valid())) {
ret = OB_ITER_END;
} else if (OB_ISNULL(cur_sort_stmt_)) {
// set cur_sort_stmt_ to the first stmt of cur_sort_redo
DmlRedoLogNode *dml_redo_node = NULL;
if (OB_ISNULL(dml_redo_node = static_cast<DmlRedoLogNode*>(cur_sort_redo_))) {
if (OB_ISNULL(dml_redo_node = static_cast<DmlRedoLogNode*>(&(*cur_sort_redo_)))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("cast RedoLogMetaNode to DmlRedoLogNode fail", KR(ret), K_(cur_sort_redo), KP(this), KPC(this));
} else if (!dml_redo_node->is_formatted()) {
@ -400,11 +359,11 @@ int SortedRedoLogList::next_dml_stmt(ObLink *&dml_stmt_task)
// switch redo node:
// 1. found dml_stmt_task and it is the last stmt of cur_sort_redo
// 2. cur_sort_redo doesn't has any row
cur_sort_redo_ = cur_sort_redo_->get_next();
cur_sort_redo_++;
sorted_progress_.inc_sorted_redo_count();
}
}
}
} // end while
}
if (OB_ITER_END == ret) {

View File

@ -18,7 +18,8 @@
#include "storage/tx/ob_trans_log.h" // ObTransPrepareLog, ObTransCommitLog
#include "lib/queue/ob_link.h" // ObLink
#include "lib/allocator/ob_mod_define.h"
#include "ob_cdc_lightly_sorted_list.h" // SortedLightyList
#include "ob_cdc_sorted_treeify_list.h" // SortedTreeifyList
#include "ob_cdc_sorted_list_iterator.h" // SortedListIterator
#include "ob_log_utils.h" // ObLogLSNArray
namespace oceanbase
@ -70,19 +71,26 @@ private:
LogEntryNode *next_; // next LogEntry(for the same PartTransTask)
};
typedef SortedLightyList<LogEntryNode> SortedLogEntryArray;
struct LogEntryNodeComparator
{
static int compare(const LogEntryNode& node1, const LogEntryNode& node2) {
return node1.get_lsn().val_ > node2.get_lsn().val_ ? 1 : (node1.get_lsn().val_ == node2.get_lsn().val_ ? 0 : -1);
}
};
typedef SortedTreeifyList<LogEntryNode, LogEntryNodeComparator> SortedLogEntryList;
class SortedLogEntryInfo
{
public:
SortedLogEntryInfo() :
SortedLogEntryInfo(ObIAllocator& allocator) :
last_fetched_redo_log_entry_(NULL),
fetched_log_entry_arr_(true), /*is_unique*/
fetched_log_entry_arr_(allocator, true), /* auto_treeify_mode=true */
recorded_lsn_arr_() {}
~SortedLogEntryInfo() { reset(); }
void reset()
{
last_fetched_redo_log_entry_ = NULL;
fetched_log_entry_arr_.reset_data();
fetched_log_entry_arr_.reset();
recorded_lsn_arr_.reset();
}
@ -100,7 +108,10 @@ public:
// note: RollbackTo is treated as Redo.
int is_all_log_entry_fetched(bool &is_all_redo_fetched);
SortedLogEntryArray &get_fetched_log_entry_node_arr() { return fetched_log_entry_arr_; }
SortedLogEntryList &get_fetched_log_entry_node_arr() { return fetched_log_entry_arr_; }
int treeify_fetched_log_entry_list() { return fetched_log_entry_arr_.treeify(); }
int untreeify_fetched_log_entry_list() { return fetched_log_entry_arr_.untreeify(); }
TO_STRING_KV(
"fetched_log_entry_count", fetched_log_entry_arr_.count(),
@ -110,7 +121,7 @@ public:
private:
LogEntryNode *last_fetched_redo_log_entry_;
// hold all fetched log_entry_info.(include lsn of log_entry which contains redo_log and rollback_to log)
SortedLogEntryArray fetched_log_entry_arr_;
SortedLogEntryList fetched_log_entry_arr_;
// hold all prev_redo_log_lsn in all TxLog:
// 1. prev_redo_lsn_arr recorded by RecordLog/CommitInfoLog
// 2. lsn of commit_info_log_entry that contains redo_log.
@ -150,7 +161,7 @@ public:
void set_host_logentry_node(LogEntryNode *log_entry_node) { host_log_entry_ = log_entry_node; };
LogEntryNode *get_host_logentry_node() { return host_log_entry_; }
// Is the log ID sequentially located before the target node
bool before(const RedoLogMetaNode &node) { return start_log_lsn_ < node.start_log_lsn_; }
bool before(const RedoLogMetaNode &node) const { return start_log_lsn_ < node.start_log_lsn_; }
void set_data(char *data, int64_t data_len)
{
@ -330,76 +341,73 @@ private:
class IStmtTask;
class DmlStmtTask;
struct RedoLogMetaComparator
{
static int compare(const RedoLogMetaNode& a, const RedoLogMetaNode& b)
{
return (a.before(b)) ? -1 : ((b.before(a)) ? 1 : 0);
}
};
typedef SortedTreeifyList<RedoLogMetaNode, RedoLogMetaComparator> RedoNodeList;
typedef SortedTreeifyList<RedoLogMetaNode, RedoLogMetaComparator>::Iterator RedoNodeIterator;
// Ordered Redo log list
struct SortedRedoLogList
{
int32_t node_num_;
// When the data of node need be stored, than need callback to increase ready_node_num
// Otherwise, we can increase ready_node_num directly
int32_t ready_node_num_;
int32_t log_num_;
bool is_dml_stmt_iter_end_;
RedoLogMetaNode *head_;
RedoLogMetaNode *tail_;
RedoLogMetaNode *last_push_node_;
RedoLogMetaNode *cur_dispatch_redo_;
RedoLogMetaNode *cur_sort_redo_;
RedoNodeList redo_node_list_;
RedoNodeIterator cur_dispatch_redo_;
RedoNodeIterator cur_sort_redo_;
ObLink *cur_sort_stmt_;
RedoSortedProgress sorted_progress_;
SortedRedoLogList() :
node_num_(0),
SortedRedoLogList(ObIAllocator& allocator) :
ready_node_num_(0),
log_num_(0),
is_dml_stmt_iter_end_(false),
head_(NULL),
tail_(NULL),
last_push_node_(NULL),
cur_dispatch_redo_(NULL),
cur_sort_redo_(NULL),
redo_node_list_(allocator, true), /* auto_treeify_mode=true */
cur_dispatch_redo_(&redo_node_list_, nullptr),
cur_sort_redo_(&redo_node_list_, nullptr),
cur_sort_stmt_(NULL),
sorted_progress_()
{}
~SortedRedoLogList() { reset(); }
int32_t get_node_number() const { return ATOMIC_LOAD(&node_num_); }
int32_t get_node_number() const { return redo_node_list_.count(); }
int32_t get_ready_node_number() const { return ATOMIC_LOAD(&ready_node_num_); }
void inc_ready_node_num() { ATOMIC_INC(&ready_node_num_); }
int check_node_num_equality(bool &is_equal);
void reset()
{
node_num_ = 0;
ready_node_num_ = 0;
log_num_ = 0;
is_dml_stmt_iter_end_ = false;
head_ = NULL;
tail_ = NULL;
last_push_node_ = NULL;
cur_dispatch_redo_ = NULL;
cur_sort_redo_ = NULL;
cur_dispatch_redo_.reset();
cur_sort_redo_.reset();
cur_sort_stmt_ = NULL;
sorted_progress_.reset();
redo_node_list_.reset();
}
bool is_valid() const
{
return node_num_ > 0
&& log_num_ > 0
&& NULL != head_
&& NULL != tail_
&& NULL != last_push_node_;
return get_node_number() > 0;
}
OB_INLINE bool is_dispatch_finish() const { return node_num_ == sorted_progress_.get_dispatched_redo_count(); }
OB_INLINE bool is_dispatch_finish() const { return get_node_number() == sorted_progress_.get_dispatched_redo_count(); }
OB_INLINE bool has_dispatched_but_unsorted_redo() const
{ return sorted_progress_.get_dispatched_not_sort_redo_count() > 0; }
OB_INLINE void set_sorted_row_seq_no(const transaction::ObTxSEQ &row_seq_no) { sorted_progress_.set_sorted_row_seq_no(row_seq_no); }
int treeify() { return redo_node_list_.treeify(); }
int untreeify() { return redo_node_list_.untreeify(); }
RedoNodeIterator redo_iter_begin() { return redo_node_list_.begin(); }
RedoNodeIterator redo_iter_end() { return redo_node_list_.end(); }
bool is_dml_stmt_iter_end() const { return is_dml_stmt_iter_end_; }
/// Push RedoLogMetaNode with order
@ -442,25 +450,19 @@ struct SortedRedoLogList
int next_dml_stmt(ObLink *&dml_stmt_task);
TO_STRING_KV(
K_(node_num),
K_(log_num),
K_(ready_node_num),
KP_(head),
KP_(tail),
KP_(last_push_node),
"redo_sorted_progress", sorted_progress_,
KP_(cur_dispatch_redo),
KP_(cur_sort_redo),
"cur_sort_redo", static_cast<DmlRedoLogNode*>(cur_sort_redo_),
K_(cur_dispatch_redo),
K_(cur_sort_redo),
KP_(cur_sort_stmt),
K_(is_dml_stmt_iter_end));
void mark_sys_ls_dml_trans_dispatched()
{
cur_dispatch_redo_ = NULL;
cur_sort_redo_ = NULL;
cur_dispatch_redo_.reset();
cur_sort_redo_.reset();
cur_sort_stmt_ = NULL;
sorted_progress_.reset_for_sys_ls_dml_trans(node_num_);
sorted_progress_.reset_for_sys_ls_dml_trans(get_node_number());
}
};

View File

@ -589,6 +589,14 @@ struct CDCLSNComparator
return a < b;
}
};
struct LSNComparator
{
static int compare(const palf::LSN& lsn1, const palf::LSN& lsn2) {
return lsn1.val_ > lsn2.val_ ? 1 : (lsn1.val_ == lsn2.val_ ? 0 : -1);
}
};
// sort and unique lsn arr.
// NOT THREAD_SAFE
int sort_and_unique_lsn_arr(ObLogLSNArray &lsn_arr);

View File

@ -118,6 +118,14 @@ int ObRemoteLogRawReader::raw_read(const palf::LSN &start_lsn,
return ret;
}
void ObRemoteLogRawReader::update_source_cb()
{
int ret = OB_SUCCESS;
if (inited_ && OB_FAIL(update_source_func_(id_, source_guard_.get_source()))) {
CLOG_LOG(WARN, "update source failed", KPC(this));
}
}
int ObRemoteLogRawReader::raw_read_(char *buffer, const int64_t buffer_size, int64_t &total_read_size)
{
int ret = OB_SUCCESS;

View File

@ -53,7 +53,7 @@ public:
// iteraotr should be inited again to follow this change
ObRemoteLogRawReader(GetSourceFunc &get_source_func,
UpdateSourceFunc &update_source_func = DefaultUpdateSourceFunctor(),
RefreshStorageInfoFunc &refresh_storage_info_func = DefaultRefreshStorageInfoFunctor());
RefreshStorageInfoFunc &refresh_storage_info_func = DefaultRefreshStorageInfoFunctor());
virtual ~ObRemoteLogRawReader();
public:
// init remote log raw reader, as archive log directory is splitted by scn, the pre_scn is indispensable to help locate directory
@ -86,6 +86,8 @@ public:
const int64_t nbytes,
int64_t &read_size);
void update_source_cb();
TO_STRING_KV(K_(inited), K_(tenant_id), K_(id), K_(pre_scn), K_(start_lsn), K_(cur_lsn), K_(max_lsn), K_(log_ext_handler));
private:

View File

@ -29,3 +29,5 @@ libobcdc_unittest(test_ob_cdc_part_trans_resolver)
libobcdc_unittest(test_log_svr_blacklist)
libobcdc_unittest(test_ob_cdc_sorted_list)
libobcdc_unittest(test_ob_log_safe_arena)
libobcdc_unittest(test_cdc_rbtree)
libobcdc_unittest(test_cdc_sorted_list)

View File

@ -0,0 +1,333 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include <unistd.h>
#include <stdarg.h>
#include <errno.h>
#include <time.h>
#include "lib/container/ob_rbtree.h"
#include "gtest/gtest.h"
#include "logservice/palf/lsn.h"
#include "ob_log_utils.h"
#include "ob_cdc_lightly_sorted_list.h"
#include "ob_log_trans_log.h"
#define USING_LOG_PREFIX OBLOG
#define INIT_NODES \
int ret = OB_SUCCESS; \
Node* nodes = static_cast<Node*>(ob_malloc(node_cnt * sizeof(Node), "rbtree_test")); \
for(int i = 0; i < node_cnt; i++) { \
LSN lsn(i); \
new(nodes+i) Node(lsn); \
EXPECT_EQ(nodes[i].lsn_.val_, lsn.val_); \
} \
#define FREE_NODES \
for (int i = 0; i < node_cnt; i++) { \
nodes[i].~Node(); \
} \
ob_free(nodes); \
nodes = nullptr; \
namespace oceanbase
{
using namespace oceanbase::container;
using namespace oceanbase::palf;
namespace libobcdc
{
struct Node
{
Node() : lsn_(), next_(nullptr) {}
Node(LSN &lsn) : lsn_(lsn), next_(nullptr) {}
~Node() { lsn_.reset(); next_ = nullptr; }
OB_INLINE int compare(const Node *other) const
{
return lsn_.val_ - other->lsn_.val_;
}
OB_INLINE bool operator==(const Node &other) {return compare(&other) == 0;}
OB_INLINE bool operator<(const Node &other) {return compare(&other) < 0;}
OB_INLINE void set_next(Node *node) {next_ = node;}
OB_INLINE Node *get_next() const {return next_;}
RBNODE(Node, rblink);
LSN lsn_;
Node* next_;
TO_STRING_KV(K_(lsn));
};
typedef ObRbTree<Node, ObDummyCompHelper<Node>> tree_t;
static void init_nodes(int64_t node_cnt, Node *&nodes)
{
nodes = static_cast<Node*>(ob_malloc(node_cnt * sizeof(Node), "rbtree_test"));
for(int i = 0; i < node_cnt; i++) {
LSN lsn(i);
new(nodes+i) Node(lsn);
EXPECT_EQ(nodes[i].lsn_.val_, lsn.val_);
}
}
static void free_nodes(int64_t node_cnt, Node*& nodes)
{
for (int i = 0; i < node_cnt; i++) {
nodes[i].~Node();
}
ob_free(nodes);
nodes = nullptr;
}
static void build_tree(tree_t &tree, Node* nodes, int64_t node_cnt, int64_t reverse_cnt = 0, bool total_reverse = false)
{
tree.init_tree();
if (total_reverse) {
for (int i = node_cnt - 1; i >=0; i--) {
EXPECT_EQ(OB_SUCCESS, tree.insert(&nodes[i]));
}
} else {
for (int i = node_cnt - reverse_cnt; i < node_cnt; i++) {
EXPECT_EQ(OB_SUCCESS, tree.insert(&nodes[i]));
}
for (int i = 0; i < node_cnt-reverse_cnt; i++) {
EXPECT_EQ(OB_SUCCESS, tree.insert(&nodes[i]));
}
}
}
static void build_list(SortedLightyList<Node> &list, Node* nodes, int64_t node_cnt, int64_t reverse_cnt = 0, bool total_reverse = false)
{
list.reset();
if (total_reverse) {
for (int i = node_cnt - 1; i >=0; i--) {
EXPECT_EQ(OB_SUCCESS, list.push(&nodes[i]));
}
} else {
for (int i = node_cnt - reverse_cnt; i < node_cnt; i++) {
EXPECT_EQ(OB_SUCCESS, list.push(&nodes[i]));
}
for (int i = 0; i < node_cnt-reverse_cnt; i++) {
EXPECT_EQ(OB_SUCCESS, list.push(&nodes[i]));
}
}
}
static void iter_tree(tree_t &tree, int64_t node_cnt)
{
int node_cntt = 0;
Node* node = tree.get_first();
while (NULL != node) {
EXPECT_EQ(node->lsn_.val_, node_cntt);
node_cntt++;
Node *next = NULL;
tree.get_next(node, next);
node = next;
}
EXPECT_EQ(node_cntt, node_cnt);
}
static void iter_list(SortedLightyList<Node> &list, int64_t node_cnt)
{
int node_cntt = 0;
Node *node = list.get_first_node();
while (NULL != node) {
EXPECT_EQ(node->lsn_.val_, node_cntt);
node_cntt++;
Node *next = node->get_next();
node = next;
}
EXPECT_EQ(node_cntt, node_cnt);
}
TEST(TESTCDCRbTree, init_and_free)
{
LOG_INFO("========== test begin ==========");
int node_cnt = 500000;
INIT_NODES;
FREE_NODES;
}
TEST(TESTCDCRbTree, sequential_verify)
{
int node_cnt = 500000;
INIT_NODES;
int64_t start_ts = get_timestamp();
tree_t tree;
build_tree(tree, nodes, node_cnt);
int64_t built_ts = get_timestamp();
iter_tree(tree, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("sequential_verify", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
FREE_NODES;
};
TEST(TESTCDCRbTree, part_reverse_verify)
{
int node_cnt = 500000;
int reverse_cnt = 1000;
INIT_NODES;
int64_t start_ts = get_timestamp();
tree_t tree;
build_tree(tree, nodes, node_cnt, reverse_cnt);
int64_t built_ts = get_timestamp();
iter_tree(tree, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("part_reverse_verify", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
FREE_NODES;
};
TEST(TESTCDCRbTree, part_reverse_verify_5w)
{
int node_cnt = 50000;
int reverse_cnt = 1000;
INIT_NODES;
int64_t start_ts = get_timestamp();
tree_t tree;
build_tree(tree, nodes, node_cnt, reverse_cnt);
int64_t built_ts = get_timestamp();
iter_tree(tree, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("part_reverse_verify_5w", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
FREE_NODES;
};
TEST(TESTCDCRbTree, total_reverse_verify)
{
int node_cnt = 500000;
INIT_NODES;
int64_t start_ts = get_timestamp();
tree_t tree;
build_tree(tree, nodes, node_cnt, 0, true);
int64_t built_ts = get_timestamp();
iter_tree(tree, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("total_reverse_verify", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
FREE_NODES;
};
TEST(TESTCDCLightyList, sequential_verify)
{
int node_cnt = 500000;
INIT_NODES;
int64_t start_ts = get_timestamp();
SortedLightyList<Node> list(true);
build_list(list, nodes, node_cnt);
int64_t built_ts = get_timestamp();
iter_list(list, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("sequential_verify", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
list.reset();
FREE_NODES;
}
TEST(TESTCDCLightyList, part_reverse_verify)
{
int node_cnt = 5000;
int reverse_cnt = 1000;
INIT_NODES;
int64_t start_ts = get_timestamp();
SortedLightyList<Node> list(true);
build_list(list, nodes, node_cnt, reverse_cnt);
int64_t built_ts = get_timestamp();
iter_list(list, node_cnt);
int64_t verify_ts = get_timestamp();
LOG_INFO("part_reverse_verify", "build_cost", built_ts - start_ts, "verify_cost", verify_ts - built_ts);
list.reset();
FREE_NODES;
}
TEST(TESTCDCRbTree, detect_balance_node_count)
{
int64_t node_cnt = 4;
while(node_cnt++ < 50) {
Node *nodes = nullptr;
tree_t tree;
SortedLightyList<Node> list(true);
init_nodes(node_cnt, nodes);
int64_t start_ts1 = common::ObTimeUtility::current_time_ns();
build_tree(tree, nodes, node_cnt, 1);
int64_t end_ts1 = common::ObTimeUtility::current_time_ns();
free_nodes(node_cnt, nodes);
init_nodes(node_cnt, nodes);
int64_t start_ts2 = common::ObTimeUtility::current_time_ns();
build_list(list, nodes, node_cnt, 1);
int64_t end_ts2 = common::ObTimeUtility::current_time_ns();
list.reset();
free_nodes(node_cnt, nodes);
int64_t tree_build_time = (end_ts1 - start_ts1);
int64_t list_build_time = (end_ts2 - start_ts2);
bool is_tree_better = (tree_build_time < list_build_time);
if (is_tree_better) {
LOG_INFO("tree_build_time less than list_build_time", K(node_cnt), K(tree_build_time), K(list_build_time));
} else {
LOG_INFO("tree_build_time greater than list_build_time", K(node_cnt), K(tree_build_time), K(list_build_time));
}
}
};
TEST(TESTCDCRbTree, list_to_tree)
{
int64_t node_cnt = 1000000;
Node* nodes = nullptr;
init_nodes(node_cnt, nodes);
tree_t tree;
SortedLightyList<Node> list(true);
build_list(list, nodes, node_cnt);
int64_t start_ts = get_timestamp();
Node *node = list.get_first_node();
while (NULL != node) {
tree.insert(node);
Node *next = node->get_next();
node = next;
}
int64_t end_ts = get_timestamp();
list.reset();
LOG_INFO("list_to_tree", K(node_cnt), "cost_us", end_ts-start_ts);
int64_t start_ts1 = get_timestamp();
node = tree.get_first();
while (NULL != node) {
EXPECT_EQ(OB_SUCCESS, list.push(node));
Node *next = NULL;
tree.get_next(node, next);
node = next;
}
int64_t end_ts1 = get_timestamp();
LOG_INFO("tree_to_list", K(node_cnt), "cost_us", end_ts1-start_ts1);
};
}
}
int main(int argc, char **argv) {
// system("rm -rf cdc_sorted_list_test.log");
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name("cdc_sorted_list_test.log", true);
logger.set_log_level(OB_LOG_LEVEL_INFO);
testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View File

@ -0,0 +1,469 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include <unistd.h>
#include <stdarg.h>
#include <errno.h>
#include <time.h>
#include "lib/container/ob_rbtree.h"
#include "gtest/gtest.h"
#include "logservice/palf/lsn.h"
#include "ob_log_utils.h"
#include "ob_cdc_sorted_treeify_list.h"
#include "ob_cdc_sorted_list_iterator.h"
#include "math.h"
#define USING_LOG_PREFIX OBLOG
#define ASSERT_SUCC(statement) \
ASSERT_EQ(OB_SUCCESS, statement);
namespace oceanbase
{
using namespace oceanbase::container;
using namespace oceanbase::palf;
namespace libobcdc
{
int build_lsn_list(ObIArray<LSN> &lsn_arr, const int64_t lsn_cnt)
{
int ret = OB_SUCCESS;
for (int i = 0; OB_SUCC(ret) && i < lsn_cnt; ++i) {
LSN lsn(i);
if (OB_FAIL(lsn_arr.push_back(lsn))) {
LOG_ERROR("push_back lsn into array failed", KR(ret));
}
}
return ret;
}
int build_list(
const ObIArray<LSN> &lsn_array,
const int64_t node_cnt,
SortedList<LSN, LSNComparator> &list,
const int64_t reverse_cnt = 0,
const bool total_reverse = false)
{
int ret = OB_SUCCESS;
ob_assert(node_cnt <= lsn_array.count());
ob_assert(reverse_cnt <= node_cnt && reverse_cnt >= 0);
if (total_reverse) {
for (int i = node_cnt-1; OB_SUCC(ret) && i >=0 ; --i) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
} else {
for (int i = node_cnt - reverse_cnt; OB_SUCC(ret) && i < node_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
for (int i = 0; OB_SUCC(ret) && i < node_cnt - reverse_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
}
if (OB_SUCC(ret) && OB_UNLIKELY(node_cnt != list.count())) {
ret = OB_STATE_NOT_MATCH;
LOG_ERROR("list node count not match", KR(ret), K(node_cnt), K(list));
}
LOG_INFO("build_list finish", KR(ret), K(list));
return ret;
}
int iter_list_and_verify(const ObIArray<LSN> &lsn_array, const int64_t node_cnt, const SortedList<LSN, LSNComparator> &list)
{
int ret = OB_SUCCESS;
SortedList<LSN, LSNComparator>::Iterator iter = list.begin();
int64_t idx = 0;
while(OB_SUCC(ret) && iter != list.end()) {
LSN &lsn = *iter;
const LSN &arr = lsn_array.at(idx);
if (lsn != lsn_array.at(idx)) {
ret = OB_STATE_NOT_MATCH;
LOG_ERROR("value check failed", KR(ret), K(idx), K(*iter), K(lsn), K(arr), K(list));
} else {
iter++;
idx++;
}
}
if (OB_SUCC(ret)) {
if (idx != node_cnt || idx != list.count()) {
ret = OB_STATE_NOT_MATCH;
LOG_ERROR("iterate count not match", KR(ret), K(idx), K(node_cnt), K(list));
}
}
return ret;
}
int pop_and_check_val(const ObIArray<LSN> &lsn_array, const int64_t node_cnt, SortedList<LSN, LSNComparator> &list, const int64_t pop = -1)
{
int ret = OB_SUCCESS;
int64_t pop_cnt = pop <= 0 ? node_cnt : pop;
const int64_t remain_cnt = node_cnt - pop_cnt;
LSN *val_ptr = nullptr;
for (int i = 0; OB_SUCC(ret) && i < pop_cnt; i++) {
if (OB_FAIL(list.pop(val_ptr))) {
LOG_ERROR("SortedTreeifyList pop failed", KR(ret), K(i), K(list));
} else if (OB_ISNULL(val_ptr)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("val poped from list is null", KR(ret), K(i), K(list));
} else if (OB_UNLIKELY(lsn_array.at(i) != *val_ptr)) {
ret = OB_STATE_NOT_MATCH;
LOG_ERROR("value check failed", KR(ret), K(ret), K(i), KPC(val_ptr), K(list));
} else {
val_ptr = nullptr;
}
}
if (OB_SUCC(ret) && OB_UNLIKELY(!list.empty())) {
ret = OB_STATE_NOT_MATCH;
LOG_WARN("list not empry after pop", K(list));
}
return ret;
}
TEST(SortedLinkedList, init_list_and_free)
{
const int64_t node_cnt = 50000;
const int64_t local_cnt = node_cnt;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedLinkedList<LSN, LSNComparator> linked_list(allocator);
LOG_INFO("========== test SortedLinkedList begin ==========");
ObTimeGuard time_guard("init sorted linked_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, linked_list));
time_guard.click("build_linked_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, linked_list));
time_guard.click("iter_linked_list");
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, linked_list));
time_guard.click("pop_linked_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, linked_list));
time_guard.click("rebuild_linked_list");
ASSERT_EQ(OB_ENTRY_EXIST, linked_list.push(lsn_array.at(0)));
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, linked_list));
time_guard.click("pop_linked_list after rebuild");
linked_list.reset();
linked_list.reset();
ASSERT_TRUE(linked_list.empty());
ASSERT_TRUE(linked_list.is_list_empty());
LOG_INFO("init_list_and_free finish", K(node_cnt), K(time_guard));
}
TEST(SortedTreeifyList, init_list_and_free)
{
const int64_t node_cnt = 5;
const int64_t local_cnt = node_cnt;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedTreeifyList<LSN, LSNComparator> treeify_list(allocator);
LOG_INFO("========== test SortedLinkedList begin ==========");
ObTimeGuard time_guard("init sorted linked_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("build_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_list");
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
time_guard.click("pop_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("rebuild_list");
ASSERT_EQ(OB_ENTRY_EXIST, treeify_list.push(lsn_array.at(0)));
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
time_guard.click("pop_list after rebuild");
treeify_list.reset();
treeify_list.reset();
ASSERT_TRUE(treeify_list.empty());
LOG_INFO("init_list_and_free finish", K(node_cnt), K(time_guard));
}
TEST(SortedTreeifyList, init_treeify_list_and_free_manual_treeify_mode)
{
const int64_t node_cnt = 3;
const int64_t local_cnt = node_cnt;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedTreeifyList<LSN, LSNComparator> treeify_list(allocator, false/* manual_treeify_mode */);
LOG_INFO("========== test SortedTreeifyList manual_treeify_mode begin ==========");
ObTimeGuard time_guard("init sorted treeify_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("build_treeify_list manual_treeify_mode");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
ASSERT_EQ(OB_ENTRY_EXIST, treeify_list.push(lsn_array.at(0)));
time_guard.click("iter_treeify_list manual_treeify_mode");
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
time_guard.click("pop_treeify_list manual_treeify_mode");
treeify_list.reset();
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("rebuild_treeify_list manual_treeify_mode");
ASSERT_SUCC(treeify_list.treeify());
time_guard.click("treeify list manual_treeify_mode");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_list after treeify manual_treeify_mode");
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
time_guard.click("pop_list after treeify");
treeify_list.reset();
treeify_list.reset();
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
ASSERT_SUCC(treeify_list.treeify());
ASSERT_EQ(OB_ENTRY_EXIST, treeify_list.push(lsn_array.at(0)));
ASSERT_SUCC(treeify_list.untreeify());
ASSERT_EQ(OB_ENTRY_EXIST, treeify_list.push(lsn_array.at(0)));
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
treeify_list.reset();
treeify_list.reset();
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
ASSERT_SUCC(treeify_list.treeify());
treeify_list.reset();
treeify_list.reset();
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
ASSERT_SUCC(treeify_list.treeify());
ASSERT_SUCC(treeify_list.untreeify());
ASSERT_SUCC(treeify_list.treeify());
ASSERT_SUCC(treeify_list.untreeify());
ASSERT_SUCC(treeify_list.treeify());
ASSERT_EQ(OB_ENTRY_EXIST, treeify_list.push(lsn_array.at(0)));
treeify_list.reset();
treeify_list.reset();
ASSERT_TRUE(treeify_list.empty());
LOG_INFO("init_treeify_list_and_free_manual_treeify_mode finish", K(node_cnt), K(time_guard));
}
TEST(SortedTreeifyList, manual_treeify)
{
int ret = OB_SUCCESS;
const int64_t node_cnt = 500000;
const int64_t local_cnt = node_cnt;
const int64_t reverse_cnt = 1000;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedTreeifyList<LSN, LSNComparator> treeify_list(allocator, false/* manual_treeify_mode */);
LOG_INFO("========== test SortedTreeifyList manual_treeify_mode begin ==========", "sizeof", sizeof(SortedTreeifyList<LSN, LSNComparator>::NodeType));
ObTimeGuard time_guard("init sorted treeify_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("build_treeify_list manual_treeify_mode");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_treeify_list manual_treeify_mode");
treeify_list.reset();
time_guard.click("reset_treeify_list manual_treeify_mode");
// test manually
for (int i = node_cnt - reverse_cnt; OB_SUCC(ret) && i < node_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(treeify_list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
time_guard.click("push_reverse_nodes");
treeify_list.treeify();
time_guard.click("treeify");
for (int i = 0; OB_SUCC(ret) && i < node_cnt - reverse_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(treeify_list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
time_guard.click("push_remain_nodes");
treeify_list.untreeify();
time_guard.click("untreeify");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_list finish");
LOG_INFO("manual_treeify finish", K(node_cnt), K(reverse_cnt), K(allocator), K(time_guard));
}
TEST(SortedTreeifyList, manual_treeify_from_middle)
{
int ret = OB_SUCCESS;
const int64_t node_cnt = 500000;
const int64_t local_cnt = node_cnt;
const int64_t start_idx = node_cnt / 2;
const int64_t reverse_cnt = 1000;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedTreeifyList<LSN, LSNComparator> treeify_list(allocator, false/* manual_treeify_mode */);
LOG_INFO("========== test SortedTreeifyList manual_treeify_mode from middle begin ==========", "sizeof", sizeof(SortedTreeifyList<LSN, LSNComparator>::NodeType));
ObTimeGuard time_guard("init sorted treeify_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("build_treeify_list manual_treeify_mode");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_treeify_list manual_treeify_mode");
treeify_list.reset();
time_guard.click("reset_treeify_list manual_treeify_mode");
// test manually
for (int i = start_idx; OB_SUCC(ret) && i < start_idx + reverse_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(treeify_list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
ASSERT_TRUE((lsn_array.at(start_idx) == *(treeify_list.top())));
time_guard.click("push_reverse_nodes");
treeify_list.treeify();
ASSERT_TRUE((lsn_array.at(start_idx) == *(treeify_list.top())));
time_guard.click("treeify");
for (int i = 0; OB_SUCC(ret) && i < start_idx; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(treeify_list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
time_guard.click("push_front_nodes");
treeify_list.untreeify();
time_guard.click("untreeify");
for (int i = start_idx + reverse_cnt; OB_SUCC(ret) && i < node_cnt; i++) {
const LSN &lsn = lsn_array.at(i);
if (OB_FAIL(treeify_list.push(const_cast<LSN&>(lsn)))) {
LOG_ERROR("SortedTreeifyList push failed", KR(ret), K(lsn), K(node_cnt));
}
}
time_guard.click("push_remain_nodes");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_list finish");
LOG_INFO("manual_treeify finish", K(node_cnt), K(reverse_cnt), K(allocator), K(time_guard));
}
TEST(SortedTreeifyList, init_treeify_list_and_free_auto_treeify_mode)
{
const int64_t node_cnt = 5000000;
const int64_t local_cnt = 1000000;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedTreeifyList<LSN, LSNComparator> treeify_list(allocator, true/*auto_treeify_mode*/);
LOG_INFO("========== test SortedTreeifyList auto_treeify_mode begin ==========");
ObTimeGuard time_guard("init sorted treeify_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list));
time_guard.click("build_treeify_list auto_treeify_mode");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list));
time_guard.click("iter_treeify_list auto_treeify_mode");
ASSERT_SUCC(pop_and_check_val(lsn_array, node_cnt, treeify_list));
time_guard.click("pop_treeify_list auto_treeify_mode");
treeify_list.reset();
treeify_list.reset();
ASSERT_TRUE(treeify_list.empty());
LOG_INFO("init_treeify_list_and_free_auto_treeify_mode finish", K(node_cnt), K(time_guard));
}
TEST(SortedLinkedList, sequential_verify)
{
const int64_t node_cnt = 50000;
const int64_t local_cnt = node_cnt;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedLinkedList<LSN, LSNComparator> linked_list(allocator);
SortedTreeifyList<LSN, LSNComparator> treeify_list_1(allocator, false/*manual_treeify_mode*/);
SortedTreeifyList<LSN, LSNComparator> treeify_list_2(allocator, true/*auto_treeify_mode*/);
LOG_INFO("========== test SortedLinkedList begin sequential_verify ==========");
ObTimeGuard time_guard("sequential_verify linked_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, linked_list));
time_guard.click("build_linked_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list_1));
time_guard.click("build_manual_treeify_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list_2));
time_guard.click("build_auto_treeify_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, linked_list));
time_guard.click("iter_linked_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list_1));
time_guard.click("iter_manual_treeify_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list_2));
time_guard.click("iter_auto_treeify_list");
linked_list.reset();
time_guard.click("reset_linked_list");
treeify_list_1.reset();
time_guard.click("reset_manual_treeify_list");
treeify_list_2.reset();
time_guard.click("reset_auto_treeify_list");
LOG_INFO("init_treeify_list_and_free_auto_treeify_mode sequential_verify finish", K(node_cnt), K(time_guard));
}
TEST(SortedLinkedList, part_reverse_verify)
{
const int64_t node_cnt = 5000;
const int64_t local_cnt = node_cnt;
const int64_t reverse_cnt = 100;
ObArenaAllocator allocator;
ObSEArray<LSN, local_cnt> lsn_array;
SortedLinkedList<LSN, LSNComparator> linked_list(allocator);
SortedTreeifyList<LSN, LSNComparator> treeify_list_1(allocator, false/*manual_treeify_mode*/);
SortedTreeifyList<LSN, LSNComparator> treeify_list_2(allocator, true/*auto_treeify_mode*/);
LOG_INFO("========== test SortedLinkedList part_reverse_verify begin ==========");
ObTimeGuard time_guard("sequential_verify linked_list");
ASSERT_SUCC(build_lsn_list(lsn_array, node_cnt));
time_guard.click("build_lsn_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, linked_list, reverse_cnt));
time_guard.click("build_linked_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list_1, reverse_cnt));
time_guard.click("build_manual_treeify_list");
ASSERT_SUCC(build_list(lsn_array, node_cnt, treeify_list_2, reverse_cnt));
time_guard.click("build_auto_treeify_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, linked_list));
time_guard.click("iter_linked_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list_1));
time_guard.click("iter_manual_treeify_list");
ASSERT_SUCC(iter_list_and_verify(lsn_array, node_cnt, treeify_list_2));
time_guard.click("iter_auto_treeify_list");
linked_list.reset();
time_guard.click("reset_linked_list");
treeify_list_1.reset();
time_guard.click("reset_manual_treeify_list");
treeify_list_2.reset();
time_guard.click("reset_auto_treeify_list");
LOG_INFO("init_treeify_list_and_free_auto_treeify_mode part_reverse_verify finish", K(node_cnt), K(reverse_cnt), K(time_guard));
}
}
}
int main(int argc, char **argv) {
system("rm -rf cdc_sorted_treeify_list.log");
ObLogger &logger = ObLogger::get_logger();
logger.set_file_name("cdc_sorted_treeify_list.log", true);
logger.set_log_level("DEBUG");
oceanbase::lib::reload_trace_log_config(true);
oceanbase::lib::reload_diagnose_info_config(true);
testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}