fix wrong flag in standby rs major checker

This commit is contained in:
obdev
2024-02-07 02:40:40 +00:00
committed by ob-robot
parent 92bfa74556
commit 93bd93e86e
9 changed files with 144 additions and 61 deletions

View File

@ -134,8 +134,12 @@ int ObChecksumValidator::set_need_validate()
if (is_primary_service_) { if (is_primary_service_) {
// need to check index checksum on primary tenant // need to check index checksum on primary tenant
need_validate_index_ckm_ = true; need_validate_index_ckm_ = true;
if (OB_FAIL(check_tablet_checksum_sync_finish())) { if (OB_FAIL(check_tablet_checksum_sync_finish(true /*force_check*/))) {
LOG_WARN("failed to check tablet checksum sync finish", K(ret), K_(is_primary_service)); LOG_WARN("failed to check tablet checksum sync finish", K(ret), K_(is_primary_service));
} else {
// for primary service, if cross cluster ckm sync finish, need to validate cur round checksum & inner_table
// else: write ckm into inner table
need_validate_cross_cluster_ckm_ = cross_cluster_ckm_sync_finish_;
} }
} else { // standby tenant } else { // standby tenant
need_validate_index_ckm_ = false; need_validate_index_ckm_ = false;
@ -173,7 +177,7 @@ int ObChecksumValidator::check_inner_status()
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid compaction_scn/expected_epoch/schema_guard_", KR(ret), K_(tenant_id), LOG_WARN("invalid compaction_scn/expected_epoch/schema_guard_", KR(ret), K_(tenant_id),
K_(compaction_scn), K_(expected_epoch), KP_(schema_guard)); K_(compaction_scn), K_(expected_epoch), KP_(schema_guard));
} else if (OB_FAIL(check_tablet_checksum_sync_finish())) { } else if (OB_FAIL(check_tablet_checksum_sync_finish(false /*force_check*/))) {
LOG_WARN("failed to set need_validate", K(ret), K_(tenant_id)); LOG_WARN("failed to set need_validate", K(ret), K_(tenant_id));
} }
return ret; return ret;
@ -334,6 +338,7 @@ int ObChecksumValidator::update_table_compaction_info_by_tablet()
if (OB_HASH_NOT_EXIST == ret) { if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS; ret = OB_SUCCESS;
table_compaction_info_.set_uncompacted(); table_compaction_info_.set_uncompacted();
(void) uncompact_info_.add_tablet(tenant_id_, cur_tablet_ls_pair_array_.at(idx).get_ls_id(), tablet_id);
LOG_TRACE("tablet not exist in tablet status map", KR(ret), K(tablet_id), LOG_TRACE("tablet not exist in tablet status map", KR(ret), K(tablet_id),
K_(cur_tablet_ls_pair_array), K_(table_compaction_info)); K_(cur_tablet_ls_pair_array), K_(table_compaction_info));
#ifdef ERRSIM #ifdef ERRSIM
@ -522,24 +527,25 @@ int ObChecksumValidator::batch_update_report_scn()
return ret; return ret;
} }
int ObChecksumValidator::check_tablet_checksum_sync_finish() int ObChecksumValidator::check_tablet_checksum_sync_finish(const bool force_check)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
bool is_exist = false; bool is_exist = false;
if (cross_cluster_ckm_sync_finish_) { // need check inner table:
// 1) force check when first init
// 2) ckm not sync finish in standby service
if (!force_check && (is_primary_service_ || cross_cluster_ckm_sync_finish_)) {
} else if (OB_FAIL(ObTabletChecksumOperator::is_first_tablet_in_sys_ls_exist(*sql_proxy_, } else if (OB_FAIL(ObTabletChecksumOperator::is_first_tablet_in_sys_ls_exist(*sql_proxy_,
tenant_id_, compaction_scn_, is_exist))) { tenant_id_, compaction_scn_, is_exist))) {
LOG_WARN("fail to check is first tablet in first ls exist", KR(ret), K_(tenant_id), K_(compaction_scn)); LOG_WARN("fail to check is first tablet in first ls exist", KR(ret), K_(tenant_id), K_(compaction_scn));
} else if (is_exist) { } else if (is_exist) {
cross_cluster_ckm_sync_finish_ = true; cross_cluster_ckm_sync_finish_ = true;
need_validate_cross_cluster_ckm_ = true;
} else { } else {
need_validate_cross_cluster_ckm_ = false;
cross_cluster_ckm_sync_finish_ = check_waiting_tablet_checksum_timeout(); cross_cluster_ckm_sync_finish_ = check_waiting_tablet_checksum_timeout();
if (!is_primary_service_ && TC_REACH_TIME_INTERVAL(PRINT_CROSS_CLUSTER_LOG_INVERVAL)) { if (!is_primary_service_ && TC_REACH_TIME_INTERVAL(PRINT_CROSS_CLUSTER_LOG_INVERVAL)) {
LOG_WARN("can not check cross-cluster checksum now, please wait until first tablet" LOG_WARN("can not check cross-cluster checksum now, please wait until first tablet"
"in sys ls exists", K_(tenant_id), K_(compaction_scn), K_(major_merge_start_us), "in sys ls exists", K_(tenant_id), K_(compaction_scn), K_(major_merge_start_us),
"fast_current_time_us", ObTimeUtil::fast_current_time()); "fast_current_time_us", ObTimeUtil::fast_current_time(), K(is_exist), K_(is_primary_service));
} }
} }
return ret; return ret;

View File

@ -63,7 +63,8 @@ public:
compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array, compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array,
compaction::ObCkmValidatorStatistics &statistics, compaction::ObCkmValidatorStatistics &statistics,
ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array, ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array,
ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array) ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array,
compaction::ObUncompactInfo &uncompact_info)
: is_inited_(false), : is_inited_(false),
is_primary_service_(false), is_primary_service_(false),
need_validate_index_ckm_(false), need_validate_index_ckm_(false),
@ -83,6 +84,7 @@ public:
idx_ckm_validate_array_(idx_ckm_validate_array), idx_ckm_validate_array_(idx_ckm_validate_array),
finish_tablet_ls_pair_array_(finish_tablet_ls_pair_array), finish_tablet_ls_pair_array_(finish_tablet_ls_pair_array),
finish_tablet_ckm_array_(finish_tablet_ckm_array), finish_tablet_ckm_array_(finish_tablet_ckm_array),
uncompact_info_(uncompact_info),
schema_guard_(nullptr), schema_guard_(nullptr),
simple_schema_(nullptr), simple_schema_(nullptr),
table_compaction_info_(), table_compaction_info_(),
@ -141,7 +143,7 @@ private:
/* Cross Cluster Checksum Section */ /* Cross Cluster Checksum Section */
int validate_cross_cluster_checksum(); int validate_cross_cluster_checksum();
int check_tablet_checksum_sync_finish(); int check_tablet_checksum_sync_finish(const bool force_check);
int validate_replica_and_tablet_checksum(); int validate_replica_and_tablet_checksum();
int check_column_checksum( int check_column_checksum(
const ObArray<share::ObTabletReplicaChecksumItem> &tablet_replica_checksum_items, const ObArray<share::ObTabletReplicaChecksumItem> &tablet_replica_checksum_items,
@ -171,6 +173,7 @@ private:
compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array_; compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array_;
ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array_; ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array_;
ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array_; ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array_;
compaction::ObUncompactInfo &uncompact_info_;
/* different for every table */ /* different for every table */
share::schema::ObSchemaGetterGuard *schema_guard_; share::schema::ObSchemaGetterGuard *schema_guard_;

View File

@ -53,10 +53,8 @@ ObMajorMergeProgressChecker::ObMajorMergeProgressChecker(
tablet_status_map_(), table_compaction_map_(), tablet_status_map_(), table_compaction_map_(),
ckm_validator_(tenant_id, stop_, tablet_ls_pair_cache_, tablet_status_map_, ckm_validator_(tenant_id, stop_, tablet_ls_pair_cache_, tablet_status_map_,
table_compaction_map_, idx_ckm_validate_array_, validator_statistics_, table_compaction_map_, idx_ckm_validate_array_, validator_statistics_,
finish_tablet_ls_pair_array_, finish_tablet_ckm_array_), finish_tablet_ls_pair_array_, finish_tablet_ckm_array_, uncompact_info_),
uncompacted_tablets_(), uncompacted_table_ids_(), uncompact_info_(), ls_locality_cache_(), total_time_guard_(), validator_statistics_(), batch_size_mgr_() {}
diagnose_rw_lock_(ObLatchIds::MAJOR_FREEZE_DIAGNOSE_LOCK),
ls_locality_cache_(), total_time_guard_(), validator_statistics_(), batch_size_mgr_() {}
int ObMajorMergeProgressChecker::init( int ObMajorMergeProgressChecker::init(
const bool is_primary_service, const bool is_primary_service,
@ -163,11 +161,7 @@ int ObMajorMergeProgressChecker::clear_cached_info()
ckm_validator_.clear_cached_info(); ckm_validator_.clear_cached_info();
loop_cnt_ = 0; loop_cnt_ = 0;
tablet_ls_pair_cache_.reuse(); tablet_ls_pair_cache_.reuse();
{ reset_uncompacted_tablets();
SpinWLockGuard w_guard(diagnose_rw_lock_);
uncompacted_tablets_.reset();
uncompacted_table_ids_.reset();
}
} }
return ret; return ret;
} }
@ -180,13 +174,8 @@ int ObMajorMergeProgressChecker::get_uncompacted_tablets(
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret), K_(tenant_id)); LOG_WARN("not init", KR(ret), K_(tenant_id));
} else { } else if (OB_FAIL(uncompact_info_.get_uncompact_info(input_tablets, input_table_ids))) {
SpinRLockGuard r_guard(diagnose_rw_lock_); LOG_WARN("fail to get uncompacted info", KR(ret), K_(tenant_id));
if (OB_FAIL(input_tablets.assign(uncompacted_tablets_))) {
LOG_WARN("fail to assign uncompacted_tablets", KR(ret), K_(tenant_id), K_(uncompacted_tablets));
} else if (OB_FAIL(input_table_ids.assign(uncompacted_table_ids_))) {
LOG_WARN("fail to assign uncompacted_tablets", KR(ret), K_(tenant_id), K_(uncompacted_table_ids));
}
} }
return ret; return ret;
} }
@ -231,11 +220,7 @@ int ObMajorMergeProgressChecker::check_verification(
if (OB_TMP_FAIL(unfinish_table_id_array.push_back(table_id))) { if (OB_TMP_FAIL(unfinish_table_id_array.push_back(table_id))) {
LOG_WARN("failed to push table_id into finish_array", KR(tmp_ret), KPC(table_compaction_info_ptr)); LOG_WARN("failed to push table_id into finish_array", KR(tmp_ret), KPC(table_compaction_info_ptr));
} }
} else if (progress_.table_cnt_[INITIAL]++ < DEBUG_INFO_CNT) { // add into uncompacted tablets array to show in diagnose (void) uncompact_info_.add_table(table_id);
SpinWLockGuard w_guard(diagnose_rw_lock_);
if (OB_TMP_FAIL(uncompacted_table_ids_.push_back(table_id))) {
LOG_WARN("fail to push_back", KR(tmp_ret), K_(tenant_id), K_(compaction_scn), K(table_id));
}
} }
if (0 >= index_cnt // data & index should be in same batch if (0 >= index_cnt // data & index should be in same batch
&& (++table_cnt >= table_batch_size)) { && (++table_cnt >= table_batch_size)) {
@ -251,13 +236,6 @@ int ObMajorMergeProgressChecker::check_verification(
return ret; return ret;
} }
void ObMajorMergeProgressChecker::reset_uncompacted_tablets()
{
SpinWLockGuard w_guard(diagnose_rw_lock_);
uncompacted_tablets_.reuse();
uncompacted_table_ids_.reuse();
}
bool ObMajorMergeProgressChecker::should_ignore_cur_table(const ObSimpleTableSchemaV2 *simple_schema) bool ObMajorMergeProgressChecker::should_ignore_cur_table(const ObSimpleTableSchemaV2 *simple_schema)
{ {
bool bret = true; bool bret = true;
@ -620,19 +598,13 @@ int ObMajorMergeProgressChecker::check_progress(
void ObMajorMergeProgressChecker::print_unfinish_info(const int64_t cost_us) void ObMajorMergeProgressChecker::print_unfinish_info(const int64_t cost_us)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSEArray<uint64_t, DEBUG_INFO_CNT> tmp_table_id_array; const int64_t array_cnt = ObUncompactInfo::DEBUG_INFO_CNT;
ObSEArray<ObTabletReplica, DEBUG_INFO_CNT> uncompacted_replica_array; ObSEArray<uint64_t, array_cnt> tmp_table_id_array;
ObSEArray<uint64_t, DEBUG_INFO_CNT> uncompacted_table_array; ObSEArray<ObTabletReplica, array_cnt> uncompacted_replica_array;
{ ObSEArray<uint64_t, array_cnt> uncompacted_table_array;
SpinRLockGuard r_guard(diagnose_rw_lock_); (void) uncompact_info_.get_uncompact_info(uncompacted_replica_array, uncompacted_table_array);
if (OB_FAIL(uncompacted_replica_array.assign(uncompacted_tablets_))) {
LOG_WARN("failed to assgin array", KR(ret));
} else if (OB_FAIL(uncompacted_table_array.assign(uncompacted_table_ids_))) {
LOG_WARN("failed to assgin array", KR(ret));
}
}
if (table_ids_.count() > 0) { if (table_ids_.count() > 0) {
const int64_t table_id_cnt = MIN(DEBUG_INFO_CNT, table_ids_.count()); const int64_t table_id_cnt = MIN(array_cnt, table_ids_.count());
for (int64_t idx = 0; OB_SUCC(ret) && idx < table_id_cnt; ++idx) { for (int64_t idx = 0; OB_SUCC(ret) && idx < table_id_cnt; ++idx) {
if (OB_FAIL(tmp_table_id_array.push_back(table_ids_.at(idx)))) { if (OB_FAIL(tmp_table_id_array.push_back(table_ids_.at(idx)))) {
LOG_WARN("failed to push array", KR(ret)); LOG_WARN("failed to push array", KR(ret));
@ -957,12 +929,7 @@ int ObMajorMergeProgressChecker::generate_tablet_status_map()
LOG_ERROR("ERROR! ERROR! ERROR! find error status tablet replica", KR(ret), K(tablet_info)); LOG_ERROR("ERROR! ERROR! ERROR! find error status tablet replica", KR(ret), K(tablet_info));
} else if (replica_snapshot_scn < compaction_scn_) { } else if (replica_snapshot_scn < compaction_scn_) {
status = ObTabletCompactionStatus::INITIAL; status = ObTabletCompactionStatus::INITIAL;
if (progress_.unmerged_tablet_cnt_++ < DEBUG_INFO_CNT) { // add into uncompacted tablets array to show in diagnose (void) uncompact_info_.add_tablet(*replica);
SpinWLockGuard w_guard(diagnose_rw_lock_);
if (OB_TMP_FAIL(uncompacted_tablets_.push_back(*replica))) {
LOG_WARN("fail to push_back", KR(tmp_ret), K_(tenant_id), K_(compaction_scn), KPC(replica));
}
}
LOG_TRACE("unfinish tablet", KR(ret), K(replica_snapshot_scn), K_(compaction_scn)); LOG_TRACE("unfinish tablet", KR(ret), K(replica_snapshot_scn), K_(compaction_scn));
break; break;
} else if (OB_FAIL(report_scn.convert_for_tx(replica->get_report_scn()))) { // check report_scn } else if (OB_FAIL(report_scn.convert_for_tx(replica->get_report_scn()))) { // check report_scn

View File

@ -68,7 +68,7 @@ public:
int get_uncompacted_tablets( int get_uncompacted_tablets(
common::ObArray<share::ObTabletReplica> &uncompacted_tablets, common::ObArray<share::ObTabletReplica> &uncompacted_tablets,
common::ObArray<uint64_t> &uncompacted_table_ids) const; common::ObArray<uint64_t> &uncompacted_table_ids) const;
void reset_uncompacted_tablets(); OB_INLINE void reset_uncompacted_tablets() { uncompact_info_.reset(); }
int check_progress(compaction::ObMergeProgress &progress); int check_progress(compaction::ObMergeProgress &progress);
const compaction::ObTabletLSPairCache &get_tablet_ls_pair_cache() const { return tablet_ls_pair_cache_; } const compaction::ObTabletLSPairCache &get_tablet_ls_pair_cache() const { return tablet_ls_pair_cache_; }
private: private:
@ -123,7 +123,6 @@ private:
private: private:
static const int64_t ADD_RS_EVENT_INTERVAL = 10L * 60 * 1000 * 1000; // 10m static const int64_t ADD_RS_EVENT_INTERVAL = 10L * 60 * 1000 * 1000; // 10m
static const int64_t PRINT_LOG_INTERVAL = 2 * 60 * 1000 * 1000; // 2m static const int64_t PRINT_LOG_INTERVAL = 2 * 60 * 1000 * 1000; // 2m
static const int64_t DEBUG_INFO_CNT = 3;
static const int64_t DEAL_REST_TABLE_CNT_THRESHOLD = 100; static const int64_t DEAL_REST_TABLE_CNT_THRESHOLD = 100;
static const int64_t DEAL_REST_TABLE_INTERVAL = 10 * 60 * 1000 * 1000L; // 10m static const int64_t DEAL_REST_TABLE_INTERVAL = 10 * 60 * 1000 * 1000L; // 10m
private: private:
@ -147,9 +146,7 @@ private:
// record each table compaction/verify status // record each table compaction/verify status
compaction::ObTableCompactionInfoMap table_compaction_map_; // <table_id, compaction_info> compaction::ObTableCompactionInfoMap table_compaction_map_; // <table_id, compaction_info>
ObChecksumValidator ckm_validator_; ObChecksumValidator ckm_validator_;
common::ObSEArray<share::ObTabletReplica, DEBUG_INFO_CNT> uncompacted_tablets_; // record for diagnose compaction::ObUncompactInfo uncompact_info_;
common::ObSEArray<uint64_t, DEBUG_INFO_CNT> uncompacted_table_ids_; // record for diagnose
common::SpinRWLock diagnose_rw_lock_;
// cache of ls_infos in __all_ls_meta_table // cache of ls_infos in __all_ls_meta_table
share::ObCompactionLocalityCache ls_locality_cache_; share::ObCompactionLocalityCache ls_locality_cache_;
// statistics section // statistics section

View File

@ -295,5 +295,75 @@ int ObTabletLSPairCache::get_tablet_ls_pairs(
return ret; return ret;
} }
/**
* -------------------------------------------------------------------ObUncompactInfo-------------------------------------------------------------------
*/
ObUncompactInfo::ObUncompactInfo()
: diagnose_rw_lock_(ObLatchIds::MAJOR_FREEZE_DIAGNOSE_LOCK),
tablets_(),
table_ids_()
{}
ObUncompactInfo::~ObUncompactInfo()
{
reset();
}
void ObUncompactInfo::reset()
{
SpinWLockGuard w_guard(diagnose_rw_lock_);
tablets_.reuse();
table_ids_.reuse();
}
void ObUncompactInfo::add_table(const uint64_t table_id)
{
int ret = OB_SUCCESS;
SpinWLockGuard w_guard(diagnose_rw_lock_);
if (table_ids_.count() < DEBUG_INFO_CNT
&& OB_FAIL(table_ids_.push_back(table_id))) {
LOG_WARN("fail to push_back", KR(ret), K(table_id));
}
}
void ObUncompactInfo::add_tablet(const share::ObTabletReplica &replica)
{
int ret = OB_SUCCESS;
SpinWLockGuard w_guard(diagnose_rw_lock_);
if (tablets_.count() < DEBUG_INFO_CNT
&& OB_FAIL(tablets_.push_back(replica))) {
LOG_WARN("fail to push_back", KR(ret), K(replica));
}
}
void ObUncompactInfo::add_tablet(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id)
{
int ret = OB_SUCCESS;
ObTabletReplica fake_replica;
fake_replica.fake_for_diagnose(tenant_id, ls_id, tablet_id);
SpinWLockGuard w_guard(diagnose_rw_lock_);
if (tablets_.count() < DEBUG_INFO_CNT
&& OB_FAIL(tablets_.push_back(fake_replica))) {
LOG_WARN("fail to push_back", KR(ret), K(fake_replica));
}
}
int ObUncompactInfo::get_uncompact_info(
ObIArray<ObTabletReplica> &input_tablets,
ObIArray<uint64_t> &input_table_ids) const
{
int ret = OB_SUCCESS;
SpinRLockGuard r_guard(diagnose_rw_lock_);
if (OB_FAIL(input_tablets.assign(tablets_))) {
LOG_WARN("fail to assign uncompacted_tablets", KR(ret), K_(tablets));
} else if (OB_FAIL(input_table_ids.assign(table_ids_))) {
LOG_WARN("fail to assign uncompacted_tablets", KR(ret), K_(table_ids));
}
return ret;
}
} // namespace compaction } // namespace compaction
} // namespace oceanbase } // namespace oceanbase

View File

@ -15,6 +15,10 @@
#include "share/ob_balance_define.h" #include "share/ob_balance_define.h"
namespace oceanbase namespace oceanbase
{ {
namespace share
{
class ObTabletReplica;
}
namespace compaction namespace compaction
{ {
@ -268,6 +272,27 @@ private:
hash::ObHashMap<common::ObTabletID, share::ObLSID> map_; hash::ObHashMap<common::ObTabletID, share::ObLSID> map_;
}; };
struct ObUncompactInfo
{
public:
ObUncompactInfo();
~ObUncompactInfo();
void reset();
void add_table(const uint64_t table_id);
void add_tablet(const share::ObTabletReplica &replica);
void add_tablet(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id);
int get_uncompact_info(
common::ObIArray<share::ObTabletReplica> &input_tablets,
common::ObIArray<uint64_t> &input_table_ids) const;
static const int64_t DEBUG_INFO_CNT = 3;
common::SpinRWLock diagnose_rw_lock_;
common::ObSEArray<share::ObTabletReplica, DEBUG_INFO_CNT> tablets_; // record for diagnose
common::ObSEArray<uint64_t, DEBUG_INFO_CNT> table_ids_; // record for diagnose
};
} // namespace compaction } // namespace compaction
} // namespace oceanbase } // namespace oceanbase

View File

@ -3685,7 +3685,8 @@ int ObDagNetScheduler::loop_running_dag_net_map()
int ObDagNetScheduler::loop_blocking_dag_net_list() int ObDagNetScheduler::loop_blocking_dag_net_list()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
if (OB_UNLIKELY(OB_ISNULL(scheduler_))) { if (OB_ISNULL(scheduler_)) {
ret = OB_ERR_UNEXPECTED;
COMMON_LOG(WARN, "scheduler is null", KP(scheduler_)); COMMON_LOG(WARN, "scheduler is null", KP(scheduler_));
} else { } else {
ObMutexGuard guard(dag_net_map_lock_); ObMutexGuard guard(dag_net_map_lock_);

View File

@ -124,6 +124,16 @@ bool ObTabletReplica::is_equal_for_report(const ObTabletReplica &other) const
return is_equal; return is_equal;
} }
void ObTabletReplica::fake_for_diagnose(const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id)
{
reset();
tenant_id_ = tenant_id;
ls_id_ = ls_id;
tablet_id_ = tablet_id;
}
ObTabletInfo::ObTabletInfo() ObTabletInfo::ObTabletInfo()
: tenant_id_(OB_INVALID_TENANT_ID), : tenant_id_(OB_INVALID_TENANT_ID),
tablet_id_(), tablet_id_(),

View File

@ -74,6 +74,10 @@ public:
const int64_t required_size, const int64_t required_size,
const int64_t report_scn, const int64_t report_scn,
const ScnStatus status); const ScnStatus status);
void fake_for_diagnose(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id);
bool is_equal_for_report(const ObTabletReplica &other) const; bool is_equal_for_report(const ObTabletReplica &other) const;
static bool is_status_valid(const ScnStatus status) static bool is_status_valid(const ScnStatus status)
{ {