[BUG] fix memtable mgr op deadlock

This commit is contained in:
Handora
2024-02-08 10:58:15 +00:00
committed by ob-robot
parent 598ab3753c
commit 6a9f9781b0
5 changed files with 37 additions and 80 deletions

View File

@ -105,7 +105,7 @@ ObMemtable::ObMemtable()
is_inited_(false),
ls_handle_(),
freezer_(nullptr),
memtable_mgr_(nullptr),
memtable_mgr_handle_(),
freeze_clock_(0),
local_allocator_(*this),
query_engine_(local_allocator_),
@ -114,7 +114,6 @@ ObMemtable::ObMemtable()
max_data_schema_version_(0),
pending_cb_cnt_(0),
unsubmitted_cnt_(0),
memtable_mgr_op_cnt_(0),
logging_blocked_(false),
logging_blocked_start_time(0),
unset_active_memtable_logging_blocked_(false),
@ -168,7 +167,8 @@ int ObMemtable::init(const ObITable::TableKey &table_key,
ret = OB_INVALID_ARGUMENT;
TRANS_LOG(WARN, "invalid param", K(ret), K(table_key), KP(freezer), KP(memtable_mgr),
K(schema_version), K(freeze_clock), K(ls_handle));
} else if (FALSE_IT(set_memtable_mgr(memtable_mgr))) {
} else if (OB_FAIL(set_memtable_mgr_(memtable_mgr))) {
TRANS_LOG(WARN, "fail to set memtable mgr", K(ret), KP(memtable_mgr));
} else if (FALSE_IT(set_freeze_clock(freeze_clock))) {
} else if (FALSE_IT(set_max_schema_version(schema_version))) {
} else if (OB_FAIL(set_freezer(freezer))) {
@ -254,7 +254,7 @@ void ObMemtable::destroy()
time_guard.click();
ls_handle_.reset();
freezer_ = nullptr;
memtable_mgr_ = nullptr;
memtable_mgr_handle_.reset();
freeze_clock_ = 0;
max_schema_version_ = 0;
max_data_schema_version_ = 0;
@ -263,7 +263,6 @@ void ObMemtable::destroy()
state_ = ObMemtableState::INVALID;
freeze_state_ = ObMemtableFreezeState::INVALID;
unsubmitted_cnt_ = 0;
memtable_mgr_op_cnt_ = 0;
logging_blocked_ = false;
logging_blocked_start_time = 0;
unset_active_memtable_logging_blocked_ = false;
@ -1734,8 +1733,7 @@ int64_t ObMemtable::dec_write_ref()
void ObMemtable::unset_logging_blocked_for_active_memtable()
{
int ret = OB_SUCCESS;
MemtableMgrOpGuard memtable_mgr_op_guard(this);
storage::ObTabletMemtableMgr *memtable_mgr = memtable_mgr_op_guard.get_memtable_mgr();
storage::ObTabletMemtableMgr *memtable_mgr = get_memtable_mgr_();
if (OB_NOT_NULL(memtable_mgr)) {
do {
@ -1750,8 +1748,7 @@ void ObMemtable::unset_logging_blocked_for_active_memtable()
void ObMemtable::resolve_left_boundary_for_active_memtable()
{
int ret = OB_SUCCESS;
MemtableMgrOpGuard memtable_mgr_op_guard(this);
storage::ObTabletMemtableMgr *memtable_mgr = memtable_mgr_op_guard.get_memtable_mgr();
storage::ObTabletMemtableMgr *memtable_mgr = get_memtable_mgr_();
const SCN new_start_scn = get_end_scn();
if (OB_NOT_NULL(memtable_mgr)) {
@ -2075,8 +2072,7 @@ bool ObMemtable::ready_for_flush_()
// ensure unset all frozen memtables'logging_block
ObTableHandleV2 handle;
ObMemtable *first_frozen_memtable = nullptr;
MemtableMgrOpGuard memtable_mgr_op_guard(this);
storage::ObTabletMemtableMgr *memtable_mgr = memtable_mgr_op_guard.get_memtable_mgr();
storage::ObTabletMemtableMgr *memtable_mgr = get_memtable_mgr_();
if (OB_ISNULL(memtable_mgr)) {
} else if (OB_FAIL(memtable_mgr->get_first_frozen_memtable(handle))) {
TRANS_LOG(WARN, "fail to get first_frozen_memtable", K(ret));
@ -3288,5 +3284,16 @@ int ObMemtable::get_tx_table_guard(ObTxTableGuard &tx_table_guard)
return ret;
}
int ObMemtable::set_memtable_mgr_(storage::ObTabletMemtableMgr *mgr)
{
ObTabletMemtableMgrPool *pool = MTL(ObTabletMemtableMgrPool*);
return memtable_mgr_handle_.set_memtable_mgr(mgr, pool);
}
storage::ObTabletMemtableMgr *ObMemtable::get_memtable_mgr_()
{
return static_cast<ObTabletMemtableMgr *>(memtable_mgr_handle_.get_memtable_mgr());
}
} // namespace memtable
} // namespace ocenabase

View File

@ -25,6 +25,7 @@
#include "storage/memtable/ob_memtable_key.h"
#include "storage/memtable/ob_row_compactor.h"
#include "storage/memtable/ob_multi_source_data.h"
#include "storage/ob_i_memtable_mgr.h"
#include "storage/checkpoint/ob_freeze_checkpoint.h"
#include "storage/compaction/ob_medium_compaction_mgr.h"
#include "storage/tx_storage/ob_ls_handle.h" //ObLSHandle
@ -331,9 +332,8 @@ public:
int set_freezer(storage::ObFreezer *handler);
storage::ObFreezer *get_freezer() { return freezer_; }
int get_ls_id(share::ObLSID &ls_id);
void set_memtable_mgr(storage::ObTabletMemtableMgr *mgr) { ATOMIC_STORE(&memtable_mgr_, mgr); }
void clear_memtable_mgr() { ATOMIC_STORE(&memtable_mgr_, nullptr); }
storage::ObTabletMemtableMgr *get_memtable_mgr() { return ATOMIC_LOAD(&memtable_mgr_); }
int set_memtable_mgr_(storage::ObTabletMemtableMgr *mgr);
storage::ObTabletMemtableMgr *get_memtable_mgr_();
void set_freeze_clock(const uint32_t freeze_clock) { ATOMIC_STORE(&freeze_clock_, freeze_clock); }
uint32_t get_freeze_clock() const { return ATOMIC_LOAD(&freeze_clock_); }
int set_emergency(const bool emergency);
@ -405,9 +405,6 @@ public:
ObMvccEngine &get_mvcc_engine() { return mvcc_engine_; }
const ObMvccEngine &get_mvcc_engine() const { return mvcc_engine_; }
OB_INLINE bool is_inited() const { return is_inited_;}
int64_t get_memtable_mgr_op_cnt() { return ATOMIC_LOAD(&memtable_mgr_op_cnt_); }
int64_t inc_memtable_mgr_op_cnt() { return ATOMIC_AAF(&memtable_mgr_op_cnt_, 1); }
int64_t dec_memtable_mgr_op_cnt() { return ATOMIC_SAF(&memtable_mgr_op_cnt_, 1); }
void pre_batch_destroy_keybtree();
static int batch_remove_unused_callback_for_uncommited_txn(
const share::ObLSID ls_id,
@ -505,7 +502,7 @@ public:
int dump2text(const char *fname);
// TODO(handora.qc) ready_for_flush interface adjustment
bool is_can_flush() { return ObMemtableFreezeState::READY_FOR_FLUSH == freeze_state_ && share::SCN::max_scn() != get_end_scn(); }
INHERIT_TO_STRING_KV("ObITable", ObITable, KP(this), KP_(memtable_mgr), K_(timestamp), K_(state),
INHERIT_TO_STRING_KV("ObITable", ObITable, KP(this), K_(timestamp), K_(state),
K_(freeze_clock), K_(max_schema_version), K_(max_data_schema_version), K_(max_column_cnt),
K_(write_ref_cnt), K_(local_allocator), K_(unsubmitted_cnt),
K_(logging_blocked), K_(unset_active_memtable_logging_blocked), K_(resolved_active_memtable_left_boundary),
@ -627,7 +624,7 @@ private:
bool is_inited_;
storage::ObLSHandle ls_handle_;
storage::ObFreezer *freezer_;
storage::ObTabletMemtableMgr *memtable_mgr_;
storage::ObMemtableMgrHandle memtable_mgr_handle_;
mutable uint32_t freeze_clock_;
ObSingleMemstoreAllocator local_allocator_;
ObMTKVBuilder kv_builder_;
@ -638,7 +635,6 @@ private:
int64_t max_data_schema_version_; // to record the max schema version of write data
int64_t pending_cb_cnt_; // number of transactions have to sync log
int64_t unsubmitted_cnt_; // number of trans node to be submitted logs
int64_t memtable_mgr_op_cnt_; // number of operations for memtable_mgr
bool logging_blocked_; // flag whether the memtable can submit log, cannot submit if true
int64_t logging_blocked_start_time; // record the start time of logging blocked
bool unset_active_memtable_logging_blocked_;
@ -681,31 +677,6 @@ private:
uint32_t modify_count_;
uint32_t acc_checksum_;
};
class MemtableMgrOpGuard
{
public:
explicit MemtableMgrOpGuard(ObMemtable *memtable): memtable_(memtable),
memtable_mgr_(nullptr)
{
if (OB_NOT_NULL(memtable_)) {
memtable_->inc_memtable_mgr_op_cnt();
memtable_mgr_ = memtable_->get_memtable_mgr();
}
}
~MemtableMgrOpGuard()
{
if (OB_NOT_NULL(memtable_)) {
memtable_->dec_memtable_mgr_op_cnt();
memtable_mgr_ = nullptr;
}
}
storage::ObTabletMemtableMgr *get_memtable_mgr() { return memtable_mgr_; }
private:
ObMemtable *memtable_;
storage::ObTabletMemtableMgr *memtable_mgr_;
};
}
}

View File

@ -67,7 +67,6 @@ void ObTabletMemtableMgr::destroy()
STORAGE_LOG(WARN, "memtable is nullptr", K(ret), KP(imemtable), K(pos));
} else if (imemtable->is_data_memtable()) {
memtable::ObMemtable *memtable = static_cast<memtable::ObMemtable *>(imemtable);
unlink_memtable_mgr_and_memtable_(memtable);
memtable->remove_from_data_checkpoint();
memtable->set_frozen();
}
@ -694,7 +693,6 @@ int ObTabletMemtableMgr::release_head_memtable_(memtable::ObIMemtable *imemtable
if (!memtable->is_empty()) {
memtable->set_read_barrier();
}
unlink_memtable_mgr_and_memtable_(memtable);
memtable->remove_from_data_checkpoint();
memtable->set_is_flushed();
memtable->set_freeze_state(ObMemtableFreezeState::RELEASED);
@ -712,36 +710,6 @@ int ObTabletMemtableMgr::release_head_memtable_(memtable::ObIMemtable *imemtable
return ret;
}
void ObTabletMemtableMgr::unlink_memtable_mgr_and_memtable_(memtable::ObMemtable *memtable)
{
// unlink memtable_mgr and memtable
// and wait the running ops about memtable_mgr in the memtable
int ret = OB_SUCCESS;
if (OB_ISNULL(memtable)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("memtable is null", K(ret), KPC(this));
} else {
memtable->clear_memtable_mgr();
wait_memtable_mgr_op_cnt_(memtable);
}
}
void ObTabletMemtableMgr::wait_memtable_mgr_op_cnt_(memtable::ObMemtable *memtable)
{
if (OB_NOT_NULL(memtable)) {
const int64_t start = ObTimeUtility::current_time();
int ret = OB_SUCCESS;
while (0 != memtable->get_memtable_mgr_op_cnt()) {
const int64_t cost_time = ObTimeUtility::current_time() - start;
if (cost_time > 1000 * 1000) {
if (TC_REACH_TIME_INTERVAL(1000 * 1000)) {
LOG_WARN("wait_memtable_mgr_op_cnt costs too much time", KPC(memtable));
}
}
}
}
}
int ObTabletMemtableMgr::get_first_frozen_memtable(ObTableHandleV2 &handle) const
{
int ret = OB_SUCCESS;
@ -815,7 +783,6 @@ void ObTabletMemtableMgr::clean_tail_memtable_()
if (memtable_tail_ > memtable_head_) {
ObMemtable *memtable = get_memtable_(memtable_tail_ - 1);
if (OB_NOT_NULL(memtable)) {
unlink_memtable_mgr_and_memtable_(memtable);
memtable->set_frozen();
} else {
LOG_WARN_RET(OB_ERR_UNEXPECTED, "memtable is null when clean_tail_memtable_", KPC(this));

View File

@ -132,8 +132,6 @@ private:
share::SCN start_scn,
share::SCN snapshot_version);
int unset_logging_blocked_for_active_memtable(memtable::ObIMemtable *memtable);
void unlink_memtable_mgr_and_memtable_(memtable::ObMemtable *memtable);
void wait_memtable_mgr_op_cnt_(memtable::ObMemtable *memtable);
DISALLOW_COPY_AND_ASSIGN(ObTabletMemtableMgr);

View File

@ -81,6 +81,9 @@ TEST_F(TestTabletMemtableMgr, tablet_memtable_mgr) {
ASSERT_EQ(OB_SUCCESS, tablet_handle.get_obj()->create_memtable(1, scn1));
ASSERT_EQ(OB_SUCCESS, tablet_handle.get_obj()->create_memtable(2, scn2));
ObSEArray<ObTableHandleV2, 64> handles;
ASSERT_EQ(OB_SUCCESS, tablet_handle.get_obj()->get_all_memtables(handles));
// memtable_mgr exist
ASSERT_EQ(OB_SUCCESS, protected_handle->get_active_memtable(handle));
ASSERT_EQ(1, pool->count_);
@ -97,6 +100,17 @@ TEST_F(TestTabletMemtableMgr, tablet_memtable_mgr) {
// memtable_mgr not exist
ASSERT_EQ(OB_ENTRY_NOT_EXIST, protected_handle->get_active_memtable(handle));
// memtable mgr is not destroyed so memtable also hold an reference
ASSERT_EQ(1, pool->count_);
// remove memtable mgr reference from memtable
ASSERT_EQ(1, handles.count());
for (int i = 0; i < handles.count(); i++) {
memtable::ObIMemtable *i_mt = nullptr;
EXPECT_EQ(OB_SUCCESS, handles[i].get_memtable(i_mt));
memtable::ObMemtable *mt = (memtable::ObMemtable *)(i_mt);
mt->memtable_mgr_handle_.reset();
}
ASSERT_EQ(0, pool->count_);
ASSERT_EQ(OB_SUCCESS, MTL(ObLSService*)->remove_ls(ls_id));