make sure all meta released when the observer exits

This commit is contained in:
windye 2023-07-18 08:42:18 +00:00 committed by ob-robot
parent 928f831487
commit 9b0e6cd960
11 changed files with 183 additions and 164 deletions

View File

@ -673,7 +673,7 @@ int MockTenantModuleEnv::init()
MTL_BIND2(mtl_new_default, ObTimestampService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObTransIDService::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObXAService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObLSService::mtl_init, mtl_start_default, mtl_stop_default, nullptr, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObLSService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObAccessService::mtl_init, nullptr, mtl_stop_default, nullptr, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObTenantFreezer::mtl_init, nullptr, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, checkpoint::ObCheckPointService::mtl_init, nullptr, mtl_stop_default, mtl_wait_default, mtl_destroy_default);

View File

@ -371,6 +371,10 @@ void TestConcurrentT3M::run1()
if (0 == count % N) {
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
} else {
ret = t3m_.tablet_map_.erase(key, tmp_handle);
ASSERT_EQ(common::OB_SUCCESS, ret);
tmp_handle.reset();
}
}
}
@ -433,7 +437,8 @@ TEST_F(TestTenantMetaMemMgr, test_memtable)
ASSERT_EQ(1, t3m_.memtable_pool_.inner_used_num_);
ASSERT_EQ(1, t3m_.free_tables_queue_.size());
t3m_.release_memtable(static_cast<memtable::ObMemtable *>(handle.table_));
bool all_table_cleaned = false; // no use
ASSERT_EQ(OB_SUCCESS, t3m_.gc_tables_in_queue(all_table_cleaned));
}
TEST_F(TestTenantMetaMemMgr, test_tx_ctx_memtable)
@ -461,7 +466,8 @@ TEST_F(TestTenantMetaMemMgr, test_tx_ctx_memtable)
ASSERT_EQ(1, t3m_.tx_ctx_memtable_pool_.inner_used_num_);
ASSERT_EQ(1, t3m_.free_tables_queue_.size());
t3m_.release_tx_ctx_memtable_(static_cast<ObTxCtxMemtable *>(handle.table_));
bool all_table_cleaned = false; // no use
ASSERT_EQ(OB_SUCCESS, t3m_.gc_tables_in_queue(all_table_cleaned));
}
TEST_F(TestTenantMetaMemMgr, test_tx_data_memtable)
@ -489,7 +495,8 @@ TEST_F(TestTenantMetaMemMgr, test_tx_data_memtable)
ASSERT_EQ(1, t3m_.tx_data_memtable_pool_.inner_used_num_);
ASSERT_EQ(1, t3m_.free_tables_queue_.size());
t3m_.release_tx_data_memtable_(static_cast<ObTxDataMemtable *>(handle.table_));
bool all_table_cleaned = false; // no use
ASSERT_EQ(OB_SUCCESS, t3m_.gc_tables_in_queue(all_table_cleaned));
}
TEST_F(TestTenantMetaMemMgr, test_lock_memtable)
@ -517,7 +524,8 @@ TEST_F(TestTenantMetaMemMgr, test_lock_memtable)
ASSERT_EQ(1, t3m_.lock_memtable_pool_.inner_used_num_);
ASSERT_EQ(1, t3m_.free_tables_queue_.size());
t3m_.release_lock_memtable_(static_cast<transaction::tablelock::ObLockMemtable *>(handle.table_));
bool all_table_cleaned = false; // no use
ASSERT_EQ(OB_SUCCESS, t3m_.gc_tables_in_queue(all_table_cleaned));
}
TEST_F(TestTenantMetaMemMgr, test_tablet)
@ -626,6 +634,8 @@ TEST_F(TestTenantMetaMemMgr, test_tablet)
ptr_hdl.reset();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
ASSERT_EQ(1, t3m_.full_tablet_creator_.get_used_obj_cnt());
gc_all_tablets();
}
TEST_F(TestTenantMetaMemMgr, test_wash_tablet)
@ -717,11 +727,11 @@ TEST_F(TestTenantMetaMemMgr, test_wash_tablet)
ASSERT_EQ(common::OB_INVALID_ARGUMENT, ret);
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
ObTabletHandle tmp_handle;
ret = t3m_.tablet_map_.erase(key, tmp_handle);
handle.reset();
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
}
@ -828,11 +838,11 @@ TEST_F(TestTenantMetaMemMgr, test_wash_inner_tablet)
ret = t3m_.compare_and_swap_tablet(key, addr, mem_addr);
ASSERT_EQ(common::OB_INVALID_ARGUMENT, ret);
tmp_handle.reset();
ret = t3m_.tablet_map_.erase(key, tmp_handle);
handle.reset();
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
}
@ -910,11 +920,11 @@ TEST_F(TestTenantMetaMemMgr, test_wash_no_sstable_tablet)
ASSERT_EQ(1, t3m_.tablet_map_.map_.size());
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
ObTabletHandle tmp_handle;
ret = t3m_.tablet_map_.erase(key, tmp_handle);
handle.reset();
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
}
@ -1099,11 +1109,11 @@ TEST_F(TestTenantMetaMemMgr, test_get_tablet_with_allocator)
ASSERT_EQ(common::OB_SUCCESS, t3m_.get_tablet_with_allocator(WashTabletPriority::WTP_HIGH, key, allocator, handle));
ASSERT_TRUE(handle.is_valid());
ObTabletHandle tmp_handle;
ret = t3m_.tablet_map_.erase(key, tmp_handle);
handle.reset();
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
}
@ -1220,11 +1230,11 @@ TEST_F(TestTenantMetaMemMgr, test_wash_mem_tablet)
ret = t3m_.compare_and_swap_tablet(key, addr, none_addr);
ASSERT_EQ(common::OB_INVALID_ARGUMENT, ret);
ObTabletHandle tmp_handle;
ret = t3m_.tablet_map_.erase(key, tmp_handle);
handle.reset();
ret = t3m_.del_tablet(key);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
ASSERT_EQ(0, t3m_.tablet_buffer_pool_.inner_used_num_);
}
@ -1359,6 +1369,13 @@ TEST_F(TestTenantMetaMemMgr, test_replace_tablet)
handle.reset();
ASSERT_EQ(1, t3m_.tablet_buffer_pool_.inner_used_num_);
ret = t3m_.tablet_map_.erase(key, tmp_handle);
ASSERT_EQ(common::OB_SUCCESS, ret);
ASSERT_EQ(0, t3m_.tablet_map_.map_.size());
tmp_handle.reset();
gc_all_tablets();
}
TEST_F(TestTenantMetaMemMgr, test_multi_tablet)
@ -1370,6 +1387,8 @@ TEST_F(TestTenantMetaMemMgr, test_multi_tablet)
int ret = multi_thread.start();
ASSERT_EQ(OB_SUCCESS, ret);
multi_thread.wait();
gc_all_tablets();
}
TEST_F(TestTenantMetaMemMgr, test_tablet_wash_priority)

View File

@ -460,11 +460,16 @@ int ObTenantNodeBalancer::fetch_effective_tenants(const TenantUnits &old_tenants
if (!found) {
ObTenant *tenant = nullptr;
MTL_SWITCH(tenant_config.tenant_id_) {
if (OB_FAIL(MTL(ObTenantMetaMemMgr*)->check_all_meta_mem_released(*MTL(ObLSService *),
is_released, "[DELETE_TENANT]"))) {
if (OB_FAIL(MTL(ObTenantMetaMemMgr*)->check_all_meta_mem_released(is_released, "[DELETE_TENANT]"))) {
LOG_WARN("fail to check_all_meta_mem_released", K(ret), K(tenant_config));
} else if (!is_released) {
// can not release now. do nothing
// can not release now. dump some debug info
const uint64_t interval = 180 * 1000 * 1000; // 180s
if (!is_released && REACH_TIME_INTERVAL(interval)) {
MTL(ObTenantMetaMemMgr*)->dump_tablet_info();
MTL(ObLSService *)->dump_ls_info();
PRINT_OBJ_LEAK(MTL_ID(), share::LEAK_CHECK_OBJ_MAX_NUM);
}
} else {
// check ls service safe to destroy.
is_released = MTL(ObLSService *)->safe_to_destroy();

View File

@ -197,13 +197,13 @@ using ObPartTransCtxObjPool = common::ObServerObjectPool<transaction::ObPartTran
using ObTableScanIteratorObjPool = common::ObServerObjectPool<oceanbase::storage::ObTableScanIterator>;
#define MTL_MEMBERS \
MTL_LIST( \
storage::ObTenantMetaMemMgr*, \
ObPartTransCtxObjPool*, \
ObTableScanIteratorObjPool*, \
common::ObTenantIOManager*, \
storage::mds::ObTenantMdsService*, \
storage::ObStorageLogger*, \
blocksstable::ObSharedMacroBlockMgr*, \
storage::ObTenantMetaMemMgr*, \
transaction::ObTransService*, \
logservice::coordinator::ObLeaderCoordinator*, \
logservice::coordinator::ObFailureDetector*, \

View File

@ -280,8 +280,6 @@ int ObLSTabletService::safe_to_destroy(bool &is_safe)
{
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
bool all_table_released = false;
bool all_tablet_released = false;
is_safe = true;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
@ -305,19 +303,12 @@ int ObLSTabletService::safe_to_destroy(bool &is_safe)
mds_table_mgr_.destroy();
}
if (is_safe) {
if (OB_FAIL(t3m->gc_tablets_in_queue(all_tablet_released))) {
LOG_WARN("failed to check all tablet released", K(ret));
bool is_wait_gc = false;
if (OB_FAIL(t3m->has_meta_wait_gc(is_wait_gc))) {
LOG_WARN("failed to check has_meta_wait_gc", K(ret));
is_safe = false;
} else {
is_safe = all_tablet_released;
}
}
if (is_safe) {
if (OB_FAIL(t3m->gc_tables_in_queue(all_table_released))) {
LOG_WARN("failed to check all table released", K(ret));
is_safe = false;
} else {
is_safe = all_table_released;
is_safe = !is_wait_gc;
}
}
}

View File

@ -253,40 +253,56 @@ int ObTenantMetaMemMgr::start()
void ObTenantMetaMemMgr::stop()
{
if (OB_LIKELY(is_inited_)) {
TG_STOP(tg_id_);
TG_STOP(persist_tg_id_);
LOG_INFO("t3m's three tasks have been stopped", K(tg_id_), K(persist_tg_id_));
}
// When the observer exits by kill -15, the release of meta is triggered by prepare_safe_destory
// called by ObLSService::stop(), then the ObLSService::wait() infinitely check if all meta release completed,
// so t3m can't stop gc thread until check_all_meta_mem_released is true in ObTenantMetaMemMgr::wait()
}
void ObTenantMetaMemMgr::wait()
{
if (OB_LIKELY(is_inited_)) {
int ret = OB_SUCCESS;
full_tablet_creator_.destroy_queue();
bool is_all_meta_released = false;
while (!is_all_meta_released) {
if (OB_FAIL(check_all_meta_mem_released(is_all_meta_released, "t3m_wait"))) {
is_all_meta_released = false;
LOG_WARN("fail to check_all_meta_mem_released", K(ret));
}
if (!is_all_meta_released) {
if (REACH_TIME_INTERVAL(5 * 1000 * 1000)) {
if (OB_FAIL(dump_tablet_info())) {
LOG_WARN("fail to dump tablet info", K(ret));
}
}
LOG_WARN("wait all meta released in t3m", K(ret));
ob_usleep(1 * 1000 * 1000); // 1s
}
}
TG_STOP(tg_id_);
TG_STOP(persist_tg_id_);
TG_WAIT(tg_id_);
TG_WAIT(persist_tg_id_);
LOG_INFO("t3m's three tasks have finished wait", K(tg_id_), K(persist_tg_id_));
}
}
void ObTenantMetaMemMgr::destroy()
{
int ret = OB_SUCCESS;
TG_DESTROY(tg_id_);
TG_DESTROY(persist_tg_id_);
tg_id_ = -1;
persist_tg_id_ = -1;
bool is_all_clean = false;
full_tablet_creator_.destroy_queue();
ObT3mTabletMapIterator::GCTabletItemOp gc_op(tablet_map_);
tablet_map_.for_each_value_store(gc_op);
gc_op.~GCTabletItemOp();
destroy_gc_tablets_queue();
if (tg_id_ != -1) {
TG_DESTROY(tg_id_);
tg_id_ = -1;
}
if (persist_tg_id_ != -1) {
TG_DESTROY(persist_tg_id_);
persist_tg_id_ = -1;
}
full_tablet_creator_.reset(); // must reset after gc_tablets
tablet_map_.destroy();
while (!is_all_clean && OB_SUCC(gc_tables_in_queue(is_all_clean)));
for (auto iter = gc_memtable_map_.begin();
OB_SUCC(ret) && iter != gc_memtable_map_.end(); ++iter) {
for (common::hash::ObHashMap<share::ObLSID, memtable::ObMemtableSet*>::iterator iter = gc_memtable_map_.begin();
OB_SUCC(ret) && iter != gc_memtable_map_.end(); ++iter) {
memtable::ObMemtableSet *memtable_set = iter->second;
if (OB_NOT_NULL(memtable_set)) {
if (0 != memtable_set->size()) {
@ -482,12 +498,12 @@ int ObTenantMetaMemMgr::gc_tables_in_queue(bool &all_table_cleaned)
FLOG_INFO("Successfully finish table gc", 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_),
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),
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(ddl_kv_pool_), K(memtable_pool_), K(pending_cnt), "wait_gc_count", free_tables_queue_.size(),
"tablet count", tablet_map_.count());
}
}
@ -736,6 +752,24 @@ int ObTenantMetaMemMgr::gc_tablets_in_queue(bool &all_tablet_cleaned)
}
return ret;
}
int ObTenantMetaMemMgr::has_meta_wait_gc(bool &is_wait)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
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_);
int64_t wait_gc_tables_cnt = 0;
{
lib::ObLockGuard<common::ObSpinLock> lock_guard(gc_queue_lock_);
wait_gc_tables_cnt = free_tables_queue_.size();
}
is_wait = (0 != wait_gc_tablets_cnt || 0 != wait_gc_tables_cnt);
LOG_INFO("check has meta wait gc in t3m", K(wait_gc_tablets_cnt), K(wait_gc_tables_cnt));
}
return ret;
}
void *ObTenantMetaMemMgr::recycle_tablet(ObTablet *tablet, TabletBufferList *header)
{
@ -1880,8 +1914,7 @@ int ObTenantMetaMemMgr::has_tablet(const ObTabletMapKey &key, bool &is_exist)
return ret;
}
int ObTenantMetaMemMgr::check_all_meta_mem_released(ObLSService &ls_service, bool &is_released,
const char *module)
int ObTenantMetaMemMgr::check_all_meta_mem_released(bool &is_released, const char *module)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited_)) {
@ -1905,20 +1938,18 @@ int ObTenantMetaMemMgr::check_all_meta_mem_released(ObLSService &ls_service, boo
} else {
is_released = true;
}
const int64_t wait_gc_tablets_cnt = ATOMIC_LOAD(&wait_gc_tablets_cnt_);
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),
K(tablet_cnt), K(large_tablet_cnt), K(ddl_kv_mgr_cnt), K(tablet_memtable_mgr_cnt), K(tx_data_memtable_cnt_),
K(tx_ctx_memtable_cnt_), K(lock_memtable_cnt_), K(full_tablet_cnt));
const uint64_t interval = 180 * 1000 * 1000; // 180s
if (!is_released && REACH_TIME_INTERVAL(interval)) {
dump_tablet();
dump_ls(ls_service);
PRINT_OBJ_LEAK(MTL_ID(), share::LEAK_CHECK_OBJ_MAX_NUM);
}
K(tx_ctx_memtable_cnt_), K(lock_memtable_cnt_), K(full_tablet_cnt),
K(wait_gc_tablets_cnt), K(wait_gc_tables_cnt), K(tablet_cnt_in_map));
}
return ret;
}
void ObTenantMetaMemMgr::dump_tablet()
int ObTenantMetaMemMgr::dump_tablet_info()
{
int ret = OB_SUCCESS;
common::ObFunction<int(common::hash::HashMapPair<ObTabletMapKey, TabletValueStore *>&)> func =
@ -1927,43 +1958,24 @@ void ObTenantMetaMemMgr::dump_tablet()
FLOG_INFO("dump tablet in map", K(key));
return OB_SUCCESS;
};
if (OB_FAIL(tablet_map_.for_each_value_store(func))) {
LOG_WARN("fail to traverse tablet map", K(ret));
}
SpinWLockGuard guard(wash_lock_);
for (ObMetaObjBufferNode *node = normal_tablet_header_.get_first();
node != normal_tablet_header_.get_header(); node = node->get_next()) {
FLOG_INFO("dump normal tablet buffer", KP(ObMetaObjBufferHelper::get_obj_buffer(node)), KP(node));
}
for (ObMetaObjBufferNode *node = large_tablet_header_.get_first();
node != large_tablet_header_.get_header(); node = node->get_next()) {
FLOG_INFO("dump large tablet buffer", KP(ObMetaObjBufferHelper::get_obj_buffer(node)), KP(node));
}
}
void ObTenantMetaMemMgr::dump_ls(ObLSService &ls_service) const
{
int ret = OB_SUCCESS;
common::ObSharedGuard<ObLSIterator> ls_iter;
ObLS *ls = nullptr;
ObLSMeta ls_meta;
if (OB_FAIL(ls_service.get_ls_iter(ls_iter, ObLSGetMod::STORAGE_MOD))) {
LOG_WARN("failed to get ls iter", K(ret));
}
while (OB_SUCC(ret)) {
if (OB_FAIL(ls_iter->get_next(ls))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next ls", K(ret));
}
} else if (OB_ISNULL(ls)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls is null", K(ret));
} else if (OB_FAIL(ls->get_ls_meta(ls_meta))) {
LOG_WARN("fail to get ls meta", K(ret));
} else {
FLOG_INFO("dump ls", K(ls_meta));
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("ObTenantMetaMemMgr hasn't been initialized", K(ret));
} else if (OB_FAIL(tablet_map_.for_each_value_store(func))) {
LOG_WARN("fail to traverse tablet map", K(ret));
} else {
SpinWLockGuard guard(wash_lock_);
for (ObMetaObjBufferNode *node = normal_tablet_header_.get_first();
node != normal_tablet_header_.get_header(); node = node->get_next()) {
FLOG_INFO("dump normal tablet buffer", KP(ObMetaObjBufferHelper::get_obj_buffer(node)), KP(node));
}
for (ObMetaObjBufferNode *node = large_tablet_header_.get_first();
node != large_tablet_header_.get_header(); node = node->get_next()) {
FLOG_INFO("dump large tablet buffer", KP(ObMetaObjBufferHelper::get_obj_buffer(node)), KP(node));
}
}
return ret;
}
ObTenantMetaMemMgr::HeapCompare::HeapCompare(int &ret)
@ -2254,33 +2266,6 @@ int ObT3mTabletMapIterator::FetchTabletItemOp::operator()(TabletPair &pair)
return ret;
}
ObT3mTabletMapIterator::GCTabletItemOp::GCTabletItemOp(TabletMap &tablet_map)
: tablet_map_(tablet_map)
{
}
int ObT3mTabletMapIterator::GCTabletItemOp::operator()(TabletPair &pair)
{
int ret = OB_SUCCESS;
/* we tranfer in_memory_tablet from tablet_map to tmp_handle,
and rely on tmp_handle::reset() to push tablet into gc_queue */
ObTabletHandle tmp_handle;
ObTabletPointer::ObMetaPointer *ptr = nullptr;
ObMetaDiskAddr addr;
if (OB_NOT_NULL(ptr = pair.second->get_value_ptr())) {
addr = ptr->get_addr();
ptr->get_in_memory_obj(tmp_handle); // ignore err
tmp_handle.set_wash_priority(WashTabletPriority::WTP_LOW);
}
if (addr.is_valid() && !addr.is_none()) {
LOG_INFO("macro blocks' ref cnt may be leaked, please check", K(pair.first), KPC(ptr));
addr.set_none_addr();
ptr->set_addr_with_reset_obj(addr); // update none addr to avoid trigger load_meta_obj
}
tmp_handle.reset();
return ret;
}
ObTenantTabletIterator::ObTenantTabletIterator(
ObTenantMetaMemMgr &t3m,
common::ObArenaAllocator &allocator)

View File

@ -249,10 +249,7 @@ public:
int get_tablet_addr(const ObTabletMapKey &key, ObMetaDiskAddr &addr);
int has_tablet(const ObTabletMapKey &key, bool &is_exist);
int del_tablet(const ObTabletMapKey &key);
int check_all_meta_mem_released(
ObLSService &ls_service,
bool &is_released,
const char *module);
int check_all_meta_mem_released(bool &is_released, const char *module);
// only used for replay and compat, others mustn't call this func
int compare_and_swap_tablet(
const ObTabletMapKey &key,
@ -273,6 +270,9 @@ public:
ObFullTabletCreator &get_mstx_tablet_creator() { return full_tablet_creator_; }
OB_INLINE int64_t get_total_tablet_cnt() const { return tablet_map_.count(); }
int has_meta_wait_gc(bool &is_wait);
int dump_tablet_info();
TO_STRING_KV(K_(tenant_id), K_(is_inited), "tablet count", tablet_map_.count());
private:
int fill_buffer_infos(
@ -445,8 +445,6 @@ private:
const int64_t buf_len,
TabletBufferList &header,
void *&free_obj);
void dump_tablet();
void dump_ls(ObLSService &ls_service) const;
void init_pool_arr();
void *recycle_tablet(ObTablet *tablet, TabletBufferList *header = nullptr);
void release_memtable(memtable::ObMemtable *memtable);
@ -558,16 +556,7 @@ private:
TabletMap &tablet_map_;
common::ObIArray<ObTabletMapKey> &items_;
};
protected:
class GCTabletItemOp final
{
public:
GCTabletItemOp(TabletMap &tablet_map);
~GCTabletItemOp() = default;
int operator()(TabletPair &pair);
private:
TabletMap &tablet_map_;
};
protected:
static const int64_t DEFAULT_TABLET_ITEM_CNT = 8;

View File

@ -30,6 +30,7 @@
#include "storage/tx_storage/ob_ls_map.h"
#include "storage/tx/ob_trans_service.h"
#include "storage/tx_storage/ob_ls_handle.h" //ObLSHandle
#include "storage/meta_mem/ob_tenant_meta_mem_mgr.h"
#include "rootserver/ob_tenant_info_loader.h"
#include "observer/ob_server_event_history_table_operator.h"
@ -99,8 +100,10 @@ bool ObLSService::safe_to_destroy()
ATOMIC_LOAD(&safe_ls_destroy_task_cnt_) == 0 &&
ATOMIC_LOAD(&iter_cnt_) == 0);
if (!is_safe && REACH_TIME_INTERVAL(10 * 1000 * 1000)) {
bool is_t3m_meta_released = false;
MTL(ObTenantMetaMemMgr*)->check_all_meta_mem_released(is_t3m_meta_released, "ObLSService"); //just for debug
LOG_INFO("ls service is not safe to destroy", K(ls_map_.is_empty()),
K_(safe_ls_destroy_task_cnt), K_(iter_cnt));
K_(safe_ls_destroy_task_cnt), K_(iter_cnt), K(is_t3m_meta_released));
}
return is_safe;
}
@ -1318,6 +1321,36 @@ int ObLSService::get_restore_status_(
return ret;
}
int ObLSService::dump_ls_info()
{
int ret = OB_SUCCESS;
common::ObSharedGuard<ObLSIterator> ls_iter;
ObLS *ls = nullptr;
ObLSMeta ls_meta;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(get_ls_iter(ls_iter, ObLSGetMod::TXSTORAGE_MOD))) {
LOG_WARN("failed to get ls iter", K(ret));
}
while (OB_SUCC(ret)) {
if (OB_FAIL(ls_iter->get_next(ls))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next ls", K(ret));
}
} else if (OB_ISNULL(ls)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls is null", K(ret));
} else if (OB_FAIL(ls->get_ls_meta(ls_meta))) {
LOG_WARN("fail to get ls meta", K(ret));
} else {
FLOG_INFO("dump ls info", K(ls_meta));
}
}
return ret;
}
} // storage
} // oceanbase

View File

@ -142,6 +142,7 @@ public:
obrpc::ObStorageRpcProxy *get_storage_rpc_proxy() { return &storage_svr_rpc_proxy_; }
storage::ObStorageRpc *get_storage_rpc() { return &storage_rpc_; }
ObLSMap *get_ls_map() { return &ls_map_; }
int dump_ls_info();
TO_STRING_KV(K_(tenant_id), K_(is_inited));
private:

View File

@ -177,12 +177,6 @@ void TestCompactionPolicy::SetUp()
{
ASSERT_TRUE(MockTenantModuleEnv::get_instance().is_inited());
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
t3m->stop();
t3m->wait();
t3m->destroy();
ret = t3m->init();
ASSERT_EQ(OB_SUCCESS, ret);
share::schema::ObTableSchema table_schema;
prepare_schema(table_schema);
@ -202,9 +196,8 @@ void TestCompactionPolicy::TearDown()
memtables_.reset();
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
t3m->stop();
t3m->wait();
t3m->destroy();
bool all_released = false;
t3m->check_all_meta_mem_released(all_released, "TestCompactionPolicy");
ObTenantFreezeInfoMgr *freeze_info_mgr = MTL(ObTenantFreezeInfoMgr *);
ASSERT_TRUE(nullptr != freeze_info_mgr);
@ -229,21 +222,21 @@ void TestCompactionPolicy::SetUpTestCase()
if (OB_SUCCESS != ret) {
LOG_ERROR("[FATAL ERROR] failed to create ls", K(ret));
}
ASSERT_EQ(OB_SUCCESS, ret);
}
void TestCompactionPolicy::TearDownTestCase()
{
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
t3m->stop();
t3m->wait();
t3m->destroy();
ret = t3m->init();
ASSERT_EQ(OB_SUCCESS, ret);
ret = MTL(ObLSService*)->remove_ls(ObLSID(TEST_LS_ID), false);
ASSERT_EQ(OB_SUCCESS, ret);
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ASSERT_EQ(OB_SUCCESS, ret);
ObLSID ls_id = ObLSID(TEST_LS_ID);
ObTabletID tablet_id = ObTabletID(TEST_TABLET_ID);
ObTabletMapKey key(ls_id, tablet_id);
ASSERT_EQ(OB_SUCCESS, t3m->del_tablet(key));
MockTenantModuleEnv::get_instance().destroy();
}

View File

@ -141,12 +141,15 @@ protected:
virtual void TearDown() override
{
ctx_mt_mgr_->reset();
t3m_.destroy();
ls_tx_ctx_mgr_.reset();
delete mt_mgr_;
mt_mgr_ = NULL;
ctx_mt_mgr_ = NULL;
bool all_table_cleaned = false; // no use
ASSERT_EQ(OB_SUCCESS, t3m_.gc_tables_in_queue(all_table_cleaned));
t3m_.destroy();
ASSERT_EQ(0, ref_count_);
tenant_base_.destroy();