From 17a96954c76af329d2362d2f7347ddfc00360b3d Mon Sep 17 00:00:00 2001 From: Tyshawn Date: Tue, 25 Jul 2023 08:42:27 +0000 Subject: [PATCH] [BUG.FIX] fix coredump in tmp file. --- .../blocksstable/ob_tmp_file_cache.cpp | 144 +++++++++--------- src/storage/blocksstable/ob_tmp_file_cache.h | 5 +- .../blocksstable/ob_tmp_file_store.cpp | 75 ++++++--- src/storage/blocksstable/ob_tmp_file_store.h | 6 +- 4 files changed, 139 insertions(+), 91 deletions(-) diff --git a/src/storage/blocksstable/ob_tmp_file_cache.cpp b/src/storage/blocksstable/ob_tmp_file_cache.cpp index 5b29dbdfb6..87656d8c32 100644 --- a/src/storage/blocksstable/ob_tmp_file_cache.cpp +++ b/src/storage/blocksstable/ob_tmp_file_cache.cpp @@ -655,7 +655,7 @@ void ObTmpFileMemTask::runTimerTask() ObTmpTenantMemBlockManager::IOWaitInfo::IOWaitInfo( 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) { 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())); + } else if (OB_FAIL(ret = ret_code_)) { + STORAGE_LOG(WARN, "fail to wait io info", K(ret), KPC(this)); } else if (block_.is_washing()) { 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())); } 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())); - } 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; } @@ -738,7 +739,12 @@ ObTmpTenantMemBlockManager::IOWaitInfo::~IOWaitInfo() void ObTmpTenantMemBlockManager::IOWaitInfo::destroy() { + ret_code_ = OB_SUCCESS; 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() @@ -1128,7 +1134,7 @@ int ObTmpTenantMemBlockManager::cleanup() const int64_t clean_nums = t_mblk_map_.size() - wash_threshold - ATOMIC_LOAD(&washing_count_); if (clean_nums <= 0) { 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)); } else { bool wash_success = false; @@ -1295,7 +1301,9 @@ 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)); } else if (OB_FAIL(m_blk->check_and_set_status( ObTmpMacroBlock::BlockStatus::MEMORY, ObTmpMacroBlock::BlockStatus::WASHING))) { - STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*m_blk)); + if (OB_STATE_NOT_MATCH != ret) { + STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*m_blk)); + } } return ret; @@ -1326,7 +1334,7 @@ int ObTmpTenantMemBlockManager::wash_block(const int64_t block_id, ObIOWaitInfoH STORAGE_LOG(WARN, "fail to get wash io info", K(ret), K_(tenant_id), K(m_blk)); } else if (OB_FAIL(write_io(info, mb_handle))) { STORAGE_LOG(WARN, "fail to write tmp block", K(ret), K_(tenant_id), K(info), K(*m_blk)); - } else if(OB_ISNULL(buf = static_cast(allocator_->alloc(sizeof(IOWaitInfo))))) { + } else if(OB_ISNULL(buf = static_cast(allocator_->alloc(sizeof(IOWaitInfo))))) { ret = OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(WARN, "fail to alloc io wait info memory", K(ret), K_(tenant_id)); } else if (FALSE_IT(wait_info = new (buf) IOWaitInfo(mb_handle, *m_blk, *allocator_))) { @@ -1346,13 +1354,18 @@ int ObTmpTenantMemBlockManager::wash_block(const int64_t block_id, ObIOWaitInfoH } if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) { 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(); allocator_->free(wait_info); wait_info = nullptr; } - m_blk->check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING, - ObTmpMacroBlock::BlockStatus::MEMORY); + handle.reset(); + 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; @@ -1380,8 +1393,6 @@ int ObTmpTenantMemBlockManager::write_io( write_info.io_desc_.set_group_id(ObIOModule::TMP_TENANT_MEM_BLOCK_IO); 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)); - } 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(), 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()); @@ -1415,83 +1426,70 @@ int64_t ObTmpTenantMemBlockManager::get_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 ret = OB_SUCCESS; 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) { ret = OB_NOT_INIT; STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); } else if (!stopped_) { common::ObSpLinkQueue::Link *node = NULL; - ObIOWaitInfoHandle wait_handle; - IOWaitInfo *wait_info = NULL; 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)) && (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(node))) { ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected error", K(ret)); - } else if (FALSE_IT(wait_info = static_cast(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())) { + STORAGE_LOG(ERROR, "unexpected error, wait info is nullptr", K(ret), KP(node)); } else { - bool need_rollback = true; ObTmpMacroBlock &blk = wait_info->get_block(); + const MacroBlockId ¯o_id = wait_info->block_handle_.get_macro_id(); const int64_t block_id = blk.get_block_id(); - STORAGE_LOG(INFO, "start to wash a block", KPC(&blk)); - ObThreadCondGuard cond_guard(cond_); - if (OB_FAIL(cond_guard.get_ret())) { - STORAGE_LOG(ERROR, "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))) { + 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 { - 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_); - } - - if (OB_FAIL(ret) && need_rollback) { - STORAGE_LOG(WARN, "fail to wash a block, need rollback", K(ret), K(blk)); - int tmp_ret = OB_SUCCESS; - 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())); + STORAGE_LOG(INFO, "start to wash a block", K(block_id), KPC(&blk)); + ObThreadCondGuard cond_guard(cond_); + if (OB_FAIL(cond_guard.get_ret())) { + STORAGE_LOG(WARN, "fail to guard request condition", K(ret)); } else { - ret = OB_SUCCESS; + 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)); + } else if (OB_FAIL(t_mblk_map_.erase_refactored(block_id))) { + if (OB_HASH_NOT_EXIST != ret) { + STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret), K(block_id)); + } else { + 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())); + } + } + } + wait_info->ret_code_ = ret; + int64_t tmp_ret = OB_SUCCESS; + // The broadcast() is executed regardless of success or failure, and the error code is ignored + // 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)); } } - } - } - // no wait handle to process. - if (OB_FAIL(ret)) { - if (OB_EAGAIN != ret) { - STORAGE_LOG(ERROR, "unexpected error", K(ret)); } } } @@ -1500,6 +1498,16 @@ int ObTmpTenantMemBlockManager::exec_wait() if (OB_TMP_FAIL(cond_.broadcast())) { 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; } diff --git a/src/storage/blocksstable/ob_tmp_file_cache.h b/src/storage/blocksstable/ob_tmp_file_cache.h index 5acaa5dab8..b6c670349b 100644 --- a/src/storage/blocksstable/ob_tmp_file_cache.h +++ b/src/storage/blocksstable/ob_tmp_file_cache.h @@ -285,7 +285,7 @@ public: void reset_io(); int broadcast(); 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: void destroy(); @@ -296,7 +296,7 @@ public: ObThreadCond cond_; ObIAllocator &allocator_; volatile int64_t ref_cnt_; - + int64_t ret_code_; private: DISALLOW_COPY_AND_ASSIGN(IOWaitInfo); }; @@ -331,7 +331,6 @@ public: int alloc_block_all_pages(ObTmpMacroBlock *t_mblk, ObTmpFileExtent &extent); int free_macro_block(const int64_t block_id); int wash_block(const int64_t block_id, ObIOWaitInfoHandle &handle); - int erase_block_from_map(const int64_t block_id); int cleanup(); int add_macro_block(ObTmpMacroBlock *&t_mblk); int wait_write_finish(const int64_t block_id, const int64_t timeout_ms); diff --git a/src/storage/blocksstable/ob_tmp_file_store.cpp b/src/storage/blocksstable/ob_tmp_file_store.cpp index 40abe2a9da..ae62391cf5 100644 --- a/src/storage/blocksstable/ob_tmp_file_store.cpp +++ b/src/storage/blocksstable/ob_tmp_file_store.cpp @@ -569,9 +569,16 @@ int ObTmpMacroBlock::give_back_buf_into_cache(const bool is_wash) SpinWLockGuard guard(lock_); if (OB_FAIL(ObTmpBlockCache::get_instance().put_block(key, handle_))) { 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))) { - STORAGE_LOG(WARN, "fail to check and set status", K(ret), K(key)); + } + if (is_wash) {// set block status disked in lock_ to avoid concurrency issues. + 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; } @@ -993,7 +1000,9 @@ int ObTmpTenantFileStore::alloc(const int64_t dir_id, const uint64_t tenant_id, if (OB_SUCC(ret)) { if (alloc_size > max_cont_size_per_block) { if (OB_FAIL(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)); + if (OB_SIZE_OVERFLOW != ret) { + STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); + } } else if (OB_ISNULL(t_mblk)) { ret = OB_ERR_NULL_VALUE; 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))) { if (OB_STATE_NOT_MATCH == ret) { if (OB_FAIL(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)); + if (OB_SIZE_OVERFLOW != ret) { + STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); + } } else if (OB_ISNULL(t_mblk)) { ret = OB_ERR_UNEXPECTED; 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)); } 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)); - } 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 { - ObTaskController::get().allow_next_syslog(); - STORAGE_LOG(INFO, "finish to free a block", K(ret), K(*t_mblk)); - t_mblk->~ObTmpMacroBlock(); - allocator_.free(t_mblk); + 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(); + STORAGE_LOG(INFO, "finish to free a block", K(ret), KPC(t_mblk)); + t_mblk->~ObTmpMacroBlock(); + allocator_.free(t_mblk); + t_mblk = nullptr; + } } 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)); } else if (tmp_mem_block_manager_.check_block_full()) { 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))) { STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_id)); } 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; } +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 ret = OB_SUCCESS; diff --git a/src/storage/blocksstable/ob_tmp_file_store.h b/src/storage/blocksstable/ob_tmp_file_store.h index f6b4aa23fa..2234531361 100644 --- a/src/storage/blocksstable/ob_tmp_file_store.h +++ b/src/storage/blocksstable/ob_tmp_file_store.h @@ -156,7 +156,6 @@ public: void set_io_desc(const common::ObIOFlag &io_desc); 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 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_disked() const { return ATOMIC_LOAD(&block_status_) == DISKED; } 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) { 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(); 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); int64_t get_memory_limit(const uint64_t tenant_id) const; - private: static const uint64_t IO_LIMIT = 4 * 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 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 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: ObTmpFileStore(); ~ObTmpFileStore();