Fix tmp file core at log
This commit is contained in:
@ -706,7 +706,7 @@ int MockTenantModuleEnv::init()
|
|||||||
STORAGE_LOG(ERROR, "reload memory config failed", K(ret));
|
STORAGE_LOG(ERROR, "reload memory config failed", K(ret));
|
||||||
} else if (OB_FAIL(start_())) {
|
} else if (OB_FAIL(start_())) {
|
||||||
STORAGE_LOG(ERROR, "mock env start failed", K(ret));
|
STORAGE_LOG(ERROR, "mock env start failed", K(ret));
|
||||||
} else if (ObTmpFileManager::get_instance().init()) {
|
} else if (OB_FAIL(ObTmpFileManager::get_instance().init())) {
|
||||||
STORAGE_LOG(WARN, "init_tmp_file_manager failed", K(ret));
|
STORAGE_LOG(WARN, "init_tmp_file_manager failed", K(ret));
|
||||||
} else {
|
} else {
|
||||||
inited_ = true;
|
inited_ = true;
|
||||||
|
|||||||
@ -472,10 +472,10 @@ int ObTmpFileExtent::read(const ObTmpFileIOInfo &io_info, const int64_t offset,
|
|||||||
if (OB_UNLIKELY(!is_alloced_)) {
|
if (OB_UNLIKELY(!is_alloced_)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret));
|
STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret));
|
||||||
} else if (OB_UNLIKELY(offset < 0 || offset >= offset_ || size <= 0
|
} else if (OB_UNLIKELY(offset < 0 || offset >= get_offset() || size <= 0
|
||||||
|| offset + size > offset_) || OB_ISNULL(buf)) {
|
|| offset + size > get_offset()) || OB_ISNULL(buf)) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
STORAGE_LOG(WARN, "invalid argument", K(ret), K(offset), K(offset_), K(size), K(buf));
|
STORAGE_LOG(WARN, "invalid argument", K(ret), K(offset), K(get_offset()), K(size), K(buf));
|
||||||
} else {
|
} else {
|
||||||
ObTmpBlockIOInfo info;
|
ObTmpBlockIOInfo info;
|
||||||
info.buf_ = buf;
|
info.buf_ = buf;
|
||||||
@ -506,25 +506,25 @@ int ObTmpFileExtent::write(const ObTmpFileIOInfo &io_info,int64_t &size, char *&
|
|||||||
} else if (OB_UNLIKELY(!is_alloced_)) {
|
} else if (OB_UNLIKELY(!is_alloced_)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret));
|
STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret));
|
||||||
} else if (offset_ == page_nums_ * ObTmpMacroBlock::get_default_page_size()) {
|
} else if (get_offset() == page_nums_ * ObTmpMacroBlock::get_default_page_size()) {
|
||||||
need_close = true;
|
need_close = true;
|
||||||
} else {
|
} else {
|
||||||
SpinWLockGuard guard(lock_);
|
SpinWLockGuard guard(lock_);
|
||||||
if (!is_closed()) {
|
if (!is_closed()) {
|
||||||
remain = page_nums_ * ObTmpMacroBlock::get_default_page_size() - offset_;
|
remain = page_nums_ * ObTmpMacroBlock::get_default_page_size() - get_offset();
|
||||||
write_size = std::min(remain, size);
|
write_size = std::min(remain, size);
|
||||||
ObTmpBlockIOInfo info;
|
ObTmpBlockIOInfo info;
|
||||||
info.block_id_ = block_id_;
|
info.block_id_ = block_id_;
|
||||||
info.buf_ = buf;
|
info.buf_ = buf;
|
||||||
info.io_desc_ = io_info.io_desc_;
|
info.io_desc_ = io_info.io_desc_;
|
||||||
info.offset_ = start_page_id_ * ObTmpMacroBlock::get_default_page_size() + offset_;
|
info.offset_ = start_page_id_ * ObTmpMacroBlock::get_default_page_size() + get_offset();
|
||||||
info.size_ = write_size;
|
info.size_ = write_size;
|
||||||
info.tenant_id_ = io_info.tenant_id_;
|
info.tenant_id_ = io_info.tenant_id_;
|
||||||
if (OB_FAIL(OB_TMP_FILE_STORE.write(owner_->get_tenant_id(), info))) {
|
if (OB_FAIL(OB_TMP_FILE_STORE.write(owner_->get_tenant_id(), info))) {
|
||||||
STORAGE_LOG(WARN, "fail to write the extent", K(ret));
|
STORAGE_LOG(WARN, "fail to write the extent", K(ret));
|
||||||
} else {
|
} else {
|
||||||
offset_ += write_size;
|
ATOMIC_FAA(&offset_, write_size);
|
||||||
g_offset_end_ = offset_ + g_offset_start_;
|
g_offset_end_ = get_offset() + g_offset_start_;
|
||||||
buf += write_size;
|
buf += write_size;
|
||||||
size -= write_size;
|
size -= write_size;
|
||||||
if (remain == write_size) {
|
if (remain == write_size) {
|
||||||
@ -547,7 +547,7 @@ void ObTmpFileExtent::reset()
|
|||||||
fd_ = -1;
|
fd_ = -1;
|
||||||
g_offset_start_ = 0;
|
g_offset_start_ = 0;
|
||||||
g_offset_end_ = 0;
|
g_offset_end_ = 0;
|
||||||
offset_ = 0;
|
ATOMIC_SET(&offset_, 0);
|
||||||
owner_ = NULL;
|
owner_ = NULL;
|
||||||
start_page_id_ = -1;
|
start_page_id_ = -1;
|
||||||
page_nums_ = 0;
|
page_nums_ = 0;
|
||||||
@ -592,11 +592,11 @@ bool ObTmpFileExtent::close(uint8_t &free_page_start_id, uint8_t &free_page_nums
|
|||||||
free_page_nums = 0;
|
free_page_nums = 0;
|
||||||
SpinWLockGuard guard(lock_);
|
SpinWLockGuard guard(lock_);
|
||||||
if (!is_closed()) {
|
if (!is_closed()) {
|
||||||
if (!force && 0 != page_nums_ && 0 == offset_) {
|
if (!force && 0 != page_nums_ && 0 == get_offset()) {
|
||||||
// Nothing to do. This extent is alloced just now, so it cannot be closed.
|
// Nothing to do. This extent is alloced just now, so it cannot be closed.
|
||||||
} else {
|
} else {
|
||||||
if (offset_ != page_nums_ * ObTmpMacroBlock::get_default_page_size()) {
|
if (get_offset() != page_nums_ * ObTmpMacroBlock::get_default_page_size()) {
|
||||||
uint8_t offset_page_id = common::upper_align(offset_, ObTmpMacroBlock::get_default_page_size())
|
uint8_t offset_page_id = common::upper_align(get_offset(), ObTmpMacroBlock::get_default_page_size())
|
||||||
/ ObTmpMacroBlock::get_default_page_size();
|
/ ObTmpMacroBlock::get_default_page_size();
|
||||||
free_page_nums = page_nums_ - offset_page_id;
|
free_page_nums = page_nums_ - offset_page_id;
|
||||||
free_page_start_id = start_page_id_ + offset_page_id;
|
free_page_start_id = start_page_id_ + offset_page_id;
|
||||||
@ -1120,6 +1120,10 @@ int ObTmpFile::aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &hand
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (OB_EAGAIN == ret) {
|
||||||
|
ret = OB_SUCCESS;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
handle.sub_data_size(io_info.size_ - size);
|
handle.sub_data_size(io_info.size_ - size);
|
||||||
if (OB_SUCC(ret) && !is_big_){
|
if (OB_SUCC(ret) && !is_big_){
|
||||||
|
|||||||
@ -195,8 +195,7 @@ public:
|
|||||||
OB_INLINE uint8_t get_page_nums() const { return page_nums_; }
|
OB_INLINE uint8_t get_page_nums() const { return page_nums_; }
|
||||||
OB_INLINE void set_block_id(const int64_t block_id) { block_id_ = block_id; }
|
OB_INLINE void set_block_id(const int64_t block_id) { block_id_ = block_id; }
|
||||||
OB_INLINE int64_t get_block_id() const { return block_id_; }
|
OB_INLINE int64_t get_block_id() const { return block_id_; }
|
||||||
OB_INLINE void set_offset(const int32_t offset) { offset_ = offset; }
|
OB_INLINE int32_t get_offset() const { return ATOMIC_LOAD(&offset_); }
|
||||||
OB_INLINE int32_t get_offset() const { return offset_; }
|
|
||||||
OB_INLINE ObTmpFile &get_owner() { return *owner_; }
|
OB_INLINE ObTmpFile &get_owner() { return *owner_; }
|
||||||
TO_STRING_KV(K_(is_alloced), K_(fd), K_(g_offset_start), K_(g_offset_end), KP_(owner),
|
TO_STRING_KV(K_(is_alloced), K_(fd), K_(g_offset_start), K_(g_offset_end), KP_(owner),
|
||||||
K_(start_page_id), K_(page_nums), K_(block_id), K_(offset), K_(is_closed));
|
K_(start_page_id), K_(page_nums), K_(block_id), K_(offset), K_(is_closed));
|
||||||
|
|||||||
@ -1266,7 +1266,7 @@ int ObTmpTenantMemBlockManager::get_block_and_set_washing(int64_t block_id, ObTm
|
|||||||
hash_val = murmurhash(&block_id, sizeof(block_id), hash_val);
|
hash_val = murmurhash(&block_id, sizeof(block_id), hash_val);
|
||||||
ObBucketHashRLockGuard lock_guard(map_lock_, hash_val);
|
ObBucketHashRLockGuard lock_guard(map_lock_, hash_val);
|
||||||
if (OB_FAIL(t_mblk_map_.get_refactored(block_id, m_blk))) {
|
if (OB_FAIL(t_mblk_map_.get_refactored(block_id, m_blk))) {
|
||||||
STORAGE_LOG(WARN, "tenant mem block manager get block failed", K(ret), K(block_id));
|
STORAGE_LOG(DEBUG, "tenant mem block manager get block failed", K(ret), K(block_id));
|
||||||
} else if (OB_ISNULL(m_blk)) {
|
} else if (OB_ISNULL(m_blk)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
STORAGE_LOG(WARN, "the block is null", K(ret), K(*m_blk));
|
STORAGE_LOG(WARN, "the block is null", K(ret), K(*m_blk));
|
||||||
@ -1280,6 +1280,8 @@ int ObTmpTenantMemBlockManager::get_block_and_set_washing(int64_t block_id, ObTm
|
|||||||
if (OB_FAIL(refresh_dir_to_blk_map(m_blk->get_dir_id(), m_blk))) {
|
if (OB_FAIL(refresh_dir_to_blk_map(m_blk->get_dir_id(), m_blk))) {
|
||||||
STORAGE_LOG(WARN, "fail to refresh dir_to_blk_map", K(ret), K(*m_blk));
|
STORAGE_LOG(WARN, "fail to refresh dir_to_blk_map", K(ret), K(*m_blk));
|
||||||
}
|
}
|
||||||
|
// refresh ret can be ignored. overwrite the ret.
|
||||||
|
ret = OB_STATE_NOT_MATCH;
|
||||||
} else if (OB_FAIL(m_blk->close(is_closed))) {
|
} else if (OB_FAIL(m_blk->close(is_closed))) {
|
||||||
STORAGE_LOG(WARN, "fail to close block", K(ret), K(*m_blk));
|
STORAGE_LOG(WARN, "fail to close block", K(ret), K(*m_blk));
|
||||||
} else if (!is_closed) {
|
} else if (!is_closed) {
|
||||||
@ -1332,14 +1334,24 @@ int ObTmpTenantMemBlockManager::wash_block(const int64_t block_id, ObIOWaitInfoH
|
|||||||
STORAGE_LOG(WARN, "fail to set block into write_handles_map", K(ret), "block_id", m_blk->get_block_id());
|
STORAGE_LOG(WARN, "fail to set block into write_handles_map", K(ret), "block_id", m_blk->get_block_id());
|
||||||
} else if (OB_FAIL(wait_info_queue_.push(wait_info))) {
|
} else if (OB_FAIL(wait_info_queue_.push(wait_info))) {
|
||||||
STORAGE_LOG(WARN, "fail to push back into write_handles", K(ret), K(wait_info));
|
STORAGE_LOG(WARN, "fail to push back into write_handles", K(ret), K(wait_info));
|
||||||
|
int tmp_ret = OB_SUCCESS;
|
||||||
|
if (OB_TMP_FAIL(wait_handles_map_.erase_refactored(m_blk->get_block_id()))) {
|
||||||
|
STORAGE_LOG(WARN, "fail to erase block from wait handles map", K(tmp_ret), K(m_blk->get_block_id()));
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
ATOMIC_INC(&washing_count_);
|
ATOMIC_INC(&washing_count_);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) {
|
if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) {
|
||||||
|
mb_handle.reset();
|
||||||
|
if (OB_NOT_NULL(wait_info)) {
|
||||||
|
wait_info->~IOWaitInfo();
|
||||||
|
allocator_->free(wait_info);
|
||||||
|
wait_info = nullptr;
|
||||||
|
}
|
||||||
m_blk->check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING,
|
m_blk->check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING,
|
||||||
ObTmpMacroBlock::BlockStatus::MEMORY);
|
ObTmpMacroBlock::BlockStatus::MEMORY);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1399,6 +1411,21 @@ 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;
|
||||||
@ -1424,51 +1451,49 @@ int ObTmpTenantMemBlockManager::exec_wait()
|
|||||||
STORAGE_LOG(WARN, "fail to exec iohandle wait", K(ret), K_(tenant_id));
|
STORAGE_LOG(WARN, "fail to exec iohandle wait", K(ret), K_(tenant_id));
|
||||||
} else if (FALSE_IT(wait_info->reset_io())) {
|
} 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 int64_t block_id = blk.get_block_id();
|
||||||
|
STORAGE_LOG(INFO, "start to wash a block", 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(ERROR, "fail to guard request condition", K(ret));
|
||||||
} else if (OB_FAIL(t_mblk_map_.erase_refactored(blk.get_block_id()))) {
|
} else if (OB_FAIL(erase_block_from_map(block_id))) {
|
||||||
STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret));
|
STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret), K(block_id));
|
||||||
} else if (OB_FAIL(wait_handles_map_.erase_refactored(blk.get_block_id()))) {
|
} else if (OB_FAIL(blk.give_back_buf_into_cache(true/*set block disked for washed block*/))) {
|
||||||
STORAGE_LOG(WARN, "fail to erase t_mblk_map", K(ret));
|
STORAGE_LOG(WARN, "fail to give back buf into cache", K(ret), K(block_id));
|
||||||
} else if (OB_FAIL(blk.check_and_set_status(ObTmpMacroBlock::BlockStatus::WASHING,
|
} else if (FALSE_IT(need_rollback = false)) {
|
||||||
ObTmpMacroBlock::BlockStatus::DISKED))) {
|
|
||||||
STORAGE_LOG(WARN, "set block status failed", K(ret), K(blk.get_block_id()));
|
|
||||||
} else if (OB_FAIL(blk.give_back_buf_into_cache())) {
|
|
||||||
STORAGE_LOG(WARN, "fail to put tmp block cache", K(ret), K_(tenant_id));
|
|
||||||
} else if (OB_FAIL(wait_info->broadcast())) {
|
} else if (OB_FAIL(wait_info->broadcast())) {
|
||||||
STORAGE_LOG(ERROR, "signal io request condition failed", K(ret), K(blk.get_block_id()));
|
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 if (FALSE_IT(OB_TMP_FILE_STORE.dec_block_cache_num(tenant_id_, 1))) {
|
||||||
} else {
|
} else {
|
||||||
ObTaskController::get().allow_next_syslog();
|
ObTaskController::get().allow_next_syslog();
|
||||||
STORAGE_LOG(INFO, "succeed to wash a block", K(blk), K(t_mblk_map_.size()));
|
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(ret)) {
|
if (OB_FAIL(ret) && need_rollback) {
|
||||||
STORAGE_LOG(WARN, "fail to wash a block", K(blk));
|
STORAGE_LOG(WARN, "fail to wash a block, need rollback", K(ret), K(blk));
|
||||||
int tmp_ret = OB_SUCCESS;
|
int tmp_ret = OB_SUCCESS;
|
||||||
blk.set_block_status(ObTmpMacroBlock::BlockStatus::MEMORY);
|
blk.set_block_status(ObTmpMacroBlock::BlockStatus::MEMORY);
|
||||||
if (OB_SUCCESS != (tmp_ret = t_mblk_map_.set_refactored(blk.get_block_id(), &blk, 1/*overwrite*/))) {
|
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(blk), K(t_mblk_map_.size()));
|
STORAGE_LOG(INFO, "fail to retry wash block", K(tmp_ret), K(block_id), K(t_mblk_map_.size()));
|
||||||
} else if (OB_SUCCESS != (tmp_ret = dir_to_blk_map_.set_refactored(
|
|
||||||
blk.get_dir_id(), blk.get_block_id(), 1/*overwrite*/))) {
|
|
||||||
STORAGE_LOG(INFO, "fail to set dir to blk map", K(tmp_ret), K(blk), K(dir_to_blk_map_.size()));
|
|
||||||
} else {
|
} else {
|
||||||
ret = OB_SUCCESS;
|
ret = OB_SUCCESS;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (OB_SUCC(ret) && OB_FAIL(cond_.broadcast())) {
|
if (OB_SUCC(ret) && OB_FAIL(cond_.broadcast())) {
|
||||||
STORAGE_LOG(ERROR, "signal wash condition failed", K(ret), K(blk.get_block_id()));
|
STORAGE_LOG(ERROR, "signal wash condition failed", K(ret), K(block_id));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// no wait handle to process.
|
// no wait handle to process.
|
||||||
|
if (OB_FAIL(ret)) {
|
||||||
if (OB_EAGAIN != ret) {
|
if (OB_EAGAIN != ret) {
|
||||||
STORAGE_LOG(ERROR, "unexpected error", K(ret));
|
STORAGE_LOG(ERROR, "unexpected error", K(ret));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1512,7 +1537,7 @@ int ObTmpTenantMemBlockManager::wait_write_finish(const int64_t block_id, const
|
|||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
bool is_found = false;
|
bool is_found = false;
|
||||||
while (OB_SUCC(ret) && blk->is_washing()) {
|
while (OB_SUCC(ret) && blk->is_washing() && !is_found) {
|
||||||
if (OB_FAIL(wait_handles_map_.get_refactored(block_id, handle))) {
|
if (OB_FAIL(wait_handles_map_.get_refactored(block_id, handle))) {
|
||||||
if (OB_HASH_NOT_EXIST == ret) {
|
if (OB_HASH_NOT_EXIST == ret) {
|
||||||
ret = OB_SUCCESS;
|
ret = OB_SUCCESS;
|
||||||
|
|||||||
@ -331,6 +331,7 @@ 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);
|
||||||
|
|||||||
@ -562,13 +562,16 @@ int ObTmpMacroBlock::get_wash_io_info(ObTmpBlockIOInfo &info)
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObTmpMacroBlock::give_back_buf_into_cache()
|
int ObTmpMacroBlock::give_back_buf_into_cache(const bool is_wash)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
ObTmpBlockCacheKey key(tmp_file_header_.block_id_, tmp_file_header_.tenant_id_);
|
ObTmpBlockCacheKey key(tmp_file_header_.block_id_, tmp_file_header_.tenant_id_);
|
||||||
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))) {
|
||||||
|
STORAGE_LOG(WARN, "fail to check and set status", K(ret), K(key));
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
@ -1158,8 +1161,7 @@ int ObTmpTenantFileStore::alloc_macro_block(const int64_t dir_id, const uint64_t
|
|||||||
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(WARN, "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),
|
STORAGE_LOG(WARN, "cannot allocate a tmp macro block", K(ret), K(dir_id), K(tenant_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 is null", K(ret));
|
STORAGE_LOG(WARN, "block is null", K(ret));
|
||||||
@ -1349,9 +1351,8 @@ int ObTmpTenantFileStore::write(const ObTmpBlockIOInfo &io_info)
|
|||||||
} else {
|
} else {
|
||||||
// Skip washing and disked status
|
// Skip washing and disked status
|
||||||
// Won't change io_info for retry alloc block.
|
// Won't change io_info for retry alloc block.
|
||||||
ret = OB_NOT_SUPPORTED;
|
ret = OB_EAGAIN;
|
||||||
STORAGE_LOG(WARN, "fail to write tmp block, because of not support update-in-place",
|
STORAGE_LOG(WARN, "block status is not correct", K(ret), K(io_info));
|
||||||
K(ret), K(io_info));
|
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|||||||
@ -189,7 +189,7 @@ public:
|
|||||||
bool is_empty() const { return page_buddy_.is_empty(); }
|
bool is_empty() const { return page_buddy_.is_empty(); }
|
||||||
int close(bool &is_all_close);
|
int close(bool &is_all_close);
|
||||||
int is_extents_closed(bool &is_extents_closed);
|
int is_extents_closed(bool &is_extents_closed);
|
||||||
int give_back_buf_into_cache();
|
int give_back_buf_into_cache(const bool is_wash = false);
|
||||||
|
|
||||||
TO_STRING_KV(KP_(buffer), K_(page_buddy), K_(handle), K_(macro_block_handle), K_(tmp_file_header),
|
TO_STRING_KV(KP_(buffer), K_(page_buddy), K_(handle), K_(macro_block_handle), K_(tmp_file_header),
|
||||||
K_(io_desc), K_(block_status), K_(is_inited), K_(alloc_time), K_(access_time));
|
K_(io_desc), K_(block_status), K_(is_inited), K_(alloc_time), K_(access_time));
|
||||||
|
|||||||
Reference in New Issue
Block a user