transform empty shell with tiny tablet, and rm persist queue

This commit is contained in:
obdev
2023-10-09 12:13:30 +00:00
committed by ob-robot
parent 686326ab59
commit a81e6710b5
12 changed files with 68 additions and 448 deletions

View File

@ -132,9 +132,6 @@ void TestLSTabletService::SetUp()
void TestLSTabletService::TearDownTestCase() void TestLSTabletService::TearDownTestCase()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
while (MTL(ObTenantMetaMemMgr*)->full_tablet_creator_.transform_head_.is_valid()) {
MTL(ObTenantMetaMemMgr*)->full_tablet_creator_.persist_tablet();
}
ret = MTL(ObLSService*)->remove_ls(ObLSID(TEST_LS_ID), false); ret = MTL(ObLSService*)->remove_ls(ObLSID(TEST_LS_ID), false);
ASSERT_EQ(OB_SUCCESS, ret); ASSERT_EQ(OB_SUCCESS, ret);

View File

@ -1533,68 +1533,6 @@ TEST_F(TestTenantMetaMemMgr, test_heap)
ASSERT_EQ(0, heap.count()); ASSERT_EQ(0, heap.count());
} }
TEST_F(TestTenantMetaMemMgr, test_full_tablet_queue)
{
ObFullTabletCreator full_creator;
ObTablet *tmp_tablet;
ASSERT_EQ(OB_SUCCESS, full_creator.init(500));
ASSERT_NE(nullptr, tmp_tablet = OB_NEWx(ObTablet, &allocator_));
MacroBlockId tmp_id;
tmp_id.second_id_ = 100;
ASSERT_EQ(OB_SUCCESS, tmp_tablet->tablet_addr_.set_mem_addr(0, 2112));
tmp_tablet->inc_ref();
ObTabletHandle tablet_handle;
tablet_handle.set_obj(tmp_tablet, &allocator_, &t3m_);
tablet_handle.set_wash_priority(WashTabletPriority::WTP_LOW);
ASSERT_FALSE(tmp_tablet->is_valid()); // test invalid tablet
ASSERT_EQ(OB_INVALID_ARGUMENT, full_creator.push_tablet_to_queue(tablet_handle));
ASSERT_EQ(0, full_creator.persist_queue_cnt_);
// mock valid empty shell tablet
tmp_tablet->table_store_addr_.addr_.set_none_addr();
tmp_tablet->storage_schema_addr_.addr_.set_none_addr();
tmp_tablet->mds_data_.auto_inc_seq_.addr_.set_none_addr();
tmp_tablet->rowkey_read_info_ = nullptr;
ASSERT_TRUE(tmp_tablet->is_valid());
ASSERT_EQ(OB_SUCCESS, tmp_tablet->tablet_addr_.set_block_addr(tmp_id, 0, 2112)); // test addr
ASSERT_EQ(OB_INVALID_ARGUMENT, full_creator.push_tablet_to_queue(tablet_handle));
ASSERT_EQ(0, full_creator.persist_queue_cnt_);
ASSERT_EQ(OB_SUCCESS, tmp_tablet->tablet_addr_.set_mem_addr(0, 2112));
ASSERT_EQ(OB_SUCCESS, full_creator.push_tablet_to_queue(tablet_handle));
ASSERT_EQ(1, full_creator.persist_queue_cnt_);
ASSERT_EQ(OB_SUCCESS, tmp_tablet->tablet_addr_.set_block_addr(tmp_id, 0, 2112));
ASSERT_EQ(OB_SUCCESS, full_creator.remove_tablet_from_queue(tablet_handle)); // skip block
ASSERT_EQ(1, full_creator.persist_queue_cnt_);
ASSERT_EQ(OB_SUCCESS, tmp_tablet->tablet_addr_.set_mem_addr(0, 2112));
ASSERT_EQ(OB_SUCCESS, full_creator.remove_tablet_from_queue(tablet_handle));
ASSERT_EQ(0, full_creator.persist_queue_cnt_);
ASSERT_EQ(full_creator.transform_head_.get_obj(), full_creator.transform_tail_.get_obj());
ASSERT_FALSE(full_creator.transform_tail_.is_valid());
ASSERT_EQ(OB_SUCCESS, full_creator.push_tablet_to_queue(tablet_handle));
ASSERT_EQ(1, full_creator.persist_queue_cnt_);
ASSERT_EQ(full_creator.transform_head_.get_obj(), full_creator.transform_tail_.get_obj());
ASSERT_EQ(full_creator.transform_head_.get_obj(), tablet_handle.get_obj());
ASSERT_FALSE(tablet_handle.get_obj()->next_full_tablet_guard_.is_valid());
ASSERT_EQ(OB_SUCCESS, full_creator.pop_tablet(tablet_handle));
ASSERT_EQ(0, full_creator.persist_queue_cnt_);
ASSERT_FALSE(tablet_handle.get_obj()->next_full_tablet_guard_.is_valid());
ASSERT_EQ(full_creator.transform_head_.get_obj(), full_creator.transform_tail_.get_obj());
ASSERT_FALSE(full_creator.transform_tail_.is_valid());
ASSERT_EQ(OB_ITER_END, full_creator.pop_tablet(tablet_handle));
ASSERT_FALSE(full_creator.transform_head_.is_valid());
tablet_handle.obj_ = nullptr; // do not use handle to gc invalid tablet
tablet_handle.reset();
}
} // end namespace storage } // end namespace storage
} // end namespace oceanbase } // end namespace oceanbase

View File

@ -1251,69 +1251,41 @@ int ObLSTabletService::update_tablet_to_empty_shell(const common::ObTabletID &ta
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
const share::ObLSID &ls_id = ls_->get_ls_id(); const share::ObLSID &ls_id = ls_->get_ls_id();
const ObTabletMapKey key(ls_id, tablet_id); const ObTabletMapKey key(ls_id, tablet_id);
ObTabletHandle new_tablet_hdl; common::ObArenaAllocator allocator(common::ObMemAttr(MTL_ID(), "UpdEmptySh"));
ObTabletHandle new_tablet_handle;
ObTabletHandle tmp_tablet_handle;
ObTabletHandle old_tablet_handle;
ObTimeGuard time_guard("UpdateTabletToEmptyShell", 3_s); ObTimeGuard time_guard("UpdateTabletToEmptyShell", 3_s);
ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_id.hash());
time_guard.click("Lock");
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("ls tablet svr hasn't been inited", K(ret)); LOG_WARN("ls tablet svr hasn't been inited", K(ret));
} else if (OB_UNLIKELY(!tablet_id.is_valid())) { } else if (OB_UNLIKELY(!tablet_id.is_valid())) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(tablet_id)); LOG_WARN("invalid args", K(ret), K(tablet_id));
} else if (OB_FAIL(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) { } else if (OB_FAIL(direct_get_tablet(tablet_id, old_tablet_handle))) {
LOG_WARN("fail to prepare create msd tablet", K(ret));
} else if (FALSE_IT(time_guard.click("PreFinish"))) {
} else if (OB_FAIL(ObTabletCreateDeleteHelper::acquire_msd_tablet(key, new_tablet_hdl))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_TABLET_NOT_EXIST;
} else {
LOG_WARN("fail to acquire temporary tablet", K(ret), K(key));
}
} else {
time_guard.click("Acquire");
ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_id.hash());
time_guard.click("Lock");
if (OB_FAIL(no_lock_update_tablet_to_empty_shell(key, new_tablet_hdl))) {
LOG_WARN("failed to do no lock update tablet to empty shell", K(ret), K(key));
} else {
time_guard.click("Update");
}
}
if (OB_SUCC(ret)) {
if (OB_UNLIKELY(!new_tablet_hdl.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected invalid new tablet hdl", K(ret), K(tablet_id), K(new_tablet_hdl));
} else if (new_tablet_hdl.get_obj()->is_valid() /* tablet may not init if empty shell exists */
&& OB_FAIL(ObTabletCreateDeleteHelper::push_msd_tablet_to_queue(new_tablet_hdl))) {
LOG_WARN("failed to push msd tablet to queue", K(ret), K(new_tablet_hdl));
}
}
return ret;
}
int ObLSTabletService::no_lock_update_tablet_to_empty_shell(
const ObTabletMapKey &key,
ObTabletHandle &new_tablet_handle)
{
int ret = OB_SUCCESS;
ObTabletHandle old_tablet_handle;
const common::ObTabletID &tablet_id = key.tablet_id_;
ObTimeGuard time_guard("NoLockUpdateTabletToEmptyShell", 2_s);
if (OB_FAIL(direct_get_tablet(tablet_id, old_tablet_handle))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id)); LOG_WARN("failed to get tablet", K(ret), K(tablet_id));
} else if (old_tablet_handle.get_obj()->is_empty_shell()) { } else if (old_tablet_handle.get_obj()->is_empty_shell()) {
LOG_INFO("old tablet is empty shell tablet, should skip this operation", K(ret), "old_tablet", old_tablet_handle.get_obj()); LOG_INFO("old tablet is empty shell tablet, should skip this operation", K(ret), "old_tablet", old_tablet_handle.get_obj());
} else if (FALSE_IT(time_guard.click("GetOld"))) {
} else if (OB_FAIL(ObTabletCreateDeleteHelper::acquire_tmp_tablet(key, allocator, tmp_tablet_handle))) {
LOG_WARN("fail to acquire temporary tablet", K(ret), K(key));
} else { } else {
time_guard.click("GetOld"); time_guard.click("Acquire");
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*); ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTablet *old_tablet = old_tablet_handle.get_obj(); ObTablet *old_tablet = old_tablet_handle.get_obj();
ObTablet *new_tablet = new_tablet_handle.get_obj(); ObTablet *tmp_tablet = tmp_tablet_handle.get_obj();
ObTablet *new_tablet = nullptr;
ObMetaDiskAddr disk_addr; ObMetaDiskAddr disk_addr;
if (OB_FAIL(new_tablet->init_empty_shell(*new_tablet_handle.get_allocator(), *old_tablet))) { if (OB_FAIL(tmp_tablet->init_empty_shell(*tmp_tablet_handle.get_allocator(), *old_tablet))) {
LOG_WARN("failed to init tablet", K(ret), KPC(old_tablet)); LOG_WARN("failed to init tablet", K(ret), KPC(old_tablet));
} else if (FALSE_IT(time_guard.click("InitNew"))) { } else if (FALSE_IT(time_guard.click("InitNew"))) {
} else if (OB_FAIL(ObTabletPersister::transform_empty_shell(*tmp_tablet, new_tablet_handle))) {
LOG_WARN("fail to transform emtpy shell", K(ret), K(tablet_id));
} else if (FALSE_IT(time_guard.click("Trans"))) {
} else if (FALSE_IT(new_tablet = new_tablet_handle.get_obj())) {
} else if (OB_FAIL(ObTabletSlogHelper::write_empty_shell_tablet_slog(new_tablet, disk_addr))) { } else if (OB_FAIL(ObTabletSlogHelper::write_empty_shell_tablet_slog(new_tablet, disk_addr))) {
LOG_WARN("fail to write emtpy shell tablet slog", K(ret), K(tablet_id), K(disk_addr)); LOG_WARN("fail to write emtpy shell tablet slog", K(ret), K(tablet_id), K(disk_addr));
} else if (FALSE_IT(new_tablet->tablet_addr_ = disk_addr)) { } else if (FALSE_IT(new_tablet->tablet_addr_ = disk_addr)) {
@ -1328,7 +1300,6 @@ int ObLSTabletService::no_lock_update_tablet_to_empty_shell(
LOG_INFO("succeeded to build empty shell tablet", K(ret), K(key), K(disk_addr)); LOG_INFO("succeeded to build empty shell tablet", K(ret), K(key), K(disk_addr));
} }
} }
return ret; return ret;
} }
@ -2028,9 +1999,7 @@ int ObLSTabletService::create_tablet(
if (OB_FAIL(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) { if (OB_FAIL(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) {
LOG_WARN("fail to prepare create msd tablet", K(ret)); LOG_WARN("fail to prepare create msd tablet", K(ret));
} } else {
if (OB_SUCC(ret)) {
ObBucketHashWLockGuard lock_guard(bucket_lock_, key.tablet_id_.hash()); ObBucketHashWLockGuard lock_guard(bucket_lock_, key.tablet_id_.hash());
if (OB_FAIL(ObTabletCreateDeleteHelper::create_msd_tablet(key, tablet_handle))) { if (OB_FAIL(ObTabletCreateDeleteHelper::create_msd_tablet(key, tablet_handle))) {
LOG_WARN("failed to create msd tablet", K(ret), K(key)); LOG_WARN("failed to create msd tablet", K(ret), K(key));
@ -2066,10 +2035,6 @@ int ObLSTabletService::create_tablet(
} }
} }
if (FAILEDx(ObTabletCreateDeleteHelper::push_msd_tablet_to_queue(tablet_handle))) {
LOG_WARN("failed to push msd tablet to queue", K(ret), K(tablet_handle));
}
return ret; return ret;
} }
@ -2190,9 +2155,7 @@ int ObLSTabletService::create_transfer_in_tablet(
time_guard.click("Prepare"); time_guard.click("Prepare");
if (FAILEDx(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) { if (FAILEDx(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) {
LOG_WARN("failed to prepare create msd tablet", K(ret)); LOG_WARN("failed to prepare create msd tablet", K(ret));
} } else {
{
ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_meta.tablet_id_.hash()); ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_meta.tablet_id_.hash());
time_guard.click("Lock"); time_guard.click("Lock");
if (FAILEDx(ObTabletCreateDeleteHelper::create_msd_tablet(key, tablet_handle))) { if (FAILEDx(ObTabletCreateDeleteHelper::create_msd_tablet(key, tablet_handle))) {
@ -2213,10 +2176,6 @@ int ObLSTabletService::create_transfer_in_tablet(
} }
} }
if (FAILEDx(ObTabletCreateDeleteHelper::push_msd_tablet_to_queue(tablet_handle))) {
LOG_WARN("failed to push msd tablet to queue", K(ret), K(tablet_handle));
}
return ret; return ret;
} }
@ -2247,36 +2206,35 @@ int ObLSTabletService::create_empty_shell_tablet(
LOG_WARN("failed to remove tablet", K(ret), K(key)); LOG_WARN("failed to remove tablet", K(ret), K(key));
} }
if (FAILEDx(ObTabletCreateDeleteHelper::prepare_create_msd_tablet())) { ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_id.hash()); // must lock after prepare
LOG_WARN("fail to prepare create msd tablet", K(ret)); common::ObArenaAllocator allocator(common::ObMemAttr(MTL_ID(), "MigEmptyT"));
ObTabletHandle tmp_tablet_hdl;
if (FAILEDx(ObTabletCreateDeleteHelper::create_tmp_tablet(key, allocator, tmp_tablet_hdl))) {
LOG_WARN("fail to create temporary tablet", K(ret), K(key));
} else { } else {
ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_id.hash()); // must lock after prepare ObFreezer *freezer = ls_->get_freezer();
if (OB_FAIL(ObTabletCreateDeleteHelper::create_msd_tablet(key, tablet_handle))) { ObTablet *tmp_tablet = tmp_tablet_hdl.get_obj();
LOG_WARN("fail to create msd tablet", K(ret), K(key)); ObTabletHandle tablet_handle;
ObTablet *new_tablet = nullptr;
ObMetaDiskAddr disk_addr;
if (OB_FAIL(tmp_tablet->init_with_migrate_param(allocator, param, false/*is_update*/, freezer))) {
LOG_WARN("failed to init tablet", K(ret), K(param));
} else if (OB_FAIL(ObTabletPersister::transform_empty_shell(*tmp_tablet, tablet_handle))) {
LOG_WARN("failed to transform empty shell", K(ret), KPC(tmp_tablet));
} else if (FALSE_IT(new_tablet = tablet_handle.get_obj())) {
} else if (OB_FAIL(ObTabletSlogHelper::write_empty_shell_tablet_slog(new_tablet, disk_addr))) {
LOG_WARN("fail to write emtpy shell tablet slog", K(ret), K(tablet_id), K(disk_addr));
} else if (FALSE_IT(new_tablet->tablet_addr_ = disk_addr)) {
} else if (OB_FAIL(refresh_tablet_addr(ls_id, tablet_id, tablet_handle))) {
LOG_WARN("failed to refresh tablet addr", K(ret), K(ls_id), K(tablet_id), K(lbt()));
ob_usleep(1000 * 1000);
ob_abort();
} else { } else {
ObFreezer *freezer = ls_->get_freezer(); ls_->get_tablet_gc_handler()->set_tablet_gc_trigger();
ObTablet *new_tablet = tablet_handle.get_obj(); LOG_INFO("succeeded to create empty shell tablet", K(ret), K(key), K(param));
ObMetaDiskAddr disk_addr;
if (OB_FAIL(new_tablet->init_with_migrate_param(*tablet_handle.get_allocator(), param, false/*is_update*/, freezer))) {
LOG_WARN("failed to init tablet", K(ret), K(param));
} else if (OB_FAIL(ObTabletSlogHelper::write_empty_shell_tablet_slog(new_tablet, disk_addr))) {
LOG_WARN("fail to write update tablet slog", K(ret), K(ls_id), K(tablet_id));
} else if (FALSE_IT(new_tablet->tablet_addr_ = disk_addr)) {
} else if (OB_FAIL(refresh_tablet_addr(ls_id, tablet_id, tablet_handle))) {
LOG_WARN("failed to refresh tablet addr", K(ret), K(ls_id), K(tablet_id), K(lbt()));
ob_usleep(1000 * 1000);
ob_abort();
} else {
ls_->get_tablet_gc_handler()->set_tablet_gc_trigger();
LOG_INFO("succeeded to create empty shell tablet", K(ret), K(key), K(param));
}
} }
} }
if (FAILEDx(ObTabletCreateDeleteHelper::push_msd_tablet_to_queue(tablet_handle))) {
LOG_WARN("fail to push msd tablet to queue", K(ret), K(tablet_handle));
}
return ret; return ret;
} }

View File

@ -717,9 +717,6 @@ private:
int create_empty_shell_tablet( int create_empty_shell_tablet(
const ObMigrationTabletParam &param, const ObMigrationTabletParam &param,
ObTabletHandle &tablet_handle); ObTabletHandle &tablet_handle);
int no_lock_update_tablet_to_empty_shell(
const ObTabletMapKey &key,
ObTabletHandle &new_tablet_handle);
private: private:
int direct_insert_rows(const uint64_t table_id, int direct_insert_rows(const uint64_t table_id,

View File

@ -116,17 +116,6 @@ void ObTenantMetaMemMgr::RefreshConfigTask::runTimerTask()
} }
} }
void ObTenantMetaMemMgr::TabletPersistTask::runTimerTask()
{
int ret = OB_SUCCESS;
if (!ObServerCheckpointSlogHandler::get_instance().is_started()) { /* for compatibility */
// do nothing
STORAGE_LOG(DEBUG, "ob block manager has not started");
} else if (OB_FAIL(t3m_->get_mstx_tablet_creator().persist_tablet())) {
LOG_WARN("fail to persist tablet in tablet creator", K(ret));
}
}
ObTenantMetaMemMgr::ObTenantMetaMemMgr(const uint64_t tenant_id) ObTenantMetaMemMgr::ObTenantMetaMemMgr(const uint64_t tenant_id)
: wash_lock_(common::ObLatchIds::TENANT_META_MEM_MGR_LOCK), : wash_lock_(common::ObLatchIds::TENANT_META_MEM_MGR_LOCK),
wash_func_(*this), wash_func_(*this),
@ -135,10 +124,8 @@ ObTenantMetaMemMgr::ObTenantMetaMemMgr(const uint64_t tenant_id)
full_tablet_creator_(), full_tablet_creator_(),
tablet_map_(), tablet_map_(),
tg_id_(-1), tg_id_(-1),
persist_tg_id_(-1),
table_gc_task_(this), table_gc_task_(this),
refresh_config_task_(), refresh_config_task_(),
tablet_persist_task_(this),
tablet_gc_task_(this), tablet_gc_task_(this),
gc_head_(nullptr), gc_head_(nullptr),
wait_gc_tablets_cnt_(0), wait_gc_tablets_cnt_(0),
@ -203,8 +190,6 @@ int ObTenantMetaMemMgr::init()
LOG_WARN("fail to initialize gc memtable map", K(ret)); LOG_WARN("fail to initialize gc memtable map", K(ret));
} else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::TenantMetaMemMgr, tg_id_))) { } else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::TenantMetaMemMgr, tg_id_))) {
LOG_WARN("fail to create thread for t3m", K(ret)); LOG_WARN("fail to create thread for t3m", K(ret));
} else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::TenantMetaMemMgr, persist_tg_id_))) {
LOG_WARN("fail to create thread for t3m", K(ret));
} else if (OB_FAIL(meta_cache_io_allocator_.init(OB_MALLOC_MIDDLE_BLOCK_SIZE, "StorMetaCacheIO", tenant_id_, mem_limit))) { } else if (OB_FAIL(meta_cache_io_allocator_.init(OB_MALLOC_MIDDLE_BLOCK_SIZE, "StorMetaCacheIO", tenant_id_, mem_limit))) {
LOG_WARN("fail to init storage meta cache io allocator", K(ret), K_(tenant_id), K(mem_limit)); LOG_WARN("fail to init storage meta cache io allocator", K(ret), K_(tenant_id), K(mem_limit));
} else { } else {
@ -235,21 +220,16 @@ int ObTenantMetaMemMgr::start()
LOG_WARN("ObTenantMetaMemMgr hasn't been inited", K(ret)); LOG_WARN("ObTenantMetaMemMgr hasn't been inited", K(ret));
} else if (OB_FAIL(TG_START(tg_id_))) { } else if (OB_FAIL(TG_START(tg_id_))) {
LOG_WARN("fail to start thread for t3m", K(ret), K(tg_id_)); LOG_WARN("fail to start thread for t3m", K(ret), K(tg_id_));
} else if (OB_FAIL(TG_START(persist_tg_id_))) {
LOG_WARN("fail to start thread for t3m", K(ret), K(persist_tg_id_));
} else if (OB_FAIL(TG_SCHEDULE(tg_id_, table_gc_task_, TABLE_GC_INTERVAL_US, true/*repeat*/))) { } else if (OB_FAIL(TG_SCHEDULE(tg_id_, table_gc_task_, TABLE_GC_INTERVAL_US, true/*repeat*/))) {
LOG_WARN("fail to schedule itables gc task", K(ret)); LOG_WARN("fail to schedule itables gc task", K(ret));
} else if (OB_FAIL(TG_SCHEDULE( } else if (OB_FAIL(TG_SCHEDULE(
tg_id_, refresh_config_task_, REFRESH_CONFIG_INTERVAL_US, true/*repeat*/))) { tg_id_, refresh_config_task_, REFRESH_CONFIG_INTERVAL_US, true/*repeat*/))) {
LOG_WARN("fail to schedule refresh config task", K(ret)); LOG_WARN("fail to schedule refresh config task", K(ret));
} else if (OB_FAIL(TG_SCHEDULE(
persist_tg_id_, tablet_persist_task_, TABLET_TRANSFORM_INTERVAL_US, true/*repeat*/))) {
LOG_WARN("fail to schedule tablet persist task", K(ret));
} else if (OB_FAIL(TG_SCHEDULE( } else if (OB_FAIL(TG_SCHEDULE(
tg_id_, tablet_gc_task_, TABLE_GC_INTERVAL_US, true/*repeat*/))) { tg_id_, tablet_gc_task_, TABLE_GC_INTERVAL_US, true/*repeat*/))) {
LOG_WARN("fail to schedule tablet gc task", K(ret)); LOG_WARN("fail to schedule tablet gc task", K(ret));
} else { } else {
LOG_INFO("successfully to start t3m's three tasks", K(ret), K(tg_id_), K(persist_tg_id_)); LOG_INFO("successfully to start t3m's three tasks", K(ret), K(tg_id_));
} }
return ret; return ret;
} }
@ -265,7 +245,6 @@ void ObTenantMetaMemMgr::wait()
{ {
if (OB_LIKELY(is_inited_)) { if (OB_LIKELY(is_inited_)) {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
full_tablet_creator_.destroy_queue();
bool is_all_meta_released = false; bool is_all_meta_released = false;
while (!is_all_meta_released) { while (!is_all_meta_released) {
if (OB_FAIL(check_all_meta_mem_released(is_all_meta_released, "t3m_wait"))) { if (OB_FAIL(check_all_meta_mem_released(is_all_meta_released, "t3m_wait"))) {
@ -284,10 +263,8 @@ void ObTenantMetaMemMgr::wait()
} }
TG_STOP(tg_id_); TG_STOP(tg_id_);
TG_STOP(persist_tg_id_);
TG_WAIT(tg_id_); TG_WAIT(tg_id_);
TG_WAIT(persist_tg_id_);
} }
} }
@ -298,10 +275,6 @@ void ObTenantMetaMemMgr::destroy()
TG_DESTROY(tg_id_); TG_DESTROY(tg_id_);
tg_id_ = -1; 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 full_tablet_creator_.reset(); // must reset after gc_tablets
tablet_map_.destroy(); tablet_map_.destroy();
for (common::hash::ObHashMap<share::ObLSID, memtable::ObMemtableSet*>::iterator iter = gc_memtable_map_.begin(); for (common::hash::ObHashMap<share::ObLSID, memtable::ObMemtableSet*>::iterator iter = gc_memtable_map_.begin();
@ -1825,11 +1798,6 @@ int ObTenantMetaMemMgr::del_tablet(const ObTabletMapKey &key)
} }
handle.set_wash_priority(WashTabletPriority::WTP_LOW); handle.set_wash_priority(WashTabletPriority::WTP_LOW);
} }
if (OB_SUCC(ret) && handle.is_valid()) { /* do not get t3m::lock when remove from queue */
if (OB_FAIL(full_tablet_creator_.remove_tablet_from_queue(handle))) {
LOG_WARN("failed to remove tablet from full queue", K(ret), K(key), K(handle));
}
}
return ret; return ret;
} }

View File

@ -358,15 +358,6 @@ private:
virtual ~RefreshConfigTask() = default; virtual ~RefreshConfigTask() = default;
virtual void runTimerTask() override; virtual void runTimerTask() override;
}; };
class TabletPersistTask : public common::ObTimerTask
{
public:
explicit TabletPersistTask(ObTenantMetaMemMgr *t3m) : t3m_(t3m) {}
virtual ~TabletPersistTask() = default;
virtual void runTimerTask() override;
private:
ObTenantMetaMemMgr *t3m_;
};
class MinMinorSSTableInfo final class MinMinorSSTableInfo final
{ {
public: public:
@ -490,7 +481,6 @@ private:
int persist_tg_id_; // since persist task may cost too much time, we use another thread to exec. int persist_tg_id_; // since persist task may cost too much time, we use another thread to exec.
TableGCTask table_gc_task_; TableGCTask table_gc_task_;
RefreshConfigTask refresh_config_task_; RefreshConfigTask refresh_config_task_;
TabletPersistTask tablet_persist_task_;
TabletGCTask tablet_gc_task_; TabletGCTask tablet_gc_task_;
ObTablet *gc_head_; ObTablet *gc_head_;
int64_t wait_gc_tablets_cnt_; int64_t wait_gc_tablets_cnt_;

View File

@ -26,12 +26,8 @@ namespace storage
ObFullTabletCreator::ObFullTabletCreator() ObFullTabletCreator::ObFullTabletCreator()
: is_inited_(false), : is_inited_(false),
tiny_allocator_(), tiny_allocator_(),
transform_head_(),
transform_tail_(),
wait_create_tablets_cnt_(0), wait_create_tablets_cnt_(0),
created_tablets_cnt_(0), created_tablets_cnt_(0),
persist_queue_cnt_(0),
mutex_(),
mstx_mem_ctx_(nullptr) mstx_mem_ctx_(nullptr)
{ {
} }
@ -70,9 +66,6 @@ int ObFullTabletCreator::init(const uint64_t tenant_id)
void ObFullTabletCreator::reset() void ObFullTabletCreator::reset()
{ {
transform_head_.reset();
transform_tail_.reset();
persist_queue_cnt_ = 0;
wait_create_tablets_cnt_ = 0; wait_create_tablets_cnt_ = 0;
created_tablets_cnt_ = 0; created_tablets_cnt_ = 0;
tiny_allocator_.reset(); tiny_allocator_.reset();
@ -110,7 +103,6 @@ int ObFullTabletCreator::throttle_tablet_creation()
if (need_wait) { if (need_wait) {
ob_usleep(10); // sleep 10us, do not get mutex here ob_usleep(10); // sleep 10us, do not get mutex here
} }
lib::ObMutexGuard guard(mutex_);
if (total() < limit_size) { if (total() < limit_size) {
need_wait = false; need_wait = false;
} else if (ObTimeUtility::fast_current_time() - start_time >= timeout) { } else if (ObTimeUtility::fast_current_time() - start_time >= timeout) {
@ -120,10 +112,9 @@ int ObFullTabletCreator::throttle_tablet_creation()
} else { } else {
need_wait = true; need_wait = true;
if (REACH_TENANT_TIME_INTERVAL(log_timeout)) { if (REACH_TENANT_TIME_INTERVAL(log_timeout)) {
const int64_t hanging_tablets_cnt = ATOMIC_LOAD(&created_tablets_cnt_) - ATOMIC_LOAD(&persist_queue_cnt_);
const int64_t wait_create_tablets_cnt = ATOMIC_LOAD(&wait_create_tablets_cnt_); const int64_t wait_create_tablets_cnt = ATOMIC_LOAD(&wait_create_tablets_cnt_);
LOG_WARN("prepare create tablet timeout", LOG_WARN("prepare create tablet timeout",
K_(created_tablets_cnt), K_(persist_queue_cnt), K(wait_create_tablets_cnt), K(hanging_tablets_cnt), K(limit_size), K_(created_tablets_cnt), K(wait_create_tablets_cnt), K(limit_size),
K(tiny_allocator_.total()), K(tiny_allocator_.used()), K(tiny_allocator_.total()), K(tiny_allocator_.used()),
K(mstx_mem_ctx_->hold()), K(mstx_mem_ctx_->used())); K(mstx_mem_ctx_->hold()), K(mstx_mem_ctx_->used()));
} }
@ -177,225 +168,5 @@ int ObFullTabletCreator::create_tablet(ObTabletHandle &tablet_handle)
return ret; return ret;
} }
int ObFullTabletCreator::persist_tablet()
{
int ret = OB_SUCCESS;
int64_t persist_tablets_cnt = 0;
int64_t error_tablets_cnt = 0;
int64_t retry_tablets_cnt = 0;
ObTabletHandle old_handle;
const int64_t per_round_time = ObTenantMetaMemMgr::TABLET_TRANSFORM_INTERVAL_US;
const int64_t start_time = ObTimeUtility::fast_current_time();
while (OB_SUCC(ret) && ObTimeUtility::fast_current_time() - start_time < per_round_time && OB_SUCC(pop_tablet(old_handle))) {
const ObTablet *old_tablet = old_handle.get_obj();
const ObMetaDiskAddr old_addr = old_tablet->get_tablet_addr();
const ObTabletMeta &tablet_meta = old_tablet->get_tablet_meta();
ObTabletMapKey key(tablet_meta.ls_id_, tablet_meta.tablet_id_);
ObMetaDiskAddr addr;
ObTabletHandle new_handle;
ObLSHandle ls_handle;
ObLSTabletService *ls_tablet_svr = nullptr;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTabletCreateDeleteMdsUserData mds_data;
ObTimeGuard single_guard("try persist tablet", 5 * 1000); // 5ms
bool tmp_fail = false;
if (OB_UNLIKELY(!old_tablet->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("unexpected old tablet", K(ret), K(key), K(old_handle), KPC(old_tablet));
} else if (FALSE_IT(single_guard.click("start persist"))) {
} else if (OB_FAIL(t3m->get_tablet_addr(key, addr))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS; // deleted, skip
} else {
LOG_ERROR("fail to get meta addr", K(ret), K(key), K(old_handle), K(old_tablet->is_empty_shell()));
}
} else if (!addr.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("unexpected not memory tablet addr", K(ret), K(key), K(addr), K(old_handle), K(old_tablet->is_empty_shell()));
} else if (addr != old_addr) {
if (addr.is_disked()) {
LOG_INFO("full tablet has been persisted, skip this", K(ret), K(key), K(old_addr), K(addr));
} else {
ret = OB_NOT_THE_OBJECT; // create_memtable may change the addr, push back to queue
tmp_fail = true;
LOG_INFO("memory addr changed, push back to queue", K(ret), K(key), K(old_addr), K(addr));
}
} else if (OB_FAIL(old_tablet->ObITabletMdsInterface::get_tablet_status(share::SCN::max_scn(), mds_data, 0))) {
if (OB_EMPTY_RESULT != ret && OB_ERR_SHARED_LOCK_CONFLICT != ret && OB_VERSION_NOT_MATCH != ret) {
LOG_ERROR("fail to get tablet status", K(ret), K(key), K(addr), K(old_handle), K(old_tablet->is_empty_shell()));
} else {
tmp_fail = true;
}
} else if (OB_FAIL(ObTabletPersister::persist_and_transform_tablet(*old_tablet, new_handle))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS; // deleted, skip
} else {
LOG_WARN("fail to persist old tablet", K(ret), K(key), K(old_handle), K(old_tablet->is_empty_shell()));
}
} else if (FALSE_IT(single_guard.click("end persist"))) {
} else if (OB_FAIL(MTL(ObLSService*)->get_ls(tablet_meta.ls_id_, ls_handle, ObLSGetMod::STORAGE_MOD))) {
LOG_ERROR("fail to get ls", K(ret), K(tablet_meta.ls_id_));
} else if (OB_ISNULL(ls_tablet_svr = ls_handle.get_ls()->get_tablet_svr())) {
ret = OB_ERR_NULL_VALUE;
LOG_ERROR("null ls tablet svr", K(ret), K(ls_handle));
} else if (OB_FAIL(ls_tablet_svr->update_tablet_mstx(key, old_addr, old_handle, new_handle))) {
LOG_WARN("fail to update tablet mstx", K(ret), K(key), K(old_addr), K(old_handle), K(new_handle), K(old_tablet->is_empty_shell()));
} else {
single_guard.click("end update mstx");
}
if (OB_FAIL(ret)) {
if (tmp_fail) {
++retry_tablets_cnt;
} else {
++error_tablets_cnt;
}
if (OB_FAIL(push_tablet_to_queue(old_handle))) {
LOG_ERROR("fail to push tablet, wrong tablet may be leaked", K(ret), K(key), K(old_handle), K(old_tablet->is_empty_shell()));
}
ret = OB_SUCCESS; // continue to persist other tablet
} else {
++persist_tablets_cnt;
LOG_DEBUG("succeed to persist one tablet", KP(old_tablet), K(single_guard));
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
// persist_tablets_cnt: the cnt of tablets that have been persisted in this round
// error_tablets_cnt: the cnt of tablets that couldn't be persisted in this round
// tablets_cnt: the cnt of tablets left in queue (including error_tablets_cnt)
if (persist_tablets_cnt + error_tablets_cnt + retry_tablets_cnt > 0) {
lib::ObMutexGuard guard(mutex_);
const int64_t hanging_tablets_cnt = ATOMIC_LOAD(&created_tablets_cnt_) - ATOMIC_LOAD(&persist_queue_cnt_);
const int64_t wait_create_tablets_cnt = ATOMIC_LOAD(&wait_create_tablets_cnt_);
FLOG_INFO("Finish persist task one round", K(persist_tablets_cnt), K(error_tablets_cnt), K(retry_tablets_cnt),
K_(created_tablets_cnt), K_(persist_queue_cnt), K(wait_create_tablets_cnt), K(hanging_tablets_cnt),
K(tiny_allocator_.total()), K(tiny_allocator_.used()),
K(mstx_mem_ctx_->hold()), K(mstx_mem_ctx_->used()));
}
return ret;
}
int ObFullTabletCreator::pop_tablet(ObTabletHandle &tablet_handle)
{
int ret = OB_SUCCESS;
tablet_handle.reset();
lib::ObMutexGuard guard(mutex_);
if (OB_UNLIKELY(0 > persist_queue_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("unexpected <0 tablets cnt", K(ret), K_(persist_queue_cnt));
} else if (0 == persist_queue_cnt_) {
ret = OB_ITER_END;
} else if (OB_UNLIKELY(!transform_head_.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected invalid tablet handle to pop", K(ret), K_(persist_queue_cnt), K_(transform_head));
} else {
tablet_handle = transform_head_;
transform_head_ = transform_head_.get_obj()->get_next_full_tablet();
ObTabletHandle empty_handle;
tablet_handle.get_obj()->set_next_full_tablet(empty_handle);
--persist_queue_cnt_;
if (!persist_queue_cnt_) {
transform_tail_.reset();
}
}
return ret;
}
int ObFullTabletCreator::push_tablet_to_queue(const ObTabletHandle &tablet_handle)
{
int ret = OB_SUCCESS;
lib::ObMutexGuard guard(mutex_);
bool hdl_valid, tablet_valid, addr_valid;
hdl_valid = tablet_valid = addr_valid = true;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("full tablet creator not inited", K(ret));
} else if (OB_UNLIKELY(!(hdl_valid = tablet_handle.is_valid())
|| !(tablet_valid = tablet_handle.get_obj()->is_valid())
|| !(addr_valid = tablet_handle.get_obj()->get_tablet_addr().is_valid())
|| tablet_handle.get_obj()->get_tablet_addr().is_block())) {
// TODO (@chenqingxiang.cqx) use !is_memory() to skip tablet if empty shell is allocated from pool
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tablet handle or not memory tablet addr", K(ret), K(hdl_valid), K(tablet_valid), K(addr_valid),
K(tablet_handle), KPC(tablet_handle.get_obj()));
} else if (0 == persist_queue_cnt_) {
tablet_handle.get_obj()->set_next_full_tablet(transform_head_);
transform_head_ = transform_tail_ = tablet_handle;
++persist_queue_cnt_;
} else {
transform_tail_.get_obj()->set_next_full_tablet(tablet_handle);
transform_tail_ = tablet_handle;
++persist_queue_cnt_;
}
return ret;
}
int ObFullTabletCreator::remove_tablet_from_queue(const ObTabletHandle &tablet_handle)
{
int ret = OB_SUCCESS;
lib::ObMutexGuard guard(mutex_);
bool hdl_valid, tablet_valid, addr_valid;
hdl_valid = tablet_valid = addr_valid = true;
ObMetaDiskAddr tablet_addr;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("full tablet creator not inited", K(ret));
} else if (OB_UNLIKELY(!(hdl_valid = tablet_handle.is_valid())
|| !(tablet_valid = tablet_handle.get_obj()->is_valid())
|| !(addr_valid = (tablet_addr = tablet_handle.get_obj()->get_tablet_addr()).is_valid()))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("fail to remove invalid tablet", K(ret), K(hdl_valid), K(tablet_valid), K(addr_valid),
K(tablet_handle), K(tablet_addr), KPC(tablet_handle.get_obj()));
} else if (tablet_addr.is_block()
|| tablet_addr.is_none()
|| 0 == persist_queue_cnt_) {
// skip persisted or none-addr tablet
// TODO (@chenqingxiang.cqx) use !is_memory() to skip tablet if empty shell is allocated from pool
} else {
ObTabletHandle curr_handle = transform_head_;
if (OB_UNLIKELY(!curr_handle.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid tranform head", K(ret), K_(transform_head));
} else if (curr_handle.get_obj() == tablet_handle.get_obj()) {
transform_head_ = transform_head_.get_obj()->get_next_full_tablet();
--persist_queue_cnt_;
if (!persist_queue_cnt_) {
transform_tail_.reset();
}
} else {
ObTabletHandle prev_handle = curr_handle;
while (curr_handle.is_valid()) {
if (curr_handle.get_obj() == tablet_handle.get_obj()) {
prev_handle.get_obj()->set_next_full_tablet(curr_handle.get_obj()->get_next_full_tablet());
if (curr_handle.get_obj() == transform_tail_.get_obj()) {
transform_tail_ = prev_handle;
}
--persist_queue_cnt_;
break;
}
prev_handle = curr_handle;
curr_handle = curr_handle.get_obj()->get_next_full_tablet();
}
}
}
return ret;
}
void ObFullTabletCreator::destroy_queue()
{
int ret = OB_SUCCESS;
lib::ObMutexGuard guard(mutex_);
while (transform_head_.is_valid()) {
transform_head_ = transform_head_.get_obj()->get_next_full_tablet();
--persist_queue_cnt_;
}
transform_tail_.reset();
if (OB_UNLIKELY(0 != persist_queue_cnt_)) {
LOG_ERROR("unexpected tablets cnt", K_(persist_queue_cnt));
}
}
} // namespace storage } // namespace storage
} // namespace oceanbase } // namespace oceanbase

View File

@ -41,12 +41,8 @@ public:
int init(const uint64_t tenant_id); int init(const uint64_t tenant_id);
void reset(); void reset();
int create_tablet(ObTabletHandle &tablet_handle); int create_tablet(ObTabletHandle &tablet_handle);
int persist_tablet(); /* ATTENTION: below functions should be called without any ls_tablet or t3m locks */
void destroy_queue(); // used to release tablets when t3m::destroy
/* ATTENTION: below functions should be called without any ls_tablet or t3m locks */
int throttle_tablet_creation(); int throttle_tablet_creation();
int push_tablet_to_queue(const ObTabletHandle &tablet_handle);
int remove_tablet_from_queue(const ObTabletHandle &tablet_handle);
void free_tablet(ObTablet *tablet); void free_tablet(ObTablet *tablet);
OB_INLINE int64_t total() const { OB_INLINE int64_t total() const {
return tiny_allocator_.total() + (nullptr == mstx_mem_ctx_ ? 0 : mstx_mem_ctx_->hold()); } return tiny_allocator_.total() + (nullptr == mstx_mem_ctx_ ? 0 : mstx_mem_ctx_->hold()); }
@ -56,17 +52,12 @@ public:
TO_STRING_KV(K(tiny_allocator_.used()), K(tiny_allocator_.total()), TO_STRING_KV(K(tiny_allocator_.used()), K(tiny_allocator_.total()),
"full allocator used", used(), "full allocator total", total()); "full allocator used", used(), "full allocator total", total());
private: private:
int pop_tablet(ObTabletHandle &tablet_handle);
common::ObIAllocator &get_allocator() { return mstx_mem_ctx_->get_malloc_allocator(); } common::ObIAllocator &get_allocator() { return mstx_mem_ctx_->get_malloc_allocator(); }
private: private:
bool is_inited_; bool is_inited_;
common::ObFIFOAllocator tiny_allocator_; common::ObFIFOAllocator tiny_allocator_;
ObTabletHandle transform_head_; // for transform thread
ObTabletHandle transform_tail_; // for transform thread
int64_t wait_create_tablets_cnt_; // tablets waiting to be created int64_t wait_create_tablets_cnt_; // tablets waiting to be created
int64_t created_tablets_cnt_; // tablets has been created int64_t created_tablets_cnt_; // tablets has been created
int64_t persist_queue_cnt_; // tablets in persist queue
lib::ObMutex mutex_;
lib::MemoryContext mstx_mem_ctx_; lib::MemoryContext mstx_mem_ctx_;
DISALLOW_COPY_AND_ASSIGN(ObFullTabletCreator); DISALLOW_COPY_AND_ASSIGN(ObFullTabletCreator);
}; };

View File

@ -433,16 +433,6 @@ int ObTabletCreateDeleteHelper::prepare_create_msd_tablet()
return ret; return ret;
} }
int ObTabletCreateDeleteHelper::push_msd_tablet_to_queue(ObTabletHandle &handle)
{
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
if (OB_FAIL(t3m->get_mstx_tablet_creator().push_tablet_to_queue(handle))) {
LOG_WARN("fail to push full tablet to queue", K(ret));
}
return ret;
}
int ObTabletCreateDeleteHelper::create_msd_tablet( int ObTabletCreateDeleteHelper::create_msd_tablet(
const ObTabletMapKey &key, const ObTabletMapKey &key,
ObTabletHandle &handle) ObTabletHandle &handle)

View File

@ -112,7 +112,6 @@ public:
common::ObArenaAllocator &allocator, common::ObArenaAllocator &allocator,
ObTabletHandle &handle); ObTabletHandle &handle);
static int prepare_create_msd_tablet(); static int prepare_create_msd_tablet();
static int push_msd_tablet_to_queue(ObTabletHandle &handle);
static int create_msd_tablet( static int create_msd_tablet(
const ObTabletMapKey &key, const ObTabletMapKey &key,
ObTabletHandle &handle); ObTabletHandle &handle);

View File

@ -17,6 +17,7 @@
#include "storage/tx_storage/ob_ls_service.h" #include "storage/tx_storage/ob_ls_service.h"
#include "storage/meta_mem/ob_tenant_meta_mem_mgr.h" #include "storage/meta_mem/ob_tenant_meta_mem_mgr.h"
#include "storage/tablet/ob_tablet_obj_load_helper.h" #include "storage/tablet/ob_tablet_obj_load_helper.h"
#include "storage/tablet/ob_tablet_slog_helper.h"
using namespace std::placeholders; using namespace std::placeholders;
using namespace oceanbase::common; using namespace oceanbase::common;
@ -311,6 +312,25 @@ int ObTabletPersister::persist_and_fill_tablet(
return ret; return ret;
} }
int ObTabletPersister::transform_empty_shell(const ObTablet &old_tablet, ObTabletHandle &new_handle)
{
int ret = OB_SUCCESS;
ObArray<ObSharedBlocksWriteCtx> tmp_tablet_meta_write_ctxs;
ObArray<ObSharedBlocksWriteCtx> tmp_sstable_meta_write_ctxs;
ObArenaAllocator tmp_allocator;
if (OB_UNLIKELY(!old_tablet.is_empty_shell())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("only support transform empty shell", K(ret), K(old_tablet));
} else if (OB_FAIL(persist_and_fill_tablet(
old_tablet, tmp_allocator, tmp_tablet_meta_write_ctxs, tmp_sstable_meta_write_ctxs, new_handle))) {
LOG_WARN("fail to persist old empty shell", K(ret), K(old_tablet));
}
return ret;
}
int ObTabletPersister::check_tablet_meta_ids( int ObTabletPersister::check_tablet_meta_ids(
const common::ObIArray<ObSharedBlocksWriteCtx> &tablet_meta_write_ctxs, const common::ObIArray<ObSharedBlocksWriteCtx> &tablet_meta_write_ctxs,
const ObTablet &tablet) const ObTablet &tablet)

View File

@ -94,6 +94,7 @@ public:
const ObTablet &old_tablet, const ObTablet &old_tablet,
char *buf, char *buf,
const int64_t len); const int64_t len);
static int transform_empty_shell(const ObTablet &old_tablet, ObTabletHandle &new_handle);
private: private:
static int check_tablet_meta_ids( static int check_tablet_meta_ids(
const common::ObIArray<ObSharedBlocksWriteCtx> &tablet_meta_write_ctxs, const common::ObIArray<ObSharedBlocksWriteCtx> &tablet_meta_write_ctxs,