[CP] [BUG.FIX] fix unexpected wash tablet from map

This commit is contained in:
Tyshawn 2024-02-10 11:09:40 +00:00 committed by ob-robot
parent a680a564b8
commit a0b5fa8b48
7 changed files with 181 additions and 118 deletions

View File

@ -117,7 +117,7 @@ void TestLSTabletService::SetUp()
ls_tablet_service_ = ls->get_tablet_svr();
while (true) {
if (nullptr != MTL(ObTenantMetaMemMgr*)->gc_head_) {
if (!MTL(ObTenantMetaMemMgr*)->tablet_gc_queue_.is_empty()) {
LOG_INFO("wait t3m gc tablet clean");
usleep(300 * 1000); // wait 300ms
} else {
@ -703,7 +703,7 @@ TEST_F(TestLSTabletService, test_replay_empty_shell)
// gc empty tablet
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr *);
ret = t3m->gc_tablet(empty_tablet);
ret = t3m->push_tablet_into_gc_queue(empty_tablet);
ASSERT_EQ(OB_SUCCESS, ret);
bool cleared = false;
ret = t3m->gc_tablets_in_queue(cleared);

View File

@ -1659,6 +1659,41 @@ TEST_F(TestTenantMetaMemMgr, test_leak_checker_switch)
t3m->is_tablet_leak_checker_enabled_ = before_enabled;
}
TEST_F(TestTenantMetaMemMgr, test_tablet_gc_queue)
{
ObTenantMetaMemMgr::TabletGCQueue gc_queue;
ASSERT_EQ(0, gc_queue.count());
ASSERT_EQ(nullptr, gc_queue.gc_head_);
ASSERT_EQ(nullptr, gc_queue.gc_tail_);
ObTablet *tablet = new ObTablet();
ASSERT_EQ(OB_SUCCESS, gc_queue.push(tablet));
ASSERT_EQ(1, gc_queue.count());
ASSERT_EQ(tablet, gc_queue.pop());
ASSERT_TRUE(gc_queue.is_empty());
delete tablet;
ASSERT_EQ(OB_INVALID_ARGUMENT, gc_queue.push(nullptr));
ASSERT_TRUE(gc_queue.is_empty());
const int64_t tablet_cnt = 1000;
for (int i = 0; i < tablet_cnt; ++i) {
tablet = new ObTablet();
ASSERT_EQ(OB_SUCCESS, gc_queue.push(tablet));
ASSERT_EQ(i + 1, gc_queue.count());
ASSERT_EQ(OB_INVALID_ARGUMENT, gc_queue.push(nullptr));
ASSERT_EQ(i + 1, gc_queue.count());
}
ASSERT_EQ(tablet_cnt, gc_queue.count());
for (int j = 0; j < tablet_cnt; ++j) {
ASSERT_NE(nullptr, tablet = gc_queue.pop());
ASSERT_EQ(tablet_cnt - 1 - j, gc_queue.count());
delete tablet;
}
ASSERT_TRUE(gc_queue.is_empty());
}
} // end namespace storage
} // end namespace oceanbase

View File

@ -113,7 +113,7 @@ void ObTabletHandle::reset()
obj_->dec_macro_ref_cnt();
obj_->~ObTablet();
allocator_->free(obj_);
} else if (OB_FAIL(t3m_->gc_tablet(obj_))) {
} else if (OB_FAIL(t3m_->push_tablet_into_gc_queue(obj_))) {
STORAGE_LOG(ERROR, "fail to gc tablet", K(ret), KPC_(obj), K_(obj_pool), K_(allocator));
}
}

View File

@ -653,7 +653,7 @@ int ObTabletPointer::release_obj(ObTablet *&t)
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "object pool or allocator is nullptr", K(ret), K(obj_));
} else if (nullptr == t->get_allocator()) {
obj_.t3m_->release_tablet(t);
obj_.t3m_->release_tablet_from_pool(t, true/*give_back_tablet_into_pool*/);
t = nullptr;
} else {
t->~ObTablet();

View File

@ -116,6 +116,59 @@ void ObTenantMetaMemMgr::RefreshConfigTask::runTimerTask()
}
}
ObTablet *ObTenantMetaMemMgr::TabletGCQueue::pop()
{
SpinWLockGuard lock(queue_lock_);
ObTablet *tablet = gc_head_;
const int64_t tablet_cnt = count();
OB_ASSERT(tablet_cnt >= 0);
if (0 == tablet_cnt) {
OB_ASSERT(nullptr == gc_head_);
OB_ASSERT(nullptr == gc_tail_);
} else if (tablet_cnt > 1) {
OB_ASSERT(nullptr != gc_head_);
OB_ASSERT(nullptr != gc_tail_);
OB_ASSERT(gc_head_ != gc_tail_);
gc_head_ = gc_head_->get_next_tablet();
} else if (tablet_cnt == 1) {
OB_ASSERT(gc_head_ == gc_tail_);
OB_ASSERT(gc_head_ != nullptr);
gc_head_ = gc_tail_ = nullptr;
}
if (OB_NOT_NULL(tablet)) {
tablet->set_next_tablet(nullptr);
ATOMIC_DEC(&tablet_count_);
}
return tablet;
}
int ObTenantMetaMemMgr::TabletGCQueue::push(ObTablet *tablet)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(tablet)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablet is nullptr", K(ret), KP(tablet));
} else {
SpinWLockGuard lock(queue_lock_);
const int64_t tablet_cnt = count();
OB_ASSERT(tablet_cnt >= 0);
tablet->set_next_tablet(nullptr);
if (0 == tablet_cnt) {
OB_ASSERT(nullptr == gc_head_);
OB_ASSERT(nullptr == gc_tail_);
gc_head_ = gc_tail_ = tablet;
} else if (tablet_cnt >= 1) {
OB_ASSERT(nullptr != gc_head_);
OB_ASSERT(nullptr != gc_tail_);
OB_ASSERT(1 == tablet_cnt ? gc_head_ == gc_tail_ : gc_head_ != gc_tail_);
gc_tail_->set_next_tablet(tablet);
gc_tail_ = tablet;
}
ATOMIC_INC(&tablet_count_);
}
return ret;
}
ObTenantMetaMemMgr::ObTenantMetaMemMgr(const uint64_t tenant_id)
: wash_lock_(common::ObLatchIds::TENANT_META_MEM_MGR_LOCK),
wash_func_(*this),
@ -127,8 +180,7 @@ ObTenantMetaMemMgr::ObTenantMetaMemMgr(const uint64_t tenant_id)
table_gc_task_(this),
refresh_config_task_(),
tablet_gc_task_(this),
gc_head_(nullptr),
wait_gc_tablets_cnt_(0),
tablet_gc_queue_(),
free_tables_queue_(),
gc_queue_lock_(common::ObLatchIds::TENANT_META_MEM_MGR_LOCK),
memtable_pool_(tenant_id, get_default_memtable_pool_count(), "MemTblObj", ObCtxIds::DEFAULT_CTX_ID),
@ -501,13 +553,13 @@ int ObTenantMetaMemMgr::gc_tables_in_queue(bool &all_table_cleaned)
}
if (recycled_cnt > 0) {
FLOG_INFO("Successfully finish table gc", K(sstable_cnt), K(data_memtable_cnt),
FLOG_INFO("gc tables in queue", K(sstable_cnt), K(data_memtable_cnt),
K(tx_data_memtable_cnt), K(tx_ctx_memtable_cnt), K(lock_memtable_cnt), K(pending_cnt), K(recycled_cnt),
K(tablet_buffer_pool_), K(large_tablet_buffer_pool_), K(full_tablet_creator_), K(tablets_mem), K(tablets_mem_limit),
K(ddl_kv_pool_), K(memtable_pool_), "wait_gc_count", free_tables_queue_.size(),
"tablet count", tablet_map_.count());
} else if (REACH_COUNT_INTERVAL(100)) {
FLOG_INFO("Recycle 0 table", K(ret),
FLOG_INFO("gc tables in queue: recycle 0 table", K(ret),
K(tablet_buffer_pool_), K(large_tablet_buffer_pool_), K(full_tablet_creator_), K(tablets_mem), K(tablets_mem_limit),
K(ddl_kv_pool_), K(memtable_pool_), K(pending_cnt), "wait_gc_count", free_tables_queue_.size(),
"tablet count", tablet_map_.count());
@ -666,41 +718,23 @@ int ObTenantMetaMemMgr::push_memtable_into_gc_map_(memtable::ObMemtable *memtabl
return ret;
}
int ObTenantMetaMemMgr::push_tablet_into_gc_queue(ObTablet *tablet)
int ObTenantMetaMemMgr::inner_push_tablet_into_gc_queue(ObTablet *tablet)
{
int ret = OB_SUCCESS;
ObTabletHandle empty_handle;
if (OB_UNLIKELY(nullptr == tablet)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("fail to push invalid tablet into gc queue", K(ret), KP(tablet));
} else if (FALSE_IT(tablet->set_next_tablet_guard(empty_handle))) { // release the ref_cnt of next_tablet_guard_
} else if (OB_FAIL(tablet_gc_queue_.push(tablet))) {
LOG_WARN("fail to push tablet into gc queue", K(ret), KP(tablet));
} else {
ObTabletHandle empty_handle;
tablet->set_next_tablet_guard(empty_handle); // release the ref_cnt of next_tablet_guard_
do {
tablet->set_next_tablet(ATOMIC_LOAD(&gc_head_));
} while (tablet->get_next_tablet() != ATOMIC_VCAS(&gc_head_, tablet->get_next_tablet(), tablet));
ATOMIC_INC(&wait_gc_tablets_cnt_);
// can't do anything to tablet as long as tablet has been pushed into gc queue.
LOG_DEBUG("push tablet into gc queue", K(ret), KP(tablet));
LOG_DEBUG("inner push tablet into gc queue", K(ret), KP(tablet));
}
return ret;
}
void ObTenantMetaMemMgr::push_tablet_list_into_gc_queue(ObTablet *tablet)
{
ObTablet *head = tablet;
ObTablet *tail = head;
while (OB_NOT_NULL(tail) && OB_NOT_NULL(tail->get_next_tablet())) {
tail = tail->get_next_tablet();
}
if (OB_NOT_NULL(tail)) {
do {
tail->set_next_tablet(ATOMIC_LOAD(&gc_head_));
} while (tail->get_next_tablet() != ATOMIC_VCAS(&gc_head_, tail->get_next_tablet(), head));
// no need to update wait_gc_tablets_cnt_
}
}
int ObTenantMetaMemMgr::gc_tablet(ObTablet *tablet)
int ObTenantMetaMemMgr::push_tablet_into_gc_queue(ObTablet *tablet)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
@ -709,8 +743,11 @@ int ObTenantMetaMemMgr::gc_tablet(ObTablet *tablet)
} else if (OB_UNLIKELY(nullptr == tablet || tablet->get_ref() != 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("push invalid tablet into gc queue", K(ret), KPC(tablet));
} else if (!tablet->is_valid()) { // push invalid tablet into gc queue directly
LOG_INFO("gc invalid tablet", KP(tablet));
} else if (!tablet->is_valid()) {
// If a tablet is invalid, it is not expensive to recycle it. Recycle it directly, to avoid asynchronous
// tablet gc queue backlog。
LOG_INFO("release an invalid tablet", KP(tablet));
release_tablet(tablet, false/*return tablet buffer ptr after release*/);
} else {
const ObTabletMeta &tablet_meta = tablet->get_tablet_meta();
const ObTabletMapKey key(tablet_meta.ls_id_, tablet_meta.tablet_id_);
@ -723,36 +760,17 @@ int ObTenantMetaMemMgr::gc_tablet(ObTablet *tablet)
LOG_WARN("unexpected null tablet pointer", K(ret), K(key), K(ptr_handle));
} else if (OB_FAIL(tablet_ptr->remove_tablet_from_old_version_chain(tablet))) {
LOG_WARN("fail to remove tablet from old version chain", K(ret), K(key), KPC(tablet));
} else {
tablet->reset_memtable();
} else if (FALSE_IT(tablet->reset_memtable())) {
} else if (OB_FAIL(inner_push_tablet_into_gc_queue(tablet))) {
LOG_WARN("fail to push tablet into gc queue", K(ret), KPC(tablet));
}
}
if (OB_SUCC(ret) && OB_FAIL(push_tablet_into_gc_queue(tablet))) {
LOG_WARN("fail to push tablet into gc queue", K(ret), KPC(tablet));
}
#ifndef OB_BUILD_RPM
FLOG_INFO("push tablet into gc queue", K(ret), KP(tablet), K(common::lbt()));
#endif
return ret;
}
void ObTenantMetaMemMgr::destroy_gc_tablets_queue()
{
// only release memory of tablets
while (OB_NOT_NULL(gc_head_)) {
ObTablet *tablet = gc_head_;
gc_head_ = gc_head_->get_next_tablet();
ObIAllocator *allocator = tablet->get_allocator();
if (OB_ISNULL(allocator)) {
tablet_buffer_pool_.free_obj(tablet);
} else {
tablet->~ObTablet();
allocator->~ObIAllocator();
}
}
wait_gc_tablets_cnt_ = 0;
}
int ObTenantMetaMemMgr::gc_tablets_in_queue(bool &all_tablet_cleaned)
{
int ret = OB_SUCCESS;
@ -765,24 +783,20 @@ int ObTenantMetaMemMgr::gc_tablets_in_queue(bool &all_tablet_cleaned)
int64_t err_tablets_cnt = 0;
int64_t left_recycle_cnt = ONE_ROUND_TABLET_GC_COUNT_THRESHOLD;
ObTablet *tablet = nullptr;
do {
tablet = ATOMIC_LOAD(&gc_head_);
} while (tablet != ATOMIC_VCAS(&gc_head_, tablet, nullptr));
while(OB_SUCC(ret) && OB_NOT_NULL(tablet) && left_recycle_cnt > 0) {
ObTablet *next_tablet = static_cast<ObTablet *>(tablet->get_next_tablet());
ATOMIC_DEC(&wait_gc_tablets_cnt_);
--left_recycle_cnt;
if (OB_UNLIKELY(tablet->get_ref() != 0)) {
while(OB_SUCC(ret) && left_recycle_cnt > 0) {
if (OB_ISNULL(tablet = tablet_gc_queue_.pop())) {
break; // tablet gc queue is empty.
} else if (OB_UNLIKELY(tablet->get_ref() != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected tablet in gc queue", K(ret), KPC(tablet));
} else {
recycle_tablet(tablet);
release_tablet(tablet, false/* return tablet buffer ptr after release*/);
}
if (OB_FAIL(ret)) {
++err_tablets_cnt;
FLOG_INFO("fail to gc tablet", K(ret), KP(tablet), KPC(tablet));
if (OB_FAIL(push_tablet_into_gc_queue(tablet))) {
if (OB_FAIL(inner_push_tablet_into_gc_queue(tablet))) {
LOG_ERROR("fail to push tablet into gc queue, tablet may be leaked", K(ret), KP(tablet));
}
ret = OB_SUCCESS; // continue to gc other tablet
@ -790,18 +804,16 @@ int ObTenantMetaMemMgr::gc_tablets_in_queue(bool &all_tablet_cleaned)
++gc_tablets_cnt;
FLOG_INFO("succeed to gc tablet", K(ret), KP(tablet));
}
tablet = static_cast<ObTablet *>(next_tablet);
}
if (OB_NOT_NULL(tablet)) {
push_tablet_list_into_gc_queue(tablet);
--left_recycle_cnt;
}
if (left_recycle_cnt < ONE_ROUND_TABLET_GC_COUNT_THRESHOLD) {
FLOG_INFO("Finish tablets gc task one round", K(gc_tablets_cnt), K(err_tablets_cnt), K_(wait_gc_tablets_cnt));
FLOG_INFO("gc tablets in queue", K(gc_tablets_cnt), K(err_tablets_cnt), K(tablet_gc_queue_.count()));
}
all_tablet_cleaned = (0 == ATOMIC_LOAD(&wait_gc_tablets_cnt_));
all_tablet_cleaned = tablet_gc_queue_.is_empty();
}
return ret;
}
int ObTenantMetaMemMgr::has_meta_wait_gc(bool &is_wait)
{
int ret = OB_SUCCESS;
@ -809,7 +821,7 @@ int ObTenantMetaMemMgr::has_meta_wait_gc(bool &is_wait)
ret = OB_NOT_INIT;
LOG_WARN("t3m has not been inited", K(ret));
} else {
const int64_t wait_gc_tablets_cnt = ATOMIC_LOAD(&wait_gc_tablets_cnt_);
const int64_t wait_gc_tablets_cnt = tablet_gc_queue_.count();
int64_t wait_gc_tables_cnt = 0;
{
lib::ObLockGuard<common::ObSpinLock> lock_guard(gc_queue_lock_);
@ -821,20 +833,18 @@ int ObTenantMetaMemMgr::has_meta_wait_gc(bool &is_wait)
return ret;
}
void *ObTenantMetaMemMgr::recycle_tablet(ObTablet *tablet, TabletBufferList *header)
void *ObTenantMetaMemMgr::release_tablet(ObTablet *tablet, const bool return_buf_ptr_after_release)
{
void *free_obj = nullptr;
const bool need_free_obj = nullptr != header ? true : false;
tablet->dec_macro_ref_cnt();
ObIAllocator *allocator = tablet->get_allocator();
if (OB_ISNULL(allocator)) {
if (need_free_obj) {
if (return_buf_ptr_after_release) {
ObMetaObjBufferHelper::del_meta_obj(tablet);
free_obj = ObMetaObjBufferHelper::get_meta_obj_buffer_ptr(reinterpret_cast<char*>(tablet));
SpinWLockGuard guard(wash_lock_);
header->remove(static_cast<ObMetaObjBufferNode *>(free_obj));
release_tablet_from_pool(tablet, false/*give_back_tablet_into_pool*/);
} else {
release_tablet(tablet);
release_tablet_from_pool(tablet, true/*give_back_tablet_into_pool*/);
}
} else {
full_tablet_creator_.free_tablet(tablet);
@ -2011,7 +2021,7 @@ int ObTenantMetaMemMgr::compare_and_swap_tablet(
return ret;
}
void ObTenantMetaMemMgr::release_tablet(ObTablet *tablet)
void ObTenantMetaMemMgr::release_tablet_from_pool(ObTablet *tablet, const bool give_back_buf_into_pool)
{
if (OB_NOT_NULL(tablet)) {
ObMetaObjBufferNode &linked_node = ObMetaObjBufferHelper::get_linked_node(reinterpret_cast<char *>(tablet));
@ -2023,10 +2033,14 @@ void ObTenantMetaMemMgr::release_tablet(ObTablet *tablet)
LOG_ERROR_RET(OB_ERR_UNEXPECTED, "ObTablet reference count may be leak", KP(tablet));
} else if (NORMAL_TABLET_POOL_SIZE == buf_header.buf_len_) {
normal_tablet_header_.remove(&linked_node);
tablet_buffer_pool_.free_obj(buf);
if (give_back_buf_into_pool) {
tablet_buffer_pool_.free_obj(buf);
}
} else if (LARGE_TABLET_POOL_SIZE == buf_header.buf_len_) {
large_tablet_header_.remove(&linked_node);
large_tablet_buffer_pool_.free_obj(buf);
if (give_back_buf_into_pool) {
large_tablet_buffer_pool_.free_obj(buf);
}
}
}
}
@ -2071,7 +2085,7 @@ int ObTenantMetaMemMgr::check_all_meta_mem_released(bool &is_released, const cha
} else {
is_released = true;
}
const int64_t wait_gc_tablets_cnt = ATOMIC_LOAD(&wait_gc_tablets_cnt_);
const int64_t wait_gc_tablets_cnt = tablet_gc_queue_.count();
const int64_t wait_gc_tables_cnt = free_tables_queue_.size();
const int64_t tablet_cnt_in_map = tablet_map_.count();
LOG_INFO("check all meta mem in t3m", K(module), K(is_released), K(memtable_cnt), K(ddl_kv_cnt),
@ -2230,12 +2244,7 @@ int ObTenantMetaMemMgr::try_wash_tablet_from_gc_queue(
{
int ret = OB_SUCCESS;
ObTablet *tablet = nullptr;
do {
tablet = ATOMIC_LOAD(&gc_head_);
} while (tablet != ATOMIC_VCAS(&gc_head_, tablet, nullptr));
while (OB_ISNULL(free_obj) && OB_SUCC(ret) && OB_NOT_NULL(tablet)) {
ObTablet *next_tablet = static_cast<ObTablet *>(tablet->get_next_tablet());
ATOMIC_DEC(&wait_gc_tablets_cnt_);
while (OB_ISNULL(free_obj) && OB_SUCC(ret) && OB_NOT_NULL(tablet = tablet_gc_queue_.pop())) {
if (OB_UNLIKELY(tablet->get_ref() != 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected tablet in gc queue", K(ret), KPC(tablet));
@ -2245,19 +2254,15 @@ int ObTenantMetaMemMgr::try_wash_tablet_from_gc_queue(
&& buf_len == ObMetaObjBufferHelper::get_buffer_header(reinterpret_cast<char *>(tablet)).buf_len_) {
h = &header;
}
free_obj = recycle_tablet(tablet, h);
free_obj = release_tablet(tablet, true/*return tablet buffer ptr after release*/);
}
if (OB_FAIL(ret)) {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(push_tablet_into_gc_queue(tablet))) {
if (OB_TMP_FAIL(inner_push_tablet_into_gc_queue(tablet))) {
LOG_ERROR("fail to push tablet into gc queue, tablet may be leaked", K(tmp_ret), KP(tablet));
}
tablet = nullptr;
}
tablet = static_cast<ObTablet *>(next_tablet);
}
if (OB_NOT_NULL(tablet)) {
push_tablet_list_into_gc_queue(tablet);
}
return ret;
}

View File

@ -198,7 +198,7 @@ public:
// garbage collector for sstable and memtable.
int push_table_into_gc_queue(ObITable *table, const ObITable::TableType table_type);
int gc_tables_in_queue(bool &all_table_cleaned);
int gc_tablet(ObTablet *tablet); // add tablet into gc queue
int push_tablet_into_gc_queue(ObTablet *tablet); // add tablet into gc queue
int gc_tablets_in_queue(bool &all_tablet_cleaned); // trigger to gc tablets
// ddl kv interface
@ -298,8 +298,7 @@ private:
ObMetaObjBufferNode *tablet_buffer_node,
ObIArray<ObTabletBufferInfo> &buffer_infos);
int64_t cal_adaptive_bucket_num();
int push_tablet_into_gc_queue(ObTablet *tablet);
void push_tablet_list_into_gc_queue(ObTablet *tablet);
int inner_push_tablet_into_gc_queue(ObTablet *tablet);
int get_min_end_scn_from_single_tablet(ObTablet *tablet,
const bool is_old,
const share::SCN &ls_checkpoint,
@ -415,6 +414,27 @@ private:
public:
int operator()(common::hash::HashMapPair<ObTabletMapKey, TabletValueStore *> &entry);
};
class TabletGCQueue final
{
public:
TabletGCQueue()
: gc_head_(nullptr),
gc_tail_(nullptr),
tablet_count_(0),
queue_lock_(common::ObLatchIds::TENANT_META_MEM_MGR_LOCK)
{}
~TabletGCQueue() = default;
OB_INLINE int64_t count() const { return ATOMIC_LOAD(&tablet_count_); }
OB_INLINE bool is_empty() const { return 0 == count(); }
int push(ObTablet *tablet);
ObTablet *pop();
private:
ObTablet *gc_head_;
ObTablet *gc_tail_;
int64_t tablet_count_;
common::SpinRWLock queue_lock_;
DISALLOW_COPY_AND_ASSIGN(TabletGCQueue);
};
private:
friend class ObT3mTabletMapIterator;
friend class TableGCTask;
@ -450,9 +470,9 @@ private:
TabletBufferList &header,
void *&free_obj);
void init_pool_arr();
void *recycle_tablet(ObTablet *tablet, TabletBufferList *header = nullptr);
void *release_tablet(ObTablet *tablet, const bool return_buf_ptr_after_release);
void release_tablet_from_pool(ObTablet *tablet, const bool give_back_tablet_into_pool);
void release_memtable(memtable::ObMemtable *memtable);
void release_tablet(ObTablet *tablet);
void release_tablet_ddl_kv_mgr(ObTabletDDLKvMgr *ddl_kv_mgr);
void release_tx_data_memtable_(ObTxDataMemtable *memtable);
void release_tx_ctx_memtable_(ObTxCtxMemtable *memtable);
@ -469,7 +489,6 @@ private:
const char *name,
common::ObIArray<ObTenantMetaMemStatus> &info) const;
int get_wash_tablet_candidate(const std::type_info &type_info, CandidateTabletInfo &info);
void destroy_gc_tablets_queue();
int push_memtable_into_gc_map_(memtable::ObMemtable *memtable);
void batch_gc_memtable_();
void batch_destroy_memtable_(memtable::ObMemtableSet *memtable_set);
@ -487,8 +506,7 @@ private:
TableGCTask table_gc_task_;
RefreshConfigTask refresh_config_task_;
TabletGCTask tablet_gc_task_;
ObTablet *gc_head_;
int64_t wait_gc_tablets_cnt_;
TabletGCQueue tablet_gc_queue_;
common::ObLinkQueue free_tables_queue_;
common::ObSpinLock gc_queue_lock_;
common::hash::ObHashMap<share::ObLSID, memtable::ObMemtableSet*> gc_memtable_map_;

View File

@ -245,9 +245,11 @@ TEST_F(TestMetaPointerMap, test_meta_pointer_map)
handle.reset();
ObTablet *old_tablet = new ObTablet();
ObMetaObj<ObTablet> old_tablet_obj;
old_tablet_obj.ptr_ = old_tablet;
ObTenantMetaMemMgr::ObNormalTabletBuffer *tablet_buffer = nullptr;
MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_.acquire(tablet_buffer);
ASSERT_NE(nullptr, tablet_buffer);
ObMetaObjBufferHelper::new_meta_obj(tablet_buffer, old_tablet_obj.ptr_);
old_tablet_obj.pool_ = &MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_;
handle.set_obj(old_tablet_obj);
@ -262,8 +264,8 @@ TEST_F(TestMetaPointerMap, test_meta_pointer_map)
phy_addr.size_ = 4096;
phy_addr.type_ = ObMetaDiskAddr::DiskType::BLOCK;
old_tablet->is_inited_ = true;
old_tablet->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
old_tablet_obj.ptr_->is_inited_ = true;
old_tablet_obj.ptr_->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
ObUpdateTabletPointerParam param;
ret = handle.get_obj()->get_updating_tablet_pointer_param(param);
ASSERT_EQ(common::OB_SUCCESS, ret);
@ -271,16 +273,17 @@ TEST_F(TestMetaPointerMap, test_meta_pointer_map)
ret = tablet_map_.compare_and_swap_addr_and_object(key, handle, handle, param);
ASSERT_EQ(common::OB_SUCCESS, ret);
ObTablet *tablet = new ObTablet();
tablet->tablet_addr_ = phy_addr;
ObMetaObj<ObTablet> tablet_obj;
tablet_obj.ptr_ = tablet;
MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_.acquire(tablet_buffer);
ASSERT_NE(nullptr, tablet_buffer);
ObMetaObjBufferHelper::new_meta_obj(tablet_buffer, tablet_obj.ptr_);
tablet_obj.ptr_->tablet_addr_ = phy_addr;
tablet_obj.pool_ = &MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_;
ObTabletHandle tablet_handle;
tablet_handle.set_obj(tablet_obj);
tablet->is_inited_ = true;
tablet->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
tablet_obj.ptr_->is_inited_ = true;
tablet_obj.ptr_->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
ret = handle.get_obj()->get_updating_tablet_pointer_param(param);
ASSERT_EQ(common::OB_SUCCESS, ret);
param.tablet_addr_ = phy_addr;
@ -291,7 +294,7 @@ TEST_F(TestMetaPointerMap, test_meta_pointer_map)
ret = tablet_map_.get_meta_obj(key, handle);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_TRUE(handle.is_valid());
ASSERT_EQ(tablet, handle.get_obj());
ASSERT_EQ(tablet_obj.ptr_, handle.get_obj());
ObTabletHandle tmp_handle;
ret = tablet_map_.erase(key, tmp_handle);
@ -341,9 +344,11 @@ TEST_F(TestMetaPointerMap, test_erase_and_load_concurrency)
handle.reset();
ObTablet *old_tablet = new ObTablet();
ObTenantMetaMemMgr::ObNormalTabletBuffer *tablet_buffer = nullptr;
ObMetaObj<ObTablet> old_tablet_obj;
old_tablet_obj.ptr_ = old_tablet;
MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_.acquire(tablet_buffer);
ASSERT_NE(nullptr, tablet_buffer);
ObMetaObjBufferHelper::new_meta_obj(tablet_buffer, old_tablet_obj.ptr_);
old_tablet_obj.pool_ = &MTL(ObTenantMetaMemMgr*)->tablet_buffer_pool_;
handle.set_obj(old_tablet_obj);
@ -358,8 +363,8 @@ TEST_F(TestMetaPointerMap, test_erase_and_load_concurrency)
phy_addr.size_ = 4096;
phy_addr.type_ = ObMetaDiskAddr::DiskType::BLOCK;
old_tablet->is_inited_ = true;
old_tablet->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
old_tablet_obj.ptr_->is_inited_ = true;
old_tablet_obj.ptr_->table_store_addr_.addr_.set_none_addr(); // mock empty_shell to pass test
ObUpdateTabletPointerParam param;
ret = handle.get_obj()->get_updating_tablet_pointer_param(param);