BUGFIX: fix deadlock at ls gc and migrate
This commit is contained in:
@ -140,11 +140,12 @@ int ObLogHandler::stop()
|
||||
|
||||
//判断is_apply_done依赖log handler不能再继续append
|
||||
//所以需要is_in_stop_state_置true表示stop阶段已经不能再提交日志
|
||||
int ObLogHandler::safe_to_destroy()
|
||||
int ObLogHandler::safe_to_destroy(bool &is_safe_destroy)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_done = false;
|
||||
LSN end_lsn;
|
||||
is_safe_destroy = true;
|
||||
WLockGuard guard(lock_);
|
||||
if (IS_INIT) {
|
||||
if (palf_handle_.is_valid() || !is_in_stop_state_) {
|
||||
@ -158,6 +159,9 @@ int ObLogHandler::safe_to_destroy()
|
||||
CLOG_LOG(INFO, "wait apply done finish", K(ret), K(is_done), K(end_lsn), KPC(apply_status_));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
is_safe_destroy = false;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
@ -162,7 +162,7 @@ public:
|
||||
// @breif, wait cb append onto apply service all be called
|
||||
// is reentrant and should be called before destroy(),
|
||||
// protect cb will not be used when log handler destroyed
|
||||
int safe_to_destroy();
|
||||
int safe_to_destroy(bool &is_safe_destroy);
|
||||
// @brief append count bytes from the buffer starting at buf to the palf handle, return the LSN and timestamp
|
||||
// @param[in] const void *, the data buffer.
|
||||
// @param[in] const uint64_t, the length of data buffer.
|
||||
|
||||
@ -173,7 +173,7 @@ int ObCheckpointExecutor::update_clog_checkpoint()
|
||||
STORAGE_LOG(ERROR, "locate lsn by logts failed", K(ret), K(ls_id),
|
||||
K(checkpoint_ts), K(checkpoint_ts_in_ls_meta));
|
||||
}
|
||||
} else if (OB_FAIL(ls_->set_clog_checkpoint_without_lock(clog_checkpoint_lsn, checkpoint_ts))) {
|
||||
} else if (OB_FAIL(ls_->set_clog_checkpoint(clog_checkpoint_lsn, checkpoint_ts))) {
|
||||
STORAGE_LOG(WARN, "set clog checkpoint failed", K(ret), K(clog_checkpoint_lsn), K(checkpoint_ts), K(ls_id));
|
||||
} else if (OB_FAIL(loghandler_->advance_base_lsn(clog_checkpoint_lsn))) {
|
||||
if (OB_NOT_INIT == ret) {
|
||||
|
||||
@ -167,10 +167,11 @@ int ObDataCheckpoint::init(ObLS *ls)
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObDataCheckpoint::safe_to_destroy()
|
||||
int ObDataCheckpoint::safe_to_destroy(bool &is_safe_destroy)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
is_safe_destroy = true;
|
||||
// avoid start ls_freeze again after waiting ls_freeze finish
|
||||
is_inited_ = false;
|
||||
// wait until ls_freeze finish
|
||||
@ -191,6 +192,10 @@ int ObDataCheckpoint::safe_to_destroy()
|
||||
prepare_list_.reset();
|
||||
ls_ = nullptr;
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
is_safe_destroy = false;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
@ -87,7 +87,7 @@ public:
|
||||
// used for virtual table
|
||||
static const uint64_t LS_DATA_CHECKPOINT_TABLET_ID = 40000;
|
||||
int init(ObLS *ls);
|
||||
int safe_to_destroy();
|
||||
int safe_to_destroy(bool &is_safe_destroy);
|
||||
int64_t get_rec_log_ts() override;
|
||||
// if min_rec_log_ts <= the input rec_log_ts
|
||||
// logstream freeze
|
||||
|
||||
@ -358,22 +358,18 @@ int ObTenantTabletScheduler::update_upper_trans_version_and_gc_sstable()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTenantTabletScheduler::wait_ls_compaction_finish(const share::ObLSID &ls_id)
|
||||
int ObTenantTabletScheduler::check_ls_compaction_finish(const share::ObLSID &ls_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool exist = false;
|
||||
if (OB_UNLIKELY(!ls_id.is_valid())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(ls_id));
|
||||
}
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(MTL(ObTenantDagScheduler*)->check_ls_compaction_dag_exist(ls_id, exist))) {
|
||||
LOG_WARN("failed to check ls compaction dag", K(ret), K(ls_id));
|
||||
} else if (!exist) {
|
||||
break;
|
||||
} else {
|
||||
ob_usleep(100 * 1000); // 100ms
|
||||
}
|
||||
} else if (OB_FAIL(MTL(ObTenantDagScheduler*)->check_ls_compaction_dag_exist(ls_id, exist))) {
|
||||
LOG_WARN("failed to check ls compaction dag", K(ret), K(ls_id));
|
||||
} else if (exist) {
|
||||
// the compaction dag exists, need retry later.
|
||||
ret = OB_EAGAIN;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -112,7 +112,7 @@ public:
|
||||
int merge_all();
|
||||
int schedule_merge(const int64_t broadcast_version);
|
||||
int update_upper_trans_version_and_gc_sstable();
|
||||
int wait_ls_compaction_finish(const share::ObLSID &ls_id);
|
||||
int check_ls_compaction_finish(const share::ObLSID &ls_id);
|
||||
|
||||
// Schedule an async task to build bloomfilter for the given macro block.
|
||||
// The bloomfilter build task will be ignored if a same build task exists in the queue.
|
||||
|
||||
@ -338,6 +338,7 @@ int ObLSCompleteMigrationDagNet::update_migration_status_(ObLS *ls)
|
||||
LOG_WARN("tenant dag scheduler has set stop, stop migration dag net", K(ret), K(ctx_));
|
||||
break;
|
||||
} else {
|
||||
// TODO: muwei should not do this before ls create finished.
|
||||
if (OB_FAIL(ls->get_migration_status(current_migration_status))) {
|
||||
LOG_WARN("failed to get migration status", K(ret), K(ctx_));
|
||||
} else if (ctx_.is_failed()) {
|
||||
@ -353,7 +354,7 @@ int ObLSCompleteMigrationDagNet::update_migration_status_(ObLS *ls)
|
||||
}
|
||||
} else {
|
||||
if (ObMigrationOpType::REBUILD_LS_OP == ctx_.arg_.type_
|
||||
&& OB_FAIL(ls->clear_saved_info_without_lock())) {
|
||||
&& OB_FAIL(ls->clear_saved_info())) {
|
||||
LOG_WARN("failed to clear ls saved info", K(ret), KPC(ls));
|
||||
} else {
|
||||
new_migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_NONE;
|
||||
|
||||
@ -1190,11 +1190,10 @@ int ObStartMigrationTask::update_ls_()
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls should not be NULL", K(ret), KPC(ctx_));
|
||||
} else {
|
||||
ObLSMetaPackage local_ls_meta_package;
|
||||
ObLSLockGuard lock_ls(ls);
|
||||
const bool is_rebuild = (ctx_->arg_.type_ == ObMigrationOpType::REBUILD_LS_OP);
|
||||
if (OB_FAIL(ls->update_ls_meta_without_lock(update_restore_status,
|
||||
ctx_->src_ls_meta_package_.ls_meta_))) {
|
||||
if (OB_FAIL(ls->update_ls_meta(update_restore_status,
|
||||
ctx_->src_ls_meta_package_.ls_meta_))) {
|
||||
LOG_WARN("failed to update ls meta", K(ret), KPC(ctx_));
|
||||
} else if (OB_FAIL(ls->get_end_lsn(end_lsn))) {
|
||||
LOG_WARN("failed to get end lsn", K(ret), KPC(ctx_));
|
||||
|
||||
@ -932,11 +932,8 @@ int ObLSMigrationHandler::report_meta_table_()
|
||||
LOG_WARN("failed ot get result", K(ret));
|
||||
} else if (OB_SUCCESS != result) {
|
||||
//do nothing
|
||||
} else {
|
||||
ObLSLockGuard lock_ls(ls_);
|
||||
if (OB_FAIL(ls_->report_replica_info())) {
|
||||
LOG_WARN("failed to report replica info", K(ret), KPC(ls_));
|
||||
}
|
||||
} else if (OB_FAIL(ls_->report_replica_info())) {
|
||||
LOG_WARN("failed to report replica info", K(ret), KPC(ls_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1179,27 +1176,26 @@ void ObLSMigrationHandler::stop()
|
||||
}
|
||||
}
|
||||
|
||||
int ObLSMigrationHandler::safe_to_destroy(
|
||||
bool &is_safe_to_destroy)
|
||||
void ObLSMigrationHandler::wait(bool &wait_finished)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_safe_to_destroy = false;
|
||||
wait_finished = false;
|
||||
ObLSMigrationTask task;
|
||||
|
||||
if (OB_FAIL(get_ls_migration_task_(task))) {
|
||||
if (OB_ENTRY_NOT_EXIST == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
is_safe_to_destroy = true;
|
||||
wait_finished = true;
|
||||
} else {
|
||||
LOG_WARN("failed to get ls migration task", K(ret), KPC(ls_));
|
||||
}
|
||||
} else {
|
||||
is_safe_to_destroy = false;
|
||||
wait_finished = false;
|
||||
wakeup_();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@ -83,7 +83,7 @@ public:
|
||||
int check_task_exist(const share::ObTaskId &task_id, bool &is_exist);
|
||||
void destroy();
|
||||
void stop();
|
||||
int safe_to_destroy(bool &is_safe_to_destroy);
|
||||
void wait(bool &wait_finished);
|
||||
|
||||
private:
|
||||
void reuse_();
|
||||
|
||||
@ -205,6 +205,7 @@ ObFreezer::ObFreezer()
|
||||
high_priority_freeze_cnt_(0),
|
||||
low_priority_freeze_cnt_(0),
|
||||
need_resubmit_log_(false),
|
||||
enable_(true),
|
||||
is_inited_(false)
|
||||
{}
|
||||
|
||||
@ -228,6 +229,7 @@ ObFreezer::ObFreezer(ObLSWRSHandler *ls_loop_worker,
|
||||
high_priority_freeze_cnt_(0),
|
||||
low_priority_freeze_cnt_(0),
|
||||
need_resubmit_log_(false),
|
||||
enable_(true),
|
||||
is_inited_(false)
|
||||
{}
|
||||
|
||||
@ -274,6 +276,7 @@ void ObFreezer::reset()
|
||||
high_priority_freeze_cnt_ = 0;
|
||||
low_priority_freeze_cnt_ = 0;
|
||||
need_resubmit_log_ = false;
|
||||
enable_ = true;
|
||||
is_inited_ = false;
|
||||
}
|
||||
|
||||
@ -316,6 +319,8 @@ int ObFreezer::logstream_freeze()
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("[Freezer] not inited", K(ret), K_(ls_id));
|
||||
} else if (OB_UNLIKELY(!enable_)) {
|
||||
LOG_WARN("freezer is offline, can not freeze now", K(ret), K_(ls_id));
|
||||
} else if (OB_FAIL(decide_max_decided_log_ts(max_decided_log_ts))) {
|
||||
TRANS_LOG(WARN, "[Freezer] decide max decided log ts failure", K(ret), K_(ls_id));
|
||||
} else if (OB_FAIL(get_ls_weak_read_ts(freeze_snapshot_version))) {
|
||||
@ -402,6 +407,8 @@ int ObFreezer::tablet_freeze(const ObTabletID &tablet_id)
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
TRANS_LOG(WARN, "[Freezer] not inited", K(ret), K_(ls_id), K(tablet_id));
|
||||
} else if (OB_UNLIKELY(!enable_)) {
|
||||
LOG_WARN("freezer is offline, can not freeze now", K(ret), K_(ls_id));
|
||||
} else if (OB_FAIL(guard.try_set_tablet_freeze_begin())) {
|
||||
// no need freeze now, a ls freeze is running or will be running
|
||||
ret = OB_SUCCESS;
|
||||
@ -478,6 +485,8 @@ int ObFreezer::force_tablet_freeze(const ObTabletID &tablet_id)
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
TRANS_LOG(WARN, "[Freezer] not inited", K(ret), K_(ls_id), K(tablet_id));
|
||||
} else if (OB_UNLIKELY(!enable_)) {
|
||||
LOG_WARN("freezer is offline, can not freeze now", K(ret), K_(ls_id));
|
||||
} else if (OB_FAIL(loop_set_freeze_flag())) {
|
||||
TRANS_LOG(WARN, "[Freezer] failed to set freeze_flag", K(ret), K_(ls_id), K(tablet_id));
|
||||
} else if (FALSE_IT(stat_.state_ = ObFreezeState::NOT_SUBMIT_LOG)) {
|
||||
|
||||
@ -205,6 +205,8 @@ public:
|
||||
const share::ObLSID &ls_id,
|
||||
uint32_t freeze_flag = 0);
|
||||
void reset();
|
||||
void offline() { enable_ = false; }
|
||||
void online() { enable_ = true; }
|
||||
|
||||
public:
|
||||
/* freeze */
|
||||
@ -320,6 +322,7 @@ private:
|
||||
int64_t low_priority_freeze_cnt_; // freeze tablet cnt
|
||||
|
||||
bool need_resubmit_log_;
|
||||
bool enable_; // whether we can do freeze now
|
||||
|
||||
bool is_inited_;
|
||||
};
|
||||
|
||||
@ -12,6 +12,7 @@
|
||||
|
||||
#define USING_LOG_PREFIX STORAGE
|
||||
|
||||
#include "lib/utility/utility.h"
|
||||
#include "share/ob_tenant_info_proxy.h"
|
||||
#include "storage/ls/ob_ls.h"
|
||||
#include "share/leak_checker/obj_leak_checker.h"
|
||||
@ -22,6 +23,7 @@
|
||||
#include "logservice/ob_log_service.h"
|
||||
#include "storage/tx/ob_trans_service.h"
|
||||
#include "observer/report/ob_i_meta_report.h"
|
||||
#include "storage/compaction/ob_tenant_tablet_scheduler.h"
|
||||
#include "storage/tx/ob_tx_log_adapter.h"
|
||||
#include "storage/tx_table/ob_tx_table.h"
|
||||
#include "storage/slog/ob_storage_log.h"
|
||||
@ -42,6 +44,7 @@ namespace oceanbase
|
||||
{
|
||||
using namespace share;
|
||||
using namespace logservice;
|
||||
using namespace transaction;
|
||||
|
||||
namespace storage
|
||||
{
|
||||
@ -62,6 +65,7 @@ ObLS::ObLS()
|
||||
is_inited_(false),
|
||||
tenant_id_(OB_INVALID_TENANT_ID),
|
||||
is_stopped_(false),
|
||||
is_offlined_(false),
|
||||
ls_meta_(),
|
||||
rs_reporter_(nullptr)
|
||||
{}
|
||||
@ -339,12 +343,12 @@ int ObLS::remove_ls()
|
||||
|
||||
void ObLS::set_create_state(const ObInnerLSStatus new_status)
|
||||
{
|
||||
ls_meta_.ls_create_status_ = new_status;
|
||||
ls_meta_.set_ls_create_status(new_status);
|
||||
}
|
||||
|
||||
ObInnerLSStatus ObLS::get_create_state()
|
||||
ObInnerLSStatus ObLS::get_create_state() const
|
||||
{
|
||||
return ls_meta_.ls_create_status_;
|
||||
return ls_meta_.get_ls_create_status();
|
||||
}
|
||||
|
||||
bool ObLS::is_need_gc() const
|
||||
@ -352,14 +356,15 @@ bool ObLS::is_need_gc() const
|
||||
int ret = OB_SUCCESS;
|
||||
bool bool_ret = false;
|
||||
ObMigrationStatus migration_status;
|
||||
ObInnerLSStatus create_status = ls_meta_.get_ls_create_status();
|
||||
|
||||
if (OB_FAIL(ls_meta_.get_migration_status(migration_status))) {
|
||||
LOG_WARN("get migration status failed", K(ret), K(ls_meta_.ls_id_));
|
||||
}
|
||||
|
||||
bool_ret = (OB_FAIL(ret) ||
|
||||
(ObInnerLSStatus::CREATING == ls_meta_.ls_create_status_) ||
|
||||
(ObInnerLSStatus::REMOVED == ls_meta_.ls_create_status_) ||
|
||||
(ObInnerLSStatus::CREATING == create_status) ||
|
||||
(ObInnerLSStatus::REMOVED == create_status) ||
|
||||
(OB_MIGRATION_STATUS_NONE != migration_status &&
|
||||
ObMigrationStatusHelper::check_allow_gc(migration_status)));
|
||||
if (bool_ret) {
|
||||
@ -383,11 +388,13 @@ bool ObLS::is_need_load_inner_tablet() const
|
||||
|
||||
void ObLS::finish_create(const bool is_commit)
|
||||
{
|
||||
ObInnerLSStatus status = ObInnerLSStatus::CREATING;
|
||||
if (is_commit) {
|
||||
ls_meta_.ls_create_status_ = ObInnerLSStatus::COMMITTED;
|
||||
status = ObInnerLSStatus::COMMITTED;
|
||||
} else {
|
||||
ls_meta_.ls_create_status_ = ObInnerLSStatus::ABORTED;
|
||||
status = ObInnerLSStatus::ABORTED;
|
||||
}
|
||||
ls_meta_.set_ls_create_status(status);
|
||||
}
|
||||
|
||||
int ObLS::start()
|
||||
@ -408,34 +415,88 @@ int ObLS::start()
|
||||
|
||||
int ObLS::stop()
|
||||
{
|
||||
int64_t read_lock = 0;
|
||||
int64_t write_lock = LSLOCKALL;
|
||||
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
int ret = OB_SUCCESS;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls is not inited", K(ret));
|
||||
} else if (OB_FAIL(stop_())) {
|
||||
LOG_WARN("stop ls failed", K(ret), K(ls_meta_));
|
||||
} else {
|
||||
tx_table_.stop();
|
||||
ls_restore_handler_.stop();
|
||||
keep_alive_ls_handler_.stop();
|
||||
log_handler_.reset_election_priority();
|
||||
restore_handler_.stop();
|
||||
if (OB_FAIL(log_handler_.stop())) {
|
||||
LOG_WARN("stop log handler failed", K(ret), KPC(this));
|
||||
}
|
||||
ls_migration_handler_.stop();
|
||||
is_stopped_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(prepare_for_safe_destroy_())) {
|
||||
LOG_WARN("fail to prepare_for_safe_destroy", K(ret));
|
||||
} else {
|
||||
LOG_INFO("stop_ls finish", KR(ret), KPC(this));
|
||||
}
|
||||
int ObLS::stop_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
tx_table_.stop();
|
||||
ls_restore_handler_.stop();
|
||||
keep_alive_ls_handler_.stop();
|
||||
log_handler_.reset_election_priority();
|
||||
restore_handler_.stop();
|
||||
if (OB_FAIL(log_handler_.stop())) {
|
||||
LOG_WARN("stop log handler failed", K(ret), KPC(this));
|
||||
}
|
||||
ls_migration_handler_.stop();
|
||||
is_stopped_ = true;
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(prepare_for_safe_destroy_())) {
|
||||
LOG_WARN("fail to prepare_for_safe_destroy", K(ret));
|
||||
} else {
|
||||
LOG_INFO("stop_ls finish", KR(ret), KPC(this));
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObLS::wait()
|
||||
{
|
||||
ObTimeGuard time_guard("ObLS::wait", 10 * 1000 * 1000);
|
||||
int64_t read_lock = LSLOCKALL;
|
||||
int64_t write_lock = 0;
|
||||
bool wait_finished = true;
|
||||
int64_t start_ts = ObTimeUtility::current_time();
|
||||
int64_t retry_times = 0;
|
||||
|
||||
do {
|
||||
retry_times++;
|
||||
{
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
ls_migration_handler_.wait(wait_finished);
|
||||
}
|
||||
if (!wait_finished) {
|
||||
ob_usleep(100 * 1000); // 100 ms
|
||||
if (retry_times % 100 == 0) { // every 10 s
|
||||
LOG_WARN("ls wait not finished.", K(ls_meta_), K(start_ts));
|
||||
}
|
||||
}
|
||||
} while (!wait_finished);
|
||||
}
|
||||
|
||||
void ObLS::wait_()
|
||||
{
|
||||
ObTimeGuard time_guard("ObLS::wait", 10 * 1000 * 1000);
|
||||
bool wait_finished = true;
|
||||
int64_t start_ts = ObTimeUtility::current_time();
|
||||
int64_t retry_times = 0;
|
||||
do {
|
||||
retry_times++;
|
||||
ls_migration_handler_.wait(wait_finished);
|
||||
if (!wait_finished) {
|
||||
ob_usleep(100 * 1000); // 100 ms
|
||||
if (retry_times % 100 == 0) { // every 10 s
|
||||
LOG_WARN("ls wait not finished.", K(ls_meta_), K(start_ts));
|
||||
}
|
||||
}
|
||||
} while (!wait_finished);
|
||||
}
|
||||
|
||||
// a class should implement prepare_for_safe_destroy() if it has
|
||||
// resource which depend on ls. the resource here is refer to all kinds of
|
||||
// memtables which are delayed GC in t3m due to performance problem.
|
||||
@ -458,21 +519,22 @@ bool ObLS::safe_to_destroy()
|
||||
bool is_safe = false;
|
||||
bool is_ls_restore_handler_safe = false;
|
||||
bool is_tablet_service_safe = false;
|
||||
bool is_ls_migration_handler_safe = false;
|
||||
bool is_data_check_point_safe = false;
|
||||
bool is_log_handler_safe = false;
|
||||
|
||||
if (OB_FAIL(ls_tablet_svr_.safe_to_destroy(is_tablet_service_safe))) {
|
||||
LOG_WARN("ls tablet service check safe to destroy failed", K(ret), KPC(this));
|
||||
} else if (OB_FAIL(data_checkpoint_.safe_to_destroy())) {
|
||||
} else if (!is_tablet_service_safe) {
|
||||
} else if (OB_FAIL(data_checkpoint_.safe_to_destroy(is_data_check_point_safe))) {
|
||||
LOG_WARN("data_checkpoint check safe to destroy failed", K(ret), KPC(this));
|
||||
} else if (!is_data_check_point_safe) {
|
||||
} else if (OB_FAIL(ls_restore_handler_.safe_to_destroy(is_ls_restore_handler_safe))) {
|
||||
LOG_WARN("ls restore handler safe to destroy failed", K(ret), KPC(this));
|
||||
} else if (!is_ls_restore_handler_safe) {
|
||||
} else if (OB_FAIL(log_handler_.safe_to_destroy())) {
|
||||
} else if (OB_FAIL(log_handler_.safe_to_destroy(is_log_handler_safe))) {
|
||||
LOG_WARN("log_handler_ check safe to destroy failed", K(ret), KPC(this));
|
||||
} else if (OB_FAIL(ls_migration_handler_.safe_to_destroy(is_ls_migration_handler_safe))) {
|
||||
LOG_WARN("ls migration handler check safe to destroy failed", K(ret), KPC(this));
|
||||
} else if (!is_ls_migration_handler_safe) {
|
||||
} else if (is_tablet_service_safe) {
|
||||
} else if (!is_log_handler_safe) {
|
||||
} else {
|
||||
if (1 == ref_mgr_.get_total_ref_cnt()) { // only has one ref at the safe destroy task
|
||||
is_safe = true;
|
||||
}
|
||||
@ -482,8 +544,10 @@ bool ObLS::safe_to_destroy()
|
||||
if (OB_SUCC(ret)) {
|
||||
if (!is_safe) {
|
||||
if (REACH_TIME_INTERVAL(60 * 1000 * 1000)) {
|
||||
LOG_WARN("this ls is not safe to destroy", K(is_safe), "ls_ref", ref_mgr_.get_total_ref_cnt(),
|
||||
K(is_tablet_service_safe), K(is_ls_restore_handler_safe), K(is_ls_migration_handler_safe),
|
||||
LOG_WARN("this ls is not safe to destroy", K(is_safe),
|
||||
K(is_tablet_service_safe), K(is_data_check_point_safe),
|
||||
K(is_ls_restore_handler_safe), K(is_log_handler_safe),
|
||||
"ls_ref", ref_mgr_.get_total_ref_cnt(),
|
||||
K(ret), KP(this), KPC(this));
|
||||
ref_mgr_.print();
|
||||
PRINT_OBJ_LEAK(MTL_ID(), share::LEAK_CHECK_OBJ_LS_HANDLE);
|
||||
@ -510,8 +574,10 @@ void ObLS::destroy()
|
||||
FLOG_INFO("ObLS destroy", K(this), K(*this), K(lbt()));
|
||||
if (OB_TMP_FAIL(offline_())) {
|
||||
LOG_WARN("ls offline failed.", K(tmp_ret), K(ls_meta_.ls_id_));
|
||||
} else if (OB_TMP_FAIL(stop())) {
|
||||
} else if (OB_TMP_FAIL(stop_())) {
|
||||
LOG_WARN("ls stop failed.", K(tmp_ret), K(ls_meta_.ls_id_));
|
||||
} else {
|
||||
wait_();
|
||||
}
|
||||
UNREGISTER_FROM_LOGSERVICE(logservice::TRANS_SERVICE_LOG_BASE_TYPE, &ls_tx_svr_);
|
||||
UNREGISTER_FROM_LOGSERVICE(logservice::STORAGE_SCHEMA_LOG_BASE_TYPE, &ls_tablet_svr_);
|
||||
@ -597,6 +663,17 @@ int ObLS::offline_tx_()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::offline_compaction_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (FALSE_IT(ls_freezer_.offline())) {
|
||||
} else if (OB_FAIL(MTL(ObTenantTabletScheduler *)->
|
||||
check_ls_compaction_finish(ls_meta_.ls_id_))) {
|
||||
LOG_WARN("check compaction finish failed", K(ret), K(ls_meta_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::offline_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -604,8 +681,11 @@ int ObLS::offline_()
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls is not inited", K(ret));
|
||||
} else if (FALSE_IT(is_offlined_ = true)) {
|
||||
} else if (FALSE_IT(checkpoint_executor_.offline())) {
|
||||
LOG_WARN("checkpoint executor offline failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(offline_compaction_())) {
|
||||
LOG_WARN("compaction offline failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(ls_wrs_handler_.offline())) {
|
||||
LOG_WARN("weak read handler offline failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(log_handler_.offline())) {
|
||||
@ -654,6 +734,13 @@ int ObLS::online_tx_()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::online_compaction_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ls_freezer_.online();
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::online()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -676,9 +763,12 @@ int ObLS::online()
|
||||
LOG_WARN("failed to online log", K(ret));
|
||||
} else if (OB_FAIL(ls_wrs_handler_.online())) {
|
||||
LOG_WARN("weak read handler online failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(online_compaction_())) {
|
||||
LOG_WARN("compaction online failed", K(ret), K(ls_meta_));
|
||||
} else if (FALSE_IT(checkpoint_executor_.online())) {
|
||||
LOG_WARN("checkpoint executor online failed", K(ret), K(ls_meta_));
|
||||
} else {
|
||||
is_offlined_ = false;
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@ -711,8 +801,13 @@ int ObLS::set_ls_meta(const ObLSMeta &ls_meta)
|
||||
LOG_WARN("ls is not inited", K(ret));
|
||||
} else {
|
||||
ls_meta_ = ls_meta;
|
||||
if (OB_FAIL(ls_meta_.update_id_service())) {
|
||||
LOG_WARN("update id service fail", K(ret));
|
||||
if (IDS_LS == ls_meta_.ls_id_) {
|
||||
ObAllIDMeta all_id_meta;
|
||||
if (OB_FAIL(ls_meta_.get_all_id_meta(all_id_meta))) {
|
||||
LOG_WARN("get all id meta failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(ObIDService::update_id_service(all_id_meta))) {
|
||||
LOG_WARN("update id service fail", K(ret), K(all_id_meta), K(*this));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -845,6 +940,9 @@ int ObLS::update_tablet_table_store(
|
||||
ObTabletHandle &handle)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t read_lock = LSLOCKLOGMETA;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls is not inited", K(ret));
|
||||
@ -852,9 +950,6 @@ int ObLS::update_tablet_table_store(
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("update tablet table store get invalid argument", K(ret), K(tablet_id), K(param));
|
||||
} else {
|
||||
int64_t read_lock = LSLOCKSTORAGE;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
const int64_t rebuild_seq = ls_meta_.get_rebuild_seq();
|
||||
if (param.rebuild_seq_ != rebuild_seq) {
|
||||
ret = OB_EAGAIN;
|
||||
@ -872,6 +967,10 @@ int ObLS::build_ha_tablet_new_table_store(
|
||||
const ObBatchUpdateTableStoreParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t read_lock = LSLOCKLOGMETA;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
const int64_t rebuild_seq = ls_meta_.get_rebuild_seq();
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls is not inited", K(ret));
|
||||
@ -879,10 +978,6 @@ int ObLS::build_ha_tablet_new_table_store(
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("build ha tablet new table store get invalid argument", K(ret), K(tablet_id), K(param));
|
||||
} else {
|
||||
int64_t read_lock = LSLOCKSTORAGE;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
const int64_t rebuild_seq = ls_meta_.get_rebuild_seq();
|
||||
if (param.rebuild_seq_ != rebuild_seq) {
|
||||
ret = OB_EAGAIN;
|
||||
LOG_WARN("build ha tablet new table store rebuild seq not same, need retry",
|
||||
@ -909,7 +1004,7 @@ int ObLS::finish_slog_replay()
|
||||
new_migration_status))) {
|
||||
LOG_WARN("failed to trans fail status", K(ret), K(current_migration_status),
|
||||
K(new_migration_status));
|
||||
} else if (can_update_ls_meta(ls_meta_.ls_create_status_) &&
|
||||
} else if (can_update_ls_meta(ls_meta_.get_ls_create_status()) &&
|
||||
OB_FAIL(ls_meta_.set_migration_status(new_migration_status, false /*no need write slog*/))) {
|
||||
LOG_WARN("failed to set migration status", K(ret), K(new_migration_status));
|
||||
} else if (is_need_gc()) {
|
||||
@ -1203,7 +1298,7 @@ int ObLS::try_update_uppder_trans_version()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
int64_t read_lock = LSLOCKLS | LSLOCKSTORAGE;
|
||||
int64_t read_lock = LSLOCKLOGMETA;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
|
||||
@ -1251,11 +1346,15 @@ int ObLS::try_update_uppder_trans_version()
|
||||
int ObLS::set_tablet_change_checkpoint_ts(const int64_t log_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t read_lock = LSLOCKLS;
|
||||
int64_t read_lock = 0;
|
||||
int64_t write_lock = LSLOCKLOGMETA;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
UPDATE_LS_META_CHECK();
|
||||
if (OB_FAIL(ret)) {
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ls is not inited", K(ret), K(ls_meta_));
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
LOG_WARN("ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (OB_FAIL(ls_meta_.set_tablet_change_checkpoint_ts(log_ts))) {
|
||||
LOG_WARN("fail to set tablet_change_checkpoint_ts", K(ret), K(log_ts), K_(ls_meta));
|
||||
} else {
|
||||
@ -1264,47 +1363,25 @@ int ObLS::set_tablet_change_checkpoint_ts(const int64_t log_ts)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::update_id_meta_with_writing_slog(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts)
|
||||
int ObLS::update_ls_meta(const bool update_restore_status,
|
||||
const ObLSMeta &src_ls_meta)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t read_lock = LSLOCKLS;
|
||||
int64_t write_lock = 0;
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
|
||||
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
STORAGE_LOG(WARN, "ls is not inited", K(ret), K(ls_meta_));
|
||||
LOG_WARN("ls is not inited", K(ret), K(ls_meta_));
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(ls_meta_.update_id_meta(service_type, limited_id, latest_log_ts, true))) {
|
||||
STORAGE_LOG(WARN, "update id meta with slog fail", K(ret), K_(ls_meta), K(service_type), K(limited_id), K(latest_log_ts));
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLS::update_id_meta_without_writing_slog(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
STORAGE_LOG(WARN, "ls is not inited", K(ret), K(ls_meta_));
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (OB_FAIL(ls_meta_.update_id_meta(service_type, limited_id, latest_log_ts, false))) {
|
||||
STORAGE_LOG(WARN, "update id meta fail", K(ret), K_(ls_meta), K(service_type), K(limited_id), K(latest_log_ts));
|
||||
LOG_WARN("ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (OB_FAIL(ls_meta_.update_ls_meta(update_restore_status, src_ls_meta))) {
|
||||
LOG_WARN("update ls meta fail", K(ret), K_(ls_meta), K(update_restore_status), K(src_ls_meta));
|
||||
} else if (IDS_LS == ls_meta_.ls_id_) {
|
||||
ObAllIDMeta all_id_meta;
|
||||
if (OB_FAIL(ls_meta_.get_all_id_meta(all_id_meta))) {
|
||||
LOG_WARN("get all id meta failed", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(ObIDService::update_id_service(all_id_meta))) {
|
||||
LOG_WARN("update id service fail", K(ret), K(all_id_meta), K(*this));
|
||||
}
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
@ -1363,10 +1440,9 @@ int ObLS::set_migration_status(
|
||||
} else if (!ObMigrationStatusHelper::is_valid(migration_status)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("set migration status get invalid argument", K(ret), K(migration_status));
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
|
||||
// migration status should be update after ls stopped, to make sure migrate task
|
||||
// will be finished later.
|
||||
} else if (!can_update_ls_meta(ls_meta_.get_ls_create_status())) {
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
|
||||
} else if (ls_meta_.get_rebuild_seq() != rebuild_seq) {
|
||||
@ -1402,10 +1478,9 @@ int ObLS::set_restore_status(
|
||||
} else if (!restore_status.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("set restore status get invalid argument", K(ret), K(restore_status));
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
|
||||
// restore status should be update after ls stopped, to make sure restore task
|
||||
// will be finished later.
|
||||
} else if (!can_update_ls_meta(ls_meta_.get_ls_create_status())) {
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
|
||||
} else if (ls_meta_.get_rebuild_seq() != rebuild_seq) {
|
||||
@ -1438,7 +1513,7 @@ int ObLS::set_ls_rebuild()
|
||||
} else if (OB_UNLIKELY(is_stopped_)) {
|
||||
ret = OB_NOT_RUNNING;
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta));
|
||||
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
|
||||
} else if (!can_update_ls_meta(ls_meta_.get_ls_create_status())) {
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
|
||||
} else if (OB_FAIL(ls_meta_.set_ls_rebuild())) {
|
||||
|
||||
@ -65,48 +65,6 @@ class ObIMetaReport;
|
||||
|
||||
namespace storage
|
||||
{
|
||||
#define UPDATE_LS_META_CHECK() \
|
||||
if (IS_NOT_INIT) { \
|
||||
ret = OB_NOT_INIT; \
|
||||
STORAGE_LOG(WARN, "ls is not inited", K(ret), K(ls_meta_)); \
|
||||
} else if (OB_UNLIKELY(is_stopped_)) { \
|
||||
ret = OB_NOT_RUNNING; \
|
||||
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta)); \
|
||||
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) { \
|
||||
ret = OB_STATE_NOT_MATCH; \
|
||||
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_)); \
|
||||
}
|
||||
|
||||
#define UPDATE_LS_META_CHECK_WITH_LOCK() \
|
||||
int64_t read_lock = LSLOCKLS; \
|
||||
int64_t write_lock = 0; \
|
||||
ObLSLockGuard lock_myself(lock_, read_lock, write_lock); \
|
||||
UPDATE_LS_META_CHECK();
|
||||
|
||||
#define UPDATE_LSMETA_WITH_LOCK(delegate_obj, func_name) \
|
||||
template <typename ...Args> \
|
||||
int func_name(Args &&...args) { \
|
||||
int ret = OB_SUCCESS; \
|
||||
UPDATE_LS_META_CHECK_WITH_LOCK(); \
|
||||
if (OB_FAIL(ret)) { \
|
||||
} else { \
|
||||
ret = delegate_obj.func_name(std::forward<Args>(args)...); \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define UPDATE_LSMETA_WITHOUT_LOCK(delegate_obj, func_name) \
|
||||
template <typename ...Args> \
|
||||
int func_name##_without_lock(Args &&...args) { \
|
||||
int ret = OB_SUCCESS; \
|
||||
UPDATE_LS_META_CHECK(); \
|
||||
if (OB_FAIL(ret)) { \
|
||||
} else { \
|
||||
ret = delegate_obj.func_name(std::forward<Args>(args)...); \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
const static int64_t LS_INNER_TABLET_FROZEN_TIMESTAMP = 1;
|
||||
|
||||
struct ObLSVTInfo
|
||||
@ -185,11 +143,12 @@ public:
|
||||
// I am ready to work now.
|
||||
int start();
|
||||
int stop();
|
||||
void wait();
|
||||
bool safe_to_destroy();
|
||||
void destroy();
|
||||
int offline();
|
||||
int online();
|
||||
bool is_offline() const { return false; } // mock function, TODO(@yanyuan)
|
||||
bool is_offline() const { return is_offlined_; } // mock function, TODO(@yanyuan)
|
||||
|
||||
ObLSTxService *get_tx_svr() { return &ls_tx_svr_; }
|
||||
ObLockTable *get_lock_table() { return &lock_table_; }
|
||||
@ -233,7 +192,7 @@ public:
|
||||
// set create state of ls.
|
||||
// @param[in] new_status, the new create state which will be set.
|
||||
void set_create_state(const ObInnerLSStatus new_status);
|
||||
ObInnerLSStatus get_create_state();
|
||||
ObInnerLSStatus get_create_state() const;
|
||||
bool is_need_gc() const;
|
||||
bool is_need_load_inner_tablet() const;
|
||||
// for rebuild
|
||||
@ -307,41 +266,40 @@ public:
|
||||
|
||||
TO_STRING_KV(K_(ls_meta), K_(log_handler), K_(restore_handler), K_(is_inited), K_(tablet_gc_handler));
|
||||
private:
|
||||
int stop_();
|
||||
void wait_();
|
||||
int prepare_for_safe_destroy_();
|
||||
int flush_if_need_(const bool need_flush);
|
||||
int offline_();
|
||||
int offline_compaction_();
|
||||
int online_compaction_();
|
||||
int offline_tx_();
|
||||
int online_tx_();
|
||||
public:
|
||||
// ObLSMeta interface:
|
||||
int update_id_meta_with_writing_slog(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts);
|
||||
int update_id_meta_without_writing_slog(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts);
|
||||
int update_ls_meta(const bool update_restore_status,
|
||||
const ObLSMeta &src_ls_meta);
|
||||
|
||||
// int update_id_meta(const int64_t service_type,
|
||||
// const int64_t limited_id,
|
||||
// const int64_t latest_log_ts,
|
||||
// const bool write_slog);
|
||||
DELEGATE_WITH_RET(ls_meta_, update_id_meta, int);
|
||||
int set_ls_rebuild();
|
||||
// protect in ls lock
|
||||
// int set_gc_state(const logservice::LSGCState &gc_state);
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, set_gc_state);
|
||||
DELEGATE_WITH_RET(ls_meta_, set_gc_state, int);
|
||||
// int set_clog_checkpoint(const palf::LSN &clog_checkpoint_lsn,
|
||||
// const int64_t clog_checkpoint_ts,
|
||||
// const bool write_slog = true);
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, set_clog_checkpoint);
|
||||
UPDATE_LSMETA_WITHOUT_LOCK(ls_meta_, set_clog_checkpoint);
|
||||
DELEGATE_WITH_RET(ls_meta_, set_clog_checkpoint, int);
|
||||
CONST_DELEGATE_WITH_RET(ls_meta_, get_clog_checkpoint_ts, int64_t);
|
||||
DELEGATE_WITH_RET(ls_meta_, get_clog_base_lsn, palf::LSN &);
|
||||
DELEGATE_WITH_RET(ls_meta_, get_saved_info, int);
|
||||
// int build_saved_info();
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, build_saved_info);
|
||||
DELEGATE_WITH_RET(ls_meta_, build_saved_info, int);
|
||||
// int clear_saved_info_without_lock();
|
||||
UPDATE_LSMETA_WITHOUT_LOCK(ls_meta_, clear_saved_info);
|
||||
// int update_ls_meta(const bool update_restore_status,
|
||||
// const ObLSMeta &src_ls_meta);
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, update_ls_meta);
|
||||
// int update_ls_meta_without_lock(const bool update_restore_status,
|
||||
// const ObLSMeta &src_ls_meta);
|
||||
UPDATE_LSMETA_WITHOUT_LOCK(ls_meta_, update_ls_meta);
|
||||
DELEGATE_WITH_RET(ls_meta_, clear_saved_info, int);
|
||||
CONST_DELEGATE_WITH_RET(ls_meta_, get_rebuild_seq, int64_t);
|
||||
CONST_DELEGATE_WITH_RET(ls_meta_, get_tablet_change_checkpoint_ts, int64_t);
|
||||
|
||||
@ -367,7 +325,7 @@ public:
|
||||
// set offline ts
|
||||
// @param [in] offline ts.
|
||||
// int set_offline_ts_ns(const int64_t offline_ts_ns);
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, set_offline_ts_ns);
|
||||
DELEGATE_WITH_RET(ls_meta_, set_offline_ts_ns, int);
|
||||
// get offline ts
|
||||
// @param [in] offline ts.
|
||||
// int get_offline_ts_ns(int64_t &offline_ts_ns);
|
||||
@ -375,7 +333,8 @@ public:
|
||||
// update replayable point
|
||||
// @param [in] replayable point.
|
||||
// int update_ls_replayable_point(const int64_t replayable_point);
|
||||
UPDATE_LSMETA_WITH_LOCK(ls_meta_, update_ls_replayable_point);
|
||||
DELEGATE_WITH_RET(ls_meta_, update_ls_replayable_point, int);
|
||||
// update replayable point
|
||||
// get replayable point
|
||||
// @param [in] replayable point
|
||||
// int get_ls_replayable_point(int64_t &replayable_point);
|
||||
@ -721,6 +680,7 @@ private:
|
||||
bool is_inited_;
|
||||
uint64_t tenant_id_;
|
||||
bool is_stopped_;
|
||||
bool is_offlined_;
|
||||
ObLSMeta ls_meta_;
|
||||
observer::ObIMetaReport *rs_reporter_;
|
||||
ObLSLock lock_;
|
||||
|
||||
@ -22,6 +22,7 @@ using namespace common;
|
||||
using namespace palf;
|
||||
using namespace logservice;
|
||||
using namespace share;
|
||||
using namespace transaction;
|
||||
namespace storage
|
||||
{
|
||||
|
||||
@ -83,6 +84,17 @@ ObLSMeta::ObLSMeta(const ObLSMeta &ls_meta)
|
||||
all_id_meta_.update_all_id_meta(ls_meta.all_id_meta_);
|
||||
}
|
||||
|
||||
void ObLSMeta::set_ls_create_status(const ObInnerLSStatus &status)
|
||||
{
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ls_create_status_ = status;
|
||||
}
|
||||
|
||||
ObInnerLSStatus ObLSMeta::get_ls_create_status() const
|
||||
{
|
||||
return ls_create_status_;
|
||||
}
|
||||
|
||||
ObLSMeta &ObLSMeta::operator=(const ObLSMeta &other)
|
||||
{
|
||||
ObSpinLockTimeGuard guard(other.lock_);
|
||||
@ -140,20 +152,24 @@ int ObLSMeta::set_clog_checkpoint(const LSN &clog_checkpoint_lsn,
|
||||
const int64_t clog_checkpoint_ts,
|
||||
const bool write_slog)
|
||||
{
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.clog_base_lsn_ = clog_checkpoint_lsn;
|
||||
tmp.clog_checkpoint_ts_ = clog_checkpoint_ts;
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
if (write_slog) {
|
||||
if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("clog_checkpoint write slog failed", K(ret));
|
||||
}
|
||||
}
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.clog_base_lsn_ = clog_checkpoint_lsn;
|
||||
tmp.clog_checkpoint_ts_ = clog_checkpoint_ts;
|
||||
|
||||
clog_base_lsn_ = clog_checkpoint_lsn;
|
||||
clog_checkpoint_ts_ = clog_checkpoint_ts;
|
||||
if (write_slog) {
|
||||
if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("clog_checkpoint write slog failed", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
clog_base_lsn_ = clog_checkpoint_lsn;
|
||||
clog_checkpoint_ts_ = clog_checkpoint_ts;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
@ -168,9 +184,12 @@ int ObLSMeta::set_tablet_change_checkpoint_ts(const int64_t tablet_change_checkp
|
||||
{
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
int ret = OB_SUCCESS;
|
||||
if (tablet_change_checkpoint_ts_ > tablet_change_checkpoint_ts) {
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (tablet_change_checkpoint_ts_ > tablet_change_checkpoint_ts) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("tablet_change_checkpoint_ts is small", KR(ret), K(tablet_change_checkpoint_ts), K_(tablet_change_checkpoint_ts));
|
||||
LOG_WARN("tablet_change_checkpoint_ts is small", KR(ret), K(tablet_change_checkpoint_ts),
|
||||
K_(tablet_change_checkpoint_ts));
|
||||
} else {
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.tablet_change_checkpoint_ts_ = tablet_change_checkpoint_ts;
|
||||
@ -206,22 +225,30 @@ int ObLSMeta::set_migration_status(const ObMigrationStatus &migration_status,
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool can_change = false;
|
||||
if (!ObMigrationStatusHelper::is_valid(migration_status)) {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!ObMigrationStatusHelper::is_valid(migration_status)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(migration_status), KPC(this));
|
||||
} else if (migration_status_ == migration_status) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(ObMigrationStatusHelper::check_can_change_status(migration_status_,
|
||||
migration_status,
|
||||
can_change))) {
|
||||
LOG_WARN("failed to check can change stauts", K(ret), K(migration_status_),
|
||||
K(migration_status));
|
||||
} else if (!can_change) {
|
||||
ret = OB_OP_NOT_ALLOW;
|
||||
LOG_WARN("ls can not change to migrate status", K(ret), K(migration_status_),
|
||||
K(migration_status));
|
||||
} else {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (migration_status_ == migration_status) {
|
||||
//do nothing
|
||||
} else if (ObMigrationStatusHelper::check_can_change_status(migration_status_, migration_status, can_change)) {
|
||||
LOG_WARN("failed to check can change stauts", K(ret), K(migration_status_), K(migration_status));
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.migration_status_ = migration_status;
|
||||
if (write_slog && OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("migration_status write slog failed", K(ret));
|
||||
} else {
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.migration_status_ = migration_status;
|
||||
if (write_slog && OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("migration_status write slog failed", K(ret));
|
||||
} else {
|
||||
migration_status_ = migration_status;
|
||||
}
|
||||
migration_status_ = migration_status;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -243,11 +270,13 @@ int ObLSMeta::get_migration_status(ObMigrationStatus &migration_status) const
|
||||
int ObLSMeta::set_gc_state(const logservice::LSGCState &gc_state)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!ObGCHandler::is_valid_ls_gc_state(gc_state)) {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!ObGCHandler::is_valid_ls_gc_state(gc_state)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("gc_state invalid", K(ret), K(gc_state));
|
||||
} else {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.gc_state_ = gc_state;
|
||||
if (OB_FAIL(write_slog_(tmp))) {
|
||||
@ -277,7 +306,11 @@ int ObLSMeta::set_offline_ts_ns(const int64_t offline_ts_ns)
|
||||
// 不主动写slog
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
offline_ts_ns_ = offline_ts_ns;
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
offline_ts_ns_ = offline_ts_ns;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -297,21 +330,21 @@ int ObLSMeta::get_offline_ts_ns(int64_t &offline_ts_ns)
|
||||
int ObLSMeta::set_restore_status(const ObLSRestoreStatus &restore_status)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!restore_status.is_valid()) {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!restore_status.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid restore status", K(ret), K(restore_status_), K(restore_status));
|
||||
} else if (restore_status_ == restore_status) {
|
||||
//do nothing
|
||||
} else {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (restore_status_ == restore_status) {
|
||||
//do nothing
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.restore_status_ = restore_status;
|
||||
if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("restore_status write slog failed", K(ret));
|
||||
} else {
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.restore_status_ = restore_status;
|
||||
if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("restore_status write slog failed", K(ret));
|
||||
} else {
|
||||
restore_status_ = restore_status;
|
||||
}
|
||||
restore_status_ = restore_status;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -334,7 +367,9 @@ int ObLSMeta::update_ls_replayable_point(const int64_t replayable_point)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (replayable_point < replayable_point_) {
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (replayable_point < replayable_point_) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("replayable_point invalid", K(ret), K(replayable_point), K(replayable_point_));
|
||||
} else if (replayable_point_ == replayable_point) {
|
||||
@ -372,9 +407,9 @@ int ObLSMeta::update_ls_meta(
|
||||
int ret = OB_SUCCESS;
|
||||
ObLSRestoreStatus ls_restore_status;
|
||||
|
||||
if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!src_ls_meta.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("update ls meta get invalid argument", K(ret), K(src_ls_meta));
|
||||
@ -382,44 +417,35 @@ int ObLSMeta::update_ls_meta(
|
||||
&& OB_FAIL(src_ls_meta.get_restore_status(ls_restore_status))) {
|
||||
LOG_WARN("failed to get restore status", K(ret), K(src_ls_meta));
|
||||
} else {
|
||||
{
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
|
||||
tmp.clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
|
||||
tmp.replayable_point_ = src_ls_meta.replayable_point_;
|
||||
tmp.tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_;
|
||||
tmp.rebuild_seq_++;
|
||||
if (update_restore_status) {
|
||||
tmp.restore_status_ = ls_restore_status;
|
||||
}
|
||||
guard.click();
|
||||
tmp.all_id_meta_.update_all_id_meta(src_ls_meta.all_id_meta_);
|
||||
if (tmp.clog_checkpoint_ts_ < clog_checkpoint_ts_) {
|
||||
ObLSMeta tmp(*this);
|
||||
tmp.clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
|
||||
tmp.clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
|
||||
tmp.replayable_point_ = src_ls_meta.replayable_point_;
|
||||
tmp.tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_;
|
||||
tmp.rebuild_seq_++;
|
||||
if (update_restore_status) {
|
||||
tmp.restore_status_ = ls_restore_status;
|
||||
}
|
||||
guard.click();
|
||||
tmp.all_id_meta_.update_all_id_meta(src_ls_meta.all_id_meta_);
|
||||
if (tmp.clog_checkpoint_ts_ < clog_checkpoint_ts_) {
|
||||
// TODO: now do not allow clog checkpoint ts rollback, may support it in 4.1
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("do not allow clog checkpoint ts rollback", K(ret), K(src_ls_meta), KPC(this));
|
||||
} else if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("clog_checkpoint write slog failed", K(ret));
|
||||
} else {
|
||||
guard.click();
|
||||
clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
|
||||
clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
|
||||
replayable_point_ = src_ls_meta.replayable_point_;
|
||||
tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_;
|
||||
|
||||
if (update_restore_status) {
|
||||
restore_status_ = ls_restore_status;
|
||||
}
|
||||
all_id_meta_.update_all_id_meta(src_ls_meta.all_id_meta_);
|
||||
}
|
||||
LOG_INFO("update ls meta", K(ret), K(tmp), K(src_ls_meta), K(*this));
|
||||
}
|
||||
if (OB_SUCC(ret) && IDS_LS == ls_id_) {
|
||||
if (OB_FAIL(all_id_meta_.update_id_service())) {
|
||||
LOG_WARN("update id service with ls meta fail", K(ret), K(*this));
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("do not allow clog checkpoint ts rollback", K(ret), K(src_ls_meta), KPC(this));
|
||||
} else if (OB_FAIL(write_slog_(tmp))) {
|
||||
LOG_WARN("clog_checkpoint write slog failed", K(ret));
|
||||
} else {
|
||||
guard.click();
|
||||
clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
|
||||
clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
|
||||
replayable_point_ = src_ls_meta.replayable_point_;
|
||||
tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_;
|
||||
all_id_meta_.update_all_id_meta(src_ls_meta.all_id_meta_);
|
||||
if (update_restore_status) {
|
||||
restore_status_ = ls_restore_status;
|
||||
}
|
||||
}
|
||||
LOG_INFO("update ls meta", K(ret), K(tmp), K(src_ls_meta), K(*this));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -430,13 +456,11 @@ int ObLSMeta::set_ls_rebuild()
|
||||
const ObMigrationStatus change_status = ObMigrationStatus::OB_MIGRATION_STATUS_REBUILD;
|
||||
bool can_change = false;
|
||||
|
||||
if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSMeta tmp(*this);
|
||||
|
||||
if (OB_FAIL(ObMigrationStatusHelper::check_can_change_status(tmp.migration_status_, change_status, can_change))) {
|
||||
LOG_WARN("failed to check can change status", K(ret), K(migration_status_), K(change_status));
|
||||
} else if (!can_change) {
|
||||
@ -490,9 +514,8 @@ int ObLSMeta::set_saved_info(const ObLSSavedInfo &saved_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!saved_info.is_valid()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("set saved info get invalid argument", K(ret), K(saved_info));
|
||||
@ -511,12 +534,11 @@ int ObLSMeta::set_saved_info(const ObLSSavedInfo &saved_info)
|
||||
int ObLSMeta::build_saved_info()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSSavedInfo saved_info;
|
||||
|
||||
if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else if (!saved_info_.is_empty()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("saved info is not empty, can not build saved info", K(ret), K(*this));
|
||||
@ -538,12 +560,11 @@ int ObLSMeta::build_saved_info()
|
||||
int ObLSMeta::clear_saved_info()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
ObLSSavedInfo saved_info;
|
||||
|
||||
if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
saved_info.reset();
|
||||
ObLSMeta tmp(*this);
|
||||
@ -601,19 +622,47 @@ int ObLSMeta::update_id_meta(const int64_t service_type,
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
all_id_meta_.update_id_meta(service_type, limited_id, latest_log_ts);
|
||||
guard.click();
|
||||
if (write_slog) {
|
||||
if (OB_FAIL(write_slog_(*this))) {
|
||||
LOG_WARN("id service flush write slog failed", K(ret));
|
||||
if (OB_FAIL(check_can_update_())) {
|
||||
LOG_WARN("ls meta cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
// TODO: write slog may failed, but the content is updated.
|
||||
all_id_meta_.update_id_meta(service_type, limited_id, latest_log_ts);
|
||||
guard.click();
|
||||
if (write_slog) {
|
||||
if (OB_FAIL(write_slog_(*this))) {
|
||||
LOG_WARN("id service flush write slog failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_INFO("update id meta", K(ret), K(service_type), K(limited_id), K(latest_log_ts),
|
||||
K(*this));
|
||||
K(*this));
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLSMeta::get_all_id_meta(ObAllIDMeta &all_id_meta) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
ObSpinLockTimeGuard guard(lock_);
|
||||
all_id_meta.update_all_id_meta(all_id_meta_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLSMeta::check_can_update_()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!can_update_ls_meta(ls_create_status_)) {
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
LOG_WARN("state not match, cannot update ls meta", K(ret), KPC(this));
|
||||
} else if (!is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLSMeta::get_migration_and_restore_status(
|
||||
ObMigrationStatus &migration_status,
|
||||
share::ObLSRestoreStatus &ls_restore_status)
|
||||
|
||||
@ -40,14 +40,22 @@ public:
|
||||
ObLSMeta();
|
||||
ObLSMeta(const ObLSMeta &ls_meta);
|
||||
~ObLSMeta() {}
|
||||
int init(const uint64_t tenant_id,
|
||||
const share::ObLSID &ls_id,
|
||||
const ObReplicaType &replica_type,
|
||||
const ObMigrationStatus &migration_status,
|
||||
const share::ObLSRestoreStatus &restore_status,
|
||||
const int64_t create_scn);
|
||||
void reset();
|
||||
bool is_valid() const;
|
||||
void set_ls_create_status(const ObInnerLSStatus &status);
|
||||
ObInnerLSStatus get_ls_create_status() const;
|
||||
ObLSMeta &operator=(const ObLSMeta &other);
|
||||
int64_t get_clog_checkpoint_ts() const;
|
||||
palf::LSN &get_clog_base_lsn();
|
||||
int set_clog_checkpoint(const palf::LSN &clog_checkpoint_lsn,
|
||||
const int64_t clog_checkpoint_ts,
|
||||
const bool write_slog = true);
|
||||
void reset();
|
||||
bool is_valid() const;
|
||||
int64_t get_rebuild_seq() const;
|
||||
int set_migration_status(const ObMigrationStatus &migration_status,
|
||||
const bool write_slog = true);
|
||||
@ -73,10 +81,7 @@ public:
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts,
|
||||
const bool write_slog);
|
||||
int update_id_service()
|
||||
{
|
||||
return all_id_meta_.update_id_service();
|
||||
}
|
||||
int get_all_id_meta(transaction::ObAllIDMeta &all_id_meta) const;
|
||||
int get_saved_info(ObLSSavedInfo &saved_info);
|
||||
int build_saved_info();
|
||||
int set_saved_info(const ObLSSavedInfo &saved_info);
|
||||
@ -84,14 +89,6 @@ public:
|
||||
int get_migration_and_restore_status(
|
||||
ObMigrationStatus &migration_status,
|
||||
share::ObLSRestoreStatus &ls_restore_status);
|
||||
|
||||
int init(
|
||||
const uint64_t tenant_id,
|
||||
const share::ObLSID &ls_id,
|
||||
const ObReplicaType &replica_type,
|
||||
const ObMigrationStatus &migration_status,
|
||||
const share::ObLSRestoreStatus &restore_status,
|
||||
const int64_t create_scn);
|
||||
class ObSpinLockTimeGuard
|
||||
{
|
||||
public:
|
||||
@ -108,14 +105,15 @@ public:
|
||||
K_(rebuild_seq), K_(migration_status), K(gc_state_), K(offline_ts_ns_),
|
||||
K_(restore_status), K_(replayable_point), K_(tablet_change_checkpoint_ts),
|
||||
K_(all_id_meta));
|
||||
private:
|
||||
int check_can_update_();
|
||||
public:
|
||||
mutable common::ObSpinLock lock_;
|
||||
uint64_t tenant_id_;
|
||||
share::ObLSID ls_id_;
|
||||
ObReplicaType replica_type_;
|
||||
ObInnerLSStatus ls_create_status_;
|
||||
|
||||
private:
|
||||
ObInnerLSStatus ls_create_status_;
|
||||
typedef common::ObFunction<int(ObLSMeta &)> WriteSlog;
|
||||
// for test
|
||||
void set_write_slog_func_(WriteSlog write_slog);
|
||||
|
||||
@ -239,11 +239,17 @@ int ObIDService::update_ls_id_meta(const bool write_slog)
|
||||
if (OB_FAIL(check_and_fill_ls())) {
|
||||
TRANS_LOG(WARN, "ls set fail", K(ret));
|
||||
} else if (write_slog) {
|
||||
ret = ls_->update_id_meta_with_writing_slog(service_type_, ATOMIC_LOAD(&limited_id_), ATOMIC_LOAD(&latest_log_ts_));
|
||||
ret = ls_->update_id_meta(service_type_,
|
||||
ATOMIC_LOAD(&limited_id_),
|
||||
ATOMIC_LOAD(&latest_log_ts_),
|
||||
true /* write slog */);
|
||||
} else {
|
||||
ret = ls_->update_id_meta_without_writing_slog(service_type_, ATOMIC_LOAD(&limited_id_), ATOMIC_LOAD(&latest_log_ts_));
|
||||
ret = ls_->update_id_meta(service_type_,
|
||||
ATOMIC_LOAD(&limited_id_),
|
||||
ATOMIC_LOAD(&latest_log_ts_),
|
||||
false /* not write slog */);
|
||||
}
|
||||
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
TRANS_LOG(WARN, "update id meta fail", K(ret), K_(service_type));
|
||||
}
|
||||
@ -420,7 +426,30 @@ int ObIDService::get_id_service(const int64_t id_service_type, ObIDService *&id_
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(ERROR, "get wrong id_service_type", K(ret), K(MTL_ID()), K(id_service_type));
|
||||
TRANS_LOG(ERROR, "get wrong id_service_type", K(ret), K(MTL_ID()), K(id_service_type));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObIDService::update_id_service(const ObAllIDMeta &id_meta)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
for(int i = 0; i < ObIDService::MAX_SERVICE_TYPE && OB_SUCC(ret); i++) {
|
||||
ObIDService *id_service = NULL;
|
||||
int64_t limited_id = 0;
|
||||
int64_t latest_log_ts = 0;
|
||||
if (OB_FAIL(id_meta.get_id_meta(i, limited_id, latest_log_ts))) {
|
||||
TRANS_LOG(WARN, "get id meta fail", K(ret), K(id_meta));
|
||||
} else if (OB_FAIL(get_id_service(i, id_service))) {
|
||||
TRANS_LOG(WARN, "get id service fail", K(ret), K(i));
|
||||
} else if (OB_ISNULL(id_service)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
TRANS_LOG(WARN, "id service is null", K(ret));
|
||||
} else {
|
||||
(void) id_service->update_limited_id(limited_id, latest_log_ts);
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
@ -636,39 +665,47 @@ OB_DEF_DESERIALIZE(ObAllIDMeta)
|
||||
|
||||
void ObAllIDMeta::update_all_id_meta(const ObAllIDMeta &all_id_meta)
|
||||
{
|
||||
ObSpinLockGuard lock_guard(lock_);
|
||||
for(int i=0; i<ObIDService::MAX_SERVICE_TYPE; i++) {
|
||||
(void)inc_update(&id_meta_[i].limited_id_, all_id_meta.id_meta_[i].limited_id_);
|
||||
(void)inc_update(&id_meta_[i].latest_log_ts_, all_id_meta.id_meta_[i].latest_log_ts_);
|
||||
}
|
||||
}
|
||||
|
||||
int ObAllIDMeta::update_id_service()
|
||||
int ObAllIDMeta::update_id_meta(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
for(int i=0; i<ObIDService::MAX_SERVICE_TYPE; i++) {
|
||||
ObIDService *id_service = NULL;
|
||||
if (OB_FAIL(ObIDService::get_id_service(i, id_service))) {
|
||||
SERVER_LOG(WARN, "get id service fail", K(ret), K(*this));
|
||||
} else if (OB_ISNULL(id_service)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "id service is null", K(ret), K(*this));
|
||||
} else {
|
||||
id_service->update_limited_id(id_meta_[i].limited_id_,
|
||||
id_meta_[i].latest_log_ts_);
|
||||
}
|
||||
ObSpinLockGuard lock_guard(lock_);
|
||||
if (service_type <= ObIDService::INVALID_ID_SERVICE_TYPE ||
|
||||
service_type >= ObIDService::MAX_SERVICE_TYPE) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
TRANS_LOG(WARN, "invalid argument", K(ret), K(service_type));
|
||||
} else {
|
||||
(void)inc_update(&id_meta_[service_type].limited_id_, limited_id);
|
||||
(void)inc_update(&id_meta_[service_type].latest_log_ts_, latest_log_ts);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObAllIDMeta::update_id_meta(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts)
|
||||
int ObAllIDMeta::get_id_meta(const int64_t service_type,
|
||||
int64_t &limited_id,
|
||||
int64_t &latest_log_ts) const
|
||||
{
|
||||
(void)inc_update(&id_meta_[service_type].limited_id_, limited_id);
|
||||
(void)inc_update(&id_meta_[service_type].latest_log_ts_, latest_log_ts);
|
||||
int ret = OB_SUCCESS;
|
||||
ObSpinLockGuard lock_guard(lock_);
|
||||
if (service_type <= ObIDService::INVALID_ID_SERVICE_TYPE ||
|
||||
service_type >= ObIDService::MAX_SERVICE_TYPE) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
TRANS_LOG(WARN, "invalid argument", K(ret), K(service_type));
|
||||
} else {
|
||||
limited_id = id_meta_[service_type].limited_id_;
|
||||
latest_log_ts = id_meta_[service_type].latest_log_ts_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@ -29,6 +29,7 @@ class ObLS;
|
||||
namespace transaction
|
||||
{
|
||||
|
||||
class ObAllIDMeta;
|
||||
|
||||
class ObPresistIDLog
|
||||
{
|
||||
@ -97,6 +98,7 @@ public:
|
||||
|
||||
static void get_all_id_service_type(int64_t *service_type);
|
||||
static int get_id_service(const int64_t id_service_type, ObIDService *&id_service);
|
||||
static int update_id_service(const ObAllIDMeta &id_meta);
|
||||
|
||||
//获取数值或数值组
|
||||
int get_number(const int64_t range, const int64_t base_id, int64_t &start_id, int64_t &end_id);
|
||||
@ -180,10 +182,12 @@ public:
|
||||
ObAllIDMeta() : count_(ObIDService::MAX_SERVICE_TYPE) {}
|
||||
~ObAllIDMeta() {}
|
||||
void update_all_id_meta(const ObAllIDMeta &all_id_meta);
|
||||
int update_id_service();
|
||||
void update_id_meta(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts);
|
||||
int update_id_meta(const int64_t service_type,
|
||||
const int64_t limited_id,
|
||||
const int64_t latest_log_ts);
|
||||
int get_id_meta(const int64_t service_type,
|
||||
int64_t &limited_id,
|
||||
int64_t &latest_log_ts) const;
|
||||
int64_t to_string(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
@ -196,6 +200,7 @@ public:
|
||||
}
|
||||
OB_UNIS_VERSION(1);
|
||||
private:
|
||||
mutable common::ObSpinLock lock_;
|
||||
int64_t count_;
|
||||
ObIDMeta id_meta_[ObIDService::MAX_SERVICE_TYPE];
|
||||
};
|
||||
|
||||
@ -137,14 +137,15 @@ int ObLSService::stop()
|
||||
LOG_ERROR("ls is null", K(ret));
|
||||
} else if (OB_FAIL(ls->offline())) {
|
||||
LOG_WARN("ls offline failed", K(ret), K(ls->get_ls_id()), KP(ls));
|
||||
} else if (OB_FAIL(ls->stop())) {
|
||||
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id));
|
||||
} else if (FALSE_IT(ls->wait())) {
|
||||
} else if (OB_FAIL(handle.set_ls(ls_map_, *ls, ObLSGetMod::TXSTORAGE_MOD))) {
|
||||
LOG_WARN("get ls handle failed", K(ret), KPC(ls));
|
||||
} else {
|
||||
ObLSLockGuard lock_ls(ls);
|
||||
if (OB_FAIL(ls->stop())) {
|
||||
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id));
|
||||
} else if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
|
||||
"LSSafeDestroy"))) {
|
||||
if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
|
||||
"LSSafeDestroy"))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("alloc memory failed", K(ret));
|
||||
} else if (FALSE_IT(task = new(task) ObLSSafeDestroyTask())) {
|
||||
@ -635,9 +636,10 @@ int ObLSService::gc_ls_after_replay_slog()
|
||||
ls_status = ls->get_create_state();
|
||||
if (ObInnerLSStatus::CREATING == ls_status || ObInnerLSStatus::REMOVED == ls_status) {
|
||||
do {
|
||||
ObLSLockGuard lock_ls(ls);
|
||||
if (OB_TMP_FAIL(ls->stop())) {
|
||||
LOG_WARN("ls stop failed", K(tmp_ret), K(ls->get_ls_id()), KP(ls));
|
||||
} else {
|
||||
ls->wait();
|
||||
}
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
usleep(SLEEP_TS);
|
||||
@ -872,14 +874,15 @@ int ObLSService::remove_ls(
|
||||
// ls leader gc must has block tx start, gracefully kill tx and write offline log before here.
|
||||
} else if (OB_FAIL(ls->offline())) {
|
||||
LOG_WARN("ls offline failed", K(ret), K(ls_id), KP(ls));
|
||||
} else if (OB_FAIL(ls->stop())) {
|
||||
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id));
|
||||
} else if (FALSE_IT(ls->wait())) {
|
||||
} else {
|
||||
ObLSSafeDestroyTask *task = nullptr;
|
||||
static const int64_t SLEEP_TS = 100_ms;
|
||||
ObLSLockGuard lock_ls(ls);
|
||||
if (OB_FAIL(ls->stop())) {
|
||||
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id));
|
||||
} else if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
|
||||
"LSSafeDestroy"))) {
|
||||
if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
|
||||
"LSSafeDestroy"))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("alloc memory failed", K(ret));
|
||||
} else if (FALSE_IT(task = new(task) ObLSSafeDestroyTask())) {
|
||||
|
||||
Reference in New Issue
Block a user