723 lines
27 KiB
C++
723 lines
27 KiB
C++
/**
|
|
* 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.
|
|
*/
|
|
|
|
#include "ob_external_start_log_locator.h"
|
|
#include "storage/ob_partition_service.h"
|
|
#include "storage/ob_pg_storage.h"
|
|
#include "clog/ob_log_file_pool.h"
|
|
#include "clog/ob_log_reader_interface.h"
|
|
#include "clog/ob_i_log_engine.h"
|
|
#include "clog/ob_log_reader_interface.h"
|
|
#include "clog/ob_partition_log_service.h"
|
|
|
|
namespace oceanbase {
|
|
using namespace common;
|
|
using namespace obrpc;
|
|
using namespace clog;
|
|
using namespace storage;
|
|
namespace logservice {
|
|
|
|
int ObExtRpcQit::init(const int64_t deadline)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// a small engouh timestamp, used to avoid misuse time_interval and timestamp
|
|
const int64_t BASE_DEADLINE = 1000000000000000; // 2001-09-09
|
|
if (OB_UNLIKELY(deadline <= BASE_DEADLINE)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
EXTLOG_LOG(WARN, "invalid deadline", K(ret), K(deadline));
|
|
} else {
|
|
deadline_ = deadline;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObExtRpcQit::should_hurry_quit() const
|
|
{
|
|
bool bool_ret = false;
|
|
|
|
if (OB_LIKELY(common::OB_INVALID_TIMESTAMP != deadline_)) {
|
|
int64_t now = ObTimeUtility::current_time();
|
|
bool_ret = (now > deadline_ - RESERVED_INTERVAL);
|
|
}
|
|
|
|
return bool_ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::init(ObPartitionService* partition_service, ObILogEngine* log_engine)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(partition_service) || OB_ISNULL(log_engine)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
EXTLOG_LOG(WARN, "ObExtStartLogLocatorForDir init error", K(ret), KP(partition_service), KP(log_engine));
|
|
} else {
|
|
partition_service_ = partition_service;
|
|
log_engine_ = log_engine;
|
|
is_inited_ = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
void ObExtStartLogLocatorForDir::destroy()
|
|
{
|
|
if (is_inited_) {
|
|
is_inited_ = false;
|
|
partition_service_ = NULL;
|
|
log_engine_ = NULL;
|
|
}
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::build_search_map(
|
|
const ObLocateByTsReq& req_msg, SearchParam& search_param, SearchMap& search_map)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObLocateByTsReq::ParamArray& params = req_msg.get_params();
|
|
const int64_t count = params.count();
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
|
const ObLocateByTsReq::Param& param = params[i];
|
|
const ObPartitionKey& pkey = param.pkey_;
|
|
char* buf = static_cast<char*>(search_param.map_allocator_->alloc(sizeof(SearchStatus)));
|
|
if (OB_UNLIKELY(NULL == buf)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
EXTLOG_LOG(ERROR, "log external executor alloc memory error", K(ret));
|
|
} else {
|
|
SearchStatus* search_status = new (buf) SearchStatus(param.start_tstamp_);
|
|
if (OB_FAIL(search_map.insert(pkey, search_status))) {
|
|
EXTLOG_LOG(WARN, "search map insert error", K(ret), K(pkey));
|
|
} else {
|
|
EXTLOG_LOG(TRACE, "search map insert success", K(pkey), KPC(search_status));
|
|
}
|
|
}
|
|
} // end for
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::get_last_slide_log(const ObPartitionKey& pkey, uint64_t& last_log_id, int64_t& last_ts)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObIPartitionLogService* pls = NULL;
|
|
ObIPartitionGroupGuard guard;
|
|
uint64_t start_log_id = OB_INVALID_ID;
|
|
uint64_t end_log_id = OB_INVALID_ID;
|
|
int64_t start_ts = OB_INVALID_TIMESTAMP;
|
|
int64_t end_ts = OB_INVALID_TIMESTAMP;
|
|
if (OB_FAIL(partition_service_->get_partition(pkey, guard)) || NULL == guard.get_partition_group() ||
|
|
NULL == (pls = guard.get_partition_group()->get_log_service())) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
} else if (!(guard.get_partition_group()->is_valid())) {
|
|
ret = OB_INVALID_PARTITION;
|
|
} else if (guard.get_partition_group()->get_pg_storage().is_restore()) {
|
|
ret = OB_STATE_NOT_MATCH;
|
|
EXTLOG_LOG(ERROR, "partition group is not in normal status", KR(ret), K(pkey));
|
|
} else {
|
|
if (OB_FAIL(pls->get_log_id_range(start_log_id, start_ts, end_log_id, end_ts))) {
|
|
EXTLOG_LOG(WARN, "get log id range from pls error", K(ret), K(pkey));
|
|
} else {
|
|
last_log_id = end_log_id;
|
|
last_ts = end_ts;
|
|
if (OB_UNLIKELY(OB_INVALID_ID == last_log_id) || OB_UNLIKELY(OB_INVALID_TIMESTAMP == last_ts)) {
|
|
// sw has initialized last_slide to last_replay_log_id/ts
|
|
ret = OB_ERR_UNEXPECTED;
|
|
EXTLOG_LOG(ERROR, "get invalid last log id or ts", K(ret), K(last_log_id), K(last_ts));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::handle_cold_pkey_by_sw_(const ObPartitionKey& pkey, SearchStatus& search_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t start_ts = search_status.start_ts_;
|
|
uint64_t last_slide_log_id = OB_INVALID_ID;
|
|
int64_t last_slide_log_ts = OB_INVALID_TIMESTAMP;
|
|
if (OB_FAIL(get_last_slide_log(pkey, last_slide_log_id, last_slide_log_ts))) {
|
|
if (OB_PARTITION_NOT_EXIST == ret || OB_INVALID_PARTITION == ret) {
|
|
EXTLOG_LOG(INFO, "get_last_slide_log partition not exist or invalid", K(ret), K(pkey));
|
|
// partition not exists, not cold partition. Confirm by scanning InfoBlock
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
EXTLOG_LOG(WARN, "get_last_slide_log error", K(ret), K(pkey));
|
|
}
|
|
} else if (last_slide_log_ts <= start_ts) {
|
|
// If the target timestamp is greater than the maximum log timestamp,
|
|
// it is safe to return the next log as the startup log.
|
|
// BAD CASE is: if visited server is a backward backup machine, there will be many rollbacks
|
|
//
|
|
// analysis of several situations:
|
|
// Note 1:
|
|
// liboblog will pull logs from start_log_id (including this start_log_id),
|
|
// last_slide_log_id is a safe result for liboblog, actually, even last_slide_log_id + 1 is safe as well.
|
|
// Here should return last_slide_log_id + 1 as result.
|
|
// Because for cold partitions, pulling the last_slide_log_id log may trigger IlogStorage to load older files.
|
|
// If there are more cold partitions, it may cause serious IlogStorage pollution and poor locality.
|
|
//
|
|
// Note 2:
|
|
// liboblog should ask the master first, or should ask all copies,
|
|
// Otherwise, if you ask about a backward standby machine,
|
|
// it may cause the given security log_id to be too small.
|
|
//
|
|
// Note 3:
|
|
// For the following Corner Case, the code logic is also applicable:
|
|
// The partition has just been created, and no logs have been slid out.
|
|
// At this time, last_slide_log_id = 0, last_slide_log_ts = 0.
|
|
uint64_t start_log_id = last_slide_log_id + 1;
|
|
int64_t start_log_ts = last_slide_log_ts;
|
|
|
|
search_status.mark_finish(OB_SUCCESS, start_log_id, start_log_ts);
|
|
|
|
EXTLOG_LOG(INFO,
|
|
"cold pkey locate success, last_slide_log_id is safe",
|
|
K(pkey),
|
|
K(search_status),
|
|
K(last_slide_log_id),
|
|
K(last_slide_log_ts));
|
|
} else {
|
|
// The maximum log timestamp of the partition is greater than the target timestamp, set the upper bound value
|
|
search_status.update_min_greater_log(last_slide_log_id, last_slide_log_ts);
|
|
|
|
EXTLOG_LOG(TRACE,
|
|
"not cold pkey, last_slide_log_ts > start_ts",
|
|
K(pkey),
|
|
K(start_ts),
|
|
K(last_slide_log_ts),
|
|
K(last_slide_log_id),
|
|
K(search_status));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::locate_from_last_file_(
|
|
const ObPartitionKey& pkey, const int64_t start_ts, const uint64_t min_log_id, SearchStatus& search_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (OB_UNLIKELY(!pkey.is_valid()) || OB_UNLIKELY(OB_INVALID_TIMESTAMP == start_ts) ||
|
|
OB_UNLIKELY(OB_INVALID_ID == min_log_id)) {
|
|
EXTLOG_LOG(WARN, "invalid argument", K(pkey), K(start_ts), K(min_log_id));
|
|
ret = OB_INVALID_ARGUMENT;
|
|
} else {
|
|
bool finished = false;
|
|
|
|
// Polling all logs of log_index, find the first log greater than or equal to start_ts
|
|
for (uint64_t cur_log_id = min_log_id; OB_SUCCESS == ret && !finished; cur_log_id++) {
|
|
ObLogCursorExt cursor;
|
|
|
|
if (OB_FAIL(log_engine_->get_cursor(pkey, cur_log_id, cursor))) {
|
|
if (OB_ERR_OUT_OF_LOWER_BOUND == ret || OB_ERR_OUT_OF_UPPER_BOUND == ret || OB_PARTITION_NOT_EXIST == ret) {
|
|
EXTLOG_LOG(TRACE, "get_cursor out of range", K(ret), K(pkey), K(cur_log_id), K(min_log_id), K(start_ts));
|
|
} else {
|
|
EXTLOG_LOG(WARN, "pls get_cursor fail", K(ret), K(pkey), K(cur_log_id));
|
|
}
|
|
} else {
|
|
// Find the first log greater than or equal to the start timestamp
|
|
if (cursor.get_submit_timestamp() >= start_ts) {
|
|
search_status.mark_finish(OB_SUCCESS, cur_log_id, cursor.get_submit_timestamp());
|
|
finished = true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// Locate the log id from the last ILOG
|
|
//
|
|
// If a file is cut during execution, new flying InfoBlock can be treated as an old file simply.
|
|
// When locating from IlogStorage later, the target file will be scanned, so it is safe not to process it here.
|
|
int ObExtStartLogLocatorForDir::locate_pkey_from_flying_ilog_(const ObPartitionKey& pkey, SearchStatus& search_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t start_ts = search_status.start_ts_;
|
|
uint64_t info_min_log_id = OB_INVALID_ID;
|
|
int64_t info_min_ts = OB_INVALID_TIMESTAMP;
|
|
|
|
if (OB_ISNULL(log_engine_)) {
|
|
ret = OB_NOT_INIT;
|
|
EXTLOG_LOG(ERROR, "not init", K(ret), KP(log_engine_));
|
|
} else {
|
|
if (OB_FAIL(log_engine_->get_ilog_memstore_min_log_id_and_ts(pkey, info_min_log_id, info_min_ts))) {
|
|
if (OB_PARTITION_NOT_EXIST == ret) {
|
|
// partition not exist, expected
|
|
ret = OB_SUCCESS;
|
|
EXTLOG_LOG(INFO, "get flying ilog info block info fail, partition not exist", K(pkey));
|
|
} else {
|
|
EXTLOG_LOG(WARN, "get_min_log_id from flying info block fail", K(ret), K(pkey));
|
|
}
|
|
} else if (OB_UNLIKELY(OB_INVALID_ID == info_min_log_id) || OB_UNLIKELY(OB_INVALID_TIMESTAMP == info_min_ts)) {
|
|
// return value invalid
|
|
EXTLOG_LOG(WARN, "invalid flying ilog info block info", K(pkey), K(info_min_log_id), K(info_min_ts));
|
|
ret = OB_INVALID_DATA;
|
|
} else {
|
|
if (start_ts >= info_min_ts) {
|
|
// If the start timestamp falls in the last ilog file, query last_file to locate the log accurately
|
|
if (OB_FAIL(locate_from_last_file_(pkey, start_ts, info_min_log_id, search_status))) {
|
|
EXTLOG_LOG(
|
|
WARN, "locate_from_last_file_ fail", K(ret), K(pkey), K(info_min_log_id), K(start_ts), K(info_min_ts));
|
|
|
|
// If the partition does not exist or the log does not exist, the return is OB_SUCCESS,
|
|
// and it is required to locate from IlogStorage
|
|
if (OB_PARTITION_NOT_EXIST == ret || OB_INVALID_PARTITION == ret || OB_ERR_OUT_OF_LOWER_BOUND == ret ||
|
|
OB_ERR_OUT_OF_UPPER_BOUND == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
} else {
|
|
EXTLOG_LOG(INFO,
|
|
"locate start log id from log index success",
|
|
K(pkey),
|
|
K(search_status),
|
|
K(info_min_log_id),
|
|
K(info_min_ts));
|
|
}
|
|
} else {
|
|
// Keep the smallest log information greater than or equal to the target timestamp
|
|
// If all InfoBlocks are scanned forward and there is no log of this pkey,
|
|
// return this log and mark OB_ERR_OUT_OF_LOWER_BOUND at well.
|
|
search_status.update_min_greater_log(info_min_log_id, info_min_ts);
|
|
EXTLOG_LOG(
|
|
TRACE, "find min greater log from flying ilog", K(pkey), K(start_ts), K(info_min_ts), K(info_min_log_id));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::locate_pkey_from_ilog_storage_(const ObPartitionKey& pkey, SearchStatus& search_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int locate_err = OB_SUCCESS;
|
|
uint64_t target_log_id = OB_INVALID_ID;
|
|
int64_t target_log_timestamp = OB_INVALID_TIMESTAMP;
|
|
const int64_t SLEEP_FOR_RETRY = 10 * 1000; // 10ms
|
|
const int64_t RETRY_CNT = 10;
|
|
int64_t start_ts = search_status.start_ts_;
|
|
|
|
if (OB_ISNULL(log_engine_)) {
|
|
ret = OB_NOT_INIT;
|
|
} else {
|
|
int64_t retry_times = 0;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
|
|
// locate from IlogStorage
|
|
// If OB_NEED_RETRY is returned, it will retry locally,
|
|
// if the fixed number of retry is still invalid, return OB_EXT_HANDLE_UNFINISH.
|
|
do {
|
|
ret = log_engine_->locate_by_timestamp(pkey, start_ts, target_log_id, target_log_timestamp);
|
|
|
|
if (OB_NEED_RETRY == ret) {
|
|
EXTLOG_LOG(WARN,
|
|
"cursor cache locate_by_timestamp need retry",
|
|
K(ret),
|
|
K(pkey),
|
|
K(start_ts),
|
|
K(target_log_id),
|
|
K(target_log_timestamp),
|
|
K(retry_times));
|
|
usleep(SLEEP_FOR_RETRY);
|
|
}
|
|
} while (OB_NEED_RETRY == ret && ++retry_times < RETRY_CNT);
|
|
|
|
int64_t end_time = ObTimeUtility::current_time();
|
|
|
|
// set positioning results
|
|
locate_err = ret;
|
|
|
|
// handle error code
|
|
if (OB_SUCCESS == ret) {
|
|
EXTLOG_LOG(INFO,
|
|
"locate_by_timestamp from cursor cache success",
|
|
K(pkey),
|
|
K(start_ts),
|
|
K(target_log_id),
|
|
K(target_log_timestamp),
|
|
K(retry_times),
|
|
"locate_time",
|
|
end_time - start_time);
|
|
} else {
|
|
EXTLOG_LOG(WARN,
|
|
"locate_by_timestamp from cursor cache fail",
|
|
K(ret),
|
|
K(pkey),
|
|
K(start_ts),
|
|
K(target_log_id),
|
|
K(target_log_timestamp),
|
|
K(retry_times),
|
|
K(search_status));
|
|
|
|
// Continue to deal with cases OB_ERR_OUT_OF_UPPER_BOUND
|
|
// Compare the upper bound value and min_greater_log_id, if it is continuous,
|
|
// It means that the log falls in the last ilog. In this case, min_greater_log_id is returned.
|
|
if (OB_ERR_OUT_OF_UPPER_BOUND == ret) {
|
|
if (target_log_id + 1 == search_status.min_greater_log_id_) {
|
|
// update positioning results
|
|
target_log_id = search_status.min_greater_log_id_;
|
|
target_log_timestamp = search_status.min_greater_log_ts_;
|
|
locate_err = OB_SUCCESS;
|
|
|
|
EXTLOG_LOG(INFO,
|
|
"locate_by_timestamp success, use first log of flying ilog",
|
|
K(pkey),
|
|
K(start_ts),
|
|
K(target_log_id),
|
|
K(target_log_timestamp),
|
|
K(retry_times),
|
|
K(search_status));
|
|
}
|
|
}
|
|
// partition not exist, continue processing
|
|
else if (OB_PARTITION_NOT_EXIST == ret) {
|
|
// last ilog not served yet, return NO_LOG
|
|
if (OB_INVALID_ID == search_status.min_greater_log_id_) {
|
|
locate_err = OB_NO_LOG;
|
|
} else {
|
|
// last ilog service, return OB_ERR_OUT_OF_LOWER_BOUND
|
|
target_log_id = search_status.min_greater_log_id_;
|
|
target_log_timestamp = search_status.min_greater_log_ts_;
|
|
locate_err = OB_ERR_OUT_OF_LOWER_BOUND;
|
|
}
|
|
}
|
|
// partition needs to be retried will return OB_EXT_HANDLE_UNFINISH status,
|
|
// and the client will continue to retry next loop
|
|
else if (OB_NEED_RETRY == ret) {
|
|
locate_err = OB_EXT_HANDLE_UNFINISH;
|
|
} else {
|
|
// other error codes remain unchanged
|
|
}
|
|
|
|
// all error code expected should return OB_SUCCESS
|
|
if (OB_ERR_OUT_OF_LOWER_BOUND == ret || OB_ERR_OUT_OF_UPPER_BOUND == ret || OB_PARTITION_NOT_EXIST == ret ||
|
|
OB_NEED_RETRY == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
|
|
// Dealing with OB_ERR_OUT_OF_LOWER_BOUND
|
|
if (OB_SUCCESS == ret && OB_ERR_OUT_OF_LOWER_BOUND == locate_err) {
|
|
// Whether the processing is successful or not, should return OB_SUCCESS
|
|
//
|
|
// NOTE: the following functions will modify the positioning results:
|
|
// target_log_id/target_log_timestamp/locate_err
|
|
int tmp_ret =
|
|
handle_when_locate_out_of_lower_bound_(pkey, search_status, target_log_id, target_log_timestamp, locate_err);
|
|
|
|
if (OB_UNLIKELY(OB_SUCCESS != tmp_ret)) {
|
|
EXTLOG_LOG(WARN,
|
|
"handle_when_locate_out_of_lower_bound_ fail",
|
|
K(tmp_ret),
|
|
K(pkey),
|
|
K(locate_err),
|
|
K(target_log_timestamp),
|
|
K(target_log_id));
|
|
}
|
|
}
|
|
|
|
// Finally, set ending regardless of success
|
|
search_status.mark_finish(locate_err, target_log_id, target_log_timestamp);
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// When locating based on the timestamp to be OB_ERR_OUT_OF_LOWER_BOUND,
|
|
// it means that all logs less than target_log_id will be recycled.
|
|
// In order to reduce the probability of positioning return failure,
|
|
// the log information recorded in storage info is verified.
|
|
// If the log recorded in storage info is continuous with the lower bound log,
|
|
// and the location timestamp falls between the two logs, means the positioning is successful.
|
|
int ObExtStartLogLocatorForDir::handle_when_locate_out_of_lower_bound_(const ObPartitionKey& pkey,
|
|
const SearchStatus& search_status, uint64_t& target_log_id, int64_t& target_log_timestamp, int& locate_err)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObIPartitionGroupGuard guard;
|
|
ObIPartitionGroup* partition = NULL;
|
|
int64_t start_ts = search_status.start_ts_;
|
|
ObBaseStorageInfo clog_info;
|
|
int old_locate_err = locate_err;
|
|
|
|
if (OB_UNLIKELY(OB_ERR_OUT_OF_LOWER_BOUND != locate_err) || OB_UNLIKELY(OB_INVALID_ID == target_log_id) ||
|
|
OB_UNLIKELY(OB_INVALID_TIMESTAMP == target_log_timestamp) || OB_UNLIKELY(start_ts > target_log_timestamp)) {
|
|
EXTLOG_LOG(
|
|
ERROR, "invalid argument", K(pkey), K(locate_err), K(target_log_id), K(target_log_timestamp), K(search_status));
|
|
ret = OB_INVALID_ARGUMENT;
|
|
} else if (OB_ISNULL(partition_service_)) {
|
|
EXTLOG_LOG(WARN, "partition service is NULL, not init", K(partition_service_));
|
|
ret = OB_NOT_INIT;
|
|
} else if (OB_FAIL(partition_service_->get_partition(pkey, guard)) ||
|
|
(NULL == (partition = guard.get_partition_group()))) {
|
|
// partition not exists, expected, not process.
|
|
ret = OB_SUCCESS;
|
|
} else if (OB_FAIL(partition->get_saved_clog_info(clog_info))) {
|
|
EXTLOG_LOG(WARN, "get_saved_clog_info fail", K(ret), K(pkey));
|
|
} else {
|
|
int64_t base_log_tstamp = clog_info.get_submit_timestamp();
|
|
uint64_t base_log_id = clog_info.get_last_replay_log_id();
|
|
|
|
// Check whether the lower bound log of the location is continuous with that recorded in the storage info.
|
|
if (base_log_id == (target_log_id - 1)) {
|
|
// Check whether the location timestamp falls in the middle of the two logs.
|
|
// clog_info < start_ts < target_log_timestamp
|
|
if (start_ts > base_log_tstamp) {
|
|
locate_err = OB_SUCCESS;
|
|
}
|
|
// If the timestamp is exactly equal to the log timestamp in storage info,
|
|
// the log in storage info is returned.
|
|
else if (start_ts == base_log_tstamp) {
|
|
locate_err = OB_SUCCESS;
|
|
target_log_id = base_log_id;
|
|
target_log_timestamp = base_log_tstamp;
|
|
}
|
|
}
|
|
|
|
EXTLOG_LOG(INFO,
|
|
"locate_by_timestamp handle OUT_OF_LOWER_BOUND case",
|
|
K(locate_err),
|
|
K(old_locate_err),
|
|
K(pkey),
|
|
K(target_log_id),
|
|
K(target_log_timestamp),
|
|
K(start_ts),
|
|
K(clog_info));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::build_search_result(
|
|
const ObLocateByTsReq& req, SearchMap& search_map, ObLocateByTsResp& resp)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObLocateByTsReq::ParamArray& params = req.get_params();
|
|
const int64_t count = params.count();
|
|
ObLocateByTsResp::Result result;
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
|
const ObPartitionKey& pkey = params[i].pkey_;
|
|
SearchStatus* search_status = NULL;
|
|
|
|
if (OB_FAIL(search_map.get(pkey, search_status))) {
|
|
EXTLOG_LOG(WARN, "search map get error", K(ret), K(pkey), KP(search_status));
|
|
} else if (OB_ISNULL(search_status)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
EXTLOG_LOG(ERROR, "search map get null value", K(ret), K(pkey));
|
|
} else {
|
|
result.reset(search_status->err_, search_status->start_log_id_, search_status->start_log_ts_);
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(resp.append_result(result))) {
|
|
EXTLOG_LOG(WARN, "locate by ts resp append result error", K(ret), K(pkey), K(result));
|
|
} else {
|
|
EXTLOG_LOG(INFO, "locate_by_timestamp append result success", K(pkey), K(result));
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
|
|
if (OB_SUCC(ret)) {
|
|
EXTLOG_LOG(INFO, "locate_by_timestamp build result success", K(count));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::do_locate_partition_(const common::ObPartitionKey& pkey, SearchStatus& search_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t begin_ts = ObTimeUtility::current_time();
|
|
|
|
// step 1: prioritize cold partitions based on the largest log
|
|
// These cold partitions do not write logs for a long time,
|
|
// and the target timestamp is likely to be greater than the last log timestamp
|
|
// In this case, return to the next service log.
|
|
if (OB_FAIL(handle_cold_pkey_by_sw_(pkey, search_status))) {
|
|
EXTLOG_LOG(WARN, "handle_cold_pkey_by_sw_ fail", K(ret), K(pkey), K(search_status));
|
|
search_status.mark_finish(ret, OB_INVALID_ID, OB_INVALID_TIMESTAMP);
|
|
}
|
|
// step 2: locate from last ilog
|
|
else if (!search_status.is_finished() && OB_FAIL(locate_pkey_from_flying_ilog_(pkey, search_status))) {
|
|
EXTLOG_LOG(WARN, "locate_pkey_from_flying_ilog_ fail", K(ret), K(pkey), K(search_status));
|
|
}
|
|
// step 3: locate from IlogStorage
|
|
else if (!search_status.is_finished() && OB_FAIL(locate_pkey_from_ilog_storage_(pkey, search_status))) {
|
|
EXTLOG_LOG(WARN, "locate_pkey_from_ilog_storage_ fail", K(ret), K(pkey), K(search_status));
|
|
}
|
|
// status must be marked as complete finally
|
|
else if (OB_UNLIKELY(!search_status.is_finished())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
EXTLOG_LOG(WARN,
|
|
"locate_by_timestamp unexpected error, partition does not mark finished",
|
|
K(ret),
|
|
K(pkey),
|
|
K(search_status));
|
|
} else {
|
|
// success
|
|
}
|
|
|
|
int64_t end_ts = ObTimeUtility::current_time();
|
|
EXTLOG_LOG(TRACE,
|
|
"partition locate_by_timestamp finish",
|
|
K(ret),
|
|
K(pkey),
|
|
K(search_status),
|
|
"locate_time",
|
|
end_ts - begin_ts);
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::do_locator_req(
|
|
const ObLocateByTsReq& req_msg, ObLocateByTsResp& resp, const ObExtRpcQit& qit, bool& is_hurry_quit)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
SearchMap search_map;
|
|
PageArena<> map_allocator;
|
|
SearchParam search_param;
|
|
|
|
search_map.init(ObModIds::OB_CLOG_EXT_RPC);
|
|
|
|
is_hurry_quit = false;
|
|
|
|
// prepare data
|
|
if (OB_FAIL(search_param.init(&map_allocator))) {
|
|
EXTLOG_LOG(WARN, "search param init error", K(ret));
|
|
} else if (OB_FAIL(build_search_map(req_msg, search_param, search_map))) {
|
|
EXTLOG_LOG(WARN, "build search map error", K(ret), K(req_msg));
|
|
} else {
|
|
SearchMap::BlurredIterator iter(search_map);
|
|
ObPartitionKey pkey;
|
|
SearchStatus* search_status = NULL;
|
|
|
|
while (OB_SUCCESS == ret && OB_SUCCESS == (ret = iter.next(pkey, search_status))) {
|
|
if (OB_ISNULL(search_status)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
EXTLOG_LOG(ERROR, "get null search status", K(ret), K(pkey), K(search_status));
|
|
} else {
|
|
// If exit halfway, mark every subsequent partition with unfinished error code
|
|
if (OB_UNLIKELY(is_hurry_quit = qit.should_hurry_quit())) {
|
|
search_status->mark_finish(OB_EXT_HANDLE_UNFINISH, OB_INVALID_ID, OB_INVALID_TIMESTAMP);
|
|
EXTLOG_LOG(
|
|
INFO, "partition locate_by_timestamp mark unfinish", K(ret), K(pkey), K(search_status), K(is_hurry_quit));
|
|
}
|
|
// locate a partition, and set the result regardless of success or not
|
|
else if (OB_FAIL(do_locate_partition_(pkey, *search_status))) {
|
|
EXTLOG_LOG(WARN, "do_locate_partition_ fail", K(ret), K(pkey), KPC(search_status));
|
|
} else {
|
|
// success
|
|
}
|
|
}
|
|
} // end while
|
|
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
|
|
// build result
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(build_search_result(req_msg, search_map, resp))) {
|
|
EXTLOG_LOG(WARN, "build result error", K(ret), K(req_msg));
|
|
}
|
|
}
|
|
|
|
(void)search_map.destroy();
|
|
map_allocator.free();
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocatorForDir::locator_req(const ObLocateByTsReq& req_msg, ObLocateByTsResp& resp, bool& is_hurry_quit)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (IS_NOT_INIT) {
|
|
ret = OB_NOT_INIT;
|
|
EXTLOG_LOG(WARN, "log external executor not init", K(ret), K(req_msg));
|
|
} else if (OB_UNLIKELY(!req_msg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
EXTLOG_LOG(WARN, "ObLocateByTsReq is not valid", K(ret), K(req_msg));
|
|
} else {
|
|
ObExtRpcQit qit;
|
|
if (OB_FAIL(qit.init(get_rpc_deadline()))) {
|
|
EXTLOG_LOG(WARN, "init qit error", K(ret));
|
|
} else if (OB_FAIL(do_locator_req(req_msg, resp, qit, is_hurry_quit))) {
|
|
EXTLOG_LOG(WARN, "do locator request error", K(ret), K(req_msg), K(resp), K(qit), K(is_hurry_quit));
|
|
} else {
|
|
EXTLOG_LOG(TRACE, "do locator success", K(req_msg), K(resp), K(qit), K(is_hurry_quit));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
resp.set_err(OB_SUCCESS);
|
|
} else {
|
|
resp.set_err(OB_ERR_SYS);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocator::init(ObPartitionService* partition_service, clog::ObILogEngine* log_engine)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(partition_service) || OB_ISNULL(log_engine)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
EXTLOG_LOG(WARN, "invalid partition_service_ or log_engine", K(ret), KP(partition_service), KP(log_engine));
|
|
} else {
|
|
partition_service_ = partition_service;
|
|
log_engine_ = log_engine;
|
|
is_inited_ = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocator::do_req_start_log_id(const ObLocateByTsReq& req, ObLocateByTsResp& resp)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObExtStartLogLocatorForDir locator_impl;
|
|
|
|
bool is_hurry_quit = false;
|
|
if (0 == req.get_params().count()) {
|
|
EXTLOG_LOG(INFO, "no partition in request", K(req));
|
|
} else if (OB_FAIL(locator_impl.init(partition_service_, log_engine_))) {
|
|
EXTLOG_LOG(WARN, "locator_impl init error", K(ret));
|
|
} else if (OB_FAIL(locator_impl.locator_req(req, resp, is_hurry_quit))) {
|
|
EXTLOG_LOG(WARN, "locator req error", K(ret), K(req), K(resp));
|
|
} else {
|
|
locator_impl.destroy();
|
|
EXTLOG_LOG(INFO, "locator req success", K(req), K(resp), K(is_hurry_quit));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObExtStartLogLocator::req_start_log_id_by_ts_with_breakpoint(
|
|
const ObLocateByTsReq& req_msg, ObLocateByTsResp& result)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (IS_NOT_INIT) {
|
|
ret = OB_NOT_INIT;
|
|
EXTLOG_LOG(WARN, "log external executor not init", K(ret), K(req_msg));
|
|
} else if (OB_UNLIKELY(!req_msg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
EXTLOG_LOG(WARN, "ObLocateByTsReq is not valid", K(ret), K(req_msg));
|
|
} else {
|
|
if (OB_FAIL(do_req_start_log_id(req_msg, result))) {
|
|
EXTLOG_LOG(WARN, "do req_start_log_id error", K(ret), K(req_msg), K(result));
|
|
} else {
|
|
EXTLOG_LOG(INFO, "do req_start_log_id success", K(ret), K(req_msg), K(result));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
result.set_err(OB_SUCCESS);
|
|
} else {
|
|
result.set_err(OB_ERR_SYS);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
} // namespace logservice
|
|
} // namespace oceanbase
|