diff --git a/deps/oblib/src/lib/container/ob_rbtree.h b/deps/oblib/src/lib/container/ob_rbtree.h index e86b62582..ae224b156 100644 --- a/deps/oblib/src/lib/container/ob_rbtree.h +++ b/deps/oblib/src/lib/container/ob_rbtree.h @@ -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(pathp) >= reinterpret_cast(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(pathp) >= reinterpret_cast(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; diff --git a/src/logservice/cdcservice/ob_cdc_fetcher.cpp b/src/logservice/cdcservice/ob_cdc_fetcher.cpp index 9a5563d73..5ce19ed4e 100644 --- a/src/logservice/cdcservice/ob_cdc_fetcher.cpp +++ b/src/logservice/cdcservice/ob_cdc_fetcher.cpp @@ -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(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; } diff --git a/src/logservice/cdcservice/ob_cdc_fetcher.h b/src/logservice/cdcservice/ob_cdc_fetcher.h index fa1528fd0..553a17211 100644 --- a/src/logservice/cdcservice/ob_cdc_fetcher.h +++ b/src/logservice/cdcservice/ob_cdc_fetcher.h @@ -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) { diff --git a/src/logservice/cdcservice/ob_cdc_req.h b/src/logservice/cdcservice/ob_cdc_req.h index 8a5966027..fdae1d9d6 100644 --- a/src/logservice/cdcservice/ob_cdc_req.h +++ b/src/logservice/cdcservice/ob_cdc_req.h @@ -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_; } diff --git a/src/logservice/cdcservice/ob_cdc_service.cpp b/src/logservice/cdcservice/ob_cdc_service.cpp index ba63cd10b..a9d9589a1 100644 --- a/src/logservice/cdcservice/ob_cdc_service.cpp +++ b/src/logservice/cdcservice/ob_cdc_service.cpp @@ -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)); diff --git a/src/logservice/libobcdc/src/CMakeLists.txt b/src/logservice/libobcdc/src/CMakeLists.txt index f7bf97e1d..e3f3d7ae8 100644 --- a/src/logservice/libobcdc/src/CMakeLists.txt +++ b/src/logservice/libobcdc/src/CMakeLists.txt @@ -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 diff --git a/src/logservice/libobcdc/src/ob_cdc_define.cpp b/src/logservice/libobcdc/src/ob_cdc_define.cpp index fdb335c78..df2d85060 100644 --- a/src/logservice/libobcdc/src/ob_cdc_define.cpp +++ b/src/logservice/libobcdc/src/ob_cdc_define.cpp @@ -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; diff --git a/src/logservice/libobcdc/src/ob_cdc_define.h b/src/logservice/libobcdc/src/ob_cdc_define.h index 9a671e77b..17f88854a 100644 --- a/src/logservice/libobcdc/src/ob_cdc_define.h +++ b/src/logservice/libobcdc/src/ob_cdc_define.h @@ -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(); diff --git a/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.cpp b/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.cpp new file mode 100644 index 000000000..1925d943b --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.cpp @@ -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 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 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 &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 &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 &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 &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 ¶m = 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 diff --git a/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.h b/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.h new file mode 100644 index 000000000..ec5473340 --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_miss_log_handler.h @@ -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 &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 &miss_log_array, + FetchLogSRpc &fetch_srpc, + volatile bool &stop_flag); + int fetch_miss_log_( + const ObIArray &miss_log_array, + const int64_t timeout, + MissLogTask &misslog_task, + FetchLogSRpc &fetch_srpc, + volatile bool &stop_flag); + int fetch_miss_log_direct_( + const ObIArray &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 diff --git a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp index 4cb660b1d..0df7f33e5 100644 --- a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp +++ b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.cpp @@ -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))) { diff --git a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.h b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.h index 01db14b5c..4fe5fc72d 100644 --- a/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.h +++ b/src/logservice/libobcdc/src/ob_cdc_part_trans_resolver.h @@ -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: diff --git a/src/logservice/libobcdc/src/ob_cdc_sorted_linked_list.h b/src/logservice/libobcdc/src/ob_cdc_sorted_linked_list.h new file mode 100644 index 000000000..1d59d842a --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_sorted_linked_list.h @@ -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 > +class SortedLinkedList : public SortedList +{ +public: + typedef ListNode ListNodeType; + typedef SortedList 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(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 diff --git a/src/logservice/libobcdc/src/ob_cdc_sorted_list.h b/src/logservice/libobcdc/src/ob_cdc_sorted_list.h new file mode 100644 index 000000000..6bd0b63d2 --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_sorted_list.h @@ -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 +struct DefaultValComparator +{ + static int compare(const T &a, const T &b) { return a.compare(b); } +}; + +template +class ListNode +{ +public: + typedef ListNode 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 > +class SortedListIterator; + +// interface for sorted list +template > +class SortedList +{ + friend class SortedListIterator; + typedef ListNode 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 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 \ No newline at end of file diff --git a/src/logservice/libobcdc/src/ob_cdc_sorted_list_iterator.h b/src/logservice/libobcdc/src/ob_cdc_sorted_list_iterator.h new file mode 100644 index 000000000..0e26ee688 --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_sorted_list_iterator.h @@ -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 SortedList::Iterator SortedList::begin() const +{ + return Iterator(this, get_first_node()); +} + +template +typename SortedList::Iterator SortedList::end() const +{ + return Iterator(this, 0); +} + +template +class SortedListIterator +{ + typedef ListNode 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 *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* list_; + ListNode* node_ptr_; +}; + +} // end namespace libobcdc +} // end namespace oceanbase + + +#endif \ No newline at end of file diff --git a/src/logservice/libobcdc/src/ob_cdc_sorted_treeify_list.h b/src/logservice/libobcdc/src/ob_cdc_sorted_treeify_list.h new file mode 100644 index 000000000..b5d36ba94 --- /dev/null +++ b/src/logservice/libobcdc/src/ob_cdc_sorted_treeify_list.h @@ -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 +class TreeNode : public ListNode +{ + public: + typedef TreeNode NodeType; + explicit TreeNode(T &val) : ListNode(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> +class SortedTreeifyList : public SortedList +{ + friend class SortedLinkedList; +public: + typedef TreeNode NodeType; + typedef ListNode BaseNodeType; + typedef SortedList 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(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(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(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(&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(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(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> tree_t; + // allocator to alloc memory for NODE + ObIAllocator &allocator_; + bool auto_treeify_mode_; + bool is_tree_mode_; + SortedLinkedList 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 diff --git a/src/logservice/libobcdc/src/ob_log_cluster_id_filter.cpp b/src/logservice/libobcdc/src/ob_log_cluster_id_filter.cpp index 01e17cb42..100287462 100644 --- a/src/logservice/libobcdc/src/ob_log_cluster_id_filter.cpp +++ b/src/logservice/libobcdc/src/ob_log_cluster_id_filter.cpp @@ -127,7 +127,7 @@ void ObLogClusterIDFilter::stat_ignored_tps() if (last_stat_time_ > 0 && delta_time > 0) { double tps = static_cast(cur_count - last_count) / static_cast(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; diff --git a/src/logservice/libobcdc/src/ob_log_config.h b/src/logservice/libobcdc/src/ob_log_config.h index 523d41c56..2b85a7e98 100644 --- a/src/logservice/libobcdc/src/ob_log_config.h +++ b/src/logservice/libobcdc/src/ob_log_config.h @@ -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 diff --git a/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.cpp b/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.cpp index d2266f8f4..e78d13559 100644 --- a/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.cpp +++ b/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.cpp @@ -66,14 +66,15 @@ int FetchLogSRpc::fetch_log(IObLogRpc &rpc, const share::ObLSID &ls_id, const ObIArray &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 &miss_log_array) + const ObIArray &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(getpid())); + req_.set_progress(progress); ARRAY_FOREACH_N(miss_log_array, idx, count) { const ObCdcLSFetchMissLogReq::MissLogParam &miss_log_param = miss_log_array.at(idx); diff --git a/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.h b/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.h index 1b4d86f72..1ed07adae 100644 --- a/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.h +++ b/src/logservice/libobcdc/src/ob_log_fetch_log_rpc.h @@ -55,7 +55,8 @@ public: const share::ObLSID &ls_id, const ObIArray &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 &miss_log_array); + const ObIArray &miss_log_array, + const int64_t progress); private: ////////////////////////////// RpcCB ////////////////////////////// diff --git a/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.cpp b/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.cpp index b11d95749..cf2baa421 100644 --- a/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.cpp +++ b/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.cpp @@ -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; diff --git a/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.h b/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.h index f876d3807..4f47a3f5d 100644 --- a/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.h +++ b/src/logservice/libobcdc/src/ob_log_ls_fetch_ctx.h @@ -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; diff --git a/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.cpp b/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.cpp index bbd6f8b5d..48dfb255d 100644 --- a/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.cpp +++ b/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.cpp @@ -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 &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 ¶m = 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 &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 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 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 &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; diff --git a/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.h b/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.h index c4f107350..000bb9cd2 100644 --- a/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.h +++ b/src/logservice/libobcdc/src/ob_log_ls_fetch_stream.h @@ -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 &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 &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 &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_( diff --git a/src/logservice/libobcdc/src/ob_log_part_trans_parser.cpp b/src/logservice/libobcdc/src/ob_log_part_trans_parser.cpp index d138f72e3..f06aa9758 100644 --- a/src/logservice/libobcdc/src/ob_log_part_trans_parser.cpp +++ b/src/logservice/libobcdc/src/ob_log_part_trans_parser.cpp @@ -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(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(redo_node->get_next()); + DdlRedoLogNode *ddl_redo = static_cast(&(*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; } diff --git a/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp b/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp index 39f21c920..7a7f12f3f 100644 --- a/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp +++ b/src/logservice/libobcdc/src/ob_log_part_trans_task.cpp @@ -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; diff --git a/src/logservice/libobcdc/src/ob_log_part_trans_task.h b/src/logservice/libobcdc/src/ob_log_part_trans_task.h index 6c387c8dd..3323cfbbf 100644 --- a/src/logservice/libobcdc/src/ob_log_part_trans_task.h +++ b/src/logservice/libobcdc/src/ob_log_part_trans_task.h @@ -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 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); }; diff --git a/src/logservice/libobcdc/src/ob_log_resource_collector.cpp b/src/logservice/libobcdc/src/ob_log_resource_collector.cpp index b78b26d9e..ef71cf67a 100644 --- a/src/logservice/libobcdc/src/ob_log_resource_collector.cpp +++ b/src/logservice/libobcdc/src/ob_log_resource_collector.cpp @@ -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(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(&(*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(dml_redo_node->get_next()); + redo_iter++; } } diff --git a/src/logservice/libobcdc/src/ob_log_timezone_info_getter.h b/src/logservice/libobcdc/src/ob_log_timezone_info_getter.h index 53606ecea..7ab5df52e 100644 --- a/src/logservice/libobcdc/src/ob_log_timezone_info_getter.h +++ b/src/logservice/libobcdc/src/ob_log_timezone_info_getter.h @@ -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 ObLogTZInfoMap; class ObCDCTimeZoneInfoGetter : public IObCDCTimeZoneInfoGetter { diff --git a/src/logservice/libobcdc/src/ob_log_trans_log.cpp b/src/logservice/libobcdc/src/ob_log_trans_log.cpp index 32dabf141..179cc19dd 100644 --- a/src/logservice/libobcdc/src/ob_log_trans_log.cpp +++ b/src/logservice/libobcdc/src/ob_log_trans_log.cpp @@ -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(cur_sort_redo_))) { + if (OB_ISNULL(dml_redo_node = static_cast(&(*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) { diff --git a/src/logservice/libobcdc/src/ob_log_trans_log.h b/src/logservice/libobcdc/src/ob_log_trans_log.h index 917268fac..5a45a3b01 100644 --- a/src/logservice/libobcdc/src/ob_log_trans_log.h +++ b/src/logservice/libobcdc/src/ob_log_trans_log.h @@ -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 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 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 RedoNodeList; +typedef SortedTreeifyList::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(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()); } }; diff --git a/src/logservice/libobcdc/src/ob_log_utils.h b/src/logservice/libobcdc/src/ob_log_utils.h index 54fed901e..03dba3ddc 100644 --- a/src/logservice/libobcdc/src/ob_log_utils.h +++ b/src/logservice/libobcdc/src/ob_log_utils.h @@ -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); diff --git a/src/logservice/restoreservice/ob_remote_log_raw_reader.cpp b/src/logservice/restoreservice/ob_remote_log_raw_reader.cpp index bdbb68e90..933ead776 100644 --- a/src/logservice/restoreservice/ob_remote_log_raw_reader.cpp +++ b/src/logservice/restoreservice/ob_remote_log_raw_reader.cpp @@ -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; diff --git a/src/logservice/restoreservice/ob_remote_log_raw_reader.h b/src/logservice/restoreservice/ob_remote_log_raw_reader.h index d3eb4a408..1d53d04f3 100644 --- a/src/logservice/restoreservice/ob_remote_log_raw_reader.h +++ b/src/logservice/restoreservice/ob_remote_log_raw_reader.h @@ -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: diff --git a/unittest/libobcdc/CMakeLists.txt b/unittest/libobcdc/CMakeLists.txt index a82bd4b65..0e450190d 100644 --- a/unittest/libobcdc/CMakeLists.txt +++ b/unittest/libobcdc/CMakeLists.txt @@ -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) diff --git a/unittest/libobcdc/test_cdc_rbtree.cpp b/unittest/libobcdc/test_cdc_rbtree.cpp new file mode 100644 index 000000000..43837e5b3 --- /dev/null +++ b/unittest/libobcdc/test_cdc_rbtree.cpp @@ -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 +#include +#include +#include +#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(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> tree_t; + +static void init_nodes(int64_t node_cnt, Node *&nodes) +{ + nodes = static_cast(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 &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 &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 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 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 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 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(); +} diff --git a/unittest/libobcdc/test_cdc_sorted_list.cpp b/unittest/libobcdc/test_cdc_sorted_list.cpp new file mode 100644 index 000000000..31bf29284 --- /dev/null +++ b/unittest/libobcdc/test_cdc_sorted_list.cpp @@ -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 +#include +#include +#include +#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_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_array, + const int64_t node_cnt, + SortedList &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)))) { + 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)))) { + 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)))) { + 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_array, const int64_t node_cnt, const SortedList &list) +{ + int ret = OB_SUCCESS; + SortedList::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_array, const int64_t node_cnt, SortedList &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_array; + SortedLinkedList 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_array; + SortedTreeifyList 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_array; + SortedTreeifyList 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_array; + SortedTreeifyList treeify_list(allocator, false/* manual_treeify_mode */); + LOG_INFO("========== test SortedTreeifyList manual_treeify_mode begin ==========", "sizeof", sizeof(SortedTreeifyList::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)))) { + 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)))) { + 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_array; + SortedTreeifyList treeify_list(allocator, false/* manual_treeify_mode */); + LOG_INFO("========== test SortedTreeifyList manual_treeify_mode from middle begin ==========", "sizeof", sizeof(SortedTreeifyList::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)))) { + 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)))) { + 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)))) { + 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_array; + SortedTreeifyList 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_array; + SortedLinkedList linked_list(allocator); + SortedTreeifyList treeify_list_1(allocator, false/*manual_treeify_mode*/); + SortedTreeifyList 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_array; + SortedLinkedList linked_list(allocator); + SortedTreeifyList treeify_list_1(allocator, false/*manual_treeify_mode*/); + SortedTreeifyList 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(); +} \ No newline at end of file