[CP] [BUG.FIX] fix unexpected wash tablet from map
This commit is contained in:
parent
a680a564b8
commit
a0b5fa8b48
@ -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);
|
||||
|
@ -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
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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_;
|
||||
|
@ -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);
|
||||
|
Loading…
x
Reference in New Issue
Block a user