[BUG.FIX] fix coredump in tmp file.

This commit is contained in:
Tyshawn
2023-07-25 08:42:27 +00:00
committed by ob-robot
parent 0870baa73f
commit 17a96954c7
4 changed files with 139 additions and 91 deletions

View File

@ -655,7 +655,7 @@ void ObTmpFileMemTask::runTimerTask()
ObTmpTenantMemBlockManager::IOWaitInfo::IOWaitInfo( ObTmpTenantMemBlockManager::IOWaitInfo::IOWaitInfo(
ObMacroBlockHandle &block_handle, ObTmpMacroBlock &block, ObIAllocator &allocator) ObMacroBlockHandle &block_handle, ObTmpMacroBlock &block, ObIAllocator &allocator)
: block_handle_(block_handle), block_(block), allocator_(allocator), ref_cnt_(0) : block_handle_(block_handle), block_(block), allocator_(allocator), ref_cnt_(0), ret_code_(OB_SUCCESS)
{ {
} }
@ -691,6 +691,8 @@ int ObTmpTenantMemBlockManager::IOWaitInfo::wait(int64_t timeout_ms)
while (OB_SUCC(ret) && block_.is_washing() && wait_ms > 0) { while (OB_SUCC(ret) && block_.is_washing() && wait_ms > 0) {
if (OB_FAIL(cond_.wait(wait_ms))) { if (OB_FAIL(cond_.wait(wait_ms))) {
STORAGE_LOG(WARN, "fail to wait block write condition", K(ret), K(wait_ms), K(block_.get_block_id())); STORAGE_LOG(WARN, "fail to wait block write condition", K(ret), K(wait_ms), K(block_.get_block_id()));
} else if (OB_FAIL(ret = ret_code_)) {
STORAGE_LOG(WARN, "fail to wait io info", K(ret), KPC(this));
} else if (block_.is_washing()) { } else if (block_.is_washing()) {
wait_ms = timeout_ms - (ObTimeUtility::fast_current_time() - begin_us) / 1000; wait_ms = timeout_ms - (ObTimeUtility::fast_current_time() - begin_us) / 1000;
} }
@ -713,9 +715,8 @@ int ObTmpTenantMemBlockManager::IOWaitInfo::exec_wait(int64_t io_timeout_ms)
STORAGE_LOG(ERROR, "lock io request condition failed", K(ret), K(block_.get_block_id())); STORAGE_LOG(ERROR, "lock io request condition failed", K(ret), K(block_.get_block_id()));
} else if (OB_FAIL(block_handle_.wait(io_timeout_ms))) { } else if (OB_FAIL(block_handle_.wait(io_timeout_ms))) {
STORAGE_LOG(WARN, "wait handle wait io failed", K(ret), K(block_.get_block_id())); STORAGE_LOG(WARN, "wait handle wait io failed", K(ret), K(block_.get_block_id()));
} else if (OB_SERVER_BLOCK_MGR.dec_ref(block_handle_.get_macro_id())) {
STORAGE_LOG(WARN, "dec block ref failed", K(ret), K(block_.get_block_id()));
} }
reset_io();
return ret; return ret;
} }
@ -738,7 +739,12 @@ ObTmpTenantMemBlockManager::IOWaitInfo::~IOWaitInfo()
void ObTmpTenantMemBlockManager::IOWaitInfo::destroy() void ObTmpTenantMemBlockManager::IOWaitInfo::destroy()
{ {
ret_code_ = OB_SUCCESS;
block_handle_.get_io_handle().reset(); block_handle_.get_io_handle().reset();
if (0 != ATOMIC_LOAD(&ref_cnt_)) {
int ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(ERROR, "unexpected error, ref cnt isn't zero", K(ret), KPC(this));
}
} }
void ObTmpTenantMemBlockManager::IOWaitInfo::reset_io() void ObTmpTenantMemBlockManager::IOWaitInfo::reset_io()
@ -1128,7 +1134,7 @@ int ObTmpTenantMemBlockManager::cleanup()
const int64_t clean_nums = t_mblk_map_.size() - wash_threshold - ATOMIC_LOAD(&washing_count_); const int64_t clean_nums = t_mblk_map_.size() - wash_threshold - ATOMIC_LOAD(&washing_count_);
if (clean_nums <= 0) { if (clean_nums <= 0) {
STORAGE_LOG(DEBUG, "there is no need to wash blocks", K(ret), K(clean_nums)); STORAGE_LOG(DEBUG, "there is no need to wash blocks", K(ret), K(clean_nums));
} else if OB_FAIL(t_mblk_map_.foreach_refactored(op)) { } else if (OB_FAIL(t_mblk_map_.foreach_refactored(op))) {
STORAGE_LOG(WARN, "choose blks failed", K(ret)); STORAGE_LOG(WARN, "choose blks failed", K(ret));
} else { } else {
bool wash_success = false; bool wash_success = false;
@ -1295,8 +1301,10 @@ int ObTmpTenantMemBlockManager::get_block_and_set_washing(int64_t block_id, ObTm
STORAGE_LOG(WARN, "the block write has not been finished", K(ret), K(*m_blk)); STORAGE_LOG(WARN, "the block write has not been finished", K(ret), K(*m_blk));
} else if (OB_FAIL(m_blk->check_and_set_status( } else if (OB_FAIL(m_blk->check_and_set_status(
ObTmpMacroBlock::BlockStatus::MEMORY, ObTmpMacroBlock::BlockStatus::WASHING))) { ObTmpMacroBlock::BlockStatus::MEMORY, ObTmpMacroBlock::BlockStatus::WASHING))) {
if (OB_STATE_NOT_MATCH != ret) {
STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*m_blk)); STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*m_blk));
} }
}
return ret; return ret;
} }
@ -1346,13 +1354,18 @@ int ObTmpTenantMemBlockManager::wash_block(const int64_t block_id, ObIOWaitInfoH
} }
if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) { if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) {
mb_handle.reset(); mb_handle.reset();
if (OB_NOT_NULL(wait_info)) { // don't release wait info unless ObIOWaitInfoHandle doesn't hold its ref
if (OB_NOT_NULL(wait_info) && OB_ISNULL(handle.get_wait_info())) {
wait_info->~IOWaitInfo(); wait_info->~IOWaitInfo();
allocator_->free(wait_info); allocator_->free(wait_info);
wait_info = nullptr; wait_info = nullptr;
} }
m_blk->check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING, handle.reset();
ObTmpMacroBlock::BlockStatus::MEMORY); int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(m_blk->check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING,
ObTmpMacroBlock::BlockStatus::MEMORY))) {
STORAGE_LOG(ERROR, "fail to rollback block status", K(ret), K(tmp_ret));
}
} }
} }
return ret; return ret;
@ -1380,8 +1393,6 @@ int ObTmpTenantMemBlockManager::write_io(
write_info.io_desc_.set_group_id(ObIOModule::TMP_TENANT_MEM_BLOCK_IO); write_info.io_desc_.set_group_id(ObIOModule::TMP_TENANT_MEM_BLOCK_IO);
if (OB_FAIL(ObBlockManager::async_write_block(write_info, handle))) { if (OB_FAIL(ObBlockManager::async_write_block(write_info, handle))) {
STORAGE_LOG(WARN, "Fail to async write block", K(ret), K(write_info), K(handle)); STORAGE_LOG(WARN, "Fail to async write block", K(ret), K(write_info), K(handle));
} else if (OB_FAIL(OB_SERVER_BLOCK_MGR.inc_ref(handle.get_macro_id()))) {
STORAGE_LOG(WARN, "fail to add macro id", K(ret), "macro id", handle.get_macro_id());
} else if (OB_FAIL(OB_SERVER_BLOCK_MGR.update_write_time(handle.get_macro_id(), } else if (OB_FAIL(OB_SERVER_BLOCK_MGR.update_write_time(handle.get_macro_id(),
true/*update_to_max_time*/))) { //just to skip bad block inspect true/*update_to_max_time*/))) { //just to skip bad block inspect
STORAGE_LOG(WARN, "fail to update macro id write time", K(ret), "macro id", handle.get_macro_id()); STORAGE_LOG(WARN, "fail to update macro id write time", K(ret), "macro id", handle.get_macro_id());
@ -1415,83 +1426,70 @@ int64_t ObTmpTenantMemBlockManager::get_tenant_mem_block_num()
return tenant_mem_block_num; return tenant_mem_block_num;
} }
int ObTmpTenantMemBlockManager::erase_block_from_map(const int64_t block_id)
{
int ret = OB_SUCCESS;
if (OB_FAIL(t_mblk_map_.erase_refactored(block_id))) {
STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret), K(block_id));
}
int tmp_ret = ret;
if (OB_FAIL(wait_handles_map_.erase_refactored(block_id))) {
STORAGE_LOG(WARN, "fail to erase wait handles map", K(ret), K(block_id));
}
// need erase block_id from both map, return the error if not OB_SUCCESS
return OB_SUCCESS == tmp_ret ? ret : tmp_ret;
}
int ObTmpTenantMemBlockManager::exec_wait() int ObTmpTenantMemBlockManager::exec_wait()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
const int64_t io_timeout_ms = GCONF._data_storage_io_timeout / 1000L; const int64_t io_timeout_ms = GCONF._data_storage_io_timeout / 1000L;
int64_t wait_io_cnt = 0;
int64_t loop_nums = 0;
if (IS_NOT_INIT) { if (IS_NOT_INIT) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret));
} else if (!stopped_) { } else if (!stopped_) {
common::ObSpLinkQueue::Link *node = NULL; common::ObSpLinkQueue::Link *node = NULL;
ObIOWaitInfoHandle wait_handle;
IOWaitInfo *wait_info = NULL;
SpinWLockGuard io_guard(io_lock_); SpinWLockGuard io_guard(io_lock_);
int64_t begin_us = ObTimeUtility::fast_current_time(); const int64_t begin_us = ObTimeUtility::fast_current_time();
while (OB_SUCC(wait_info_queue_.pop(node)) && while (OB_SUCC(wait_info_queue_.pop(node)) &&
(ObTimeUtility::fast_current_time() - begin_us)/1000 < TASK_INTERVAL) { (ObTimeUtility::fast_current_time() - begin_us)/1000 < TASK_INTERVAL) {
if (OB_ISNULL(node)) { IOWaitInfo *wait_info = NULL;
++loop_nums;
if (OB_ISNULL(wait_info = static_cast<IOWaitInfo*>(node))) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(ERROR, "unexpected error", K(ret)); STORAGE_LOG(ERROR, "unexpected error, wait info is nullptr", K(ret), KP(node));
} else if (FALSE_IT(wait_info = static_cast<IOWaitInfo*>(node))) {
} else if (OB_FAIL(wait_handles_map_.get_refactored(wait_info->get_block().get_block_id(), wait_handle))) {
STORAGE_LOG(WARN, "fail to get wait handles", K(ret), K(wait_info->get_block().get_block_id()), K_(tenant_id));
} else if (OB_FAIL(wait_info->exec_wait(io_timeout_ms))) {
STORAGE_LOG(WARN, "fail to exec iohandle wait", K(ret), K_(tenant_id));
} else if (FALSE_IT(wait_info->reset_io())) {
} else { } else {
bool need_rollback = true;
ObTmpMacroBlock &blk = wait_info->get_block(); ObTmpMacroBlock &blk = wait_info->get_block();
const MacroBlockId &macro_id = wait_info->block_handle_.get_macro_id();
const int64_t block_id = blk.get_block_id(); const int64_t block_id = blk.get_block_id();
STORAGE_LOG(INFO, "start to wash a block", KPC(&blk)); const int64_t free_page_nums = blk.get_free_page_nums();
if (OB_FAIL(wait_info->exec_wait(io_timeout_ms))) {
STORAGE_LOG(WARN, "fail to exec io handle wait", K(ret), K_(tenant_id), KPC(wait_info));
} else {
STORAGE_LOG(INFO, "start to wash a block", K(block_id), KPC(&blk));
ObThreadCondGuard cond_guard(cond_); ObThreadCondGuard cond_guard(cond_);
if (OB_FAIL(cond_guard.get_ret())) { if (OB_FAIL(cond_guard.get_ret())) {
STORAGE_LOG(ERROR, "fail to guard request condition", K(ret)); STORAGE_LOG(WARN, "fail to guard request condition", K(ret));
} else if (OB_FAIL(erase_block_from_map(block_id))) {
STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret), K(block_id));
} else if (OB_FAIL(blk.give_back_buf_into_cache(true/*set block disked for washed block*/))) {
STORAGE_LOG(WARN, "fail to give back buf into cache", K(ret), K(block_id));
} else if (FALSE_IT(need_rollback = false)) {
} else if (OB_FAIL(wait_info->broadcast())) {
STORAGE_LOG(ERROR, "signal io request condition failed", K(ret), K(block_id));
} else if (FALSE_IT(OB_TMP_FILE_STORE.dec_block_cache_num(tenant_id_, 1))) {
} else { } else {
ObTaskController::get().allow_next_syslog();
STORAGE_LOG(INFO, "succeed to wash a block", K(block_id), K(t_mblk_map_.size()));
ATOMIC_DEC(&washing_count_); ATOMIC_DEC(&washing_count_);
} if (OB_FAIL(blk.give_back_buf_into_cache(true/*set block disked for washed block*/))) {
STORAGE_LOG(WARN, "fail to give back buf into cache", K(ret), K(block_id));
if (OB_FAIL(ret) && need_rollback) { } else if (OB_FAIL(t_mblk_map_.erase_refactored(block_id))) {
STORAGE_LOG(WARN, "fail to wash a block, need rollback", K(ret), K(blk)); if (OB_HASH_NOT_EXIST != ret) {
int tmp_ret = OB_SUCCESS; STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret), K(block_id));
blk.set_block_status(ObTmpMacroBlock::BlockStatus::MEMORY);
if (OB_SUCCESS != (tmp_ret = t_mblk_map_.set_refactored(block_id, &blk, 1/*overwrite*/))) {
STORAGE_LOG(INFO, "fail to retry wash block", K(tmp_ret), K(block_id), K(t_mblk_map_.size()));
} else { } else {
ret = OB_SUCCESS; ret = OB_SUCCESS;
} }
} else {
++wait_io_cnt;
OB_TMP_FILE_STORE.dec_block_cache_num(tenant_id_, 1);
ObTaskController::get().allow_next_syslog();
STORAGE_LOG(INFO, "succeed to wash a block", K(block_id), K(macro_id),
"free_page_nums:", free_page_nums, K(t_mblk_map_.size()));
} }
} }
} }
// no wait handle to process. wait_info->ret_code_ = ret;
if (OB_FAIL(ret)) { int64_t tmp_ret = OB_SUCCESS;
if (OB_EAGAIN != ret) { // The broadcast() is executed regardless of success or failure, and the error code is ignored
STORAGE_LOG(ERROR, "unexpected error", K(ret)); // so that the next request can be executed.
if (OB_TMP_FAIL(wait_info->broadcast())) {
STORAGE_LOG(ERROR, "signal io request condition failed", K(ret), K(tmp_ret), K(block_id));
}
// Regardless of success or failure, need to erase wait info handle from map.
if (OB_TMP_FAIL(wait_handles_map_.erase_refactored(block_id))) {
if (OB_HASH_NOT_EXIST != tmp_ret) {
STORAGE_LOG(ERROR, "fail to erase wait handles map", K(ret), K(tmp_ret), K(block_id));
}
}
} }
} }
} }
@ -1500,6 +1498,16 @@ int ObTmpTenantMemBlockManager::exec_wait()
if (OB_TMP_FAIL(cond_.broadcast())) { if (OB_TMP_FAIL(cond_.broadcast())) {
STORAGE_LOG(ERROR, "signal wash condition failed", K(ret), K(tmp_ret)); STORAGE_LOG(ERROR, "signal wash condition failed", K(ret), K(tmp_ret));
} }
if (loop_nums > 0) {
const int64_t washing_count = ATOMIC_LOAD(&washing_count_);
int64_t block_cache_num = -1;
int64_t page_cache_num = -1;
OB_TMP_FILE_STORE.get_block_cache_num(tenant_id_, block_cache_num);
OB_TMP_FILE_STORE.get_page_cache_num(tenant_id_, page_cache_num);
ObTaskController::get().allow_next_syslog();
STORAGE_LOG(INFO, "succeed to do one round of tmp block io", K(ret), K(loop_nums),
K(wait_io_cnt), K(washing_count), K(block_cache_num), K(page_cache_num));
}
} }
return ret; return ret;
} }

View File

@ -285,7 +285,7 @@ public:
void reset_io(); void reset_io();
int broadcast(); int broadcast();
OB_INLINE ObTmpMacroBlock& get_block() { return block_; }; OB_INLINE ObTmpMacroBlock& get_block() { return block_; };
TO_STRING_KV(K_(block), K_(block_handle), K_(ref_cnt)); TO_STRING_KV(K_(block), K_(block_handle), K_(ref_cnt), K_(ret_code));
private: private:
void destroy(); void destroy();
@ -296,7 +296,7 @@ public:
ObThreadCond cond_; ObThreadCond cond_;
ObIAllocator &allocator_; ObIAllocator &allocator_;
volatile int64_t ref_cnt_; volatile int64_t ref_cnt_;
int64_t ret_code_;
private: private:
DISALLOW_COPY_AND_ASSIGN(IOWaitInfo); DISALLOW_COPY_AND_ASSIGN(IOWaitInfo);
}; };
@ -331,7 +331,6 @@ public:
int alloc_block_all_pages(ObTmpMacroBlock *t_mblk, ObTmpFileExtent &extent); int alloc_block_all_pages(ObTmpMacroBlock *t_mblk, ObTmpFileExtent &extent);
int free_macro_block(const int64_t block_id); int free_macro_block(const int64_t block_id);
int wash_block(const int64_t block_id, ObIOWaitInfoHandle &handle); int wash_block(const int64_t block_id, ObIOWaitInfoHandle &handle);
int erase_block_from_map(const int64_t block_id);
int cleanup(); int cleanup();
int add_macro_block(ObTmpMacroBlock *&t_mblk); int add_macro_block(ObTmpMacroBlock *&t_mblk);
int wait_write_finish(const int64_t block_id, const int64_t timeout_ms); int wait_write_finish(const int64_t block_id, const int64_t timeout_ms);

View File

@ -569,9 +569,16 @@ int ObTmpMacroBlock::give_back_buf_into_cache(const bool is_wash)
SpinWLockGuard guard(lock_); SpinWLockGuard guard(lock_);
if (OB_FAIL(ObTmpBlockCache::get_instance().put_block(key, handle_))) { if (OB_FAIL(ObTmpBlockCache::get_instance().put_block(key, handle_))) {
STORAGE_LOG(WARN, "fail to put block into block cache", K(ret), K(key)); STORAGE_LOG(WARN, "fail to put block into block cache", K(ret), K(key));
// set block status disked in lock_ to avoid concurrency issues. }
} else if (is_wash && OB_FAIL(check_and_set_status(BlockStatus::WASHING, BlockStatus::DISKED))) { if (is_wash) {// set block status disked in lock_ to avoid concurrency issues.
STORAGE_LOG(WARN, "fail to check and set status", K(ret), K(key)); if (OB_FAIL(ret)) {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(check_and_set_status(BlockStatus::WASHING, BlockStatus::MEMORY))) {
STORAGE_LOG(ERROR, "fail to rollback block status", K(ret), K(tmp_ret), K(key), KPC(this));
}
} else if (OB_FAIL(check_and_set_status(BlockStatus::WASHING, BlockStatus::DISKED))) {
STORAGE_LOG(WARN, "fail to check and set status", K(ret), K(key), KPC(this));
}
} }
return ret; return ret;
} }
@ -993,7 +1000,9 @@ int ObTmpTenantFileStore::alloc(const int64_t dir_id, const uint64_t tenant_id,
if (OB_SUCC(ret)) { if (OB_SUCC(ret)) {
if (alloc_size > max_cont_size_per_block) { if (alloc_size > max_cont_size_per_block) {
if (OB_FAIL(alloc_macro_block(dir_id, tenant_id, t_mblk))) { if (OB_FAIL(alloc_macro_block(dir_id, tenant_id, t_mblk))) {
if (OB_SIZE_OVERFLOW != ret) {
STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id));
}
} else if (OB_ISNULL(t_mblk)) { } else if (OB_ISNULL(t_mblk)) {
ret = OB_ERR_NULL_VALUE; ret = OB_ERR_NULL_VALUE;
STORAGE_LOG(WARN, "block alloced is NULL", K(ret), K(dir_id), K(tenant_id)); STORAGE_LOG(WARN, "block alloced is NULL", K(ret), K(dir_id), K(tenant_id));
@ -1011,7 +1020,9 @@ int ObTmpTenantFileStore::alloc(const int64_t dir_id, const uint64_t tenant_id,
} else if (OB_FAIL(tmp_mem_block_manager_.alloc_extent(dir_id, tenant_id, alloc_size, extent))) { } else if (OB_FAIL(tmp_mem_block_manager_.alloc_extent(dir_id, tenant_id, alloc_size, extent))) {
if (OB_STATE_NOT_MATCH == ret) { if (OB_STATE_NOT_MATCH == ret) {
if (OB_FAIL(alloc_macro_block(dir_id, tenant_id, t_mblk))) { if (OB_FAIL(alloc_macro_block(dir_id, tenant_id, t_mblk))) {
if (OB_SIZE_OVERFLOW != ret) {
STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id));
}
} else if (OB_ISNULL(t_mblk)) { } else if (OB_ISNULL(t_mblk)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "unexpected error, t_mblk is nullptr", K(ret), KP(t_mblk)); STORAGE_LOG(WARN, "unexpected error, t_mblk is nullptr", K(ret), KP(t_mblk));
@ -1132,19 +1143,23 @@ int ObTmpTenantFileStore::free_macro_block(ObTmpMacroBlock *&t_mblk)
STORAGE_LOG(WARN, "invalid argument", K(ret)); STORAGE_LOG(WARN, "invalid argument", K(ret));
} else if (OB_FAIL(tmp_block_manager_.free_macro_block(t_mblk->get_block_id()))) { } else if (OB_FAIL(tmp_block_manager_.free_macro_block(t_mblk->get_block_id()))) {
STORAGE_LOG(WARN, "fail to free tmp macro block for block manager", K(ret)); STORAGE_LOG(WARN, "fail to free tmp macro block for block manager", K(ret));
} else if (t_mblk->is_memory() && OB_FAIL(tmp_mem_block_manager_.check_and_free_mem_block(t_mblk))) {
STORAGE_LOG(WARN, "fail to check and free mem block", K(ret));
}
if (OB_FAIL(ret)) {
} else if (t_mblk->is_washing() &&
OB_FAIL(tmp_mem_block_manager_.wait_write_finish(t_mblk->get_block_id(), ObTmpTenantMemBlockManager::get_default_timeout_ms()))) {
STORAGE_LOG(WARN, "fail to wait write io finish", K(ret), K(t_mblk));
} else { } else {
while (OB_SUCC(ret) && !t_mblk->is_disked()) {
if (t_mblk->is_memory() && OB_FAIL(tmp_mem_block_manager_.check_and_free_mem_block(t_mblk))) {
STORAGE_LOG(WARN, "fail to check and free mem block", K(ret), KPC(t_mblk));
} else if (t_mblk->is_washing() &&
OB_FAIL(tmp_mem_block_manager_.wait_write_finish(t_mblk->get_block_id(),
ObTmpTenantMemBlockManager::get_default_timeout_ms()))) {
STORAGE_LOG(WARN, "fail to wait write io finish", K(ret), KPC(t_mblk));
}
}
if (OB_SUCC(ret)) {
ObTaskController::get().allow_next_syslog(); ObTaskController::get().allow_next_syslog();
STORAGE_LOG(INFO, "finish to free a block", K(ret), K(*t_mblk)); STORAGE_LOG(INFO, "finish to free a block", K(ret), KPC(t_mblk));
t_mblk->~ObTmpMacroBlock(); t_mblk->~ObTmpMacroBlock();
allocator_.free(t_mblk); allocator_.free(t_mblk);
t_mblk = nullptr;
}
} }
return ret; return ret;
} }
@ -1159,7 +1174,7 @@ int ObTmpTenantFileStore::alloc_macro_block(const int64_t dir_id, const uint64_t
STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret));
} else if (tmp_mem_block_manager_.check_block_full()) { } else if (tmp_mem_block_manager_.check_block_full()) {
ret = OB_SIZE_OVERFLOW; ret = OB_SIZE_OVERFLOW;
STORAGE_LOG(WARN, "mem block is full", K(ret), K(tenant_id), K(dir_id)); STORAGE_LOG(DEBUG, "mem block is full", K(ret), K(tenant_id), K(dir_id));
} else if (OB_FAIL(tmp_block_manager_.alloc_macro_block(dir_id, tenant_id, t_mblk))) { } else if (OB_FAIL(tmp_block_manager_.alloc_macro_block(dir_id, tenant_id, t_mblk))) {
STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id));
} else if (OB_ISNULL(t_mblk)) { } else if (OB_ISNULL(t_mblk)) {
@ -1680,6 +1695,30 @@ int ObTmpFileStore::dec_page_cache_num(const uint64_t tenant_id, const int64_t n
return ret; return ret;
} }
int ObTmpFileStore::get_page_cache_num(const uint64_t tenant_id, int64_t &num)
{
int ret = OB_SUCCESS;
ObTmpTenantFileStoreHandle store_handle;
if (OB_FAIL(get_store(tenant_id, store_handle))) {
STORAGE_LOG(WARN, "fail to get tmp tenant file store", K(ret), K(tenant_id));
} else {
num = store_handle.get_tenant_store()->get_page_cache_num();
}
return ret;
}
int ObTmpFileStore::get_block_cache_num(const uint64_t tenant_id, int64_t &num)
{
int ret = OB_SUCCESS;
ObTmpTenantFileStoreHandle store_handle;
if (OB_FAIL(get_store(tenant_id, store_handle))) {
STORAGE_LOG(WARN, "fail to get tmp tenant file store", K(ret), K(tenant_id));
} else {
num = store_handle.get_tenant_store()->get_block_cache_num();
}
return ret;
}
int ObTmpFileStore::free(const uint64_t tenant_id, ObTmpFileExtent *extent) int ObTmpFileStore::free(const uint64_t tenant_id, ObTmpFileExtent *extent)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;

View File

@ -156,7 +156,6 @@ public:
void set_io_desc(const common::ObIOFlag &io_desc); void set_io_desc(const common::ObIOFlag &io_desc);
int check_and_set_status(const BlockStatus old_block_status, const BlockStatus new_block_status); int check_and_set_status(const BlockStatus old_block_status, const BlockStatus new_block_status);
OB_INLINE int get_block_status() const { return ATOMIC_LOAD(&block_status_); } OB_INLINE int get_block_status() const { return ATOMIC_LOAD(&block_status_); }
OB_INLINE void set_block_status(BlockStatus block_status) { ATOMIC_SET(&block_status_, block_status); }
OB_INLINE bool is_memory() const { return ATOMIC_LOAD(&block_status_) == MEMORY; } OB_INLINE bool is_memory() const { return ATOMIC_LOAD(&block_status_) == MEMORY; }
OB_INLINE bool is_disked() const { return ATOMIC_LOAD(&block_status_) == DISKED; } OB_INLINE bool is_disked() const { return ATOMIC_LOAD(&block_status_) == DISKED; }
OB_INLINE bool is_washing() const { return ATOMIC_LOAD(&block_status_) == WASHING; } OB_INLINE bool is_washing() const { return ATOMIC_LOAD(&block_status_) == WASHING; }
@ -264,6 +263,8 @@ public:
OB_INLINE void dec_block_cache_num(const int64_t num) { OB_INLINE void dec_block_cache_num(const int64_t num) {
ATOMIC_FAS(&block_cache_num_, num); ATOMIC_FAS(&block_cache_num_, num);
}; };
OB_INLINE int64_t get_page_cache_num() const { return ATOMIC_LOAD(&page_cache_num_); }
OB_INLINE int64_t get_block_cache_num() const { return ATOMIC_LOAD(&block_cache_num_); }
void inc_ref(); void inc_ref();
int64_t dec_ref(); int64_t dec_ref();
@ -275,7 +276,6 @@ private:
int alloc_macro_block(const int64_t dir_id, const uint64_t tenant_id, ObTmpMacroBlock *&t_mblk); int alloc_macro_block(const int64_t dir_id, const uint64_t tenant_id, ObTmpMacroBlock *&t_mblk);
int64_t get_memory_limit(const uint64_t tenant_id) const; int64_t get_memory_limit(const uint64_t tenant_id) const;
private: private:
static const uint64_t IO_LIMIT = 4 * 1024L * 1024L * 1024L; static const uint64_t IO_LIMIT = 4 * 1024L * 1024L * 1024L;
static const uint64_t TOTAL_LIMIT = 15 * 1024L * 1024L * 1024L; static const uint64_t TOTAL_LIMIT = 15 * 1024L * 1024L * 1024L;
@ -351,6 +351,8 @@ public:
int dec_page_cache_num(const uint64_t tenant_id, const int64_t num); int dec_page_cache_num(const uint64_t tenant_id, const int64_t num);
int inc_block_cache_num(const uint64_t tenant_id, const int64_t num); int inc_block_cache_num(const uint64_t tenant_id, const int64_t num);
int dec_block_cache_num(const uint64_t tenant_id, const int64_t num); int dec_block_cache_num(const uint64_t tenant_id, const int64_t num);
int get_page_cache_num(const uint64_t tenant_id, int64_t &num);
int get_block_cache_num(const uint64_t tenant_id, int64_t &num);
private: private:
ObTmpFileStore(); ObTmpFileStore();
~ObTmpFileStore(); ~ObTmpFileStore();