BUGFIX: fix deadlock at ls gc and migrate

This commit is contained in:
obdev
2022-11-16 04:05:39 +00:00
committed by wangzelin.wzl
parent 9847c3139a
commit 318f835b88
20 changed files with 480 additions and 340 deletions

View File

@ -140,11 +140,12 @@ int ObLogHandler::stop()
//判断is_apply_done依赖log handler不能再继续append //判断is_apply_done依赖log handler不能再继续append
//所以需要is_in_stop_state_置true表示stop阶段已经不能再提交日志 //所以需要is_in_stop_state_置true表示stop阶段已经不能再提交日志
int ObLogHandler::safe_to_destroy() int ObLogHandler::safe_to_destroy(bool &is_safe_destroy)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
bool is_done = false; bool is_done = false;
LSN end_lsn; LSN end_lsn;
is_safe_destroy = true;
WLockGuard guard(lock_); WLockGuard guard(lock_);
if (IS_INIT) { if (IS_INIT) {
if (palf_handle_.is_valid() || !is_in_stop_state_) { 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_)); 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; return ret;
} }

View File

@ -162,7 +162,7 @@ public:
// @breif, wait cb append onto apply service all be called // @breif, wait cb append onto apply service all be called
// is reentrant and should be called before destroy(), // is reentrant and should be called before destroy(),
// protect cb will not be used when log handler destroyed // 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 // @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 void *, the data buffer.
// @param[in] const uint64_t, the length of data buffer. // @param[in] const uint64_t, the length of data buffer.

View File

@ -173,7 +173,7 @@ int ObCheckpointExecutor::update_clog_checkpoint()
STORAGE_LOG(ERROR, "locate lsn by logts failed", K(ret), K(ls_id), STORAGE_LOG(ERROR, "locate lsn by logts failed", K(ret), K(ls_id),
K(checkpoint_ts), K(checkpoint_ts_in_ls_meta)); 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)); 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))) { } else if (OB_FAIL(loghandler_->advance_base_lsn(clog_checkpoint_lsn))) {
if (OB_NOT_INIT == ret) { if (OB_NOT_INIT == ret) {

View File

@ -167,10 +167,11 @@ int ObDataCheckpoint::init(ObLS *ls)
return OB_SUCCESS; return OB_SUCCESS;
} }
int ObDataCheckpoint::safe_to_destroy() int ObDataCheckpoint::safe_to_destroy(bool &is_safe_destroy)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
is_safe_destroy = true;
// avoid start ls_freeze again after waiting ls_freeze finish // avoid start ls_freeze again after waiting ls_freeze finish
is_inited_ = false; is_inited_ = false;
// wait until ls_freeze finish // wait until ls_freeze finish
@ -191,6 +192,10 @@ int ObDataCheckpoint::safe_to_destroy()
prepare_list_.reset(); prepare_list_.reset();
ls_ = nullptr; ls_ = nullptr;
if (OB_FAIL(ret)) {
is_safe_destroy = false;
}
return ret; return ret;
} }

View File

@ -87,7 +87,7 @@ public:
// used for virtual table // used for virtual table
static const uint64_t LS_DATA_CHECKPOINT_TABLET_ID = 40000; static const uint64_t LS_DATA_CHECKPOINT_TABLET_ID = 40000;
int init(ObLS *ls); int init(ObLS *ls);
int safe_to_destroy(); int safe_to_destroy(bool &is_safe_destroy);
int64_t get_rec_log_ts() override; int64_t get_rec_log_ts() override;
// if min_rec_log_ts <= the input rec_log_ts // if min_rec_log_ts <= the input rec_log_ts
// logstream freeze // logstream freeze

View File

@ -358,22 +358,18 @@ int ObTenantTabletScheduler::update_upper_trans_version_and_gc_sstable()
return ret; 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; int ret = OB_SUCCESS;
bool exist = false; bool exist = false;
if (OB_UNLIKELY(!ls_id.is_valid())) { if (OB_UNLIKELY(!ls_id.is_valid())) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ls_id)); LOG_WARN("invalid argument", K(ret), K(ls_id));
} } else if (OB_FAIL(MTL(ObTenantDagScheduler*)->check_ls_compaction_dag_exist(ls_id, exist))) {
while (OB_SUCC(ret)) { LOG_WARN("failed to check ls compaction dag", K(ret), K(ls_id));
if (OB_FAIL(MTL(ObTenantDagScheduler*)->check_ls_compaction_dag_exist(ls_id, exist))) { } else if (exist) {
LOG_WARN("failed to check ls compaction dag", K(ret), K(ls_id)); // the compaction dag exists, need retry later.
} else if (!exist) { ret = OB_EAGAIN;
break;
} else {
ob_usleep(100 * 1000); // 100ms
}
} }
return ret; return ret;
} }

View File

@ -112,7 +112,7 @@ public:
int merge_all(); int merge_all();
int schedule_merge(const int64_t broadcast_version); int schedule_merge(const int64_t broadcast_version);
int update_upper_trans_version_and_gc_sstable(); 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. // 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. // The bloomfilter build task will be ignored if a same build task exists in the queue.

View File

@ -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_)); LOG_WARN("tenant dag scheduler has set stop, stop migration dag net", K(ret), K(ctx_));
break; break;
} else { } else {
// TODO: muwei should not do this before ls create finished.
if (OB_FAIL(ls->get_migration_status(current_migration_status))) { if (OB_FAIL(ls->get_migration_status(current_migration_status))) {
LOG_WARN("failed to get migration status", K(ret), K(ctx_)); LOG_WARN("failed to get migration status", K(ret), K(ctx_));
} else if (ctx_.is_failed()) { } else if (ctx_.is_failed()) {
@ -353,7 +354,7 @@ int ObLSCompleteMigrationDagNet::update_migration_status_(ObLS *ls)
} }
} else { } else {
if (ObMigrationOpType::REBUILD_LS_OP == ctx_.arg_.type_ 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)); LOG_WARN("failed to clear ls saved info", K(ret), KPC(ls));
} else { } else {
new_migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_NONE; new_migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_NONE;

View File

@ -1190,11 +1190,10 @@ int ObStartMigrationTask::update_ls_()
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls should not be NULL", K(ret), KPC(ctx_)); LOG_WARN("ls should not be NULL", K(ret), KPC(ctx_));
} else { } else {
ObLSMetaPackage local_ls_meta_package;
ObLSLockGuard lock_ls(ls); ObLSLockGuard lock_ls(ls);
const bool is_rebuild = (ctx_->arg_.type_ == ObMigrationOpType::REBUILD_LS_OP); const bool is_rebuild = (ctx_->arg_.type_ == ObMigrationOpType::REBUILD_LS_OP);
if (OB_FAIL(ls->update_ls_meta_without_lock(update_restore_status, if (OB_FAIL(ls->update_ls_meta(update_restore_status,
ctx_->src_ls_meta_package_.ls_meta_))) { ctx_->src_ls_meta_package_.ls_meta_))) {
LOG_WARN("failed to update ls meta", K(ret), KPC(ctx_)); LOG_WARN("failed to update ls meta", K(ret), KPC(ctx_));
} else if (OB_FAIL(ls->get_end_lsn(end_lsn))) { } else if (OB_FAIL(ls->get_end_lsn(end_lsn))) {
LOG_WARN("failed to get end lsn", K(ret), KPC(ctx_)); LOG_WARN("failed to get end lsn", K(ret), KPC(ctx_));

View File

@ -932,11 +932,8 @@ int ObLSMigrationHandler::report_meta_table_()
LOG_WARN("failed ot get result", K(ret)); LOG_WARN("failed ot get result", K(ret));
} else if (OB_SUCCESS != result) { } else if (OB_SUCCESS != result) {
//do nothing //do nothing
} else { } else if (OB_FAIL(ls_->report_replica_info())) {
ObLSLockGuard lock_ls(ls_); LOG_WARN("failed to report replica info", K(ret), KPC(ls_));
if (OB_FAIL(ls_->report_replica_info())) {
LOG_WARN("failed to report replica info", K(ret), KPC(ls_));
}
} }
return ret; return ret;
} }
@ -1179,27 +1176,26 @@ void ObLSMigrationHandler::stop()
} }
} }
int ObLSMigrationHandler::safe_to_destroy( void ObLSMigrationHandler::wait(bool &wait_finished)
bool &is_safe_to_destroy)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
is_safe_to_destroy = false; wait_finished = false;
ObLSMigrationTask task; ObLSMigrationTask task;
if (OB_FAIL(get_ls_migration_task_(task))) { if (OB_FAIL(get_ls_migration_task_(task))) {
if (OB_ENTRY_NOT_EXIST == ret) { if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS; ret = OB_SUCCESS;
is_safe_to_destroy = true; wait_finished = true;
} else { } else {
LOG_WARN("failed to get ls migration task", K(ret), KPC(ls_)); LOG_WARN("failed to get ls migration task", K(ret), KPC(ls_));
} }
} else { } else {
is_safe_to_destroy = false; wait_finished = false;
wakeup_(); wakeup_();
} }
return ret;
} }
} }
} }

View File

@ -83,7 +83,7 @@ public:
int check_task_exist(const share::ObTaskId &task_id, bool &is_exist); int check_task_exist(const share::ObTaskId &task_id, bool &is_exist);
void destroy(); void destroy();
void stop(); void stop();
int safe_to_destroy(bool &is_safe_to_destroy); void wait(bool &wait_finished);
private: private:
void reuse_(); void reuse_();

View File

@ -205,6 +205,7 @@ ObFreezer::ObFreezer()
high_priority_freeze_cnt_(0), high_priority_freeze_cnt_(0),
low_priority_freeze_cnt_(0), low_priority_freeze_cnt_(0),
need_resubmit_log_(false), need_resubmit_log_(false),
enable_(true),
is_inited_(false) is_inited_(false)
{} {}
@ -228,6 +229,7 @@ ObFreezer::ObFreezer(ObLSWRSHandler *ls_loop_worker,
high_priority_freeze_cnt_(0), high_priority_freeze_cnt_(0),
low_priority_freeze_cnt_(0), low_priority_freeze_cnt_(0),
need_resubmit_log_(false), need_resubmit_log_(false),
enable_(true),
is_inited_(false) is_inited_(false)
{} {}
@ -274,6 +276,7 @@ void ObFreezer::reset()
high_priority_freeze_cnt_ = 0; high_priority_freeze_cnt_ = 0;
low_priority_freeze_cnt_ = 0; low_priority_freeze_cnt_ = 0;
need_resubmit_log_ = false; need_resubmit_log_ = false;
enable_ = true;
is_inited_ = false; is_inited_ = false;
} }
@ -316,6 +319,8 @@ int ObFreezer::logstream_freeze()
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("[Freezer] not inited", K(ret), K_(ls_id)); 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))) { } 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)); 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))) { } 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) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
TRANS_LOG(WARN, "[Freezer] not inited", K(ret), K_(ls_id), K(tablet_id)); 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())) { } else if (OB_FAIL(guard.try_set_tablet_freeze_begin())) {
// no need freeze now, a ls freeze is running or will be running // no need freeze now, a ls freeze is running or will be running
ret = OB_SUCCESS; ret = OB_SUCCESS;
@ -478,6 +485,8 @@ int ObFreezer::force_tablet_freeze(const ObTabletID &tablet_id)
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
TRANS_LOG(WARN, "[Freezer] not inited", K(ret), K_(ls_id), K(tablet_id)); 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())) { } 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)); 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)) { } else if (FALSE_IT(stat_.state_ = ObFreezeState::NOT_SUBMIT_LOG)) {

View File

@ -205,6 +205,8 @@ public:
const share::ObLSID &ls_id, const share::ObLSID &ls_id,
uint32_t freeze_flag = 0); uint32_t freeze_flag = 0);
void reset(); void reset();
void offline() { enable_ = false; }
void online() { enable_ = true; }
public: public:
/* freeze */ /* freeze */
@ -320,6 +322,7 @@ private:
int64_t low_priority_freeze_cnt_; // freeze tablet cnt int64_t low_priority_freeze_cnt_; // freeze tablet cnt
bool need_resubmit_log_; bool need_resubmit_log_;
bool enable_; // whether we can do freeze now
bool is_inited_; bool is_inited_;
}; };

View File

@ -12,6 +12,7 @@
#define USING_LOG_PREFIX STORAGE #define USING_LOG_PREFIX STORAGE
#include "lib/utility/utility.h"
#include "share/ob_tenant_info_proxy.h" #include "share/ob_tenant_info_proxy.h"
#include "storage/ls/ob_ls.h" #include "storage/ls/ob_ls.h"
#include "share/leak_checker/obj_leak_checker.h" #include "share/leak_checker/obj_leak_checker.h"
@ -22,6 +23,7 @@
#include "logservice/ob_log_service.h" #include "logservice/ob_log_service.h"
#include "storage/tx/ob_trans_service.h" #include "storage/tx/ob_trans_service.h"
#include "observer/report/ob_i_meta_report.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/ob_tx_log_adapter.h"
#include "storage/tx_table/ob_tx_table.h" #include "storage/tx_table/ob_tx_table.h"
#include "storage/slog/ob_storage_log.h" #include "storage/slog/ob_storage_log.h"
@ -42,6 +44,7 @@ namespace oceanbase
{ {
using namespace share; using namespace share;
using namespace logservice; using namespace logservice;
using namespace transaction;
namespace storage namespace storage
{ {
@ -62,6 +65,7 @@ ObLS::ObLS()
is_inited_(false), is_inited_(false),
tenant_id_(OB_INVALID_TENANT_ID), tenant_id_(OB_INVALID_TENANT_ID),
is_stopped_(false), is_stopped_(false),
is_offlined_(false),
ls_meta_(), ls_meta_(),
rs_reporter_(nullptr) rs_reporter_(nullptr)
{} {}
@ -339,12 +343,12 @@ int ObLS::remove_ls()
void ObLS::set_create_state(const ObInnerLSStatus new_status) 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 bool ObLS::is_need_gc() const
@ -352,14 +356,15 @@ bool ObLS::is_need_gc() const
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
bool bool_ret = false; bool bool_ret = false;
ObMigrationStatus migration_status; ObMigrationStatus migration_status;
ObInnerLSStatus create_status = ls_meta_.get_ls_create_status();
if (OB_FAIL(ls_meta_.get_migration_status(migration_status))) { if (OB_FAIL(ls_meta_.get_migration_status(migration_status))) {
LOG_WARN("get migration status failed", K(ret), K(ls_meta_.ls_id_)); LOG_WARN("get migration status failed", K(ret), K(ls_meta_.ls_id_));
} }
bool_ret = (OB_FAIL(ret) || bool_ret = (OB_FAIL(ret) ||
(ObInnerLSStatus::CREATING == ls_meta_.ls_create_status_) || (ObInnerLSStatus::CREATING == create_status) ||
(ObInnerLSStatus::REMOVED == ls_meta_.ls_create_status_) || (ObInnerLSStatus::REMOVED == create_status) ||
(OB_MIGRATION_STATUS_NONE != migration_status && (OB_MIGRATION_STATUS_NONE != migration_status &&
ObMigrationStatusHelper::check_allow_gc(migration_status))); ObMigrationStatusHelper::check_allow_gc(migration_status)));
if (bool_ret) { if (bool_ret) {
@ -383,11 +388,13 @@ bool ObLS::is_need_load_inner_tablet() const
void ObLS::finish_create(const bool is_commit) void ObLS::finish_create(const bool is_commit)
{ {
ObInnerLSStatus status = ObInnerLSStatus::CREATING;
if (is_commit) { if (is_commit) {
ls_meta_.ls_create_status_ = ObInnerLSStatus::COMMITTED; status = ObInnerLSStatus::COMMITTED;
} else { } else {
ls_meta_.ls_create_status_ = ObInnerLSStatus::ABORTED; status = ObInnerLSStatus::ABORTED;
} }
ls_meta_.set_ls_create_status(status);
} }
int ObLS::start() int ObLS::start()
@ -408,34 +415,88 @@ int ObLS::start()
int ObLS::stop() 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; int ret = OB_SUCCESS;
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("ls is not inited", K(ret)); LOG_WARN("ls is not inited", K(ret));
} else if (OB_FAIL(stop_())) {
LOG_WARN("stop ls failed", K(ret), K(ls_meta_));
} else { } else {
tx_table_.stop(); }
ls_restore_handler_.stop(); return ret;
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)) { int ObLS::stop_()
if (OB_FAIL(prepare_for_safe_destroy_())) { {
LOG_WARN("fail to prepare_for_safe_destroy", K(ret)); int ret = OB_SUCCESS;
} else { tx_table_.stop();
LOG_INFO("stop_ls finish", KR(ret), KPC(this)); 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; 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 // 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 // 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. // 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_safe = false;
bool is_ls_restore_handler_safe = false; bool is_ls_restore_handler_safe = false;
bool is_tablet_service_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))) { 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)); 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)); 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))) { } 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)); LOG_WARN("ls restore handler safe to destroy failed", K(ret), KPC(this));
} else if (!is_ls_restore_handler_safe) { } 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)); 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))) { } else if (!is_log_handler_safe) {
LOG_WARN("ls migration handler check safe to destroy failed", K(ret), KPC(this)); } else {
} else if (!is_ls_migration_handler_safe) {
} else if (is_tablet_service_safe) {
if (1 == ref_mgr_.get_total_ref_cnt()) { // only has one ref at the safe destroy task if (1 == ref_mgr_.get_total_ref_cnt()) { // only has one ref at the safe destroy task
is_safe = true; is_safe = true;
} }
@ -482,8 +544,10 @@ bool ObLS::safe_to_destroy()
if (OB_SUCC(ret)) { if (OB_SUCC(ret)) {
if (!is_safe) { if (!is_safe) {
if (REACH_TIME_INTERVAL(60 * 1000 * 1000)) { 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(), LOG_WARN("this ls is not safe to destroy", K(is_safe),
K(is_tablet_service_safe), K(is_ls_restore_handler_safe), K(is_ls_migration_handler_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)); K(ret), KP(this), KPC(this));
ref_mgr_.print(); ref_mgr_.print();
PRINT_OBJ_LEAK(MTL_ID(), share::LEAK_CHECK_OBJ_LS_HANDLE); 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())); FLOG_INFO("ObLS destroy", K(this), K(*this), K(lbt()));
if (OB_TMP_FAIL(offline_())) { if (OB_TMP_FAIL(offline_())) {
LOG_WARN("ls offline failed.", K(tmp_ret), K(ls_meta_.ls_id_)); 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_)); 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::TRANS_SERVICE_LOG_BASE_TYPE, &ls_tx_svr_);
UNREGISTER_FROM_LOGSERVICE(logservice::STORAGE_SCHEMA_LOG_BASE_TYPE, &ls_tablet_svr_); UNREGISTER_FROM_LOGSERVICE(logservice::STORAGE_SCHEMA_LOG_BASE_TYPE, &ls_tablet_svr_);
@ -597,6 +663,17 @@ int ObLS::offline_tx_()
return ret; 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 ObLS::offline_()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
@ -604,8 +681,11 @@ int ObLS::offline_()
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("ls is not inited", K(ret)); LOG_WARN("ls is not inited", K(ret));
} else if (FALSE_IT(is_offlined_ = true)) {
} else if (FALSE_IT(checkpoint_executor_.offline())) { } else if (FALSE_IT(checkpoint_executor_.offline())) {
LOG_WARN("checkpoint executor offline failed", K(ret), K(ls_meta_)); 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())) { } else if (OB_FAIL(ls_wrs_handler_.offline())) {
LOG_WARN("weak read handler offline failed", K(ret), K(ls_meta_)); LOG_WARN("weak read handler offline failed", K(ret), K(ls_meta_));
} else if (OB_FAIL(log_handler_.offline())) { } else if (OB_FAIL(log_handler_.offline())) {
@ -654,6 +734,13 @@ int ObLS::online_tx_()
return ret; return ret;
} }
int ObLS::online_compaction_()
{
int ret = OB_SUCCESS;
ls_freezer_.online();
return ret;
}
int ObLS::online() int ObLS::online()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
@ -676,9 +763,12 @@ int ObLS::online()
LOG_WARN("failed to online log", K(ret)); LOG_WARN("failed to online log", K(ret));
} else if (OB_FAIL(ls_wrs_handler_.online())) { } else if (OB_FAIL(ls_wrs_handler_.online())) {
LOG_WARN("weak read handler online failed", K(ret), K(ls_meta_)); 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())) { } else if (FALSE_IT(checkpoint_executor_.online())) {
LOG_WARN("checkpoint executor online failed", K(ret), K(ls_meta_)); LOG_WARN("checkpoint executor online failed", K(ret), K(ls_meta_));
} else { } else {
is_offlined_ = false;
// do nothing // do nothing
} }
@ -711,8 +801,13 @@ int ObLS::set_ls_meta(const ObLSMeta &ls_meta)
LOG_WARN("ls is not inited", K(ret)); LOG_WARN("ls is not inited", K(ret));
} else { } else {
ls_meta_ = ls_meta; ls_meta_ = ls_meta;
if (OB_FAIL(ls_meta_.update_id_service())) { if (IDS_LS == ls_meta_.ls_id_) {
LOG_WARN("update id service fail", K(ret)); 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; return ret;
@ -845,6 +940,9 @@ int ObLS::update_tablet_table_store(
ObTabletHandle &handle) ObTabletHandle &handle)
{ {
int ret = OB_SUCCESS; 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) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("ls is not inited", K(ret)); LOG_WARN("ls is not inited", K(ret));
@ -852,9 +950,6 @@ int ObLS::update_tablet_table_store(
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("update tablet table store get invalid argument", K(ret), K(tablet_id), K(param)); LOG_WARN("update tablet table store get invalid argument", K(ret), K(tablet_id), K(param));
} else { } 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(); const int64_t rebuild_seq = ls_meta_.get_rebuild_seq();
if (param.rebuild_seq_ != rebuild_seq) { if (param.rebuild_seq_ != rebuild_seq) {
ret = OB_EAGAIN; ret = OB_EAGAIN;
@ -872,6 +967,10 @@ int ObLS::build_ha_tablet_new_table_store(
const ObBatchUpdateTableStoreParam &param) const ObBatchUpdateTableStoreParam &param)
{ {
int ret = OB_SUCCESS; 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) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("ls is not inited", K(ret)); LOG_WARN("ls is not inited", K(ret));
@ -879,10 +978,6 @@ int ObLS::build_ha_tablet_new_table_store(
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("build ha tablet new table store get invalid argument", K(ret), K(tablet_id), K(param)); LOG_WARN("build ha tablet new table store get invalid argument", K(ret), K(tablet_id), K(param));
} else { } 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) { if (param.rebuild_seq_ != rebuild_seq) {
ret = OB_EAGAIN; ret = OB_EAGAIN;
LOG_WARN("build ha tablet new table store rebuild seq not same, need retry", 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))) { new_migration_status))) {
LOG_WARN("failed to trans fail status", K(ret), K(current_migration_status), LOG_WARN("failed to trans fail status", K(ret), K(current_migration_status),
K(new_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*/))) { 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)); LOG_WARN("failed to set migration status", K(ret), K(new_migration_status));
} else if (is_need_gc()) { } else if (is_need_gc()) {
@ -1203,7 +1298,7 @@ int ObLS::try_update_uppder_trans_version()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS;
int64_t read_lock = LSLOCKLS | LSLOCKSTORAGE; int64_t read_lock = LSLOCKLOGMETA;
int64_t write_lock = 0; int64_t write_lock = 0;
ObLSLockGuard lock_myself(lock_, read_lock, write_lock); 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 ObLS::set_tablet_change_checkpoint_ts(const int64_t log_ts)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
int64_t read_lock = LSLOCKLS; int64_t read_lock = 0;
int64_t write_lock = LSLOCKLOGMETA; int64_t write_lock = LSLOCKLOGMETA;
ObLSLockGuard lock_myself(lock_, read_lock, write_lock); ObLSLockGuard lock_myself(lock_, read_lock, write_lock);
UPDATE_LS_META_CHECK(); if (IS_NOT_INIT) {
if (OB_FAIL(ret)) { 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))) { } 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)); LOG_WARN("fail to set tablet_change_checkpoint_ts", K(ret), K(log_ts), K_(ls_meta));
} else { } else {
@ -1264,47 +1363,25 @@ int ObLS::set_tablet_change_checkpoint_ts(const int64_t log_ts)
return ret; return ret;
} }
int ObLS::update_id_meta_with_writing_slog(const int64_t service_type, int ObLS::update_ls_meta(const bool update_restore_status,
const int64_t limited_id, const ObLSMeta &src_ls_meta)
const int64_t latest_log_ts)
{ {
int ret = OB_SUCCESS; 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) { if (IS_NOT_INIT) {
ret = OB_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_)) { } else if (OB_UNLIKELY(is_stopped_)) {
ret = OB_NOT_RUNNING; ret = OB_NOT_RUNNING;
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta)); LOG_WARN("ls stopped", K(ret), K_(ls_meta));
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) { } else if (OB_FAIL(ls_meta_.update_ls_meta(update_restore_status, src_ls_meta))) {
ret = OB_STATE_NOT_MATCH; LOG_WARN("update ls meta fail", K(ret), K_(ls_meta), K(update_restore_status), K(src_ls_meta));
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_)); } else if (IDS_LS == ls_meta_.ls_id_) {
} else if (OB_FAIL(ls_meta_.update_id_meta(service_type, limited_id, latest_log_ts, true))) { ObAllIDMeta all_id_meta;
STORAGE_LOG(WARN, "update id meta with slog fail", K(ret), K_(ls_meta), K(service_type), K(limited_id), K(latest_log_ts)); if (OB_FAIL(ls_meta_.get_all_id_meta(all_id_meta))) {
} else { LOG_WARN("get all id meta failed", K(ret), K(ls_meta_));
// do nothing } 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;
}
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));
} else { } else {
// do nothing // do nothing
} }
@ -1363,10 +1440,9 @@ int ObLS::set_migration_status(
} else if (!ObMigrationStatusHelper::is_valid(migration_status)) { } else if (!ObMigrationStatusHelper::is_valid(migration_status)) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("set migration status get invalid argument", K(ret), K(migration_status)); LOG_WARN("set migration status get invalid argument", K(ret), K(migration_status));
} else if (OB_UNLIKELY(is_stopped_)) { // migration status should be update after ls stopped, to make sure migrate task
ret = OB_NOT_RUNNING; // will be finished later.
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta)); } else if (!can_update_ls_meta(ls_meta_.get_ls_create_status())) {
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
ret = OB_STATE_NOT_MATCH; ret = OB_STATE_NOT_MATCH;
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_)); STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
} else if (ls_meta_.get_rebuild_seq() != rebuild_seq) { } else if (ls_meta_.get_rebuild_seq() != rebuild_seq) {
@ -1402,10 +1478,9 @@ int ObLS::set_restore_status(
} else if (!restore_status.is_valid()) { } else if (!restore_status.is_valid()) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("set restore status get invalid argument", K(ret), K(restore_status)); LOG_WARN("set restore status get invalid argument", K(ret), K(restore_status));
} else if (OB_UNLIKELY(is_stopped_)) { // restore status should be update after ls stopped, to make sure restore task
ret = OB_NOT_RUNNING; // will be finished later.
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta)); } else if (!can_update_ls_meta(ls_meta_.get_ls_create_status())) {
} else if (!can_update_ls_meta(ls_meta_.ls_create_status_)) {
ret = OB_STATE_NOT_MATCH; ret = OB_STATE_NOT_MATCH;
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_)); STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
} else if (ls_meta_.get_rebuild_seq() != rebuild_seq) { } else if (ls_meta_.get_rebuild_seq() != rebuild_seq) {
@ -1438,7 +1513,7 @@ int ObLS::set_ls_rebuild()
} else if (OB_UNLIKELY(is_stopped_)) { } else if (OB_UNLIKELY(is_stopped_)) {
ret = OB_NOT_RUNNING; ret = OB_NOT_RUNNING;
STORAGE_LOG(WARN, "ls stopped", K(ret), K_(ls_meta)); 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; ret = OB_STATE_NOT_MATCH;
STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_)); STORAGE_LOG(WARN, "state not match, cannot update ls meta", K(ret), K(ls_meta_));
} else if (OB_FAIL(ls_meta_.set_ls_rebuild())) { } else if (OB_FAIL(ls_meta_.set_ls_rebuild())) {

View File

@ -65,48 +65,6 @@ class ObIMetaReport;
namespace storage 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; const static int64_t LS_INNER_TABLET_FROZEN_TIMESTAMP = 1;
struct ObLSVTInfo struct ObLSVTInfo
@ -185,11 +143,12 @@ public:
// I am ready to work now. // I am ready to work now.
int start(); int start();
int stop(); int stop();
void wait();
bool safe_to_destroy(); bool safe_to_destroy();
void destroy(); void destroy();
int offline(); int offline();
int online(); 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_; } ObLSTxService *get_tx_svr() { return &ls_tx_svr_; }
ObLockTable *get_lock_table() { return &lock_table_; } ObLockTable *get_lock_table() { return &lock_table_; }
@ -233,7 +192,7 @@ public:
// set create state of ls. // set create state of ls.
// @param[in] new_status, the new create state which will be set. // @param[in] new_status, the new create state which will be set.
void set_create_state(const ObInnerLSStatus new_status); 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_gc() const;
bool is_need_load_inner_tablet() const; bool is_need_load_inner_tablet() const;
// for rebuild // 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)); TO_STRING_KV(K_(ls_meta), K_(log_handler), K_(restore_handler), K_(is_inited), K_(tablet_gc_handler));
private: private:
int stop_();
void wait_();
int prepare_for_safe_destroy_(); int prepare_for_safe_destroy_();
int flush_if_need_(const bool need_flush); int flush_if_need_(const bool need_flush);
int offline_(); int offline_();
int offline_compaction_();
int online_compaction_();
int offline_tx_(); int offline_tx_();
int online_tx_(); int online_tx_();
public: public:
// ObLSMeta interface: // ObLSMeta interface:
int update_id_meta_with_writing_slog(const int64_t service_type, int update_ls_meta(const bool update_restore_status,
const int64_t limited_id, const ObLSMeta &src_ls_meta);
const int64_t latest_log_ts);
int update_id_meta_without_writing_slog(const int64_t service_type, // int update_id_meta(const int64_t service_type,
const int64_t limited_id, // const int64_t limited_id,
const int64_t latest_log_ts); // const int64_t latest_log_ts,
// const bool write_slog);
DELEGATE_WITH_RET(ls_meta_, update_id_meta, int);
int set_ls_rebuild(); int set_ls_rebuild();
// protect in ls lock // protect in ls lock
// int set_gc_state(const logservice::LSGCState &gc_state); // 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, // int set_clog_checkpoint(const palf::LSN &clog_checkpoint_lsn,
// const int64_t clog_checkpoint_ts, // const int64_t clog_checkpoint_ts,
// const bool write_slog = true); // const bool write_slog = true);
UPDATE_LSMETA_WITH_LOCK(ls_meta_, set_clog_checkpoint); DELEGATE_WITH_RET(ls_meta_, set_clog_checkpoint, int);
UPDATE_LSMETA_WITHOUT_LOCK(ls_meta_, set_clog_checkpoint);
CONST_DELEGATE_WITH_RET(ls_meta_, get_clog_checkpoint_ts, int64_t); 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_clog_base_lsn, palf::LSN &);
DELEGATE_WITH_RET(ls_meta_, get_saved_info, int); DELEGATE_WITH_RET(ls_meta_, get_saved_info, int);
// int build_saved_info(); // 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(); // int clear_saved_info_without_lock();
UPDATE_LSMETA_WITHOUT_LOCK(ls_meta_, clear_saved_info); DELEGATE_WITH_RET(ls_meta_, clear_saved_info, int);
// 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);
CONST_DELEGATE_WITH_RET(ls_meta_, get_rebuild_seq, int64_t); CONST_DELEGATE_WITH_RET(ls_meta_, get_rebuild_seq, int64_t);
CONST_DELEGATE_WITH_RET(ls_meta_, get_tablet_change_checkpoint_ts, int64_t); CONST_DELEGATE_WITH_RET(ls_meta_, get_tablet_change_checkpoint_ts, int64_t);
@ -367,7 +325,7 @@ public:
// set offline ts // set offline ts
// @param [in] offline ts. // @param [in] offline ts.
// int set_offline_ts_ns(const int64_t offline_ts_ns); // 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 // get offline ts
// @param [in] offline ts. // @param [in] offline ts.
// int get_offline_ts_ns(int64_t &offline_ts_ns); // int get_offline_ts_ns(int64_t &offline_ts_ns);
@ -375,7 +333,8 @@ public:
// update replayable point // update replayable point
// @param [in] replayable point. // @param [in] replayable point.
// int update_ls_replayable_point(const int64_t 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 // get replayable point
// @param [in] replayable point // @param [in] replayable point
// int get_ls_replayable_point(int64_t &replayable_point); // int get_ls_replayable_point(int64_t &replayable_point);
@ -721,6 +680,7 @@ private:
bool is_inited_; bool is_inited_;
uint64_t tenant_id_; uint64_t tenant_id_;
bool is_stopped_; bool is_stopped_;
bool is_offlined_;
ObLSMeta ls_meta_; ObLSMeta ls_meta_;
observer::ObIMetaReport *rs_reporter_; observer::ObIMetaReport *rs_reporter_;
ObLSLock lock_; ObLSLock lock_;

View File

@ -22,6 +22,7 @@ using namespace common;
using namespace palf; using namespace palf;
using namespace logservice; using namespace logservice;
using namespace share; using namespace share;
using namespace transaction;
namespace storage namespace storage
{ {
@ -83,6 +84,17 @@ ObLSMeta::ObLSMeta(const ObLSMeta &ls_meta)
all_id_meta_.update_all_id_meta(ls_meta.all_id_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) ObLSMeta &ObLSMeta::operator=(const ObLSMeta &other)
{ {
ObSpinLockTimeGuard guard(other.lock_); 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 int64_t clog_checkpoint_ts,
const bool write_slog) 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; int ret = OB_SUCCESS;
if (write_slog) { ObSpinLockTimeGuard guard(lock_);
if (OB_FAIL(write_slog_(tmp))) { if (OB_FAIL(check_can_update_())) {
LOG_WARN("clog_checkpoint write slog failed", K(ret)); 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; if (write_slog) {
clog_checkpoint_ts_ = clog_checkpoint_ts; 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; return ret;
} }
@ -168,9 +184,12 @@ int ObLSMeta::set_tablet_change_checkpoint_ts(const int64_t tablet_change_checkp
{ {
ObSpinLockTimeGuard guard(lock_); ObSpinLockTimeGuard guard(lock_);
int ret = OB_SUCCESS; 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; 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 { } else {
ObLSMeta tmp(*this); ObLSMeta tmp(*this);
tmp.tablet_change_checkpoint_ts_ = tablet_change_checkpoint_ts; 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; int ret = OB_SUCCESS;
bool can_change = false; 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; 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 { } else {
ObSpinLockTimeGuard guard(lock_); ObLSMeta tmp(*this);
if (migration_status_ == migration_status) { tmp.migration_status_ = migration_status;
//do nothing if (write_slog && OB_FAIL(write_slog_(tmp))) {
} else if (ObMigrationStatusHelper::check_can_change_status(migration_status_, migration_status, can_change)) { LOG_WARN("migration_status write slog failed", K(ret));
LOG_WARN("failed to check can change stauts", K(ret), K(migration_status_), K(migration_status));
} else { } else {
ObLSMeta tmp(*this); migration_status_ = migration_status;
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;
}
} }
} }
return ret; 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 ObLSMeta::set_gc_state(const logservice::LSGCState &gc_state)
{ {
int ret = OB_SUCCESS; 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; ret = OB_ERR_UNEXPECTED;
LOG_WARN("gc_state invalid", K(ret), K(gc_state)); LOG_WARN("gc_state invalid", K(ret), K(gc_state));
} else { } else {
ObSpinLockTimeGuard guard(lock_);
ObLSMeta tmp(*this); ObLSMeta tmp(*this);
tmp.gc_state_ = gc_state; tmp.gc_state_ = gc_state;
if (OB_FAIL(write_slog_(tmp))) { if (OB_FAIL(write_slog_(tmp))) {
@ -277,7 +306,11 @@ int ObLSMeta::set_offline_ts_ns(const int64_t offline_ts_ns)
// 不主动写slog // 不主动写slog
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_); 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; 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 ObLSMeta::set_restore_status(const ObLSRestoreStatus &restore_status)
{ {
int ret = OB_SUCCESS; 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; ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid restore status", K(ret), K(restore_status_), K(restore_status)); LOG_WARN("invalid restore status", K(ret), K(restore_status_), K(restore_status));
} else if (restore_status_ == restore_status) {
//do nothing
} else { } else {
ObSpinLockTimeGuard guard(lock_); ObLSMeta tmp(*this);
if (restore_status_ == restore_status) { tmp.restore_status_ = restore_status;
//do nothing if (OB_FAIL(write_slog_(tmp))) {
LOG_WARN("restore_status write slog failed", K(ret));
} else { } else {
ObLSMeta tmp(*this); restore_status_ = restore_status;
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;
}
} }
} }
return ret; return ret;
@ -334,7 +367,9 @@ int ObLSMeta::update_ls_replayable_point(const int64_t replayable_point)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_); 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; ret = OB_INVALID_ARGUMENT;
LOG_WARN("replayable_point invalid", K(ret), K(replayable_point), K(replayable_point_)); LOG_WARN("replayable_point invalid", K(ret), K(replayable_point), K(replayable_point_));
} else if (replayable_point_ == replayable_point) { } else if (replayable_point_ == replayable_point) {
@ -372,9 +407,9 @@ int ObLSMeta::update_ls_meta(
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObLSRestoreStatus ls_restore_status; ObLSRestoreStatus ls_restore_status;
if (!is_valid()) { ObSpinLockTimeGuard guard(lock_);
ret = OB_ERR_UNEXPECTED; if (OB_FAIL(check_can_update_())) {
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this)); LOG_WARN("ls meta cannot update", K(ret), K(*this));
} else if (!src_ls_meta.is_valid()) { } else if (!src_ls_meta.is_valid()) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("update ls meta get invalid argument", K(ret), K(src_ls_meta)); 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))) { && OB_FAIL(src_ls_meta.get_restore_status(ls_restore_status))) {
LOG_WARN("failed to get restore status", K(ret), K(src_ls_meta)); LOG_WARN("failed to get restore status", K(ret), K(src_ls_meta));
} else { } else {
{ ObLSMeta tmp(*this);
ObSpinLockTimeGuard guard(lock_); tmp.clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
ObLSMeta tmp(*this); tmp.clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
tmp.clog_base_lsn_ = src_ls_meta.clog_base_lsn_; tmp.replayable_point_ = src_ls_meta.replayable_point_;
tmp.clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_; tmp.tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_;
tmp.replayable_point_ = src_ls_meta.replayable_point_; tmp.rebuild_seq_++;
tmp.tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_; if (update_restore_status) {
tmp.rebuild_seq_++; tmp.restore_status_ = ls_restore_status;
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_);
guard.click(); if (tmp.clog_checkpoint_ts_ < clog_checkpoint_ts_) {
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 // TODO: now do not allow clog checkpoint ts rollback, may support it in 4.1
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("do not allow clog checkpoint ts rollback", K(ret), K(src_ls_meta), KPC(this)); LOG_WARN("do not allow clog checkpoint ts rollback", K(ret), K(src_ls_meta), KPC(this));
} else if (OB_FAIL(write_slog_(tmp))) { } else if (OB_FAIL(write_slog_(tmp))) {
LOG_WARN("clog_checkpoint write slog failed", K(ret)); LOG_WARN("clog_checkpoint write slog failed", K(ret));
} else { } else {
guard.click(); guard.click();
clog_base_lsn_ = src_ls_meta.clog_base_lsn_; clog_base_lsn_ = src_ls_meta.clog_base_lsn_;
clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_; clog_checkpoint_ts_ = src_ls_meta.clog_checkpoint_ts_;
replayable_point_ = src_ls_meta.replayable_point_; replayable_point_ = src_ls_meta.replayable_point_;
tablet_change_checkpoint_ts_ = src_ls_meta.tablet_change_checkpoint_ts_; 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) { if (update_restore_status) {
restore_status_ = ls_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));
} }
} }
LOG_INFO("update ls meta", K(ret), K(tmp), K(src_ls_meta), K(*this));
} }
return ret; return ret;
} }
@ -430,13 +456,11 @@ int ObLSMeta::set_ls_rebuild()
const ObMigrationStatus change_status = ObMigrationStatus::OB_MIGRATION_STATUS_REBUILD; const ObMigrationStatus change_status = ObMigrationStatus::OB_MIGRATION_STATUS_REBUILD;
bool can_change = false; bool can_change = false;
if (!is_valid()) { ObSpinLockTimeGuard guard(lock_);
ret = OB_ERR_UNEXPECTED; if (OB_FAIL(check_can_update_())) {
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this)); LOG_WARN("ls meta cannot update", K(ret), K(*this));
} else { } else {
ObSpinLockTimeGuard guard(lock_);
ObLSMeta tmp(*this); ObLSMeta tmp(*this);
if (OB_FAIL(ObMigrationStatusHelper::check_can_change_status(tmp.migration_status_, change_status, can_change))) { 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)); LOG_WARN("failed to check can change status", K(ret), K(migration_status_), K(change_status));
} else if (!can_change) { } else if (!can_change) {
@ -490,9 +514,8 @@ int ObLSMeta::set_saved_info(const ObLSSavedInfo &saved_info)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_); ObSpinLockTimeGuard guard(lock_);
if (!is_valid()) { if (OB_FAIL(check_can_update_())) {
ret = OB_ERR_UNEXPECTED; LOG_WARN("ls meta cannot update", K(ret), K(*this));
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this));
} else if (!saved_info.is_valid()) { } else if (!saved_info.is_valid()) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("set saved info get invalid argument", K(ret), K(saved_info)); 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 ObLSMeta::build_saved_info()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_);
ObLSSavedInfo saved_info; ObLSSavedInfo saved_info;
if (!is_valid()) { ObSpinLockTimeGuard guard(lock_);
ret = OB_ERR_UNEXPECTED; if (OB_FAIL(check_can_update_())) {
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this)); LOG_WARN("ls meta cannot update", K(ret), K(*this));
} else if (!saved_info_.is_empty()) { } else if (!saved_info_.is_empty()) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("saved info is not empty, can not build saved info", K(ret), K(*this)); 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 ObLSMeta::clear_saved_info()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_);
ObLSSavedInfo saved_info; ObLSSavedInfo saved_info;
if (!is_valid()) { ObSpinLockTimeGuard guard(lock_);
ret = OB_ERR_UNEXPECTED; if (OB_FAIL(check_can_update_())) {
LOG_WARN("ls meta is not valid, cannot update", K(ret), K(*this)); LOG_WARN("ls meta cannot update", K(ret), K(*this));
} else { } else {
saved_info.reset(); saved_info.reset();
ObLSMeta tmp(*this); ObLSMeta tmp(*this);
@ -601,19 +622,47 @@ int ObLSMeta::update_id_meta(const int64_t service_type,
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObSpinLockTimeGuard guard(lock_); ObSpinLockTimeGuard guard(lock_);
all_id_meta_.update_id_meta(service_type, limited_id, latest_log_ts); if (OB_FAIL(check_can_update_())) {
guard.click(); LOG_WARN("ls meta cannot update", K(ret), K(*this));
if (write_slog) { } else {
if (OB_FAIL(write_slog_(*this))) { // TODO: write slog may failed, but the content is updated.
LOG_WARN("id service flush write slog failed", K(ret)); 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), LOG_INFO("update id meta", K(ret), K(service_type), K(limited_id), K(latest_log_ts),
K(*this)); K(*this));
return ret; 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( int ObLSMeta::get_migration_and_restore_status(
ObMigrationStatus &migration_status, ObMigrationStatus &migration_status,
share::ObLSRestoreStatus &ls_restore_status) share::ObLSRestoreStatus &ls_restore_status)

View File

@ -40,14 +40,22 @@ public:
ObLSMeta(); ObLSMeta();
ObLSMeta(const ObLSMeta &ls_meta); ObLSMeta(const ObLSMeta &ls_meta);
~ObLSMeta() {} ~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); ObLSMeta &operator=(const ObLSMeta &other);
int64_t get_clog_checkpoint_ts() const; int64_t get_clog_checkpoint_ts() const;
palf::LSN &get_clog_base_lsn(); palf::LSN &get_clog_base_lsn();
int set_clog_checkpoint(const palf::LSN &clog_checkpoint_lsn, int set_clog_checkpoint(const palf::LSN &clog_checkpoint_lsn,
const int64_t clog_checkpoint_ts, const int64_t clog_checkpoint_ts,
const bool write_slog = true); const bool write_slog = true);
void reset();
bool is_valid() const;
int64_t get_rebuild_seq() const; int64_t get_rebuild_seq() const;
int set_migration_status(const ObMigrationStatus &migration_status, int set_migration_status(const ObMigrationStatus &migration_status,
const bool write_slog = true); const bool write_slog = true);
@ -73,10 +81,7 @@ public:
const int64_t limited_id, const int64_t limited_id,
const int64_t latest_log_ts, const int64_t latest_log_ts,
const bool write_slog); const bool write_slog);
int update_id_service() int get_all_id_meta(transaction::ObAllIDMeta &all_id_meta) const;
{
return all_id_meta_.update_id_service();
}
int get_saved_info(ObLSSavedInfo &saved_info); int get_saved_info(ObLSSavedInfo &saved_info);
int build_saved_info(); int build_saved_info();
int set_saved_info(const ObLSSavedInfo &saved_info); int set_saved_info(const ObLSSavedInfo &saved_info);
@ -84,14 +89,6 @@ public:
int get_migration_and_restore_status( int get_migration_and_restore_status(
ObMigrationStatus &migration_status, ObMigrationStatus &migration_status,
share::ObLSRestoreStatus &ls_restore_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 class ObSpinLockTimeGuard
{ {
public: public:
@ -108,14 +105,15 @@ public:
K_(rebuild_seq), K_(migration_status), K(gc_state_), K(offline_ts_ns_), 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_(restore_status), K_(replayable_point), K_(tablet_change_checkpoint_ts),
K_(all_id_meta)); K_(all_id_meta));
private:
int check_can_update_();
public: public:
mutable common::ObSpinLock lock_; mutable common::ObSpinLock lock_;
uint64_t tenant_id_; uint64_t tenant_id_;
share::ObLSID ls_id_; share::ObLSID ls_id_;
ObReplicaType replica_type_; ObReplicaType replica_type_;
ObInnerLSStatus ls_create_status_;
private: private:
ObInnerLSStatus ls_create_status_;
typedef common::ObFunction<int(ObLSMeta &)> WriteSlog; typedef common::ObFunction<int(ObLSMeta &)> WriteSlog;
// for test // for test
void set_write_slog_func_(WriteSlog write_slog); void set_write_slog_func_(WriteSlog write_slog);

View File

@ -239,11 +239,17 @@ int ObIDService::update_ls_id_meta(const bool write_slog)
if (OB_FAIL(check_and_fill_ls())) { if (OB_FAIL(check_and_fill_ls())) {
TRANS_LOG(WARN, "ls set fail", K(ret)); TRANS_LOG(WARN, "ls set fail", K(ret));
} else if (write_slog) { } 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 { } 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)) { if (OB_FAIL(ret)) {
TRANS_LOG(WARN, "update id meta fail", K(ret), K_(service_type)); 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; break;
default: default:
ret = OB_ERR_UNEXPECTED; 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; return ret;
@ -636,39 +665,47 @@ OB_DEF_DESERIALIZE(ObAllIDMeta)
void ObAllIDMeta::update_all_id_meta(const ObAllIDMeta &all_id_meta) 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++) { 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].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_); (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; int ret = OB_SUCCESS;
ObSpinLockGuard lock_guard(lock_);
for(int i=0; i<ObIDService::MAX_SERVICE_TYPE; i++) { if (service_type <= ObIDService::INVALID_ID_SERVICE_TYPE ||
ObIDService *id_service = NULL; service_type >= ObIDService::MAX_SERVICE_TYPE) {
if (OB_FAIL(ObIDService::get_id_service(i, id_service))) { ret = OB_INVALID_ARGUMENT;
SERVER_LOG(WARN, "get id service fail", K(ret), K(*this)); TRANS_LOG(WARN, "invalid argument", K(ret), K(service_type));
} else if (OB_ISNULL(id_service)) { } else {
ret = OB_ERR_UNEXPECTED; (void)inc_update(&id_meta_[service_type].limited_id_, limited_id);
SERVER_LOG(WARN, "id service is null", K(ret), K(*this)); (void)inc_update(&id_meta_[service_type].latest_log_ts_, latest_log_ts);
} else {
id_service->update_limited_id(id_meta_[i].limited_id_,
id_meta_[i].latest_log_ts_);
}
} }
return ret; return ret;
} }
void ObAllIDMeta::update_id_meta(const int64_t service_type, int ObAllIDMeta::get_id_meta(const int64_t service_type,
const int64_t limited_id, int64_t &limited_id,
const int64_t latest_log_ts) int64_t &latest_log_ts) const
{ {
(void)inc_update(&id_meta_[service_type].limited_id_, limited_id); int ret = OB_SUCCESS;
(void)inc_update(&id_meta_[service_type].latest_log_ts_, 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 {
limited_id = id_meta_[service_type].limited_id_;
latest_log_ts = id_meta_[service_type].latest_log_ts_;
}
return ret;
} }
} }
} }

View File

@ -29,6 +29,7 @@ class ObLS;
namespace transaction namespace transaction
{ {
class ObAllIDMeta;
class ObPresistIDLog class ObPresistIDLog
{ {
@ -97,6 +98,7 @@ public:
static void get_all_id_service_type(int64_t *service_type); 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 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); 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() : count_(ObIDService::MAX_SERVICE_TYPE) {}
~ObAllIDMeta() {} ~ObAllIDMeta() {}
void update_all_id_meta(const ObAllIDMeta &all_id_meta); void update_all_id_meta(const ObAllIDMeta &all_id_meta);
int update_id_service(); int update_id_meta(const int64_t service_type,
void update_id_meta(const int64_t service_type, const int64_t limited_id,
const int64_t limited_id, const int64_t latest_log_ts);
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 to_string(char* buf, const int64_t buf_len) const
{ {
int64_t pos = 0; int64_t pos = 0;
@ -196,6 +200,7 @@ public:
} }
OB_UNIS_VERSION(1); OB_UNIS_VERSION(1);
private: private:
mutable common::ObSpinLock lock_;
int64_t count_; int64_t count_;
ObIDMeta id_meta_[ObIDService::MAX_SERVICE_TYPE]; ObIDMeta id_meta_[ObIDService::MAX_SERVICE_TYPE];
}; };

View File

@ -137,14 +137,15 @@ int ObLSService::stop()
LOG_ERROR("ls is null", K(ret)); LOG_ERROR("ls is null", K(ret));
} else if (OB_FAIL(ls->offline())) { } else if (OB_FAIL(ls->offline())) {
LOG_WARN("ls offline failed", K(ret), K(ls->get_ls_id()), KP(ls)); 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))) { } else if (OB_FAIL(handle.set_ls(ls_map_, *ls, ObLSGetMod::TXSTORAGE_MOD))) {
LOG_WARN("get ls handle failed", K(ret), KPC(ls)); LOG_WARN("get ls handle failed", K(ret), KPC(ls));
} else { } else {
ObLSLockGuard lock_ls(ls); ObLSLockGuard lock_ls(ls);
if (OB_FAIL(ls->stop())) { if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id)); "LSSafeDestroy"))) {
} else if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
"LSSafeDestroy"))) {
ret = OB_ALLOCATE_MEMORY_FAILED; ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret)); LOG_WARN("alloc memory failed", K(ret));
} else if (FALSE_IT(task = new(task) ObLSSafeDestroyTask())) { } 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(); ls_status = ls->get_create_state();
if (ObInnerLSStatus::CREATING == ls_status || ObInnerLSStatus::REMOVED == ls_status) { if (ObInnerLSStatus::CREATING == ls_status || ObInnerLSStatus::REMOVED == ls_status) {
do { do {
ObLSLockGuard lock_ls(ls);
if (OB_TMP_FAIL(ls->stop())) { if (OB_TMP_FAIL(ls->stop())) {
LOG_WARN("ls stop failed", K(tmp_ret), K(ls->get_ls_id()), KP(ls)); LOG_WARN("ls stop failed", K(tmp_ret), K(ls->get_ls_id()), KP(ls));
} else {
ls->wait();
} }
if (OB_SUCCESS != tmp_ret) { if (OB_SUCCESS != tmp_ret) {
usleep(SLEEP_TS); 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. // ls leader gc must has block tx start, gracefully kill tx and write offline log before here.
} else if (OB_FAIL(ls->offline())) { } else if (OB_FAIL(ls->offline())) {
LOG_WARN("ls offline failed", K(ret), K(ls_id), KP(ls)); 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 { } else {
ObLSSafeDestroyTask *task = nullptr; ObLSSafeDestroyTask *task = nullptr;
static const int64_t SLEEP_TS = 100_ms; static const int64_t SLEEP_TS = 100_ms;
ObLSLockGuard lock_ls(ls); ObLSLockGuard lock_ls(ls);
if (OB_FAIL(ls->stop())) { if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
LOG_WARN("stop ls failed", K(ret), KP(ls), K(ls_id)); "LSSafeDestroy"))) {
} else if (OB_ISNULL(task = (ObLSSafeDestroyTask*)ob_malloc(sizeof(ObLSSafeDestroyTask),
"LSSafeDestroy"))) {
ret = OB_ALLOCATE_MEMORY_FAILED; ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret)); LOG_WARN("alloc memory failed", K(ret));
} else if (FALSE_IT(task = new(task) ObLSSafeDestroyTask())) { } else if (FALSE_IT(task = new(task) ObLSSafeDestroyTask())) {