add diagnose mgr unittest & placeholder for RPC request

This commit is contained in:
yangqise7en
2024-08-09 09:33:40 +00:00
committed by ob-robot
parent 75c6d07cac
commit ee0deb14ba
20 changed files with 281 additions and 45 deletions

View File

@ -343,7 +343,7 @@ int ObMajorFreezeHelper::do_one_tenant_major_freeze(
const int64_t launch_start_time = ObTimeUtility::current_time();
obrpc::ObMajorFreezeRpcProxy proxy;
ObAddr leader;
obrpc::ObMajorFreezeRequest req(freeze_info);
obrpc::ObMajorFreezeRequest req(freeze_info, MF_DAILY_MERGE/*placeholder*/); // right logic is on SS feature
obrpc::ObMajorFreezeResponse resp;
uint64_t tenant_id = freeze_info.tenant_id_;

View File

@ -22,7 +22,7 @@ namespace obrpc
OB_SERIALIZE_MEMBER(ObSimpleFreezeInfo, tenant_id_);
OB_SERIALIZE_MEMBER(ObMajorFreezeRequest, info_);
OB_SERIALIZE_MEMBER(ObMajorFreezeRequest, info_, freeze_reason_);
OB_SERIALIZE_MEMBER(ObMajorFreezeResponse, err_code_);

View File

@ -20,6 +20,7 @@
#include "rpc/obrpc/ob_rpc_proxy_macros.h"
#include "observer/ob_server_struct.h"
#include "share/config/ob_server_config.h"
#include "rootserver/freeze/ob_major_freeze_util.h"
namespace oceanbase
{
@ -63,15 +64,24 @@ struct ObMajorFreezeRequest
{
public:
ObSimpleFreezeInfo info_;
rootserver::ObMajorFreezeReason freeze_reason_; // placeholder for SS feature
ObMajorFreezeRequest() {}
ObMajorFreezeRequest(const ObSimpleFreezeInfo &info) : info_(info) {}
ObMajorFreezeRequest()
: info_(),
freeze_reason_(rootserver::MF_REASON_MAX)
{}
ObMajorFreezeRequest(
const ObSimpleFreezeInfo &info,
const rootserver::ObMajorFreezeReason freeze_reason)
: info_(info),
freeze_reason_(freeze_reason)
{}
bool is_valid() const { return info_.is_valid(); }
uint64_t tenant_id() const { return info_.tenant_id_; }
TO_STRING_KV(K_(info));
TO_STRING_KV(K_(info), "freeze_reason", major_freeze_reason_to_str(freeze_reason_));
OB_UNIS_VERSION(1);
};

View File

@ -110,5 +110,29 @@ ObFreezeTimeGuard::~ObFreezeTimeGuard()
}
}
const char * ObMajorFreezeReasonStr[] = {
"DAILY_MERGE",
"USER_REQUEST",
"MAJOR_COMPACT_TRIGGER"
};
const char *major_freeze_reason_to_str(const int64_t freeze_reason)
{
STATIC_ASSERT(static_cast<int64_t>(MF_REASON_MAX) == ARRAYSIZEOF(ObMajorFreezeReasonStr),
"major freeze reason str len is mismatch");
const char *str = "";
if (OB_UNLIKELY(!is_valid_major_freeze_reason((ObMajorFreezeReason)freeze_reason))) {
str = "invalid_freeze_reason";
} else {
str = ObMajorFreezeReasonStr[freeze_reason];
}
return str;
}
bool is_valid_major_freeze_reason(const ObMajorFreezeReason &freeze_reason)
{
return freeze_reason >= ObMajorFreezeReason::MF_DAILY_MERGE
&& freeze_reason < ObMajorFreezeReason::MF_REASON_MAX;
}
} // end namespace rootserver
} // end namespace oceanbase

View File

@ -66,6 +66,15 @@ private:
const char * const log_mod_;
};
enum ObMajorFreezeReason : uint8_t {
MF_DAILY_MERGE = 0,
MF_USER_REQUEST,
MF_MAJOR_COMPACT_TRIGGER,
MF_REASON_MAX,
};
const char *major_freeze_reason_to_str(const int64_t freeze_reason);
bool is_valid_major_freeze_reason(const ObMajorFreezeReason &freeze_reason);
} // end namespace rootserver
} // end namespace oceanbase

View File

@ -61,7 +61,7 @@ ObDagWarningInfo::~ObDagWarningInfo()
void ObDagWarningInfo::shallow_copy(ObIDiagnoseInfo *other)
{
ObDagWarningInfo *info = nullptr;
if (OB_NOT_NULL(other) && OB_NOT_NULL(info = dynamic_cast<ObDagWarningInfo *>(other))) {
if (OB_NOT_NULL(other) && OB_NOT_NULL(info = static_cast<ObDagWarningInfo *>(other))) {
priority_ = info->priority_;
task_id_ = info->task_id_;
dag_type_ = info->dag_type_;

View File

@ -10,6 +10,7 @@
* See the Mulan PubL v2 for more details.
*/
//SUSPECT_INFO_TYPE_DEF(suspect_info_type, info_priority, with_comment, info_str, int_info_cnt, ...)
#ifdef SUSPECT_INFO_TYPE_DEF
SUSPECT_INFO_TYPE_DEF(SUSPECT_MEMTABLE_CANT_MINOR_MERGE, ObDiagnoseInfoPrio::DIAGNOSE_PRIORITY_LOW, false, "memtable can not minor merge",
2, {"memtable end_scn", "memtable timestamp"})
@ -82,6 +83,11 @@ enum ObDiagnoseTabletType {
TYPE_DIAGNOSE_TABLET_MAX
};
static bool is_valid_diagnose_tablet_type(const ObDiagnoseTabletType type)
{
return type >= TYPE_SPECIAL && type < TYPE_DIAGNOSE_TABLET_MAX;
}
static constexpr ObDiagnoseInfoStruct OB_SUSPECT_INFO_TYPES[] = {
#define SUSPECT_INFO_TYPE_DEF(suspect_info_type, info_priority, with_comment, info_str, int_info_cnt, ...) \
{int_info_cnt, info_priority, with_comment, info_str, ##__VA_ARGS__},

View File

@ -3093,7 +3093,7 @@ int ObDagPrioScheduler::diagnose_dag(
} else if (OB_ISNULL(stored_dag)) {
ret = OB_ERR_SYS;
LOG_WARN("dag is null", K(ret));
} else if (stored_dag->get_priority() != dag.get_priority()) {
} else if (OB_UNLIKELY(stored_dag->get_priority() != dag.get_priority())) {
ret = OB_ERR_UNEXPECTED;
COMMON_LOG(WARN, "unexpected priority value", K(ret), K(stored_dag->get_priority()), K(dag.get_priority()));
} else if (OB_FAIL(stored_dag->diagnose_compaction_info(progress))) {

View File

@ -719,7 +719,7 @@ void ObBasicTabletMergeCtx::add_sstable_merge_info(
}
ObInfoParamBuffer info_allocator;
if (OB_SUCCESS == MTL(ObDagWarningHistoryManager *)->get_with_param(hash, &warning_info, info_allocator)) {
if (OB_SUCCESS == MTL(ObDagWarningHistoryManager *)->get_with_param(hash, warning_info, info_allocator)) {
sstable_merge_info.dag_ret_ = warning_info.dag_ret_;
sstable_merge_info.task_id_ = warning_info.task_id_;
sstable_merge_info.retry_cnt_ = warning_info.retry_cnt_;
@ -730,7 +730,7 @@ void ObBasicTabletMergeCtx::add_sstable_merge_info(
ObScheduleSuspectInfo ret_info;
info_allocator.reuse();
if (OB_SUCCESS == MTL(compaction::ObScheduleSuspectInfoMgr *)->get_with_param(hash, &ret_info, info_allocator)) {
if (OB_SUCCESS == MTL(compaction::ObScheduleSuspectInfoMgr *)->get_with_param(hash, ret_info, info_allocator)) {
sstable_merge_info.suspect_add_time_ = ret_info.add_time_;
sstable_merge_info.info_param_ = ret_info.info_param_;
if (OB_TMP_FAIL(MTL(compaction::ObScheduleSuspectInfoMgr *)->delete_info(hash))) {

View File

@ -62,7 +62,7 @@ bool ObScheduleSuspectInfo::is_valid() const
void ObScheduleSuspectInfo::shallow_copy(ObIDiagnoseInfo *other)
{
ObScheduleSuspectInfo *info = nullptr;
if (OB_NOT_NULL(other) && OB_NOT_NULL(info = dynamic_cast<ObScheduleSuspectInfo *>(other))) {
if (OB_NOT_NULL(other) && OB_NOT_NULL(info = static_cast<ObScheduleSuspectInfo *>(other))) {
merge_type_ = info->merge_type_;
ls_id_ = info->ls_id_;
tablet_id_ = info->tablet_id_;
@ -72,11 +72,6 @@ void ObScheduleSuspectInfo::shallow_copy(ObIDiagnoseInfo *other)
}
}
int64_t ObScheduleSuspectInfo::get_add_time() const
{
return add_time_;
}
int64_t ObScheduleSuspectInfo::get_hash() const
{
return hash_;
@ -274,15 +269,12 @@ int ObIDiagnoseInfoMgr::size()
return info_list_.get_size();
}
int ObIDiagnoseInfoMgr::get_with_param(const int64_t key, ObIDiagnoseInfo *out_info, ObIAllocator &allocator)
int ObIDiagnoseInfoMgr::get_with_param(const int64_t key, ObIDiagnoseInfo &out_info, ObIAllocator &allocator)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
STORAGE_LOG(WARN, "ObIDiagnoseInfoMgr is not init", K(ret));
} else if (OB_ISNULL(out_info)) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid argument", K(ret), K(out_info));
} else {
common::SpinWLockGuard guard(lock_);
ObIDiagnoseInfo *info = NULL;
@ -294,8 +286,8 @@ int ObIDiagnoseInfoMgr::get_with_param(const int64_t key, ObIDiagnoseInfo *out_i
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "info_param is null", K(ret), K(info));
} else {
out_info->shallow_copy(info/*src*/);
if (OB_FAIL(info->info_param_->deep_copy(allocator, out_info->info_param_/*dst*/))) {
out_info.shallow_copy(info/*src*/);
if (OB_FAIL(info->info_param_->deep_copy(allocator, out_info.info_param_/*dst*/))) {
STORAGE_LOG(WARN, "failed to deep copy info param", K(ret));
}
}
@ -766,7 +758,7 @@ int ObCompactionDiagnoseMgr::get_suspect_info(
input_info.ls_id_ = ls_id;
input_info.tablet_id_ = tablet_id;
ObInfoParamBuffer allocator; // info_param_ will be invalid after return
if (OB_FAIL(MTL(ObScheduleSuspectInfoMgr *)->get_with_param(input_info.hash(), &ret_info, allocator))) {
if (OB_FAIL(MTL(ObScheduleSuspectInfoMgr *)->get_with_param(input_info.hash(), ret_info, allocator))) {
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("failed to get suspect info", K(ret), K(input_info));
}
@ -828,7 +820,7 @@ int ObCompactionDiagnoseMgr::diagnose_tenant( //TODO(mingqiao): check tenant res
ObTimeUtility::fast_current_time(),
"error_code", ret,
"freeze_info is invalid, merged_version", merged_version))) {
LOG_WARN("failed to add dignose info about freeze_info", K(ret), K(merged_version));
LOG_WARN("failed to add dignose info about freeze_info", K(tmp_ret), K(merged_version));
}
} else {
compaction_scn = freeze_info.frozen_scn_.get_val_for_tx();
@ -1566,7 +1558,7 @@ int ObCompactionDiagnoseMgr::get_suspect_and_warning_info(
dag_hash.merge_type_ = merge_type;
dag_hash.ls_id_ = ls_id;
dag_hash.tablet_id_ = tablet_id;
if (OB_FAIL(MTL(ObScheduleSuspectInfoMgr *)->get_with_param(dag_hash.inner_hash(), &info, allocator))) {
if (OB_FAIL(MTL(ObScheduleSuspectInfoMgr *)->get_with_param(dag_hash.inner_hash(), info, allocator))) {
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("failed to get suspect info", K(ret), K(dag_hash));
} else { // no schedule suspect info
@ -1575,7 +1567,7 @@ int ObCompactionDiagnoseMgr::get_suspect_and_warning_info(
allocator.reuse();
char tmp_str[common::OB_DAG_WARNING_INFO_LENGTH] = "\0";
if (OB_FAIL(MTL(ObDagWarningHistoryManager *)->get_with_param(
dag_key, &warning_info, allocator))) {
dag_key, warning_info, allocator))) {
// check __all_virtual_dag_warning_history
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("failed to get dag warning info", K(ret), K(dag_hash));

View File

@ -113,7 +113,6 @@ struct ObIDiagnoseInfo : public common::ObDLinkBase<ObIDiagnoseInfo> {
}
virtual void shallow_copy(ObIDiagnoseInfo *other) = 0;
virtual void update(ObIDiagnoseInfo *other) {}
virtual int64_t get_add_time() const { return INT_MAX64; }
virtual int64_t get_hash() const { return 0; }
template<typename T>
int deep_copy(ObIAllocator &allocator, T *&out_info);
@ -173,7 +172,6 @@ struct ObScheduleSuspectInfo : public ObIDiagnoseInfo, public ObMergeDagHash
int64_t hash() const;
bool is_valid() const;
virtual void shallow_copy(ObIDiagnoseInfo *other) override;
virtual int64_t get_add_time() const override;
virtual int64_t get_hash() const override;
TO_STRING_KV("merge_type", merge_type_to_str(merge_type_), K_(ls_id), K_(tablet_id), K_(add_time), K_(hash));
@ -248,7 +246,7 @@ public:
template<typename T>
int alloc_and_add(const int64_t key, T *input_info);
int get_with_param(const int64_t key, ObIDiagnoseInfo *out_info, ObIAllocator &allocator);
int get_with_param(const int64_t key, ObIDiagnoseInfo &out_info, ObIAllocator &allocator);
int delete_info(const int64_t key);
int set_max(const int64_t size);
@ -332,6 +330,10 @@ int ObIDiagnoseInfoMgr::alloc_and_add(const int64_t key, T *input_info)
return ret;
}
/*
ObScheduleSuspectInfoMgr only provide add func, will purge suspect_info when alloc_mem_fail,
so there may have remain suspect info after compaction finish
*/
class ObScheduleSuspectInfoMgr : public ObIDiagnoseInfoMgr {
public:
static int mtl_init(ObScheduleSuspectInfoMgr *&schedule_suspect_info);
@ -554,7 +556,6 @@ public:
typedef common::hash::ObHashMap<ObLSID, ObLSCheckStatus> LSStatusMap;
private:
static const int64_t NS_TIME = 1000L * 1000L * 1000L;
static const int64_t TOLERATE_DAG_NET_HANG_INTERVAL = 1000L * 1000L * 60L * 10; // 10hour
static const int64_t WAIT_MEDIUM_SCHEDULE_INTERVAL = NS_TIME * 60L * 5; // 5min // ns
static const int64_t TOLERATE_MEDIUM_SCHEDULE_INTERVAL = NS_TIME * 60L * 60L * 36; // 36 hour
static const int64_t DIAGNOSE_TABELT_MAX_COUNT = 10; // same type diagnose tablet max count
@ -798,6 +799,9 @@ ADD_SUSPECT_INFO(merge_type, diagnose_type, UNKNOW_LS_ID, UNKNOW_TABLET_ID, info
return ret; \
}
/*
the input 'INFO_PARAM_INT##n_int, LOG_PARAMETER_KV##n' should match the ObSuspectInfoType in SUSPECT_INFO_TYPE_DEF
*/
#define DEFINE_SUSPECT_INFO_ADD_EXTRA(n, n_int) \
template <typename T = int64_t, LOG_TYPENAME_TN##n> \
int ADD_SUSPECT_INFO(compaction::ObMergeType type, share::ObDiagnoseTabletType diagnose_type, \

View File

@ -59,7 +59,7 @@ int ObCompactionHistogramStat::add_value(const int64_t time, const bool failed)
{
int ret = OB_SUCCESS;
int64_t index = ObCompactionHistogramBucketUtil::get_index(time);
if (ObCompactionHistogramBucketUtil::BUCKET_MAX_COUNT < index || 0 > index) {
if (OB_UNLIKELY(ObCompactionHistogramBucketUtil::BUCKET_MAX_COUNT < index || 0 > index)) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "get unexpected index", K(ret), K(index));
} else {
@ -250,7 +250,7 @@ void ObCompactionDagStatus::update_finish_cnt(
int64_t ObCompactionDagStatus::get_cost_long_time(const int64_t prio)
{
int64_t ret_time = INT64_MAX;
if (0 < prio && COMPACTION_PRIORITY_MAX > prio) {
if (0 <= prio && COMPACTION_PRIORITY_MAX > prio) {
ret_time = COST_LONG_TIME[prio];
}
return ret_time;
@ -532,7 +532,7 @@ int ObCompactionSuggestionMgr::analyze_merge_info(
"concurrent_cnt", merge_info.concurrent_cnt_);
need_suggestion = true;
}
if (1 != merge_info.concurrent_cnt_) { // parallel compaction
if (1 != merge_info.concurrent_cnt_ && merge_info.total_row_count_ >= ROW_COUNT_TO_CHECK_PARALLEL_EVEN) { // parallel compaction
const ObParalleMergeInfo &paral_info = merge_info.parallel_merge_info_;
const int64_t count = paral_info.info_[ObParalleMergeInfo::SCAN_UNITS].count_;
if (0 < count) {

View File

@ -247,6 +247,7 @@ public:
static const int64_t TOO_MANY_FAILED_COUNT = 20;
static const int64_t SCAN_AVERAGE_RAITO = 4; // 2 * 2
static const int64_t INC_ROW_CNT_PARAM = 5 * 1000 * 1000; // 5 Million
static const int64_t ROW_COUNT_TO_CHECK_PARALLEL_EVEN = 1 * 1000 * 1000; // 1 Million
static const int64_t SINGLE_PARTITION_MACRO_CNT_PARAM = 256 * 1024; // single partition size 500G
static const int64_t MACRO_CNT_PARAM = 5 * 1000; // 5 k

View File

@ -69,8 +69,7 @@ int ObDiagnoseTabletMgr::add_diagnose_tablet(
LOG_WARN("ObDiagnoseTabletMgr is not init", K(ret));
} else {
if (!ls_id.is_valid() || !tablet_id.is_valid()
|| share::ObDiagnoseTabletType::TYPE_SPECIAL > type
|| share::ObDiagnoseTabletType::TYPE_DIAGNOSE_TABLET_MAX <= type) {
|| !is_valid_diagnose_tablet_type(type)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ls_id), K(tablet_id), K(type));
} else {
@ -128,8 +127,7 @@ int ObDiagnoseTabletMgr::delete_diagnose_tablet(
LOG_WARN("ObDiagnoseTabletMgr is not init", K(ret));
} else {
if (!ls_id.is_valid() || !tablet_id.is_valid()
|| share::ObDiagnoseTabletType::TYPE_SPECIAL > type
|| share::ObDiagnoseTabletType::TYPE_DIAGNOSE_TABLET_MAX <= type) {
|| !is_valid_diagnose_tablet_type(type)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ls_id), K(tablet_id), K(type));
} else {
@ -146,7 +144,7 @@ int ObDiagnoseTabletMgr::delete_diagnose_tablet(
if (OB_FAIL(diagnose_tablet_map_.erase_refactored(diagnose_tablet))) {
LOG_WARN("fail to delete diagnose tablet", K(ret), K(diagnose_tablet));
}
} else if (OB_FAIL(diagnose_tablet_map_.set_refactored(diagnose_tablet, flag, 1))) {
} else if (OB_FAIL(diagnose_tablet_map_.set_refactored(diagnose_tablet, flag, 1/*overwrite*/))) {
LOG_WARN("fail to add diagnose tablet into map", K(ret), K(diagnose_tablet), K(flag));
}
}

View File

@ -34,7 +34,6 @@ public:
static int get_next_info(compaction::ObIDiagnoseInfoMgr::Iterator &major_iter,
compaction::ObIDiagnoseInfoMgr::Iterator &minor_iter,
ObSSTableMergeInfo &info, char *buf, const int64_t buf_len);
// TODO need init memory limit with tenant config
ObTenantSSTableMergeInfoMgr();
virtual ~ObTenantSSTableMergeInfoMgr();
int init(const int64_t page_size=compaction::ObIDiagnoseInfoMgr::INFO_PAGE_SIZE);

View File

@ -363,7 +363,6 @@ private:
static constexpr ObMergeType MERGE_TYPES[] = {MINOR_MERGE, HISTORY_MINOR_MERGE, MDS_MINOR_MERGE};
static const int64_t ADD_LOOP_EVENT_INTERVAL = 120 * 1000 * 1000L; // 120s
static const int64_t PRINT_LOG_INVERVAL = 2 * 60 * 1000 * 1000L; // 2m
static const int64_t WAIT_MEDIUM_CHECK_THRESHOLD = 10 * 60 * 1000 * 1000 * 1000L; // 10m
static const int64_t REFRESH_TENANT_STATUS_INTERVAL = 30 * 1000 * 1000L; // 30s
static const int64_t MERGE_BACTH_FREEZE_CNT = 100L;
private:

View File

@ -42,7 +42,7 @@ struct ObTenantTabletSchedulerTaskMgr
virtual void runTimerTask() override; \
};
DEFINE_TIMER_TASK(MergeLoopTask);
DEFINE_TIMER_TASK(SSTableGCTask);
DEFINE_TIMER_TASK_WITHOUT_TIMEOUT_CHECK(SSTableGCTask);
DEFINE_TIMER_TASK(InfoPoolResizeTask);
DEFINE_TIMER_TASK(TabletUpdaterRefreshTask);
DEFINE_TIMER_TASK_WITHOUT_TIMEOUT_CHECK(MediumLoopTask);

View File

@ -122,6 +122,7 @@ storage_dml_unittest(test_major_rows_merger)
storage_dml_unittest(test_tablet tablet/test_tablet.cpp)
storage_unittest(test_medium_list_checker compaction/test_medium_list_checker.cpp)
storage_dml_unittest(test_ls_reserved_snapshot_mgr compaction/test_ls_reserved_snapshot_mgr.cpp)
storage_unittest(test_diagnose_info_mgr compaction/test_diagnose_info_mgr.cpp)
storage_unittest(test_protected_memtable_mgr_handle test_protected_memtable_mgr_handle.cpp)
storage_unittest(test_choose_migration_source_policy migration/test_choose_migration_source_policy.cpp)

View File

@ -0,0 +1,193 @@
/**
* 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.
*/
#define USING_LOG_PREFIX STORAGE
#include <gtest/gtest.h>
#include <gmock/gmock.h>
#define private public
#define protected public
#include <string.h>
#include "storage/compaction/ob_compaction_diagnose.h"
#include "mittest/mtlenv/mock_tenant_module_env.h"
#include "share/rc/ob_tenant_base.h"
namespace oceanbase
{
using namespace common;
using namespace compaction;
using namespace storage;
namespace unittest
{
class TestDiagnoseInfoMgr : public ::testing::Test
{
public:
TestDiagnoseInfoMgr()
: ls_id_(TEST_LS_ID),
tenant_base_(TEST_TENANT_ID),
suspect_info_mgr_(NULL),
diagnose_tablet_mgr_(NULL),
is_inited_(false)
{}
~TestDiagnoseInfoMgr() = default;
void SetUp();
void TearDown();
int gene_suspect_info(
const ObDiagnoseInfoPrio &prio,
const ObMergeType &merge_type,
const ObTabletID &tablet_id,
ObScheduleSuspectInfo &info);
static constexpr int64_t TEST_TENANT_ID = 1001;
static constexpr int64_t TEST_LS_ID = 9001;
share::ObLSID ls_id_;
ObTenantBase tenant_base_;
ObScheduleSuspectInfoMgr *suspect_info_mgr_;
ObDiagnoseTabletMgr *diagnose_tablet_mgr_;
ObDiagnoseInfoParam<2, 0> param_;
bool is_inited_;
};
void TestDiagnoseInfoMgr::SetUp()
{
if (!is_inited_) {
ObMallocAllocator::get_instance()->create_and_add_tenant_allocator(TEST_TENANT_ID);
is_inited_ = true;
}
if (OB_ISNULL(suspect_info_mgr_)) {
suspect_info_mgr_ = OB_NEW(ObScheduleSuspectInfoMgr, ObModIds::TEST);
}
tenant_base_.set(suspect_info_mgr_);
if (OB_ISNULL(diagnose_tablet_mgr_)) {
diagnose_tablet_mgr_ = OB_NEW(ObDiagnoseTabletMgr, ObModIds::TEST);
}
tenant_base_.set(diagnose_tablet_mgr_);
ObTenantEnv::set_tenant(&tenant_base_);
ASSERT_EQ(OB_SUCCESS, tenant_base_.init());
ObMallocAllocator *ma = ObMallocAllocator::get_instance();
ASSERT_EQ(OB_SUCCESS, ma->set_tenant_limit(TEST_TENANT_ID, 1LL << 30));
}
void TestDiagnoseInfoMgr::TearDown()
{
if (OB_NOT_NULL(suspect_info_mgr_)) {
suspect_info_mgr_->destroy();
suspect_info_mgr_ = nullptr;
}
if (OB_NOT_NULL(diagnose_tablet_mgr_)) {
diagnose_tablet_mgr_->destroy();
diagnose_tablet_mgr_ = nullptr;
}
tenant_base_.destroy();
ObTenantEnv::set_tenant(nullptr);
}
int TestDiagnoseInfoMgr::gene_suspect_info(
const ObDiagnoseInfoPrio &prio,
const ObMergeType &merge_type,
const ObTabletID &tablet_id,
ObScheduleSuspectInfo &info)
{
int ret = OB_SUCCESS;
info.priority_ = static_cast<uint32_t>(prio);
info.merge_type_ = merge_type;
info.ls_id_ = ls_id_;
info.tablet_id_ = tablet_id;
info.info_param_ = &param_;
return ret;
}
bool judge_equal(const ObScheduleSuspectInfo &a, const ObScheduleSuspectInfo &b)
{
return a.priority_ == b.priority_
&& a.ls_id_ == b.ls_id_
&& a.tablet_id_ == b.tablet_id_
&& a.merge_type_ == b.merge_type_;
}
TEST_F(TestDiagnoseInfoMgr, test_add_del_suspect_info)
{
int ret = OB_SUCCESS;
ObArenaAllocator allocator;
const ObTabletID tablet_id(1);
ASSERT_EQ(OB_SUCCESS, ObScheduleSuspectInfoMgr::mtl_init(suspect_info_mgr_));
ObScheduleSuspectInfo info;
ObScheduleSuspectInfo ret_info;
ret = gene_suspect_info(ObDiagnoseInfoPrio::DIAGNOSE_PRIORITY_LOW, MINOR_MERGE, tablet_id, info);
ASSERT_EQ(OB_SUCCESS, ret);
ret = suspect_info_mgr_->add_suspect_info(info.hash(), info);
ASSERT_EQ(OB_SUCCESS, ret);
// high priority could cover low priority
info.priority_ = static_cast<uint32_t>(ObDiagnoseInfoPrio::DIAGNOSE_PRIORITY_HIGH);
ret = suspect_info_mgr_->add_suspect_info(info.hash(), info);
ASSERT_EQ(OB_SUCCESS, ret);
ret = suspect_info_mgr_->get_with_param(info.hash(), ret_info, allocator);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(true, judge_equal(info, ret_info));
// mid priority couldn't cover high priority
info.priority_ = static_cast<uint32_t>(ObDiagnoseInfoPrio::DIAGNOSE_PRIORITY_MID);
ret = suspect_info_mgr_->add_suspect_info(info.hash(), info);
ASSERT_EQ(OB_SUCCESS, ret);
ret = suspect_info_mgr_->get_with_param(info.hash(), ret_info, allocator);
ASSERT_EQ(OB_SUCCESS, ret);
// still be high priority
ASSERT_EQ(static_cast<uint32_t>(ObDiagnoseInfoPrio::DIAGNOSE_PRIORITY_HIGH), ret_info.priority_);
}
TEST_F(TestDiagnoseInfoMgr, test_diagnose_tablet_mgr)
{
int ret = OB_SUCCESS;
const ObTabletID tablet_id(1);
ASSERT_EQ(OB_SUCCESS, ObDiagnoseTabletMgr::mtl_init(diagnose_tablet_mgr_));
ret = diagnose_tablet_mgr_->add_diagnose_tablet(ls_id_, tablet_id, TYPE_DIAGNOSE_TABLET_MAX);
ASSERT_EQ(OB_INVALID_ARGUMENT, ret);
ret = diagnose_tablet_mgr_->add_diagnose_tablet(ls_id_, tablet_id, TYPE_MINOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
ret = diagnose_tablet_mgr_->add_diagnose_tablet(ls_id_, tablet_id, TYPE_MINOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
// same diagnose type is registed, return success anyway
ret = diagnose_tablet_mgr_->add_diagnose_tablet(ls_id_, tablet_id, TYPE_MINOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
// same diagnose type is registed, return success anyway
ret = diagnose_tablet_mgr_->add_diagnose_tablet(ls_id_, tablet_id, TYPE_RS_MAJOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
ret = diagnose_tablet_mgr_->delete_diagnose_tablet(ls_id_, tablet_id, TYPE_MINOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(1, diagnose_tablet_mgr_->diagnose_tablet_map_.size());
// after remove all flag, registed tablet is deleted
ret = diagnose_tablet_mgr_->delete_diagnose_tablet(ls_id_, tablet_id, TYPE_RS_MAJOR_MERGE);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(0, diagnose_tablet_mgr_->diagnose_tablet_map_.size());
}
}//end namespace unittest
}//end namespace oceanbase
int main(int argc, char **argv)
{
system("rm -f test_diagnose_info_mgr.log*");
OB_LOGGER.set_file_name("test_diagnose_info_mgr.log");
OB_LOGGER.set_log_level("DEBUG");
testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View File

@ -186,11 +186,11 @@ TEST_F(TestDagWarningHistory, simple_add)
compaction::ObInfoParamBuffer allocator;
ObDagWarningInfo ret_info;
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START+1, &ret_info, allocator);
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START+1, ret_info, allocator);
ASSERT_EQ(OB_HASH_NOT_EXIST, ret);
allocator.reuse();
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, &ret_info, allocator);
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, ret_info, allocator);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(TRUE, ret_info.dag_ret_ == ObBasicDag::DAG_RET_START);
STORAGE_LOG(DEBUG, "", K(ret_info));
@ -198,7 +198,7 @@ TEST_F(TestDagWarningHistory, simple_add)
char comment[common::OB_DAG_WARNING_INFO_LENGTH];
memset(comment, '\0', sizeof(comment));
allocator.reuse();
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, &ret_info, allocator);
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, ret_info, allocator);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ (TRUE, ret_info.dag_ret_ == ObBasicDag::DAG_RET_START);
memset(comment, '\0', sizeof(comment));
@ -226,7 +226,7 @@ TEST_F(TestDagWarningHistory, simple_del)
compaction::ObInfoParamBuffer allocator;
ObDagWarningInfo ret_info;
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, &ret_info, allocator);
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, ret_info, allocator);
ASSERT_EQ(OB_HASH_NOT_EXIST, ret);
ASSERT_EQ(OB_SUCCESS, MTL(ObDagWarningHistoryManager *)->add_dag_warning_info(&dag));
@ -239,7 +239,7 @@ TEST_F(TestDagWarningHistory, simple_del)
ASSERT_EQ(OB_ITER_END, iterator.get_next(&ret_info, nullptr, 0));
allocator.reuse();
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, &ret_info, allocator);
ret = MTL(ObDagWarningHistoryManager *)->get_with_param(ObBasicDag::KEY_START, ret_info, allocator);
ASSERT_EQ(OB_HASH_NOT_EXIST, ret);
}