Add ddl_commit_scn to guarantee single ddl commit log
This commit is contained in:
@ -32,8 +32,7 @@ using namespace oceanbase::storage;
|
||||
ObTabletDDLKvMgr::ObTabletDDLKvMgr()
|
||||
: is_inited_(false), success_start_scn_(SCN::min_scn()), ls_id_(), tablet_id_(), table_key_(), cluster_version_(0),
|
||||
start_scn_(SCN::min_scn()), commit_scn_(SCN::min_scn()), max_freeze_scn_(SCN::min_scn()),
|
||||
table_id_(0), execution_id_(-1), head_(0), tail_(0), lock_(ObLatchIds::TABLET_DDL_KV_MGR_LOCK), ref_cnt_(0),
|
||||
can_schedule_major_compaction_(false)
|
||||
table_id_(0), execution_id_(-1), head_(0), tail_(0), lock_(), ref_cnt_(0)
|
||||
{
|
||||
}
|
||||
|
||||
@ -47,7 +46,7 @@ void ObTabletDDLKvMgr::destroy()
|
||||
if (is_started()) {
|
||||
LOG_INFO("start destroy ddl kv manager", K(ls_id_), K(tablet_id_), K(start_scn_), K(head_), K(tail_), K(lbt()));
|
||||
}
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
ATOMIC_STORE(&ref_cnt_, 0);
|
||||
for (int64_t pos = head_; pos < tail_; ++pos) {
|
||||
const int64_t idx = get_idx(pos);
|
||||
@ -69,7 +68,6 @@ void ObTabletDDLKvMgr::destroy()
|
||||
execution_id_ = -1;
|
||||
success_start_scn_.set_min();
|
||||
is_inited_ = false;
|
||||
can_schedule_major_compaction_ = false;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::init(const share::ObLSID &ls_id, const common::ObTabletID &tablet_id)
|
||||
@ -94,7 +92,8 @@ int ObTabletDDLKvMgr::init(const share::ObLSID &ls_id, const common::ObTabletID
|
||||
// ddl start from checkpoint
|
||||
// keep ddl sstable table
|
||||
|
||||
int ObTabletDDLKvMgr::ddl_start(const ObITable::TableKey &table_key,
|
||||
int ObTabletDDLKvMgr::ddl_start(ObTablet &tablet,
|
||||
const ObITable::TableKey &table_key,
|
||||
const SCN &start_scn,
|
||||
const int64_t cluster_version,
|
||||
const int64_t execution_id,
|
||||
@ -115,7 +114,7 @@ int ObTabletDDLKvMgr::ddl_start(const ObITable::TableKey &table_key,
|
||||
ret = OB_ERR_SYS;
|
||||
LOG_WARN("tablet id not same", K(ret), K(table_key), K(tablet_id_));
|
||||
} else {
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (start_scn_.is_valid_and_not_min()) {
|
||||
if (execution_id >= execution_id_ && start_scn >= start_scn_) {
|
||||
LOG_INFO("execution id changed, need cleanup", K(ls_id_), K(tablet_id_), K(execution_id_), K(execution_id), K(start_scn_), K(start_scn));
|
||||
@ -169,6 +168,8 @@ int ObTabletDDLKvMgr::ddl_commit(const SCN &start_scn,
|
||||
} else if (start_scn < start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_INFO("skip ddl commit log", K(start_scn), K(*this));
|
||||
} else if (OB_FAIL(set_commit_scn(commit_scn))) {
|
||||
LOG_WARN("failed to set commit scn", K(ret));
|
||||
} else if (OB_FAIL(freeze_ddl_kv(commit_scn))) {
|
||||
LOG_WARN("freeze ddl kv failed", K(ret), K(commit_scn));
|
||||
} else {
|
||||
@ -183,7 +184,6 @@ int ObTabletDDLKvMgr::ddl_commit(const SCN &start_scn,
|
||||
}
|
||||
table_id_ = table_id;
|
||||
ddl_task_id_ = ddl_task_id;
|
||||
commit_scn_ = commit_scn;
|
||||
|
||||
ObDDLTableMergeDagParam param;
|
||||
param.ls_id_ = ls_id_;
|
||||
@ -318,6 +318,116 @@ int ObTabletDDLKvMgr::get_ddl_major_merge_param(ObDDLTableMergeDagParam ¶m)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::get_rec_scn(SCN &rec_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLSHandle ls_handle;
|
||||
ObTabletHandle tablet_handle;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret), K(is_inited_));
|
||||
} else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) {
|
||||
LOG_WARN("failed to get log stream", K(ret), K(ls_id_));
|
||||
} else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id_,
|
||||
tablet_handle,
|
||||
ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) {
|
||||
LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_));
|
||||
}
|
||||
|
||||
// rec scn of ddl start log
|
||||
if (OB_SUCC(ret)) {
|
||||
const ObTabletMeta &tablet_meta = tablet_handle.get_obj()->get_tablet_meta();
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (start_scn_.is_valid_and_not_min() && start_scn_ != tablet_meta.ddl_start_scn_) {
|
||||
// has a latest start log and not flushed to tablet meta, keep it
|
||||
rec_scn = SCN::min(rec_scn, start_scn_);
|
||||
}
|
||||
}
|
||||
|
||||
// rec scn of ddl commit log
|
||||
if (OB_SUCC(ret)) {
|
||||
const ObTabletMeta &tablet_meta = tablet_handle.get_obj()->get_tablet_meta();
|
||||
if (tablet_meta.ddl_commit_scn_.is_valid_and_not_min()) {
|
||||
// has commit log and already dumped to tablet meta, skip
|
||||
} else {
|
||||
const SCN commit_scn = get_commit_scn(tablet_meta);
|
||||
if (commit_scn.is_valid_and_not_min()) {
|
||||
// has commit log and not yet dumped to tablet meta
|
||||
rec_scn = SCN::min(rec_scn, commit_scn);
|
||||
} else {
|
||||
// no commit log
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// rec scn of ddl redo
|
||||
if (OB_SUCC(ret)) {
|
||||
bool has_ddl_kv = false;
|
||||
if (OB_FAIL(check_has_effective_ddl_kv(has_ddl_kv))) {
|
||||
LOG_WARN("failed to check ddl kv", K(ret));
|
||||
} else if (has_ddl_kv) {
|
||||
SCN min_scn;
|
||||
if (OB_FAIL(get_ddl_kv_min_scn(min_scn))) {
|
||||
LOG_WARN("fail to get ddl kv min log ts", K(ret));
|
||||
} else {
|
||||
rec_scn = SCN::min(rec_scn, min_scn);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::set_commit_scn(const SCN &commit_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLSHandle ls_handle;
|
||||
ObTabletHandle tablet_handle;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret), K(is_inited_));
|
||||
} else if (OB_UNLIKELY(commit_scn <= SCN::min_scn())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(commit_scn));
|
||||
} else if (OB_FAIL(MTL(ObLSService *)->get_ls(ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) {
|
||||
LOG_WARN("failed to get log stream", K(ret), K(ls_id_));
|
||||
} else if (OB_FAIL(ls_handle.get_ls()->get_tablet(tablet_id_,
|
||||
tablet_handle,
|
||||
ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) {
|
||||
LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_));
|
||||
} else {
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
const SCN old_commit_scn = get_commit_scn_nolock(tablet_handle.get_obj()->get_tablet_meta());
|
||||
if (old_commit_scn.is_valid_and_not_min() && old_commit_scn != commit_scn) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("already committed by others", K(ret), K(commit_scn), K(*this));
|
||||
} else {
|
||||
commit_scn_ = commit_scn;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
SCN ObTabletDDLKvMgr::get_commit_scn(const ObTabletMeta &tablet_meta)
|
||||
{
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
return get_commit_scn_nolock(tablet_meta);
|
||||
}
|
||||
|
||||
SCN ObTabletDDLKvMgr::get_commit_scn_nolock(const ObTabletMeta &tablet_meta)
|
||||
{
|
||||
SCN commit_scn = SCN::min_scn();
|
||||
if (tablet_meta.ddl_commit_scn_.is_valid_and_not_min() || commit_scn_.is_valid_and_not_min()) {
|
||||
if (tablet_meta.ddl_commit_scn_.is_valid_and_not_min()) {
|
||||
commit_scn = tablet_meta.ddl_commit_scn_;
|
||||
} else {
|
||||
commit_scn = commit_scn_;
|
||||
}
|
||||
} else {
|
||||
commit_scn = SCN::min_scn();
|
||||
}
|
||||
return commit_scn;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::set_commit_success(const SCN &start_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -328,7 +438,7 @@ int ObTabletDDLKvMgr::set_commit_success(const SCN &start_scn)
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(start_scn));
|
||||
} else {
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (start_scn < start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_WARN("ddl task expired", K(ret), K(start_scn), K(*this));
|
||||
@ -349,9 +459,9 @@ int ObTabletDDLKvMgr::set_commit_success(const SCN &start_scn)
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObTabletDDLKvMgr::is_commit_success() const
|
||||
bool ObTabletDDLKvMgr::is_commit_success()
|
||||
{
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
return is_commit_success_unlock();
|
||||
}
|
||||
|
||||
@ -360,10 +470,15 @@ bool ObTabletDDLKvMgr::is_commit_success_unlock() const
|
||||
return success_start_scn_ > SCN::min_scn() && success_start_scn_ == start_scn_;
|
||||
}
|
||||
|
||||
bool ObTabletDDLKvMgr::can_schedule_major_compaction() const
|
||||
bool ObTabletDDLKvMgr::can_schedule_major_compaction(const ObTabletMeta &tablet_meta)
|
||||
{
|
||||
TCRLockGuard guard(lock_);
|
||||
return can_schedule_major_compaction_ && !is_commit_success_unlock();
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
return can_schedule_major_compaction_nolock(tablet_meta);
|
||||
}
|
||||
|
||||
bool ObTabletDDLKvMgr::can_schedule_major_compaction_nolock(const ObTabletMeta &tablet_meta)
|
||||
{
|
||||
return get_commit_scn_nolock(tablet_meta).is_valid_and_not_min() && !is_commit_success_unlock();
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::cleanup()
|
||||
@ -373,7 +488,7 @@ int ObTabletDDLKvMgr::cleanup()
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret));
|
||||
} else {
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
cleanup_unlock();
|
||||
}
|
||||
return ret;
|
||||
@ -399,15 +514,40 @@ void ObTabletDDLKvMgr::cleanup_unlock()
|
||||
table_id_ = 0;
|
||||
execution_id_ = -1;
|
||||
success_start_scn_.set_min();
|
||||
can_schedule_major_compaction_ = false;
|
||||
}
|
||||
|
||||
bool ObTabletDDLKvMgr::is_execution_id_older(const int64_t execution_id)
|
||||
{
|
||||
TCRLockGuard guard(lock_);
|
||||
return execution_id < execution_id_;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::set_execution_id_nolock(const int64_t execution_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (execution_id < execution_id_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_WARN("ddl task expired", K(ret), K(execution_id), K(*this));
|
||||
} else {
|
||||
execution_id_ = execution_id;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::set_execution_id(const int64_t execution_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret), K(is_inited_));
|
||||
} else {
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (OB_FAIL(set_execution_id_nolock(execution_id))) {
|
||||
LOG_WARN("failed to set execution id", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::online()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -430,7 +570,8 @@ int ObTabletDDLKvMgr::online()
|
||||
table_key.version_range_.base_version_ = 0;
|
||||
table_key.version_range_.snapshot_version_ = tablet_meta.ddl_snapshot_version_;
|
||||
const SCN &start_scn = tablet_meta.ddl_start_scn_;
|
||||
if (OB_FAIL(ddl_start(table_key,
|
||||
if (OB_FAIL(ddl_start(*tablet_handle.get_obj(),
|
||||
table_key,
|
||||
start_scn,
|
||||
tablet_meta.ddl_cluster_version_,
|
||||
tablet_meta.ddl_execution_id_,
|
||||
@ -463,7 +604,7 @@ int ObTabletDDLKvMgr::register_to_tablet(const SCN &ddl_start_scn, ObDDLKvMgrHan
|
||||
ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) {
|
||||
LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_));
|
||||
} else {
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (ddl_start_scn < start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_INFO("ddl task expired", K(ret), K(ls_id_), K(tablet_id_), K(start_scn_), K(ddl_start_scn));
|
||||
@ -502,7 +643,7 @@ int ObTabletDDLKvMgr::unregister_from_tablet(const SCN &ddl_start_scn, ObDDLKvMg
|
||||
ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) {
|
||||
LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_));
|
||||
} else {
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (ddl_start_scn < start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_INFO("ddl task expired", K(ret), K(ls_id_), K(tablet_id_), K(start_scn_), K(ddl_start_scn));
|
||||
@ -523,6 +664,31 @@ int ObTabletDDLKvMgr::unregister_from_tablet(const SCN &ddl_start_scn, ObDDLKvMg
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::rdlock(const int64_t timeout_us, uint32_t &tid)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCC(lock_.rdlock(ObLatchIds::TABLET_DDL_KV_MGR_LOCK, timeout_us))) {
|
||||
tid = static_cast<uint32_t>(GETTID());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::wrlock(const int64_t timeout_us, uint32_t &tid)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCC(lock_.wrlock(ObLatchIds::TABLET_DDL_KV_MGR_LOCK, timeout_us))) {
|
||||
tid = static_cast<uint32_t>(GETTID());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTabletDDLKvMgr::unlock(const uint32_t tid)
|
||||
{
|
||||
if (OB_SUCCESS != lock_.unlock(&tid)) {
|
||||
ob_abort();
|
||||
}
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::update_tablet(const SCN &start_scn, const int64_t snapshot_version, const SCN &ddl_checkpoint_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -620,16 +786,13 @@ int ObTabletDDLKvMgr::update_ddl_major_sstable()
|
||||
ObTabletCommon::NO_CHECK_GET_TABLET_TIMEOUT_US))) {
|
||||
LOG_WARN("get tablet handle failed", K(ret), K(ls_id_), K(tablet_id_));
|
||||
} else {
|
||||
{
|
||||
TCWLockGuard guard(lock_);
|
||||
can_schedule_major_compaction_ = true;
|
||||
}
|
||||
ObTabletHandle new_tablet_handle;
|
||||
ObUpdateTableStoreParam param(tablet_handle.get_obj()->get_snapshot_version(),
|
||||
ObVersionRange::MIN_VERSION, // multi_version_start
|
||||
&tablet_handle.get_obj()->get_storage_schema(),
|
||||
ls_handle.get_ls()->get_rebuild_seq());
|
||||
param.ddl_info_.keep_old_ddl_sstable_ = true;
|
||||
param.ddl_info_.ddl_commit_scn_ = commit_scn_;
|
||||
if (OB_FAIL(ls_handle.get_ls()->update_tablet_table_store(tablet_id_, param, new_tablet_handle))) {
|
||||
LOG_WARN("failed to update tablet table store", K(ret), K(ls_id_), K(tablet_id_), K(param));
|
||||
}
|
||||
@ -647,10 +810,12 @@ int ObTabletDDLKvMgr::get_ddl_param(ObTabletDDLParam &ddl_param)
|
||||
ret = OB_STATE_NOT_MATCH;
|
||||
LOG_WARN("ddl not started", K(ret));
|
||||
} else {
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
ddl_param.tenant_id_ = MTL_ID();
|
||||
ddl_param.ls_id_ = ls_id_;
|
||||
ddl_param.table_key_ = table_key_;
|
||||
ddl_param.start_scn_ = start_scn_;
|
||||
ddl_param.commit_scn_ = commit_scn_;
|
||||
ddl_param.snapshot_version_ = table_key_.get_snapshot_version();
|
||||
ddl_param.cluster_version_ = cluster_version_;
|
||||
}
|
||||
@ -667,7 +832,7 @@ int ObTabletDDLKvMgr::get_freezed_ddl_kv(const SCN &freeze_scn, ObTableHandleV2
|
||||
LOG_WARN("not init", K(ret), K(is_inited_));
|
||||
} else {
|
||||
bool found = false;
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
for (int64_t i = head_; OB_SUCC(ret) && !found && i < tail_; ++i) {
|
||||
const int64_t idx = get_idx(i);
|
||||
ObTableHandleV2 &cur_kv_handle = ddl_kv_handles_[idx];
|
||||
@ -720,7 +885,7 @@ int ObTabletDDLKvMgr::get_active_ddl_kv_impl(ObTableHandleV2 &kv_handle)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::get_or_create_ddl_kv(const SCN &scn, ObTableHandleV2 &kv_handle)
|
||||
int ObTabletDDLKvMgr::get_or_create_ddl_kv(const SCN &start_scn, const SCN &scn, ObTableHandleV2 &kv_handle)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
kv_handle.reset();
|
||||
@ -731,17 +896,26 @@ int ObTabletDDLKvMgr::get_or_create_ddl_kv(const SCN &scn, ObTableHandleV2 &kv_h
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(scn));
|
||||
} else {
|
||||
ObTableHandleV2 tmp_kv_handle;
|
||||
TCRLockGuard guard(lock_);
|
||||
try_get_ddl_kv_unlock(scn, kv_handle);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (start_scn != start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_WARN("ddl task expired", K(ret), K(start_scn), KPC(this));
|
||||
} else {
|
||||
try_get_ddl_kv_unlock(scn, kv_handle);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && !kv_handle.is_valid()) {
|
||||
TCWLockGuard guard(lock_);
|
||||
try_get_ddl_kv_unlock(scn, kv_handle);
|
||||
if (kv_handle.is_valid()) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(alloc_ddl_kv(kv_handle))) {
|
||||
LOG_WARN("create ddl kv failed", K(ret));
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (start_scn != start_scn_) {
|
||||
ret = OB_TASK_EXPIRED;
|
||||
LOG_WARN("ddl task expired", K(ret), K(start_scn), KPC(this));
|
||||
} else {
|
||||
try_get_ddl_kv_unlock(scn, kv_handle);
|
||||
if (kv_handle.is_valid()) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(alloc_ddl_kv(kv_handle))) {
|
||||
LOG_WARN("create ddl kv failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -770,7 +944,7 @@ int ObTabletDDLKvMgr::freeze_ddl_kv(const SCN &freeze_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTableHandleV2 kv_handle;
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
@ -809,7 +983,7 @@ int ObTabletDDLKvMgr::release_ddl_kvs(const SCN &end_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
DEBUG_SYNC(BEFORE_RELEASE_DDL_KV);
|
||||
TCWLockGuard guard(lock_);
|
||||
ObLatchWGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
@ -844,7 +1018,7 @@ int ObTabletDDLKvMgr::release_ddl_kvs(const SCN &end_scn)
|
||||
int ObTabletDDLKvMgr::get_ddl_kv_min_scn(SCN &min_scn)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
min_scn = SCN::max_scn();
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
@ -868,6 +1042,7 @@ int ObTabletDDLKvMgr::get_ddl_kvs_unlock(const bool frozen_only, ObTablesHandleA
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
kv_handle_array.reset();
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
@ -893,7 +1068,7 @@ int ObTabletDDLKvMgr::get_ddl_kvs(const bool frozen_only, ObTablesHandleArray &k
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
kv_handle_array.reset();
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
@ -903,15 +1078,15 @@ int ObTabletDDLKvMgr::get_ddl_kvs(const bool frozen_only, ObTablesHandleArray &k
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTabletDDLKvMgr::get_ddl_kvs_for_query(ObTablesHandleArray &kv_handle_array)
|
||||
int ObTabletDDLKvMgr::get_ddl_kvs_for_query(ObTablet &tablet, ObTablesHandleArray &kv_handle_array)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
kv_handle_array.reset();
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
} else if (!can_schedule_major_compaction_) {
|
||||
} else if (!can_schedule_major_compaction_nolock(tablet.get_tablet_meta())) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(get_ddl_kvs_unlock(true/*frozen_only*/, kv_handle_array))) {
|
||||
LOG_WARN("get ddl kv unlock failed", K(ret));
|
||||
@ -922,7 +1097,7 @@ int ObTabletDDLKvMgr::get_ddl_kvs_for_query(ObTablesHandleArray &kv_handle_array
|
||||
int ObTabletDDLKvMgr::check_has_effective_ddl_kv(bool &has_ddl_kv)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
TCRLockGuard guard(lock_);
|
||||
ObLatchRGuard guard(lock_, ObLatchIds::TABLET_DDL_KV_MGR_LOCK);
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTabletDDLKvMgr is not inited", K(ret));
|
||||
|
||||
Reference in New Issue
Block a user