diff --git a/deps/oblib/src/lib/allocator/ob_slice_alloc.h b/deps/oblib/src/lib/allocator/ob_slice_alloc.h index d9816ca631..ffaf6210cd 100644 --- a/deps/oblib/src/lib/allocator/ob_slice_alloc.h +++ b/deps/oblib/src/lib/allocator/ob_slice_alloc.h @@ -293,7 +293,7 @@ private: class ObSliceAlloc { public: - enum { MAX_ARENA_NUM = 32, MAX_REF_NUM = 4096, DEFAULT_BLOCK_SIZE = OB_MALLOC_NORMAL_BLOCK_SIZE }; + enum { MAX_ARENA_NUM = 32, MAX_REF_NUM = 128, DEFAULT_BLOCK_SIZE = OB_MALLOC_NORMAL_BLOCK_SIZE }; typedef ObSimpleSync Sync; typedef ObBlockSlicer Block; typedef ObBlockAllocMgr BlockAlloc; diff --git a/mittest/mtlenv/mock_tenant_module_env.h b/mittest/mtlenv/mock_tenant_module_env.h index 86f696697e..6ad735fbd0 100644 --- a/mittest/mtlenv/mock_tenant_module_env.h +++ b/mittest/mtlenv/mock_tenant_module_env.h @@ -95,6 +95,7 @@ #include "observer/table/ob_table_session_pool.h" #include "share/index_usage/ob_index_usage_info_mgr.h" #include "storage/tenant_snapshot/ob_tenant_snapshot_service.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" // ObTenantTmpFileManager #include "storage/memtable/ob_lock_wait_mgr.h" namespace oceanbase @@ -658,6 +659,10 @@ int MockTenantModuleEnv::init_before_start_mtl() STORAGE_LOG(WARN, "fail to init env", K(ret)); } else if (OB_FAIL(oceanbase::palf::election::GLOBAL_INIT_ELECTION_MODULE())) { STORAGE_LOG(WARN, "fail to init env", K(ret)); + } else if (OB_FAIL(tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1))) { + STORAGE_LOG(WARN, "init tmp block cache failed", KR(ret)); + } else if (OB_FAIL(tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1))) { + STORAGE_LOG(WARN, "init tmp page cache failed", KR(ret)); } else if (OB_SUCCESS != (ret = bandwidth_throttle_.init(1024 * 1024 * 60))) { STORAGE_LOG(ERROR, "failed to init bandwidth_throttle_", K(ret)); } else if (OB_FAIL(TG_START(lib::TGDefIDs::ServerGTimer))) { @@ -691,6 +696,7 @@ int MockTenantModuleEnv::init() } else { oceanbase::ObClusterVersion::get_instance().update_data_version(DATA_CURRENT_VERSION); MTL_BIND2(ObTenantIOManager::mtl_new, ObTenantIOManager::mtl_init, mtl_start_default, mtl_stop_default, nullptr, ObTenantIOManager::mtl_destroy); + MTL_BIND2(mtl_new_default, tmp_file::ObTenantTmpFileManager::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); MTL_BIND2(mtl_new_default, omt::ObSharedTimer::mtl_init, omt::ObSharedTimer::mtl_start, omt::ObSharedTimer::mtl_stop, omt::ObSharedTimer::mtl_wait, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObTenantSchemaService::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObStorageLogger::mtl_init, ObStorageLogger::mtl_start, ObStorageLogger::mtl_stop, ObStorageLogger::mtl_wait, mtl_destroy_default); @@ -753,8 +759,6 @@ int MockTenantModuleEnv::init() STORAGE_LOG(ERROR, "reload memory config failed", K(ret)); } else if (OB_FAIL(start_())) { STORAGE_LOG(ERROR, "mock env start failed", K(ret)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().init())) { - STORAGE_LOG(WARN, "init_tmp_file_manager failed", K(ret)); } else { inited_ = true; } @@ -853,7 +857,6 @@ void MockTenantModuleEnv::destroy() ObKVGlobalCache::get_instance().destroy(); ObServerCheckpointSlogHandler::get_instance().destroy(); SLOGGERMGR.destroy(); - ObTmpFileManager::get_instance().destroy(); OB_SERVER_BLOCK_MGR.stop(); OB_SERVER_BLOCK_MGR.wait(); @@ -867,7 +870,8 @@ void MockTenantModuleEnv::destroy() net_frame_.stop(); net_frame_.wait(); net_frame_.destroy(); - + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); TG_STOP(lib::TGDefIDs::ServerGTimer); TG_WAIT(lib::TGDefIDs::ServerGTimer); TG_DESTROY(lib::TGDefIDs::ServerGTimer); diff --git a/mittest/mtlenv/storage/CMakeLists.txt b/mittest/mtlenv/storage/CMakeLists.txt index faec8222c6..13abc2993c 100644 --- a/mittest/mtlenv/storage/CMakeLists.txt +++ b/mittest/mtlenv/storage/CMakeLists.txt @@ -42,5 +42,6 @@ add_subdirectory(checkpoint) add_subdirectory(blocksstable) add_subdirectory(tenant_snapshot) add_subdirectory(tablet_memtable) +add_subdirectory(tmp_file) target_link_libraries(test_memtable PUBLIC mock_tx_ctx mock_tx_log_adapter) diff --git a/mittest/mtlenv/storage/test_lob_manager.cpp b/mittest/mtlenv/storage/test_lob_manager.cpp index 1e33c2f28b..d37acf4fa6 100644 --- a/mittest/mtlenv/storage/test_lob_manager.cpp +++ b/mittest/mtlenv/storage/test_lob_manager.cpp @@ -23,7 +23,6 @@ #include "lib/random/ob_random.h" #include "storage/blocksstable/ob_data_file_prepare.h" #include "share/ob_simple_mem_limit_getter.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "storage/lob/ob_lob_piece.h" #include "sql/engine/ob_exec_context.h" #include "lib/objectpool/ob_server_object_pool.h" diff --git a/mittest/mtlenv/storage/tmp_file/CMakeLists.txt b/mittest/mtlenv/storage/tmp_file/CMakeLists.txt new file mode 100644 index 0000000000..ae96ad2a51 --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/CMakeLists.txt @@ -0,0 +1,6 @@ +storage_unittest(test_tmp_file) +storage_unittest(test_tmp_file_meta_tree) +storage_unittest(test_tmp_file_write_buffer_pool_index_cache) +storage_unittest(test_tmp_file_buffer_pool) +storage_unittest(test_tmp_file_flush_list) +storage_unittest(test_tmp_file_block_manager) diff --git a/mittest/mtlenv/storage/tmp_file/ob_tmp_file_test_helper.h b/mittest/mtlenv/storage/tmp_file/ob_tmp_file_test_helper.h new file mode 100644 index 0000000000..643277a2f0 --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/ob_tmp_file_test_helper.h @@ -0,0 +1,510 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OB_TMP_FILE_TEST_HELPER_ +#define OB_TMP_FILE_TEST_HELPER_ +#include +#include +#include +#include +#include +#include +#include +#define USING_LOG_PREFIX STORAGE +#define protected public +#define private public +#include "share/ob_thread_pool.h" +#include "share/rc/ob_tenant_base.h" +#include "storage/tmp_file/ob_tmp_file_io_define.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" + +namespace oceanbase +{ +using namespace common; +using namespace tmp_file; +using namespace share; +/* ------------------------------ Test Helper ------------------------------ */ +void print_hex_data(const char *buffer, int64_t length) +{ + std::cout << std::hex << std::setfill('0'); + for (int64_t i = 0; i < length; ++i) { + std::cout << std::setw(2) << static_cast(static_cast(buffer[i])); + } + std::cout << std::dec << std::endl; +} + +void dump_hex_data(const char *buffer, int length, const std::string &filename) +{ + static SpinRWLock lock_; + SpinWLockGuard guard(lock_); + std::ifstream ifile(filename); + if (ifile) { + } else { + std::ofstream file(filename, std::ios::out | std::ios::binary); + if (file.is_open()) { + for (int i = 0; i < length; ++i) { + if (i != 0 && i % 16 == 0) { + file << std::endl; + } else if (i != 0 && i % 2 == 0) { + file << " "; + } + file << std::hex << std::setw(2) << std::setfill('0') + << (static_cast(buffer[i]) & 0xFF); + } + file.close(); + std::cout << "Data has been written to " << filename << " in hex format." << std::endl; + } else { + std::cerr << "Error opening file " << filename << " for writing." << std::endl; + } + } +} + +bool compare_and_print_hex_data(const char *lhs, const char *rhs, + int64_t buf_length, int64_t print_length, + std::string &filename) +{ + bool is_equal = true; + static SpinRWLock lock_; + SpinWLockGuard guard(lock_); + static int64_t idx = 0; + filename.clear(); + filename = std::to_string(ATOMIC_FAA(&idx, 1)) + "_cmp_and_dump_hex_data.txt"; + std::ofstream file(filename, std::ios::out | std::ios::binary); + if (file.is_open()) { + for (int i = 0; i < buf_length; ++i) { + if (lhs[i] != rhs[i]) { + is_equal = false; + int64_t print_begin = i - print_length / 2 >= 0 ? i - print_length / 2 : 0; + int64_t print_end = print_begin + print_length < buf_length ? print_begin + print_length : buf_length; + file << "First not equal happen at " << i + << ", print length: " << print_end - print_begin + << ", print begin: " << print_begin + << ", print end: " << print_end << std::endl; + file << std::endl << "lhs:" << std::endl; + { + const char *buffer = lhs + print_begin; + int64_t length = print_end - print_begin; + for (int64_t i = 0; i < length; ++i) { + file << std::hex << std::setw(2) << std::setfill('0') << (static_cast(buffer[i]) & 0xFF); + } + } + file << std::endl << "rhs:" << std::endl; + { + const char *buffer = rhs + print_begin; + int64_t length = print_end - print_begin; + for (int64_t i = 0; i < length; ++i) { + file << std::hex << std::setw(2) << std::setfill('0') << (static_cast(buffer[i]) & 0xFF); + } + } + std::cout << "not equal at " << i << std::endl; + break; + } + } + file.close(); + } else { + std::cerr << "Error opening file " << filename << " for writing." << std::endl; + } + return is_equal; +} + +int64_t generate_random_int(const int64_t lower_bound, const int64_t upper_bound) +{ + std::random_device rd; + std::mt19937 gen(rd()); + std::uniform_int_distribution dis(lower_bound, upper_bound); + int64_t random_number = dis(gen); + return random_number; +} + +std::vector generate_random_sequence(const int64_t lower_bound, + const int64_t upper_bound, + const int64_t sequence_sum, + unsigned seed = std::random_device{}()) +{ + std::mt19937 gen(seed); + std::uniform_int_distribution dis(lower_bound, upper_bound); + std::vector random_sequence; + int64_t sum = 0; + while (sum < sequence_sum) { + int64_t rand_num = std::min(sequence_sum - sum, dis(gen)); + random_sequence.push_back(rand_num); + sum += rand_num; + } + return random_sequence; +} + +/* -------------------------- TestTmpFileStress --------------------------- */ +enum TmpFileOp { + WRITE, + READ, + TRUNCATE, + OP_MAX +}; + +class TestTmpFileStress : public share::ObThreadPool +{ +public: + TestTmpFileStress(ObTenantBase *tenant_ctx); + virtual ~TestTmpFileStress(); + int init(const int fd, const TmpFileOp op, const int64_t thread_cnt, + char *buf, const int64_t offset, const int64_t size); + void reset(); + virtual void run1(); + TO_STRING_KV(K_(thread_cnt), K_(fd), K_(op), KP_(buf), K_(offset), K_(size)); +private: + void write_data_(const int64_t write_size); + void truncate_data_(); + void read_data_(const int64_t read_offset, const int64_t read_size); +private: + int64_t thread_cnt_; + int fd_; + TmpFileOp op_; + char *buf_; + int64_t offset_; + int64_t size_; + ObTenantBase *tenant_ctx_; +}; + +TestTmpFileStress::TestTmpFileStress(ObTenantBase *tenant_ctx) + : thread_cnt_(0), fd_(0), + op_(OP_MAX), + buf_(nullptr), offset_(0), + size_(0), + tenant_ctx_(tenant_ctx) +{ +} + +TestTmpFileStress::~TestTmpFileStress() +{ +} + +int TestTmpFileStress::init(const int fd, const TmpFileOp op, + const int64_t thread_cnt, + char *buf, int64_t offset, + const int64_t size) +{ + int ret = OB_SUCCESS; + if (thread_cnt < 0 || OB_ISNULL(buf) || offset < 0 || size <= 0) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(ret), K(thread_cnt), KP(buf), K(offset), K(size)); + } else if (TmpFileOp::OP_MAX == op) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(ret), K(op)); + } else if ((op == TmpFileOp::WRITE || op == TmpFileOp::TRUNCATE) && 1 != thread_cnt) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(ret), K(op), K(thread_cnt)); + } else { + buf_ = buf; + thread_cnt_ = thread_cnt; + fd_ = fd; + op_ = op; + offset_ = offset; + size_ = size; + set_thread_count(static_cast(thread_cnt)); + } + return ret; +} + +void TestTmpFileStress::reset() +{ + thread_cnt_ = 0; + fd_ = 0; + op_ = OP_MAX; + buf_ = nullptr; + offset_ = 0; + size_ = 0; +} + +void TestTmpFileStress::write_data_(const int64_t write_size) +{ + STORAGE_LOG(INFO, "TestTmpFileStress write thread", K(fd_), K(thread_idx_), KP(buf_), K(size_)); + int ret = OB_SUCCESS; + ObArray size_array; + ObTmpFileIOInfo io_info; + ASSERT_EQ(OB_SUCCESS, ret); + io_info.fd_ = fd_; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + int64_t already_write = 0; + std::vector turn_write_size = generate_random_sequence(1, write_size / 3, write_size, 3); + for (int i = 0; i < turn_write_size.size(); ++i) { + int64_t this_turn_write_size = turn_write_size[i]; + STORAGE_LOG(INFO, "random write size", K(fd_), K(thread_idx_), KP(buf_), K(size_), K(this_turn_write_size)); + // write data + io_info.buf_ = buf_ + already_write; + if (this_turn_write_size % ObTmpFileGlobal::PAGE_SIZE == 0 && i == 0) { + io_info.size_ = this_turn_write_size - 2 * 1024; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + + io_info.size_ = 2 * 1024; + io_info.buf_ = buf_ + already_write + this_turn_write_size - 2 * 1024; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + } else { + io_info.size_ = this_turn_write_size; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + } + already_write += this_turn_write_size; + } + + ASSERT_EQ(OB_SUCCESS, ret); + STORAGE_LOG(INFO, "TestTmpFileStress write thread finished", K(fd_), K(thread_idx_), KP(buf_), K(size_)); +} + +void TestTmpFileStress::read_data_(const int64_t read_offset, const int64_t read_size) +{ + STORAGE_LOG(INFO, "TestTmpFileStress read thread start", K(fd_), K(thread_idx_), KP(buf_), K(read_offset), K(read_size)); + int ret = OB_SUCCESS; + char *read_buf = new char[read_size]; + ObTmpFileIOInfo io_info; + ObTmpFileIOHandle handle; + io_info.fd_ = fd_; + io_info.size_ = read_size; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + io_info.buf_ = read_buf; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + int cmp = memcmp(handle.get_buffer(), buf_ + read_offset, io_info.size_); + if (cmp != 0 || OB_FAIL(ret)) { + STORAGE_LOG(WARN, "TestTmpFileStress read thread failed", KR(ret), K(fd_), K(cmp), K(thread_idx_), KP(buf_), K(read_offset), K(read_size)); + ob_abort(); + } + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + STORAGE_LOG(INFO, "TestTmpFileStress read thread finished", K(fd_), K(thread_idx_), KP(buf_), K(read_offset), K(read_size)); +} + +void TestTmpFileStress::truncate_data_() +{ + int64_t truncate_offset = offset_ + MIN(size_, MAX(size_ / 10, 8 * 1024)); + STORAGE_LOG(INFO, "TestTmpFileStress truncate thread start", K(fd_), K(thread_idx_), KP(buf_), + K(truncate_offset), K(offset_), K(size_)); + int ret = MTL(ObTenantTmpFileManager *)->truncate(fd_, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileIOInfo io_info; + io_info.fd_ = fd_; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + const int64_t invalid_size = truncate_offset - offset_; + const int64_t valid_size = size_ - invalid_size; + + char *zero_buf = new char[invalid_size]; + MEMSET(zero_buf, 0, invalid_size); + char *read_buf = new char[size_]; + io_info.size_ = size_; + io_info.buf_ = read_buf; + ObTmpFileIOHandle handle; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, offset_, handle); + int cmp = memcmp(handle.get_buffer()+invalid_size, buf_ + truncate_offset, valid_size); + if (cmp != 0 || OB_FAIL(ret)) { + STORAGE_LOG(INFO, "TestTmpFileStress truncate thread failed. " + "fail to compare valid part.", KR(ret), K(cmp), K(fd_), K(thread_idx_), KP(buf_), + K(truncate_offset), K(valid_size), K(invalid_size), K(offset_), K(size_)); + ob_abort(); + } + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, cmp); + cmp = memcmp(handle.get_buffer(), zero_buf, invalid_size); + if (cmp != 0) { + STORAGE_LOG(INFO, "TestTmpFileStress truncate thread failed. " + "fail to compare zero part.", KR(ret), K(cmp), K(fd_), K(thread_idx_), KP(buf_), + K(truncate_offset), K(valid_size), K(invalid_size), K(offset_), K(size_)); + } + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + delete[] zero_buf; + + truncate_offset = offset_ + size_; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd_, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + + zero_buf = new char[size_]; + MEMSET(zero_buf, 0, size_); + read_buf = new char[size_]; + io_info.buf_ = read_buf; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, offset_, handle); + cmp = memcmp(handle.get_buffer(), zero_buf, size_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + delete[] zero_buf; + STORAGE_LOG(INFO, "TestTmpFileStress truncate thread finished", K(fd_), K(thread_idx_), KP(buf_), K(offset_), K(size_)); +} + +void TestTmpFileStress::run1() +{ + ObTenantEnv::set_tenant(tenant_ctx_); + common::ObCurTraceId::TraceId trace_id; + ObCurTraceId::TraceId *cur_trace_id = ObCurTraceId::get_trace_id(); + if (nullptr != cur_trace_id && cur_trace_id->is_valid()) { + trace_id = *cur_trace_id; + LOG_INFO("init TestTmpFileStress with an old trace_id", KPC(cur_trace_id), KPC(this)); + } else { + trace_id.init(GCONF.self_addr_); + LOG_INFO("init TestTmpFileStress with a new trace_id", K(trace_id), KPC(this)); + } + ObTraceIDGuard trace_guard(trace_id); + + if (op_ == TmpFileOp::WRITE) { + write_data_(size_); + } else if (op_ == TmpFileOp::READ) { + int64_t read_offset = offset_ + (size_ / thread_cnt_) * thread_idx_; + int64_t read_size = 0; + if (thread_idx_ == thread_cnt_ - 1) { + read_size = size_ / thread_cnt_ + size_ % thread_cnt_; + } else { + read_size = size_ / thread_cnt_; + } + read_data_(read_offset, read_size); + } else { + truncate_data_(); + } +} + +/* -------------------------- TestMultiTmpFileStress --------------------------- */ +class TestMultiTmpFileStress : public share::ObThreadPool +{ +public: + TestMultiTmpFileStress(ObTenantBase *tenant_ctx); + virtual ~TestMultiTmpFileStress(); + int init(const int64_t file_cnt, const int64_t dir_id, const int64_t thread_cnt, + const int64_t batch_size, const int64_t batch_num); + virtual void run1(); +private: + int64_t file_cnt_; + int64_t dir_id_; + int64_t read_thread_cnt_perf_file_; + int64_t batch_size_; + int64_t batch_num_; + ObTenantBase *tenant_ctx_; +}; + +TestMultiTmpFileStress::TestMultiTmpFileStress(ObTenantBase *tenant_ctx) + : file_cnt_(0), + dir_id_(-1), + read_thread_cnt_perf_file_(0), + batch_size_(0), + batch_num_(0), + tenant_ctx_(tenant_ctx) +{ +} + +TestMultiTmpFileStress::~TestMultiTmpFileStress() +{ +} + +int TestMultiTmpFileStress::init(const int64_t file_cnt, + const int64_t dir_id, + const int64_t thread_cnt, + const int64_t batch_size, + const int64_t batch_num) +{ + int ret = OB_SUCCESS; + if (file_cnt < 0 || thread_cnt < 0) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", K(ret), K(file_cnt), K(thread_cnt)); + } else { + file_cnt_ = file_cnt; + dir_id_ = dir_id; + read_thread_cnt_perf_file_ = thread_cnt; + batch_size_ = batch_size; + batch_num_ = batch_num; + set_thread_count(static_cast(file_cnt)); + } + return ret; +} + +void TestMultiTmpFileStress::run1() +{ + STORAGE_LOG(INFO, "TestMultiTmpFileStress thread run start"); + int ret = OB_SUCCESS; + int64_t fd = 0; + ObTenantEnv::set_tenant(tenant_ctx_); + + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir_id_); + std::cout << "normal case, fd: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + STORAGE_LOG(INFO, "open file success", K(fd)); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = ObTmpFileWBPIndexCache::INIT_BUCKET_ARRAY_CAPACITY * 2; + file_handle.reset(); + + int64_t file_size = batch_size_ * batch_num_; + char * data_buffer = new char[file_size]; + for (int64_t i = 0; i < file_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < file_size; ++j) { + data_buffer[i + j] = random_int; + } + i += random_length; + } + + + TestTmpFileStress test_truncate(tenant_ctx_); + for (int64_t i = 0; i < batch_num_; ++i) { + if (i > 0) { + // truncate read data in previous round + test_truncate.init(fd, TmpFileOp::TRUNCATE, 1, data_buffer, (i-1) * batch_size_, batch_size_); + ASSERT_EQ(OB_SUCCESS, ret); + STORAGE_LOG(INFO, "test_truncate run start", K(i), K(batch_size_)); + test_truncate.start(); + } + TestTmpFileStress test_write(tenant_ctx_); + ret = test_write.init(fd, TmpFileOp::WRITE, 1, data_buffer + i * batch_size_, 0, batch_size_); + ASSERT_EQ(OB_SUCCESS, ret); + STORAGE_LOG(INFO, "test_write run start"); + test_write.start(); + test_write.wait(); + STORAGE_LOG(INFO, "test_write run end"); + + TestTmpFileStress test_read(tenant_ctx_); + ret = test_read.init(fd, TmpFileOp::READ, read_thread_cnt_perf_file_, data_buffer, i * batch_size_, batch_size_); + ASSERT_EQ(OB_SUCCESS, ret); + + STORAGE_LOG(INFO, "test_read run start", K(i), K(batch_size_)); + test_read.start(); + test_read.wait(); + STORAGE_LOG(INFO, "test_read run end"); + + if (i > 0) { + // wait to truncate read data in last round + test_truncate.wait(); + test_truncate.reset(); + STORAGE_LOG(INFO, "test_truncate run end", K(i)); + } + + STORAGE_LOG(INFO, "TestMultiTmpFileStress thread run a batch end", K(i)); + } + + test_truncate.init(fd, TmpFileOp::TRUNCATE, 1, data_buffer, file_size - batch_size_, batch_size_); + ASSERT_EQ(OB_SUCCESS, ret); + STORAGE_LOG(INFO, "test_truncate run start"); + test_truncate.start(); + test_truncate.wait(); + STORAGE_LOG(INFO, "test_truncate run end"); + + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + + delete[] data_buffer; + STORAGE_LOG(INFO, "TestMultiTmpFileStress thread run end"); +} + +} // namespace oceanbase + +#endif diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file.cpp new file mode 100644 index 0000000000..8e745dc6cf --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file.cpp @@ -0,0 +1,1354 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include "mittest/mtlenv/storage/tmp_file/ob_tmp_file_test_helper.h" +#define USING_LOG_PREFIX STORAGE +#define protected public +#define private public +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "share/ob_simple_mem_limit_getter.h" +#include "lib/alloc/ob_malloc_allocator.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_meta_tree.h" + +namespace oceanbase +{ +using namespace common; +using namespace blocksstable; +using namespace tmp_file; +using namespace storage; +using namespace share::schema; +/* ------------------------------ Mock Parameter ---------------------------- */ +static const int64_t TENANT_MEMORY = 8L * 1024L * 1024L * 1024L /* 8 GB */; +/********************************* Mock WBP *************************** */ +static const int64_t WBP_BLOCK_SIZE = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; // each wbp block has 253 pages (253 * 8KB == 2024KB) +static const int64_t SMALL_WBP_BLOCK_COUNT = 3; +static const int64_t SMALL_WBP_MEM_LIMIT = SMALL_WBP_BLOCK_COUNT * WBP_BLOCK_SIZE; // the wbp mem size is 5.93MB +static const int64_t BIG_WBP_BLOCK_COUNT = 40; +static const int64_t BIG_WBP_MEM_LIMIT = BIG_WBP_BLOCK_COUNT * WBP_BLOCK_SIZE; // the wbp mem size is 79.06MB +/********************************* Mock WBP Index Cache*************************** */ +// each bucket could indicate a 256KB data in wbp. +// SMALL_WBP_IDX_CACHE_MAX_CAPACITY will indicate 4MB data in wbp +static const int64_t SMALL_WBP_IDX_CACHE_MAX_CAPACITY = ObTmpFileWBPIndexCache::INIT_BUCKET_ARRAY_CAPACITY * 2; +/********************************* Mock Meta Tree *************************** */ +static const int64_t MAX_DATA_ITEM_ARRAY_COUNT = 2; +static const int64_t MAX_PAGE_ITEM_COUNT = 4; // MAX_PAGE_ITEM_COUNT * ObTmpFileGlobal::PAGE_SIZE means + // the max representation range of a meta page (4 * 2MB == 8MB). + // according to the formula of summation for geometric sequence + // (S_n = a_1 * (1-q^n)/(1-q), where a_1 = 8MB, q = 4), + // a two-level meta tree could represent at most 40MB disk data of tmp file + // a three-level meta tree could represent at most 168MB disk data of tmp file + // a four-level meta tree could represent at most 680MB disk data of tmp file + +/* ---------------------------- Unittest Class ----------------------------- */ + +class TestTmpFile : public ::testing::Test +{ +public: + TestTmpFile() = default; + virtual ~TestTmpFile() = default; + virtual void SetUp(); + virtual void TearDown(); + static void SetUpTestCase(); + static void TearDownTestCase(); +}; +static ObSimpleMemLimitGetter getter; +static const int64_t TEST_ROWKEY_COLUMN_CNT = 2; + +// ATTENTION! +// currently, we only initialize modules about tmp file at the beginning of unit test and +// never restart them in the end of test case. +// please make sure that all test cases will not affect the others. +void TestTmpFile::SetUpTestCase() +{ + int ret = OB_SUCCESS; + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); + + CHUNK_MGR.set_limit(TENANT_MEMORY); + ObMallocAllocator::get_instance()->set_tenant_limit(MTL_ID(), TENANT_MEMORY); + + MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = SMALL_WBP_MEM_LIMIT; + ObSharedNothingTmpFileMetaTree::set_max_array_item_cnt(MAX_DATA_ITEM_ARRAY_COUNT); + ObSharedNothingTmpFileMetaTree::set_max_page_item_cnt(MAX_PAGE_ITEM_COUNT); +} + +void TestTmpFile::SetUp() +{ + int ret = OB_SUCCESS; + ASSERT_EQ(true, MockTenantModuleEnv::get_instance().is_inited()); +// if (!MTL(ObTenantTmpFileManager *)->is_inited_) { +// ret = MTL(ObTenantTmpFileManager *)->init(); +// ASSERT_EQ(OB_SUCCESS, ret); +// ret = MTL(ObTenantTmpFileManager *)->start(); +// ASSERT_EQ(OB_SUCCESS, ret); +// MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = SMALL_WBP_MEM_LIMIT; +// } +} + +void TestTmpFile::TearDownTestCase() +{ + MockTenantModuleEnv::get_instance().destroy(); +} + +void TestTmpFile::TearDown() +{ +// if (MTL(ObTenantTmpFileManager *)->is_inited_) { +// MTL(ObTenantTmpFileManager *)->stop(); +// MTL(ObTenantTmpFileManager *)->wait(); +// MTL(ObTenantTmpFileManager *)->destroy(); +// } +} + +// generate 2MB random data (will not trigger flush and evict logic) +// 1. test write pages and append write tail page +// 2. test write after reading +TEST_F(TestTmpFile, test_unaligned_data_read_write) +{ + int ret = OB_SUCCESS; + const int64_t write_size = 2 * 1024 * 1024; + const int64_t wbp_mem_limit = MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.get_memory_limit(); + ASSERT_LT(write_size, wbp_mem_limit); + char * write_buffer = new char[write_size]; + for (int64_t i = 0; i < write_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < write_size; ++j) { + write_buffer[i + j] = random_int; + } + i += random_length; + } + int64_t dir = -1; + int64_t fd = -1; + const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + ObTmpFileIOInfo io_info; + ObTmpFileIOHandle handle; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + // dump random data + { + std::string r_file_name = std::to_string(fd) + "_raw_write_data.txt"; + dump_hex_data(write_buffer, write_size, r_file_name); + } + + // random write, read, and check + int64_t already_write = 0; + std::vector turn_write_size = generate_random_sequence(1, write_size / 3, write_size, 3); + for (int i = 0; i < turn_write_size.size(); ++i) { + int64_t this_turn_write_size = turn_write_size[i]; + std::cout << "random write and read " << this_turn_write_size << std::endl; + // write data + { + ObTmpFileIOInfo io_info; + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + io_info.buf_ = write_buffer + already_write; + if (this_turn_write_size % ObTmpFileGlobal::PAGE_SIZE == 0 && i == 0) { + io_info.size_ = this_turn_write_size - 2 * 1024; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + + io_info.size_ = 2 * 1024; + io_info.buf_ = write_buffer + already_write + this_turn_write_size - 2 * 1024; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + } else { + io_info.size_ = this_turn_write_size; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->write(io_info)); + } + } + // read data + char * read_check_buffer = new char[this_turn_write_size]; + { + ObTmpFileIOInfo io_info; + ObTmpFileIOHandle handle; + io_info.fd_ = fd; + io_info.size_ = this_turn_write_size; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + io_info.buf_ = read_check_buffer; + ASSERT_EQ(OB_SUCCESS, MTL(ObTenantTmpFileManager *)->read(io_info, handle)); + } + // check data + { + std::string compare_file_name = std::to_string(fd) + "_compare_result.txt"; + bool is_equal = compare_and_print_hex_data( + write_buffer + already_write, read_check_buffer, + this_turn_write_size, 200, compare_file_name); + if (!is_equal) { + // dump write data + std::string w_file_name = std::to_string(fd) + "_write_data.txt"; + dump_hex_data(write_buffer + already_write, this_turn_write_size, w_file_name); + // dump read check data + std::string r_file_name = std::to_string(fd) + "_read_data.txt"; + dump_hex_data(read_check_buffer, this_turn_write_size, r_file_name); + // abort + std::cout << "not equal in random data test" + << "\nwrite dumped file: " << w_file_name + << "\nread check dumped file: " << r_file_name + << "\ncompare result file: " << compare_file_name << std::endl; + ob_abort(); + } + } + // update already_write + delete [] read_check_buffer; + already_write += this_turn_write_size; + } + + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + LOG_INFO("test_unaligned_data_read_write"); +} + +// generate 7MB random data +// this test will trigger flush and evict logic for data pages. +// meta tree will not be evicted in this test. +// 1. test pread +// 1.1 read disk data +// 1.2 read memory data +// 1.3 read both disk and memory data +// 1.4 read OB_ITER_END +// 2. test read +// 2.1 read aligned data +// 2.2 read unaligned data +TEST_F(TestTmpFile, test_read) +{ + int ret = OB_SUCCESS; + const int64_t write_size = 7 * 1024 * 1024; // 7MB + const int64_t wbp_mem_limit = MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.get_memory_limit(); + ASSERT_GT(write_size, wbp_mem_limit); + char *write_buf = new char [write_size]; + for (int64_t i = 0; i < write_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < write_size; ++j) { + write_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir = -1; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + + ObTmpFileIOInfo io_info; + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.buf_ = write_buf; + io_info.size_ = write_size; + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + // Write data + int64_t write_time = ObTimeUtility::current_time(); + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + write_time = ObTimeUtility::current_time() - write_time; + ASSERT_EQ(OB_SUCCESS, ret); + + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t wbp_begin_offset = file_handle.get()->cal_wbp_begin_offset(); + ASSERT_GT(wbp_begin_offset, 0); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + + int64_t read_time = ObTimeUtility::current_time(); + /************** test pread **************/ + // 1. read memory data + char *read_buf = new char [write_size - wbp_begin_offset]; + ObTmpFileIOHandle handle; + io_info.buf_ = read_buf; + io_info.size_ = write_size - wbp_begin_offset; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, wbp_begin_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + int cmp = memcmp(handle.get_buffer(), write_buf + wbp_begin_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 2. read disk data + read_buf = new char [wbp_begin_offset]; + io_info.buf_ = read_buf; + io_info.size_ = wbp_begin_offset; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, 0, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 3. read both disk and memory data + int64_t read_size = wbp_begin_offset / 2 + 9 * 1024; + int64_t read_offset = wbp_begin_offset / 2 + 1024; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 4. read OB_ITER_END + read_buf = new char [200]; + io_info.buf_ = read_buf; + io_info.size_ = 200; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, write_size - 100, handle); + ASSERT_EQ(OB_ITER_END, ret); + ASSERT_EQ(100, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + write_size - 100, 100); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + /************** test read **************/ + // 1. read aligned data + read_buf = new char [3 * ObTmpFileGlobal::PAGE_SIZE]; + io_info.buf_ = read_buf; + io_info.size_ = 3 * ObTmpFileGlobal::PAGE_SIZE; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + // 2. read unaligned data + read_buf = new char [ObTmpFileGlobal::PAGE_SIZE]; + io_info.buf_ = read_buf; + io_info.size_ = 100; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + 3 * ObTmpFileGlobal::PAGE_SIZE, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + + io_info.buf_ = read_buf + 100; + io_info.size_ = ObTmpFileGlobal::PAGE_SIZE - 100; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + 3 * ObTmpFileGlobal::PAGE_SIZE + 100, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + read_time = ObTimeUtility::current_time() - read_time; + + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + LOG_INFO("test_read"); + LOG_INFO("io time", K(write_time), K(read_time)); +} + +// generate 8MB random data +// this test will check whether kv_cache caches correct pages in disk +TEST_F(TestTmpFile, test_cached_read) +{ + int ret = OB_SUCCESS; + const int64_t write_size = 8 * 1024 * 1024; // 8MB + const int64_t wbp_mem_limit = MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.get_memory_limit(); + ASSERT_GT(write_size, wbp_mem_limit); + char *write_buf = new char [write_size]; + for (int64_t i = 0; i < write_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < write_size; ++j) { + write_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir = -1; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + + ObTmpFileIOInfo io_info; + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.buf_ = write_buf; + io_info.size_ = write_size; + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + + // 1. Write data and wait flushing over + int64_t write_time = ObTimeUtility::current_time(); + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + write_time = ObTimeUtility::current_time() - write_time; + ASSERT_EQ(OB_SUCCESS, ret); + sleep(2); + + int64_t wbp_begin_offset = file_handle.get()->cal_wbp_begin_offset(); + ASSERT_GT(wbp_begin_offset, 0); + ASSERT_EQ(wbp_begin_offset % ObTmpFileGlobal::PAGE_SIZE, 0); + + // 2. check block kv cache + common::ObArray data_items; + ret = file_handle.get()->meta_tree_.search_data_items(0,wbp_begin_offset, data_items); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, data_items.empty()); + for (int64_t i = 0; OB_SUCC(ret) && i < data_items.count(); i++) { + const int64_t block_index = data_items[i].block_index_; + ObTmpBlockValueHandle block_value_handle; + ret = ObTmpBlockCache::get_instance().get_block(ObTmpBlockCacheKey(block_index, MTL_ID()), + block_value_handle); + ASSERT_EQ(OB_SUCCESS, ret); + } + + // 3. read data from block kv cache + int64_t read_size = write_size; + int64_t read_offset = 0; + char *read_buf = new char [read_size]; + ObTmpFileIOHandle handle; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + io_info.disable_page_cache_ = true; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + int cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 4. clear block kv cache + for (int64_t i = 0; OB_SUCC(ret) && i < data_items.count(); i++) { + const int64_t block_index = data_items[i].block_index_; + ObTmpBlockValueHandle block_value_handle; + ret = ObTmpBlockCache::get_instance().erase(ObTmpBlockCacheKey(block_index, MTL_ID())); + ASSERT_EQ(OB_SUCCESS, ret); + } + + // 5. read disk data and puts them into kv_cache + int64_t read_time = ObTimeUtility::current_time(); + read_size = wbp_begin_offset - ObTmpFileGlobal::PAGE_SIZE; + read_offset = ObTmpFileGlobal::PAGE_SIZE / 2; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + io_info.disable_page_cache_ = false; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 6. read disk data to check whether kv_cache caches correct pages + read_size = wbp_begin_offset; + read_offset = 0; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + io_info.disable_page_cache_ = false; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + read_time = ObTimeUtility::current_time() - read_time; + + // 7. check pages in kv_cache + int64_t previous_virtual_page_id = data_items.at(0).virtual_page_id_; + for (int64_t i = 0; i < data_items.count() && OB_SUCC(ret); ++i) { + const ObSharedNothingTmpFileDataItem &data_item = data_items.at(i); + if (i > 0) { + ASSERT_GT(data_item.virtual_page_id_, previous_virtual_page_id); + previous_virtual_page_id = data_item.virtual_page_id_; + } + for (int64_t j = 0; j < data_item.physical_page_num_; j++) { + int64_t physical_page_id = data_item.physical_page_id_ + j; + ObTmpPageCacheKey key(data_item.block_index_, physical_page_id, MTL_ID()); + ObTmpPageValueHandle handle; + ret = ObTmpPageCache::get_instance().get_page(key, handle); + if (OB_FAIL(ret)) { + std::cout << "get cached page failed" << i <<" "<< data_item.block_index_<<" "<< physical_page_id << std::endl; + } + ASSERT_EQ(OB_SUCCESS, ret); + cmp = memcmp(handle.value_->get_buffer(), write_buf + (data_item.virtual_page_id_ + j) * ObTmpFileGlobal::PAGE_SIZE, ObTmpFileGlobal::PAGE_SIZE); + ASSERT_EQ(0, cmp); + } + } + + file_handle.reset(); + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + LOG_INFO("test_cached_read"); + LOG_INFO("io time", K(write_time), K(read_time)); +} + +// 1. append write a uncompleted tail page in memory +// 2. append write a uncompleted tail page in disk +TEST_F(TestTmpFile, test_write_tail_page) +{ + int ret = OB_SUCCESS; + const int64_t write_size = 10 * 1024; // 10KB + int64_t already_write_size = 0; + char *write_buf = new char [write_size]; + for (int64_t i = 0; i < write_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < write_size; ++j) { + write_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir = -1; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + + // 1. write 2KB data and check rightness of writing + ObTmpFileIOInfo io_info; + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.buf_ = write_buf; + io_info.size_ = 2 * 1024; // 2KB + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + already_write_size += io_info.size_; + + int64_t read_size = 2 * 1024; // 2KB + int64_t read_offset = 0; + char *read_buf = new char [read_size]; + ObTmpFileIOHandle handle; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + int cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 2. append write 2KB data in memory and check rightness of writing + io_info.buf_ = write_buf + 2 * 1024; // 2KB + io_info.size_ = 2 * 1024; // 2KB + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + already_write_size += io_info.size_; + + read_size = 4 * 1024; // 4KB + read_offset = 0; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 3. forcibly evict current page + ObTmpFilePageCacheController &pc_ctrl = MTL(ObTenantTmpFileManager *)->get_page_cache_controller(); + ATOMIC_SET(&pc_ctrl.flush_all_data_, true); + pc_ctrl.flush_tg_.notify_doing_flush(); + sleep(2); + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ret = file_handle.get()->page_cache_controller_->invoke_swap_and_wait(write_size); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t wbp_begin_offset = file_handle.get()->cal_wbp_begin_offset(); + ASSERT_EQ(wbp_begin_offset, already_write_size); + + // 4. read disk page and add it into kv_cache + read_offset = 5; + read_size = already_write_size - read_offset; // 4KB - 5B + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 5. append write 6KB data in memory and check rightness of writing + io_info.buf_ = write_buf + already_write_size; + io_info.size_ = write_size - already_write_size; // 6KB + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + already_write_size += io_info.size_; + + read_size = write_size; + read_offset = 0; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 6. forcibly evict all pages and read them from disk to check whether hit old cached page in kv_cache + pc_ctrl.flush_tg_.notify_doing_flush(); + sleep(2); + ret = file_handle.get()->page_cache_controller_->invoke_swap_and_wait(write_size); + ASSERT_EQ(OB_SUCCESS, ret); + wbp_begin_offset = file_handle.get()->cal_wbp_begin_offset(); + ASSERT_EQ(wbp_begin_offset, already_write_size); + ATOMIC_SET(&pc_ctrl.flush_all_data_, false); + + read_offset = 20; + read_size = write_size - read_offset; // 10KB - 20B + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + file_handle.reset(); + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + LOG_INFO("test_write_tail_page"); +} + +// 1. truncate special cases +// 2. truncate disk data (truncate_offset < wbp begin offset) +// 3. truncate memory data and disk data (wbp begin offset < truncate_offset < file_size_) +// 4. truncate() do nothing (truncate_offset < file's truncate_offset_) +// 5. invalid truncate_offset checking +TEST_F(TestTmpFile, test_tmp_file_truncate) +{ + int ret = OB_SUCCESS; + const int64_t data_size = 30 * 1024 * 1024; // 30MB + const int64_t wbp_mem_limit = MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.get_memory_limit(); + ASSERT_GT(data_size, wbp_mem_limit); + char *write_buf = new char [data_size]; + int64_t already_write_size = 0; + for (int64_t i = 0; i < data_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < data_size; ++j) { + write_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir = -1; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + + // 1. truncate special cases + // 1.1 truncate a file with several pages + // 1.1.1 write two pages and check rightness of writing + ObTmpFileIOInfo io_info; + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + io_info.buf_ = write_buf; + io_info.size_ = 2 * ObTmpFileGlobal::PAGE_SIZE; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + already_write_size += io_info.size_; + + int64_t read_offset = 0; + int64_t read_size = already_write_size; + char *read_buf = new char [read_size]; + ObTmpFileIOHandle handle; + io_info.buf_ = read_buf; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + int cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + MEMSET(read_buf, 0, read_size); + + // 1.1.2 truncate to the middle offset of the first page + ASSERT_EQ(file_handle.get()->cached_page_nums_, 2); + uint32_t begin_page_id = file_handle.get()->begin_page_id_; + uint32_t end_page_id = file_handle.get()->end_page_id_; + int64_t truncate_offset = ObTmpFileGlobal::PAGE_SIZE / 2; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(file_handle.get()->begin_page_id_, begin_page_id); + + // read_offset = 0; + // read_size = already_write_size; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + MEMSET(read_buf, 0, read_size); + + // 1.1.3 truncate the first page + truncate_offset = ObTmpFileGlobal::PAGE_SIZE; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(file_handle.get()->begin_page_id_, end_page_id); + ASSERT_EQ(file_handle.get()->cached_page_nums_, 1); + + // read_offset = 0; + // read_size = already_write_size; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + MEMSET(read_buf, 0, read_size); + + // 1.1.4 truncate whole pages + truncate_offset = already_write_size; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(file_handle.get()->begin_page_id_, ObTmpFileGlobal::INVALID_PAGE_ID); + ASSERT_EQ(file_handle.get()->cached_page_nums_, 0); + + // read_offset = 0; + // read_size = already_write_size; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 1.2 truncate a offset of a page whose page index is not in index cache (to mock the sparsify case of index cache) + // 1.2.1 write three pages and check rightness of writing + read_offset = already_write_size; + io_info.buf_ = write_buf + already_write_size; + io_info.size_ = 3 * ObTmpFileGlobal::PAGE_SIZE; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + already_write_size += io_info.size_; + + read_size = io_info.size_; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + delete[] read_buf; + + // 1.2.2 pop the first page index of index cache of file + ASSERT_NE(file_handle.get()->page_idx_cache_.page_buckets_, nullptr); + ASSERT_EQ(file_handle.get()->page_idx_cache_.size(), 1); + ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket *bucket = file_handle.get()->page_idx_cache_.page_buckets_->at(0); + ASSERT_NE(bucket, nullptr); + ASSERT_EQ(bucket->size(), 3); + begin_page_id = file_handle.get()->begin_page_id_; + end_page_id = file_handle.get()->end_page_id_; + ASSERT_EQ(bucket->page_indexes_.at(bucket->left_), begin_page_id); + ASSERT_EQ(bucket->page_indexes_.at(bucket->right_), end_page_id); + ret = bucket->pop_(); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(bucket->size(), 2); + ASSERT_NE(bucket->page_indexes_.at(bucket->left_), begin_page_id); + ASSERT_EQ(bucket->page_indexes_.at(bucket->right_), end_page_id); + + // 1.2.3 truncate the first page + ASSERT_EQ(file_handle.get()->cached_page_nums_, 3); + truncate_offset = read_offset + ObTmpFileGlobal::PAGE_SIZE; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(file_handle.get()->cached_page_nums_, 2); + + read_size = already_write_size - read_offset; + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 2. truncate disk data (truncate_offset < wbp begin offset) + read_offset = already_write_size; + io_info.buf_ = write_buf + already_write_size; + io_info.size_ = data_size - already_write_size; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t wbp_begin_offset = file_handle.get()->cal_wbp_begin_offset(); + ASSERT_GT(wbp_begin_offset, 0); + + truncate_offset = wbp_begin_offset/2; + read_size = wbp_begin_offset - read_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 3. truncate memory data (truncate_offset < file_size_) + // 3.1 truncate_offset is unaligned + read_offset = truncate_offset; + truncate_offset = (wbp_begin_offset + data_size) / 2 - ObTmpFileGlobal::PAGE_SIZE / 2; + read_size = data_size - read_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + read_offset = truncate_offset; + truncate_offset = upper_align(truncate_offset, ObTmpFileGlobal::PAGE_SIZE) + ObTmpFileGlobal::PAGE_SIZE; + read_size = data_size - read_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 3.2 truncate_offset is aligned + ASSERT_EQ(truncate_offset % ObTmpFileGlobal::PAGE_SIZE, 0); + read_offset = truncate_offset; + truncate_offset = truncate_offset + 5 * ObTmpFileGlobal::PAGE_SIZE; + read_size = data_size - read_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + read_offset = truncate_offset; + truncate_offset = data_size; + read_size = data_size - read_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + MEMSET(write_buf, 0, truncate_offset); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, read_size); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + + // 4. truncate() do nothing (truncate_offset < file's truncate_offset_) + int64_t old_truncate_offset = truncate_offset; + ASSERT_EQ(old_truncate_offset, file_handle.get()->truncated_offset_); + truncate_offset = wbp_begin_offset; + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, truncate_offset); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(old_truncate_offset, file_handle.get()->truncated_offset_); + + // 5. invalid truncate_offset checking + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, -1); + ASSERT_NE(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->truncate(fd, data_size + 10); + ASSERT_NE(OB_SUCCESS, ret); + + file_handle.reset(); + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + LOG_INFO("test_tmp_file_truncate"); +} + +// generate 750MB random data. +// this test will trigger flush and evict logic for both data and meta pages. +void test_big_file(const int64_t write_size, const int64_t wbp_mem_limit, ObTmpFileIOInfo io_info) +{ + int ret = OB_SUCCESS; + ASSERT_GT(write_size, wbp_mem_limit); + MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = wbp_mem_limit; + const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + int cmp = 0; + char *write_buf = (char *)malloc(write_size); + for (int64_t i = 0; i < write_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < write_size; ++j) { + write_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir = -1; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << " tenant_id:"<< MTL_ID() << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + io_info.fd_ = fd; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + + // 1. write 750MB data + io_info.buf_ = write_buf; + io_info.size_ = write_size; + int64_t write_time = ObTimeUtility::current_time(); + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + write_time = ObTimeUtility::current_time() - write_time; + + // 2. read 750MB data + ObTmpFileIOHandle handle; + int64_t read_size = write_size; + char *read_buf = new char [read_size]; + io_info.buf_ = read_buf; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + cmp = memcmp(handle.get_buffer(), write_buf, handle.get_done_size()); + ASSERT_EQ(read_size, handle.get_done_size()); + handle.reset(); + ASSERT_EQ(0, cmp); + memset(read_buf, 0, read_size); + + // 3. attempt to read data when reach the end of file + int64_t read_time = ObTimeUtility::current_time(); + io_info.size_ = 10; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_ITER_END, ret); + handle.reset(); + + // 4. pread 2MB + int64_t read_offset = 100; + read_size = macro_block_size; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, handle.get_done_size()); + handle.reset(); + ASSERT_EQ(0, cmp); + memset(read_buf + read_offset, 0, read_size); + + // 5. attempt to read data when reach the end of file (after pread) + io_info.size_ = 10; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_ITER_END, ret); + handle.reset(); + + // 6. pread data which has been read to use kv_cache + int loop_count = 30; + for (int i = 0; i < loop_count; ++i) { + read_offset = macro_block_size * (40 + i); + read_size = macro_block_size * 2; + io_info.size_ = read_size; + ret = MTL(ObTenantTmpFileManager *)->pread(io_info, read_offset, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(read_size, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_buf + read_offset, handle.get_done_size()); + handle.reset(); + ASSERT_EQ(0, cmp); + memset(read_buf + read_offset, 0, read_size); + } + read_time = ObTimeUtility::current_time() - read_time; + + free(write_buf); + free(read_buf); + + ret = MTL(ObTenantTmpFileManager *)->remove(fd); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + STORAGE_LOG(INFO, "test_big_file", K(io_info.disable_page_cache_)); + STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); +} + +TEST_F(TestTmpFile, test_big_file_with_small_wbp) +{ + const int64_t write_size = 150 * 1024 * 1024; // write 150MB data + const int64_t wbp_mem_limit = SMALL_WBP_MEM_LIMIT; + ObTmpFileIOInfo io_info; + io_info.disable_page_cache_ = true; + test_big_file(write_size, wbp_mem_limit, io_info); +} + +// generate 16MB random data for four files. (total 64MB) +// 1. the first three files write and read 1020KB data (will not trigger flushing) +// 2. the 4th file writes and reads 3MB+1020KB data (will trigger flushing in the processing of writing) +// 3. the first three files write and read 1MB data 3 times (total 3MB) +// 4. each file read and write 12MB+4KB data +TEST_F(TestTmpFile, test_multi_file_single_thread_read_write) +{ + int ret = OB_SUCCESS; + const int64_t buf_size = 64 * 1024 * 1024; // 64MB + const int64_t wbp_mem_limit = MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.get_memory_limit(); + ASSERT_GT(buf_size, wbp_mem_limit); + char *random_buf = new char [buf_size]; + for (int64_t i = 0; i < buf_size;) { + int64_t random_length = generate_random_int(1024, 8 * 1024); + int64_t random_int = generate_random_int(0, 256); + for (int64_t j = 0; j < random_length && i + j < buf_size; ++j) { + random_buf[i + j] = random_int; + } + i += random_length; + } + + int64_t dir1 = -1; + int64_t dir2 = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir2); + ASSERT_EQ(OB_SUCCESS, ret); + const int64_t file_num = 4; + char *write_bufs[file_num] = {nullptr}; + int64_t already_write_sizes[file_num] = {0}; + int64_t fds[file_num] = {-1}; + for (int i = 0; i < file_num; ++i) { + int64_t dir = i % 2 == 0 ? dir1 : dir2; + int64_t fd = -1; + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + std::cout << "open temporary file: " << fd << std::endl; + ASSERT_EQ(OB_SUCCESS, ret); + fds[i] = fd; + write_bufs[i] = random_buf + i * buf_size / file_num; + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + file_handle.get()->page_idx_cache_.max_bucket_array_capacity_ = SMALL_WBP_IDX_CACHE_MAX_CAPACITY; + file_handle.reset(); + } + ObTmpFileIOInfo io_info; + io_info.io_desc_.set_wait_event(2); + io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + ObTmpFileIOHandle handle; + int cmp = 0; + + // 1. the first three files write and read 1020KB data (will not trigger flushing) + int64_t write_size = 1020; + io_info.size_ = write_size; + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + io_info.fd_ = fds[i]; + io_info.buf_ = write_bufs[i] + already_write_sizes[i]; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + } + + char *read_buf = new char [write_size]; + io_info.buf_ = read_buf; + io_info.size_ = write_size; + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + io_info.fd_ = fds[i]; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_bufs[i] + already_write_sizes[i], io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + memset(read_buf, 0, write_size); + } + delete[] read_buf; + + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + already_write_sizes[i] += write_size; + } + // 2. the 4th file writes and reads 3MB+1020KB data (will trigger flushing in the processing of writing) + write_size = 1020 + 3 * 1024 * 1024; + io_info.size_ = write_size; + io_info.fd_ = fds[file_num - 1]; + io_info.buf_ = write_bufs[file_num - 1] + already_write_sizes[file_num - 1]; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + + read_buf = new char [write_size]; + io_info.buf_ = read_buf; + io_info.size_ = write_size; + io_info.fd_ = fds[file_num - 1]; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_bufs[file_num - 1] + already_write_sizes[file_num - 1], io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + delete[] read_buf; + already_write_sizes[file_num - 1] += write_size; + + // 3. the first three files write and read 1MB data 3 times + write_size = 1024 * 1024; + io_info.size_ = write_size; + const int loop_cnt = 3; + read_buf = new char [write_size]; + for (int cnt = 0; OB_SUCC(ret) && cnt < loop_cnt; cnt++) { + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + io_info.fd_ = fds[i]; + io_info.buf_ = write_bufs[i] + already_write_sizes[i]; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + } + + io_info.buf_ = read_buf; + io_info.size_ = write_size; + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + io_info.fd_ = fds[i]; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_bufs[i] + already_write_sizes[i], io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + memset(read_buf, 0, write_size); + } + for (int i = 0; OB_SUCC(ret) && i < file_num - 1; i++) { + already_write_sizes[i] += write_size; + } + } + delete[] read_buf; + // 4. each file read and write 12MB+4KB data + write_size = 12 * 1024 * 1024 + 4 * 1024; + io_info.size_ = write_size; + read_buf = new char [write_size]; + for (int i = 0; OB_SUCC(ret) && i < file_num; i++) { + io_info.fd_ = fds[i]; + io_info.buf_ = write_bufs[i] + already_write_sizes[i]; + ret = MTL(ObTenantTmpFileManager *)->write(io_info); + ASSERT_EQ(OB_SUCCESS, ret); + } + + io_info.buf_ = read_buf; + io_info.size_ = write_size; + for (int i = 0; OB_SUCC(ret) && i < file_num; i++) { + io_info.fd_ = fds[i]; + ret = MTL(ObTenantTmpFileManager *)->read(io_info, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(io_info.size_, handle.get_done_size()); + cmp = memcmp(handle.get_buffer(), write_bufs[i] + already_write_sizes[i], io_info.size_); + ASSERT_EQ(0, cmp); + handle.reset(); + memset(read_buf, 0, write_size); + } + delete[] read_buf; + for (int i = 0; OB_SUCC(ret) && i < file_num; i++) { + already_write_sizes[i] += write_size; + } + + for (int i = 0; OB_SUCC(ret) && i < file_num; i++) { + ret = MTL(ObTenantTmpFileManager *)->remove(fds[i]); + ASSERT_EQ(OB_SUCCESS, ret); + } + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.flush_priority_mgr_.get_file_size()); + ASSERT_EQ(0, MTL(ObTenantTmpFileManager *)->page_cache_controller_.evict_mgr_.get_file_size()); + + LOG_INFO("test_multi_file_single_thread_read_write"); +} + +TEST_F(TestTmpFile, test_single_file_multi_thread_read_write) +{ + int ret = OB_SUCCESS; + const int64_t read_thread_cnt = 4; + const int64_t file_cnt = 1; + const int64_t batch_size = 64 * 1024 * 1024; // 64MB + const int64_t batch_num = 4; + TestMultiTmpFileStress test(MTL_CTX()); + int64_t dir = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = test.init(file_cnt, dir, read_thread_cnt, batch_size, batch_num); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t io_time = ObTimeUtility::current_time(); + test.start(); + test.wait(); + io_time = ObTimeUtility::current_time() - io_time; + + STORAGE_LOG(INFO, "test_single_file_multi_thread_read_write"); + STORAGE_LOG(INFO, "io time", K(io_time)); +} + +TEST_F(TestTmpFile, test_multi_file_multi_thread_read_write) +{ + int ret = OB_SUCCESS; + MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.set_max_data_page_usage_ratio_(0.99); + const int64_t read_thread_cnt = 4; + const int64_t file_cnt = 4; + const int64_t batch_size = 16 * 1024 * 1024; // 16MB + const int64_t batch_num = 4; + TestMultiTmpFileStress test(MTL_CTX()); + int64_t dir = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = test.init(file_cnt, dir, read_thread_cnt, batch_size, batch_num); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t io_time = ObTimeUtility::current_time(); + test.start(); + test.wait(); + io_time = ObTimeUtility::current_time() - io_time; + MTL(ObTenantTmpFileManager *)->page_cache_controller_.write_buffer_pool_.set_max_data_page_usage_ratio_(0.90); + STORAGE_LOG(INFO, "test_multi_file_multi_thread_read_write"); + STORAGE_LOG(INFO, "io time", K(io_time)); +} + +TEST_F(TestTmpFile, test_more_files_more_threads_read_write) +{ + int ret = OB_SUCCESS; + const int64_t read_thread_cnt = 1; + const int64_t file_cnt = 128; + const int64_t batch_size = 3 * 1024 * 1024; + const int64_t batch_num = 2; // total 128 * 3MB * 2 = 768MB + TestMultiTmpFileStress test(MTL_CTX()); + int64_t dir = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + ret = test.init(file_cnt, dir, read_thread_cnt, batch_size, batch_num); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t io_time = ObTimeUtility::current_time(); + test.start(); + test.wait(); + io_time = ObTimeUtility::current_time() - io_time; + + STORAGE_LOG(INFO, "test_more_files_more_threads_read_write"); + STORAGE_LOG(INFO, "io time", K(io_time)); +} + +TEST_F(TestTmpFile, test_big_file) +{ + const int64_t write_size = 750 * 1024 * 1024; // write 750MB data + const int64_t wbp_mem_limit = BIG_WBP_MEM_LIMIT; + ObTmpFileIOInfo io_info; + io_info.disable_page_cache_ = false; + test_big_file(write_size, wbp_mem_limit, io_info); +} + +TEST_F(TestTmpFile, test_big_file_disable_page_cache) +{ + const int64_t write_size = 750 * 1024 * 1024; // write 750MB data + const int64_t wbp_mem_limit = BIG_WBP_MEM_LIMIT; + ObTmpFileIOInfo io_info; + io_info.disable_page_cache_ = true; + test_big_file(write_size, wbp_mem_limit, io_info); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_sn_tmp_file.log*"); + system("rm -rf ./run*"); + OB_LOGGER.set_file_name("test_sn_tmp_file.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file_block_manager.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file_block_manager.cpp new file mode 100644 index 0000000000..8b94e6ebea --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file_block_manager.cpp @@ -0,0 +1,598 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include +#include +#include +#define USING_LOG_PREFIX STORAGE +#define protected public +#define private public +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_block_manager.h" + +namespace oceanbase +{ +using namespace tmp_file; +/* ---------------------------- Unittest Class ----------------------------- */ +class TestTmpFileBlock : public ::testing::Test +{ +public: + TestTmpFileBlock() = default; + virtual ~TestTmpFileBlock() = default; + static void SetUpTestCase(); + static void TearDownTestCase(); +}; + +void TestTmpFileBlock::SetUpTestCase() +{ + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); +} + +void TestTmpFileBlock::TearDownTestCase() +{ + MockTenantModuleEnv::get_instance().destroy(); +} + +TEST_F(TestTmpFileBlock, test_block_manager_op) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockManager block_mgr; + static const int64_t META_DEFAULT_LIMIT = 15 * 1024L * 1024L * 1024L; + ret = block_mgr.init(MTL_ID(), META_DEFAULT_LIMIT); + ASSERT_EQ(OB_SUCCESS, ret); + + int64_t begin_page_id = 0; + int64_t page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS; + int64_t block_index1 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index2 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index3 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index4 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index5 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t invalid_logic_block_index = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + // 1. create tmp file blocks + ret = block_mgr.create_tmp_file_block(begin_page_id + 10, page_num - 20, block_index1); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index1, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num, block_index2); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index2, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id + 10, page_num - 10, block_index3); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index3, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num, block_index4); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index4, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num, block_index5); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index5, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(-1, page_num, invalid_logic_block_index); + ASSERT_EQ(OB_INVALID_ARGUMENT, ret); + ret = block_mgr.create_tmp_file_block(begin_page_id, -1, invalid_logic_block_index); + ASSERT_EQ(OB_INVALID_ARGUMENT, ret); + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num+1, invalid_logic_block_index); + ASSERT_EQ(OB_INVALID_ARGUMENT, ret); + ret = block_mgr.create_tmp_file_block(begin_page_id+1, page_num, invalid_logic_block_index); + ASSERT_EQ(OB_INVALID_ARGUMENT, ret); + + // 2. alloc three macro block + ObMacroBlockHandle macro_block_handle1; + ObMacroBlockHandle macro_block_handle2; + ObMacroBlockHandle macro_block_handle3; + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle3); + ASSERT_EQ(OB_SUCCESS, ret); + MacroBlockId macro_block_id1 = macro_block_handle1.get_macro_id(); + ASSERT_EQ(true, macro_block_id1.is_valid()); + MacroBlockId macro_block_id2 = macro_block_handle2.get_macro_id(); + ASSERT_EQ(true, macro_block_id2.is_valid()); + MacroBlockId macro_block_id3 = macro_block_handle3.get_macro_id(); + ASSERT_EQ(true, macro_block_id3.is_valid()); + + // 3. switch state op + ret = block_mgr.write_back_failed(block_index1); + ASSERT_EQ(OB_OP_NOT_ALLOW, ret); + ret = block_mgr.write_back_succ(block_index1, macro_block_id1); + ASSERT_EQ(OB_OP_NOT_ALLOW, ret); + + ret = block_mgr.write_back_start(block_index2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_start(block_index3); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_start(block_index4); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_start(block_index5); + ASSERT_EQ(OB_SUCCESS, ret); + + // 4. test releasing all pages of block which is writing back + // 4.1 block exists in block manager before releasing all pages of block + ObTmpFileBlockHandle handle; + ret = block_mgr.get_tmp_file_block_handle(block_index4, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(nullptr, handle.get()); + handle.reset(); + ret = block_mgr.get_tmp_file_block_handle(block_index5, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(nullptr, handle.get()); + handle.reset(); + // 4.2 block exists in block manager after releasing block + ret = block_mgr.release_tmp_file_page(block_index4, begin_page_id, page_num); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.release_tmp_file_page(block_index5, begin_page_id, page_num); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = block_mgr.get_tmp_file_block_handle(block_index4, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(nullptr, handle.get()); + handle.reset(); + ret = block_mgr.get_tmp_file_block_handle(block_index5, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(nullptr, handle.get()); + handle.reset(); + // 4.3 block doesn't exist in block manager after writing back over + ret = block_mgr.write_back_succ(block_index4, macro_block_id1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_failed(block_index5); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.get_tmp_file_block_handle(block_index4, handle); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, ret); + ret = block_mgr.get_tmp_file_block_handle(block_index5, handle); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, ret); + + // 5. test releasing pages of block which is writing back over + ret = block_mgr.write_back_succ(block_index2, macro_block_id2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index2, macro_block_id2); + ASSERT_NE(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index3, MacroBlockId()); + ASSERT_NE(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index3, macro_block_id3); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = block_mgr.release_tmp_file_page(block_index1, begin_page_id + 10, page_num - 20); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.release_tmp_file_page(block_index2, begin_page_id, page_num); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.release_tmp_file_page(block_index3, begin_page_id + 10, 10); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = block_mgr.get_tmp_file_block_handle(block_index1, handle); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, ret); + ret = block_mgr.get_tmp_file_block_handle(block_index2, handle); + ASSERT_EQ(OB_ENTRY_NOT_EXIST, ret); + ret = block_mgr.get_tmp_file_block_handle(block_index3, handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(nullptr, handle.get()); + + LOG_INFO("test_block_manager_op"); +} + +TEST_F(TestTmpFileBlock, test_block_manager_stat) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockManager block_mgr; + static const int64_t META_DEFAULT_LIMIT = 15 * 1024L * 1024L * 1024L; + ret = block_mgr.init(MTL_ID(), META_DEFAULT_LIMIT); + ASSERT_EQ(OB_SUCCESS, ret); + + int64_t begin_page_id = 0; + int64_t page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS; + int64_t block_index1 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index2 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index3 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + int64_t block_index4 = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + // 1. create tmp file blocks + ret = block_mgr.create_tmp_file_block(begin_page_id + 10, page_num - 20, block_index1); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index1, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num, block_index2); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index2, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id + 10, page_num - 10, block_index3); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index3, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + ret = block_mgr.create_tmp_file_block(begin_page_id, page_num, block_index4); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(block_index4, ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX); + + // 2. alloc three macro block + ObMacroBlockHandle macro_block_handle1; + ObMacroBlockHandle macro_block_handle2; + ObMacroBlockHandle macro_block_handle3; + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle3); + ASSERT_EQ(OB_SUCCESS, ret); + MacroBlockId macro_block_id1 = macro_block_handle1.get_macro_id(); + ASSERT_EQ(true, macro_block_id1.is_valid()); + MacroBlockId macro_block_id2 = macro_block_handle2.get_macro_id(); + ASSERT_EQ(true, macro_block_id2.is_valid()); + MacroBlockId macro_block_id3 = macro_block_handle3.get_macro_id(); + ASSERT_EQ(true, macro_block_id3.is_valid()); + + // 3. switch state op + ret = block_mgr.write_back_start(block_index1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_start(block_index2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_start(block_index3); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index1, macro_block_id1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index2, macro_block_id2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block_mgr.write_back_succ(block_index3, macro_block_id3); + ASSERT_EQ(OB_SUCCESS, ret); + + // 4. test statistic + int64_t macro_block_count = 0; + ret = block_mgr.get_macro_block_count(macro_block_count); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(3, macro_block_count); + + ObArray macro_id_list; + ret = macro_id_list.push_back(macro_block_id1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = macro_id_list.push_back(macro_block_id2); + ASSERT_EQ(OB_SUCCESS, ret); + ret = macro_id_list.push_back(macro_block_id3); + ASSERT_EQ(OB_SUCCESS, ret); + std::sort(macro_id_list.begin(), macro_id_list.end()); + + ObArray macro_id_list_stat; + ret = block_mgr.get_macro_block_list(macro_id_list_stat); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(macro_id_list.count(), macro_id_list_stat.count()); + std::sort(macro_id_list_stat.begin(), macro_id_list_stat.end()); + for (int64_t i = 0; i < macro_id_list.count(); ++i) { + ASSERT_EQ(true, macro_id_list[i] == macro_id_list_stat[i]); + } + + int64_t used_page_num = 0; + macro_block_count = 0; + ret = block_mgr.get_block_usage_stat(used_page_num, macro_block_count); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(3, macro_block_count); + ASSERT_EQ(page_num * 3 - 30, used_page_num); + + LOG_INFO("test_block_manager_stat"); +} + +TEST_F(TestTmpFileBlock, test_block) +{ + int ret = OB_SUCCESS; + ObTmpFileBlock block; + int64_t block_index = 1; + int64_t begin_page_id = 0; + int64_t page_num = 0; + + begin_page_id = -1; + page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS; + ret = block.init_block(block_index, begin_page_id, page_num); + ASSERT_NE(OB_SUCCESS, ret); + + begin_page_id = 0; + page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS + 1; + ret = block.init_block(block_index, begin_page_id, page_num); + ASSERT_NE(OB_SUCCESS, ret); + begin_page_id = 0; + + begin_page_id = 0; + page_num = 0; + ret = block.init_block(block_index, begin_page_id, page_num); + ASSERT_NE(OB_SUCCESS, ret); + + begin_page_id = 10; + page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS; + ret = block.init_block(block_index, begin_page_id, page_num); + ASSERT_NE(OB_SUCCESS, ret); + + begin_page_id = 10; + page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS-20; + ret = block.init_block(block_index, begin_page_id, page_num); + ASSERT_EQ(OB_SUCCESS, ret); + + ObMacroBlockHandle macro_block_handle1; + ret = OB_SERVER_BLOCK_MGR.alloc_block(macro_block_handle1); + ASSERT_EQ(OB_SUCCESS, ret); + MacroBlockId macro_block_id1 = macro_block_handle1.get_macro_id(); + ASSERT_EQ(true, macro_block_id1.is_valid()); + + ret = block.write_back_failed(); + ASSERT_NE(OB_SUCCESS, ret); + ret = block.write_back_succ(macro_block_id1); + ASSERT_NE(OB_SUCCESS, ret); + + int64_t end_page_id = begin_page_id + page_num - 1; + ret = block.release_pages(0, 15); + ASSERT_NE(OB_SUCCESS, ret); + ret = block.release_pages(end_page_id - 30, 35); + ASSERT_NE(OB_SUCCESS, ret); + ret = block.release_pages(9, 1); + ASSERT_NE(OB_SUCCESS, ret); + ret = block.release_pages(end_page_id + 1, 1); + ASSERT_NE(OB_SUCCESS, ret); + ret = block.release_pages(end_page_id, 1); + ASSERT_EQ(OB_SUCCESS, ret); + + ASSERT_EQ(false, block.on_disk()); + int64_t used_page_num = 0; + ret = block.get_page_usage(used_page_num); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(page_num - 1, used_page_num); + bool can_remove = false; + ret = block.can_remove(can_remove); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, can_remove); + + ret = block.write_back_start(); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block.write_back_succ(macro_block_id1); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, block.on_disk()); + ret = block.get_page_usage(used_page_num); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(page_num - 1, used_page_num); + ret = block.release_pages(10, used_page_num); + ASSERT_EQ(OB_SUCCESS, ret); + ret = block.can_remove(can_remove); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, can_remove); + + LOG_INFO("test_block"); +} + +TEST_F(TestTmpFileBlock, test_block_page_bit_map) +{ + int ret = OB_SUCCESS; + bool value; + ObTmpFileBlockPageBitmap bitmap; + // 1. get/set for one bit of bitmap + ret = bitmap.get_value(7, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.get_value(16, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + ret = bitmap.set_bitmap(7, true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.set_bitmap(16, true); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = bitmap.get_value(7, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + ret = bitmap.get_value(16, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + + ret = bitmap.get_value(bitmap.get_capacity(), value); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.get_value(-1, value); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.set_bitmap(bitmap.get_capacity(), true); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.set_bitmap(-1, true); + ASSERT_NE(OB_SUCCESS, ret); + + // 2. set batch bits of bitmap + int64_t start = 13; + int64_t end = 80; + ret = bitmap.get_value(start, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.get_value(40, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.get_value(end, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + ret = bitmap.set_bitmap_batch(start, end - start + 1, true); + ASSERT_EQ(OB_SUCCESS, ret); + for (int i = start; i <= end; i++) { + ret = bitmap.get_value(i, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + } + ret = bitmap.get_value(start - 1, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.get_value(end + 1, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + start = 5; + end = 40; + ret = bitmap.set_bitmap_batch(start , end - start + 1, true); + ASSERT_EQ(OB_SUCCESS, ret); + for (int i = start; i <= end; i++) { + ret = bitmap.get_value(i, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + } + + start = 60; + end = 100; + ret = bitmap.set_bitmap_batch(start, end - start + 1, false); + ASSERT_EQ(OB_SUCCESS, ret); + for (int i = start; i <= end; i++) { + ret = bitmap.get_value(i, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + } + + ret = bitmap.set_bitmap_batch(-1, 100, true); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.set_bitmap_batch(0, bitmap.get_capacity() + 1, true); + ASSERT_NE(OB_SUCCESS, ret); + // 3. test is_all_true() and is_all_false() + ret = bitmap.is_all_false(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.is_all_true(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + bitmap.reset(); + + ret = bitmap.is_all_false(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + ret = bitmap.is_all_true(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + start = 60; + end = 100; + ret = bitmap.set_bitmap_batch(start, end - start + 1 , true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.is_all_true(start, end, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + ret = bitmap.is_all_true(start - 1, end, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.is_all_true(start, end + 1, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + ret = bitmap.is_all_true(-1, end, value); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.is_all_true(100, bitmap.get_capacity(), value); + ASSERT_NE(OB_SUCCESS, ret); + + ret = bitmap.set_bitmap_batch(0, bitmap.get_capacity(), true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.is_all_true(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + ret = bitmap.is_all_false(value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + start = 60; + end = 100; + ret = bitmap.set_bitmap_batch(start, end - start + 1 , false); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.is_all_false(start, end, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, value); + ret = bitmap.is_all_false(start - 1, end, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + ret = bitmap.is_all_false(start, end + 1, value); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, value); + + ret = bitmap.is_all_false(-1, end, value); + ASSERT_NE(OB_SUCCESS, ret); + ret = bitmap.is_all_false(100, bitmap.get_capacity(), value); + ASSERT_NE(OB_SUCCESS, ret); + LOG_INFO("test_block_page_bit_map"); +} + +TEST_F(TestTmpFileBlock, test_block_page_bit_map_iter) +{ + int ret = OB_SUCCESS; + bool value; + ObTmpFileBlockPageBitmap bitmap; + int64_t start_page_id = 0; + int64_t end_page_id = bitmap.get_capacity() - 1; + ObTmpFileBlockPageBitmapIterator iter; + ret = iter.init(nullptr, start_page_id, end_page_id); + ASSERT_NE(OB_SUCCESS, ret); + ret = iter.init(&bitmap, -1, end_page_id); + ASSERT_NE(OB_SUCCESS, ret); + ret = iter.init(&bitmap, start_page_id, bitmap.get_capacity()); + ASSERT_NE(OB_SUCCESS, ret); + ret = iter.init(&bitmap, start_page_id, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + + int64_t starts[5] = {0, 10, 84, 163, bitmap.get_capacity() - 1}; + int64_t ends[5] = {0, 40, 101, 230, bitmap.get_capacity() - 1}; + ret = bitmap.set_bitmap(starts[0], true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.set_bitmap_batch(starts[1], ends[1] - starts[1] + 1, true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.set_bitmap_batch(starts[2], ends[2] - starts[2] + 1, true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.set_bitmap_batch(starts[3], ends[3] - starts[3] + 1, true); + ASSERT_EQ(OB_SUCCESS, ret); + ret = bitmap.set_bitmap(starts[4], true); + ASSERT_EQ(OB_SUCCESS, ret); + + ASSERT_EQ(true, iter.has_next()); + + int i = 0; + while (iter.has_next()) { + bool value = false; + ret = iter.next_range(value, start_page_id, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + if (value) { + ASSERT_EQ(start_page_id, starts[i]); + ASSERT_EQ(end_page_id, ends[i]); + i++; + } else { + ASSERT_NE(i, 0); + ASSERT_EQ(start_page_id, ends[i-1]+1); + ASSERT_EQ(end_page_id, starts[i]-1); + } + } + + iter.reset(); + ret = iter.init(&bitmap, 5, 180); + ASSERT_EQ(OB_SUCCESS, ret); + ends[0] = 4; + ends[3] = 180; + i = 1; + while (iter.has_next()) { + bool value = false; + ret = iter.next_range(value, start_page_id, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + if (value) { + ASSERT_EQ(start_page_id, starts[i]); + ASSERT_EQ(end_page_id, ends[i]); + i++; + } else { + ASSERT_EQ(start_page_id, ends[i-1]+1); + ASSERT_EQ(end_page_id, starts[i]-1); + } + } + LOG_INFO("test_block_page_bit_map_iter"); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_tmp_file_block_manager.log*"); + OB_LOGGER.set_file_name("test_tmp_file_block_manager.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} \ No newline at end of file diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file_buffer_pool.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file_buffer_pool.cpp new file mode 100644 index 0000000000..98af9fac2d --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file_buffer_pool.cpp @@ -0,0 +1,690 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE +#include +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "lib/random/ob_random.h" + +namespace oceanbase +{ +using namespace common; +using namespace tmp_file; +using namespace storage; + +struct WBPTestHelper +{ +public: + WBPTestHelper(const int64_t fd, ObTmpWriteBufferPool &wbp) + : fd_(fd), + data_size_(0), + data_page_num_(0), + meta_page_num_(0), + data_page_ids_(), + begin_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + begin_virtual_page_id_(-1), + end_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + wbp_(wbp) {} + int alloc_data_pages(const int64_t num); + // int alloc_meta_pages(const int64_t num); + int free_all_pages(); +public: + int64_t fd_; + int64_t data_size_; + int64_t data_page_num_; + int64_t meta_page_num_; + ObArray data_page_ids_; + uint32_t begin_page_id_; + int64_t begin_virtual_page_id_; + uint32_t end_page_id_; + ObTmpWriteBufferPool &wbp_; +}; + +int WBPTestHelper::alloc_data_pages(const int64_t num) +{ + int ret = OB_SUCCESS; + uint32_t previous_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t previous_virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + for (int64_t i = 0; OB_SUCC(ret) && i < num; ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + int64_t new_virtual_page_id_ = data_size_ / ObTmpFileGlobal::PAGE_SIZE; + if (OB_FAIL(wbp_.alloc_page(fd_, ObTmpFilePageUniqKey(new_virtual_page_id_), new_page_id, buf))) { + LOG_WARN("fail to alloc page", K(fd_), K(previous_page_id)); + } else if (OB_FAIL(data_page_ids_.push_back(new_page_id))) { + LOG_WARN("fail to push back", K(ret)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID != previous_page_id && + OB_FAIL(wbp_.link_page(fd_, new_page_id, previous_page_id, ObTmpFilePageUniqKey(previous_virtual_page_id_)))) { + LOG_WARN("fail to link page", K(fd_), K(new_page_id), K(previous_page_id), K(previous_virtual_page_id_)); + } else { + previous_virtual_page_id_ = new_virtual_page_id_; + if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_) { + begin_page_id_ = new_page_id; + begin_virtual_page_id_ = data_size_ / ObTmpFileGlobal::PAGE_SIZE; + } + data_size_ += ObTmpFileGlobal::PAGE_SIZE; + previous_page_id = new_page_id; + } + } + return ret; +} + +int WBPTestHelper::free_all_pages() +{ + int ret = OB_SUCCESS; + uint32_t free_page_id = begin_page_id_; + for (int64_t free_cnt = 0; OB_SUCC(ret) && free_cnt < data_page_num_ + && ObTmpFileGlobal::INVALID_PAGE_ID != free_page_id; ++free_cnt) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp_.free_page(fd_, free_page_id, ObTmpFilePageUniqKey(begin_virtual_page_id_), next_page_id); + free_page_id = next_page_id; + } + return ret; +} + +struct WBPTestFunctor +{ +public: + WBPTestFunctor(const int64_t fd, const int64_t capacity, const int64_t loop, + ObTmpWriteBufferPool *wbp) + : fd_(fd), wbp_capacity_(capacity), loop_(loop), wbp_(wbp), data_(), + begin_data_page_virtual_id_(-1), end_data_page_virtual_id_(-1) {} + void operator() (); + bool check_wbp_data_success(); + bool check_no_page_belong_self(); + void print_deque(std::deque * dq); +public: + int64_t fd_; + int64_t wbp_capacity_; + int64_t loop_; + ObTmpWriteBufferPool * wbp_; + std::deque data_; + int64_t begin_data_page_virtual_id_; + int64_t end_data_page_virtual_id_; +}; + +/* + * Randomly allocate several pages (0 ~ `wbp_capacity_`), then randomly free + * several pages (0 ~ alloced_page_nums), repeat for `loop_` times, and at the + * end, all pages are automatically returned. + */ +void WBPTestFunctor::operator() () +{ + // alloc and free + for (int i = 0; i < loop_; ++i) { + int ret = OB_SUCCESS; + // random alloc pages + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buf = nullptr; + uint32_t prev_page_id = data_.size() == 0 + ? ObTmpFileGlobal::INVALID_PAGE_ID + : data_.back(); + int64_t alloc_page_nums = ObRandom::rand(0, wbp_capacity_); + for (int64_t j = 0; OB_SUCC(ret) && j < alloc_page_nums; ++j) { + int64_t new_page_begin_virtual_id = end_data_page_virtual_id_ < 0 ? 0 : end_data_page_virtual_id_ + 1; + ret = wbp_->alloc_page(fd_, ObTmpFilePageUniqKey(new_page_begin_virtual_id), new_page_id, new_page_buf); + ASSERT_EQ(OB_SUCCESS, ret); + if (prev_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + ret = wbp_->link_page(fd_, new_page_id, prev_page_id, ObTmpFilePageUniqKey(end_data_page_virtual_id_)); + ASSERT_EQ(OB_SUCCESS, ret); + } else { + begin_data_page_virtual_id_ = new_page_begin_virtual_id; + } + data_.push_back(new_page_id); + LOG_INFO("alloc page succeed", K(fd_), K(new_page_id), K(prev_page_id)); + end_data_page_virtual_id_ = new_page_begin_virtual_id; + prev_page_id = new_page_id; + } + + if (!check_wbp_data_success()) { + std::cout << "check data fail after alloc, loop: " << i << std::endl; + break; + } + + int64_t free_page_nums = ObRandom::rand(0, data_.size()); + for (int64_t j = 0; OB_SUCC(ret) && j < free_page_nums && data_.size() > 0; ++j) { + uint32_t page_to_free = data_.front(); + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp_->free_page(fd_, page_to_free, ObTmpFilePageUniqKey(begin_data_page_virtual_id_), next_page_id); + data_.pop_front(); + ASSERT_EQ(ret, OB_SUCCESS); + begin_data_page_virtual_id_ += 1; + } + + if (!check_wbp_data_success()) { + std::cout << "check data fail after free, loop: " << i << std::endl; + break; + } + } + // free all + { + int ret = OB_SUCCESS; + int64_t total_page_nums = data_.size(); + for (int64_t j = 0; OB_SUCC(ret) && j < total_page_nums; ++j) { + uint32_t page_to_free = data_.front(); + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp_->free_page(fd_, page_to_free, ObTmpFilePageUniqKey(begin_data_page_virtual_id_), next_page_id); + ASSERT_EQ(ret, OB_SUCCESS); + data_.pop_front(); + begin_data_page_virtual_id_ += 1; + } + if (data_.size() != 0) { + std::cout << "free all pages error, data size: " << data_.size() << std::endl; + } + if (!check_no_page_belong_self()) { + std::cout << fd_ << " check no page belong self fail" << std::endl; + } + } +} + +bool WBPTestFunctor::check_wbp_data_success() +{ + bool check_res = true; + std::deque wbp_data; + // collect wbp data + int ret = OB_SUCCESS; + uint32_t curr_page_id = data_.size() > 0 ? data_.front() : ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t curr_page_virtual_id = begin_data_page_virtual_id_; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char * page_buff = nullptr; + while (OB_SUCC(ret) && curr_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + ret = wbp_->read_page(fd_, curr_page_id, ObTmpFilePageUniqKey(curr_page_virtual_id), page_buff, next_page_id); + if (OB_SUCC(ret)) { + wbp_data.push_back(curr_page_id); + curr_page_virtual_id += 1; + } else { + std::cout << "fetch page error, ret: " << ret << std::endl; + } + curr_page_id = next_page_id; + } + // compare data + if (data_.size() != wbp_data.size()) { + std::cout << "check wbp data fail, data size: " << data_.size() + << ", wbp data size: " << wbp_data.size() << std::endl; + check_res = false; + print_deque(&data_); + print_deque(&wbp_data); + } else { + for (int64_t i = 0; i < data_.size(); ++i) { + if (data_.at(i) != wbp_data.at(i)) { + std::cout << "check wbp data fail, not equal happen at: " << i + << ", data: " << data_.at(i) + << ", wbp data: " << wbp_data.at(i) << std::endl; + check_res = false; + print_deque(&data_); + print_deque(&wbp_data); + break; + } + } + } + return check_res; +} + +bool WBPTestFunctor::check_no_page_belong_self() +{ + bool no_page_belong_self = true; + int64_t page_belong_self_nums = 0; + for (int64_t i = 0; i < wbp_capacity_; ++i) { + if (wbp_->fat_[i].fd_ == fd_) { + std::cout << fd_ << " find self page, idx: " << i + << ", fd: " << wbp_->fat_[i].fd_ + << ", next_page_id: " << wbp_->fat_[i].next_page_id_ + << std::endl; + page_belong_self_nums++; + no_page_belong_self = false; + } + } + if (!no_page_belong_self) { + std::cout << fd_ << " occupy " << page_belong_self_nums << " pages" << std::endl; + } + return no_page_belong_self; +} + +void WBPTestFunctor::print_deque(std::deque * dq) +{ + ObArray data; + for (int64_t i = 0; i < dq->size(); ++i) { + data.push_back(dq->at(i)); + } + LOG_INFO("print_deque", K(fd_), K(data)); +} + +class TestBufferPool : public ::testing::Test +{ +public: + virtual void SetUp(); + virtual void TearDown(); + static void SetUpTestCase(); + static void TearDownTestCase(); +}; + +void TestBufferPool::SetUp() +{ + ObTmpFilePageCacheController &pc_ctrl = MTL(ObTenantTmpFileManager *)->get_page_cache_controller(); + ObTmpWriteBufferPool &wbp = pc_ctrl.get_write_buffer_pool(); + wbp.destroy(); + ASSERT_EQ(OB_SUCCESS, wbp.init()); +} + +void TestBufferPool::TearDown() +{ + MTL(ObTenantTmpFileManager *)->get_page_cache_controller().get_write_buffer_pool().destroy(); +} + +void TestBufferPool::SetUpTestCase() +{ + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); +} + +void TestBufferPool::TearDownTestCase() +{ + MockTenantModuleEnv::get_instance().destroy(); +} + +TEST_F(TestBufferPool, test_buffer_pool_basic) +{ + ObTmpFilePageCacheController &pc_ctrl = MTL(ObTenantTmpFileManager *)->get_page_cache_controller(); + ObTmpWriteBufferPool &wbp = pc_ctrl.get_write_buffer_pool(); + WBPTestFunctor wbp_test_functor(0, ObTmpWriteBufferPool::BLOCK_PAGE_NUMS, 100, &wbp); + wbp_test_functor(); +} + +TEST_F(TestBufferPool, test_buffer_pool_concurrent) +{ + ObTmpFilePageCacheController &pc_ctrl = MTL(ObTenantTmpFileManager *)->get_page_cache_controller(); + ObTmpWriteBufferPool &wbp = pc_ctrl.get_write_buffer_pool(); + const int64_t MAX_THREAD_NUM = 5; + const int64_t MAX_LOOP_NUM = 100; + std::vector t_vec; + for (int64_t i = 0; i < MAX_THREAD_NUM; ++i) { + WBPTestFunctor functor = WBPTestFunctor(10 + i, ObTmpWriteBufferPool::BLOCK_PAGE_NUMS, MAX_LOOP_NUM, &wbp); + t_vec.push_back(std::thread(functor)); + } + for (int64_t i = 0; i < t_vec.size(); ++i) { + t_vec[i].join(); + } +} + +TEST_F(TestBufferPool, test_entry_state_switch_write_back) +{ + ObTmpWriteBufferPool &wbp = MTL(ObTenantTmpFileManager *)->page_cache_controller_.get_write_buffer_pool(); + int ret = OB_SUCCESS; + int64_t fd = 0; + const int64_t ALLOC_PAGE_NUM = 200; + WBPTestHelper wbp_test(fd, wbp); + ret = wbp_test.alloc_data_pages(ALLOC_PAGE_NUM); + ASSERT_EQ(OB_SUCCESS, ret); + + // dirty + uint32_t cur_page_id = wbp_test.begin_page_id_; + int64_t cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_dirty(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + ASSERT_EQ(ALLOC_PAGE_NUM, wbp.dirty_page_num_); + + // write back + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_write_back(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + ASSERT_EQ(0, wbp.dirty_page_num_); + + // write back fail, page entry return to dirty + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_write_back_fail(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + printf("after write back fail\n"); + ASSERT_EQ(ALLOC_PAGE_NUM, wbp.dirty_page_num_); + + // write back again + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_write_back(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + + // write back succ + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_write_back_succ(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + ASSERT_EQ(0, wbp.dirty_page_num_); + + // write back succ re-entrant + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_write_back_succ(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + ASSERT_EQ(0, wbp.dirty_page_num_); + + ret = wbp_test.free_all_pages(); + ASSERT_EQ(OB_SUCCESS, ret); +} + +TEST_F(TestBufferPool, test_entry_state_switch_loading) +{ + ObTmpWriteBufferPool &wbp = MTL(ObTenantTmpFileManager *)->page_cache_controller_.get_write_buffer_pool(); + int ret = OB_SUCCESS; + int64_t fd = 0; + const int64_t ALLOC_PAGE_NUM = 200; + WBPTestHelper wbp_test(fd, wbp); + ret = wbp_test.alloc_data_pages(ALLOC_PAGE_NUM); + ASSERT_EQ(OB_SUCCESS, ret); + + // load + uint32_t cur_page_id = wbp_test.begin_page_id_; + int64_t cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_load(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_TRUE(wbp.is_loading(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id))); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + + // load fail + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_load_fail(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_TRUE(wbp.is_exist(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id))); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + + // load again + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_load(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_TRUE(wbp.is_loading(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id))); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + + // load succ + cur_page_id = wbp_test.begin_page_id_; + cur_page_virtual_id = 0; + for (int64_t i = 0; i < ALLOC_PAGE_NUM; ++i) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_next_page_id(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp.notify_load_succ(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_TRUE(wbp.is_cached(fd, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id))); + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + } + + ret = wbp_test.free_all_pages(); + ASSERT_EQ(OB_SUCCESS, ret); +} + +TEST_F(TestBufferPool, test_alloc_page_limit) +{ + int ret = OB_SUCCESS; + ObTmpWriteBufferPool &wbp = MTL(ObTenantTmpFileManager *)->page_cache_controller_.get_write_buffer_pool(); + int64_t max_page_num = wbp.get_max_page_num(); + std::cout << "write buffer pool max page num " << max_page_num << std::endl; + LOG_INFO("write buffer pool max page num", K(max_page_num)); + int64_t fd = 0; + int64_t offset = 0; + uint32_t data_head_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + // 分配 50% 的 data page + const int64_t BATCH_ALLOC_DATA_PAGE_NUM = max_page_num / 2; + int64_t cur_page_virtual_id = 0; + for (int64_t i = 0; i < BATCH_ALLOC_DATA_PAGE_NUM; ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + ret = wbp.alloc_page(fd, ObTmpFilePageUniqKey(cur_page_virtual_id), new_page_id, buf); // TODO: 替换成wbp_test + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_NE(buf, nullptr); + data_head_page_id = ObTmpFileGlobal::INVALID_PAGE_ID == data_head_page_id ? new_page_id : data_head_page_id; + if (ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + ret = wbp.link_page(fd, new_page_id, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id - 1)); + ASSERT_EQ(ret, OB_SUCCESS); + } + cur_page_virtual_id += 1; + cur_page_id = new_page_id; + } + + // 再分配 50% 的 data page,超过 MAX_DATA_PAGE_USAGE_RATIO(default 0.9) 后 + // 会触发 OB_ALLOCATE_TMP_FILE_PAGE_FAILED,分配页面失败 + for (int64_t i = 0; OB_SUCC(ret) && i < BATCH_ALLOC_DATA_PAGE_NUM; ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + ret = wbp.alloc_page(fd, ObTmpFilePageUniqKey(cur_page_virtual_id), new_page_id, buf); + if (ret == OB_ALLOCATE_TMP_FILE_PAGE_FAILED) { + break; + } + ASSERT_EQ(ret, OB_SUCCESS); + if (cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + ret = wbp.link_page(fd, new_page_id, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id - 1)); + ASSERT_EQ(ret, OB_SUCCESS); + } + cur_page_id = new_page_id; + cur_page_virtual_id += 1; + } + ASSERT_EQ(ret, OB_ALLOCATE_TMP_FILE_PAGE_FAILED); + + // 此时仍可分配少量 meta page(buffer pool最小为2MB,为meta page预留空间最少为25页) + cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t meta_page_num = 0; + for (int64_t i = 0; i < std::max(max_page_num * 0.01, 20.0); ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + ObTmpFilePageUniqKey meta_page_offset(1, meta_page_num); + ret = wbp.alloc_page(fd, meta_page_offset, new_page_id, buf); + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_NE(buf, nullptr); + if (ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + ret = wbp.link_page(fd, new_page_id, cur_page_id, ObTmpFilePageUniqKey(1, meta_page_num-1)); + ASSERT_EQ(ret, OB_SUCCESS); + } + cur_page_id = new_page_id; + meta_page_num += 1; + } + + // 分配 meta page 到buffer pool上限 + cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + for (int64_t i = 0; OB_SUCC(ret) && i < max_page_num; ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + ObTmpFilePageUniqKey meta_page_offset(1, meta_page_num); + ret = wbp.alloc_page(fd, meta_page_offset, new_page_id, buf); + if (ret == OB_ALLOCATE_TMP_FILE_PAGE_FAILED) { + break; + } + ASSERT_EQ(ret, OB_SUCCESS); + if (cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + ret = wbp.link_page(fd, new_page_id, cur_page_id, ObTmpFilePageUniqKey(1, meta_page_num -1)); + ASSERT_EQ(ret, OB_SUCCESS); + } + cur_page_id = new_page_id; + meta_page_num += 1; + } + ASSERT_EQ(ret, OB_ALLOCATE_TMP_FILE_PAGE_FAILED); + ASSERT_EQ(wbp.data_page_cnt_ + wbp.meta_page_cnt_, wbp.used_page_num_); + + // data page释放后,可以继续分配meta page + int64_t cur_meta_page_num = wbp.meta_page_cnt_; + int64_t free_page_id = data_head_page_id; + const int64_t FREE_DATA_PAGE_NUM = max_page_num / 2; + int64_t free_cnt = 0; + cur_page_virtual_id = 0; + for (; free_cnt < FREE_DATA_PAGE_NUM && ObTmpFileGlobal::INVALID_PAGE_ID != free_page_id; ++free_cnt) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.free_page(fd, free_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), next_page_id); + free_page_id = next_page_id; + ASSERT_EQ(ret, OB_SUCCESS); + cur_page_virtual_id += 1; + } + ASSERT_EQ(free_cnt, FREE_DATA_PAGE_NUM); + int64_t alloc_cnt = 0; + for (; OB_SUCC(ret) && alloc_cnt < FREE_DATA_PAGE_NUM; ++alloc_cnt) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *buf = nullptr; + ObTmpFilePageUniqKey meta_page_offset(1, meta_page_num); + ret = wbp.alloc_page(fd, meta_page_offset, new_page_id, buf); + ASSERT_EQ(ret, OB_SUCCESS); + if (cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + ret = wbp.link_page(fd, new_page_id, cur_page_id, ObTmpFilePageUniqKey(1, meta_page_num - 1)); + ASSERT_EQ(ret, OB_SUCCESS); + } + cur_page_id = new_page_id; + meta_page_num += 1; + } + printf("total page num: %ld, data page: %ld, meta page: %ld, capacity: %ld\n", wbp.used_page_num_, wbp.meta_page_cnt_, wbp.data_page_cnt_, wbp.capacity_); + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_EQ(wbp.meta_page_cnt_, FREE_DATA_PAGE_NUM + cur_meta_page_num); + ASSERT_EQ(wbp.meta_page_cnt_ + wbp.data_page_cnt_, wbp.used_page_num_); +} + +TEST_F(TestBufferPool, test_get_page_id_by_offset) +{ + ObTmpWriteBufferPool &wbp = MTL(ObTenantTmpFileManager *)->page_cache_controller_.get_write_buffer_pool(); + int ret = OB_SUCCESS; + int64_t fd = 0; + const int64_t ALLOC_PAGE_NUM = 400; + WBPTestHelper wbp_test(fd, wbp); + ret = wbp_test.alloc_data_pages(ALLOC_PAGE_NUM); + ASSERT_EQ(OB_SUCCESS, ret); + + uint32_t page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_page_id_by_virtual_id(fd, 0, wbp_test.begin_page_id_, page_id); + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_NE(page_id, ObTmpFileGlobal::INVALID_PAGE_ID); + ASSERT_EQ(page_id, wbp_test.data_page_ids_.at(0)); + + page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_page_id_by_virtual_id(fd, 1, wbp_test.begin_page_id_, page_id); + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_NE(page_id, ObTmpFileGlobal::INVALID_PAGE_ID); + ASSERT_EQ(page_id, wbp_test.data_page_ids_.at(1)); + + page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_page_id_by_virtual_id(fd, ALLOC_PAGE_NUM - 1, wbp_test.begin_page_id_, page_id); + ASSERT_EQ(ret, OB_SUCCESS); + ASSERT_NE(page_id, ObTmpFileGlobal::INVALID_PAGE_ID); + ASSERT_EQ(page_id, wbp_test.data_page_ids_.at(ALLOC_PAGE_NUM - 1)); + + // offset out of bound, return INVALID_PAGE_ID + page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.get_page_id_by_virtual_id(fd, ALLOC_PAGE_NUM, wbp_test.begin_page_id_, page_id); + ASSERT_EQ(page_id, ObTmpFileGlobal::INVALID_PAGE_ID); + + ret = wbp_test.free_all_pages(); + ASSERT_EQ(OB_SUCCESS, ret); +} + +TEST_F(TestBufferPool, test_truncate_page) +{ + ObTmpWriteBufferPool &wbp = MTL(ObTenantTmpFileManager *)->page_cache_controller_.get_write_buffer_pool(); + int ret = OB_SUCCESS; + int64_t fd = 0; + const int64_t ALLOC_PAGE_NUM = 200; + WBPTestHelper wbp_test(fd, wbp); + ret = wbp_test.alloc_data_pages(ALLOC_PAGE_NUM); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = wbp.truncate_page(fd, wbp_test.begin_page_id_, ObTmpFilePageUniqKey(0), -1); + ASSERT_EQ(ret, OB_INVALID_ARGUMENT); + + ret = wbp.truncate_page(fd, wbp_test.begin_page_id_, ObTmpFilePageUniqKey(0), 0); + ASSERT_EQ(ret, OB_INVALID_ARGUMENT); + + ret = wbp.truncate_page(2, wbp_test.begin_page_id_, ObTmpFilePageUniqKey(0), 4096); + ASSERT_EQ(ret, OB_STATE_NOT_MATCH); + + const int64_t truncate_size = 4096; + ret = wbp.truncate_page(fd, wbp_test.begin_page_id_, ObTmpFilePageUniqKey(0), truncate_size); + ASSERT_EQ(ret, OB_SUCCESS); + + char null_buf[truncate_size]; + memset(null_buf, 0, sizeof(null_buf)); + char *page_buf = nullptr; + uint32_t unused_next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ret = wbp.read_page(fd, wbp_test.data_page_ids_.at(0), ObTmpFilePageUniqKey(0), page_buf, unused_next_page_id); + ASSERT_EQ(ret, OB_SUCCESS); + int cmp = memcmp(null_buf, page_buf, truncate_size); + ASSERT_EQ(cmp, 0); + + ret = wbp_test.free_all_pages(); + ASSERT_EQ(OB_SUCCESS, ret); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_tmp_file_buffer_pool.log*"); + OB_LOGGER.set_file_name("test_tmp_file_buffer_pool.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file_flush_list.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file_flush_list.cpp new file mode 100644 index 0000000000..f0f2371ca3 --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file_flush_list.cpp @@ -0,0 +1,697 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE +#include +#include +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "lib/random/ob_random.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_flush_priority_manager.h" + +namespace oceanbase +{ +using namespace common; +using namespace blocksstable; +using namespace tmp_file; +using namespace storage; +using namespace std; + +typedef ObTmpFileGlobal::FlushCtxState FlushCtxState; + +struct TestDirtyPageRecord +{ + TestDirtyPageRecord() : dir(0), dirty_data_size_(0), non_rightmost_meta_page_num_(0), rightmost_meta_page_num_(0) {} + TestDirtyPageRecord(const int64_t dir, + const int64_t dirty_data_size, + const int64_t non_rightmost_meta_page_num, + const int64_t rightmost_meta_page_num) + : dir(dir), + dirty_data_size_(dirty_data_size), + non_rightmost_meta_page_num_(non_rightmost_meta_page_num), + rightmost_meta_page_num_(rightmost_meta_page_num) {} + int64_t dir; + int64_t dirty_data_size_; + int64_t non_rightmost_meta_page_num_; + int64_t rightmost_meta_page_num_; +}; + +class TestFlushListIterator : public ::testing::Test +{ +public: + TestFlushListIterator() {} + void insert_file_sequence(); + void create_files(const FlushCtxState state, + const int64_t file_num, + ObTmpFileFlushPriorityManager &flush_prio_mgr, + vector &file_handles); + void create_files_with_dir(const FlushCtxState state, + const int64_t dir, + const int64_t file_num, + ObTmpFileFlushPriorityManager &flush_prio_mgr, + vector &file_handles); + void clear_all_files(); +protected: + virtual void TearDown() + { + clear_all_files(); + } + + static void SetUpTestCase() + { + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); + } + + static void TearDownTestCase() + { + MockTenantModuleEnv::get_instance().destroy(); + } +private: + char *random_buf_; + const int64_t write_size_ = 4 * 1024 * 1024; // 4MB + unordered_map mock_dirty_record_; // 记录模拟生成的脏页数据代替实际写入文件 +}; + +void get_file_range_by_state(const FlushCtxState state, int64_t &low, int64_t &high) +{ + switch(state) { + case FlushCtxState::FSM_F1: // [2MB, 4MB] + low = 2 * 1024 * 1024; + high = 4 * 1024 * 1024; + break; + case FlushCtxState::FSM_F2: // [1MB, 2MB) + low = 1 * 1024 * 1024; + high = 2 * 1024 * 1024 - 1; + break; + case FlushCtxState::FSM_F3: // [128KB, 1MB) + low = 128 * 1024; + high = 1 * 1024 * 1024 - 1; + break; + case FlushCtxState::FSM_F4: // [8KB, 128KB) + low = 8 * 1024; + high = 128 * 1024 - 1; + break; + case FlushCtxState::FSM_F5: // (0KB, 8KB) + low = 1; + high = 8 * 1024 - 1; + break; + default: + low = 0; + high = 0; + break; + } +} + +void TestFlushListIterator::clear_all_files() +{ + int ret = OB_SUCCESS; +// ObTmpFileFlushPriorityManager &flush_prio_mgr = +// MTL(ObTenantTmpFileManager *)->get_page_cache_controller().get_flush_priority_mgr(); +// for (auto p : mock_dirty_record_) { +// int64_t fd = p.first; +// ObTmpFileHandle file_handle; +// ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); +// ASSERT_EQ(OB_SUCCESS, ret); +// ASSERT_NE(nullptr, file_handle.get()); +// ret = flush_prio_mgr.remove_file(*file_handle.get()); +// ASSERT_EQ(OB_SUCCESS, ret); +// } + mock_dirty_record_.clear(); +} + +void TestFlushListIterator::insert_file_sequence() +{ +} + +void TestFlushListIterator::create_files(const FlushCtxState state, const int64_t file_num, + ObTmpFileFlushPriorityManager &flush_prio_mgr, + vector &file_handles) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0 ; i < file_num; ++i) { + int64_t dir = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + + int64_t fd = i; + + void *buf = nullptr; + ObSharedNothingTmpFile *tmp_file = nullptr; + buf = MTL(ObTenantTmpFileManager *)->tmp_file_allocator_.alloc(sizeof(ObSharedNothingTmpFile)); + ASSERT_NE(nullptr, buf); + + tmp_file = new (buf) ObSharedNothingTmpFile(); + ret = tmp_file->init(MTL_ID(), fd, dir, + &MTL(ObTenantTmpFileManager *)->tmp_file_block_manager_, + &MTL(ObTenantTmpFileManager *)->callback_allocator_, + &MTL(ObTenantTmpFileManager *)->wbp_index_cache_allocator_, + &MTL(ObTenantTmpFileManager *)->wbp_index_cache_bucket_allocator_, + &MTL(ObTenantTmpFileManager *)->page_cache_controller_); + ASSERT_EQ(OB_SUCCESS, ret); + tmp_file->flush_prio_mgr_ = &flush_prio_mgr; + + // ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + // ASSERT_EQ(OB_SUCCESS, ret); + + int64_t low = 0, high = 0; + get_file_range_by_state(state, low, high); + + TestDirtyPageRecord dirty_record; + int64_t mock_dirty_data_size = ObRandom::rand(low, high); + + dirty_record.dirty_data_size_ = mock_dirty_data_size; + dirty_record.non_rightmost_meta_page_num_ = 0; + dirty_record.rightmost_meta_page_num_ = 0; + mock_dirty_record_[fd] = dirty_record; + + ObTmpFileHandle file_handle; + file_handle.init(tmp_file); + ObSharedNothingTmpFile &file = *file_handle.get(); + file.file_size_ = mock_dirty_data_size; + file.cached_page_nums_ = + upper_align(mock_dirty_data_size, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + file_handles.push_back(file_handle); + } + ASSERT_EQ(file_num, file_handles.size()); +} + +void TestFlushListIterator::create_files_with_dir( + const FlushCtxState state, + const int64_t dir, + const int64_t file_num, + ObTmpFileFlushPriorityManager &flush_prio_mgr, + vector &file_handles) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0 ; i < file_num; ++i) { + int64_t fd = i; + + void *buf = nullptr; + ObSharedNothingTmpFile *tmp_file = nullptr; + buf = MTL(ObTenantTmpFileManager *)->tmp_file_allocator_.alloc(sizeof(ObSharedNothingTmpFile)); + ASSERT_NE(nullptr, buf); + + tmp_file = new (buf) ObSharedNothingTmpFile(); + ret = tmp_file->init(MTL_ID(), fd, dir, + &MTL(ObTenantTmpFileManager *)->tmp_file_block_manager_, + &MTL(ObTenantTmpFileManager *)->callback_allocator_, + &MTL(ObTenantTmpFileManager *)->wbp_index_cache_allocator_, + &MTL(ObTenantTmpFileManager *)->wbp_index_cache_bucket_allocator_, + &MTL(ObTenantTmpFileManager *)->page_cache_controller_); + ASSERT_EQ(OB_SUCCESS, ret); + tmp_file->flush_prio_mgr_ = &flush_prio_mgr; + + int64_t low = 0, high = 0; + get_file_range_by_state(state, low, high); + + TestDirtyPageRecord dirty_record; + int64_t mock_dirty_data_size = ObRandom::rand(low, high); + + dirty_record.dirty_data_size_ = mock_dirty_data_size; + dirty_record.non_rightmost_meta_page_num_ = 0; + dirty_record.rightmost_meta_page_num_ = 0; + mock_dirty_record_[fd] = dirty_record; + + ObTmpFileHandle file_handle; + file_handle.init(tmp_file); + ObSharedNothingTmpFile &file = *file_handle.get(); + file.file_size_ = mock_dirty_data_size; + file.cached_page_nums_ = + upper_align(mock_dirty_data_size, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + file_handles.push_back(file_handle); + } + ASSERT_EQ(file_num, file_handles.size()); +} + +TEST_F(TestFlushListIterator, test_iter_order) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector array; + + // dir 1 has 10KB * 4 + array.push_back(TestDirtyPageRecord(1/*dir*/, 10240/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(1/*dir*/, 10240/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(1/*dir*/, 10240/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(1/*dir*/, 10240/*dirty_data_size*/, 0, 0)); + + // dir 2 has (24KB * 4 + 2MB * 4) + array.push_back(TestDirtyPageRecord(2/*dir*/, 24576/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 24576/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 24576/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 24576/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 2097152/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 2097152/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 2097152/*dirty_data_size*/, 0, 0)); + array.push_back(TestDirtyPageRecord(2/*dir*/, 2097152/*dirty_data_size*/, 0, 0)); + + // iterate order: 2MB * 4, 10KB * 4, 4KB * 4 + for (auto &mock_record: array) { + int64_t fd = -1; + int64_t dir = mock_record.dir; + ret = MTL(ObTenantTmpFileManager *)->open(fd, dir); + ASSERT_EQ(OB_SUCCESS, ret); + + mock_dirty_record_[fd] = mock_record; + + ObTmpFileHandle file_handle; + ret = MTL(ObTenantTmpFileManager *)->get_tmp_file(fd, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &file = *file_handle.get(); + file.file_size_ = mock_record.dirty_data_size_; + file.cached_page_nums_ = upper_align(file.file_size_, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + ret = flush_prio_mgr.insert_data_flush_list(file, mock_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + printf("iterating file...\n"); + bool unused = false; + for (int64_t i = 0; i < 4; i++) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F1, unused, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &file = *file_handle.get(); + TestDirtyPageRecord &mock_record = mock_dirty_record_[file.fd_]; + ASSERT_EQ(mock_record.dirty_data_size_, file.file_size_); + printf("fd: %ld, file_size: %ld\n", file.fd_, file.file_size_); + } + + for (int64_t i = 0; i < 4; i++) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F2, unused, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &file = *file_handle.get(); + TestDirtyPageRecord &mock_record = mock_dirty_record_[file.fd_]; + ASSERT_EQ(mock_record.dirty_data_size_, file.file_size_); + ASSERT_EQ(24576, file.file_size_); + printf("fd: %ld, file_size: %ld\n", file.fd_, file.file_size_); + } + + for (int64_t i = 0; i < 4; i++) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F2, unused, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &file = *file_handle.get(); + TestDirtyPageRecord &mock_record = mock_dirty_record_[file.fd_]; + ASSERT_EQ(mock_record.dirty_data_size_, file.file_size_); + ASSERT_EQ(10240, file.file_size_); + printf("fd: %ld, file_size: %ld\n", file.fd_, file.file_size_); + } + + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F3, unused, file_handle); + ASSERT_EQ(OB_ITER_END, ret); +} + +TEST_F(TestFlushListIterator, test_iter_data_basic) +{ + int ret = OB_SUCCESS; + + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector file_handles; + int total_file_num = 0; + const int64_t FILE_NUM = 10; + + // 创建文件,根据层级生成模拟的脏页数量 + int64_t total_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + file_handles.clear(); + create_files(FlushCtxState(t), FILE_NUM, flush_prio_mgr, file_handles); + for (int64_t i = 0; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + total_file_cnt += 1; + } + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + // 遍历所有刷盘层级是否能取出该层级所有文件 + int64_t iter_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + for (int64_t i = 0; OB_SUCC(ret) && i < FILE_NUM * 5; ++i) { + bool is_meta; // unused + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState(t), is_meta, file_handle); + if (OB_SUCC(ret)) { + ASSERT_NE(file_handle.get(), nullptr); + iter_file_cnt += 1; + } + } + ASSERT_EQ(ret, OB_ITER_END); + ret = OB_SUCCESS; + } + + ASSERT_EQ(iter_file_cnt, total_file_cnt); +} + +TEST_F(TestFlushListIterator, test_iter_prev_stage) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector file_handles; + int total_file_num = 0; + const int64_t FILE_NUM = 10; + + // 创建文件,根据层级生成模拟的脏页数量 + int64_t total_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + file_handles.clear(); + create_files(FlushCtxState(t), FILE_NUM, flush_prio_mgr, file_handles); + for (int64_t i = 0; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + + total_file_cnt += 1; + } + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + // 直接开始遍历F3直至OB_ITER_END + bool is_meta; // unused + for (int64_t i = 0; OB_SUCC(ret) && i < FILE_NUM * 10; ++i) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F3, is_meta, file_handle); + if (OB_SUCC(ret)) { + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + } + } + ASSERT_EQ(OB_ITER_END, ret); + + // 切换到下一层级后,无法再遍历之前的层级 + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F1, is_meta, file_handle); + ASSERT_EQ(OB_ERR_UNEXPECTED, ret); +} + +// 某个文件从iterator返回、使用之后重新插回,如果迭代层级没有变更应该能被iterator重新取出 +TEST_F(TestFlushListIterator, test_iter_reinsert_file) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector file_handles; + int total_file_num = 0; + const int64_t FILE_NUM = 10; + + // 创建文件,根据层级生成模拟的脏页数量 + int64_t total_file_cnt = 0; + create_files(FlushCtxState::FSM_F1, FILE_NUM, flush_prio_mgr, file_handles); + for (int64_t i = 0; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + total_file_cnt += 1; + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + // 通过iterator取出所有文件 + bool is_meta; // unused + int64_t iter_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + for (int64_t i = 0; OB_SUCC(ret) && i < FILE_NUM * 5; ++i) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState(t), is_meta, file_handle); + if (OB_SUCC(ret)) { + ASSERT_NE(file_handle.get(), nullptr); + iter_file_cnt += 1; + } + } + ASSERT_EQ(ret, OB_ITER_END); + } + ASSERT_EQ(iter_file_cnt, total_file_cnt); + ret = OB_SUCCESS; + + ObTmpFileHandle file_handle = file_handles[0]; + ASSERT_NE(file_handle.get(), nullptr); + ret = flush_prio_mgr.insert_data_flush_list(*file_handle.get(), 2 * 1024 * 1024); + ASSERT_EQ(ret, OB_SUCCESS); + + file_handle.reset(); + ret = iter.next(FlushCtxState::FSM_F1, is_meta, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + + ret = iter.next(FlushCtxState::FSM_F1, is_meta, file_handle); + ASSERT_EQ(OB_ITER_END, ret); +} + +TEST_F(TestFlushListIterator, test_flush_list_remove) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector file_handles; + const int64_t FILE_NUM = 10; + + // 创建文件,根据层级生成模拟的脏页数量 + int64_t total_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + vector tmp_file_handles; + create_files(FlushCtxState(t), FILE_NUM, flush_prio_mgr, tmp_file_handles); + for (int64_t i = 0; i < tmp_file_handles.size(); ++i) { + ObTmpFileHandle file_handle = tmp_file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + total_file_cnt += 1; + } + file_handles.insert(file_handles.end(), tmp_file_handles.begin(), tmp_file_handles.end()); + } + + // 从文件链表中删除文件 + const int64_t rand_remove_cnt = ObRandom::rand(1, file_handles.size()); + for (int64_t i = 0; i < rand_remove_cnt; ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(nullptr, file_handle.get()); + ASSERT_EQ(OB_SUCCESS, ret); + ObSharedNothingTmpFile &file = *file_handle.get(); + ret = flush_prio_mgr.remove_file(false/*is_meta*/, file); + ASSERT_EQ(OB_SUCCESS, ret); + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + int64_t remain_file_cnt = 0; + for (int64_t t = FlushCtxState::FSM_F1; t < FlushCtxState::FSM_FINISHED; ++t) { + for (int64_t i = 0; OB_SUCC(ret) && i < FILE_NUM * 10; ++i) { + bool is_meta; // unused + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState(t), is_meta, file_handle); + if (OB_SUCC(ret)) { + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + int64_t fd = tmp_file.get_fd(); + remain_file_cnt += 1; + } + } + ASSERT_EQ(ret, OB_ITER_END); + ret = OB_SUCCESS; + } + ASSERT_EQ(remain_file_cnt + rand_remove_cnt, total_file_cnt); +} + +TEST_F(TestFlushListIterator, test_flush_list_update) +{ + int ret = OB_SUCCESS; + + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + vector file_handles; + int total_file_num = 0; + const int64_t FILE_NUM = 10; + + // 创建文件,根据层级生成模拟的脏页数量 + int64_t total_file_cnt = 0; + create_files(FlushCtxState::FSM_F1, FILE_NUM, flush_prio_mgr, file_handles); + + // 0.插入文件0~4 + for (int64_t i = 0; i < file_handles.size() / 2; ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + total_file_cnt += 1; + } + + // 1. 更新不在链表中的文件 + for (int64_t i = file_handles.size() / 2; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.update_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_ERR_UNEXPECTED, ret); + } + + // 2. 对层级没有变动的文件进行更新 + for (int64_t i = 0; i < file_handles.size() / 2; ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.update_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + } + + // 3. 更新到新的层级 + for (int64_t i = 0; i < file_handles.size() / 2; ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + dirty_record.dirty_data_size_ = 4096; + ret = flush_prio_mgr.update_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + } + + // 4. 插入文件5~10 + for (int64_t i = file_handles.size() / 2; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + total_file_cnt += 1; + } + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + // 5. F1中可以取出5个文件 + bool is_meta; // unused + for (int64_t i = 0; i < file_handles.size() / 2; ++i) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F1, is_meta, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + } + + // 6. F5中可以取出5个更新后的文件 + for (int64_t i = 0; i < file_handles.size() / 2; ++i) { + ObTmpFileHandle file_handle; + ret = iter.next(FlushCtxState::FSM_F3, is_meta, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + } +} + +TEST_F(TestFlushListIterator, test_flush_list_reinsert_after_use) +{ + int ret = OB_SUCCESS; + int64_t dir = -1; + ret = MTL(ObTenantTmpFileManager *)->alloc_dir(dir); + ASSERT_EQ(OB_SUCCESS, ret); + + ObTmpFileFlushPriorityManager flush_prio_mgr; + ret = flush_prio_mgr.init(); + ASSERT_EQ(OB_SUCCESS, ret); + ObTmpFilePageCacheController &pc_ctrl = MTL(ObTenantTmpFileManager *)->get_page_cache_controller(); + vector file_handles; + const int64_t FILE_NUM = 10; + FlushCtxState state = FlushCtxState::FSM_F2; + create_files_with_dir(state, dir, FILE_NUM, flush_prio_mgr, file_handles); + ASSERT_EQ(FILE_NUM, file_handles.size()); + + ObTmpFileFlushListIterator iter; + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + for (int64_t i = 0; i < file_handles.size(); ++i) { + ObTmpFileHandle file_handle = file_handles.at(i); + ASSERT_NE(file_handle.get(), nullptr); + ObSharedNothingTmpFile &tmp_file = *file_handle.get(); + TestDirtyPageRecord &dirty_record = mock_dirty_record_.at(tmp_file.get_fd()); + ret = flush_prio_mgr.insert_data_flush_list(tmp_file, dirty_record.dirty_data_size_); + ASSERT_EQ(OB_SUCCESS, ret); + } + + // 取出迭代器dir中一半的文件 + const int64_t USED_FILE_CNT = file_handles.size() / 2; + bool is_meta = false; // unused + for (int64_t i = 0; i < USED_FILE_CNT; ++i) { + ObTmpFileHandle file_handle; + ret = iter.next(state, is_meta, file_handle); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_NE(file_handle.get(), nullptr); + printf("use file %ld\n", file_handle.get()->get_fd()); + } + + // 迭代器中剩余文件重新插回flush_prio_mgr + iter.destroy(); + ret = iter.init(&flush_prio_mgr); + ASSERT_EQ(OB_SUCCESS, ret); + + // 重新初始化迭代器,通过迭代器取出剩余文件 + int64_t remain_file_cnt = 0; + while (OB_SUCC(ret)) { + ObTmpFileHandle file_handle; + if (OB_SUCC(iter.next(state, is_meta, file_handle))) { + remain_file_cnt += 1; + } + } + printf("remain_file_cnt:%ld, USED_FILE_CNT:%ld, FILE_NUM:%ld\n", remain_file_cnt, USED_FILE_CNT, FILE_NUM); + ASSERT_EQ(remain_file_cnt + USED_FILE_CNT, FILE_NUM); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_tmp_file_flush_list.log*"); + OB_LOGGER.set_file_name("test_tmp_file_flush_list.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file_meta_tree.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file_meta_tree.cpp new file mode 100644 index 0000000000..661818c7c4 --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file_meta_tree.cpp @@ -0,0 +1,2381 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include +#include +#include +#include +#define USING_LOG_PREFIX STORAGE +#define protected public +#define private public +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "share/ob_simple_mem_limit_getter.h" +#include "storage/tmp_file/ob_tmp_file_meta_tree.h" + +namespace oceanbase +{ +using namespace common; +using namespace blocksstable; +using namespace tmp_file; +using namespace storage; +using namespace share::schema; + +/* --------------------------------- Mock META TREE---------------------------------- */ +class ObTmpFileTestMetaTree : public ObSharedNothingTmpFileMetaTree +{ +public: + ObTmpFileTestMetaTree() : release_pages_() {} + ~ObTmpFileTestMetaTree(); + void reset(); +private: + virtual int release_meta_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t page_index_in_level); + virtual int release_tmp_file_page_(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num); + virtual int cache_page_for_write_(const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info); +public: + ObArray release_pages_; + //mock rightmost pages in read_cache, page_level is the index + //first means page buffer in read_cache + //second means read count in read_cache + ObArray> read_cache_rightmost_pages_; +}; + +ObTmpFileTestMetaTree::~ObTmpFileTestMetaTree() +{ + reset(); +} + +void ObTmpFileTestMetaTree::reset() +{ + release_pages_.reset(); +} + +int ObTmpFileTestMetaTree::release_meta_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t page_index_in_level) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(level_page_range_array_.count() <= page_info.page_level_ + || 0 > page_index_in_level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_ or page_info", KR(ret), K(fd_), K(level_page_range_array_), + K(page_info), K(page_index_in_level)); + } else if (is_page_in_write_cache(page_info)) { + const uint32_t start_page_id_in_array = level_page_range_array_.at(page_info.page_level_).start_page_id_; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_->free_page(fd_, page_info.buffer_page_id_, + ObTmpFilePageUniqKey(page_info.page_level_, page_index_in_level), next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page in write cache", KR(ret), K(fd_), K(page_info), K(page_index_in_level)); + } else if (OB_UNLIKELY(start_page_id_in_array != page_info.buffer_page_id_)) { + //NOTE: pages must be released sequentially (from front to back in array) + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), K(fd_), K(level_page_range_array_), K(page_info)); + } else { + level_page_range_array_.at(page_info.page_level_).start_page_id_ = next_page_id; + if (start_page_id_in_array == level_page_range_array_.at(page_info.page_level_).end_page_id_) { + //next_page_id must be invalid + level_page_range_array_.at(page_info.page_level_).end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } + if (ObTmpFileGlobal::INVALID_PAGE_ID == level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_) { + level_page_range_array_.at(page_info.page_level_).flushed_page_num_ += 1; + } + if (start_page_id_in_array == level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_) { + level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } + level_page_range_array_.at(page_info.page_level_).cached_page_num_ -= 1; + level_page_range_array_.at(page_info.page_level_).evicted_page_num_ += 1; + } + } + if (OB_SUCC(ret) && is_page_flushed(page_info)) { + if (OB_FAIL(release_tmp_file_page_(page_info.block_index_, page_info.physical_page_id_, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(page_info)); + } + } + if (FAILEDx(release_pages_.push_back(page_info.buffer_page_id_))) { + STORAGE_LOG(WARN, "fail to push_back", KR(ret), K(page_info)); + } + return ret; +} + +int ObTmpFileTestMetaTree::release_tmp_file_page_(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num) +{ + int ret = OB_SUCCESS; + stat_info_.all_type_flush_page_released_cnt_ += page_num; + return ret; +} + +int ObTmpFileTestMetaTree::cache_page_for_write_( + const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!page_info.is_valid() + || page_info.page_level_ >= level_page_range_array_.count() + || (ObTmpFileGlobal::INVALID_PAGE_ID != parent_page_id + && page_info.page_level_ + 1 >= level_page_range_array_.count()))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(page_info), K(level_page_range_array_), K(parent_page_id)); + } else { + if (!is_page_in_write_cache(page_info)) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buff = NULL; + int32_t level_page_index = level_page_range_array_[page_info.page_level_].evicted_page_num_ - 1; + ObTmpFilePageUniqKey page_key(page_info.page_level_, level_page_index); + if (OB_UNLIKELY(!is_page_flushed(page_info) + || 0 < level_page_range_array_[page_info.page_level_].cached_page_num_ + || 0 > level_page_index + || ObTmpFileGlobal::INVALID_PAGE_ID != level_page_range_array_[page_info.page_level_].end_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_info", KR(ret), K(fd_), K(page_info), K(level_page_range_array_)); + } else if (OB_FAIL(wbp_->alloc_page(fd_, page_key, new_page_id, new_page_buff))) { + STORAGE_LOG(WARN, "fail to alloc meta page", KR(ret), K(fd_), K(page_info), K(level_page_range_array_)); + } else if (OB_ISNULL(new_page_buff)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected null page buff", KR(ret), K(fd_), KP(new_page_buff)); + } else if (OB_FAIL(wbp_->notify_load(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load for meta", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } else { + if (OB_UNLIKELY(read_cache_rightmost_pages_.count() <= page_info.page_level_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected read_cache_rightmost_pages_", KR(ret), K(read_cache_rightmost_pages_.count()), K(page_info)); + } else { + MEMCPY(new_page_buff, read_cache_rightmost_pages_.at(page_info.page_level_).first, ObTmpFileGlobal::PAGE_SIZE); + read_cache_rightmost_pages_.at(page_info.page_level_).second++; + } + if (FAILEDx(check_page_(new_page_buff))) { + STORAGE_LOG(WARN, "the page is invalid or corrupted", KR(ret), K(fd_), KP(new_page_buff)); + } + if (OB_SUCC(ret)) { + //change page state to cached + if (OB_FAIL(wbp_->notify_load_succ(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load succ for meta", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } + } else { + int tmp_ret = OB_SUCCESS; + //change page state to invalid + if (OB_TMP_FAIL(wbp_->notify_load_fail(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load fail for meta", KR(tmp_ret), K(fd_), K(new_page_id), K(page_key)); + } + } + } + if (OB_SUCC(ret)) { + int64_t origin_block_index = page_info.block_index_; + int16_t origin_physical_page_id = page_info.physical_page_id_; + page_info.buffer_page_id_ = new_page_id; + page_info.block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + page_info.physical_page_id_ = -1; + if (ObTmpFileGlobal::INVALID_PAGE_ID == parent_page_id) { + root_item_ = page_info; + } else { + char *parent_page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileTreePageHeader page_header; + int32_t parent_level_page_index = level_page_range_array_[page_info.page_level_ + 1].evicted_page_num_ + + level_page_range_array_[page_info.page_level_ + 1].cached_page_num_ - 1; + ObTmpFilePageUniqKey parent_page_offset(page_info.page_level_ + 1, parent_level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, parent_page_id, parent_page_offset, parent_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(parent_page_id), K(parent_page_offset)); + } else if (OB_FAIL(read_page_header_(parent_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(parent_page_buff)); + } else if (OB_FAIL(rewrite_item_(parent_page_buff, page_header.item_num_ - 1, page_info))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), K(fd_), K(page_header), K(page_info), KP(parent_page_buff)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, parent_page_id, parent_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(parent_page_id), K(parent_page_offset)); + } + } + if (OB_SUCC(ret)) { + int16_t page_level = page_info.page_level_; + if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify dirty", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } else if (OB_FAIL(release_tmp_file_page_(origin_block_index, origin_physical_page_id, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(origin_block_index), K(origin_physical_page_id)); + } else { + level_page_range_array_[page_level].start_page_id_ = new_page_id; + level_page_range_array_[page_level].end_page_id_ = new_page_id; + level_page_range_array_[page_level].cached_page_num_++; + level_page_range_array_[page_level].evicted_page_num_--; + level_page_range_array_[page_level].flushed_page_num_--; + } + } + } else if (ObTmpFileGlobal::INVALID_PAGE_ID != new_page_id) { //fail + uint32_t unused_next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(wbp_->free_page(fd_, new_page_id, page_key, unused_next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page", KR(tmp_ret), K(fd_), K(new_page_id), K(page_key)); + } + } + STORAGE_LOG(INFO, "load page to write cache", KR(ret), K(fd_), K(page_info), K(page_key)); + } else { + //still in write cache + if (!is_page_in_write_cache(page_info)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_info", KR(ret), K(fd_), K(page_info)); + } + } + } + return ret; +} + +/* ---------------------------- Unittest Class ----------------------------- */ +class TestSNTmpFileMetaTree : public ::testing::Test +{ +public: + TestSNTmpFileMetaTree() = default; + virtual ~TestSNTmpFileMetaTree() = default; + static void SetUpTestCase(); + static void TearDownTestCase(); +public: + void generate_data_items(const int64_t item_num, + const int64_t start_virtual_page_id, + ObArray &data_items); + void generate_wrong_data_items(const int64_t item_num, + const int64_t start_virtual_page_id, + ObArray &data_items); + void test_tree_flush_with_truncate_occurs_before_update_meta( + int64_t truncate_offset, bool insert_after_truncate); +}; + +// static ObSimpleMemLimitGetter getter; + +//TODO: test data_item_array +void TestSNTmpFileMetaTree::SetUpTestCase() +{ + int ret = OB_SUCCESS; + // const int64_t bucket_num = 1024; + // const int64_t max_cache_size = 1024 * 1024 * 1024; + // const int64_t block_size = common::OB_MALLOC_BIG_BLOCK_SIZE; + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); + + // ret = getter.add_tenant(1, + // 8L * 1024L * 1024L, 2L * 1024L * 1024L * 1024L); + // ASSERT_EQ(OB_SUCCESS, ret); + // ret = ObKVGlobalCache::get_instance().init(&getter, bucket_num, max_cache_size, block_size); + // if (OB_INIT_TWICE == ret) { + // ret = OB_SUCCESS; + // } else { + // ASSERT_EQ(OB_SUCCESS, ret); + // } +} + +void TestSNTmpFileMetaTree::TearDownTestCase() +{ + MockTenantModuleEnv::get_instance().destroy(); + // ObKVGlobalCache::get_instance().destroy(); +} + +//mock data items +void TestSNTmpFileMetaTree::generate_data_items( + const int64_t item_num, + const int64_t start_virtual_page_id, + ObArray &data_items) +{ + int64_t block_index = 0; + int16_t physical_page_id = 0; + int16_t physical_page_num = 128; //(OB_DEFAULT_MACRO_BLOCK_SIZE / ObTmpFileGlobal::PAGE_SIZE) / 2 + int64_t virtual_page_id = start_virtual_page_id; + ObSharedNothingTmpFileDataItem data_item; + for (int64_t i = block_index; i < item_num; i++) { + data_item.reset(); + data_item.block_index_ = block_index; + data_item.physical_page_id_ = physical_page_id; + data_item.physical_page_num_ = physical_page_num; + data_item.virtual_page_id_ = virtual_page_id; + block_index++; + virtual_page_id += physical_page_num; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(data_item)); + } +} + +void TestSNTmpFileMetaTree::generate_wrong_data_items( + const int64_t item_num, + const int64_t start_virtual_page_id, + ObArray &data_items) +{ + int64_t block_index = 0; + int16_t physical_page_id = 0; + int16_t physical_page_num = 128; //(OB_DEFAULT_MACRO_BLOCK_SIZE / ObTmpFileGlobal::PAGE_SIZE) / 2 + int64_t virtual_page_id = start_virtual_page_id; + ObSharedNothingTmpFileDataItem data_item; + for (int64_t i = block_index; i < item_num; i++) { + data_item.reset(); + data_item.block_index_ = block_index; + data_item.physical_page_id_ = physical_page_id; + data_item.physical_page_num_ = physical_page_num; + data_item.virtual_page_id_ = virtual_page_id > 0 ? virtual_page_id - std::rand() % 50 - 1 : 0; + block_index++; + virtual_page_id += physical_page_num; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(data_item)); + } +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_insert) +{ + STORAGE_LOG(INFO, "=======================test_tree_insert begin======================="); + ObTmpWriteBufferPool wbp; + // wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 30; + ObArray data_items; + ObArray data_items_1; + ObArray data_items_2; + ObArray data_items_3; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first insert======================="); + //insert 9 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 9; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + //we assume that there are some disk flushing and other operations here. + + STORAGE_LOG(INFO, "=======================second insert======================="); + //insert 18 items (insert one by one) + for (int64_t i = 9; i < 27; i++) { + data_items_2.reset(); + ASSERT_EQ(OB_SUCCESS, data_items_2.push_back(data_items.at(i))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + } + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + //we assume that there are some disk flushing and other operations here. + + STORAGE_LOG(INFO, "=======================third insert======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 27; i < item_num; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_3.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_3)); + ASSERT_EQ(4, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 30 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_insert end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_insert_fail) +{ + STORAGE_LOG(INFO, "=======================test_tree_insert_fail begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 750; + ObArray data_items; + ObArray data_items_1; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first insert======================="); + //insert 750 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_ALLOCATE_TMP_FILE_PAGE_FAILED, meta_tree_.insert_items(data_items)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(false, meta_tree_.root_item_.is_valid()); + + //we assume that there are some disk flushing and other operations here. + + STORAGE_LOG(INFO, "=======================second insert======================="); + //insert 90 items (insert one by one) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 90; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(5, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(30, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(10, meta_tree_.level_page_range_array_.at(1).cached_page_num_); + ASSERT_EQ(4, meta_tree_.level_page_range_array_.at(2).cached_page_num_); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.at(3).cached_page_num_); + ASSERT_EQ(1, meta_tree_.level_page_range_array_.at(4).cached_page_num_); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 90 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_insert_fail end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_read) +{ + STORAGE_LOG(INFO, "=======================test_tree_read begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 25; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + //we assume that there are some disk flushing and other operations here. + //After simulating the insert, we can see that the tmp file offset is [0, 25 * 128 * 8K], + // with each data item occupying 128 * 8K. + STORAGE_LOG(INFO, "=======================first tree read======================="); + ObArray get_data_items; + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(1, 2 * 128 * ObTmpFileGlobal::PAGE_SIZE - 2, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(2, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================second tree read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(7 * 128 * ObTmpFileGlobal::PAGE_SIZE, 5 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(5, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================third tree read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(0, 4 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(4, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================forth tree read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(0, 25 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(25, get_data_items.count()); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 25 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_read end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush) +{ + STORAGE_LOG(INFO, "=======================test_tree_flush begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 28; + ObArray data_items; + ObArray data_items_1; + ObArray data_items_2; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 25; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(13, total_need_flush_page_num); + ASSERT_EQ(3, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================first tree flush======================="); + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context_1; + ObArray tree_io_array_1; + int64_t write_offset_1 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_1, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array_1)); + ASSERT_EQ(3, tree_io_array_1.count()); + ASSERT_EQ(0 + 13 * ObTmpFileGlobal::PAGE_SIZE, write_offset_1); + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + //insert 3 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 25; i < 28; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_2.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(4, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(5, total_need_flush_page_num); + ASSERT_EQ(4, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================second tree flush======================="); + //NOTE: We will not flush the tree again before io returns successfully. + //So here we assume that io is successful and call this function "update_after_flush". + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context_2; + ObArray tree_io_array_2; + int64_t write_offset_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_2, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array_2)); + ASSERT_EQ(4, tree_io_array_2.count()); + //flush 7 dirty pages. + //During the flushing process, + // disk information will be changed to the upper-level pages, so upper-level pages will become dirty pages during this process. + // So you shouldn’t be surprised that the number of dirty pages changed from 5 to 7 + ASSERT_EQ(0 + 7 * ObTmpFileGlobal::PAGE_SIZE, write_offset_2); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + for (int64_t i = 0; i < 4; i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_2.at(i); + ASSERT_EQ(tree_io.flush_end_page_id_, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 28 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + delete[] block_buff_1; + delete[] block_buff_2; + STORAGE_LOG(INFO, "=======================test_tree_flush end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_multi_io) +{ + STORAGE_LOG(INFO, "=======================test_tree_flush_with_multi_io begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 25; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(13, total_need_flush_page_num); + ASSERT_EQ(3, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================tree flush======================="); + STORAGE_LOG(INFO, "=======================first block======================="); + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - 3 * ObTmpFileGlobal::PAGE_SIZE; //this block can only accommodate 3 pages + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array_1)); + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + + STORAGE_LOG(INFO, "=======================second block======================="); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_2; + int64_t write_offset_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array_2)); + ASSERT_EQ(3, tree_io_array_2.count()); + ASSERT_EQ(0 + 10 * ObTmpFileGlobal::PAGE_SIZE, write_offset_2); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 28 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + delete[] block_buff_1; + delete[] block_buff_2; + STORAGE_LOG(INFO, "=======================test_tree_flush_with_multi_io end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_major_flush) +{ + STORAGE_LOG(INFO, "=======================test_tree_major_flush begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 25; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================tree flush======================="); + char *block_buff = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context; + ObArray tree_io_array; + int64_t write_offset = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::MAJOR, + block_buff, + write_offset, + flush_context, + tree_io_array)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array)); + ASSERT_EQ(2, tree_io_array.count()); + ASSERT_EQ(0 + 10 * ObTmpFileGlobal::PAGE_SIZE, write_offset); //flush 10 pages + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(3, total_need_flush_page_num); + ASSERT_EQ(3, total_need_flush_rightmost_page_num); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array)); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 25 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + delete[] block_buff; + STORAGE_LOG(INFO, "=======================test_tree_major_flush end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_evict) +{ + STORAGE_LOG(INFO, "=======================test_tree_evict begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 25; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================first tree flush======================="); + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context_1; + ObArray tree_io_array_1; + int64_t write_offset_1 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::MAJOR, + block_buff_1, + write_offset_1, + flush_context_1, + tree_io_array_1)); + ASSERT_EQ(2, tree_io_array_1.count()); + ASSERT_EQ(2, tree_io_array_1.at(1).flush_nums_); + ASSERT_EQ(0 + 10 * ObTmpFileGlobal::PAGE_SIZE, write_offset_1); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + + STORAGE_LOG(INFO, "=======================first tree evict======================="); + int64_t total_need_evict_page_num = 0; + int64_t total_need_evict_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_evict_page_num(total_need_evict_page_num, total_need_evict_rightmost_page_num)); + ASSERT_EQ(total_need_evict_page_num, 10); + ASSERT_EQ(total_need_evict_rightmost_page_num, 0); + int64_t actual_evict_page_num_1 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.evict_meta_pages(13, + ObTmpFileTreeEvictType::FULL, + actual_evict_page_num_1)); + ASSERT_EQ(actual_evict_page_num_1, 10); + + STORAGE_LOG(INFO, "=======================second tree flush======================="); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObTmpFileTreeFlushContext flush_context_2; + ObArray tree_io_array_2; + int64_t write_offset_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_2, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array", K(tree_io_array_2)); + ASSERT_EQ(3, tree_io_array_2.count()); + ASSERT_EQ(0 + 3 * ObTmpFileGlobal::PAGE_SIZE, write_offset_2); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + + STORAGE_LOG(INFO, "=======================second tree evict======================="); + int64_t actual_evict_page_num_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.evict_meta_pages(3, + ObTmpFileTreeEvictType::FULL, + actual_evict_page_num_2)); + ASSERT_EQ(actual_evict_page_num_2, 3); + for (int64_t i = 0; i < 3; i++) { + ASSERT_EQ(0, meta_tree_.level_page_range_array_.at(i).cached_page_num_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(i).start_page_id_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(i).end_page_id_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(i).flushed_end_page_id_); + } + + meta_tree_.reset(); + delete[] block_buff_1; + delete[] block_buff_2; + STORAGE_LOG(INFO, "=======================test_tree_evict end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_clear) +{ + STORAGE_LOG(INFO, "=======================test_tree_clear begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + const int64_t item_num = 25; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================tree insert======================="); + //insert 25 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================tree clear======================="); + ObSharedNothingTmpFileMetaItem origin_root_item = meta_tree_.root_item_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(0, 25 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(13, meta_tree_.release_pages_.count()); + ASSERT_EQ(origin_root_item.buffer_page_id_, meta_tree_.release_pages_.at(12)); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_clear end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_truncate) +{ + STORAGE_LOG(INFO, "=======================test_tree_truncate begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + const int64_t item_num = 75; + ObArray data_items; + ObArray data_items_1; + ObArray data_items_2; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 5 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 5; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(5, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "data_item_array", K(meta_tree_.data_item_array_)); + + STORAGE_LOG(INFO, "=======================first tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, 3 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "data_item_array", K(meta_tree_.data_item_array_)); + ASSERT_EQ(2, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + //insert 70 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 5; i < 75; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_2.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================second tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(3 * 128 * ObTmpFileGlobal::PAGE_SIZE, 26 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + ASSERT_EQ(4, meta_tree_.release_pages_.count()); + + STORAGE_LOG(INFO, "=======================third tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(26 * 128 * ObTmpFileGlobal::PAGE_SIZE, 28 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + ASSERT_EQ(6, meta_tree_.release_pages_.count()); + + STORAGE_LOG(INFO, "=======================first tree read======================="); + ObArray get_data_items; + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(28 * 128 * ObTmpFileGlobal::PAGE_SIZE, 20 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + ASSERT_EQ(20, get_data_items.count()); + ASSERT_EQ(28 * 128, get_data_items.at(0).virtual_page_id_); + + STORAGE_LOG(INFO, "=======================forth tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(28 * 128 * ObTmpFileGlobal::PAGE_SIZE, 73 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + ASSERT_EQ(16, meta_tree_.release_pages_.count()); + + STORAGE_LOG(INFO, "=======================fifth tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(73 * 128 * ObTmpFileGlobal::PAGE_SIZE, 75 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + ASSERT_EQ(19, meta_tree_.release_pages_.count()); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(75 * 128 * ObTmpFileGlobal::PAGE_SIZE, 75 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_truncate end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_truncate_with_unfilled_page) +{ + STORAGE_LOG(INFO, "=======================test_tree_truncate_with_unfilled_page begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + const int64_t item_num = 75; + ObArray data_items; + ObArray data_items_1; + ObArray data_items_2; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 25 items (insert a array) + //each data item contains 128 pages, we assume that the last page of the last data item is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 25; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================first tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, (24 * 128 + 1 * 127.5) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(24 * 128 * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(4, meta_tree_.release_pages_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================load unfilled page to write cache======================="); + ObSharedNothingTmpFileDataItem last_data_item; + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true/*release_tail_in_disk*/)); + + //then, we write some data in write buffer + + STORAGE_LOG(INFO, "=======================second tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate((24 * 128 + 1 * 127.5) * ObTmpFileGlobal::PAGE_SIZE, (24 * 128 + 1 * 127.8) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ((24 * 128 + 1 *127) * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(6, meta_tree_.release_pages_.count()); + + STORAGE_LOG(INFO, "=======================third tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate((24 * 128 + 1 * 127.8) * ObTmpFileGlobal::PAGE_SIZE, 28 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + ASSERT_EQ((24 * 128 + 1 *127) * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(6, meta_tree_.release_pages_.count()); + + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + //insert 47 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 28; i < 75; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_2.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(10, meta_tree_.level_page_range_array_[0].evicted_page_num_ + meta_tree_.level_page_range_array_[0].cached_page_num_); + ASSERT_EQ(2, meta_tree_.level_page_range_array_[1].evicted_page_num_ + meta_tree_.level_page_range_array_[1].cached_page_num_); + ASSERT_EQ(1, meta_tree_.level_page_range_array_[2].evicted_page_num_ + meta_tree_.level_page_range_array_[2].cached_page_num_); + + STORAGE_LOG(INFO, "=======================forth tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(28 * 128 * ObTmpFileGlobal::PAGE_SIZE, 75 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + ASSERT_EQ(6 + 13, meta_tree_.release_pages_.count()); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(false, meta_tree_.root_item_.is_valid()); + ASSERT_EQ(75 * 128 * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(75 * 128 * ObTmpFileGlobal::PAGE_SIZE, 80 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_truncate_with_unfilled_page end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_truncate_with_data_item_remove) +{ + STORAGE_LOG(INFO, "=======================test_tree_truncate_with_data_item_remove begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 9; + ObArray data_items; + ObArray data_items_1; + ObArray data_items_2; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + //generate last data item, we assume that the page of the last data item is an unfilled page. + ObSharedNothingTmpFileDataItem last_data_item; + last_data_item.block_index_ = item_num; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = item_num * 128; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 10 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + for (int64_t i = 0; i < 10; i++) { + ASSERT_EQ(OB_SUCCESS, data_items_1.push_back(data_items.at(i))); + } + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + meta_tree_.print_meta_tree_total_info(); + + STORAGE_LOG(INFO, "=======================first tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, (9 * 128 + 1 * 0.5) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(9 * 128 * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(1, meta_tree_.release_pages_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================load unfilled page to write cache======================="); + last_data_item.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true/*release_tail_in_disk*/)); + + //then, we write some data in write buffer + + STORAGE_LOG(INFO, "=======================second tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate((9 * 128 + 1 * 0.5) * ObTmpFileGlobal::PAGE_SIZE, (9 * 128 + 1 * 0.8) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(9 * 128 * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(3, meta_tree_.release_pages_.count()); + ASSERT_EQ(false, meta_tree_.root_item_.is_valid()); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + item_num = 10; + generate_data_items(item_num, 9 * 128, data_items_2); + //generate last data item, we assume that the page of the last data item is an unfilled page. + last_data_item.reset(); + last_data_item.block_index_ = 30; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = (9 + 10) * 128; + ASSERT_EQ(OB_SUCCESS, data_items_2.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items_2.count()); + //insert 11 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + // The third leaf page has only one data item (with an unfilled page) + ASSERT_EQ(3, meta_tree_.level_page_range_array_[0].evicted_page_num_ + meta_tree_.level_page_range_array_[0].cached_page_num_); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + meta_tree_.print_meta_tree_total_info(); + + STORAGE_LOG(INFO, "=======================third tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate((9 * 128 + 1 * 0.8) * ObTmpFileGlobal::PAGE_SIZE, (9 * 128 + 10 * 128 + 1 * 0.5) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ((9 * 128 + 10 * 128) * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(3 + 2, meta_tree_.release_pages_.count()); + meta_tree_.print_meta_tree_total_info(); + + STORAGE_LOG(INFO, "=======================load unfilled page to write cache======================="); + last_data_item.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true/*release_tail_in_disk*/)); + + STORAGE_LOG(INFO, "=======================forth tree truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate((9 * 128 + 10 * 128 + 1 * 0.5) * ObTmpFileGlobal::PAGE_SIZE, (9 * 128 + 10 * 128 + 1 * 0.6) * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ((9 * 128 + 10 * 128) * ObTmpFileGlobal::PAGE_SIZE, meta_tree_.released_offset_); + ASSERT_EQ(3 + 2 + 2, meta_tree_.release_pages_.count()); + ASSERT_EQ(false, meta_tree_.root_item_.is_valid()); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear((9 * 128 + 10 * 128 + 1 * 0.6) * ObTmpFileGlobal::PAGE_SIZE, 30 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_truncate_with_data_item_remove end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_occurs_between_buf_generation_opers) +{ + STORAGE_LOG(INFO, "====test_tree_flush_with_truncate_occurs_between_buf_generation_opers begin=="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 19; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + //generate last data item, we assume that the page of the last data item is an unfilled page. + ObSharedNothingTmpFileDataItem last_data_item; + last_data_item.block_index_ = item_num; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = item_num * 128; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 10 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(5, total_need_flush_page_num); + ASSERT_EQ(2, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================first round tree flush ============================"); + ObTmpFileTreeFlushContext flush_context_first; + + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - ObTmpFileGlobal::PAGE_SIZE; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + + int64_t truncate_offset = 5 * 128 * ObTmpFileGlobal::PAGE_SIZE; //truncate one meta page + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, truncate_offset)); + ASSERT_EQ(1, meta_tree_.release_pages_.count()); + + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_2 = OB_DEFAULT_MACRO_BLOCK_SIZE - 4 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_2; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_first, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array_2", K(tree_io_array_2)); + + ASSERT_EQ(0, tree_io_array_2.count()); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(4, total_need_flush_page_num); + ASSERT_EQ(2, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + for (int64_t i = 0; i < tree_io_array_1.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_1.at(i); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + + STORAGE_LOG(INFO, "=======================second round tree flush ============================"); + ObTmpFileTreeFlushContext flush_context_second; + + char *block_buff_3 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_3 = OB_DEFAULT_MACRO_BLOCK_SIZE - 4 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_3; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(2/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_3, + write_offset_3, + flush_context_second, + tree_io_array_3)); + STORAGE_LOG(INFO, "tree_io_array_3", K(tree_io_array_3)); + + ASSERT_EQ(2, tree_io_array_3.count()); + ASSERT_EQ(true, flush_context_second.is_meta_reach_end_); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_3)); + for (int64_t i = 0; i < tree_io_array_3.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_3.at(i); + ASSERT_EQ(tree_io.flush_end_page_id_, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(truncate_offset, 30 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + delete[] block_buff_1; + delete[] block_buff_2; + delete[] block_buff_3; + meta_tree_.reset(); + STORAGE_LOG(INFO, "====test_tree_flush_with_truncate_occurs_between_buf_generation_opers end=="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_occurs_between_buf_generation_opers_2) +{ + STORAGE_LOG(INFO, "==test_tree_flush_with_truncate_occurs_between_buf_generation_opers_2 begin==="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 19; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + //generate last data item, we assume that the page of the last data item is an unfilled page. + ObSharedNothingTmpFileDataItem last_data_item; + last_data_item.block_index_ = item_num; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = item_num * 128; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 10 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(5, total_need_flush_page_num); + ASSERT_EQ(2, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================first tree flush ============================"); + ObTmpFileTreeFlushContext flush_context_first; + + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - 3 * ObTmpFileGlobal::PAGE_SIZE; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + + int64_t truncate_offset = 5 * 1 * 128 * ObTmpFileGlobal::PAGE_SIZE; + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, truncate_offset)); + + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_2 = OB_DEFAULT_MACRO_BLOCK_SIZE - 2 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_2; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_first, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array_2", K(tree_io_array_2)); + ASSERT_EQ(2, tree_io_array_2.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_2); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + + for (int64_t i = 0; i < tree_io_array_2.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_2.at(i); + ASSERT_EQ(tree_io.flush_end_page_id_, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + + delete[] block_buff_1; + delete[] block_buff_2; + meta_tree_.reset(); + STORAGE_LOG(INFO, "==test_tree_flush_with_truncate_occurs_between_buf_generation_opers_2 end==="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_occurs_between_buf_generation_opers_3) +{ + STORAGE_LOG(INFO, "====test_tree_flush_with_truncate_occurs_between_buf_generation_opers_3 begin=="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 19; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + //generate last data item, we assume that the page of the last data item is an unfilled page. + ObSharedNothingTmpFileDataItem last_data_item; + last_data_item.block_index_ = item_num; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = item_num * 128; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 10 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(5, total_need_flush_page_num); + ASSERT_EQ(2, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "=======================first tree flush ============================"); + ObTmpFileTreeFlushContext flush_context_first; + + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - ObTmpFileGlobal::PAGE_SIZE; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + + int64_t truncate_offset = 5 * 4 * 128 * ObTmpFileGlobal::PAGE_SIZE; + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, truncate_offset)); + + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_2 = OB_DEFAULT_MACRO_BLOCK_SIZE - 4 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_2; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_first, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array_2", K(tree_io_array_2)); + + ASSERT_EQ(0, tree_io_array_2.count()); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + ASSERT_NE(flush_context_first.tree_epoch_, meta_tree_.tree_epoch_); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + for (int64_t i = 0; i < tree_io_array_1.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_1.at(i); + ASSERT_NE(tree_io.tree_epoch_, meta_tree_.tree_epoch_); + } + + delete[] block_buff_1; + delete[] block_buff_2; + meta_tree_.reset(); + STORAGE_LOG(INFO, "====test_tree_flush_with_truncate_occurs_between_buf_generation_opers_3 end=="); +} + +void TestSNTmpFileMetaTree::test_tree_flush_with_truncate_occurs_before_update_meta( + int64_t truncate_offset, bool insert_after_truncate) +{ + STORAGE_LOG(INFO, "===============test_tree_flush_with_truncate_occurs_before_update_meta begin ===="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 70; + ObArray data_items; + ObArray data_items_1; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + //generate last data item, we assume the last data item only contain one page. + ObSharedNothingTmpFileDataItem last_data_item; + last_data_item.block_index_ = item_num; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = item_num * 128; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); + item_num++; + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + //insert 10 items (insert a array) + // last data item has only one page, and the page is an unfilled page. + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================check tree flush pages======================="); + int64_t total_need_flush_page_num = 0; + int64_t total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(15 + 3 + 1, total_need_flush_page_num); + ASSERT_EQ(3, total_need_flush_rightmost_page_num); + + STORAGE_LOG(INFO, "======================= tree flush ============================"); + ObTmpFileTreeFlushContext flush_context; + + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - ObTmpFileGlobal::PAGE_SIZE; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_2 = OB_DEFAULT_MACRO_BLOCK_SIZE - 18 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_2; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array_2", K(tree_io_array_2)); + + ASSERT_EQ(3, tree_io_array_2.count()); + ASSERT_EQ(true, flush_context.is_meta_reach_end_); + + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.get_need_flush_page_num(total_need_flush_page_num, total_need_flush_rightmost_page_num)); + ASSERT_EQ(0, total_need_flush_page_num); + ASSERT_EQ(0, total_need_flush_rightmost_page_num); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, truncate_offset)); + + if (insert_after_truncate) { + STORAGE_LOG(INFO, "======================= second tree insert ======================="); + int64_t item_num_2 = 20; + ObArray data_items_1; + //even if truncate_offset is not an integer multiple of PAGE_SIZE, this result is reasonable. + //the start_virtual_page_id we insert into the tree may be smaller than the actual truncate_offset of the tmp file. + int64_t start_virtual_page_id = + MAX(truncate_offset / ObTmpFileGlobal::PAGE_SIZE, last_data_item.virtual_page_id_ + last_data_item.physical_page_num_); + generate_data_items(item_num_2, start_virtual_page_id, data_items_1); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + } + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + + if (insert_after_truncate) { + ASSERT_EQ(true, meta_tree_.root_item_.is_valid()); + if (truncate_offset >= (last_data_item.virtual_page_id_ + last_data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE) { + for (int64_t i = 0; i < tree_io_array_2.count(); i++) { + ASSERT_NE(tree_io_array_2.at(i).tree_epoch_, meta_tree_.tree_epoch_); + } + } else { + for (int64_t i = 0; i < tree_io_array_2.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_2.at(i); + ASSERT_EQ(tree_io.flush_end_page_id_, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + } + } else { + if (truncate_offset >= (last_data_item.virtual_page_id_ + last_data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE) { + ASSERT_EQ(false, meta_tree_.root_item_.is_valid()); + } else { + for (int64_t i = 0; i < tree_io_array_2.count(); i++) { + ObTmpFileTreeIOInfo& tree_io = tree_io_array_2.at(i); + ASSERT_EQ(tree_io.flush_end_page_id_, meta_tree_.level_page_range_array_.at(tree_io.page_level_).flushed_end_page_id_); + } + } + } + + ASSERT_EQ(OB_SUCCESS, meta_tree_.clear(truncate_offset, 100 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + delete[] block_buff_1; + delete[] block_buff_2; + meta_tree_.reset(); + STORAGE_LOG(INFO, "=================test_tree_flush_with_truncate_occurs_before_update_meta end======="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_to_somewhere_in_the_middle_before_update_meta) +{ + test_tree_flush_with_truncate_occurs_before_update_meta(5 * 128 * ObTmpFileGlobal::PAGE_SIZE, false); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_to_end_before_update_meta) +{ + test_tree_flush_with_truncate_occurs_before_update_meta(30 * 128 * ObTmpFileGlobal::PAGE_SIZE, false); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_to_somewhere_in_the_middle_before_update_meta_2) +{ + test_tree_flush_with_truncate_occurs_before_update_meta(8 * 128 * ObTmpFileGlobal::PAGE_SIZE, true); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_flush_with_truncate_to_end_before_update_meta_2) +{ + test_tree_flush_with_truncate_occurs_before_update_meta(10 * 128 * ObTmpFileGlobal::PAGE_SIZE, true); +} + +//================More detailed tests of meta tree involve more test points================== + +//=========================================insert============================================ +TEST_F(TestSNTmpFileMetaTree, test_array_insert) +{ + STORAGE_LOG(INFO, "=======================test_array_insert begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(100); + meta_tree_.set_max_page_item_cnt(100); + int64_t item_num = 30; + ObArray data_items; + ObArray data_items_1; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first insert======================="); + //insert 30 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(30, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second insert======================="); + item_num = 71; + generate_data_items(item_num, 30 * 128, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + //insert 71 items (insert one by one) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_array_insert end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_prepare_for_insert) +{ + STORAGE_LOG(INFO, "=================test_tree_prepare_for_insert begin==============="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + int64_t item_num = 9; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================first tree flush and evict========================="); + ObTmpFileTreeFlushContext flush_context_first; + + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - 2 * ObTmpFileGlobal::PAGE_SIZE; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + char *rightmost_page_buf = block_buff_1 + OB_DEFAULT_MACRO_BLOCK_SIZE - ObTmpFileGlobal::PAGE_SIZE; + meta_tree_.read_cache_rightmost_pages_.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf, 0))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + int64_t actual_evict_page_num = -1; + ASSERT_EQ(OB_SUCCESS, meta_tree_.evict_meta_pages(3, ObTmpFileTreeEvictType::FULL, actual_evict_page_num)); + ASSERT_EQ(2, actual_evict_page_num); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(0).end_page_id_); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + item_num = 6; + ObArray data_items_1; + generate_data_items(item_num, 9 * 128, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(1, meta_tree_.read_cache_rightmost_pages_.at(0).second); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(1, meta_tree_.level_page_range_array_.at(0).evicted_page_num_); + STORAGE_LOG(INFO, "level_page_range_array", K(meta_tree_.level_page_range_array_)); + + STORAGE_LOG(INFO, "=======================second tree flush and evict========================="); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + int64_t write_offset_2 = OB_DEFAULT_MACRO_BLOCK_SIZE - 3 * ObTmpFileGlobal::PAGE_SIZE; + ObArray tree_io_array_2; + ObTmpFileTreeFlushContext flush_context_second; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_second, + tree_io_array_2)); + STORAGE_LOG(INFO, "tree_io_array_2", K(tree_io_array_2)); + ASSERT_EQ(2, tree_io_array_2.count()); + ASSERT_EQ(true, flush_context_second.is_meta_reach_end_); + + char *rightmost_page_buf_0 = block_buff_2 + OB_DEFAULT_MACRO_BLOCK_SIZE - 2 * ObTmpFileGlobal::PAGE_SIZE; + char *rightmost_page_buf_1 = block_buff_2 + OB_DEFAULT_MACRO_BLOCK_SIZE - 1 * ObTmpFileGlobal::PAGE_SIZE; + meta_tree_.read_cache_rightmost_pages_.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_0, 0))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_1, 0))); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_2)); + actual_evict_page_num = -1; + ASSERT_EQ(OB_SUCCESS, meta_tree_.evict_meta_pages(3, ObTmpFileTreeEvictType::FULL, actual_evict_page_num)); + ASSERT_EQ(3, actual_evict_page_num); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(1).end_page_id_); + + STORAGE_LOG(INFO, "=======================third tree insert======================="); + item_num = 20; + ObArray data_items_2; + generate_data_items(item_num, 15 * 128, data_items_2); + ASSERT_EQ(item_num, data_items_2.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(1, meta_tree_.read_cache_rightmost_pages_.at(0).second); + ASSERT_EQ(1, meta_tree_.read_cache_rightmost_pages_.at(1).second); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(5, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.at(0).evicted_page_num_); + + delete[] block_buff_1; + delete[] block_buff_2; + meta_tree_.reset(); + STORAGE_LOG(INFO, "=================test_tree_prepare_for_insert begin==============="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_prepare_for_insert_fail) +{ + STORAGE_LOG(INFO, "=======================test_tree_prepare_for_insert_fail begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(2); + + STORAGE_LOG(INFO, "=======================first insert======================="); + int64_t item_num = 10; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(4, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================first tree flush and evict========================="); + ObTmpFileTreeFlushContext flush_context_first; + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = 0; + + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + ASSERT_EQ(4, tree_io_array_1.count()); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + char *rightmost_page_buf_0 = block_buff_1 + 4 * ObTmpFileGlobal::PAGE_SIZE; + char *rightmost_page_buf_1 = block_buff_1 + 7 * ObTmpFileGlobal::PAGE_SIZE; + char *rightmost_page_buf_2 = block_buff_1 + 9 * ObTmpFileGlobal::PAGE_SIZE; + char *rightmost_page_buf_3 = block_buff_1 + 10 * ObTmpFileGlobal::PAGE_SIZE; + meta_tree_.read_cache_rightmost_pages_.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_0, 0))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_1, 0))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_2, 0))); + ASSERT_EQ(OB_SUCCESS, meta_tree_.read_cache_rightmost_pages_.push_back(std::make_pair(rightmost_page_buf_3, 0))); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + int64_t actual_evict_page_num = -1; + ASSERT_EQ(OB_SUCCESS, meta_tree_.evict_meta_pages(12, ObTmpFileTreeEvictType::FULL, actual_evict_page_num)); + ASSERT_EQ(11, actual_evict_page_num); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(3).end_page_id_); + + STORAGE_LOG(INFO, "=======================build new meta tree======================="); + //We build a new meta tree to take up most of the write cache memory + ObTmpFileTestMetaTree meta_tree_1_; + ASSERT_EQ(OB_SUCCESS, meta_tree_1_.init(2, &wbp, &callback_allocator)); + meta_tree_1_.set_max_array_item_cnt(2); + meta_tree_1_.set_max_page_item_cnt(2); + + STORAGE_LOG(INFO, "=======================new meta tree insert======================="); + item_num = 252; + ObArray data_items_1; + generate_data_items(item_num, 0/*start virtual page id*/, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_1_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_1_.insert_items(data_items_1)); + ASSERT_EQ(8, meta_tree_1_.level_page_range_array_.count());//252 pages + + STORAGE_LOG(INFO, "=======================second insert======================="); + ASSERT_EQ(OB_ALLOCATE_TMP_FILE_PAGE_FAILED, meta_tree_.prepare_for_insert_items()); + ASSERT_NE(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(3).end_page_id_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(2).end_page_id_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(1).end_page_id_); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, meta_tree_.level_page_range_array_.at(0).end_page_id_); + + STORAGE_LOG(INFO, "=======================new meta tree clear======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_1_.clear(0, 252 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + + STORAGE_LOG(INFO, "=======================third insert======================="); + item_num = 10; + ObArray data_items_2; + generate_data_items(item_num, 10 * 128/*start virtual page id*/, data_items_2); + ASSERT_EQ(item_num, data_items_2.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(5, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(6, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(4, meta_tree_.level_page_range_array_.at(0).evicted_page_num_); + + delete[] block_buff_1; + meta_tree_.reset(); + meta_tree_1_.reset(); + STORAGE_LOG(INFO, "=======================test_tree_prepare_for_insert_fail end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_insert_fail_after_array_used) +{ + STORAGE_LOG(INFO, "================test_tree_insert_fail_after_array_used begin================"); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(3); + meta_tree_.set_max_page_item_cnt(3); + STORAGE_LOG(INFO, "=======================first insert======================="); + int64_t item_num = 2; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + //insert 750 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(2, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second insert======================="); + item_num = 10; + ObArray data_items_1; + generate_data_items(item_num, 1 * 128/*start virtual page id*/, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_NE(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.data_item_array_.count()); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================third insert======================="); + item_num = 600; + ObArray data_items_2; + generate_data_items(item_num, 2 * 128/*start virtual page id*/, data_items_2); + ASSERT_EQ(item_num, data_items_2.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_ALLOCATE_TMP_FILE_PAGE_FAILED, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(2, meta_tree_.data_item_array_.count()); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "================test_tree_insert_fail_after_array_used begin================"); +} + +TEST_F(TestSNTmpFileMetaTree, test_tree_insert_fail_after_tree_build) +{ + STORAGE_LOG(INFO, "================test_tree_insert_fail_after_tree_build begin================"); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(2); + meta_tree_.set_max_page_item_cnt(3); + STORAGE_LOG(INFO, "=======================first insert======================="); + int64_t item_num = 361; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + //insert 750 items (insert a array) + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(6, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================second insert======================="); + item_num = 361; + ObArray data_items_1; + generate_data_items(item_num, 361 * 128/*start virtual page id*/, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_ALLOCATE_TMP_FILE_PAGE_FAILED, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(121, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(41, meta_tree_.level_page_range_array_.at(1).cached_page_num_); + + STORAGE_LOG(INFO, "=======================third insert======================="); + item_num = 2; + ObArray data_items_2; + generate_data_items(item_num, 361 * 128/*start virtual page id*/ - 10, data_items_2); + ASSERT_EQ(item_num, data_items_2.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_NE(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + data_items_2.reset(); + generate_data_items(item_num, 361 * 128/*start virtual page id*/, data_items_2); + ASSERT_EQ(item_num, data_items_2.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_2)); + ASSERT_EQ(121, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + ASSERT_EQ(41, meta_tree_.level_page_range_array_.at(1).cached_page_num_); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "================test_tree_insert_fail_after_tree_build begin================"); +} + +TEST_F(TestSNTmpFileMetaTree, test_array_read) +{ + STORAGE_LOG(INFO, "=======================test_array_read begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(100); + meta_tree_.set_max_page_item_cnt(100); + STORAGE_LOG(INFO, "=======================first array insert======================="); + int64_t item_num = 50; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(50, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second array insert======================="); + item_num = 50; + ObArray data_items_1; + generate_data_items(item_num, 50 * 128 - 1/*start virtual page id*/, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_NE(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + data_items_1.reset(); + generate_data_items(item_num, 50 * 128/*start virtual page id*/, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(100, meta_tree_.data_item_array_.count()); + + //After simulating the insert, we can see that the tmp file offset is [0, 100 * 128 * 8K], + // with each data item occupying 128 * 8K. + STORAGE_LOG(INFO, "=======================first array read======================="); + ObArray get_data_items; + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(10, 2 * 128 * ObTmpFileGlobal::PAGE_SIZE - 5, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(3, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================second array read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(7 * 128 * ObTmpFileGlobal::PAGE_SIZE, 5 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + ASSERT_EQ(5, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================third array read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(0, 100 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + ASSERT_EQ(100, get_data_items.count()); + + STORAGE_LOG(INFO, "=======================forth array read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(91 * 128 * ObTmpFileGlobal::PAGE_SIZE + 2, 3 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(4, get_data_items.count()); + ASSERT_EQ(91 * 128, get_data_items.at(0).virtual_page_id_); + + STORAGE_LOG(INFO, "=======================fifth array read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(1, 0.5 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(1, get_data_items.count()); + ASSERT_EQ(0, get_data_items.at(0).virtual_page_id_); + + STORAGE_LOG(INFO, "=======================fifth array read======================="); + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(1, 0.5 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + ASSERT_EQ(1, get_data_items.count()); + ASSERT_EQ(0, get_data_items.at(0).virtual_page_id_); + + STORAGE_LOG(INFO, "=======================fifth array read======================="); + get_data_items.reset(); + ASSERT_NE(OB_SUCCESS, meta_tree_.search_data_items(1, 100 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_array_read end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_read_fail) +{ + STORAGE_LOG(INFO, "=======================test_read_fail begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(3); + meta_tree_.set_max_page_item_cnt(5); + STORAGE_LOG(INFO, "=======================tree insert======================="); + int64_t item_num = 3; + ObArray data_items; + generate_wrong_data_items(item_num, 0, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(3, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================first read======================="); + ObArray get_data_items; + ASSERT_NE(OB_SUCCESS, meta_tree_.search_data_items(1 * 128 * ObTmpFileGlobal::PAGE_SIZE, 2 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + + STORAGE_LOG(INFO, "=======================second insert======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, 3 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + item_num = 5; + ObArray data_items_1; + generate_wrong_data_items(item_num, 4 * 128, data_items_1); + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + ASSERT_EQ(1, meta_tree_.level_page_range_array_.count()); + + STORAGE_LOG(INFO, "=======================second read======================="); + get_data_items.reset(); + ASSERT_NE(OB_SUCCESS, meta_tree_.search_data_items(4 * 128 * ObTmpFileGlobal::PAGE_SIZE, 2 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_read_fail end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_array_read_after_truncate) +{ + STORAGE_LOG(INFO, "=======================test_array_read_after_truncate begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(100); + meta_tree_.set_max_page_item_cnt(100); + STORAGE_LOG(INFO, "=======================array insert======================="); + int64_t item_num = 100; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(100, meta_tree_.data_item_array_.count()); + //After simulating the insert, we can see that the tmp file offset is [0, 100 * 128 * 8K], + // with each data item occupying 128 * 8K. + STORAGE_LOG(INFO, "=======================array truncate======================="); + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(0, 2 * 128 * ObTmpFileGlobal::PAGE_SIZE - 5)); + ASSERT_EQ(99, meta_tree_.data_item_array_.count()); + + ASSERT_EQ(OB_SUCCESS, meta_tree_.truncate(2 * 128 * ObTmpFileGlobal::PAGE_SIZE - 5, 20 * 128 * ObTmpFileGlobal::PAGE_SIZE)); + ASSERT_EQ(80, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================array read======================="); + ObArray get_data_items; + get_data_items.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items(20 * 128 * ObTmpFileGlobal::PAGE_SIZE, 5 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + ASSERT_EQ(5, get_data_items.count()); + STORAGE_LOG(INFO, "data_items", K(get_data_items)); + + get_data_items.reset(); + ASSERT_NE(OB_SUCCESS, meta_tree_.search_data_items(18 * 128 * ObTmpFileGlobal::PAGE_SIZE, 5 * 128 * ObTmpFileGlobal::PAGE_SIZE, get_data_items)); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_array_read_after_truncate end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_write_tail) +{ + STORAGE_LOG(INFO, "=======================test_write_tail begin======================="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + STORAGE_LOG(INFO, "=======================first insert======================="); + int64_t item_num = 4; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(0, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(4, meta_tree_.data_item_array_.count()); + //After simulating the insert, we can see that the tmp file offset is [0, 100 * 128 * 8K], + // with each data item occupying 128 * 8K. + // we assume that the last page of the last data item is an unfilled page. + STORAGE_LOG(INFO, "=======================first write tail======================="); + ObSharedNothingTmpFileDataItem last_data_item; + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + //write tail fail + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, false)); + ASSERT_EQ(128, meta_tree_.data_item_array_.at(meta_tree_.data_item_array_.count() - 1).physical_page_num_); + last_data_item.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true)); + ASSERT_EQ(127, meta_tree_.data_item_array_.at(meta_tree_.data_item_array_.count() - 1).physical_page_num_); + ASSERT_EQ(4, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second insert======================="); + data_items.reset(); + last_data_item.reset(); + last_data_item.block_index_ = 30; + last_data_item.physical_page_id_ = 0; + last_data_item.physical_page_num_ = 1; + last_data_item.virtual_page_id_ = 3 * 128 + 127; + ASSERT_EQ(OB_SUCCESS, data_items.push_back(last_data_item)); //unfilled page + ASSERT_EQ(1, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(5, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================second write tail======================="); + last_data_item.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true)); + ASSERT_EQ(4, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================third insert======================="); + item_num = 2; + ObArray data_items_1; + generate_data_items(item_num, 3 * 128 + 127/*start virtual page id*/, data_items_1); + //we assume that the last page of the last data item is an unfilled page. + ASSERT_EQ(item_num, data_items_1.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items_1)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(0, meta_tree_.data_item_array_.count()); + + STORAGE_LOG(INFO, "=======================third write tail======================="); + last_data_item.reset(); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true)); + + STORAGE_LOG(INFO, "=======================first read======================="); + ObArray data_items_2; + ASSERT_EQ(OB_SUCCESS, meta_tree_.search_data_items((3 * 128 + 127) * ObTmpFileGlobal::PAGE_SIZE, 240 * ObTmpFileGlobal::PAGE_SIZE, data_items_2)); + ASSERT_EQ(2, data_items_2.count()); + ASSERT_EQ(127, data_items_2.at(1).physical_page_num_); + + meta_tree_.reset(); + STORAGE_LOG(INFO, "=======================test_write_tail end======================="); +} + +TEST_F(TestSNTmpFileMetaTree, test_page_is_dirty_again_during_flush) +{ + STORAGE_LOG(INFO, "=================test_page_is_dirty_again_during_flush begin==============="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + int64_t item_num = 10; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + + STORAGE_LOG(INFO, "=======================first tree flush========================="); + ObTmpFileTreeFlushContext flush_context_first; + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - 2 * ObTmpFileGlobal::PAGE_SIZE; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + STORAGE_LOG(INFO, "tree_io_array_1", K(tree_io_array_1)); + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(2, tree_io_array_1.at(0).flush_nums_); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + ASSERT_EQ(false, flush_context_first.is_meta_reach_end_); + + STORAGE_LOG(INFO, "=======================first write tail======================="); + ObSharedNothingTmpFileDataItem last_data_item; + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_write_tail(last_data_item)); + ASSERT_EQ(OB_SUCCESS, meta_tree_.finish_write_tail(last_data_item, true)); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + item_num = 1; + data_items.reset(); + generate_data_items(item_num, 9 * 128 + 127/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + + STORAGE_LOG(INFO, "=======================second tree flush========================="); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_2; + int64_t write_offset_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_first, + tree_io_array_2)); + ASSERT_EQ(0, tree_io_array_2.count()); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + STORAGE_LOG(INFO, "=======================third tree flush========================="); + //flush successfully + ASSERT_EQ(OB_SUCCESS, meta_tree_.update_after_flush(tree_io_array_1)); + ASSERT_EQ(tree_io_array_1.at(0).flush_end_page_id_, meta_tree_.level_page_range_array_.at(0).flushed_end_page_id_); + + ObTmpFileTreeFlushContext flush_context_second; + char *block_buff_3 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_3; + int64_t write_offset_3 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(2/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_3, + write_offset_3, + flush_context_second, + tree_io_array_3)); + ASSERT_EQ(2, tree_io_array_3.count()); + ASSERT_EQ(2, tree_io_array_3.at(0).flush_nums_); + ASSERT_EQ(1, tree_io_array_3.at(1).flush_nums_); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + delete[] block_buff_1; + delete[] block_buff_2; + delete[] block_buff_3; + meta_tree_.reset(); + STORAGE_LOG(INFO, "=================test_page_is_dirty_again_during_flush end==============="); +} + +TEST_F(TestSNTmpFileMetaTree, test_insert_items_during_flush) +{ + STORAGE_LOG(INFO, "=================test_insert_items_during_flush begin==============="); + ObTmpWriteBufferPool wbp; + wbp.default_wbp_memory_limit_ = ObTmpWriteBufferPool::WBP_BLOCK_SIZE; //253 pages + ASSERT_EQ(OB_SUCCESS, wbp.init()); + common::ObFIFOAllocator callback_allocator; + ASSERT_EQ(OB_SUCCESS, callback_allocator.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID))); + ObTmpFileTestMetaTree meta_tree_; + ASSERT_EQ(OB_SUCCESS, meta_tree_.init(1, &wbp, &callback_allocator)); + meta_tree_.set_max_array_item_cnt(5); + meta_tree_.set_max_page_item_cnt(5); + STORAGE_LOG(INFO, "=======================first tree insert======================="); + int64_t item_num = 10; + ObArray data_items; + generate_data_items(item_num, 0/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + + STORAGE_LOG(INFO, "=======================first tree flush========================="); + ObTmpFileTreeFlushContext flush_context_first; + char *block_buff_1 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_1; + int64_t write_offset_1 = OB_DEFAULT_MACRO_BLOCK_SIZE - 2 * ObTmpFileGlobal::PAGE_SIZE; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(0/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_1, + write_offset_1, + flush_context_first, + tree_io_array_1)); + ASSERT_EQ(1, tree_io_array_1.count()); + ASSERT_EQ(2, tree_io_array_1.at(0).flush_nums_); + ASSERT_EQ(OB_DEFAULT_MACRO_BLOCK_SIZE, write_offset_1); + ASSERT_EQ(false, flush_context_first.is_meta_reach_end_); + + STORAGE_LOG(INFO, "=======================second tree insert======================="); + item_num = 1; + data_items.reset(); + generate_data_items(item_num, 10 * 128/*start virtual page id*/, data_items); + ASSERT_EQ(item_num, data_items.count()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.prepare_for_insert_items()); + ASSERT_EQ(OB_SUCCESS, meta_tree_.insert_items(data_items)); + ASSERT_EQ(2, meta_tree_.level_page_range_array_.count()); + ASSERT_EQ(3, meta_tree_.level_page_range_array_.at(0).cached_page_num_); + + STORAGE_LOG(INFO, "=======================second tree flush========================="); + char *block_buff_2 = new char[OB_DEFAULT_MACRO_BLOCK_SIZE]; + ObArray tree_io_array_2; + int64_t write_offset_2 = 0; + ASSERT_EQ(OB_SUCCESS, meta_tree_.flush_meta_pages_for_block(1/*block_index*/, + ObTmpFileTreeEvictType::FULL, + block_buff_2, + write_offset_2, + flush_context_first, + tree_io_array_2)); + ASSERT_EQ(2, tree_io_array_2.count()); + ASSERT_EQ(1, tree_io_array_2.at(0).flush_nums_); + ASSERT_EQ(true, flush_context_first.is_meta_reach_end_); + + delete[] block_buff_1; + delete[] block_buff_2; + meta_tree_.reset(); + STORAGE_LOG(INFO, "=================test_insert_items_during_flush end==============="); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_sn_tmp_file_meta_tree.log*"); + OB_LOGGER.set_file_name("test_sn_tmp_file_meta_tree.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/mittest/mtlenv/storage/tmp_file/test_tmp_file_write_buffer_pool_index_cache.cpp b/mittest/mtlenv/storage/tmp_file/test_tmp_file_write_buffer_pool_index_cache.cpp new file mode 100644 index 0000000000..63e19a8e32 --- /dev/null +++ b/mittest/mtlenv/storage/tmp_file/test_tmp_file_write_buffer_pool_index_cache.cpp @@ -0,0 +1,843 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include +#include +#define USING_LOG_PREFIX STORAGE +#define protected public +#define private public +#include "mittest/mtlenv/mock_tenant_module_env.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" + +namespace oceanbase +{ +using namespace tmp_file; + +static const int64_t INIT_BUCKET_ARRAY_CAPACITY = ObTmpFileWBPIndexCache::INIT_BUCKET_ARRAY_CAPACITY; +static const int64_t MAX_BUCKET_ARRAY_CAPACITY = ObTmpFileWBPIndexCache::MAX_BUCKET_ARRAY_CAPACITY; +static const int64_t BUCKET_CAPACITY = ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::BUCKET_CAPACITY; +static const int64_t fd = 1; +static uint64_t tenant_id = OB_SYS_TENANT_ID; +static const int64_t TENANT_MEMORY = 8L * 1024L * 1024L * 1024L /* 8 GB */; +static const double TMP_FILE_WBP_MEM_LIMIT_PROP = 50; //[0, 100] +/* ----------------------------- Mock Class -------------------------------- */ +class MockWBPIndexCache +{ +public: + MockWBPIndexCache(ObTmpWriteBufferPool &wbp) : wbp_(wbp) {} + int push(uint32_t page_index); + int truncate(const int64_t truncate_page_virtual_id); + int compare(const ObTmpFileWBPIndexCache &index_cache); +private: + int sparsify_(const int64_t sparsify_modulus); +public: + ObArray mock_index_cache_; + ObTmpWriteBufferPool &wbp_; +}; + +int MockWBPIndexCache::push(uint32_t page_index) +{ + int ret = OB_SUCCESS; + if (mock_index_cache_.count() >= MAX_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY) { + if (OB_FAIL(sparsify_(2))) { + LOG_WARN("failed to sparsify", KR(ret)); + } + } + if (FAILEDx(mock_index_cache_.push_back(page_index))) { + LOG_WARN("failed to push back page index", KR(ret), K(page_index)); + } + return ret; +} + +int MockWBPIndexCache::truncate(const int64_t truncate_page_virtual_id) +{ + int ret = OB_SUCCESS; + ObArray new_mock_index_cache; + if (truncate_page_virtual_id < 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(truncate_page_virtual_id)); + } else { + int64_t i = 0; + for (; OB_SUCC(ret) && i < mock_index_cache_.count(); ++i) { + int64_t virtual_page_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + if (OB_FAIL(wbp_.get_page_virtual_id(fd, mock_index_cache_.at(i), virtual_page_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(mock_index_cache_.at(i))); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == virtual_page_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid virtual page id", KR(ret), K(virtual_page_id)); + } else if (virtual_page_id >= truncate_page_virtual_id) { + break; + } + } + for (int64_t j = i; j < mock_index_cache_.count(); ++j) { + if (OB_FAIL(new_mock_index_cache.push_back(mock_index_cache_.at(j)))) { + LOG_WARN("fail to push back page index", KR(ret), K(mock_index_cache_.at(j))); + } + } + mock_index_cache_ = new_mock_index_cache; + } + return ret; +} + +int MockWBPIndexCache::sparsify_(const int64_t sparsify_modulus) +{ + int ret = OB_SUCCESS; + ObArray new_mock_index_cache; + if (OB_UNLIKELY(sparsify_modulus < 2 || + mock_index_cache_.count() != MAX_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", KR(ret), K(sparsify_modulus), K(mock_index_cache_.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < mock_index_cache_.count(); ++i) { + if (i % sparsify_modulus != 0 && + OB_FAIL(new_mock_index_cache.push_back(mock_index_cache_.at(i)))) { + LOG_WARN("fail to push back page index", KR(ret), K(mock_index_cache_.at(i))); + } + } + mock_index_cache_ = new_mock_index_cache; + } + return ret; +} + +int MockWBPIndexCache::compare(const ObTmpFileWBPIndexCache &index_cache) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(index_cache.is_empty() && mock_index_cache_.empty())) { + // do nothing + } else { + if (OB_UNLIKELY(index_cache.is_empty() || mock_index_cache_.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("one cache is empty", KR(ret), K(index_cache.is_empty()), K(mock_index_cache_.empty())); + } else if (OB_ISNULL(index_cache.page_buckets_) || OB_ISNULL(index_cache.page_buckets_->at(index_cache.left_)) || + OB_ISNULL(index_cache.page_buckets_->at(index_cache.right_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), KP(index_cache.page_buckets_), K(index_cache.left_), K(index_cache.right_)); + } else if (OB_UNLIKELY(index_cache.page_buckets_->at(index_cache.left_)->is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected empty bucket", KR(ret), K(index_cache.left_)); + } else if (OB_UNLIKELY(index_cache.page_buckets_->at(index_cache.right_)->is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected empty bucket", KR(ret), K(index_cache.right_)); + } else if (index_cache.size() == 1) { + if (mock_index_cache_.count() != index_cache.page_buckets_->at(index_cache.left_)->size()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected size", KR(ret), K(index_cache.size()), K(mock_index_cache_.count())); + } + } else { + int64_t real_cache_size = index_cache.page_buckets_->at(index_cache.left_)->size() + + index_cache.page_buckets_->at(index_cache.right_)->size() + + (index_cache.get_logic_tail_() - index_cache.left_ - 1) * + BUCKET_CAPACITY; + if (mock_index_cache_.count() != real_cache_size) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected size", KR(ret), K(real_cache_size), K(index_cache.page_buckets_->at(index_cache.left_)->size()), + K(index_cache.page_buckets_->at(index_cache.right_)->size()), K(index_cache.size()), K(mock_index_cache_.count())); + } + } + + if (OB_SUCC(ret)) { + const int64_t logic_end_bkt_pos = index_cache.get_logic_tail_(); + int64_t cur_logic_bkt_pos = index_cache.left_; + int64_t cur_logic_index_pos = index_cache.page_buckets_->at(cur_logic_bkt_pos)->left_; + int64_t i = 0; + for (;OB_SUCC(ret) && i < mock_index_cache_.count() && cur_logic_bkt_pos <= logic_end_bkt_pos; ++i) { + int64_t real_bkt_pos = cur_logic_bkt_pos % index_cache.capacity_; + int64_t real_index_pos = cur_logic_index_pos % BUCKET_CAPACITY; + const ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket *bkt = index_cache.page_buckets_->at(real_bkt_pos); + uint32_t page_index_in_real_cache = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t page_index_in_mock_cache = mock_index_cache_.at(i); + + if (OB_ISNULL(bkt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), KP(bkt)); + } else if (FALSE_IT(page_index_in_real_cache = bkt->page_indexes_.at(real_index_pos))) { + } else if (OB_UNLIKELY(page_index_in_mock_cache != page_index_in_real_cache)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page index is not same", KR(ret), K(page_index_in_mock_cache), K(page_index_in_real_cache), + K(i), K(cur_logic_bkt_pos), K(cur_logic_index_pos), K(index_cache.left_), K(logic_end_bkt_pos)); + } else if (OB_UNLIKELY(cur_logic_index_pos > bkt->get_logic_tail_())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected index", KR(ret), K(cur_logic_index_pos), K(bkt->get_logic_tail_())); + } else if (cur_logic_index_pos < bkt->get_logic_tail_()) { + cur_logic_index_pos += 1; + } else { // cur_logic_index_pos == bkt->get_logic_tail_() + cur_logic_bkt_pos += 1; + if (cur_logic_bkt_pos <= logic_end_bkt_pos) { + const ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket *next_bkt = index_cache.page_buckets_->at(cur_logic_bkt_pos % index_cache.capacity_); + if (OB_ISNULL(next_bkt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(mock_index_cache_.count()), + K(index_cache.page_buckets_->at(index_cache.left_)->size()), + K(index_cache.page_buckets_->at(index_cache.right_)->size()), + K(index_cache.size()), K(index_cache.capacity_), + K(i), K(cur_logic_bkt_pos), + K(cur_logic_index_pos), K(index_cache.left_), + K(logic_end_bkt_pos), KP(next_bkt)); + } else if (OB_UNLIKELY(next_bkt->is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected empty", KR(ret), K(mock_index_cache_.count()), + K(index_cache.page_buckets_->at(index_cache.left_)->size()), + K(index_cache.page_buckets_->at(index_cache.right_)->size()), + K(index_cache.size()), K(index_cache.capacity_), + K(i), K(cur_logic_bkt_pos), K(cur_logic_index_pos), + K(index_cache.left_), K(logic_end_bkt_pos), + KPC(next_bkt)); + } else { + cur_logic_index_pos = next_bkt->left_; + } + } + } + } // end for + } + } + + return ret; +} + +/* ---------------------------- Unittest Class ----------------------------- */ +class TestTmpFileWBPIndexCache : public ::testing::Test +{ +public: + TestTmpFileWBPIndexCache() = default; + virtual ~TestTmpFileWBPIndexCache() = default; + virtual void SetUp(); + virtual void TearDown(); + static void SetUpTestCase(); + static void TearDownTestCase(); + int write_and_push_pages(const uint32_t end_page_id,const int64_t page_num, + MockWBPIndexCache &mock_cache, uint32_t &new_end_page_id); + int truncate_pages(const int64_t truncate_page_num, MockWBPIndexCache &mock_cache); + int truncate_and_free_pages(const uint32_t wbp_begin_page_id, const int64_t truncate_page_num_in_cache, + MockWBPIndexCache &mock_cache, uint32_t &new_begin_page_id); +private: + int write_pages_(const int64_t page_num, const int64_t begin_page_virtual_id, ObArray &page_indexes); + int free_pages_(const uint32_t begin_page_id, const int64_t end_page_virtual_id, uint32_t &new_begin_page_id); + int check_cache_status_after_truncate_(MockWBPIndexCache &mock_cache); +public: + ObTmpWriteBufferPool wbp_; + ObTmpFileWBPIndexCache wbp_index_cache_; + common::ObFIFOAllocator wbp_index_cache_allocator_; + common::ObFIFOAllocator wbp_index_cache_bkt_allocator_; +}; + +void TestTmpFileWBPIndexCache::SetUp() +{ + ASSERT_EQ(OB_SUCCESS, wbp_index_cache_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_NORMAL_BLOCK_SIZE, + ObMemAttr(tenant_id, "TmpFileIndCache", + ObCtxIds::DEFAULT_CTX_ID))); + ASSERT_EQ(OB_SUCCESS, wbp_index_cache_bkt_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(tenant_id, "TmpFileIndCBkt", + ObCtxIds::DEFAULT_CTX_ID))); + ASSERT_EQ(OB_SUCCESS, wbp_.init()); + ASSERT_EQ(OB_SUCCESS, wbp_index_cache_.init(fd, &wbp_, &wbp_index_cache_allocator_, &wbp_index_cache_bkt_allocator_)); +} + +void TestTmpFileWBPIndexCache::TearDown() +{ + wbp_index_cache_allocator_.reset(); + wbp_index_cache_bkt_allocator_.reset(); + wbp_.destroy(); + wbp_index_cache_.destroy(); +} + +void TestTmpFileWBPIndexCache::SetUpTestCase() +{ + int ret = OB_SUCCESS; + ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init()); + + CHUNK_MGR.set_limit(TENANT_MEMORY); + ObMallocAllocator::get_instance()->set_tenant_limit(1, TENANT_MEMORY); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + ASSERT_EQ(true, tenant_config.is_valid()); + tenant_config->_temporary_file_io_area_size = TMP_FILE_WBP_MEM_LIMIT_PROP; +} + +void TestTmpFileWBPIndexCache::TearDownTestCase() +{ + MockTenantModuleEnv::get_instance().destroy(); +} + +// end_page_id: the page_id of the last page of wbp +// page_num: the number of pages to be alloced. +// new_end_page_id: page_id of the last page of alloced pages. +// this function will alloc 'page_num' pages and link them after the page of end_page_id, +// then put them into mock_cache and wbp_index_cache_. +int TestTmpFileWBPIndexCache::write_and_push_pages(const uint32_t end_page_id, const int64_t page_num, + MockWBPIndexCache &mock_cache, uint32_t &new_end_page_id) +{ + int ret = OB_SUCCESS; + ObArray page_indexes; + int64_t end_page_virtual_id = 0; + int64_t new_begin_page_virtual_id = 0; + if (OB_UNLIKELY(page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(page_num)); + } else if (end_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + if (OB_FAIL(wbp_.get_page_virtual_id(fd, end_page_id, end_page_virtual_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(end_page_id), K(end_page_virtual_id)); + } else if (OB_UNLIKELY(end_page_virtual_id < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid virtual id", KR(ret), K(end_page_virtual_id)); + } else { + new_begin_page_virtual_id = end_page_virtual_id + 1; + } + } + + if (FAILEDx(write_pages_(page_num, new_begin_page_virtual_id, page_indexes))) { + LOG_WARN("fail to write pages", KR(ret), K(page_num)); + } else if (OB_UNLIKELY(page_indexes.count() != page_num)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error", KR(ret), K(page_num), K(page_indexes.count())); + } else if (end_page_id != ObTmpFileGlobal::INVALID_PAGE_ID && + OB_FAIL(wbp_.link_page(fd, page_indexes.at(0), end_page_id, ObTmpFilePageUniqKey(end_page_virtual_id)))) { + LOG_WARN("fail to link page", KR(ret), K(page_indexes.at(0)), K(end_page_id), K(end_page_virtual_id)); + } else if (FALSE_IT(new_end_page_id = page_indexes.at(page_num - 1))) { + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < page_num; ++i) { + if (OB_FAIL(mock_cache.push(page_indexes.at(i)))) { + LOG_WARN("fail to push", KR(ret), K(i), K(page_indexes.at(i))); + } else if (OB_FAIL(wbp_index_cache_.push(page_indexes.at(i)))) { + LOG_WARN("fail to push", KR(ret), K(i), K(page_indexes.at(i))); + } + } + } + return ret; +} + +int TestTmpFileWBPIndexCache::truncate_pages(const int64_t truncate_page_num, MockWBPIndexCache &mock_cache) +{ + int ret = OB_SUCCESS; + ObArray page_indexes; + if (OB_UNLIKELY(truncate_page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(truncate_page_num)); + } else { + int64_t truncate_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + uint32_t truncate_page_id = mock_cache.mock_index_cache_.at(truncate_page_num - 1); + if (OB_FAIL(wbp_.get_page_virtual_id(fd, truncate_page_id, truncate_page_virtual_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(truncate_page_id)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == truncate_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid virtual page id", KR(ret), K(truncate_page_virtual_id)); + } else if (FALSE_IT(truncate_page_virtual_id += 1)) { + } else if (OB_FAIL(mock_cache.truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate", KR(ret), K(truncate_page_virtual_id)); + } else if (OB_FAIL(wbp_index_cache_.truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate", KR(ret), K(truncate_page_virtual_id)); + } else if (OB_FAIL(check_cache_status_after_truncate_(mock_cache))) { + LOG_WARN("fail to check cache status after truncate", KR(ret)); + } + } + return ret; +} + +// wbp_begin_page_id: the page_id of the first page of wbp, the pos of it in the wbp list should be +// less than or equal to the first cached index in wbp_index_cache_. +// truncate_page_num_in_cache: the number of page indexes to be truncated in mock_cache, +// we will free pages between [wbp_begin_page_id, begin_page_id_in_cache + truncate_page_num_in_cache] +// new_begin_page_id: page_id of the first page after free pages in wbp. +int TestTmpFileWBPIndexCache::truncate_and_free_pages(const uint32_t wbp_begin_page_id, const int64_t truncate_page_num_in_cache, + MockWBPIndexCache &mock_cache, uint32_t &new_begin_page_id) +{ + int ret = OB_SUCCESS; + ObArray page_indexes; + if (OB_UNLIKELY(truncate_page_num_in_cache <= 0 || ObTmpFileGlobal::INVALID_PAGE_ID == wbp_begin_page_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(truncate_page_num_in_cache), K(wbp_begin_page_id)); + } else { + int64_t truncate_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + uint32_t truncate_page_id = mock_cache.mock_index_cache_.at(truncate_page_num_in_cache - 1); + if (OB_FAIL(wbp_.get_page_virtual_id(fd, truncate_page_id, truncate_page_virtual_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(truncate_page_id)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == truncate_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid virtual page id", KR(ret), K(truncate_page_virtual_id)); + } else if (FALSE_IT(truncate_page_virtual_id += 1)) { + } else if (OB_FAIL(mock_cache.truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate", KR(ret), K(truncate_page_virtual_id)); + } else if (OB_FAIL(wbp_index_cache_.truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate", KR(ret), K(truncate_page_virtual_id)); + } else if (OB_FAIL(check_cache_status_after_truncate_(mock_cache))) { + LOG_WARN("fail to check cache status after truncate", KR(ret)); + } else if (OB_FAIL(free_pages_(wbp_begin_page_id, truncate_page_virtual_id, new_begin_page_id))) { + LOG_WARN("fail to free pages", KR(ret), K(wbp_begin_page_id), K(truncate_page_virtual_id)); + } + } + return ret; +} + +int TestTmpFileWBPIndexCache::write_pages_(const int64_t page_num, const int64_t begin_page_virtual_id, ObArray &page_indexes) +{ + int ret = OB_SUCCESS; + page_indexes.reset(); + if (OB_UNLIKELY(page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(page_num)); + } else{ + for (int64_t i = 0; OB_SUCC(ret) && i < page_num; ++i) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *unused_buf = nullptr; + int64_t new_page_begin_page_id = begin_page_virtual_id + i; + if (OB_FAIL(wbp_.alloc_page(fd, ObTmpFilePageUniqKey(new_page_begin_page_id), new_page_id, unused_buf))) { + LOG_WARN("fail to alloc data page", KR(ret), K(i)); + } else if (OB_FAIL(wbp_.notify_dirty(fd, new_page_id, ObTmpFilePageUniqKey(new_page_begin_page_id)))) { + LOG_WARN("fail to notify dirty", KR(ret), K(new_page_id), K(new_page_begin_page_id)); + } else if (!page_indexes.empty() && + OB_FAIL(wbp_.link_page(fd, new_page_id, page_indexes.at(page_indexes.count()-1), + ObTmpFilePageUniqKey(new_page_begin_page_id - 1)))) { + LOG_WARN("fail to link page", KR(ret), K(i), K(new_page_id), + K(page_indexes.at(page_indexes.count()-1)), K(new_page_begin_page_id)); + } else if (OB_FAIL(page_indexes.push_back(new_page_id))) { + LOG_WARN("fail to push back", KR(ret), K(i)); + } + } + } + return ret; +} + + +int TestTmpFileWBPIndexCache::free_pages_(const uint32_t begin_page_id, const int64_t end_page_virtual_id, uint32_t &new_begin_page_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(end_page_virtual_id <= 0 || begin_page_id == ObTmpFileGlobal::INVALID_PAGE_ID)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(end_page_virtual_id), K(begin_page_id)); + } else { + bool free_over = false; + uint32_t cur_page_id = begin_page_id; + int64_t cnt = 0; + while(!free_over && OB_SUCC(ret) && cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + int64_t virtual_page_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_.get_page_virtual_id(fd, cur_page_id, virtual_page_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(cur_page_id), K(cnt)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == virtual_page_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid virtual page id", KR(ret), K(virtual_page_id), K(cnt)); + } else if (virtual_page_id >= end_page_virtual_id) { + free_over = true; + new_begin_page_id = cur_page_id; + } else if (OB_FAIL(wbp_.free_page(fd, cur_page_id, ObTmpFilePageUniqKey(virtual_page_id), next_page_id))) { + LOG_WARN("fail to alloc data page", KR(ret), K(cur_page_id), K(cnt)); + } else { + cur_page_id = next_page_id; + cnt += 1; + } + } + } + return ret; +} + +int TestTmpFileWBPIndexCache::check_cache_status_after_truncate_(MockWBPIndexCache &mock_cache) +{ + int ret = OB_SUCCESS; + int64_t bkt_num = wbp_index_cache_.get_logic_tail_() - wbp_index_cache_.left_ + 1; + int64_t page_num = 0; + if (OB_UNLIKELY(bkt_num != wbp_index_cache_.size())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket num is invalid", KR(ret), K(bkt_num), K(wbp_index_cache_.size())); + } + for (int64_t bkt_arr_logic_idx = wbp_index_cache_.left_; + OB_SUCC(ret) && bkt_arr_logic_idx <= wbp_index_cache_.get_logic_tail_(); + bkt_arr_logic_idx++) { + int64_t bkt_arr_idx = bkt_arr_logic_idx % wbp_index_cache_.capacity_; + int64_t bkt_left_idx = 0; + if (OB_ISNULL(wbp_index_cache_.page_buckets_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket array is null", KR(ret)); + } else if (OB_ISNULL(wbp_index_cache_.page_buckets_->at(bkt_arr_idx))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket is null", KR(ret), K(bkt_arr_idx)); + } else if (OB_UNLIKELY(wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket is empty", KR(ret), K(bkt_arr_idx)); + } else if (FALSE_IT(bkt_left_idx = wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->left_)) { + } else if (OB_UNLIKELY(wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->min_page_index_ != + wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->page_indexes_.at(bkt_left_idx))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("min page index is not equal to left page index", KR(ret), + K(wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->min_page_index_), + K(wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->page_indexes_.at(bkt_left_idx))); + } else { + for (int64_t bkt_logic_idx = wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->left_; + OB_SUCC(ret) && bkt_logic_idx <= wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->get_logic_tail_(); + bkt_logic_idx++) { + int64_t bkt_idx = bkt_logic_idx % wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->capacity_; + uint32_t page_idx = wbp_index_cache_.page_buckets_->at(bkt_arr_idx)->page_indexes_.at(bkt_idx); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == page_idx)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page index is invalid", KR(ret), K(bkt_arr_idx), K(bkt_idx)); + } else { + page_num += 1; + } + } // end for + } + } // end for + if (OB_UNLIKELY(page_num != mock_cache.mock_index_cache_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page num is invalid", KR(ret), K(page_num), K(mock_cache.mock_index_cache_.count())); + } + return ret; +} + +TEST_F(TestTmpFileWBPIndexCache, test_push_and_pop) +{ + int ret = OB_SUCCESS; + uint32_t begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + // 1. push indexes + int64_t page_num = BUCKET_CAPACITY / 2; + MockWBPIndexCache mock_cache(wbp_); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + begin_page_id = mock_cache.mock_index_cache_.at(0); + + // 2. fill a bucket of wbp_index_cache + page_num = BUCKET_CAPACITY - page_num; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(1, wbp_index_cache_.size_); + ASSERT_EQ(true, wbp_index_cache_.page_buckets_->at(wbp_index_cache_.right_)->is_full()); + + // 3. continue to push some index when bucket is full + page_num = BUCKET_CAPACITY / 4; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(2, wbp_index_cache_.size_); + ASSERT_EQ(false, wbp_index_cache_.page_buckets_->at(wbp_index_cache_.right_)->is_full()); + + // 4. push more than a bucket number of indexes + page_num = BUCKET_CAPACITY * 2 + BUCKET_CAPACITY / 4; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(4, wbp_index_cache_.size_); + ASSERT_EQ(false, wbp_index_cache_.page_buckets_->at(wbp_index_cache_.right_)->is_full()); + + // 5. free indexes of wbp_index_cache and mock_cache + page_num = BUCKET_CAPACITY / 4; + ASSERT_LE(page_num, mock_cache.mock_index_cache_.count()); + uint32_t remove_page_id = mock_cache.mock_index_cache_.at(page_num - 1); + ret = truncate_and_free_pages(begin_page_id, page_num, mock_cache, begin_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + // 6. push some indexes after freeing + page_num = 2 * BUCKET_CAPACITY / 4; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + // 7. free more than a bucket number of indexes + page_num = BUCKET_CAPACITY * 2 + BUCKET_CAPACITY / 2; + ASSERT_LE(page_num, mock_cache.mock_index_cache_.count()); + remove_page_id = mock_cache.mock_index_cache_.at(page_num - 1); + ret = truncate_and_free_pages(begin_page_id, page_num, mock_cache, begin_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + // 8. free all indexes of wbp_index_cache and mock_cache + page_num = mock_cache.mock_index_cache_.count(); + remove_page_id = mock_cache.mock_index_cache_.at(page_num - 1); + ret = truncate_and_free_pages(begin_page_id, page_num, mock_cache, begin_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + // 9. push indexes into both wbp_index_cache and mock_cache + page_num = BUCKET_CAPACITY / 3; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + LOG_INFO("test_push_and_pop"); +} + +TEST_F(TestTmpFileWBPIndexCache, test_expand_and_sparsify) +{ + int ret = OB_SUCCESS; + uint32_t begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + // 1. push indexes into both wbp_index_cache and mock_cache + int64_t page_num = INIT_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY; + MockWBPIndexCache mock_cache(wbp_); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, wbp_index_cache_.is_full()); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + begin_page_id = mock_cache.mock_index_cache_.at(0); + + // 2. push indexes to trigger expand + page_num = 30; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, wbp_index_cache_.is_full()); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY * 2, wbp_index_cache_.capacity_); + + // 3. push indexes to trigger expand until reach the max capacity + page_num = MAX_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY + - mock_cache.mock_index_cache_.count(); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(true, wbp_index_cache_.is_full()); + ASSERT_EQ(MAX_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + + // 4. push indexes to trigger sparsify + page_num = BUCKET_CAPACITY / 2; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(false, wbp_index_cache_.is_full()); + ASSERT_EQ(wbp_index_cache_.capacity_ / 2 + 1, wbp_index_cache_.size_); + ASSERT_EQ(page_num, wbp_index_cache_.page_buckets_->at(wbp_index_cache_.right_)->size_); + ASSERT_EQ(MAX_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + + LOG_INFO("test_expand_and_sparsify"); +} + +TEST_F(TestTmpFileWBPIndexCache, test_shrink) +{ + int ret = OB_SUCCESS; + uint32_t begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + // 1. push indexes to trigger expand + int64_t page_num = 2 * INIT_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY; + MockWBPIndexCache mock_cache(wbp_); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(2 * INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.size_); + ASSERT_EQ(2 * INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + begin_page_id = mock_cache.mock_index_cache_.at(0); + + // 2. free 7/8 indexes to trigger shrink + page_num = wbp_index_cache_.capacity_ * BUCKET_CAPACITY * 7 / 8; + ASSERT_GE(page_num, mock_cache.mock_index_cache_.count() - mock_cache.mock_index_cache_.count() / ObTmpFileWBPIndexCache::SHRINK_THRESHOLD); + ASSERT_LE(page_num, mock_cache.mock_index_cache_.count()); + uint32_t remove_page_id = mock_cache.mock_index_cache_.at(page_num - 1); + ret = truncate_and_free_pages(begin_page_id, page_num, mock_cache, begin_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + + // 3. push indexes to fill buckets with INIT_BUCKET_ARRAY_CAPACITY + page_num = INIT_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY + - mock_cache.mock_index_cache_.count(); + ASSERT_LE(page_num, INIT_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY); + ASSERT_GT(page_num, 0); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.size_); + ASSERT_EQ(wbp_index_cache_.size_ * BUCKET_CAPACITY, + mock_cache.mock_index_cache_.count()); + + // 4. free 7/8 indexes, but doesn't trigger shrink + page_num = wbp_index_cache_.capacity_ * BUCKET_CAPACITY * 7 / 8; + ASSERT_GE(page_num, mock_cache.mock_index_cache_.count() - mock_cache.mock_index_cache_.count() / ObTmpFileWBPIndexCache::SHRINK_THRESHOLD); + ASSERT_LE(page_num, mock_cache.mock_index_cache_.count()); + remove_page_id = mock_cache.mock_index_cache_.at(page_num - 1); + ret = truncate_and_free_pages(begin_page_id, page_num, mock_cache, begin_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(INIT_BUCKET_ARRAY_CAPACITY, wbp_index_cache_.capacity_); + + LOG_INFO("test_shrink"); +} + +int mock_circle_bucket(ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket &bucket) +{ + int ret = OB_SUCCESS; + if (bucket.right_ < 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid bucket", KR(ret), K(bucket)); + } else if (bucket.left_ < bucket.right_) { + ObArray array; + for (int64_t i = bucket.left_; OB_SUCC(ret) && i <= bucket.right_; ++i) { + if (OB_FAIL(array.push_back(bucket.page_indexes_.at(i)))) { + LOG_WARN("fail to push back", KR(ret), K(i)); + } + } + if (OB_SUCC(ret)) { + int64_t mid = bucket.left_ + (bucket.right_ - bucket.left_) / 2; + for (int64_t i = 0; i < array.size(); ++i) { + int64_t new_pos = (bucket.left_ + i + mid) > bucket.right_ ? + bucket.left_ + i + mid - array.size() : + bucket.left_ + i + mid; + bucket.page_indexes_.at(new_pos) = array.at(i); + } + bucket.left_ = mid; + bucket.right_= mid - 1; + } + } + return ret; +} + +int mock_circle_cache(ObTmpFileWBPIndexCache &wbp_index_cache) +{ + int ret = OB_SUCCESS; + if (wbp_index_cache.right_ < 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid bucket", KR(ret), K(wbp_index_cache)); + } else if (wbp_index_cache.left_ < wbp_index_cache.right_) { + ObArray array; + for (int64_t i = wbp_index_cache.left_; OB_SUCC(ret) && i <= wbp_index_cache.right_; ++i) { + if (OB_ISNULL(wbp_index_cache.page_buckets_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", KR(ret), K(wbp_index_cache)); + } else if (OB_ISNULL(wbp_index_cache.page_buckets_->at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", KR(ret), K(wbp_index_cache)); + } else if (OB_FAIL(array.push_back(wbp_index_cache.page_buckets_->at(i)))) { + LOG_WARN("fail to push back", KR(ret), K(i)); + } + } + if (OB_SUCC(ret)) { + int64_t mid = wbp_index_cache.left_ + (wbp_index_cache.right_ - wbp_index_cache.left_) / 2; + for (int64_t i = 0; i < array.size(); ++i) { + int64_t new_pos = (wbp_index_cache.left_ + i + mid) > wbp_index_cache.right_ ? + wbp_index_cache.left_ + i + mid - array.size() : + wbp_index_cache.left_ + i + mid; + wbp_index_cache.page_buckets_->at(new_pos) = array.at(i); + } + wbp_index_cache.left_ = mid; + wbp_index_cache.right_= mid - 1; + } + } + + if (OB_SUCC(ret)) { + for (int64_t i = wbp_index_cache.left_; OB_SUCC(ret) && i <= wbp_index_cache.get_logic_tail_(); ++i) { + int64_t pos = i % wbp_index_cache.capacity_; + if (OB_ISNULL(wbp_index_cache.page_buckets_->at(pos))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", KR(ret), K(wbp_index_cache)); + } else if (OB_FAIL(mock_circle_bucket(*wbp_index_cache.page_buckets_->at(pos)))) { + LOG_WARN("fail to mock circle bucket", KR(ret), K(wbp_index_cache)); + } + } + } + return ret; +} + +TEST_F(TestTmpFileWBPIndexCache, test_search) +{ + int ret = OB_SUCCESS; + uint32_t begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + // 1. push indexes until reach the max capacity + int64_t page_num = MAX_BUCKET_ARRAY_CAPACITY * BUCKET_CAPACITY; + MockWBPIndexCache mock_cache(wbp_); + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + begin_page_id = mock_cache.mock_index_cache_.at(0); + ASSERT_NE(begin_page_id, ObTmpFileGlobal::INVALID_PAGE_ID); + ret = mock_circle_cache(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_GT(wbp_index_cache_.left_, wbp_index_cache_.right_); + ASSERT_GT(wbp_index_cache_.page_buckets_->at(wbp_index_cache_.left_)->left_, + wbp_index_cache_.page_buckets_->at(wbp_index_cache_.left_)->right_); + + // 2. push indexes to trigger sparsify + page_num = 20; + ret = write_and_push_pages(end_page_id, page_num, mock_cache, end_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + // 3. truncate indexes to make the first bucket not be full + page_num = 10; + ASSERT_LE(page_num, mock_cache.mock_index_cache_.count()); + ret = truncate_pages(page_num, mock_cache); + ASSERT_EQ(OB_SUCCESS, ret); + ret = mock_cache.compare(wbp_index_cache_); + ASSERT_EQ(OB_SUCCESS, ret); + + // 4. search page_index which exists in wbp_index_cache + int64_t search_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + uint32_t search_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t remove_page_id = mock_cache.mock_index_cache_.at(mock_cache.mock_index_cache_.count() / 5 + 1); + ret = wbp_.get_page_virtual_id(fd, remove_page_id, search_page_virtual_id); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp_index_cache_.binary_search(search_page_virtual_id, search_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(search_page_id, remove_page_id); + + // 5. search page_index which doesn't exist in wbp_index_cache and whose virtual id is not in the range of wbp_index_cache. + // the range of cache is [the page_virtual_id of the first index, INFINITY) + ret = wbp_index_cache_.binary_search(1, search_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(ObTmpFileGlobal::INVALID_PAGE_ID, search_page_id); + + // 6. search page_index which doesn't exist in wbp_index_cache and whose virtual id is in the range of wbp_index_cache. + // the range of cache is [the page_virtual_id of the first index, INFINITY) + int64_t page_virtual_id1 = -1; + int64_t page_virtual_id2 = -1; + ret = wbp_.get_page_virtual_id(fd, mock_cache.mock_index_cache_.at(0), page_virtual_id1); + ASSERT_EQ(OB_SUCCESS, ret); + ret = wbp_.get_page_virtual_id(fd, mock_cache.mock_index_cache_.at(1), page_virtual_id2); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_GT(page_virtual_id2 - page_virtual_id1, 1); + search_page_virtual_id = page_virtual_id1 + 1; + ret = wbp_index_cache_.binary_search(page_virtual_id1, search_page_id); + ASSERT_EQ(OB_SUCCESS, ret); + int64_t res_virtual_id = -1; + ret = wbp_.get_page_virtual_id(fd, search_page_id, res_virtual_id); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(page_virtual_id1, res_virtual_id); + + LOG_INFO("test_search"); +} + +} // namespace oceanbase + +int main(int argc, char **argv) +{ + int ret = 0; + system("rm -f ./test_tmp_file_write_buffer_pool_index_cache.log*"); + OB_LOGGER.set_file_name("test_tmp_file_write_buffer_pool_index_cache.log", true); + OB_LOGGER.set_log_level("INFO"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} \ No newline at end of file diff --git a/src/observer/ob_server.cpp b/src/observer/ob_server.cpp index 07ce0ff912..ee898e0bc7 100644 --- a/src/observer/ob_server.cpp +++ b/src/observer/ob_server.cpp @@ -76,6 +76,7 @@ #include "storage/compaction/ob_sstable_merge_info_mgr.h" #include "storage/tablelock/ob_table_lock_service.h" #include "storage/tx/ob_ts_mgr.h" +#include "storage/tmp_file/ob_tmp_file_cache.h" #include "storage/tx_table/ob_tx_data_cache.h" #include "storage/ob_file_system_router.h" #include "storage/ob_tablet_autoinc_seq_rpc_handler.h" @@ -419,6 +420,10 @@ int ObServer::init(const ObServerOptions &opts, const ObPLogWriterCfg &log_cfg) LOG_ERROR("init storage failed", KR(ret)); } else if (OB_FAIL(init_tx_data_cache())) { LOG_ERROR("init tx data cache failed", KR(ret)); + } else if (OB_FAIL(tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1))) { + LOG_ERROR("init tmp block cache failed", KR(ret)); + } else if (OB_FAIL(tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1))) { + LOG_ERROR("init tmp page cache failed", KR(ret)); } else if (OB_FAIL(init_log_kv_cache())) { LOG_ERROR("init log kv cache failed", KR(ret)); } else if (OB_FAIL(locality_manager_.init(self_addr_, @@ -703,10 +708,6 @@ void ObServer::destroy() disk_usage_report_task_.destroy(); FLOG_INFO("tenant disk usage report task destroyed"); - FLOG_INFO("begin to destroy tmp file manager"); - ObTmpFileManager::get_instance().destroy(); - FLOG_INFO("tmp file manager destroyed"); - FLOG_INFO("begin to destroy disk usage report task"); TG_DESTROY(lib::TGDefIDs::DiskUseReport); FLOG_INFO("disk usage report task destroyed"); @@ -719,6 +720,14 @@ void ObServer::destroy() OB_TX_DATA_KV_CACHE.destroy(); FLOG_INFO("tx data kv cache destroyed"); + FLOG_INFO("begin to destroy tmp block cache"); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + FLOG_INFO("tmp block cache destroyed"); + + FLOG_INFO("begin to destroy tmp page cache"); + tmp_file::ObTmpPageCache::get_instance().destroy(); + FLOG_INFO("tmp page cache destroyed"); + FLOG_INFO("begin to destroy log kv cache"); OB_LOG_KV_CACHE.destroy(); FLOG_INFO("log kv cache destroyed"); @@ -2917,8 +2926,6 @@ int ObServer::init_storage() storage_env_.bf_cache_miss_count_threshold_, storage_env_.storage_meta_cache_priority_))) { LOG_WARN("Fail to init OB_STORE_CACHE, ", KR(ret), K(storage_env_.data_dir_)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().init())) { - LOG_WARN("fail to init temp file manager", KR(ret)); } else if (OB_FAIL(OB_SERVER_BLOCK_MGR.init(THE_IO_DEVICE, storage_env_.default_block_size_))) { LOG_ERROR("init server block mgr fail", KR(ret)); diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index 6353db43e1..442faaa4e6 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -165,6 +165,7 @@ #include "rootserver/mview/ob_mview_maintenance_service.h" #include "share/resource_limit_calculator/ob_resource_limit_calculator.h" #include "storage/checkpoint/ob_checkpoint_diagnose.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" // ObTenantTmpFileManager #include "storage/restore/ob_tenant_restore_info_mgr.h" using namespace oceanbase; @@ -186,6 +187,7 @@ using namespace oceanbase::archive; using namespace oceanbase::observer; using namespace oceanbase::rootserver; using namespace oceanbase::blocksstable; +using namespace oceanbase::tmp_file; #define OB_TENANT_LOCK_BUCKET_NUM 10000L @@ -447,6 +449,7 @@ int ObMultiTenant::init(ObAddr myaddr, if (OB_SUCC(ret) && mtl_bind_flag) { MTL_BIND2(ObTenantIOManager::mtl_new, ObTenantIOManager::mtl_init, mtl_start_default, mtl_stop_default, nullptr, ObTenantIOManager::mtl_destroy); + MTL_BIND2(mtl_new_default, tmp_file::ObTenantTmpFileManager::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); // base mtl MTL_BIND2(mtl_new_default, storage::mds::ObTenantMdsService::mtl_init, storage::mds::ObTenantMdsService::mtl_start, storage::mds::ObTenantMdsService::mtl_stop, storage::mds::ObTenantMdsService::mtl_wait, mtl_destroy_default); diff --git a/src/observer/omt/ob_tenant.cpp b/src/observer/omt/ob_tenant.cpp index 85f1aa7cf2..7f2e74bf94 100644 --- a/src/observer/omt/ob_tenant.cpp +++ b/src/observer/omt/ob_tenant.cpp @@ -1162,17 +1162,6 @@ void ObTenant::destroy() ObTenantSwitchGuard guard(this); print_all_thread("TENANT_BEFORE_DESTROY", id_); destroy_mtl_module(); - // 1.some mtl module(eg: ObDataAccessService) remove tmp file when destroy, - // so remove_tenant_file must be after destroy_mtl_module. - // 2.there is tg in ObTmpTenantMemBlockManager, so remove_tenant_file must be before - // ObTenantBase::destroy() in which tg leak is checked. - if (OB_TMP_FAIL(FILE_MANAGER_INSTANCE_V2.remove_tenant_file(id_))) { - if (OB_ENTRY_NOT_EXIST == tmp_ret) { - tmp_ret = OB_SUCCESS; - } else { - LOG_WARN_RET(tmp_ret, "fail to free tmp tenant file store", K(ret), K_(id)); - } - } ObTenantBase::destroy(); if (nullptr != multi_level_queue_) { diff --git a/src/rootserver/backup/ob_backup_table_list_mgr.cpp b/src/rootserver/backup/ob_backup_table_list_mgr.cpp index f425c9c3d7..5e31397329 100644 --- a/src/rootserver/backup/ob_backup_table_list_mgr.cpp +++ b/src/rootserver/backup/ob_backup_table_list_mgr.cpp @@ -13,6 +13,7 @@ #define USING_LOG_PREFIX RS #include "ob_backup_table_list_mgr.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "storage/backup/ob_backup_data_store.h" #include "share/backup/ob_backup_io_adapter.h" @@ -354,10 +355,9 @@ int ObBackupTableListMgr::read_from_tmp_file_(const int64_t read_size, const int { int ret = OB_SUCCESS; table_list.reset(); - blocksstable::ObTmpFileIOInfo io_info; - blocksstable::ObTmpFileIOHandle handle; + tmp_file::ObTmpFileIOInfo io_info; + tmp_file::ObTmpFileIOHandle handle; io_info.fd_ = tmp_file_.get_fd(); - io_info.tenant_id_ = tmp_file_.get_tenant_id(); io_info.dir_id_ = tmp_file_.get_dir(); io_info.io_desc_.set_wait_event(2); io_info.size_ = read_size; @@ -370,7 +370,7 @@ int ObBackupTableListMgr::read_from_tmp_file_(const int64_t read_size, const int ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc memory", K(ret), K(read_size)); } else if (FALSE_IT(io_info.buf_ = buf)) { - } else if (OB_FAIL(blocksstable::ObTmpFileManager::get_instance().pread(io_info, offset, handle))) { + } else if (OB_FAIL(MTL(tmp_file::ObTenantTmpFileManager*)->pread(io_info, offset, handle))) { LOG_WARN("failed to pread from tmp file", K(ret), K(io_info), K(offset), K(read_size)); } else { blocksstable::ObBufferReader buffer_reader(buf, read_size); diff --git a/src/share/io/ob_io_define.cpp b/src/share/io/ob_io_define.cpp index f31d49771e..5211d26838 100644 --- a/src/share/io/ob_io_define.cpp +++ b/src/share/io/ob_io_define.cpp @@ -616,6 +616,29 @@ void ObIOResult::cancel() } } +int ObIOResult::wait(int64_t wait_ms) +{ + int ret = OB_SUCCESS; + ObThreadCondGuard guard(cond_); + if (OB_FAIL(guard.get_ret())) { + LOG_ERROR("fail to guard result condition", K(ret)); + } else { + int64_t begin_ms = ObTimeUtility::current_time(); + while (OB_SUCC(ret) && !is_finished_ && wait_ms > 0) { + if (OB_FAIL(cond_.wait(wait_ms))) { + LOG_WARN("fail to wait result condition", K(ret), K(wait_ms), K(*this)); + } else if (!is_finished_) { + int64_t duration_ms = ObTimeUtility::current_time() - begin_ms; + wait_ms -= duration_ms; + } + } + if (OB_UNLIKELY(wait_ms <= 0)) { // rarely happen + ret = OB_TIMEOUT; + } + } + return ret; +} + void ObIOResult::inc_ref(const char *msg) { (void)ATOMIC_FAA(&result_ref_cnt_, 1); @@ -1449,13 +1472,19 @@ bool ObIOHandle::is_valid() const return nullptr != result_; } -int ObIOHandle::wait() +int ObIOHandle::wait(const int64_t wait_timeout_ms) { int ret = OB_SUCCESS; if (OB_ISNULL(result_)) { ret = OB_NOT_INIT; LOG_WARN("The IOHandle has not been inited, ", K(ret)); - } else if (!result_->is_finished_) { + } else if (OB_FAIL(result_->ret_code_.io_ret_)) { + LOG_WARN("IO error, ", K(ret), K(*result_)); + } else if (result_->is_finished_) { + // do nothing + } else if (0 == wait_timeout_ms) { + ret = OB_EAGAIN; + } else if (UINT64_MAX == wait_timeout_ms) { const int64_t timeout_ms = ((result_->begin_ts_ > 0 ? result_->begin_ts_ + result_->timeout_us_ : 0) - ObTimeUtility::current_time()) / 1000L; ObWaitEventGuard wait_guard(result_->flag_.get_wait_event(), @@ -1464,24 +1493,10 @@ int ObIOHandle::wait() const int64_t real_wait_timeout = min(OB_IO_MANAGER.get_io_config().data_storage_io_timeout_ms_, timeout_ms); if (real_wait_timeout > 0) { - ObThreadCondGuard guard(result_->cond_); - if (OB_FAIL(guard.get_ret())) { - LOG_ERROR("fail to guard result condition", K(ret)); - } else { - int64_t wait_ms = real_wait_timeout; - int64_t begin_ms = ObTimeUtility::current_time(); - while (OB_SUCC(ret) && !result_->is_finished_ && wait_ms > 0) { - if (OB_FAIL(result_->cond_.wait(wait_ms))) { - LOG_WARN("fail to wait result condition", K(ret), K(wait_ms), K(*result_)); - } else if (!result_->is_finished_) { - int64_t duration_ms = ObTimeUtility::current_time() - begin_ms; - wait_ms = real_wait_timeout - duration_ms; - } - } - if (OB_UNLIKELY(wait_ms <= 0)) { // rarely happen - ret = OB_TIMEOUT; - LOG_WARN("fail to wait result condition due to spurious wakeup", - K(ret), K(wait_ms), K(*result_)); + int64_t wait_ms = real_wait_timeout; + if (OB_FAIL(result_->wait(wait_ms))) { + if (OB_TIMEOUT == ret) { // rarely happen + LOG_WARN("fail to wait result condition due to spurious wakeup", K(ret), K(wait_ms), K(*result_)); } } } else if (result_->is_finished_) { @@ -1489,6 +1504,16 @@ int ObIOHandle::wait() } else { ret = OB_TIMEOUT; } + } else { + int64_t wait_ms = wait_timeout_ms; + if (OB_FAIL(result_->wait(wait_ms))) { + if (OB_TIMEOUT == ret) { + const int64_t real_wait_timeout = min(OB_IO_MANAGER.get_io_config().data_storage_io_timeout_ms_, result_->timeout_us_ / 1000L); + if ((ObTimeUtility::current_time() - result_->begin_ts_) / 1000L < real_wait_timeout) { + ret = OB_EAGAIN; + } + } + } } if (OB_SUCC(ret)) { if (OB_FAIL(result_->ret_code_.io_ret_)) { diff --git a/src/share/io/ob_io_define.h b/src/share/io/ob_io_define.h index c412e12b72..b3652aaa55 100644 --- a/src/share/io/ob_io_define.h +++ b/src/share/io/ob_io_define.h @@ -296,6 +296,7 @@ public: void reset(); void destroy(); void cancel(); + int wait(int64_t wait_ms); void finish(const ObIORetCode &ret_code, ObIORequest *req = nullptr); void calc_io_offset_and_size(int64_t &size, int32_t &offset); ObIOMode get_mode() const; @@ -465,7 +466,7 @@ public: bool is_valid() const; OB_INLINE bool is_finished() const { return nullptr != result_ && result_->is_finished_; } - int wait(); + int wait(const int64_t wait_timeout_ms=UINT64_MAX); const char *get_buffer(); int64_t get_data_size() const; int64_t get_rt() const; diff --git a/src/share/ob_thread_define.h b/src/share/ob_thread_define.h index 6e2036499e..c23a374ace 100755 --- a/src/share/ob_thread_define.h +++ b/src/share/ob_thread_define.h @@ -168,4 +168,6 @@ TG_DEF(StartupAccelHandler, StartupAccelHandler, QUEUE_THREAD, 1, observer::ObSt TG_DEF(TenantTTLManager, TTLManager, TIMER) TG_DEF(TenantTabletTTLMgr, TTLTabletMgr, TIMER) TG_DEF(TntSharedTimer, TntSharedTimer, TIMER) + +TG_DEF(TmpFileSwap, TFSwap, THREAD_POOL, 1) #endif diff --git a/src/share/rc/ob_tenant_base.h b/src/share/rc/ob_tenant_base.h index 693eb7d5d4..170e06c2d3 100755 --- a/src/share/rc/ob_tenant_base.h +++ b/src/share/rc/ob_tenant_base.h @@ -66,6 +66,9 @@ namespace blocksstable { class ObSharedMacroBlockMgr; class ObDecodeResourcePool; } +namespace tmp_file { + class ObTenantTmpFileManager; +} namespace storage { namespace mds { class ObTenantMdsService; @@ -257,6 +260,7 @@ using ObTableScanIteratorObjPool = common::ObServerObjectPoolon_write_io(rdtsc() - begin_io_dump_time); @@ -2128,7 +2126,6 @@ int ObChunkDatumStore::write_file(void *buf, int64_t size) LOG_WARN("open file failed", K(ret)); } else { file_size_ = 0; - io_.tenant_id_ = tenant_id_; io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io_.io_timeout_ms_ = timeout_ms; LOG_INFO("open file success", K_(io_.fd), K_(io_.dir_id)); @@ -2157,7 +2154,7 @@ int ObChunkDatumStore::read_file( void *buf, const int64_t size, const int64_t offset, - blocksstable::ObTmpFileIOHandle &handle, + tmp_file::ObTmpFileIOHandle &handle, const int64_t file_size, const int64_t cur_pos, int64_t &tmp_file_size) @@ -2183,7 +2180,7 @@ int ObChunkDatumStore::read_file( CK (cur_pos >= file_size); OX (ret = OB_ITER_END); } else { - blocksstable::ObTmpFileIOInfo tmp_io = io_; + tmp_file::ObTmpFileIOInfo tmp_io = io_; set_io(size, static_cast(buf), tmp_io); tmp_io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); tmp_io.io_timeout_ms_ = timeout_ms; @@ -2195,10 +2192,10 @@ int ObChunkDatumStore::read_file( if (OB_ITER_END != ret) { LOG_WARN("read form file failed", K(ret), K(tmp_io), K(offset), K(timeout_ms)); } - } else if (handle.get_data_size() != size) { + } else if (handle.get_done_size() != size) { ret = OB_INNER_STAT_ERROR; LOG_WARN("read data less than expected", - K(ret), K(tmp_io), "read_size", handle.get_data_size()); + K(ret), K(tmp_io), "read_size", handle.get_done_size()); } } return ret; @@ -2208,7 +2205,7 @@ int ObChunkDatumStore::aio_read_file( void *buf, const int64_t size, const int64_t offset, - blocksstable::ObTmpFileIOHandle &handle) + tmp_file::ObTmpFileIOHandle &handle) { int ret = OB_SUCCESS; if (!is_inited()) { @@ -2218,7 +2215,7 @@ int ObChunkDatumStore::aio_read_file( ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(size), K(offset), KP(buf)); } else if (size > 0) { - blocksstable::ObTmpFileIOInfo tmp_io = io_; + tmp_file::ObTmpFileIOInfo tmp_io = io_; set_io(size, static_cast(buf), tmp_io); tmp_io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); if (OB_FAIL(get_timeout(tmp_io.io_timeout_ms_))) { @@ -2750,8 +2747,11 @@ int ObChunkDatumStore::Iterator::aio_read(char *buf, const int64_t size) if (!aio_read_handle_.is_valid()) { // first read, wait write finish int64_t timeout_ms = 0; - OZ(store_->get_timeout(timeout_ms)); - OZ(store_->aio_write_handle_.wait()); + if (OB_FAIL(store_->get_timeout(timeout_ms))) { + LOG_WARN("fail to exec store_->get_timeout", K(ret)); + } else if (store_->aio_write_handle_.is_valid() && OB_FAIL(store_->aio_write_handle_.wait())) { + LOG_WARN("fail to exec store_->aio_write_handle_.wait", K(ret)); + } } if (OB_SUCC(ret)) { if (size <= 0 || cur_iter_pos_ >= file_size_) { diff --git a/src/sql/engine/basic/ob_chunk_datum_store.h b/src/sql/engine/basic/ob_chunk_datum_store.h index d3388f68dc..9a94b5d452 100644 --- a/src/sql/engine/basic/ob_chunk_datum_store.h +++ b/src/sql/engine/basic/ob_chunk_datum_store.h @@ -22,7 +22,7 @@ #include "common/row/ob_row_iterator.h" #include "share/datum/ob_datum.h" #include "sql/engine/expr/ob_expr.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "sql/engine/basic/ob_sql_mem_callback.h" #include "sql/engine/basic/ob_batch_result_holder.h" @@ -831,7 +831,7 @@ public: // cp from chunk iter ObChunkDatumStore* store_; Block* cur_iter_blk_; - blocksstable::ObTmpFileIOHandle aio_read_handle_; + tmp_file::ObTmpFileIOHandle aio_read_handle_; int64_t cur_nth_blk_; //reading nth blk start from 1 int64_t cur_chunk_n_blocks_; //the number of blocks of current chunk int64_t cur_iter_pos_; //pos in file @@ -1136,16 +1136,16 @@ private: int write_file(void *buf, int64_t size); int read_file( - void *buf, const int64_t size, const int64_t offset, blocksstable::ObTmpFileIOHandle &handle, + void *buf, const int64_t size, const int64_t offset, tmp_file::ObTmpFileIOHandle &handle, const int64_t file_size, const int64_t cur_pos, int64_t &tmp_file_size); int aio_read_file(void *buf, const int64_t size, const int64_t offset, - blocksstable::ObTmpFileIOHandle &handle); + tmp_file::ObTmpFileIOHandle &handle); bool need_dump(int64_t extra_size); BlockBuffer* new_block(); void set_io(int64_t size, char *buf) { io_.size_ = size; io_.buf_ = buf; } - static void set_io(int64_t size, char *buf, blocksstable::ObTmpFileIOInfo &io) { io.size_ = size; io.buf_ = buf; } + static void set_io(int64_t size, char *buf, tmp_file::ObTmpFileIOInfo &io) { io.size_ = size; io.buf_ = buf; } bool find_block_can_hold(const int64_t size, bool &need_shrink); int get_store_row(RowIterator &it, const StoredRow *&sr); inline void callback_alloc(int64_t size) { if (callback_ != nullptr) callback_->alloc(size); } @@ -1175,7 +1175,7 @@ private: int64_t row_cnt_; int64_t col_count_; - blocksstable::ObTmpFileIOHandle aio_write_handle_; + tmp_file::ObTmpFileIOHandle aio_write_handle_; bool enable_dump_; bool has_dumped_; @@ -1184,7 +1184,7 @@ private: ObIOEventObserver *io_event_observer_; //int fd_; - blocksstable::ObTmpFileIOInfo io_; + tmp_file::ObTmpFileIOInfo io_; int64_t file_size_; int64_t n_block_in_file_; diff --git a/src/sql/engine/basic/ob_chunk_row_store.cpp b/src/sql/engine/basic/ob_chunk_row_store.cpp index 1f291c014a..633a672e2b 100644 --- a/src/sql/engine/basic/ob_chunk_row_store.cpp +++ b/src/sql/engine/basic/ob_chunk_row_store.cpp @@ -861,8 +861,6 @@ int ObChunkRowStore::finish_add_row(bool need_dump) LOG_WARN("finish_add_row dump error", K(ret)); } else if (OB_FAIL(get_timeout(timeout_ms))) { LOG_WARN("get timeout failed", K(ret)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(io_.fd_, timeout_ms))) { - LOG_WARN("sync file failed", K(ret), K_(io_.fd), K(timeout_ms)); } } else { LOG_DEBUG("finish_add_row no need to dump", K(ret)); @@ -1627,7 +1625,6 @@ int ObChunkRowStore::write_file(void *buf, int64_t size) LOG_WARN("open file failed", K(ret)); } else { file_size_ = 0; - io_.tenant_id_ = tenant_id_; io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io_.io_timeout_ms_ = timeout_ms; LOG_TRACE("open file success", K_(io_.fd), K_(io_.dir_id)); @@ -1674,7 +1671,7 @@ int ObChunkRowStore::read_file(void *buf, const int64_t size, const int64_t offs this->set_io(size, static_cast(buf)); io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); io_.io_timeout_ms_ = timeout_ms; - blocksstable::ObTmpFileIOHandle handle; + tmp_file::ObTmpFileIOHandle handle; if (0 == read_size && OB_FAIL(FILE_MANAGER_INSTANCE_V2.get_tmp_file_size(io_.fd_, tmp_file_size))) { LOG_WARN("failed to get tmp file size", K(ret)); @@ -1682,10 +1679,10 @@ int ObChunkRowStore::read_file(void *buf, const int64_t size, const int64_t offs if (OB_ITER_END != ret) { LOG_WARN("read form file failed", K(ret), K(io_), K(offset), K(timeout_ms)); } - } else if (handle.get_data_size() != size) { + } else if (handle.get_done_size() != size) { ret = OB_INNER_STAT_ERROR; LOG_WARN("read data less than expected", - K(ret), K(io_), "read_size", handle.get_data_size()); + K(ret), K(io_), "read_size", handle.get_done_size()); } } return ret; diff --git a/src/sql/engine/basic/ob_chunk_row_store.h b/src/sql/engine/basic/ob_chunk_row_store.h index c96e72878b..b82d2ef8f8 100644 --- a/src/sql/engine/basic/ob_chunk_row_store.h +++ b/src/sql/engine/basic/ob_chunk_row_store.h @@ -20,7 +20,7 @@ #include "lib/list/ob_dlist.h" #include "common/row/ob_row.h" #include "common/row/ob_row_iterator.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "sql/engine/basic/ob_sql_mem_callback.h" namespace oceanbase @@ -494,7 +494,7 @@ private: int64_t dumped_row_cnt_; //int fd_; - blocksstable::ObTmpFileIOInfo io_; + tmp_file::ObTmpFileIOInfo io_; int64_t file_size_; int64_t n_block_in_file_; diff --git a/src/sql/engine/basic/ob_material_op_impl.cpp b/src/sql/engine/basic/ob_material_op_impl.cpp index 2d64daf67c..214101a9b2 100644 --- a/src/sql/engine/basic/ob_material_op_impl.cpp +++ b/src/sql/engine/basic/ob_material_op_impl.cpp @@ -53,8 +53,8 @@ void ObMaterialOpImpl::reset() { sql_mem_processor_.unregister_profile(); io_event_observer_ = nullptr; - datum_store_.reset(); datum_store_it_.reset(); + datum_store_.reset(); got_first_row_ = false; inited_ = false; // can not destroy mem_entify here, the memory may hold by %iter_ or %datum_store_ diff --git a/src/sql/engine/basic/ob_ra_datum_store.cpp b/src/sql/engine/basic/ob_ra_datum_store.cpp index b6033435bb..5c18edaaad 100644 --- a/src/sql/engine/basic/ob_ra_datum_store.cpp +++ b/src/sql/engine/basic/ob_ra_datum_store.cpp @@ -14,7 +14,7 @@ #include "ob_ra_datum_store.h" #include "lib/container/ob_se_array_iterator.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "lib/utility/ob_tracepoint.h" #include "share/config/ob_server_config.h" #include "sql/engine/basic/ob_chunk_datum_store.h" @@ -1144,11 +1144,10 @@ int ObRADatumStore::write_file(BlockIndex &bi, void *buf, int64_t size) if (NULL != mem_stat_) { mem_stat_->dumped(size); } - blocksstable::ObTmpFileIOInfo io; + tmp_file::ObTmpFileIOInfo io; io.fd_ = fd_; io.buf_ = static_cast(buf); io.size_ = size; - io.tenant_id_ = tenant_id_; io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io.io_timeout_ms_ = timeout_ms; const uint64_t start = rdtsc(); @@ -1184,22 +1183,21 @@ int ObRADatumStore::read_file(void *buf, const int64_t size, const int64_t offse } if (OB_SUCC(ret) && size > 0) { - blocksstable::ObTmpFileIOInfo io; + tmp_file::ObTmpFileIOInfo io; io.fd_ = fd_; io.dir_id_ = dir_id_; io.buf_ = static_cast(buf); io.size_ = size; - io.tenant_id_ = tenant_id_; io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); io.io_timeout_ms_ = timeout_ms; const uint64_t start = rdtsc(); - blocksstable::ObTmpFileIOHandle handle; + tmp_file::ObTmpFileIOHandle handle; if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.pread(io, offset, handle))) { LOG_WARN("read form file failed", K(ret), K(io), K(offset), K(timeout_ms)); - } else if (OB_UNLIKELY(handle.get_data_size() != size)) { + } else if (OB_UNLIKELY(handle.get_done_size() != size)) { ret = OB_INNER_STAT_ERROR; LOG_WARN("read data less than expected", - K(ret), K(io), "read_size", handle.get_data_size()); + K(ret), K(io), "read_size", handle.get_done_size()); } if (NULL != io_observer_) { io_observer_->on_read_io(rdtsc() - start); @@ -1323,8 +1321,6 @@ int ObRADatumStore::finish_add_row() LOG_WARN("write last index block to file failed", K(ret), K(ret)); } else if (OB_FAIL(get_timeout(timeout_ms))) { LOG_WARN("get timeout failed", K(ret)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(fd_, timeout_ms))) { - LOG_WARN("sync file failed", K(ret), K(fd_), K(timeout_ms)); } else { if (blkbuf_.buf_.is_inited()) { free_blk_mem(blkbuf_.buf_.data(), blkbuf_.buf_.capacity()); diff --git a/src/sql/engine/basic/ob_ra_row_store.cpp b/src/sql/engine/basic/ob_ra_row_store.cpp index 0b34694594..e6abade66d 100644 --- a/src/sql/engine/basic/ob_ra_row_store.cpp +++ b/src/sql/engine/basic/ob_ra_row_store.cpp @@ -14,7 +14,7 @@ #include "ob_ra_row_store.h" #include "lib/container/ob_se_array_iterator.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "lib/utility/ob_tracepoint.h" #include "share/config/ob_server_config.h" @@ -962,11 +962,10 @@ int ObRARowStore::write_file(BlockIndex &bi, void *buf, int64_t size) ret = OB_E(EventTable::EN_8) ret; } if (OB_SUCC(ret) && size > 0) { - blocksstable::ObTmpFileIOInfo io; + tmp_file::ObTmpFileIOInfo io; io.fd_ = fd_; io.buf_ = static_cast(buf); io.size_ = size; - io.tenant_id_ = tenant_id_; io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io.io_timeout_ms_ = timeout_ms; if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.write(io))) { @@ -996,21 +995,20 @@ int ObRARowStore::read_file(void *buf, const int64_t size, const int64_t offset) } if (OB_SUCC(ret) && size > 0) { - blocksstable::ObTmpFileIOInfo io; + tmp_file::ObTmpFileIOInfo io; io.fd_ = fd_; io.dir_id_ = dir_id_; io.buf_ = static_cast(buf); io.size_ = size; - io.tenant_id_ = tenant_id_; io.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); io.io_timeout_ms_ = timeout_ms; - blocksstable::ObTmpFileIOHandle handle; + tmp_file::ObTmpFileIOHandle handle; if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.pread(io, offset, handle))) { LOG_WARN("read form file failed", K(ret), K(io), K(offset), K(timeout_ms)); - } else if (handle.get_data_size() != size) { + } else if (handle.get_done_size() != size) { ret = OB_INNER_STAT_ERROR; LOG_WARN("read data less than expected", - K(ret), K(io), "read_size", handle.get_data_size()); + K(ret), K(io), "read_size", handle.get_done_size()); } } return ret; @@ -1104,8 +1102,6 @@ int ObRARowStore::finish_add_row() LOG_WARN("write last index block to file failed", K(ret), K(ret)); } else if (OB_FAIL(get_timeout(timeout_ms))) { LOG_WARN("get timeout failed", K(ret)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(fd_, timeout_ms))) { - LOG_WARN("sync file failed", K(ret), K(fd_), K(timeout_ms)); } else { if (blkbuf_.buf_.is_inited()) { free_blk_mem(blkbuf_.buf_.data(), blkbuf_.buf_.capacity()); diff --git a/src/sql/engine/basic/ob_temp_block_store.cpp b/src/sql/engine/basic/ob_temp_block_store.cpp index ecf87bd820..8878909646 100644 --- a/src/sql/engine/basic/ob_temp_block_store.cpp +++ b/src/sql/engine/basic/ob_temp_block_store.cpp @@ -14,7 +14,7 @@ #include "ob_temp_block_store.h" #include "lib/container/ob_se_array_iterator.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "lib/utility/ob_tracepoint.h" #include "share/config/ob_server_config.h" #include "sql/engine/basic/ob_chunk_datum_store.h" @@ -205,8 +205,6 @@ int ObTempBlockStore::finish_add_row(bool need_dump /*true*/) LOG_WARN("get timeout failed", K(ret)); } else if (write_io_handle_.is_valid() && OB_FAIL(write_io_handle_.wait())) { LOG_WARN("fail to wait write", K(ret), K(write_io_handle_)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(io_.fd_, timeout_ms))) { - LOG_WARN("sync file failed", K(ret), K(io_.fd_), K(timeout_ms)); } if (OB_LIKELY(nullptr != io_observer_)) { io_observer_->on_write_io(rdtsc() - begin_io_dump_time); @@ -413,7 +411,7 @@ int ObTempBlockStore::decompr_block(BlockReader &reader, const Block *&blk) ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpeteced null pointer", K(ret), KP(blk), KP(reader.buf_.data())); } else { - int64_t comp_size = reader.read_io_handle_.get_data_size() - sizeof(Block); + int64_t comp_size = reader.read_io_handle_.get_done_size() - sizeof(Block); int64_t decomp_size = blk->raw_size_ - sizeof(Block); int64_t actual_uncomp_size = 0; if (OB_FAIL(ensure_reader_buffer(reader, reader.decompr_buf_, blk->raw_size_))) { @@ -1076,7 +1074,6 @@ int ObTempBlockStore::write_file(BlockIndex &bi, void *buf, int64_t size) LOG_WARN("open file failed", K(ret)); } else { file_size_ = 0; - io_.tenant_id_ = tenant_id_; io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io_.io_timeout_ms_ = timeout_ms; LOG_INFO("open file success", K_(io_.fd), K_(io_.dir_id), K(get_compressor_type())); @@ -1109,7 +1106,7 @@ int ObTempBlockStore::write_file(BlockIndex &bi, void *buf, int64_t size) } int ObTempBlockStore::read_file(void *buf, const int64_t size, const int64_t offset, - blocksstable::ObTmpFileIOHandle &handle, const bool is_async) + tmp_file::ObTmpFileIOHandle &handle, const bool is_async) { int ret = OB_SUCCESS; int64_t timeout_ms = 0; @@ -1118,12 +1115,12 @@ int ObTempBlockStore::read_file(void *buf, const int64_t size, const int64_t off LOG_WARN("invalid argument", K(size), K(offset), KP(buf)); } else if (OB_FAIL(get_timeout(timeout_ms))) { LOG_WARN("get timeout failed", K(ret)); - } else if (!handle.is_valid() && OB_FAIL(write_io_handle_.wait())) { + } else if (!handle.is_valid() && write_io_handle_.is_valid() && OB_FAIL(write_io_handle_.wait())) { LOG_WARN("fail to wait write", K(ret)); } if (OB_SUCC(ret) && size > 0) { - blocksstable::ObTmpFileIOInfo tmp_read_id = io_; + tmp_file::ObTmpFileIOInfo tmp_read_id = io_; tmp_read_id.buf_ = static_cast(buf); tmp_read_id.size_ = size; tmp_read_id.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_READ); @@ -1136,10 +1133,10 @@ int ObTempBlockStore::read_file(void *buf, const int64_t size, const int64_t off } else { if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.pread(tmp_read_id, offset, handle))) { LOG_WARN("read form file failed", K(ret), K(tmp_read_id), K(offset), K(timeout_ms)); - } else if (OB_UNLIKELY(handle.get_data_size() != size)) { + } else if (OB_UNLIKELY(handle.get_done_size() != size)) { ret = OB_INNER_STAT_ERROR; LOG_WARN("read data less than expected", K(ret), K(tmp_read_id), - "read_size", handle.get_data_size()); + "read_size", handle.get_done_size()); } } if (NULL != io_observer_) { @@ -1421,7 +1418,6 @@ void ObTempBlockStore::BlockReader::reuse() buf_.reset(); decompr_buf_.reset(); } - read_io_handle_.set_last_extent_id(0); } void ObTempBlockStore::BlockReader::reset_cursor(const int64_t file_size, const bool need_release) diff --git a/src/sql/engine/basic/ob_temp_block_store.h b/src/sql/engine/basic/ob_temp_block_store.h index cb1dc99db4..0bcdcb5db4 100644 --- a/src/sql/engine/basic/ob_temp_block_store.h +++ b/src/sql/engine/basic/ob_temp_block_store.h @@ -20,7 +20,7 @@ #include "sql/engine/basic/ob_sql_mem_callback.h" #include "lib/checksum/ob_crc64.h" #include "sql/engine/basic/chunk_store/ob_chunk_block_compressor.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" namespace oceanbase { @@ -289,7 +289,7 @@ public: inline int64_t get_block_cnt() const { return store_->get_block_cnt(); } void set_iteration_age(IterationAge *age) { age_ = age; } void set_blk_holder(BlockHolder *holder) { blk_holder_ptr_ = holder; } - blocksstable::ObTmpFileIOHandle& get_read_io_handler() { return read_io_handle_; } + tmp_file::ObTmpFileIOHandle& get_read_io_handler() { return read_io_handle_; } inline bool is_async() const { return is_async_; } void reset(); void reuse(); @@ -329,7 +329,7 @@ public: IterationAge inner_age_; // to optimize performance, record the last_extent_id to avoid do binary search every time // calling read. - blocksstable::ObTmpFileIOHandle read_io_handle_; + tmp_file::ObTmpFileIOHandle read_io_handle_; int64_t cur_file_offset_; bool is_async_; int aio_buf_idx_; @@ -507,7 +507,7 @@ private: int ensure_reader_buffer(BlockReader &reader, ShrinkBuffer &buf, const int64_t size); int write_file(BlockIndex &bi, void *buf, int64_t size); int read_file(void *buf, const int64_t size, const int64_t offset, - blocksstable::ObTmpFileIOHandle &handle, const bool is_async); + tmp_file::ObTmpFileIOHandle &handle, const bool is_async); int dump_block_if_need(const int64_t extra_size); bool need_dump(const int64_t extra_size); int write_compressed_block(Block *blk, BlockIndex *bi); @@ -590,8 +590,8 @@ private: ObSqlMemoryCallback *mem_stat_; ObChunkBlockCompressor compressor_; ObIOEventObserver *io_observer_; - blocksstable::ObTmpFileIOHandle write_io_handle_; - blocksstable::ObTmpFileIOInfo io_; + tmp_file::ObTmpFileIOHandle write_io_handle_; + tmp_file::ObTmpFileIOInfo io_; bool last_block_on_disk_; int64_t cur_file_offset_; diff --git a/src/storage/CMakeLists.txt b/src/storage/CMakeLists.txt index 6d263d2c5a..23ac9cb286 100644 --- a/src/storage/CMakeLists.txt +++ b/src/storage/CMakeLists.txt @@ -43,9 +43,6 @@ ob_set_subtarget(ob_storage blocksstable blocksstable/ob_sstable_printer.cpp blocksstable/ob_storage_cache_suite.cpp blocksstable/ob_super_block_buffer_holder.cpp - blocksstable/ob_tmp_file.cpp - blocksstable/ob_tmp_file_cache.cpp - blocksstable/ob_tmp_file_store.cpp blocksstable/ob_datum_row.cpp blocksstable/ob_datum_rowkey.cpp blocksstable/ob_data_store_desc.cpp @@ -142,6 +139,27 @@ ob_set_subtarget(ob_storage blocksstable_index_block blocksstable/index_block/ob_sstable_sec_meta_iterator.cpp ) +ob_set_subtarget(ob_storage tmp_file + tmp_file/ob_shared_nothing_tmp_file.cpp + tmp_file/ob_tmp_file_write_buffer_index_cache.cpp + tmp_file/ob_tmp_file_cache.cpp + tmp_file/ob_tmp_file_io_define.cpp + tmp_file/ob_tmp_file_io_ctx.cpp + tmp_file/ob_tmp_file_global.cpp + tmp_file/ob_tmp_file_block_manager.cpp + tmp_file/ob_tmp_file_manager.cpp + tmp_file/ob_tmp_file_eviction_manager.cpp + tmp_file/ob_tmp_file_flush_priority_manager.cpp + tmp_file/ob_tmp_file_flush_list_iterator.cpp + tmp_file/ob_tmp_file_meta_tree.cpp + tmp_file/ob_tmp_file_flush_manager.cpp + tmp_file/ob_tmp_file_flush_ctx.cpp + tmp_file/ob_tmp_file_page_cache_controller.cpp + tmp_file/ob_tmp_file_thread_wrapper.cpp + tmp_file/ob_tmp_file_thread_job.cpp + tmp_file/ob_tmp_file_write_buffer_pool.cpp +) + ob_set_subtarget(ob_storage slog slog/ob_server_slog_writer.cpp slog/ob_storage_log_batch_header.cpp diff --git a/src/storage/backup/ob_backup_tmp_file.cpp b/src/storage/backup/ob_backup_tmp_file.cpp index 7be81a84d4..ab4d73b5d4 100644 --- a/src/storage/backup/ob_backup_tmp_file.cpp +++ b/src/storage/backup/ob_backup_tmp_file.cpp @@ -16,6 +16,7 @@ #include "lib/oblog/ob_log_module.h" using namespace oceanbase::blocksstable; +using namespace oceanbase::tmp_file; using namespace oceanbase::storage; namespace oceanbase { @@ -40,9 +41,9 @@ int ObBackupTmpFile::open(const uint64_t tenant_id) if (is_opened_) { ret = OB_INIT_TWICE; LOG_WARN("backup tmp file init twice", K(ret)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().alloc_dir(file_dir_))) { + } else if (OB_FAIL(ObTenantTmpFileManager::get_instance().alloc_dir(file_dir_))) { LOG_WARN("failed to alloc dir", K(ret)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().open(file_fd_, file_dir_))) { + } else if (OB_FAIL(ObTenantTmpFileManager::get_instance().open(file_fd_, file_dir_))) { LOG_WARN("failed to open tmp file", K(ret), K(file_dir_)); } else { tenant_id_ = tenant_id; @@ -63,7 +64,7 @@ int ObBackupTmpFile::write(const char *buf, const int64_t size) LOG_WARN("backup tmp file init twice", K(ret)); } else if (OB_FAIL(get_io_info_(buf, size, timeout_ms, io_info))) { LOG_WARN("failed to get io info", K(ret), K(buf), K(size)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().write(io_info))) { + } else if (OB_FAIL(ObTenantTmpFileManager::get_instance().write(io_info))) { LOG_WARN("failed to write tmp file", K(ret), K(io_info), K(timeout_ms)); } else { file_size_ += size; @@ -78,7 +79,7 @@ int ObBackupTmpFile::close() if (!is_opened_) { ret = OB_NOT_INIT; LOG_WARN("backup tmp file do not init", K(ret)); - } else if (OB_FAIL(ObTmpFileManager::get_instance().remove(file_fd_))) { + } else if (OB_FAIL(ObTenantTmpFileManager::get_instance().remove(file_fd_))) { LOG_WARN("failed to remove tmp file fd", K(ret), K(file_fd_)); } else { is_opened_ = false; @@ -92,7 +93,7 @@ int ObBackupTmpFile::get_io_info_(const char *buf, const int64_t size, const int int ret = OB_SUCCESS; io_info.reset(); io_info.fd_ = file_fd_; - io_info.tenant_id_ = tenant_id_; + io_info.dir_id_ = file_dir_; io_info.io_desc_.set_wait_event(2); io_info.buf_ = const_cast(buf); io_info.size_ = size; diff --git a/src/storage/backup/ob_backup_tmp_file.h b/src/storage/backup/ob_backup_tmp_file.h index 63a3358264..f53363b338 100644 --- a/src/storage/backup/ob_backup_tmp_file.h +++ b/src/storage/backup/ob_backup_tmp_file.h @@ -14,7 +14,7 @@ #define STORAGE_LOG_STREAM_BACKUP_TMP_FILE_H_ #include "storage/backup/ob_backup_data_struct.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "storage/blocksstable/ob_data_buffer.h" namespace oceanbase { @@ -49,7 +49,7 @@ public: TO_STRING_KV(K_(is_opened), K_(tenant_id), K_(file_dir), K_(file_fd), K_(file_size)); private: - int get_io_info_(const char *buf, const int64_t size, const int64_t timeout_ms, blocksstable::ObTmpFileIOInfo &io_info); + int get_io_info_(const char *buf, const int64_t size, const int64_t timeout_ms, tmp_file::ObTmpFileIOInfo &io_info); private: bool is_opened_; @@ -139,10 +139,9 @@ int ObBackupIndexBufferNode::get_backup_index(T &backup_index) backup_index.reset(); const int64_t need_read_size = sizeof(T); const int64_t timeout_ms = 5000; - blocksstable::ObTmpFileIOInfo io_info; - blocksstable::ObTmpFileIOHandle handle; + tmp_file::ObTmpFileIOInfo io_info; + tmp_file::ObTmpFileIOHandle handle; io_info.fd_ = tmp_file_.get_fd(); - io_info.tenant_id_ = tmp_file_.get_tenant_id(); io_info.io_desc_.set_wait_event(2); io_info.size_ = std::min(need_read_size, estimate_size_ - read_offset_); io_info.io_timeout_ms_ = timeout_ms; @@ -158,7 +157,7 @@ int ObBackupIndexBufferNode::get_backup_index(T &backup_index) ret = OB_ALLOCATE_MEMORY_FAILED; OB_LOG(WARN, "failed to alloc memory", K(ret), K(need_read_size)); } else if (FALSE_IT(io_info.buf_ = buf)) { - } else if (OB_FAIL(blocksstable::ObTmpFileManager::get_instance().pread(io_info, read_offset_, handle))) { + } else if (OB_FAIL(tmp_file::ObTenantTmpFileManager::get_instance().pread(io_info, read_offset_, handle))) { OB_LOG(WARN, "failed to pread from tmp file", K(ret), K(io_info), K_(read_offset), K(need_read_size)); } else { blocksstable::ObBufferReader buffer_reader(buf, need_read_size); diff --git a/src/storage/blocksstable/ob_block_manager.cpp b/src/storage/blocksstable/ob_block_manager.cpp index 182c19613a..c3239d91f2 100644 --- a/src/storage/blocksstable/ob_block_manager.cpp +++ b/src/storage/blocksstable/ob_block_manager.cpp @@ -27,7 +27,7 @@ #include "storage/blocksstable/ob_block_manager.h" #include "storage/blocksstable/ob_macro_block_struct.h" #include "storage/blocksstable/ob_sstable_meta.h" -#include "storage/blocksstable/ob_tmp_file_store.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "storage/slog_ckpt/ob_server_checkpoint_slog_handler.h" #include "storage/slog_ckpt/ob_tenant_checkpoint_slog_handler.h" #include "storage/meta_mem/ob_tenant_meta_mem_mgr.h" @@ -41,6 +41,7 @@ using namespace oceanbase::common; using namespace oceanbase::common::hash; using namespace oceanbase::blocksstable; +using namespace oceanbase::tmp_file; using namespace oceanbase::storage; using namespace oceanbase::share; @@ -1493,18 +1494,29 @@ int ObBlockManager::mark_tmp_file_blocks( ObMacroBlockMarkerStatus &tmp_status) { int ret = OB_SUCCESS; - ObArray macro_block_list; + omt::ObMultiTenant *omt = GCTX.omt_; + common::ObSEArray mtl_tenant_ids; - if (OB_FAIL(macro_block_list.reserve(DEFAULT_PENDING_FREE_COUNT))) { - LOG_WARN("fail to reserve macro block list", K(ret)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.get_macro_block_list(macro_block_list))) { - LOG_WARN("fail to get macro block list", K(ret)); - } else if (OB_FAIL(update_mark_info(macro_block_list, macro_id_set, mark_info))){ - LOG_WARN("fail to update mark info", K(ret), K(macro_block_list.count())); - } else { - tmp_status.tmp_file_count_ += macro_block_list.count(); - tmp_status.hold_count_ -= macro_block_list.count(); + omt->get_mtl_tenant_ids(mtl_tenant_ids); + for (int64_t i = 0; OB_SUCC(ret) && i < mtl_tenant_ids.count(); i++) { + const uint64_t tenant_id = mtl_tenant_ids.at(i); + MTL_SWITCH(tenant_id) { + ObArray macro_block_list; + if (OB_FAIL(set_group_id(tenant_id))) { + LOG_WARN("isolate CPU and IOPS failed", K(ret)); + } else if (OB_FAIL(mark_tenant_blocks(mark_info, macro_id_set, tmp_status))) { + LOG_WARN("fail to mark tenant blocks", K(ret), K(tenant_id)); + } else if (OB_FALSE_IT(MTL(ObTenantTmpFileManager*)->get_macro_block_list(macro_block_list))) { + LOG_WARN("fail to get macro block list", K(ret)); + } else if (OB_FAIL(update_mark_info(macro_block_list, macro_id_set, mark_info))){ + LOG_WARN("fail to update mark info", K(ret), K(macro_block_list.count())); + } else { + tmp_status.tmp_file_count_ += macro_block_list.count(); + tmp_status.hold_count_ -= macro_block_list.count(); + } + } } + return ret; } diff --git a/src/storage/blocksstable/ob_macro_block_handle.cpp b/src/storage/blocksstable/ob_macro_block_handle.cpp index 0edeefcc90..9f8e439a5a 100644 --- a/src/storage/blocksstable/ob_macro_block_handle.cpp +++ b/src/storage/blocksstable/ob_macro_block_handle.cpp @@ -221,18 +221,20 @@ int ObMacroBlockHandle::async_write(const ObMacroBlockWriteInfo &write_info) return ret; } -int ObMacroBlockHandle::wait() +int ObMacroBlockHandle::wait(const int64_t wait_timeout_ms) { int ret = OB_SUCCESS; if (io_handle_.is_empty()) { // do nothing - } else if (OB_FAIL(io_handle_.wait())) { - LOG_WARN("fail to wait block io, may be retry", K(macro_id_), K(ret)); - int tmp_ret = OB_SUCCESS; - if (OB_SUCCESS != (tmp_ret = report_bad_block())) { - LOG_WARN("fail to report bad block", K(tmp_ret), K(ret)); + } else if (OB_FAIL(io_handle_.wait(wait_timeout_ms))) { + if (OB_EAGAIN != ret) { + LOG_WARN("fail to wait block io, may be retry", K(macro_id_), K(ret)); + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = report_bad_block())) { + LOG_WARN("fail to report bad block", K(tmp_ret), K(ret)); + } + io_handle_.reset(); } - io_handle_.reset(); } return ret; } diff --git a/src/storage/blocksstable/ob_macro_block_handle.h b/src/storage/blocksstable/ob_macro_block_handle.h index 777d9d2fa2..82d438b81d 100644 --- a/src/storage/blocksstable/ob_macro_block_handle.h +++ b/src/storage/blocksstable/ob_macro_block_handle.h @@ -45,7 +45,7 @@ public: int async_read(const ObMacroBlockReadInfo &read_info); int async_write(const ObMacroBlockWriteInfo &write_info); int set_macro_block_id(const MacroBlockId ¯o_block_id); - int wait(); + int wait(const int64_t wait_timeout_ms=UINT64_MAX); TO_STRING_KV(K_(macro_id), K_(io_handle)); private: int report_bad_block() const; diff --git a/src/storage/blocksstable/ob_tmp_file.cpp b/src/storage/blocksstable/ob_tmp_file.cpp deleted file mode 100644 index b7dfd6c59c..0000000000 --- a/src/storage/blocksstable/ob_tmp_file.cpp +++ /dev/null @@ -1,1836 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#include "ob_tmp_file.h" -#include "ob_tmp_file_cache.h" -#include "observer/ob_server_struct.h" -#include "share/ob_task_define.h" - -namespace oceanbase -{ -using namespace storage; -using namespace share; - -namespace blocksstable -{ - -ObTmpFileIOInfo::ObTmpFileIOInfo() - : fd_(0), dir_id_(0), size_(0), io_timeout_ms_(DEFAULT_IO_WAIT_TIME_MS), - tenant_id_(OB_INVALID_TENANT_ID), buf_(NULL), io_desc_(), - disable_page_cache_(false) -{ -} - -ObTmpFileIOInfo::~ObTmpFileIOInfo() -{ -} - -void ObTmpFileIOInfo::reset() -{ - fd_ = 0; - dir_id_ = 0; - size_ = 0; - io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - tenant_id_ = OB_INVALID_TENANT_ID; - buf_ = NULL; -} - -bool ObTmpFileIOInfo::is_valid() const -{ - return fd_ >= 0 && dir_id_ >= 0 && size_ > 0 && OB_INVALID_TENANT_ID != tenant_id_ - && NULL != buf_ && io_desc_.is_valid() && io_timeout_ms_ > 0; -} - -ObTmpFileIOHandle::ObTmpFileIOHandle() - : io_handles_(), - page_cache_handles_(), - block_cache_handles_(), - write_block_ids_(), - fd_(OB_INVALID_FD), - dir_id_(OB_INVALID_ID), - tenant_id_(OB_INVALID_TENANT_ID), - buf_(NULL), - size_(0), - is_read_(false), - has_wait_(false), - is_finished_(false), - disable_page_cache_(false), - ret_code_(OB_SUCCESS), - expect_read_size_(0), - last_read_offset_(-1), - io_flag_(), - update_offset_in_file_(false), - last_fd_(OB_INVALID_FD), - last_extent_id_(0) -{ - io_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_IO_HDL")); - page_cache_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_PCACHE_HDL")); - block_cache_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_BCACHE_HDL")); -} - -ObTmpFileIOHandle::~ObTmpFileIOHandle() -{ - reset(); -} - -int ObTmpFileIOHandle::prepare_read( - const int64_t read_size, - const int64_t read_offset, - const common::ObIOFlag io_flag, - char *read_buf, - int64_t fd, - int64_t dir_id, - uint64_t tenant_id, - bool disable_page_cache) -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(read_buf)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), KP_(buf)); - } else { - buf_ = read_buf; - size_ = 0; - fd_ = fd; - dir_id_ = dir_id; - tenant_id_ = tenant_id; - is_read_ = true; - has_wait_ = false; - expect_read_size_ = read_size; - last_read_offset_ = read_offset; - io_flag_ = io_flag; - disable_page_cache_ = disable_page_cache; - if (last_fd_ != fd_) { - last_fd_ = fd_; - last_extent_id_ = 0; - } - } - return ret; -} - -int ObTmpFileIOHandle::prepare_write( - char *write_buf, - const int64_t write_size, - int64_t fd, - int64_t dir_id, - uint64_t tenant_id) -{ - int ret = OB_SUCCESS; - const int64_t bkt_cnt = 17; - lib::ObMemAttr bkt_mem_attr(tenant_id, "TmpBlkIDBkt"); - if (OB_ISNULL(write_buf)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), KP_(buf)); - } else if (OB_FAIL(write_block_ids_.create(bkt_cnt, bkt_mem_attr))) { - STORAGE_LOG(WARN, "create write block id set failed", K(ret), K(bkt_cnt)); - } else { - buf_ = write_buf; - size_ = write_size; - fd_ = fd; - dir_id_ = dir_id; - tenant_id_ = tenant_id; - is_read_ = false; - has_wait_ = false; - expect_read_size_ = 0; - last_read_offset_ = -1; - io_flag_.reset(); - } - return ret; -} - -int ObTmpFileIOHandle::wait() -{ - int ret = OB_SUCCESS; - const int64_t timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, DEFAULT_IO_WAIT_TIME_MS); - if (timeout_ms < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument. timeout must be positive", K(ret), K(timeout_ms)); - } else if (!is_finished_) { - if (is_read_ && OB_FAIL(wait_read_finish(timeout_ms))) { - STORAGE_LOG(WARN, "wait read finish failed", K(ret), K(timeout_ms), K(is_read_)); - } else if (!is_read_ && OB_FAIL(wait_write_finish(timeout_ms))) { - STORAGE_LOG(WARN, "wait write finish failed", K(ret), K(timeout_ms), K(is_read_)); - } - ret_code_ = ret; - is_finished_ = true; - } - - if (OB_SUCC(ret)) { - if (OB_FAIL(ret_code_)) { - STORAGE_LOG(WARN, "tmp file io error", K(ret), KPC(this)); - } - } - - return ret; -} - -int ObTmpFileIOHandle::wait_write_finish(int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - if (write_block_ids_.size() == 0) { - STORAGE_LOG(DEBUG, "write block ids size is 0", K(ret), K(timeout_ms)); - } else { - // iter all blocks, execute wait - common::hash::ObHashSet::const_iterator iter; - int64_t begin_us = ObTimeUtility::fast_current_time(); - int64_t wait_ms = timeout_ms; - for (iter = write_block_ids_.begin(); OB_SUCC(ret) && iter != write_block_ids_.end(); ++iter) { - const int64_t &blk_id = iter->first; - if (OB_FAIL(OB_TMP_FILE_STORE.wait_write_finish(tenant_id_, blk_id, wait_ms))) { - STORAGE_LOG(WARN, "fail to wait write finish", K(ret), K(blk_id), K(timeout_ms)); - } - wait_ms = timeout_ms - (ObTimeUtility::fast_current_time() - begin_us) / 1000; - if (OB_SUCC(ret) && OB_UNLIKELY(wait_ms <= 0)) { - ret = OB_TIMEOUT; - STORAGE_LOG(WARN, "fail to wait tmp file write finish", K(ret), K(wait_ms), K(blk_id), K(timeout_ms)); - } - } - int bret = OB_SUCCESS; - if (OB_UNLIKELY(OB_SUCCESS != (bret = write_block_ids_.destroy()))) { - STORAGE_LOG(WARN, "fail to destroy write block id set", K(bret), K(wait_ms), K(timeout_ms)); - } - } - return ret; -} - -int ObTmpFileIOHandle::wait_read_finish(const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (OB_UNLIKELY(has_wait_ && is_read_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "read wait() isn't reentrant interface, shouldn't call again", K(ret)); - } else if (OB_FAIL(do_read_wait(timeout_ms))) { - STORAGE_LOG(WARN, "fail to wait tmp file io", K(ret), K(timeout_ms)); - } else if (is_read_ && !has_wait_) { - if (size_ == expect_read_size_) { - //do nothing - } else if (OB_UNLIKELY(size_ > expect_read_size_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "read size more than expected size", K(ret), K(timeout_ms)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.get_tmp_file_handle(fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret)); - } else { - ObTmpFileIOInfo io_info; - io_info.fd_ = fd_; - io_info.dir_id_ = dir_id_; - io_info.tenant_id_ = tenant_id_; - io_info.size_ = expect_read_size_; - io_info.buf_ = buf_; - io_info.io_desc_ = io_flag_; - io_info.io_timeout_ms_ = timeout_ms; - while (OB_SUCC(ret) && size_ < expect_read_size_) { - if (OB_FAIL(file_handle.get_resource_ptr()->once_aio_read_batch(io_info, - update_offset_in_file_, - last_read_offset_, - *this))) { - STORAGE_LOG(WARN, "fail to read once batch", K(ret), K(timeout_ms), K(io_info), K(*this)); - } else if (OB_FAIL(do_read_wait(timeout_ms))) { - STORAGE_LOG(WARN, "fail to wait tmp file io", K(ret), K(timeout_ms)); - } - } - } - } - - if (OB_SUCC(ret) || OB_ITER_END == ret) { - has_wait_ = true; - expect_read_size_ = 0; - last_read_offset_ = -1; - io_flag_.reset(); - update_offset_in_file_ = false; - } - return ret; -} - -int ObTmpFileIOHandle::do_read_wait(const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - for (int32_t i = 0; OB_SUCC(ret) && i < block_cache_handles_.count(); i++) { - ObBlockCacheHandle &tmp = block_cache_handles_.at(i); - MEMCPY(tmp.buf_, tmp.block_handle_.value_->get_buffer() + tmp.offset_, tmp.size_); - tmp.block_handle_.reset(); - } - if (0 != block_cache_handles_.count()) { - OB_TMP_FILE_STORE.dec_block_cache_num(tenant_id_, block_cache_handles_.count()); - } - block_cache_handles_.reset(); - - for (int32_t i = 0; OB_SUCC(ret) && i < page_cache_handles_.count(); i++) { - ObPageCacheHandle &tmp = page_cache_handles_.at(i); - MEMCPY(tmp.buf_, tmp.page_handle_.value_->get_buffer() + tmp.offset_, tmp.size_); - tmp.page_handle_.reset(); - } - if (0 != page_cache_handles_.count()) { - OB_TMP_FILE_STORE.dec_page_cache_num(tenant_id_, page_cache_handles_.count()); - } - page_cache_handles_.reset(); - - for (int32_t i = 0; OB_SUCC(ret) && i < io_handles_.count(); i++) { - ObIOReadHandle &tmp = io_handles_.at(i); - if (OB_FAIL(tmp.macro_handle_.wait())) { - STORAGE_LOG(WARN, "fail to wait tmp read io", K(ret)); - } else { - MEMCPY(tmp.buf_, tmp.macro_handle_.get_buffer() + tmp.offset_, tmp.size_); - tmp.macro_handle_.reset(); - } - } - io_handles_.reset(); - return ret; -} - -void ObTmpFileIOHandle::reset() -{ - for (int32_t i = 0; i < io_handles_.count(); i++) { - io_handles_.at(i).macro_handle_.reset(); - } - for (int32_t i = 0; i < block_cache_handles_.count(); i++) { - block_cache_handles_.at(i).block_handle_.reset(); - } - if (0 != block_cache_handles_.count()) { - OB_TMP_FILE_STORE.dec_block_cache_num(tenant_id_, block_cache_handles_.count()); - } - for (int32_t i = 0; i < page_cache_handles_.count(); i++) { - page_cache_handles_.at(i).page_handle_.reset(); - } - if (0 != page_cache_handles_.count()) { - OB_TMP_FILE_STORE.dec_page_cache_num(tenant_id_, page_cache_handles_.count()); - } - io_handles_.reset(); - page_cache_handles_.reset(); - block_cache_handles_.reset(); - write_block_ids_.destroy(); - fd_ = OB_INVALID_FD; - dir_id_ = OB_INVALID_ID; - tenant_id_ = OB_INVALID_TENANT_ID; - buf_ = NULL; - size_ = 0; - is_read_ = false; - has_wait_ = false; - expect_read_size_ = 0; - last_read_offset_ = -1; - io_flag_.reset(); - update_offset_in_file_ = false; - is_finished_ = false; - ret_code_ = OB_SUCCESS; -} - -bool ObTmpFileIOHandle::is_valid() const -{ - return OB_INVALID_FD != fd_ && OB_INVALID_ID != dir_id_ && OB_INVALID_TENANT_ID != tenant_id_ - && NULL != buf_ && size_ >= 0; -} - -int ObTmpFileIOHandle::record_block_id(const int64_t block_id) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(write_block_ids_.set_refactored(block_id, 1))) { - STORAGE_LOG(WARN, "record block id failed", K(ret), K(block_id)); - } - return ret; -} - -void ObTmpFileIOHandle::set_last_extent_id(const int64_t last_extent_id) -{ - last_extent_id_ = last_extent_id; -} - -int64_t ObTmpFileIOHandle::get_last_extent_id() const -{ - return last_extent_id_; -} - -ObTmpFileIOHandle::ObIOReadHandle::ObIOReadHandle() - : macro_handle_(), buf_(NULL), offset_(0), size_(0) -{ -} - -ObTmpFileIOHandle::ObIOReadHandle::ObIOReadHandle(const ObMacroBlockHandle ¯o_handle, - char *buf, const int64_t offset, const int64_t size) - : macro_handle_(macro_handle), buf_(buf), offset_(offset), size_(size) -{ -} - -ObTmpFileIOHandle::ObIOReadHandle::~ObIOReadHandle() -{ -} - -ObTmpFileIOHandle::ObIOReadHandle::ObIOReadHandle(const ObTmpFileIOHandle::ObIOReadHandle &other) - : macro_handle_(), buf_(NULL), offset_(0), size_(0) -{ - *this = other; -} - -ObTmpFileIOHandle::ObIOReadHandle &ObTmpFileIOHandle::ObIOReadHandle::operator=( - const ObTmpFileIOHandle::ObIOReadHandle &other) -{ - if (&other != this) { - macro_handle_ = other.macro_handle_; - offset_ = other.offset_; - buf_ = other.buf_; - size_ = other.size_; - } - return *this; -} -ObTmpFileIOHandle::ObPageCacheHandle::ObPageCacheHandle() - : page_handle_(), buf_(NULL), offset_(0), size_(0) -{ -} - -ObTmpFileIOHandle::ObPageCacheHandle::ObPageCacheHandle(const ObTmpPageValueHandle &page_handle, - char *buf, const int64_t offset, const int64_t size) - : page_handle_(page_handle), buf_(buf), offset_(offset), size_(size) -{ -} - -ObTmpFileIOHandle::ObPageCacheHandle::~ObPageCacheHandle() -{ -} - -ObTmpFileIOHandle::ObPageCacheHandle::ObPageCacheHandle( - const ObTmpFileIOHandle::ObPageCacheHandle &other) - : page_handle_(), buf_(NULL), offset_(0), size_(0) -{ - *this = other; -} - -ObTmpFileIOHandle::ObPageCacheHandle &ObTmpFileIOHandle::ObPageCacheHandle::operator=( - const ObTmpFileIOHandle::ObPageCacheHandle &other) -{ - if (&other != this) { - page_handle_ = other.page_handle_; - offset_ = other.offset_; - buf_ = other.buf_; - size_ = other.size_; - } - return *this; -} - -ObTmpFileIOHandle::ObBlockCacheHandle::ObBlockCacheHandle() - : block_handle_(), buf_(NULL), offset_(0), size_(0) -{ -} - -ObTmpFileIOHandle::ObBlockCacheHandle::ObBlockCacheHandle(const ObTmpBlockValueHandle &block_handle, - char *buf, const int64_t offset, const int64_t size) - : block_handle_(block_handle), buf_(buf), offset_(offset), size_(size) -{ -} - -ObTmpFileIOHandle::ObBlockCacheHandle::~ObBlockCacheHandle() -{ -} - -ObTmpFileIOHandle::ObBlockCacheHandle::ObBlockCacheHandle( - const ObTmpFileIOHandle::ObBlockCacheHandle &other) - : block_handle_(), buf_(NULL), offset_(0), size_(0) -{ - *this = other; -} - -ObTmpFileIOHandle::ObBlockCacheHandle &ObTmpFileIOHandle::ObBlockCacheHandle::operator=( - const ObTmpFileIOHandle::ObBlockCacheHandle &other) -{ - if (&other != this) { - block_handle_ = other.block_handle_; - offset_ = other.offset_; - buf_ = other.buf_; - size_ = other.size_; - } - return *this; -} - -void ObTmpFileExtent::set_global_offset(const int64_t g_offset_start, const int64_t g_offset_end) -{ - g_offset_start_ = g_offset_start; - g_offset_end_ = g_offset_end; -} - -void ObTmpFileExtent::get_global_offset(int64_t &g_offset_start, int64_t &g_offset_end) const -{ - g_offset_start = g_offset_start_; - g_offset_end = g_offset_end_; -} - -ObTmpFileExtent::ObTmpFileExtent(ObTmpFile *file) - : is_alloced_(false), - is_closed_(false), - start_page_id_(-1), - page_nums_(0), - offset_(0), - fd_(file->get_fd()), - g_offset_start_(0), - g_offset_end_(0), - owner_(file), - block_id_(-1), - lock_(common::ObLatchIds::TMP_FILE_EXTENT_LOCK), - is_truncated_(false) -{ -} - -ObTmpFileExtent::~ObTmpFileExtent() -{ -} - -int ObTmpFileExtent::read(const ObTmpFileIOInfo &io_info, const int64_t offset, const int64_t size, - char *buf, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!is_alloced_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret)); - } else if (OB_UNLIKELY(offset < 0 || offset >= get_offset() || size <= 0 - || offset + size > get_offset()) || OB_ISNULL(buf)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(offset), K(get_offset()), K(size), K(buf)); - } else { - ObTmpBlockIOInfo info; - info.buf_ = buf; - info.io_desc_ = io_info.io_desc_; - info.block_id_ = block_id_; - info.offset_ = start_page_id_ * ObTmpMacroBlock::get_default_page_size() + offset; - info.size_ = size; - info.tenant_id_ = io_info.tenant_id_; - info.io_timeout_ms_ = io_info.io_timeout_ms_; - if (OB_FAIL(OB_TMP_FILE_STORE.read(owner_->get_tenant_id(), info, handle))) { - STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(info), K(*this)); - } else { - STORAGE_LOG(DEBUG, "debug tmp file: read extent", K(ret), K(info), K(*this)); - } - } - return ret; -} - -int ObTmpFileExtent::write(const ObTmpFileIOInfo &io_info,int64_t &size, char *&buf) -{ - int ret = OB_SUCCESS; - int write_size = 0; - int64_t remain = 0; - bool is_write = false; - bool need_close = false; - if (OB_UNLIKELY(size <= 0) || OB_ISNULL(buf)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret)); - } else if (OB_UNLIKELY(!is_alloced_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "ObTmpFileExtent has not been allocated", K(ret)); - } else if (get_offset() == page_nums_ * ObTmpMacroBlock::get_default_page_size()) { - need_close = true; - } else { - SpinWLockGuard guard(lock_); - if (!is_closed()) { - remain = page_nums_ * ObTmpMacroBlock::get_default_page_size() - get_offset(); - write_size = std::min(remain, size); - ObTmpBlockIOInfo info; - info.block_id_ = block_id_; - info.buf_ = buf; - info.io_desc_ = io_info.io_desc_; - info.offset_ = start_page_id_ * ObTmpMacroBlock::get_default_page_size() + get_offset(); - info.size_ = write_size; - info.tenant_id_ = io_info.tenant_id_; - info.io_timeout_ms_ = io_info.io_timeout_ms_; - if (OB_FAIL(OB_TMP_FILE_STORE.write(owner_->get_tenant_id(), info))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret)); - } else { - ATOMIC_FAA(&offset_, write_size); - g_offset_end_ = get_offset() + g_offset_start_; - buf += write_size; - size -= write_size; - if (remain == write_size) { - need_close = true; - } - STORAGE_LOG(DEBUG, "debug tmp file: write extent", K(ret), K(info), K(*this)); - } - } - } - if (need_close) { - close(); - try_sync_block(); - } - return ret; -} - -void ObTmpFileExtent::reset() -{ - is_alloced_ = false; - fd_ = -1; - g_offset_start_ = 0; - g_offset_end_ = 0; - ATOMIC_SET(&offset_, 0); - owner_ = NULL; - start_page_id_ = -1; - page_nums_ = 0; - block_id_ = -1; - ATOMIC_STORE(&is_closed_, false); - ATOMIC_STORE(&is_truncated_, false); -} - -bool ObTmpFileExtent::is_valid() -{ - return start_page_id_ >= 0 && page_nums_ >= 0 && block_id_ > 0; -} - -bool ObTmpFileExtent::close(bool force) -{ - int ret = OB_SUCCESS; - uint8_t page_start_id = ObTmpFilePageBuddy::MAX_PAGE_NUMS; - uint8_t page_nums = 0; - if (!is_closed_) { - if (close(page_start_id, page_nums, force)) { - if (ObTmpFilePageBuddy::MAX_PAGE_NUMS == page_start_id && 0 == page_nums) { - //nothing to do - } else if (OB_UNLIKELY(page_start_id > ObTmpFilePageBuddy::MAX_PAGE_NUMS - 1 - || page_nums > ObTmpFilePageBuddy::MAX_PAGE_NUMS)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "fail to close the extent", K(ret), K_(block_id), K(page_start_id), - K(page_nums), K_(offset)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.free(owner_->get_tenant_id(), block_id_, page_start_id, - page_nums))) { - STORAGE_LOG(WARN, "fail to free", K(ret)); - } - if (OB_FAIL(ret)) { - unclose(page_nums); - } - } - } - return is_closed(); -} - -bool ObTmpFileExtent::close(uint8_t &free_page_start_id, uint8_t &free_page_nums, bool force) -{ - free_page_start_id = ObTmpFilePageBuddy::MAX_PAGE_NUMS; - free_page_nums = 0; - SpinWLockGuard guard(lock_); - if (!is_closed()) { - if (!force && 0 != page_nums_ && 0 == get_offset()) { - // Nothing to do. This extent is alloced just now, so it cannot be closed. - } else { - if (get_offset() != page_nums_ * 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(); - free_page_nums = page_nums_ - offset_page_id; - free_page_start_id = start_page_id_ + offset_page_id; - page_nums_ -= free_page_nums; - } - ATOMIC_STORE(&is_closed_, true); - } - } - return is_closed(); -} - -int ObTmpFileExtent::try_sync_block() -{ - int ret = OB_SUCCESS; - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle handle; - ObTmpMacroBlock *blk = NULL; - if (OB_FAIL(OB_TMP_FILE_STORE.get_macro_block(owner_->get_tenant_id(), block_id_, blk))) { - STORAGE_LOG(WARN, "fail to get macro block", K(ret), K(owner_->get_tenant_id()),K(block_id_)); - } else if (OB_ISNULL(blk)) { - ret = OB_ERR_NULL_VALUE; - } else if (0 != blk->get_free_page_nums()) { - STORAGE_LOG(DEBUG, "ob tmp macro block has not been used up", K(ret), K(blk->get_free_page_nums()), K(owner_->get_tenant_id()),K(block_id_)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.wash_block(owner_->get_tenant_id(), block_id_, handle))) { - // try to flush the block to the disk. If fails, do nothing. - STORAGE_LOG(DEBUG, "fail to sync block", K(ret), K(owner_->get_tenant_id()), K(block_id_)); - } else { - STORAGE_LOG(DEBUG, "succeed to sync wash block", K(block_id_)); - } - - return ret; -} - -void ObTmpFileExtent::unclose(const int32_t page_nums) -{ - SpinWLockGuard guard(lock_); - if (page_nums >= 0) { - page_nums_ += page_nums; - } - ATOMIC_STORE(&is_closed_, false); -} - -ObTmpFileMeta::~ObTmpFileMeta() -{ - clear(); -} - -int ObTmpFileMeta::init(const int64_t fd, const int64_t dir_id, common::ObIAllocator *allocator) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(fd < 0 || dir_id < 0) || OB_ISNULL(allocator)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(fd), K(dir_id)); - } else { - fd_ = fd; - dir_id_ = dir_id; - allocator_ = allocator; - } - return ret; -} - -ObTmpFileExtent *ObTmpFileMeta::get_last_extent() -{ - return extents_.count() > 0 ? extents_.at(extents_.count() - 1) : NULL; -} - - -int ObTmpFileMeta::deep_copy(const ObTmpFileMeta &other) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(extents_.assign(other.extents_))) { - STORAGE_LOG(WARN, "fail to assign extents array", K(ret)); - } else { - fd_ = other.fd_; - dir_id_ = other.dir_id_; - allocator_ = other.allocator_; - } - return ret; -} - -int ObTmpFileMeta::clear() -{ - //free extents - int ret = OB_SUCCESS; - ObTmpFileExtent *tmp = NULL; - for (int64_t i = extents_.count() - 1; OB_SUCC(ret) && i >= 0; --i) { - tmp = extents_.at(i); - if (NULL != tmp) { - if (!tmp->is_alloced() || tmp->is_truncated()) { - // nothing to do. - } else if (OB_FAIL(OB_TMP_FILE_STORE.free(tmp->get_owner().get_tenant_id(), tmp))) { - STORAGE_LOG(WARN, "fail to free extents", K(ret)); - } - if (OB_SUCC(ret)) { - tmp->~ObTmpFileExtent(); - allocator_->free(tmp); - extents_.at(i) = NULL; - } - } - } - if (OB_SUCC(ret)) { - extents_.reset(); - allocator_ = NULL; - fd_ = -1; - dir_id_ = -1; - } - return ret; -} - -ObTmpFile::ObTmpFile() - : is_inited_(false), - is_big_(false), - offset_(0), - tenant_id_(-1), - lock_(common::ObLatchIds::TMP_FILE_LOCK), - allocator_(NULL), - file_meta_(), - read_guard_(0), - next_truncated_extent_id_(0) -{ -} - -ObTmpFile::~ObTmpFile() -{ - clear(); -} - -int ObTmpFile::clear() -{ - int ret = OB_SUCCESS; - if (OB_FAIL(file_meta_.clear())) { - STORAGE_LOG(WARN, "fail to clear file meta", K(ret)); - } else { - if (is_inited_) { - is_big_ = false; - tenant_id_ = -1; - offset_ = 0; - allocator_ = NULL; - is_inited_ = false; - read_guard_ = 0; - next_truncated_extent_id_ = 0; - } - } - return ret; -} - -int ObTmpFile::init(const int64_t fd, const int64_t dir_id, common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else if (fd < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(fd)); - } else if (OB_FAIL(file_meta_.init(fd, dir_id, &allocator))) { - STORAGE_LOG(WARN, "fail to init file meta", K(ret)); - } else { - allocator_ = &allocator; - is_inited_ = true; - } - - if (!is_inited_) { - clear(); - } - return ret; -} - -int64_t ObTmpFile::find_first_extent(const int64_t offset) -{ - common::ObIArray &extents = file_meta_.get_extents(); - int64_t first_extent = 0; - int64_t left = 0; - int64_t right = extents.count() - 1; - ObTmpFileExtent *tmp = nullptr; - while(left < right) { - int64_t mid = (left + right) / 2; - tmp = extents.at(mid); - if (tmp->get_global_start() <= offset && offset < tmp->get_global_end()) { - first_extent = mid; - break; - } else if(tmp->get_global_start() > offset) { - right = mid - 1; - } else if(tmp->get_global_end() <= offset) { - left = mid + 1; - } - } - return first_extent; -} - -int ObTmpFile::aio_read_without_lock(const ObTmpFileIOInfo &io_info, - int64_t &offset, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileExtent *tmp = nullptr; - if (OB_ISNULL(tmp = file_meta_.get_last_extent())) { - ret = OB_BAD_NULL_ERROR; - STORAGE_LOG(WARN, "fail to read, because the tmp file is empty", K(ret), KP(tmp), K(io_info)); - } else if (OB_FAIL(handle.prepare_read(io_info.size_, - offset, - io_info.io_desc_, - io_info.buf_, - file_meta_.get_fd(), - file_meta_.get_dir_id(), - io_info.tenant_id_, - io_info.disable_page_cache_))) { - STORAGE_LOG(WARN, "fail to prepare read io handle", K(ret), K(io_info), K(offset)); - } else if (OB_UNLIKELY(io_info.size_ > 0 && offset >= tmp->get_global_end())) { - ret = OB_ITER_END; - } else if (OB_FAIL(once_aio_read_batch_without_lock(io_info, offset, handle))) { - STORAGE_LOG(WARN, "fail to read one batch", K(ret), K(offset), K(handle)); - } else { - handle.set_last_read_offset(offset); - } - return ret; -} - -int ObTmpFile::once_aio_read_batch( - const ObTmpFileIOInfo &io_info, - const bool need_update_offset, - int64_t &offset, - ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileExtent *tmp = nullptr; - const int64_t remain_size = io_info.size_ - handle.get_data_size(); - - { - SpinRLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been initialized", K(ret)); - } else if (OB_UNLIKELY(offset < 0 || remain_size < 0) || OB_ISNULL(io_info.buf_)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(offset), K(remain_size), KP(io_info.buf_)); - } else if (OB_ISNULL(tmp = file_meta_.get_last_extent())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, null tmp file extent", K(ret), KP(tmp), K(io_info)); - } else if (OB_UNLIKELY(remain_size > 0 && offset >= tmp->get_global_end())) { - ret = OB_ITER_END; - } else if (OB_FAIL(once_aio_read_batch_without_lock(io_info, offset, handle))) { - STORAGE_LOG(WARN, "fail to read one batch", K(ret), K(offset), K(handle)); - } else { - handle.set_last_read_offset(offset); - } - } - - if (need_update_offset) { - SpinWLockGuard guard(lock_); - offset_ = offset; - } - return ret; -} - -int ObTmpFile::fill_zero(char *buf, const int64_t size) -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || size < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "buf is null or size is negative", K(ret), K(size), KP(buf)); - } else { - MEMSET(buf, 0, size); - } - return ret; -} - -int ObTmpFile::once_aio_read_batch_without_lock( - const ObTmpFileIOInfo &io_info, - int64_t &offset, - ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - int64_t one_batch_read_size = 0; - char *buf = io_info.buf_ + handle.get_data_size(); - int64_t remain_size = io_info.size_ - handle.get_data_size(); - int64_t read_size = 0; - ObTmpFileExtent *tmp = nullptr; - common::ObIArray &extents = file_meta_.get_extents(); - int64_t ith_extent = get_extent_cache(offset, handle); - - while (OB_SUCC(ret) - && ith_extent < extents.count() - && remain_size > 0 - && one_batch_read_size < READ_SIZE_PER_BATCH) { - tmp = extents.at(ith_extent); - if (tmp->get_global_start() <= offset && offset < tmp->get_global_end()) { - if (offset + remain_size > tmp->get_global_end()) { - read_size = tmp->get_global_end() - offset; - } else { - read_size = remain_size; - } - // read from the extent. - if (tmp->is_truncated()) { - if (read_guard_ < tmp->get_global_end()) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "extent is truncated but read_guard not set correctlly", K(ret), K(tmp), K(read_guard_)); - } else if (OB_FAIL(fill_zero(buf, read_size))) { - STORAGE_LOG(WARN, "fail to fill zero data to buf", K(ret)); - } - } else if (offset >= read_guard_) { - if (OB_FAIL(tmp->read(io_info, offset - tmp->get_global_start(), read_size, buf, handle))) { - STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(io_info), K(buf), KP_(io_info.buf)); - } - } else { - if (read_guard_ < offset + read_size) { - const int64_t zero_size = read_guard_ - offset; - const int64_t file_read_size = read_size - zero_size; - if (OB_FAIL(fill_zero(buf, zero_size))) { - STORAGE_LOG(WARN, "fail to read zero from truncated pos", K(ret)); - } else if (OB_FAIL(tmp->read(io_info, read_guard_ - tmp->get_global_start(), file_read_size, buf + zero_size, handle))) { - STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(io_info), K(buf + zero_size), KP_(io_info.buf)); - } - } else { - // read 0; - if (OB_FAIL(fill_zero(buf, read_size))) { - STORAGE_LOG(WARN, "fail to read zero from truncated pos", K(ret), KP(buf), K(read_size)); - } - } - - } - if (OB_SUCC(ret)) { - offset += read_size; - remain_size -= read_size; - buf += read_size; - one_batch_read_size += read_size; - handle.add_data_size(read_size); - } - } - ++ith_extent; - } - - if (OB_SUCC(ret) && OB_LIKELY(ith_extent > 0)) { - handle.set_last_extent_id(ith_extent - 1); - } - return ret; -} - -int64_t ObTmpFile::get_extent_cache(const int64_t offset, const ObTmpFileIOHandle &handle) -{ - common::ObIArray &extents = file_meta_.get_extents(); - int64_t ith_extent = -1; - int64_t last_extent_id = handle.get_last_extent_id(); - if (OB_UNLIKELY(last_extent_id < 0 || last_extent_id >= extents.count() - 1)) { - ith_extent = find_first_extent(offset); - } else if (OB_LIKELY(extents.at(last_extent_id)->get_global_start() <= offset - && offset < extents.at(last_extent_id)->get_global_end())) { - ith_extent = last_extent_id; - } else if (offset == extents.at(last_extent_id)->get_global_end() - && last_extent_id != extents.count() - 1) { - ith_extent = last_extent_id + 1; - } else { - ith_extent = find_first_extent(offset); - } - - return ith_extent; -} - -int ObTmpFile::aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else { - tenant_id_ = io_info.tenant_id_; - SpinWLockGuard guard(lock_); - if (OB_FAIL(aio_read_without_lock(io_info, offset_, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to do aio read without lock", K(ret)); - } - } else { - handle.set_update_offset_in_file(); - } - } - return ret; -} -int ObTmpFile::aio_pread(const ObTmpFileIOInfo &io_info, const int64_t offset, - ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else { - int64_t tmp_offset = offset; - SpinRLockGuard guard(lock_); - if (OB_FAIL(aio_read_without_lock(io_info, tmp_offset, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to do aio read without lock", K(ret)); - } - } - } - return ret; -} - -int ObTmpFile::seek(const int64_t offset, const int whence) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else { - switch (whence) { - case SET_SEEK: - offset_ = offset; - break; - case CUR_SEEK: - offset_ += offset; - break; - default: - ret = OB_NOT_SUPPORTED; - STORAGE_LOG(WARN, "not supported whence", K(ret), K(whence)); - } - } - return ret; -} - -int ObTmpFile::read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(aio_read(io_info, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to read data using asynchronous io", K(ret), K(io_info)); - } else { - if (OB_FAIL(handle.wait())) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(io_info)); - } else { - ret = OB_ITER_END; - } - } - } else if (OB_FAIL(handle.wait())) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(io_info)); - } - } - return ret; -} - -int ObTmpFile::pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else if (OB_FAIL(aio_pread(io_info, offset, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to read data using asynchronous io", K(ret), K(io_info)); - } else { - if (OB_FAIL(handle.wait())) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(io_info)); - } else { - ret = OB_ITER_END; - } - } - } else if (OB_FAIL(handle.wait())) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret), K(io_info)); - } - } - return ret; -} - -int ObTmpFile::aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - // only support append at present. - int ret = OB_SUCCESS; - DISABLE_SQL_MEMLEAK_GUARD; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info), K(handle)); - } else if (OB_FAIL(handle.prepare_write(io_info.buf_, - io_info.size_, - file_meta_.get_fd(), - file_meta_.get_dir_id(), - io_info.tenant_id_))) { - STORAGE_LOG(WARN, "fail to prepare write io handle", K(ret)); - } else { - tenant_id_ = io_info.tenant_id_; - int64_t size = io_info.size_; - char *buf = io_info.buf_; - SpinWLockGuard guard(lock_); - ObTmpFileExtent *tmp = file_meta_.get_last_extent(); - void *buff = NULL; - ObTmpFileExtent *extent = NULL; - while (OB_SUCC(ret) && size > 0) { - if (NULL != tmp && tmp->is_alloced() && !tmp->is_closed()) { - if (OB_FAIL(write_file_extent(io_info, tmp, size, buf))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret)); - } - } else if (NULL == tmp || (tmp->is_alloced() && tmp->is_closed())) { - int64_t alloc_size = 0; - // Pre-Allocation Strategy: - // 1. small file - // a. 0KB < size <= 32KB, alloc_size = 32KB; - // b. 32KB < size <= 64KB, alloc_size = 64KB; - // c. 64KB < size , alloc_size = size; - // 2. big file - // a. 32KB < size <= 64KB, alloc_size = 64KB; - // b. 64KB < size , alloc_size = size; - // - // NOTE: if the size is more than block size, it will be split into - // multiple allocation. - if (size <= big_file_prealloc_size()) { - if (!is_big_ && size <= small_file_prealloc_size()) { - alloc_size = common::upper_align(size, small_file_prealloc_size()); - } else { - alloc_size = common::upper_align(size, big_file_prealloc_size()); - } - } else if (size > ObTmpFileStore::get_block_size()) { - alloc_size = ObTmpFileStore::get_block_size(); - } else { - alloc_size = size; - } - lib::ObMemAttr attr(tenant_id_, "TmpFileExtent"); - if (OB_ISNULL(buff = allocator_->alloc(sizeof(ObTmpFileExtent), attr))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "fail to alloc a buf", K(ret)); - } else if (OB_ISNULL(extent = new (buff) ObTmpFileExtent(this))) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to new a ObTmpFileExtent", K(ret)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.alloc(get_dir_id(), tenant_id_, alloc_size, *extent))) { - STORAGE_LOG(WARN, "fail to allocate extents", K(ret), K_(tenant_id)); - } else { - if (NULL != tmp) { - extent->set_global_offset(tmp->get_global_end(), tmp->get_global_end()); - } - tmp = extent; - if (OB_FAIL(write_file_extent(io_info, tmp, size, buf))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret)); - } else if (OB_FAIL(file_meta_.push_back_extent(tmp))) { - STORAGE_LOG(WARN, "fail to push the extent", K(ret)); - } else { - extent = NULL; - buff = NULL; - } - } - // If it fails, the unused extent should be returned to the macro block and cleaned up. - if (OB_FAIL(ret)) { - if (OB_NOT_NULL(extent)) { - if (extent->is_valid()) { - OB_TMP_FILE_STORE.free(tenant_id_, extent); - extent->~ObTmpFileExtent(); - } - allocator_->free(extent); - extent = NULL; - } - if (OB_NOT_NULL(tmp)) { - tmp = NULL; - } - } - } else { - // this extent has allocated, no one page, invalid extent(tmp->is_alloced = false). - ret = OB_INVALID_ERROR; - STORAGE_LOG(WARN, "invalid extent", K(ret), KPC(tmp), KPC(this)); - } - if (OB_SUCC(ret)) { - if (OB_ISNULL(tmp)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(ERROR, "invalid tmp", K(ret)); - } else if (OB_FAIL(handle.record_block_id(tmp->get_block_id()))) { - if (OB_HASH_EXIST == ret) { - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "set block id failed", K(ret)); - } - } - } - - if (OB_EAGAIN == ret) { - ret = OB_SUCCESS; - } - } - handle.sub_data_size(io_info.size_ - size); - if (OB_SUCC(ret) && !is_big_){ - is_big_ = tmp->get_global_end() >= - SMALL_FILE_MAX_THRESHOLD * ObTmpMacroBlock::get_default_page_size(); - } - } - return ret; -} - -int64_t ObTmpFile::get_dir_id() const -{ - return file_meta_.get_dir_id(); -} - -uint64_t ObTmpFile::get_tenant_id() const -{ - return tenant_id_; -} - -int64_t ObTmpFile::get_fd() const -{ - return file_meta_.get_fd(); -} - -int ObTmpFile::sync(const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - ObTmpFileExtent *tmp = file_meta_.get_last_extent(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFile has not been inited", K(ret)); - } else if (OB_ISNULL(tmp)) { - ret = OB_BAD_NULL_ERROR; - STORAGE_LOG(WARN, "the file does not have a extent", K(ret), K(timeout_ms)); - } else { - tmp->close(true/*force*/); - // all extents has been closed. - const ObIArray &extents = file_meta_.get_extents(); - common::hash::ObHashSet blk_id_set; - lib::ObMemAttr attr(tenant_id_, "TmpBlkIDSet"); - if (OB_FAIL(blk_id_set.create(min(extents.count(), 1024 * 1024), attr))){ - STORAGE_LOG(WARN, "create block id set failed", K(ret), K(timeout_ms)); - } else { - // get extents block id set. - for (int64_t i=0; OB_SUCC(ret) && i < extents.count(); ++i) { - const ObTmpFileExtent* e = extents.at(i); - const int64_t &blk_id = e->get_block_id(); - if (OB_FAIL(blk_id_set.set_refactored(blk_id))) { - STORAGE_LOG(WARN, "add block id to set failed", K(ret), K(blk_id)); - } - } - // iter all blocks, execute async wash. - common::hash::ObHashSet::const_iterator iter; - common::ObSEArray handles; - handles.set_attr(ObMemAttr(MTL_ID(), "TMP_SYNC_HDL")); - for (iter = blk_id_set.begin(); OB_SUCC(ret) && iter != blk_id_set.end(); ++iter) { - const int64_t &blk_id = iter->first; - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle handle; - if (OB_FAIL(OB_TMP_FILE_STORE.sync_block(tenant_id_, blk_id, handle))) { - // OB_HASH_NOT_EXIST: - // if multiple file sync same block, the block may be not exist in hash map. - // OB_STATE_NOT_MATCH: - // the extents in block may be not all close and shouldn't sync it now. - if (OB_HASH_NOT_EXIST == ret || OB_STATE_NOT_MATCH == ret) { - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "sync block failed", K(ret), K(blk_id)); - } - } else if (OB_NOT_NULL(handle.get_wait_info()) && OB_FAIL(handles.push_back(handle))) { - STORAGE_LOG(WARN, "push back wait handle to array failed", K(ret), K(blk_id)); - } - } - - int64_t begin_us = ObTimeUtility::fast_current_time(); - int64_t wait_ms = timeout_ms; - for (int64_t i=0; OB_SUCC(ret) && i < handles.count(); ++i) { - const ObTmpTenantMemBlockManager::ObIOWaitInfoHandle handle = handles.at(i); - if (OB_FAIL(handle.get_wait_info()->wait(timeout_ms))) { - STORAGE_LOG(WARN, "add block id to set failed", K(ret), K(timeout_ms)); - } else { - wait_ms = timeout_ms - (ObTimeUtility::fast_current_time() - begin_us) / 1000; - } - - if (OB_SUCC(ret) && OB_UNLIKELY(wait_ms <= 0)) { // rarely happen - ret = OB_TIMEOUT; - STORAGE_LOG(WARN, "fail to wait tmp file sync finish", K(ret), K(wait_ms)); - } - } - } - } - return ret; -} - -int ObTmpFile::deep_copy(char *buf, const int64_t buf_len, ObTmpFile *&value) const -{ - int ret = OB_SUCCESS; - const int64_t deep_copy_size = get_deep_copy_size(); - if (OB_ISNULL(buf) || buf_len < deep_copy_size) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), KP(buf), K(buf_len)); - } else { - ObTmpFile *pvalue = new (buf) ObTmpFile(); - if (OB_FAIL(pvalue->file_meta_.deep_copy(file_meta_))) { - STORAGE_LOG(WARN, "fail to deep copy meta", K(ret)); - } else { - pvalue->is_big_ = is_big_; - pvalue->offset_ = offset_; - pvalue->tenant_id_ = tenant_id_; - pvalue->allocator_ = allocator_; - pvalue->is_inited_ = is_inited_; - value = pvalue; - } - } - return ret; -} - -void ObTmpFile::get_file_size(int64_t &file_size) -{ - ObTmpFileExtent *tmp = file_meta_.get_last_extent(); - file_size = (nullptr == tmp) ? 0 : tmp->get_global_end(); -} - -/* - * to avoid truncating blocks that is using now (e.g., the io request is in io manager but not finish). - * we need to ensure there is no other file operation while calling truncate. - */ -int ObTmpFile::truncate(const int64_t offset) -{ - int ret = OB_SUCCESS; - - SpinWLockGuard guard(lock_); - // release extents - ObTmpFileExtent *tmp = nullptr; - //the extents before read_guard_ is truncated; - int64_t ith_extent = next_truncated_extent_id_; - common::ObIArray &extents = file_meta_.get_extents(); - STORAGE_LOG(INFO, "truncate ", K(offset), K(read_guard_), K(ith_extent)); - - if (OB_ISNULL(tmp = file_meta_.get_last_extent())) { - ret = OB_BAD_NULL_ERROR; - STORAGE_LOG(WARN, "fail to truncate, because the tmp file is empty", K(ret), KP(tmp)); - } else if (offset < 0 || offset > tmp->get_global_end()) { - ret = OB_INDEX_OUT_OF_RANGE; - STORAGE_LOG(WARN, "offset out of range", K(ret), K(tmp), K(offset)); - } - - while (OB_SUCC(ret) && ith_extent >= 0 - && ith_extent < extents.count()) { - tmp = extents.at(ith_extent); - if (tmp->get_global_start() >= offset) { - break; - } else if (!tmp->is_closed()) { - // for extent that is not closed, shouldn't truncate. - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "the truncate extent is not closed", K(ret)); - } else if (tmp->get_global_end() > offset) { - break; - } else { - // release space - if (!tmp->is_truncated()) { - tmp->set_truncated(); - if (OB_FAIL(OB_TMP_FILE_STORE.free(get_tenant_id(), tmp->get_block_id(), - tmp->get_start_page_id(), - tmp->get_page_nums()))) { - STORAGE_LOG(WARN, "fail to release space", K(ret), K(read_guard_), K(tmp)); - } - STORAGE_LOG(TRACE, "release extents", K(ith_extent), K(tmp->get_start_page_id()), K(tmp->get_page_nums())); - } - if (OB_SUCC(ret)) { - // if only part of extent is truncated, we only need to set the read_guard - ith_extent++; - } - } - } - - if (OB_SUCC(ret) && offset > read_guard_) { - read_guard_ = offset; - next_truncated_extent_id_ = ith_extent; - } - return ret; -} - -int ObTmpFile::write_file_extent(const ObTmpFileIOInfo &io_info, ObTmpFileExtent *file_extent, - int64_t &size, char *&buf) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(size <= 0)|| OB_ISNULL(file_extent) || OB_ISNULL(buf)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret)); - } else if (OB_FAIL(file_extent->write(io_info, size, buf))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret), K(size), KP(buf)); - } - return ret; -} - -int ObTmpFile::write(const ObTmpFileIOInfo &io_info) -{ - int ret = OB_SUCCESS; - ObTmpFileIOHandle handle; - if (OB_FAIL(aio_write(io_info, handle))) { - STORAGE_LOG(WARN, "fail to write using asynchronous io", K(ret), K(io_info)); - } else if (OB_FAIL(handle.wait())) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret)); - } - return ret; -} - -int64_t ObTmpFile::small_file_prealloc_size() -{ - return SMALL_FILE_MAX_THRESHOLD * ObTmpMacroBlock::get_default_page_size(); -} - -int64_t ObTmpFile::big_file_prealloc_size() -{ - return BIG_FILE_PREALLOC_EXTENT_SIZE * ObTmpMacroBlock::get_default_page_size(); -} - - -ObTmpFileHandle::ObTmpFileHandle() - : ObResourceHandle() -{ -} - -ObTmpFileHandle::~ObTmpFileHandle() -{ - reset(); -} - -void ObTmpFileHandle::reset() -{ - if (NULL != ptr_) { - int tmp_ret = OB_SUCCESS; - if (OB_SUCCESS != (tmp_ret = ObTmpFileManager::get_instance().dec_handle_ref(*this))) { - STORAGE_LOG_RET(WARN, tmp_ret, "fail to decrease handle reference count", K(tmp_ret)); - } else { - ptr_ = nullptr; - } - } -} - -ObTmpFileManager &ObTmpFileManager::get_instance() -{ - static ObTmpFileManager instance; - return instance; -} - -int ObTmpFileManager::init() -{ - int ret = OB_SUCCESS; - ObMemAttr attr = SET_USE_500(ObMemAttr(OB_SERVER_TENANT_ID, ObModIds::OB_TMP_FILE_MANAGER)); //TODO: split tmp file map into each tenant. - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.init(DEFAULT_BUCKET_NUM, attr, *lib::ObMallocAllocator::get_instance()))) { - STORAGE_LOG(WARN, "fail to init map for temporary files", K(ret)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.init())) { - STORAGE_LOG(WARN, "fail to init the block manager for temporary files", K(ret)); - } else { - is_inited_ = true; - } - - if (!is_inited_) { - destroy(); - } - return ret; -} - -int ObTmpFileManager::alloc_dir(int64_t &dir) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(get_next_dir(dir))) { - STORAGE_LOG(WARN, "The directory of ObTmpFileManager has been used up", K(ret)); - } - return ret; -} - -int ObTmpFileManager::get_next_dir(int64_t &next_dir) -{ - int ret = OB_SUCCESS; - next_value(next_dir_, next_dir); - if (INT64_MAX - 1 == next_dir_) { - ret = OB_SIZE_OVERFLOW; - } - return ret; -} - -void ObTmpFileManager::next_value(int64_t ¤t_val, int64_t &next_val) -{ - int64_t old_val = ATOMIC_LOAD(¤t_val); - int64_t new_val = 0; - bool finish = false; - while (!finish) { - new_val = (old_val + 1) % INT64_MAX; - next_val = new_val; - finish = (old_val == (new_val = ATOMIC_VCAS(¤t_val, old_val, new_val))); - old_val = new_val; - } -} - -int ObTmpFileManager::open(int64_t &fd, int64_t &dir) -{ - int ret = OB_SUCCESS; - ObTmpFile file; - common::ObIAllocator *allocator = nullptr; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.get_tenant_extent_allocator(MTL_ID(), allocator))) { - STORAGE_LOG(WARN, "fail to get extent allocator", K(ret)); - } else if (OB_FAIL(get_next_fd(fd))) { - STORAGE_LOG(WARN, "fail to get next fd", K(ret)); - } else if (OB_FAIL(file.init(fd, dir, *allocator))) { - STORAGE_LOG(WARN, "fail to open file", K(ret)); - } else if (OB_FAIL(files_.set(fd, file))) { - STORAGE_LOG(WARN, "fail to set tmp file", K(ret)); - } else { - ObTaskController::get().allow_next_syslog(); - STORAGE_LOG(INFO, "succeed to open a tmp file", K(fd), K(dir), K(common::lbt())); - } - return ret; -} - -int ObTmpFileManager::get_next_fd(int64_t &next_fd) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle handle; - next_value(next_fd_, next_fd); - if (OB_FAIL(files_.get(next_fd, handle))) { - if (OB_ENTRY_NOT_EXIST != ret) { - STORAGE_LOG(WARN, "fail to get from resource map", K(ret), K(next_fd)); - } else { - ret = OB_SUCCESS; - } - } else { - ret = OB_FILE_ALREADY_EXIST; - STORAGE_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "too much file", K(ret)); - } - return ret; -} - -int ObTmpFileManager::aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->aio_read(io_info, handle))) { - if (common::OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to do asynchronous read", K(ret), K(io_info)); - } - } - return ret; -} - -int ObTmpFileManager::aio_pread(const ObTmpFileIOInfo &io_info, const int64_t offset, - ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->aio_pread(io_info, offset, handle))) { - if (common::OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to do asynchronous pread", K(ret), K(io_info), K(offset)); - } - } - return ret; -} - -int ObTmpFileManager::read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get temporary file handle", K(ret), K(io_info)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->read(io_info, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to read", K(ret), K(io_info)); - } - } - return ret; -} - -int ObTmpFileManager::pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(io_info)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->pread(io_info, offset, handle))) { - if (OB_ITER_END != ret) { - STORAGE_LOG(WARN, "fail to pread", K(ret), K(io_info)); - } - } - return ret; -} - -int ObTmpFileManager::aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(io_info)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->aio_write(io_info, handle))) { - STORAGE_LOG(WARN, "fail to aio_write", K(ret), K(io_info)); - } - return ret; -} - -int ObTmpFileManager::write(const ObTmpFileIOInfo &io_info) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (!io_info.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(io_info)); - } else if (OB_FAIL(files_.get(io_info.fd_, file_handle))) { - STORAGE_LOG(WARN, "fail to get temporary file handle", K(ret), K(io_info)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->write(io_info))) { - STORAGE_LOG(WARN, "fail to write", K(ret), K(io_info)); - } - return ret; -} - -int ObTmpFileManager::seek(const int64_t fd, const int64_t offset, const int whence) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.get(fd, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->seek(offset, whence))) { - STORAGE_LOG(WARN, "fail to seek file", K(ret)); - } - return ret; -} - -int ObTmpFileManager::truncate(const int64_t fd, const int64_t offset) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.get(fd, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->truncate(offset))) { - STORAGE_LOG(WARN, "fail to seek file", K(ret)); - } - return ret; -} - -int ObTmpFileManager::get_tmp_file_handle(const int64_t fd, ObTmpFileHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!is_inited_)) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.get(fd, handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } - return ret; -} - -int ObTmpFileManager::remove(const int64_t fd) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else{ - if (OB_FAIL(files_.erase(fd))) { - if (common::OB_ENTRY_NOT_EXIST != ret) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } else { - ret = OB_SUCCESS; - STORAGE_LOG(INFO, "this tmp file has been removed", K(fd), K(common::lbt())); - } - } else { - ObTaskController::get().allow_next_syslog(); - STORAGE_LOG(INFO, "succeed to remove a tmp file", K(fd), K(common::lbt())); - } - } - return ret; -} - -int ObTmpFileManager::remove_tenant_file(const uint64_t tenant_id) -{ - int ret = OB_SUCCESS; - common::ObSEArray fd_list; - fd_list.set_attr(ObMemAttr(MTL_ID(), "TMP_FD_LIST")); - RmTenantTmpFileOp rm_tenant_file_op(tenant_id, &fd_list); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(tenant_id)); - } else if (OB_FAIL(files_.foreach(rm_tenant_file_op))) { - STORAGE_LOG(WARN, "fail to foreach files_", K(ret), K(tenant_id)); - } else { - for (int64_t i = 0; OB_SUCC(ret) && i < fd_list.count(); i++) { - if (OB_FAIL(remove(fd_list.at(i)))) { - STORAGE_LOG(WARN, "fail to remove tmp file", K(ret), K(fd_list.at(i)), K(i)); - } - } - if (OB_SUCC(ret)) { - if (OB_FAIL(OB_TMP_FILE_STORE.free_tenant_file_store(tenant_id))) { - if (OB_ENTRY_NOT_EXIST == ret) { - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "fail to free tmp tenant file store", K(ret), K(tenant_id)); - } - } - } - } - return ret; -} - -int ObTmpFileManager::get_all_tenant_id(common::ObIArray &tenant_ids) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(OB_TMP_FILE_STORE.get_all_tenant_id(tenant_ids))) { - STORAGE_LOG(WARN, "fail to get all tenant ids", K(ret)); - } - return ret; -} - -int ObTmpFileManager::sync(const int64_t fd, const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.get(fd, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } else if (OB_FAIL(file_handle.get_resource_ptr()->sync(timeout_ms))) { - STORAGE_LOG(WARN, "fail to close file", K(ret)); - } - return ret; -} - -ObTmpFileManager::ObTmpFileManager() - : is_inited_(false), - next_fd_(-1), - next_dir_(-1), - rm_file_lock_(common::ObLatchIds::TMP_FILE_MGR_LOCK), - files_() -{ -} - -ObTmpFileManager::~ObTmpFileManager() -{ - destroy(); -} - -void ObTmpFileManager::destroy() -{ - files_.destroy(); - OB_TMP_FILE_STORE.destroy(); - next_fd_ = -1; - next_dir_ = -1; - is_inited_ = false; -} - -int ObTmpFileManager::dec_handle_ref(ObTmpFileHandle &handle) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (OB_FAIL(files_.dec_handle_ref(handle.ptr_))) { - STORAGE_LOG(WARN, "fail to dec handle ref without lock", K(ret)); - } - return ret; -} - -int ObTmpFileManager::get_tmp_file_size(const int64_t fd, int64_t &file_size) -{ - int ret = OB_SUCCESS; - ObTmpFileHandle file_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileManager has not been inited", K(ret)); - } else if (fd < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(fd)); - } else if (OB_FAIL(files_.get(fd, file_handle))) { - STORAGE_LOG(WARN, "fail to get tmp file handle", K(ret), K(fd)); - } else { - file_handle.get_resource_ptr()->get_file_size(file_size); - } - return ret; -} - -} // end namespace blocksstable -} // end namespace oceanbase diff --git a/src/storage/blocksstable/ob_tmp_file.h b/src/storage/blocksstable/ob_tmp_file.h deleted file mode 100644 index da65938f4b..0000000000 --- a/src/storage/blocksstable/ob_tmp_file.h +++ /dev/null @@ -1,444 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_H_ -#define OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_H_ - -#include "storage/ob_resource_map.h" -#include "lib/container/ob_se_array.h" -#include "storage/ob_resource_map.h" -#include "ob_macro_block_handle.h" -#include "ob_block_manager.h" -#include "ob_tmp_file_store.h" - -namespace oceanbase -{ -namespace blocksstable -{ - -class ObTmpFile; -class ObTmpFileExtent; - -struct ObTmpFileIOInfo final -{ -public: - ObTmpFileIOInfo(); - ~ObTmpFileIOInfo(); - void reset(); - bool is_valid() const; - TO_STRING_KV(K_(fd), K_(dir_id), K_(size), K_(io_timeout_ms), K_(tenant_id), KP_(buf), K_(io_desc)); - int64_t fd_; - int64_t dir_id_; - int64_t size_; - int64_t io_timeout_ms_; - uint64_t tenant_id_; - char *buf_; - common::ObIOFlag io_desc_; - bool disable_page_cache_; -}; - -class ObTmpFileIOHandle final -{ -public: - struct ObIOReadHandle final - { - ObIOReadHandle(); - ObIOReadHandle(const ObMacroBlockHandle ¯o_handle, char *buf, const int64_t offset, - const int64_t size); - ~ObIOReadHandle(); - ObIOReadHandle(const ObIOReadHandle &other); - ObIOReadHandle &operator=(const ObIOReadHandle &other); - TO_STRING_KV(K_(macro_handle), K_(offset), K_(size), KP_(buf)); - ObMacroBlockHandle macro_handle_; - char *buf_; - int64_t offset_; - int64_t size_; - }; - - struct ObPageCacheHandle final - { - ObPageCacheHandle(); - ObPageCacheHandle(const ObTmpPageValueHandle &page_handle, char *buf, const int64_t offset, - const int64_t size); - ~ObPageCacheHandle(); - ObPageCacheHandle(const ObPageCacheHandle &other); - ObPageCacheHandle &operator=(const ObPageCacheHandle &other); - TO_STRING_KV(K_(page_handle), K_(offset), K_(size), KP_(buf)); - ObTmpPageValueHandle page_handle_; - char *buf_; - int64_t offset_; - int64_t size_; - }; - - struct ObBlockCacheHandle final - { - ObBlockCacheHandle(); - ObBlockCacheHandle(const ObTmpBlockValueHandle &block_handle, char *buf, const int64_t offset, - const int64_t size); - ~ObBlockCacheHandle(); - ObBlockCacheHandle(const ObBlockCacheHandle &other); - ObBlockCacheHandle &operator=(const ObBlockCacheHandle &other); - TO_STRING_KV(K_(block_handle), K_(offset), K_(size), KP_(buf)); - ObTmpBlockValueHandle block_handle_; - char *buf_; - int64_t offset_; - int64_t size_; - }; - - ObTmpFileIOHandle(); - ~ObTmpFileIOHandle(); - OB_INLINE char *get_buffer() { return buf_; } - OB_INLINE int64_t get_data_size() { return size_; } - OB_INLINE bool is_disable_page_cache() const { return disable_page_cache_; } - int prepare_read( - const int64_t read_size, - const int64_t read_offset, - const common::ObIOFlag io_flag, - char *read_buf, - int64_t fd, - int64_t dir_id, - uint64_t tenant_id, - const bool disable_page_cache); - int prepare_write( - char *write_buf, - const int64_t write_size, - int64_t fd, - int64_t dir_id, - uint64_t tenant_id); - OB_INLINE void add_data_size(const int64_t size) { size_ += size; } - OB_INLINE void sub_data_size(const int64_t size) { size_ -= size; } - OB_INLINE void set_update_offset_in_file() { update_offset_in_file_ = true; } - OB_INLINE void set_last_read_offset(const int64_t last_read_offset) - { - last_read_offset_ = last_read_offset; - } - int wait(); - void reset(); - bool is_valid() const; - common::ObIArray &get_io_handles() - { - return io_handles_; - } - common::ObIArray &get_page_cache_handles() - { - return page_cache_handles_; - } - common::ObIArray &get_block_cache_handles() - { - return block_cache_handles_; - } - int record_block_id(const int64_t block_it); - - OB_INLINE int64_t get_last_read_offset() const { return last_read_offset_; } - int64_t get_last_extent_id() const; - void set_last_extent_id(const int64_t last_extent_id); - - TO_STRING_KV(KP_(buf), K_(size), K_(is_read), K_(has_wait), K_(expect_read_size), - K_(last_read_offset), K_(io_flag), K_(update_offset_in_file)); - -private: - int wait_write_finish(const int64_t timeout_ms); - int wait_read_finish(const int64_t timeout_ms); - int do_read_wait(const int64_t timeout_ms); - -private: - common::ObSEArray io_handles_; - common::ObSEArray page_cache_handles_; - common::ObSEArray block_cache_handles_; - common::hash::ObHashSet write_block_ids_; - int64_t fd_; - int64_t dir_id_; - uint64_t tenant_id_; - char *buf_; - int64_t size_; //has read or to write size. - bool is_read_; - bool has_wait_; - bool is_finished_; - bool disable_page_cache_; - int ret_code_; - int64_t expect_read_size_; - int64_t last_read_offset_; // only for more than 8MB read. - common::ObIOFlag io_flag_; - bool update_offset_in_file_; - int64_t last_fd_; - int64_t last_extent_id_; - DISALLOW_COPY_AND_ASSIGN(ObTmpFileIOHandle); -}; - -class ObTmpFileExtent final -{ -public: - explicit ObTmpFileExtent(ObTmpFile *file); - ~ObTmpFileExtent(); - int read(const ObTmpFileIOInfo &io_info, const int64_t offset, const int64_t size, - char *buf, ObTmpFileIOHandle &handle); - int write(const ObTmpFileIOInfo &io_info, int64_t &size, char *&buf); - void reset(); - OB_INLINE bool is_closed() const { return ATOMIC_LOAD(&is_closed_); } - OB_INLINE bool is_truncated() const { return ATOMIC_LOAD(&is_truncated_); } - void set_truncated() { ATOMIC_STORE(&is_truncated_, true); } - bool is_valid(); - bool close(bool force = false); - bool close(uint8_t &free_page_start_id, uint8_t &free_page_nums, bool force = false); - void unclose(const int32_t page_nums = -1); - bool is_alloced() const { return is_alloced_; } - OB_INLINE void set_global_offset(const int64_t g_offset_start, const int64_t g_offset_end); - OB_INLINE void get_global_offset(int64_t &g_offset_start, int64_t &g_offset_end) const; - OB_INLINE int64_t get_global_end() const { return g_offset_end_; } - OB_INLINE int64_t get_global_start() const { return g_offset_start_; } - OB_INLINE void alloced() { is_alloced_ = true; } - OB_INLINE void set_start_page_id(const uint8_t start_page_id) { start_page_id_ = start_page_id; } - OB_INLINE uint8_t get_start_page_id() const { return start_page_id_; } - OB_INLINE void set_page_nums(const uint8_t page_nums) { page_nums_ = 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 int64_t get_block_id() const { return block_id_; } - OB_INLINE int32_t get_offset() const { return ATOMIC_LOAD(&offset_); } - 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), - K_(start_page_id), K_(page_nums), K_(block_id), K_(offset), K_(is_closed)); - -private: - int try_sync_block(); - -private: - bool is_alloced_; - bool is_closed_; // only if close, this extent cannot be used. - uint8_t start_page_id_; - uint8_t page_nums_; - int32_t offset_; - int64_t fd_; - int64_t g_offset_start_; - int64_t g_offset_end_; - ObTmpFile *owner_; - int64_t block_id_; - common::SpinRWLock lock_; - bool is_truncated_; - DISALLOW_COPY_AND_ASSIGN(ObTmpFileExtent); -}; - -class ObTmpFileMeta final -{ -public: - explicit ObTmpFileMeta() : fd_(-1), dir_id_(-1), allocator_(NULL), extents_() - { - extents_.set_attr(ObMemAttr(MTL_ID(), "TMP_META")); - } - ~ObTmpFileMeta(); - int clear(); - int init(const int64_t fd, const int64_t dir_id, common::ObIAllocator *allocator); - ObTmpFileExtent *get_last_extent(); - common::ObIArray &get_extents() { return extents_; } - int push_back_extent(ObTmpFileExtent *extent) { return extents_.push_back(extent); } - int pop_back_extent(ObTmpFileExtent *&extent) { return extents_.pop_back(extent); } - void pop_back_extent() { extents_.pop_back(); } - int deep_copy(const ObTmpFileMeta &other); - OB_INLINE int64_t get_fd() const { return fd_; } - OB_INLINE int64_t get_dir_id() const { return dir_id_; } - TO_STRING_KV(K_(fd), K_(dir_id), K_(extents)); - -private: - int64_t fd_; - int64_t dir_id_; - common::ObIAllocator *allocator_; - ExtentArray extents_; // b-tree is better - DISALLOW_COPY_AND_ASSIGN(ObTmpFileMeta); -}; - -class ObTmpFile final -{ -public: - enum FileWhence - { - SET_SEEK = 0, - CUR_SEEK, - }; - ObTmpFile(); - ~ObTmpFile(); - int init(const int64_t fd, const int64_t dir_id, common::ObIAllocator &allocator); - int aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - int aio_pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle); - int read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - int pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle); - int aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - int write(const ObTmpFileIOInfo &io_info); - int seek(const int64_t offset, const int whence); - - // the data before the offset is released - int truncate(const int64_t offset); - int clear(); - int64_t get_dir_id() const; - uint64_t get_tenant_id() const; - int64_t get_fd() const; - int sync(const int64_t timeout_ms); - int deep_copy(char *buf, const int64_t buf_len, ObTmpFile *&value) const; - // only for ObTmpFileIOHandle, once more than READ_SIZE_PER_BATCH read. - int once_aio_read_batch( - const ObTmpFileIOInfo &io_info, - const bool need_update_offset, - int64_t &offset, - ObTmpFileIOHandle &handle); - - void get_file_size(int64_t &file_size); - OB_INLINE int64_t get_deep_copy_size() const { return sizeof(*this); } ; - TO_STRING_KV(K_(file_meta), K_(is_big), K_(tenant_id), K_(is_inited)); - -private: - static int fill_zero(char *buf, const int64_t size); - int write_file_extent(const ObTmpFileIOInfo &io_info, ObTmpFileExtent *file_extent, - int64_t &size, char *&buf); - int aio_read_without_lock( - const ObTmpFileIOInfo &io_info, - int64_t &offset, - ObTmpFileIOHandle &handle); - int once_aio_read_batch_without_lock( - const ObTmpFileIOInfo &io_info, - int64_t &offset, - ObTmpFileIOHandle &handle); - int64_t small_file_prealloc_size(); - int64_t big_file_prealloc_size(); - int64_t find_first_extent(const int64_t offset); - int64_t get_extent_cache(const int64_t offset, const ObTmpFileIOHandle &handle); - -private: - // NOTE: - // 1.The pre-allocated macro should satisfy the following inequality: - // SMALL_FILE_MAX_THRESHOLD < BIG_FILE_PREALLOC_EXTENT_SIZE < block size - static const int64_t SMALL_FILE_MAX_THRESHOLD = 4; - static const int64_t BIG_FILE_PREALLOC_EXTENT_SIZE = 8; - static const int64_t READ_SIZE_PER_BATCH = 8 * 1024 * 1024; // 8MB - - bool is_inited_; - bool is_big_; - int64_t offset_; // read offset - uint64_t tenant_id_; - common::SpinRWLock lock_; - common::ObIAllocator *allocator_; - ObTmpFileMeta file_meta_; - - // content before read_guard_ is truncated, which means the space is released. read before read_guard_ will only return 0; - int64_t read_guard_; - - // to optimize truncated speed, record the last_truncated_extent_id, so that we do not need to binary search the extent id every time we truncated. - int64_t next_truncated_extent_id_; - - DISALLOW_COPY_AND_ASSIGN(ObTmpFile); -}; - -class ObTmpFileHandle final: public storage::ObResourceHandle -{ -public: - ObTmpFileHandle(); - ~ObTmpFileHandle(); - virtual void reset() override; -private: - friend class ObTmpFileManager; - DISALLOW_COPY_AND_ASSIGN(ObTmpFileHandle); -}; - -class ObTmpFileManager final -{ -public: - static ObTmpFileManager &get_instance(); - int init(); - - int alloc_dir(int64_t &dir); - int open(int64_t &fd, int64_t &dir); - // NOTE: - // default order read, if want to read random, should be seek first. - int aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - int aio_pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle); - // NOTE: - // default order read, if want to read random, should be seek first. - int read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - int pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &handle); - // NOTE: - // only support order write. - int aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &handle); - // NOTE: - // only support order write. - int write(const ObTmpFileIOInfo &io_info); - // only for read: - // 1. whence == SET_SEEK, inner offset = offset; - // 2. whence == CUR_SEEK, inner offset -= offset; - int seek(const int64_t fd, const int64_t offset, const int whence); - // NOTE: - // remove file and all of block in this file, after not used file, should be called in case - // of block leak. - int truncate(const int64_t fd, const int64_t offset); - int remove(const int64_t fd); - int remove_tenant_file(const uint64_t tenant_id); - - int get_all_tenant_id(common::ObIArray &tenant_ids); - - int sync(const int64_t fd, const int64_t timeout_ms); - - void destroy(); - int dec_handle_ref(ObTmpFileHandle &handle); - // Returns the size of the current temporary file - int get_tmp_file_size(const int64_t fd, int64_t &file_size); - -public: - friend class ObTmpFileIOHandle; - -private: - class RmTenantTmpFileOp - { - public: - RmTenantTmpFileOp(const uint64_t tenant_id, common::ObIArray *fd_list) - : tenant_id_(tenant_id), fd_list_(fd_list) - {} - ~RmTenantTmpFileOp() = default; - int operator()(common::hash::HashMapPair &entry) - { - int ret = OB_SUCCESS; - ObTmpFile *tmp_file = entry.second; - if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id_) - || OB_ISNULL(fd_list_) || OB_ISNULL(tmp_file)) { - ret = common::OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret)); - } else if (tmp_file->get_tenant_id() == tenant_id_) { - if (OB_FAIL(fd_list_->push_back(tmp_file->get_fd()))) { - STORAGE_LOG(WARN, "fd_list_ push back failed", K(ret)); - } - } - return ret; - } - private: - const uint64_t tenant_id_; - common::ObIArray *fd_list_; - }; - -private: - ObTmpFileManager(); - ~ObTmpFileManager(); - int get_next_dir(int64_t &next_dir); - int get_next_fd(int64_t &next_fd); - void next_value(int64_t ¤t_val, int64_t &next_val); - int get_tmp_file_handle(const int64_t fd, ObTmpFileHandle &handle); - -private: - static const int64_t DEFAULT_BUCKET_NUM = 10243L; - bool is_inited_; - int64_t next_fd_; - int64_t next_dir_; - common::SpinRWLock rm_file_lock_; - storage::ObResourceMap files_; - - DISALLOW_COPY_AND_ASSIGN(ObTmpFileManager); -}; - -#define FILE_MANAGER_INSTANCE_V2 (::oceanbase::blocksstable::ObTmpFileManager::get_instance()) - -} // end namespace blocksstable -} // end namespace oceanbase -#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_H_ diff --git a/src/storage/blocksstable/ob_tmp_file_cache.cpp b/src/storage/blocksstable/ob_tmp_file_cache.cpp deleted file mode 100644 index 3d8c9cd2e2..0000000000 --- a/src/storage/blocksstable/ob_tmp_file_cache.cpp +++ /dev/null @@ -1,1655 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#include "observer/omt/ob_tenant_config_mgr.h" -#include "lib/stat/ob_diagnose_info.h" -#include "common/ob_smart_var.h" -#include "storage/ob_file_system_router.h" -#include "share/ob_task_define.h" -#include "ob_tmp_file_cache.h" -#include "ob_tmp_file.h" -#include "ob_tmp_file_store.h" -#include "ob_block_manager.h" - -using namespace oceanbase::storage; -using namespace oceanbase::share; - -namespace oceanbase -{ -namespace blocksstable -{ - -ObTmpPageCacheKey::ObTmpPageCacheKey() - : block_id_(-1), page_id_(-1), tenant_id_(0) -{ -} - -ObTmpPageCacheKey::ObTmpPageCacheKey(const int64_t block_id, const int64_t page_id, - const uint64_t tenant_id) - : block_id_(block_id), page_id_(page_id), tenant_id_(tenant_id) -{ -} - -ObTmpPageCacheKey::~ObTmpPageCacheKey() -{ -} - -bool ObTmpPageCacheKey::operator ==(const ObIKVCacheKey &other) const -{ - const ObTmpPageCacheKey &other_key = reinterpret_cast (other); - return block_id_ == other_key.block_id_ - && page_id_ == other_key.page_id_ - && tenant_id_ == other_key.tenant_id_; -} - -uint64_t ObTmpPageCacheKey::get_tenant_id() const -{ - return tenant_id_; -} - -uint64_t ObTmpPageCacheKey::hash() const -{ - return murmurhash(this, sizeof(ObTmpPageCacheKey), 0); -} - -int64_t ObTmpPageCacheKey::size() const -{ - return sizeof(*this); -} - -int ObTmpPageCacheKey::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "Invalid argument, ", K(ret)); - } else if (OB_UNLIKELY(!is_valid())) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid tmp page cache key, ", K(*this), K(ret)); - } else { - key = new (buf) ObTmpPageCacheKey(block_id_, page_id_, tenant_id_); - } - return ret; -} - -bool ObTmpPageCacheKey::is_valid() const -{ - return OB_LIKELY(block_id_ > 0 && page_id_ >= 0 && tenant_id_ > 0 && size() > 0); -} - -ObTmpPageCacheValue::ObTmpPageCacheValue(char *buf) - : buf_(buf), size_(ObTmpMacroBlock::get_default_page_size()) -{ -} - -ObTmpPageCacheValue::~ObTmpPageCacheValue() -{ -} - -int64_t ObTmpPageCacheValue::size() const -{ - return sizeof(*this) + size_; -} - -int ObTmpPageCacheValue::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), KP(buf), K(buf_len), - "request_size", size()); - } else if (OB_UNLIKELY(!is_valid())) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid tmp page cache value", K(ret)); - } else { - ObTmpPageCacheValue *pblk_value = new (buf) ObTmpPageCacheValue(buf + sizeof(*this)); - MEMCPY(buf + sizeof(*this), buf_, size() - sizeof(*this)); - pblk_value->size_ = size_; - value = pblk_value; - } - return ret; -} - -int ObTmpPageCache::inner_read_io(const ObTmpBlockIOInfo &io_info, - ObITmpPageIOCallback *callback, - ObMacroBlockHandle ¯o_block_handle) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(read_io(io_info, callback, macro_block_handle))) { - if (macro_block_handle.get_io_handle().is_empty()) { - // TODO: After the continuous IO has been optimized, this should - // not happen. - if (OB_FAIL(macro_block_handle.wait())) { - STORAGE_LOG(WARN, "fail to wait tmp page io", K(ret), KP(callback)); - } else if (OB_FAIL(read_io(io_info, callback, macro_block_handle))) { - STORAGE_LOG(WARN, "fail to read tmp page from io", K(ret), KP(callback)); - } - } else { - STORAGE_LOG(WARN, "fail to read tmp page from io", K(ret), KP(callback)); - } - } - // Avoid double_free with io_handle - if (OB_FAIL(ret) && OB_NOT_NULL(callback) && OB_NOT_NULL(callback->get_allocator())) { - common::ObIAllocator *allocator = callback->get_allocator(); - callback->~ObITmpPageIOCallback(); - allocator->free(callback); - } - return ret; -} - -int ObTmpPageCache::direct_read(const ObTmpBlockIOInfo &info, - ObMacroBlockHandle &mb_handle, - common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - void *buf = nullptr; - ObTmpDirectReadPageIOCallback *callback = nullptr; - if (OB_ISNULL(buf = allocator.alloc(sizeof(ObTmpDirectReadPageIOCallback)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "allocate callback memory failed", K(ret)); - } else { - // fill the callback - callback = new (buf) ObTmpDirectReadPageIOCallback; - callback->cache_ = this; - callback->offset_ = info.offset_; - callback->allocator_ = &allocator; - if (OB_FAIL(inner_read_io(info, callback, mb_handle))) { - STORAGE_LOG(WARN, "fail to inner read io", K(ret), K(mb_handle)); - } - // There is no need to handle error cases (freeing the memory of the - // callback) because inner_read_io will handle error cases and free the - // memory of the callback. - } - return ret; -} - -int ObTmpPageCache::prefetch( - const ObTmpPageCacheKey &key, - const ObTmpBlockIOInfo &info, - ObMacroBlockHandle &mb_handle, - common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!key.is_valid() )) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "Invalid arguments", K(ret), K(key)); - } else { - // fill the callback - void *buf = nullptr; - ObTmpPageIOCallback *callback = nullptr; - if (OB_ISNULL(buf = allocator.alloc(sizeof(ObTmpPageIOCallback)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "allocate callback memory failed", K(ret)); - } else { - callback = new (buf) ObTmpPageIOCallback; - callback->cache_ = this; - callback->offset_ = info.offset_; - callback->allocator_ = &allocator; - callback->key_ = key; - if (OB_FAIL(inner_read_io(info, callback, mb_handle))) { - STORAGE_LOG(WARN, "fail to inner read io", K(ret), K(mb_handle)); - } - // There is no need to handle error cases (freeing the memory of the - // callback) because inner_read_io will handle error cases and free the - // memory of the callback. - } - } - return ret; -} - -int ObTmpPageCache::prefetch( - const ObTmpBlockIOInfo &info, - const common::ObIArray &page_io_infos, - ObMacroBlockHandle &mb_handle, - common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(page_io_infos.count() <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "Invalid arguments", K(ret), K(page_io_infos.count()), K(info)); - } else { - void *buf = nullptr; - ObTmpMultiPageIOCallback *callback = nullptr; - if (OB_ISNULL(buf = allocator.alloc(sizeof(ObTmpMultiPageIOCallback)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "allocate callback memory failed", K(ret)); - } else { - callback = new (buf) ObTmpMultiPageIOCallback; - callback->cache_ = this; - callback->offset_ = info.offset_; - callback->allocator_ = &allocator; - if (OB_FAIL(callback->page_io_infos_.assign(page_io_infos))) { - STORAGE_LOG(WARN, "fail to assign page io infos", K(ret), K(page_io_infos.count()), K(info)); - if (OB_NOT_NULL(callback)) { // handle ObArray assign fail case and free callback - callback->~ObTmpMultiPageIOCallback(); - allocator.free(callback); - callback = nullptr; - } - } else if (OB_FAIL(inner_read_io(info, callback, mb_handle))) { - STORAGE_LOG(WARN, "fail to inner read io", K(ret), K(mb_handle)); - } - // inner_read_io will handle error cases and free the memory of callback. - } - } - return ret; -} - -int ObTmpPageCache::get_cache_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!key.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key)); - } else { - const ObTmpPageCacheValue *value = NULL; - if (OB_FAIL(get(key, value, handle.handle_))) { - if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { - STORAGE_LOG(WARN, "fail to get key from page cache", K(ret)); - } else { - EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_MISS); - } - } else { - if (OB_ISNULL(value)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, the value must not be NULL", K(ret)); - } else { - handle.value_ = const_cast(value); - EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_HIT); - } - } - } - return ret; -} - -ObTmpPageCache::ObITmpPageIOCallback::ObITmpPageIOCallback() - : cache_(NULL), allocator_(NULL), offset_(0), data_buf_(NULL) -{ - static_assert(sizeof(*this) <= CALLBACK_BUF_SIZE, "IOCallback buf size not enough"); -} - -ObTmpPageCache::ObITmpPageIOCallback::~ObITmpPageIOCallback() -{ - if (NULL != allocator_ && NULL != data_buf_) { - allocator_->free(data_buf_); - data_buf_ = NULL; - } - allocator_ = NULL; -} - -int ObTmpPageCache::ObITmpPageIOCallback::alloc_data_buf(const char *io_data_buffer, const int64_t data_size) -{ - int ret = alloc_and_copy_data(io_data_buffer, data_size, allocator_, data_buf_); - return ret; -} - -int ObTmpPageCache::ObITmpPageIOCallback::process_page( - const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!key.is_valid() || !value.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key), K(value)); - } else if (OB_FAIL(cache_->put(key, value, true/*overwrite*/))) { - STORAGE_LOG(WARN, "fail to put tmp page into cache", K(ret), K(key), K(value)); - } - return ret; -} - -ObTmpPageCache::ObTmpPageIOCallback::ObTmpPageIOCallback() - : key_() -{ - static_assert(sizeof(*this) <= CALLBACK_BUF_SIZE, "IOCallback buf size not enough"); -} - -ObTmpPageCache::ObTmpPageIOCallback::~ObTmpPageIOCallback() -{ - -} - -int ObTmpPageCache::ObTmpPageIOCallback::inner_process(const char *data_buffer, const int64_t size) -{ - int ret = OB_SUCCESS; - ObTimeGuard time_guard("TmpPage_Callback_Process", 100000); //100ms - if (OB_ISNULL(cache_) || OB_ISNULL(allocator_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "Invalid tmp page cache callback or allocator", KP_(cache), KP_(allocator), K(ret)); - } else if (OB_UNLIKELY(size <= 0 || data_buffer == nullptr)) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid data buffer size", K(ret), K(size), KP(data_buffer)); - } else if (OB_FAIL(alloc_data_buf(data_buffer, size))) { - STORAGE_LOG(WARN, "Fail to allocate memory, ", K(ret), K(size)); - } else if (FALSE_IT(time_guard.click("alloc_data_buf"))) { - } else { - ObTmpPageCacheValue value(data_buf_); - if (OB_FAIL(process_page(key_, value))) { - STORAGE_LOG(WARN, "fail to process tmp page cache in callback", K(ret)); - } - time_guard.click("process_page"); - } - if (OB_FAIL(ret) && NULL != allocator_ && NULL != data_buf_) { - allocator_->free(data_buf_); - data_buf_ = NULL; - } - return ret; -} - -int64_t ObTmpPageCache::ObTmpPageIOCallback::size() const -{ - return sizeof(*this); -} - -const char *ObTmpPageCache::ObTmpPageIOCallback::get_data() -{ - return data_buf_; -} - -ObTmpPageCache::ObTmpMultiPageIOCallback::ObTmpMultiPageIOCallback() - : page_io_infos_() -{ - static_assert(sizeof(*this) <= CALLBACK_BUF_SIZE, "IOCallback buf size not enough"); -} -ObTmpPageCache::ObTmpMultiPageIOCallback::~ObTmpMultiPageIOCallback() -{ - page_io_infos_.reset(); - page_io_infos_.~ObIArray(); -} - -int ObTmpPageCache::ObTmpMultiPageIOCallback::inner_process(const char *data_buffer, const int64_t size) -{ - int ret = OB_SUCCESS; - ObTimeGuard time_guard("TmpMultiPage_Callback_Process", 100000); //100ms - if (OB_ISNULL(cache_) || OB_ISNULL(allocator_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "Invalid tmp page cache callbackor allocator", KP_(cache), KP_(allocator), K(ret)); - } else if (OB_UNLIKELY(size <= 0 || data_buffer == nullptr)) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid data buffer size", K(ret), K(size), KP(data_buffer)); - } else if (OB_FAIL(alloc_data_buf(data_buffer, size))) { - STORAGE_LOG(WARN, "Fail to allocate memory, ", K(ret), K(size)); - } else if (FALSE_IT(time_guard.click("alloc_data_buf"))) { - } else { - for (int32_t i = 0; OB_SUCC(ret) && i < page_io_infos_.count(); i++) { - int64_t cur_offset = page_io_infos_.at(i).key_.get_page_id() - * ObTmpMacroBlock::get_default_page_size() - offset_; - cur_offset += ObTmpMacroBlock::get_header_padding(); - ObTmpPageCacheValue value(data_buf_ + cur_offset); - if (OB_FAIL(process_page(page_io_infos_.at(i).key_, value))) { - STORAGE_LOG(WARN, "fail to process tmp page cache in callback", K(ret)); - } - } - time_guard.click("process_page"); - page_io_infos_.reset(); - } - if (OB_FAIL(ret) && NULL != allocator_ && NULL != data_buf_) { - allocator_->free(data_buf_); - data_buf_ = NULL; - } - return ret; -} - -int64_t ObTmpPageCache::ObTmpMultiPageIOCallback::size() const -{ - return sizeof(*this); -} - -const char *ObTmpPageCache::ObTmpMultiPageIOCallback::get_data() -{ - return data_buf_; -} - -int64_t ObTmpPageCache::ObTmpDirectReadPageIOCallback::size() const -{ - return sizeof(*this); -} - -const char * ObTmpPageCache::ObTmpDirectReadPageIOCallback::get_data() -{ - return data_buf_; -} - -int ObTmpPageCache::ObTmpDirectReadPageIOCallback::inner_process(const char *data_buffer, const int64_t size) -{ - int ret = OB_SUCCESS; - ObTimeGuard time_guard("ObTmpDirectReadPageIOCallback", 100000); //100ms - if (OB_ISNULL(cache_) || OB_ISNULL(allocator_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "Invalid tmp page cache callback allocator", KP_(cache), KP_(allocator), K(ret)); - } else if (OB_UNLIKELY(size <= 0 || data_buffer == nullptr)) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid data buffer size", K(ret), K(size), KP(data_buffer)); - } else if (OB_FAIL(alloc_data_buf(data_buffer, size))) { - STORAGE_LOG(WARN, "Fail to allocate memory, ", K(ret), K(size)); - } else if (FALSE_IT(time_guard.click("alloc_data_buf"))) { - } - if (OB_FAIL(ret) && NULL != allocator_ && NULL != data_buf_) { - allocator_->free(data_buf_); - data_buf_ = NULL; - } - return ret; -} - -int ObTmpPageCache::read_io(const ObTmpBlockIOInfo &io_info, ObITmpPageIOCallback *callback, - ObMacroBlockHandle &handle) -{ - int ret = OB_SUCCESS; - common::ObArenaAllocator allocator(ObModIds::OB_MACRO_FILE); - // fill the read info - ObMacroBlockReadInfo read_info; - read_info.io_desc_ = io_info.io_desc_; - read_info.macro_block_id_ = io_info.macro_block_id_; - read_info.io_timeout_ms_ = io_info.io_timeout_ms_; - read_info.io_callback_ = callback; - read_info.offset_ = io_info.offset_; - read_info.size_ = io_info.size_; - read_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - read_info.io_desc_.set_sys_module_id(ObIOModule::TMP_PAGE_CACHE_IO); - if (OB_FAIL(ObBlockManager::async_read_block(read_info, handle))) { - STORAGE_LOG(WARN, "fail to async read block", K(ret), K(read_info), KP(callback)); - } - return ret; -} - -ObTmpPageCache &ObTmpPageCache::get_instance() -{ - static ObTmpPageCache instance; - return instance; -} - -ObTmpPageCache::ObTmpPageCache() -{ -} - -ObTmpPageCache::~ObTmpPageCache() -{ -} - -int ObTmpPageCache::init(const char *cache_name, const int64_t priority) -{ - int ret = OB_SUCCESS; - if (OB_FAIL((common::ObKVCache::init( - cache_name, priority)))) { - STORAGE_LOG(WARN, "Fail to init kv cache, ", K(ret)); - } - return ret; -} - -void ObTmpPageCache::destroy() -{ - common::ObKVCache::destroy(); -} - -int ObTmpPageCache::put_page(const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!key.is_valid() || !value.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key), K(value)); - } else if (OB_FAIL(put(key, value, true/*overwrite*/))) { - STORAGE_LOG(WARN, "fail to put page to page cache", K(ret), K(key), K(value)); - } - return ret; -} - -int ObTmpPageCache::get_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle) -{ - int ret = OB_SUCCESS; - const ObTmpPageCacheValue *value = NULL; - if (OB_UNLIKELY(!key.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key)); - } else if (OB_FAIL(get(key, value, handle.handle_))) { - if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { - STORAGE_LOG(WARN, "fail to get key from page cache", K(ret), K(key)); - } else { - EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_MISS); - } - } else { - if (OB_ISNULL(value)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, the value must not be NULL", K(ret)); - } else { - handle.value_ = const_cast(value); - EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_HIT); - } - } - return ret; -} - -ObTmpBlockCacheKey::ObTmpBlockCacheKey(const int64_t block_id, const uint64_t tenant_id) - : block_id_(block_id), tenant_id_(tenant_id) -{ -} - -bool ObTmpBlockCacheKey::operator ==(const ObIKVCacheKey &other) const -{ - const ObTmpBlockCacheKey &other_key = reinterpret_cast (other); - return block_id_ == other_key.block_id_ - && tenant_id_ == other_key.tenant_id_; -} - -uint64_t ObTmpBlockCacheKey::get_tenant_id() const -{ - return tenant_id_; -} - -uint64_t ObTmpBlockCacheKey::hash() const -{ - return murmurhash(this, sizeof(ObTmpBlockCacheKey), 0); -} - -int64_t ObTmpBlockCacheKey::size() const -{ - return sizeof(*this); -} - -int ObTmpBlockCacheKey::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "Invalid argument, ", K(ret)); - } else if (OB_UNLIKELY(!is_valid())) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid tmp block cache key, ", K(*this), K(ret)); - } else { - key = new(buf) ObTmpBlockCacheKey(block_id_, tenant_id_); - } - return ret; -} - -ObTmpBlockCacheValue::ObTmpBlockCacheValue(char *buf) - : buf_(buf), size_(ObTmpFileStore::get_block_size()) -{ -} - -int64_t ObTmpBlockCacheValue::size() const -{ - return sizeof(*this) + size_; -} - -int ObTmpBlockCacheValue::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(ret), KP(buf), K(buf_len), - "request_size", size()); - } else if (OB_UNLIKELY(!is_valid())) { - ret = OB_INVALID_DATA; - STORAGE_LOG(WARN, "invalid tmp block cache value", K(ret)); - } else { - ObTmpBlockCacheValue *pblk_value = new (buf) ObTmpBlockCacheValue(buf + sizeof(*this)); - MEMCPY(buf + sizeof(*this), buf_, size() - sizeof(*this)); - pblk_value->size_ = size_; - value = pblk_value; - } - return ret; -} - -ObTmpBlockCache &ObTmpBlockCache::get_instance() -{ - static ObTmpBlockCache instance; - return instance; -} - -int ObTmpBlockCache::init(const char *cache_name, const int64_t priority) -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(cache_name) || OB_UNLIKELY(priority <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(cache_name), K(priority)); - } else if (OB_FAIL((common::ObKVCache::init(cache_name, priority)))) { - STORAGE_LOG(WARN, "Fail to init kv cache, ", K(ret)); - } - return ret; -} - -int ObTmpBlockCache::get_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle) -{ - int ret = OB_SUCCESS; - const ObTmpBlockCacheValue *value = NULL; - if (OB_UNLIKELY(!key.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key)); - } else if (OB_FAIL(get(key, value, handle.handle_))) { - if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { - STORAGE_LOG(WARN, "fail to get key-value from block cache", K(ret)); - } else { - EVENT_INC(ObStatEventIds::TMP_BLOCK_CACHE_MISS); - } - } else { - if (OB_ISNULL(value)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, the value must not be NULL", K(ret)); - } else { - handle.value_ = const_cast(value); - EVENT_INC(ObStatEventIds::TMP_BLOCK_CACHE_HIT); - } - } - return ret; -} - -int ObTmpBlockCache::alloc_buf(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(alloc(key.get_tenant_id(), key.size(), - sizeof(ObTmpBlockCacheValue) + ObTmpFileStore::get_block_size(), - handle.kvpair_, handle.handle_, handle.inst_handle_))) { - STORAGE_LOG(WARN, "failed to alloc kvcache buf", K(ret)); - } else if (OB_FAIL(key.deep_copy(reinterpret_cast(handle.kvpair_->key_), - key.size(), handle.kvpair_->key_))) { - STORAGE_LOG(WARN, "failed to deep copy key", K(ret), K(key)); - } else { - char *buf = reinterpret_cast(handle.kvpair_->value_); - handle.value_ = new (buf) ObTmpBlockCacheValue(buf + sizeof(ObTmpBlockCacheValue)); - } - return ret; -} - -int ObTmpBlockCache::put_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!key.is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), K(key)); - } else if (OB_FAIL(put_kvpair(handle.inst_handle_, handle.kvpair_, - handle.handle_, true/*overwrite*/))) { - STORAGE_LOG(WARN, "fail to put tmp block to block cache", K(ret)); - } else { - handle.reset(); - } - return ret; -} - -void ObTmpBlockCache::destroy() -{ - common::ObKVCache::destroy(); -} - -ObTmpFileWaitTask::ObTmpFileWaitTask(ObTmpTenantMemBlockManager &mgr) - : mgr_(mgr) -{ -} - -void ObTmpFileWaitTask::runTimerTask() -{ - int ret = OB_SUCCESS; - if (OB_FAIL(mgr_.exec_wait())) { - STORAGE_LOG(WARN, "fail to wait block", K(ret)); - } -} - -ObTmpFileMemTask::ObTmpFileMemTask(ObTmpTenantMemBlockManager &mgr) - : mgr_(mgr) -{ -} - -void ObTmpFileMemTask::runTimerTask() -{ - int ret = OB_SUCCESS; - if (OB_FAIL(mgr_.change_mem())) { - if (OB_EAGAIN != ret){ - STORAGE_LOG(WARN, "fail to wait block", K(ret)); - } - } -} - - -ObTmpTenantMemBlockManager::IOWaitInfo::IOWaitInfo( - ObMacroBlockHandle &block_handle, ObTmpMacroBlock &block, ObIAllocator &allocator) - : block_handle_(&block_handle), block_(block), allocator_(allocator), ref_cnt_(0), ret_code_(OB_SUCCESS) -{ -} - -void ObTmpTenantMemBlockManager::IOWaitInfo::inc_ref() -{ - ATOMIC_INC(&ref_cnt_); -} - -void ObTmpTenantMemBlockManager::IOWaitInfo::dec_ref() -{ - int ret = OB_SUCCESS; - const int64_t tmp_ref = ATOMIC_SAF(&ref_cnt_, 1); - if (tmp_ref < 0) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "bug: ref_cnt < 0", K(ret), K(tmp_ref), K(lbt())); - ob_abort(); - } else if (0 == tmp_ref) { - this->~IOWaitInfo(); - allocator_.free(this); - } -} - -int ObTmpTenantMemBlockManager::IOWaitInfo::wait(int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - if (block_.is_washing()) { - ObThreadCondGuard guard(cond_); - if (OB_FAIL(guard.get_ret())) { - STORAGE_LOG(ERROR, "fail to guard request condition", K(ret), K(block_.get_block_id())); - } else { - int64_t begin_us = ObTimeUtility::fast_current_time(); - int64_t wait_ms = 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; - } - } - if (OB_SUCC(ret) && OB_UNLIKELY(wait_ms <= 0)) { // rarely happen - ret = OB_TIMEOUT; - STORAGE_LOG(WARN, "fail to wait block io condition due to spurious wakeup", - K(ret), K(wait_ms), K(block_.get_block_id())); - } - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::IOWaitInfo::exec_wait() -{ - int ret = OB_SUCCESS; - ObThreadCondGuard guard(cond_); - if (OB_FAIL(guard.get_ret())) { - STORAGE_LOG(ERROR, "lock io request condition failed", K(ret), K(block_.get_block_id())); - } else if (OB_NOT_NULL(block_handle_) && OB_FAIL(block_handle_->wait())) { - STORAGE_LOG(WARN, "wait handle wait io failed", K(ret), K(block_.get_block_id())); - block_handle_->reset_macro_id(); - } - reset_io(); - return ret; -} - -int ObTmpTenantMemBlockManager::IOWaitInfo::broadcast() -{ - int ret = OB_SUCCESS; - ObThreadCondGuard guard(cond_); - if (OB_FAIL(guard.get_ret())) { - STORAGE_LOG(ERROR, "lock io request condition failed", K(ret), K(block_.get_block_id())); - } else if (OB_FAIL(cond_.broadcast())) { - STORAGE_LOG(WARN, "wait handle wait io failed", K(ret), K(block_.get_block_id())); - } - return ret; -} - -ObTmpTenantMemBlockManager::IOWaitInfo::~IOWaitInfo() -{ - destroy(); -} - -void ObTmpTenantMemBlockManager::IOWaitInfo::destroy() -{ - ret_code_ = OB_SUCCESS; - reset_io(); - 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() -{ - if (OB_NOT_NULL(block_handle_)) { - block_handle_->get_io_handle().reset(); - block_handle_ = nullptr; - } -} - -ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::ObIOWaitInfoHandle() - : wait_info_(nullptr) -{ -} - -ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::ObIOWaitInfoHandle(const ObIOWaitInfoHandle &other) - : wait_info_(nullptr) -{ - *this = other; -} - -ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::~ObIOWaitInfoHandle() -{ - reset(); -} - -void ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::set_wait_info(IOWaitInfo *wait_info) -{ - if (OB_NOT_NULL(wait_info)) { - reset(); - wait_info->inc_ref(); // ref for handle - wait_info_ = wait_info; - } -} - -ObTmpTenantMemBlockManager::ObIOWaitInfoHandle& -ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::operator=(const ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &other) -{ - if (&other != this) { - set_wait_info(other.wait_info_); - } - return *this; -} - -bool ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::is_empty() const -{ - return nullptr == wait_info_; -} - -bool ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::is_valid() const -{ - return nullptr != wait_info_; -} - -void ObTmpTenantMemBlockManager::ObIOWaitInfoHandle::reset() -{ - if (OB_NOT_NULL(wait_info_)) { - wait_info_->dec_ref(); // ref for handle - wait_info_ = nullptr; - } -} - -ObTmpTenantMemBlockManager::ObTmpTenantMemBlockManager(ObTmpTenantFileStore &tenant_store) - : tenant_store_(tenant_store), - wait_info_queue_(), - t_mblk_map_(), - dir_to_blk_map_(), - blk_nums_threshold_(0), - block_cache_(NULL), - allocator_(NULL), - tenant_id_(0), - last_access_tenant_config_ts_(0), - last_tenant_mem_block_num_(1), - is_inited_(false), - tg_id_(OB_INVALID_INDEX), - stopped_(true), - washing_count_(0), - wait_task_(*this), - mem_task_(*this), - io_lock_(), - map_lock_(), - cond_(), - compare_() -{ -} - -ObTmpTenantMemBlockManager::~ObTmpTenantMemBlockManager() -{ -} - -int ObTmpTenantMemBlockManager::init(const uint64_t tenant_id, - common::ObConcurrentFIFOAllocator &allocator, - double blk_nums_threshold) -{ - int ret = OB_SUCCESS; - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpBlockCache has been inited", K(ret)); - } else if (OB_UNLIKELY(blk_nums_threshold <= 0) || OB_UNLIKELY(blk_nums_threshold > 1)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(blk_nums_threshold)); - } else if (OB_FAIL(wait_handles_map_.create(MBLK_HASH_BUCKET_NUM, ObModIds::OB_TMP_BLOCK_MAP, - "WaitHdl", tenant_id))) { - STORAGE_LOG(WARN, "fail to create wait handles map", K(ret)); - } else if (OB_FAIL(t_mblk_map_.create(MBLK_HASH_BUCKET_NUM, ObModIds::OB_TMP_BLOCK_MAP, - "TmpMBlk", tenant_id))) { - STORAGE_LOG(WARN, "Fail to create allocating block map, ", K(ret)); - } else if (OB_FAIL(dir_to_blk_map_.create(MBLK_HASH_BUCKET_NUM, ObModIds::OB_TMP_MAP, - "DirToBlk", tenant_id))) { - STORAGE_LOG(WARN, "Fail to create tmp dir map, ", K(ret)); - } else if (OB_FAIL(map_lock_.init(MBLK_HASH_BUCKET_NUM, ObLatchIds::TMP_FILE_MEM_BLOCK_LOCK, "TmpMemBlkMgr", MTL_ID()))) { - STORAGE_LOG(WARN, "Fail to create tmp dir map, ", K(ret)); - } else if (OB_ISNULL(block_cache_ = &ObTmpBlockCache::get_instance())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to get the block cache", K(ret)); - } else if (OB_FAIL(cond_.init(ObWaitEventIds::IO_CONTROLLER_COND_WAIT))) { - STORAGE_LOG(WARN, "fail to init condition", K(ret)); - } else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::COMMON_TIMER_THREAD, tg_id_))) { - STORAGE_LOG(WARN, "TG_CREATE_TENANT failed", KR(ret)); - } else if (OB_FAIL(TG_START(tg_id_))) { - STORAGE_LOG(WARN, "TG_START failed", KR(ret), K_(tg_id)); - } else if (FALSE_IT(stopped_ = false)) { - } else if (OB_FAIL(TG_SCHEDULE(tg_id_, wait_task_, TASK_INTERVAL, true/*repeat*/))) { - STORAGE_LOG(WARN, "TG_SCHEDULE task failed", KR(ret), K_(tg_id), K(TASK_INTERVAL)); - } else if (OB_FAIL(TG_SCHEDULE(tg_id_, mem_task_, MEMORY_TASK_INTERVAL, true/*repeat*/))) { - STORAGE_LOG(WARN, "TG_SCHEDULE task failed", KR(ret), K_(tg_id), K(MEMORY_TASK_INTERVAL)); - } else { - blk_nums_threshold_ = blk_nums_threshold; - tenant_id_ = tenant_id; - allocator_ = &allocator; - is_inited_ = true; - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -int ObTmpTenantMemBlockManager::DestroyBlockMapOp::operator () (oceanbase::common::hash::HashMapPair &entry) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *blk = entry.second; - if (OB_ISNULL(blk)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "block is null", K(ret)); - } else if (blk->is_memory()) { - if (OB_FAIL(blk->check_and_set_status( - ObTmpMacroBlock::BlockStatus::MEMORY, ObTmpMacroBlock::BlockStatus::DISKED))) { - if (OB_STATE_NOT_MATCH == ret) { - STORAGE_LOG(DEBUG, "this block is washing", K(ret), K(*blk)); - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*blk)); - } - } else if (OB_FAIL(blk->give_back_buf_into_cache())) { - STORAGE_LOG(WARN, "fail to put tmp block cache", K(ret), K(blk)); - } else { - tenant_store_.dec_block_cache_num(1); - } - } else { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "block status not correct", K(ret), K(blk)); - } - return ret; -} - -void ObTmpTenantMemBlockManager::destroy() -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *tmp = NULL; - stopped_ = true; - if (OB_INVALID_INDEX != tg_id_) { - TG_STOP(tg_id_); - TG_WAIT(tg_id_); - TG_DESTROY(tg_id_); - tg_id_ = OB_INVALID_INDEX; - } - const int64_t io_timeout_ms = GCONF._data_storage_io_timeout / 1000L; - ObSpLinkQueue::Link *node = NULL; - while (!wait_info_queue_.is_empty()) { - IOWaitInfo *wait_info = NULL; - if (OB_FAIL(wait_info_queue_.pop(node))) { - STORAGE_LOG(WARN, "pop wait handle failed", K(ret)); - } else if (FALSE_IT(wait_info = static_cast(node))) { - } else if (OB_FAIL(wait_info->exec_wait())) { - // overwrite ret - STORAGE_LOG(WARN, "fail to exec iohandle wait", K(ret), K_(tenant_id)); - } - } - ATOMIC_STORE(&washing_count_, 0); - DestroyBlockMapOp op(tenant_store_); - if (OB_FAIL(t_mblk_map_.foreach_refactored(op))) { - // overwrite ret - STORAGE_LOG(WARN, "destroy mblk map failed", K(ret)); - } - t_mblk_map_.destroy(); - blk_nums_threshold_ = 0; - dir_to_blk_map_.destroy(); - if (NULL != block_cache_) { - block_cache_ = NULL; - } - map_lock_.destroy(); - allocator_ = NULL; - is_inited_ = false; -} - -int ObTmpTenantMemBlockManager::get_block(const ObTmpBlockCacheKey &key, - ObTmpBlockValueHandle &handle) -{ - return block_cache_->get_block(key, handle); -} - -int ObTmpTenantMemBlockManager::alloc_buf(const ObTmpBlockCacheKey &key, - ObTmpBlockValueHandle &handle) -{ - return block_cache_->alloc_buf(key, handle); -} - -int ObTmpTenantMemBlockManager::free_macro_block(const int64_t block_id) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *t_mblk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", 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 tmp macro block", K(ret)); - } else { - ret = OB_SUCCESS; - STORAGE_LOG(DEBUG, "macro block has been erased", K(ret)); - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::alloc_extent(const int64_t dir_id, const uint64_t tenant_id, - const int64_t size, ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - const int64_t page_nums = std::ceil(size * 1.0 / ObTmpMacroBlock::get_default_page_size()); - int64_t block_id = -1; - ObTmpMacroBlock *t_mblk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_FAIL(get_block_from_dir_cache(dir_id, tenant_id, page_nums, t_mblk))) { - bool is_found = false; - if (OB_FAIL(get_available_macro_block(dir_id, tenant_id, page_nums, t_mblk, is_found))) { - } else if (!is_found) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(DEBUG, "cannot find available macro block", K(ret)); - } - } - - if (OB_SUCC(ret)) { - if (OB_ISNULL(t_mblk)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, t_mblk is nullptr", K(ret), KP(t_mblk)); - } else if (OB_FAIL(t_mblk->alloc(page_nums, extent))){ - STORAGE_LOG(WARN, "fail to alloc tmp extent", K(ret)); - } else if (OB_FAIL(refresh_dir_to_blk_map(dir_id, t_mblk))) { - STORAGE_LOG(WARN, "fail to refresh dir_to_blk_map", K(ret), K(*t_mblk)); - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::get_block_from_dir_cache(const int64_t dir_id, const int64_t tenant_id, - const int64_t page_nums, ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - int64_t block_id = -1; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_FAIL(dir_to_blk_map_.get_refactored(dir_id, block_id))) { - STORAGE_LOG(DEBUG, "fail to get macro block from dir cache", K(ret), K(dir_id), K(block_id), K(dir_to_blk_map_.size())); - } else if (OB_FAIL(t_mblk_map_.get_refactored(block_id, t_mblk))) { - STORAGE_LOG(DEBUG, "the tmp macro block has been washed", K(ret), K(block_id)); - } else if (OB_ISNULL(t_mblk)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "block is null", K(ret)); - } else if (t_mblk->get_max_cont_page_nums() < page_nums - || t_mblk->get_tenant_id() != tenant_id - || !t_mblk->is_memory()) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(DEBUG, "the block is not suitable", K(ret), K(block_id), - "block_page_nums", t_mblk->get_max_cont_page_nums(), K(page_nums), - "block_tenant_id", t_mblk->get_tenant_id(), K(tenant_id), - "block_status", t_mblk->get_block_status()); - } - return ret; -} - -int ObTmpTenantMemBlockManager::GetAvailableBlockMapOp::operator () (oceanbase::common::hash::HashMapPair &entry) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *blk = entry.second; - if (!is_found_) { - if (OB_ISNULL(blk)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, blk is nullptr", K(ret), KP(blk)); - } else if (tenant_id_ != blk->get_tenant_id() || dir_id_ != blk->get_dir_id()) { - // do nothing - } else { - if (blk->get_max_cont_page_nums() < page_nums_ - || blk->get_block_status() != ObTmpMacroBlock::BlockStatus::MEMORY) { - // do nothing - } else { - block_ = blk; - is_found_ = true; - } - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::get_available_macro_block(const int64_t dir_id, const uint64_t tenant_id, - const int64_t page_nums, ObTmpMacroBlock *&t_mblk, - bool &is_found) -{ - int ret = OB_SUCCESS; - is_found = false; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else { - GetAvailableBlockMapOp op(dir_id, tenant_id, page_nums, t_mblk, is_found); - if (OB_FAIL(t_mblk_map_.foreach_refactored(op))) { - STORAGE_LOG(WARN, "get available macro block failed", K(ret)); - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::check_memory_limit() -{ - int ret = OB_SUCCESS; - const int64_t timeout_ts = THIS_WORKER.get_timeout_ts(); - while (OB_SUCC(ret) && get_tenant_mem_block_num() < t_mblk_map_.size() && !wait_info_queue_.is_empty()) { - ObThreadCondGuard guard(cond_); - if (OB_FAIL(guard.get_ret())) { - STORAGE_LOG(ERROR, "fail to guard request condition", K(ret)); - } else { - int64_t wait_ms = (timeout_ts - ObTimeUtility::current_time()) / 1000; - while (OB_SUCC(ret) - && get_tenant_mem_block_num() < t_mblk_map_.size() - && !wait_info_queue_.is_empty() - && wait_ms > 0) { - if (OB_FAIL(cond_.wait(wait_ms))) { - STORAGE_LOG(WARN, "fail to wait block write condition", K(ret), K(wait_ms)); - } else if (get_tenant_mem_block_num() < t_mblk_map_.size()) { - wait_ms = (timeout_ts - ObTimeUtility::current_time()) / 1000; - } - } - - if (OB_SUCC(ret) && OB_UNLIKELY(wait_ms <= 0)) { - ret = OB_TIMEOUT; - STORAGE_LOG(WARN, "fail to wait block io condition due to spurious wakeup", - K(ret), K(wait_ms), K(timeout_ts), K(ObTimeUtility::current_time())); - } - } - } - return ret; -} -bool ObTmpTenantMemBlockManager::check_block_full() -{ - return get_tenant_mem_block_num() < t_mblk_map_.size(); -} - -ObTmpTenantMemBlockManager::BlockWashScoreCompare::BlockWashScoreCompare() -{ -} - -bool ObTmpTenantMemBlockManager::BlockWashScoreCompare::operator() ( - const ObTmpTenantMemBlockManager::BlockInfo &left, - const ObTmpTenantMemBlockManager::BlockInfo &right) -{ - return left.wash_score_ < right.wash_score_; -} - -int ObTmpTenantMemBlockManager::cleanup() -{ - int ret = OB_SUCCESS; - ObArenaAllocator allocator(common::ObMemAttr(MTL_ID(), "TmpFileRank")); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_FAIL(check_memory_limit())) { - STORAGE_LOG(WARN, "fail to check memory limit", K(ret), K(t_mblk_map_.size()), K(get_tenant_mem_block_num())); - } else { - const int64_t wash_threshold = get_tenant_mem_block_num() * 0.8; - Heap heap(compare_, &allocator); - ChooseBlocksMapOp op(heap, ObTimeUtility::fast_current_time()); - 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))) { - STORAGE_LOG(WARN, "choose blks failed", K(ret)); - } else { - const int64_t candidate_cnt = heap.count(); - bool wash_success = false; - while (OB_SUCC(ret) && heap.count() > 0 && !wash_success) { - const BlockInfo info = heap.top(); - ObIOWaitInfoHandle handle; - if (OB_FAIL(wash_block(info.block_id_, handle))) { - STORAGE_LOG(WARN, "fail to wash", K(ret), K_(tenant_id), K(info.block_id_)); - } else { - wash_success = handle.is_valid(); - STORAGE_LOG(DEBUG, "succeed to wash block for cleanup", K(info)); - } - - if (OB_SUCC(ret)) { - if (OB_FAIL(heap.pop())) { - STORAGE_LOG(WARN, "pop info from heap failed", K(ret), K_(tenant_id)); - } - } - } - if (OB_SUCC(ret) && !wash_success) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(WARN, "fail to cleanup", K(ret), K(t_mblk_map_.size()), K(candidate_cnt)); - } - } - } - return ret; -} - -int ObTmpTenantMemBlockManager::free_empty_blocks(common::ObIArray &free_blocks) -{ - int ret = OB_SUCCESS; - if (free_blocks.count() > 0) { - for (int64_t i = 0; OB_SUCC(ret) && i < free_blocks.count(); ++i) { - ObTmpMacroBlock* blk = free_blocks.at(i); - if (blk->is_empty()) { - if (OB_FAIL(free_macro_block(blk->get_block_id()))) { - STORAGE_LOG(WARN, "fail to free tmp macro block", K(ret)); - } - } - free_blocks.at(i) = NULL; - } - free_blocks.reset(); - } - return ret; -} - -int ObTmpTenantMemBlockManager::check_and_free_mem_block(ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - uint64_t hash_val = 0; - int64_t block_id = t_mblk->get_block_id(); - hash_val = murmurhash(&block_id, sizeof(block_id), hash_val); - ObBucketHashWLockGuard lock_guard(map_lock_, hash_val); - if (OB_FAIL(t_mblk->check_and_set_status( - ObTmpMacroBlock::BlockStatus::MEMORY, ObTmpMacroBlock::BlockStatus::DISKED))) { - if (OB_STATE_NOT_MATCH == ret) { - STORAGE_LOG(DEBUG, "this block is washing", K(ret), K(*t_mblk)); - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*t_mblk)); - } - } else if (OB_FAIL(t_mblk->give_back_buf_into_cache())) { - STORAGE_LOG(WARN, "fail to put tmp block cache", K(ret), K(t_mblk)); - } else if (OB_FAIL(free_macro_block(t_mblk->get_block_id()))) { - STORAGE_LOG(WARN, "fail to free tmp macro block for block cache", K(ret)); - } else { - tenant_store_.dec_block_cache_num(1); - } - return ret; -} - -int ObTmpTenantMemBlockManager::ChooseBlocksMapOp::operator () (oceanbase::common::hash::HashMapPair &entry) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *blk = entry.second; - if (OB_LIKELY(NULL != blk) && OB_LIKELY(blk->is_inited()) && OB_LIKELY(blk->is_memory()) - && OB_LIKELY(0 != blk->get_used_page_nums())) { - BlockInfo info; - info.block_id_ = blk->get_block_id(); - info.wash_score_ = blk->get_wash_score(cur_time_); - if(OB_FAIL(heap_.push(info))) { - STORAGE_LOG(WARN, "insert block to array failed", K(ret)); - } - } - STORAGE_LOG(DEBUG, "choose one block", K(ret), KPC(blk)); - return ret; -} - -int ObTmpTenantMemBlockManager::add_macro_block(ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_FAIL(t_mblk_map_.set_refactored(t_mblk->get_block_id(), t_mblk))) { - STORAGE_LOG(WARN, "fail to set tmp macro block map", K(ret), K(t_mblk)); - } - return ret; -} - -int ObTmpTenantMemBlockManager::refresh_dir_to_blk_map(const int64_t dir_id, - const ObTmpMacroBlock *t_mblk) -{ - int ret = OB_SUCCESS; - int64_t block_id = 0; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpBlockCache has not been inited", K(ret)); - } else if (OB_FAIL(dir_to_blk_map_.get_refactored(dir_id, block_id))) { - if (OB_HASH_NOT_EXIST == ret) { - if (OB_FAIL(dir_to_blk_map_.set_refactored(dir_id, t_mblk->get_block_id(), 1))) { - STORAGE_LOG(WARN, "fail to set dir_to_blk_map_", K(ret), K(dir_id), K(t_mblk->get_block_id())); - } - } - } else { - ObTmpMacroBlock *dir_mblk = NULL; - if (OB_FAIL(t_mblk_map_.get_refactored(block_id, dir_mblk))) { - if (OB_HASH_NOT_EXIST == ret) { - STORAGE_LOG(DEBUG, "the tmp macro block has been removed or washed", K(ret), K(block_id)); - if (OB_FAIL(dir_to_blk_map_.set_refactored(dir_id, t_mblk->get_block_id(), 1))) { - STORAGE_LOG(WARN, "fail to set dir_to_blk_map_", K(ret), K(dir_id), K(t_mblk->get_block_id())); - } - } else { - STORAGE_LOG(WARN, "fail to get block", K(ret), K(block_id)); - } - } else if (dir_mblk->get_max_cont_page_nums() < t_mblk->get_max_cont_page_nums()) { - if (OB_FAIL(dir_to_blk_map_.set_refactored(dir_id, t_mblk->get_block_id(), 1))) { - STORAGE_LOG(WARN, "fail to set dir_to_blk_map_", K(ret), K(dir_id), K(t_mblk->get_block_id())); - } - } - } - - return ret; -} - -int ObTmpTenantMemBlockManager::get_block_and_set_washing(int64_t block_id, ObTmpMacroBlock *&m_blk) -{ - int ret = OB_SUCCESS; - bool is_sealed = false; - uint64_t hash_val = 0; - hash_val = murmurhash(&block_id, sizeof(block_id), hash_val); - ObBucketHashRLockGuard lock_guard(map_lock_, hash_val); - if (OB_FAIL(t_mblk_map_.get_refactored(block_id, m_blk))) { - STORAGE_LOG(DEBUG, "tenant mem block manager get block failed", K(ret), K(block_id)); - } else if (OB_ISNULL(m_blk)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "the block is null", K(ret), K(*m_blk)); - } else if (OB_UNLIKELY(!m_blk->is_inited())) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(WARN, "the block has not been inited", K(ret), K(*m_blk)); - } else if (OB_UNLIKELY(!m_blk->is_memory())) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(DEBUG, "the block has been disked or washing", K(ret), K(*m_blk)); - } else if (m_blk->is_empty()) { - 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)); - } - // refresh ret can be ignored. overwrite the ret. - ret = OB_STATE_NOT_MATCH; - } else if (OB_FAIL(m_blk->seal(is_sealed))) { - STORAGE_LOG(WARN, "fail to seal block", K(ret), K(*m_blk)); - } else if (OB_UNLIKELY(!is_sealed)) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(WARN, "the block has some unclosed extents", K(ret), K(is_sealed), K(*m_blk)); - } else if (OB_UNLIKELY(0 == m_blk->get_used_page_nums())) { - ret = OB_STATE_NOT_MATCH; - 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))) { - if (OB_STATE_NOT_MATCH != ret) { - STORAGE_LOG(WARN, "check and set status failed", K(ret), K(*m_blk)); - } - } - - return ret; -} - -int ObTmpTenantMemBlockManager::wash_block(const int64_t block_id, ObIOWaitInfoHandle &handle) -{ - int ret = OB_SUCCESS; - handle.reset(); - IOWaitInfo *wait_info = NULL; - ObTmpMacroBlock *m_blk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "the tenant mem block manager not been inited", K(ret)); - } else if (OB_FAIL(get_block_and_set_washing(block_id, m_blk))) { - if (OB_HASH_NOT_EXIST == ret || OB_STATE_NOT_MATCH == ret) { - ret = OB_SUCCESS; - STORAGE_LOG(DEBUG, "this block may be removed, washed or disked", K(ret), K(block_id)); - } else { - STORAGE_LOG(WARN, "check and set washing failed", K(ret), K(block_id)); - } - } else { - ObTmpBlockIOInfo info; - char *buf = NULL; - SpinWLockGuard io_guard(io_lock_); - ObMacroBlockHandle &mb_handle = m_blk->get_macro_block_handle(); - if (OB_FAIL(m_blk->get_wash_io_info(info))) { - 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))))) { - 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_))) { - } else if (OB_FAIL(wait_info->cond_.init(ObWaitEventIds::IO_CONTROLLER_COND_WAIT))) { - STORAGE_LOG(WARN, "fail to init condition", K(ret), K_(tenant_id)); - } else if (FALSE_IT(handle.set_wait_info(wait_info))) { - } else if (OB_FAIL(wait_handles_map_.set_refactored(m_blk->get_block_id(), handle))) { - 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))) { - 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 { - ATOMIC_INC(&washing_count_); - } - if (OB_FAIL(ret) && OB_NOT_NULL(m_blk)) { - mb_handle.reset(); - // 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; - } - 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; -} - -int ObTmpTenantMemBlockManager::write_io( - const ObTmpBlockIOInfo &io_info, - ObMacroBlockHandle &handle) -{ - int ret = OB_SUCCESS; - const int64_t buf_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - const int64_t page_size = ObTmpMacroBlock::get_default_page_size(); - int64_t pos = 0; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); - } else if (OB_FAIL(THE_IO_DEVICE->check_space_full(OB_SERVER_BLOCK_MGR.get_macro_block_size()))) { - STORAGE_LOG(WARN, "fail to check space full", K(ret)); - } else { - ObMacroBlockWriteInfo write_info; - write_info.io_desc_ = io_info.io_desc_; - write_info.buffer_ = io_info.buf_; - write_info.offset_ = ObTmpMacroBlock::get_header_padding(); - write_info.size_ = io_info.size_; - write_info.io_timeout_ms_ = io_info.io_timeout_ms_; - write_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - write_info.io_desc_.set_sys_module_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.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()); - } - } - return ret; -} - -int64_t ObTmpTenantMemBlockManager::get_tenant_mem_block_num() -{ - int64_t tenant_mem_block_num = TENANT_MEM_BLOCK_NUM; - int64_t last_access_ts = ATOMIC_LOAD(&last_access_tenant_config_ts_); - if (last_access_ts > 0 - && common::ObClockGenerator::getClock() - last_access_ts < 10000000) { - tenant_mem_block_num = ATOMIC_LOAD(&last_tenant_mem_block_num_); - } else { - omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id_)); - if (!tenant_config.is_valid()) { - COMMON_LOG(INFO, "failed to get tenant config", K_(tenant_id)); - } else if (0 == tenant_config->_temporary_file_io_area_size) { - tenant_mem_block_num = 1L; - } else { - const int64_t bytes = common::upper_align( - lib::get_tenant_memory_limit(tenant_id_) * tenant_config->_temporary_file_io_area_size / 100, - ObTmpFileStore::get_block_size()); - tenant_mem_block_num = bytes / ObTmpFileStore::get_block_size(); - } - ATOMIC_STORE(&last_tenant_mem_block_num_, tenant_mem_block_num); - ATOMIC_STORE(&last_access_tenant_config_ts_, common::ObClockGenerator::getClock()); - } - return tenant_mem_block_num; -} - -int ObTmpTenantMemBlockManager::exec_wait() -{ - int ret = OB_SUCCESS; - 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; - SpinWLockGuard io_guard(io_lock_); - const int64_t begin_us = ObTimeUtility::fast_current_time(); - while (OB_SUCC(ret) && (ObTimeUtility::fast_current_time() - begin_us)/1000 < TASK_INTERVAL) { - IOWaitInfo *wait_info = NULL; - if (OB_FAIL(wait_info_queue_.pop(node))) { - if (OB_EAGAIN != ret) { - STORAGE_LOG(WARN, "fail to pop wait info from queue", K(ret)); - } - } else if (FALSE_IT(++loop_nums)) { - } else if (OB_ISNULL(wait_info = static_cast(node))) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected error, wait info is nullptr", K(ret), KP(node)); - } else if (OB_ISNULL(wait_info->block_handle_)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "unexpected error, macro handle in wait info is nullptr", K(ret), KPC(wait_info)); - } else { - 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 free_page_nums = blk.get_free_page_nums(); - if (OB_FAIL(wait_info->exec_wait())) { - STORAGE_LOG(WARN, "fail to exec io handle wait", K(ret), K_(tenant_id), KPC(wait_info)); - ATOMIC_DEC(&washing_count_); - int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(blk.check_and_set_status(ObTmpMacroBlock::WASHING, ObTmpMacroBlock::MEMORY))) { - STORAGE_LOG(ERROR, "fail to rollback block status", K(ret), K(tmp_ret), K(block_id), K(blk)); - } - } else { - 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 { - 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; - tenant_store_.dec_block_cache_num(1); - ObTaskController::get().allow_next_syslog(); - STORAGE_LOG(INFO, "succeed to wash a block", K(block_id), K(macro_id), - K(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)); - } - } - } - } - if (OB_EAGAIN == ret) { - ret = OB_SUCCESS; - } - } - if (OB_SUCC(ret)) { - int tmp_ret = OB_SUCCESS; - if (OB_TMP_FAIL(cond_.broadcast())) { - STORAGE_LOG(ERROR, "signal wash condition failed", K(ret), K(tmp_ret)); - } - if (loop_nums > 0 || REACH_TIME_INTERVAL(1000 * 1000L)/*1s*/) { - const int64_t washing_count = ATOMIC_LOAD(&washing_count_); - int64_t block_cache_num = -1; - int64_t page_cache_num = -1; - block_cache_num = tenant_store_.get_block_cache_num(); - page_cache_num = tenant_store_.get_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; -} - -int ObTmpTenantMemBlockManager::change_mem() -{ - int ret = OB_SUCCESS; - // Here, this memory is used to store temporary file block metadata, which is related to the - // datafile size. So, we set the upper limit of memory to be percentage (default, 70%) of tenant memory to - // avoid excessive tenant memory, and affecting system stability. In theory, the limit - // will be reached only when the tenant's memory is extremely small and the disk is extremely - // large. - tenant_store_.refresh_memory_limit(tenant_id_); - return ret; -} - -int ObTmpTenantMemBlockManager::wait_write_finish(const int64_t block_id, const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - const int64_t wash_to_flush_wait_interval = 1000; // 1ms - ObIOWaitInfoHandle handle; - ObTmpMacroBlock *blk = nullptr; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else if (timeout_ms < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "Invalid argument, ", K(timeout_ms), K(ret)); - } else if (OB_FAIL(t_mblk_map_.get_refactored(block_id, blk))) { - if (OB_HASH_NOT_EXIST == ret) { - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "cannot find tmp block", K(ret), K(block_id)); - } - } else { - bool is_found = false; - while (OB_SUCC(ret) && blk->is_washing() && !is_found) { - if (OB_FAIL(wait_handles_map_.get_refactored(block_id, handle))) { - if (OB_HASH_NOT_EXIST == ret) { - ret = OB_SUCCESS; - } else { - STORAGE_LOG(WARN, "cannot find wait hanlde", K(ret), K(block_id)); - } - } else { - is_found = true; - } - } - - if (OB_SUCC(ret) && is_found) { - if (OB_FAIL(handle.get_wait_info()->wait(timeout_ms))) { - STORAGE_LOG(WARN, "wait write io finish failed", K(ret), K(block_id)); - } - } - } - - return ret; -} - - -} // end namespace blocksstable -} // end namespace oceanbase diff --git a/src/storage/blocksstable/ob_tmp_file_cache.h b/src/storage/blocksstable/ob_tmp_file_cache.h deleted file mode 100644 index c66878e4fc..0000000000 --- a/src/storage/blocksstable/ob_tmp_file_cache.h +++ /dev/null @@ -1,474 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_CACHE_H_ -#define OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_CACHE_H_ - -#include "lib/hash/ob_hashmap.h" -#include "lib/queue/ob_link_queue.h" -#include "share/io/ob_io_manager.h" -#include "share/cache/ob_kv_storecache.h" -#include "storage/ob_i_store.h" - -namespace oceanbase -{ -namespace blocksstable -{ - -struct ObTmpBlockIOInfo; -struct ObTmpFileMacroBlockHeader; -class ObTmpFileIOHandle; -class ObTmpMacroBlock; -class ObTmpFileExtent; -class ObMacroBlockHandle; -class ObTmpTenantFileStore; - -class ObTmpPageCacheKey final : public common::ObIKVCacheKey -{ -public: - ObTmpPageCacheKey(); - ObTmpPageCacheKey(const int64_t block_id, const int64_t page_id, const uint64_t tenant_id); - ~ObTmpPageCacheKey(); - bool operator ==(const ObIKVCacheKey &other) const override; - uint64_t get_tenant_id() const override; - uint64_t hash() const override; - int64_t size() const override; - int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const override; - bool is_valid() const; - int64_t get_page_id() const { return page_id_; } - TO_STRING_KV(K_(block_id), K_(page_id), K_(tenant_id)); - -private: - int64_t block_id_; - int64_t page_id_; - uint64_t tenant_id_; -}; - -class ObTmpPageCacheValue final : public common::ObIKVCacheValue -{ -public: - explicit ObTmpPageCacheValue(char *buf); - ~ObTmpPageCacheValue(); - int64_t size() const override; - int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const override; - bool is_valid() const { return NULL != buf_ && size() > 0; } - char *get_buffer() { return buf_; } - void set_buffer(char *buf) { buf_ = buf;} - TO_STRING_KV(KP_(buf), K_(size)); - -private: - char *buf_; - int64_t size_; - DISALLOW_COPY_AND_ASSIGN(ObTmpPageCacheValue); -}; - -struct ObTmpPageValueHandle final -{ -public: - ObTmpPageValueHandle() : value_(NULL), handle_() {} - ~ObTmpPageValueHandle() = default; - void reset() - { - handle_.reset(); - value_ = NULL; - } - TO_STRING_KV(KP_(value), K_(handle)); - ObTmpPageCacheValue *value_; - common::ObKVCacheHandle handle_; -}; - -struct ObTmpPageIOInfo final -{ -public: - ObTmpPageIOInfo() : offset_(0), size_(0), key_() {} - ~ObTmpPageIOInfo() {} - TO_STRING_KV(K_(key), K_(offset), K_(size)); - - int32_t offset_; - int32_t size_; - ObTmpPageCacheKey key_; -}; - -class ObTmpPageCache final : public common::ObKVCache -{ -public: - typedef common::ObKVCache BasePageCache; - static ObTmpPageCache &get_instance(); - int init(const char *cache_name, const int64_t priority); - int direct_read(const ObTmpBlockIOInfo &info, ObMacroBlockHandle &mb_handle, common::ObIAllocator &allocator); - int prefetch( - const ObTmpPageCacheKey &key, - const ObTmpBlockIOInfo &info, - ObMacroBlockHandle &mb_handle, - common::ObIAllocator &allocator); - // multi page prefetch - int prefetch( - const ObTmpBlockIOInfo &info, - const common::ObIArray &page_io_infos, - ObMacroBlockHandle &mb_handle, - common::ObIAllocator &allocator); - int get_cache_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle); - int get_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle); - int put_page(const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value); - void destroy(); -public: - class ObITmpPageIOCallback : public common::ObIOCallback - { - public: - ObITmpPageIOCallback(); - virtual ~ObITmpPageIOCallback(); - virtual int alloc_data_buf(const char *io_data_buffer, const int64_t data_size) override; - protected: - friend class ObTmpPageCache; - virtual int process_page(const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value); - virtual ObIAllocator *get_allocator() { return allocator_; } - - protected: - BasePageCache *cache_; - common::ObIAllocator *allocator_; - int64_t offset_; // offset in block - char *data_buf_; // actual data buffer - }; - - class ObTmpPageIOCallback final : public ObITmpPageIOCallback - { - public: - ObTmpPageIOCallback(); - ~ObTmpPageIOCallback() override; - int64_t size() const override; - int inner_process(const char *data_buffer, const int64_t size) override; - const char *get_data() override; - TO_STRING_KV("callback_type:", "ObTmpPageIOCallback", KP_(data_buf)); - DISALLOW_COPY_AND_ASSIGN(ObTmpPageIOCallback); - private: - friend class ObTmpPageCache; - ObTmpPageCacheKey key_; - }; - class ObTmpMultiPageIOCallback final : public ObITmpPageIOCallback - { - public: - ObTmpMultiPageIOCallback(); - ~ObTmpMultiPageIOCallback() override; - int64_t size() const override; - int inner_process(const char *data_buffer, const int64_t size) override; - const char *get_data() override; - TO_STRING_KV("callback_type:", "ObTmpMultiPageIOCallback", KP_(data_buf)); - DISALLOW_COPY_AND_ASSIGN(ObTmpMultiPageIOCallback); - private: - friend class ObTmpPageCache; - common::ObArray page_io_infos_; - }; - class ObTmpDirectReadPageIOCallback final : public ObITmpPageIOCallback - { - public: - ObTmpDirectReadPageIOCallback() {} - ~ObTmpDirectReadPageIOCallback() override {} - int64_t size() const override; - int inner_process(const char *data_buffer, const int64_t size) override; - const char *get_data() override; - TO_STRING_KV("callback_type:", "ObTmpDirectReadPageIOCallback", KP_(data_buf)); - DISALLOW_COPY_AND_ASSIGN(ObTmpDirectReadPageIOCallback); - }; -private: - ObTmpPageCache(); - ~ObTmpPageCache(); - int inner_read_io(const ObTmpBlockIOInfo &io_info, - ObITmpPageIOCallback *callback, - ObMacroBlockHandle &handle); - int read_io(const ObTmpBlockIOInfo &io_info, - ObITmpPageIOCallback *callback, - ObMacroBlockHandle &handle); - -private: - DISALLOW_COPY_AND_ASSIGN(ObTmpPageCache); -}; - -class ObTmpBlockCacheKey final : public common::ObIKVCacheKey -{ -public: - ObTmpBlockCacheKey(const int64_t block_id, const uint64_t tenant_id); - ~ObTmpBlockCacheKey() {} - bool operator ==(const ObIKVCacheKey &other) const override; - uint64_t get_tenant_id() const override; - uint64_t hash() const override; - int64_t size() const override; - int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const override; - int64_t get_block_id() const { return block_id_; } - bool is_valid() const { return block_id_ > 0 && tenant_id_ > 0 && size() > 0; } - TO_STRING_KV(K_(block_id), K_(tenant_id)); - -private: - int64_t block_id_; - uint64_t tenant_id_; - friend class ObTmpBlockCache; - DISALLOW_COPY_AND_ASSIGN(ObTmpBlockCacheKey); -}; - -class ObTmpBlockCacheValue final : public common::ObIKVCacheValue -{ -public: - explicit ObTmpBlockCacheValue(char *buf); - ~ObTmpBlockCacheValue() {} - int64_t size() const override; - int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const override; - bool is_valid() const { return NULL != buf_ && size() > 0; } - char *get_buffer() { return buf_; } - TO_STRING_KV(K_(size)); - -private: - char *buf_; - int64_t size_; - DISALLOW_COPY_AND_ASSIGN(ObTmpBlockCacheValue); -}; - -struct ObTmpBlockValueHandle final -{ -public: - ObTmpBlockValueHandle() - : value_(NULL), inst_handle_(), kvpair_(NULL), handle_(){} - ~ObTmpBlockValueHandle() = default; - void reset() - { - handle_.reset(); - inst_handle_.reset(); - value_ = NULL; - kvpair_ = NULL; - } - TO_STRING_KV(KP_(value), K_(inst_handle), KP_(kvpair), K_(handle)); - ObTmpBlockCacheValue *value_; - ObKVCacheInstHandle inst_handle_; - ObKVCachePair *kvpair_; - common::ObKVCacheHandle handle_; -}; - -class ObTmpBlockCache final: public common::ObKVCache -{ -public: - static ObTmpBlockCache &get_instance(); - int init(const char *cache_name, const int64_t priority); - int get_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); - int alloc_buf(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); - int put_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); - void destroy(); - -private: - ObTmpBlockCache() {} - ~ObTmpBlockCache() {} - - DISALLOW_COPY_AND_ASSIGN(ObTmpBlockCache); -}; - -class ObTmpTenantMemBlockManager; - -class ObTmpFileWaitTask : public common::ObTimerTask -{ -public: - explicit ObTmpFileWaitTask(ObTmpTenantMemBlockManager &mgr); - virtual ~ObTmpFileWaitTask() {} - virtual void runTimerTask() override; -private: - ObTmpTenantMemBlockManager &mgr_; -}; - -class ObTmpFileMemTask : public common::ObTimerTask -{ -public: - explicit ObTmpFileMemTask(ObTmpTenantMemBlockManager &mgr); - virtual ~ObTmpFileMemTask() {} - virtual void runTimerTask() override; -private: - ObTmpTenantMemBlockManager &mgr_; -}; - -class ObTmpTenantMemBlockManager final -{ -public: - struct IOWaitInfo :public common::ObSpLinkQueue::Link - { - public: - IOWaitInfo(ObMacroBlockHandle &block_handle, ObTmpMacroBlock &block, ObIAllocator &allocator); - virtual ~IOWaitInfo(); - void inc_ref(); - void dec_ref(); - int wait(const int64_t timout_ms); - int exec_wait(); - void reset_io(); - int broadcast(); - OB_INLINE ObTmpMacroBlock& get_block() { return block_; }; - TO_STRING_KV(K_(block), KPC_(block_handle), K_(ref_cnt), K_(ret_code)); - - private: - void destroy(); - - public: - ObMacroBlockHandle *block_handle_; - ObTmpMacroBlock &block_; - ObThreadCond cond_; - ObIAllocator &allocator_; - volatile int64_t ref_cnt_; - int64_t ret_code_; - private: - DISALLOW_COPY_AND_ASSIGN(IOWaitInfo); - }; - - struct ObIOWaitInfoHandle final - { - public: - ObIOWaitInfoHandle(); - ~ObIOWaitInfoHandle(); - ObIOWaitInfoHandle(const ObIOWaitInfoHandle &other); - ObIOWaitInfoHandle &operator=(const ObIOWaitInfoHandle &other); - void set_wait_info(IOWaitInfo *wait_info); - bool is_empty() const; - bool is_valid() const; - void reset(); - OB_INLINE IOWaitInfo* get_wait_info() const { return wait_info_; }; - TO_STRING_KV(KPC_(wait_info)); - - private: - IOWaitInfo *wait_info_; - }; - - explicit ObTmpTenantMemBlockManager(ObTmpTenantFileStore &tenant_store); - ~ObTmpTenantMemBlockManager(); - int init(const uint64_t tenant_id, - common::ObConcurrentFIFOAllocator &allocator, - double blk_nums_threshold = DEFAULT_MIN_FREE_BLOCK_RATIO); - void destroy(); - int get_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); - int alloc_buf(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); - int alloc_extent(const int64_t dir_id, const uint64_t tenant_id, const int64_t size, ObTmpFileExtent &extent); - 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 cleanup(); - int add_macro_block(ObTmpMacroBlock *&t_mblk); - int wait_write_finish(const int64_t block_id, const int64_t timeout_ms); - static int64_t get_default_timeout_ms() {return GCONF._data_storage_io_timeout / 1000L;} - int free_empty_blocks(common::ObIArray &free_blocks); - int refresh_dir_to_blk_map(const int64_t dir_id, const ObTmpMacroBlock *t_mblk); - int check_and_free_mem_block(ObTmpMacroBlock *&blk); - bool check_block_full(); - - int exec_wait(); - int change_mem(); - -private: - // 1/256, only one free block each 256 block. - static constexpr double DEFAULT_MIN_FREE_BLOCK_RATIO = 0.00390625; - static const uint64_t DEFAULT_BUCKET_NUM = 1543L; - static const uint64_t MBLK_HASH_BUCKET_NUM = 10243L; - static const int64_t TENANT_MEM_BLOCK_NUM = 64L; - const int64_t TASK_INTERVAL = 10 * 1000; // 10 ms - const int64_t MEMORY_TASK_INTERVAL = 1000 * 1000; // 1 s - typedef common::hash::ObHashMap - TmpMacroBlockMap; - typedef common::hash::ObHashMap Map; - typedef common::hash::ObHashMap WaitHandleMap; - - struct BlockInfo final - { - public: - BlockInfo() :block_id_(0), wash_score_(INT64_MIN) {}; - ~BlockInfo() = default; - - TO_STRING_KV(K_(block_id), K_(wash_score)); - - int64_t block_id_; - double wash_score_; - }; - - class BlockWashScoreCompare final - { - public: - BlockWashScoreCompare(); - ~BlockWashScoreCompare() = default; - bool operator() (const BlockInfo &a, const BlockInfo &b); - int get_error_code() { return OB_SUCCESS; } - }; - - typedef common::ObBinaryHeap Heap; - struct ChooseBlocksMapOp final - { - public: - ChooseBlocksMapOp(Heap &heap, int64_t cur_time) : heap_(heap), cur_time_(cur_time) {} - int operator () (oceanbase::common::hash::HashMapPair &entry); - private: - Heap &heap_; - int64_t cur_time_; - }; - - struct GetAvailableBlockMapOp final - { - public: - GetAvailableBlockMapOp(const int64_t dir_id, const int64_t tenant_id, const int64_t page_nums, - ObTmpMacroBlock *&block, bool &is_found) - : dir_id_(dir_id), tenant_id_(tenant_id), page_nums_(page_nums), block_(block), is_found_(is_found){} - int operator () (oceanbase::common::hash::HashMapPair &entry); - private: - int64_t dir_id_; - int64_t tenant_id_; - int64_t page_nums_; - ObTmpMacroBlock *&block_; - bool &is_found_; - }; - - struct DestroyBlockMapOp final - { - public: - DestroyBlockMapOp(ObTmpTenantFileStore &tenant_store) : tenant_store_(tenant_store) {} - int operator () (oceanbase::common::hash::HashMapPair &entry); - private: - ObTmpTenantFileStore &tenant_store_; // reference to tenant store from ObTmpTenantMemBlockManager - }; - -private: - int get_available_macro_block(const int64_t dir_id, const uint64_t tenant_id, const int64_t page_nums, - ObTmpMacroBlock *&t_mblk, bool &is_found); - int write_io( - const ObTmpBlockIOInfo &io_info, - ObMacroBlockHandle &handle); - int64_t get_tenant_mem_block_num(); - int check_memory_limit(); - int get_block_from_dir_cache(const int64_t dir_id, const int64_t tenant_id, - const int64_t page_nums, ObTmpMacroBlock *&t_mblk); - int get_block_and_set_washing(int64_t block_id, ObTmpMacroBlock *&m_blk); - - ObTmpTenantFileStore &tenant_store_; - ObSpLinkQueue wait_info_queue_; - WaitHandleMap wait_handles_map_; - TmpMacroBlockMap t_mblk_map_; // - Map dir_to_blk_map_; // - double blk_nums_threshold_; // free_page_nums / total_page_nums - ObTmpBlockCache *block_cache_; - common::ObConcurrentFIFOAllocator *allocator_; - uint64_t tenant_id_; - int64_t last_access_tenant_config_ts_; - int64_t last_tenant_mem_block_num_; - bool is_inited_; - int tg_id_; - bool stopped_; - int64_t washing_count_; - ObTmpFileWaitTask wait_task_; - ObTmpFileMemTask mem_task_; - SpinRWLock io_lock_; - common::ObBucketLock map_lock_; - ObThreadCond cond_; - BlockWashScoreCompare compare_; - DISALLOW_COPY_AND_ASSIGN(ObTmpTenantMemBlockManager); -}; - -} // end namespace blocksstable -} // end namespace oceanbase -#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_CACHE_H_ - - diff --git a/src/storage/blocksstable/ob_tmp_file_store.cpp b/src/storage/blocksstable/ob_tmp_file_store.cpp deleted file mode 100644 index 518ff7d666..0000000000 --- a/src/storage/blocksstable/ob_tmp_file_store.cpp +++ /dev/null @@ -1,2012 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#include "ob_tmp_file_store.h" -#include "ob_tmp_file.h" -#include "share/ob_task_define.h" -#include "observer/omt/ob_tenant_config_mgr.h" - -using namespace oceanbase::share; - -namespace oceanbase -{ -namespace blocksstable -{ - -const int64_t ObTmpMacroBlock::DEFAULT_PAGE_SIZE = 8192L; // 8kb - -ObTmpFilePageBuddy::ObTmpFilePageBuddy() - : is_inited_(false), - max_cont_page_nums_(0), - buf_(NULL), - allocator_(NULL) -{ - MEMSET(free_area_, 0, sizeof(free_area_)); -} - -ObTmpFilePageBuddy::~ObTmpFilePageBuddy() -{ - destroy(); -} - -int ObTmpFilePageBuddy::init(common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - uint8_t start_id = 0; - if (OB_UNLIKELY(is_inited_)) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpFilePageBuddy has not been inited", K(ret)); - } else { - allocator_ = &allocator; - buf_ = reinterpret_cast(allocator_->alloc(sizeof(ObTmpFileArea) * MAX_PAGE_NUMS)); - if (OB_ISNULL(buf_)) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "fail to alloc a buf", K(ret)); - } else { - max_cont_page_nums_ = std::pow(2, MAX_ORDER - 1); - /** - * page buddy free_list for a new block: - * -------------- --------- --------- --------- --------- --------- --------- --------- ------- - * |cont_page_nums| 1 | 2 | 4 | 8 | 16 | 32 | 64 | 128 | - * -------------- --------- --------- --------- --------- --------- --------- --------- ------- - * | free_area |[254,254]|[252,253]|[248,251]|[240,247]|[224,239]|[192,223]|[128,191]|[0,127]| - * -------------- --------- --------- --------- --------- --------- --------- --------- ------- - */ - uint8_t nums = max_cont_page_nums_; - for (int32_t i = MIN_ORDER - 1; i >= 0; --i) { - free_area_[i] = NULL; - } - for (int32_t i = MAX_ORDER - 1; i >= MIN_ORDER; --i) { - char *buf = reinterpret_cast(&(buf_[start_id])); - free_area_[i] = new (buf) ObTmpFileArea(start_id, nums); - start_id += nums; - nums /= 2; - } - is_inited_ = true; - } - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -void ObTmpFilePageBuddy::destroy() -{ - if (NULL != buf_) { - allocator_->free(buf_); - buf_ = NULL; - } - allocator_ = NULL; - max_cont_page_nums_ = 0; - is_inited_ = false; -} - -int ObTmpFilePageBuddy::alloc_all_pages() -{ - int ret = OB_SUCCESS; - ObTmpFileArea *tmp = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFilePageBuddy has not been inited", K(ret)); - } else if (is_empty()) { - for (int32_t i = 0; i < MAX_ORDER; ++i) { - while (NULL != free_area_[i]) { - tmp = free_area_[i]; - free_area_[i] = tmp->next_; - tmp->~ObTmpFileArea(); - } - } - - max_cont_page_nums_ = 0; - } else { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "this tmp block is not empty", K(ret)); - } - return ret; -} - -int ObTmpFilePageBuddy::alloc(const uint8_t page_nums, - uint8_t &start_page_id, - uint8_t &alloced_page_nums) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFilePageBuddy has not been inited", K(ret)); - } else if (OB_UNLIKELY(page_nums <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(page_nums)); - } else { - int32_t index = std::ceil(std::log(page_nums)/std::log(2)); - bool is_alloced = false; - for (int32_t i = index; i < MAX_ORDER && !is_alloced; ++i) { - if (NULL != free_area_[i]) { - int64_t num = i - index; - ObTmpFileArea *tmp = free_area_[i]; - free_area_[i] = tmp->next_; - tmp->next_ = NULL; - while (num--) { - tmp->page_nums_ /= 2; - char *buf = reinterpret_cast(&(buf_[tmp->start_page_id_ + tmp->page_nums_])); - ObTmpFileArea *area = new (buf) ObTmpFileArea(tmp->start_page_id_ + tmp->page_nums_, - tmp->page_nums_); - area->next_ = free_area_[static_cast(std::log(tmp->page_nums_)/std::log(2))]; - free_area_[static_cast(std::log(tmp->page_nums_)/std::log(2))] = area; - } - start_page_id = tmp->start_page_id_; - alloced_page_nums = std::pow(2, index); - is_alloced = true; - tmp->~ObTmpFileArea(); - } - } - - if (!is_alloced) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "cannot alloc the page", K(ret), K_(max_cont_page_nums), K(page_nums)); - } else { - index = std::ceil(std::log(max_cont_page_nums_)/std::log(2)); - int64_t max = 0; - for (int32_t i = index; i >= 0; --i) { - if (NULL == free_area_[i]) { - // nothing to do. - } else { - max = free_area_[i]->page_nums_; - break; - } - } - max_cont_page_nums_ = max; - } - } - return ret; -} - -void ObTmpFilePageBuddy::free_align(const int32_t start_page_id, const int32_t page_nums, - ObTmpFileArea *&area) -{ - ObTmpFileArea *tmp = NULL; - int64_t nums = page_nums; - int32_t start_id = start_page_id; - while (NULL != (tmp = find_buddy(nums, start_id))) { - // combine free area and buddy area. - if (0 != (start_id % (2 * nums))) { - start_id = tmp->start_page_id_; - std::swap(area, tmp); - } - nums *= 2; - tmp->~ObTmpFileArea(); - tmp = NULL; - } - area->start_page_id_ = start_id; - area->page_nums_ = nums; - area->next_ = free_area_[static_cast(std::log(nums)/std::log(2))]; - free_area_[static_cast(std::log(nums)/std::log(2))] = area; - if (nums > max_cont_page_nums_) { - max_cont_page_nums_ = nums; - } -} - -bool ObTmpFilePageBuddy::is_empty() const -{ - bool is_empty = true; - for (int32_t i = 0; i < MIN_ORDER && is_empty; ++i) { - if (NULL != free_area_[i]) { - is_empty = false; - break; - } - } - for (int32_t i = MIN_ORDER; i < MAX_ORDER && is_empty; ++i) { - if (NULL == free_area_[i]) { - is_empty = false; - break; - } - } - return is_empty; -} - -int64_t ObTmpFilePageBuddy::to_string(char* buf, const int64_t buf_len) const -{ - int64_t pos = 0; - bool first = true; - ObTmpFileArea *area = NULL; - common::databuff_printf(buf, buf_len, pos, "{"); - for (int32_t i = 0; i < MAX_ORDER; ++i) { - area = free_area_[i]; - if (NULL != area) { - common::databuff_print_kv(buf, buf_len, pos, "page_nums", static_cast(std::pow(2, i))); - common::databuff_printf(buf, buf_len, pos, "{"); - while (NULL != area) { - if (first) { - first = false; - } else { - common::databuff_printf(buf, buf_len, pos, ","); - } - common::databuff_printf(buf, buf_len, pos, "{"); - common::databuff_print_kv(buf, buf_len, pos, "start_page_id", area->start_page_id_, - "end_page_id", area->start_page_id_ + area->page_nums_); - common::databuff_printf(buf, buf_len, pos, "}"); - area = area->next_; - } - common::databuff_printf(buf, buf_len, pos, "}"); - } - } - common::databuff_printf(buf, buf_len, pos, "}"); - return pos; -} - -void ObTmpFilePageBuddy::free(const int32_t start_page_id, const int32_t page_nums) -{ - if (OB_UNLIKELY(start_page_id + page_nums >= std::pow(2, MAX_ORDER))) { - STORAGE_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "page id more than max numbers in block", K(start_page_id), K(page_nums)); - ob_abort(); - } else { - int32_t start_id = start_page_id; - int32_t nums = page_nums; - int32_t length = 0; - while (nums > 0) { - /** - * PURPOSE: align free area into power of 2. - * - * The probable value of alloc_start_id: - * page nums start page id - * 128 0 ---------------- 128 ------------------- 256 - * 64 0 ------ 64 ------ 128 ------- 192 ------- 256 - * 32 0 - 32 - 64 - 96 - 128 - 160 - 192 - 224 - 256 - * ... ... - * So, the maximum number of consecutive pages from a start_page_id is the - * gcd(greatest common divisor) between it and 512, except 0. The maximum - * consecutive page nums of 0 is 256. - * - * The layout of free area in alocated area : - * |<---------------alloc_page_nums--------------->| - * <---- |<--free_page_nums-->| - * |==========================|====================| - * alloc_start free_page_id alloc_end - * - * So, free_end always equal to alloc_end. - * - * Based on two observations above, the algorithm is designed as follows: - */ - length = 2; - while(0 == start_id % length && length <= nums) { - length *= 2; - } - length = std::min(length / 2, nums); - - char *buf = reinterpret_cast(&(buf_[start_id])); - ObTmpFileArea *area = new (buf) ObTmpFileArea(start_id, length); - free_align(area->start_page_id_, area->page_nums_, area); - start_id += length; - nums -= length; - } - } -} - -ObTmpFileArea *ObTmpFilePageBuddy::find_buddy(const int32_t page_nums, const int32_t start_page_id) -{ - ObTmpFileArea *tmp = NULL; - if (MAX_PAGE_NUMS < page_nums || page_nums <= 0 || start_page_id < 0 - || start_page_id >= MAX_PAGE_NUMS) { - STORAGE_LOG_RET(WARN, OB_INVALID_ARGUMENT, "invalid argument", K(page_nums), K(start_page_id)); - } else if (MAX_PAGE_NUMS == page_nums) { - // no buddy, so, nothing to do. - } else { - tmp = free_area_[static_cast(std::log(page_nums)/std::log(2))]; - ObTmpFileArea *pre = tmp; - int64_t start_id = 0; - /** - * case 1: case 2: - * |<--page_nums-->|<--page_nums-->| |<--page_nums-->|<--page_nums-->| - * |===============|===============| |===============|===============| - * start_page_id start_id(buddy) start_id(buddy) start_page_id - */ - if (0 == (start_page_id % (2 * page_nums))) { // case 1 - start_id = start_page_id + page_nums; - } else { // case 2 - start_id = start_page_id - page_nums; - } - while (NULL != tmp) { - if (tmp->start_page_id_ == start_id) { - if (pre == tmp) { - free_area_[static_cast(std::log(page_nums)/std::log(2))] = tmp->next_; - } else { - pre->next_ = tmp->next_; - } - tmp->next_ = NULL; - break; - } - pre = tmp; - tmp = tmp->next_; - } - } - return tmp; -} - -ObTmpFileMacroBlockHeader::ObTmpFileMacroBlockHeader() - : version_(TMP_FILE_MACRO_BLOCK_HEADER_VERSION), - magic_(TMP_FILE_MACRO_BLOCK_HEADER_MAGIC), - block_id_(-1), - dir_id_(-1), - tenant_id_(0), - free_page_nums_(-1) -{ -} - -bool ObTmpFileMacroBlockHeader::is_valid() const -{ - return TMP_FILE_MACRO_BLOCK_HEADER_VERSION == version_ - && TMP_FILE_MACRO_BLOCK_HEADER_MAGIC == magic_ - && block_id_ >= 0 - && dir_id_ >= 0 - && tenant_id_ > 0 - && free_page_nums_ >= 0; -} - -int ObTmpFileMacroBlockHeader::serialize(char *buf, const int64_t buf_len, int64_t &pos) const -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), KP(buf), K(buf_len)); - } else if (OB_UNLIKELY(pos + get_serialize_size() > buf_len)) { - ret = OB_BUF_NOT_ENOUGH; - STORAGE_LOG(WARN, "data buffer is not enough", K(ret), K(pos), K(buf_len), K(*this)); - } else if (OB_UNLIKELY(!is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "tmp file macro block header is invalid", K(ret), K(*this)); - } else { - ObTmpFileMacroBlockHeader *header = reinterpret_cast(buf + pos); - header->version_ = version_; - header->magic_ = magic_; - header->block_id_ = block_id_; - header->dir_id_ = dir_id_; - header->tenant_id_ = tenant_id_; - header->free_page_nums_ = free_page_nums_; - pos += header->get_serialize_size(); - } - return ret; -} - -int ObTmpFileMacroBlockHeader::deserialize(const char *buf, const int64_t data_len, int64_t &pos) -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(buf) || OB_UNLIKELY(data_len <= 0 || pos < 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid arguments", K(ret), KP(buf), K(data_len), K(pos)); - } else if (OB_UNLIKELY(data_len - pos < get_serialize_size())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "buffer not enough", K(ret), KP(buf), K(data_len), K(pos)); - } else { - const ObTmpFileMacroBlockHeader *ptr = reinterpret_cast(buf + pos); - version_ = ptr->version_; - magic_ = ptr->magic_; - block_id_ = ptr->block_id_; - dir_id_ = ptr->dir_id_; - tenant_id_ = ptr->tenant_id_; - free_page_nums_ = ptr->free_page_nums_; - if (OB_UNLIKELY(!is_valid())) { - ret = OB_DESERIALIZE_ERROR; - STORAGE_LOG(ERROR, "deserialize error", K(ret), K(*this)); - } else { - pos += get_serialize_size(); - } - } - return ret; -} - -void ObTmpFileMacroBlockHeader::reset() -{ - version_ = TMP_FILE_MACRO_BLOCK_HEADER_VERSION; - magic_ = TMP_FILE_MACRO_BLOCK_HEADER_MAGIC; - block_id_ = -1; - dir_id_ = -1; - tenant_id_ = 0; - free_page_nums_ = 0; -} - -ObTmpMacroBlock::ObTmpMacroBlock() - : buffer_(NULL), - handle_(), - using_extents_(), - macro_block_handle_(), - tmp_file_header_(), - io_desc_(), - block_status_(MAX), - is_sealed_(false), - is_inited_(false), - alloc_time_(0), - access_time_(0) -{ - using_extents_.set_attr(ObMemAttr(MTL_ID(), "TMP_US_META")); -} - -ObTmpMacroBlock::~ObTmpMacroBlock() -{ - destroy(); -} - -int ObTmpMacroBlock::init(const int64_t block_id, const int64_t dir_id, const uint64_t tenant_id, - common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else if (OB_FAIL(page_buddy_.init(allocator))) { - STORAGE_LOG(WARN, "Fail to init the page buddy", K(ret)); - } else { - tmp_file_header_.block_id_ = block_id; - tmp_file_header_.dir_id_ = dir_id; - tmp_file_header_.tenant_id_ = tenant_id; - tmp_file_header_.free_page_nums_ = ObTmpFilePageBuddy::MAX_PAGE_NUMS; - ATOMIC_STORE(&block_status_, MEMORY); - is_inited_ = true; - alloc_time_ = 0; - ATOMIC_STORE(&access_time_, 0); - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -void ObTmpMacroBlock::destroy() -{ - using_extents_.reset(); - page_buddy_.destroy(); - macro_block_handle_.reset(); - tmp_file_header_.reset(); - buffer_ = NULL; - handle_.reset(); - ATOMIC_STORE(&block_status_, MAX); - is_sealed_ = false; - alloc_time_ = 0; - ATOMIC_STORE(&access_time_, 0); - is_inited_ = false; -} - -int ObTmpMacroBlock::seal(bool &is_sealed) -{ - int ret = OB_SUCCESS; - is_sealed = false; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret), KPC(this)); - } else { - ObTmpFileExtent *tmp = NULL; - SpinWLockGuard guard(lock_); - bool is_all_closed = using_extents_.count() == 0 ? false : true; - for (int32_t i = 0; OB_SUCC(ret) && i < using_extents_.count() && is_all_closed; ++i) { - tmp = using_extents_.at(i); - if (NULL != tmp && !tmp->is_closed()) { - uint8_t start_id = ObTmpFilePageBuddy::MAX_PAGE_NUMS; - uint8_t page_nums = 0; - if (tmp->close(start_id, page_nums)) { - if (ObTmpFilePageBuddy::MAX_PAGE_NUMS== start_id && 0 == page_nums) { - //nothing to do - } else if (OB_UNLIKELY(start_id > ObTmpFilePageBuddy::MAX_PAGE_NUMS - 1 - || page_nums > ObTmpFilePageBuddy::MAX_PAGE_NUMS)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "fail to close the extent", K(ret), K(start_id), K(page_nums), K(*tmp)); - } else if (OB_FAIL(free(start_id, page_nums))) { - STORAGE_LOG(WARN, "fail to free the extent", K(ret)); - } else { - } - if (OB_FAIL(ret)) { - tmp->unclose(page_nums); - is_all_closed = false; - } - } else { - is_all_closed = false; - } - } - } - if (OB_SUCC(ret) && is_all_closed) { - is_sealed = true; - ATOMIC_SET(&is_sealed_, true); - } - } - return ret; -} - -int ObTmpMacroBlock::is_extents_closed(bool &is_extents_closed) -{ - int ret = OB_SUCCESS; - is_extents_closed = true; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited"); - } else { - SpinRLockGuard guard(lock_); - is_extents_closed = using_extents_.count() == 0 ? false : true; - for (int64_t i = 0; i< using_extents_.count(); ++i) { - if (!using_extents_.at(i)->is_closed()) { - STORAGE_LOG(DEBUG, "the tmp macro block's extents is not all closed", K(tmp_file_header_.block_id_)); - is_extents_closed = false; - break; - } - } - } - return ret; -} - -int ObTmpMacroBlock::get_block_cache_handle(ObTmpBlockValueHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpBlockCacheKey key(tmp_file_header_.block_id_, tmp_file_header_.tenant_id_); - SpinRLockGuard guard(lock_); - if (!is_disked()) { - handle = handle_; - } else if (OB_FAIL(ObTmpBlockCache::get_instance().get_block(key, handle))) { - if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { - STORAGE_LOG(WARN, "fail to get tmp block from cache", K(ret), K(key)); - } else if (REACH_COUNT_INTERVAL(100)) { // print one log per 100 times. - STORAGE_LOG(DEBUG, "block cache miss", K(ret), K(key)); - } - } - return ret; -} - -int ObTmpMacroBlock::get_wash_io_info(ObTmpBlockIOInfo &info) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else { - info.block_id_ = tmp_file_header_.block_id_; - info.offset_ = 0; - info.size_ = ObTmpFileStore::get_block_size(); - info.tenant_id_ = tmp_file_header_.tenant_id_; - info.macro_block_id_ = get_macro_block_id(); - info.buf_ = buffer_; - info.io_desc_ = io_desc_; - info.io_timeout_ms_ = max(GCONF._data_storage_io_timeout / 1000L, DEFAULT_IO_WAIT_TIME_MS); - } - return ret; -} - -int ObTmpMacroBlock::give_back_buf_into_cache(const bool is_wash) -{ - int ret = OB_SUCCESS; - ObTmpBlockCacheKey key(tmp_file_header_.block_id_, tmp_file_header_.tenant_id_); - 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)); - } - 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; -} - -int ObTmpMacroBlock::alloc_all_pages(ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - const bool sealed = is_sealed(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else if (OB_UNLIKELY(!is_memory() || sealed)) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(WARN, "the block is not in memory", K(ret), K(ATOMIC_LOAD(&block_status_)), K(sealed)); - } else if (OB_FAIL(page_buddy_.alloc_all_pages())) { - STORAGE_LOG(WARN, "Fail to allocate the tmp extent", K(ret), K_(tmp_file_header), K_(page_buddy)); - } else { - extent.set_block_id(get_block_id()); - extent.set_start_page_id(0); - extent.set_page_nums(ObTmpFilePageBuddy::MAX_PAGE_NUMS); - extent.alloced(); - if (OB_FAIL(using_extents_.push_back(&extent))) { - STORAGE_LOG(WARN, "Fail to push back into using_extexts", K(ret)); - page_buddy_.free(extent.get_start_page_id(), extent.get_page_nums()); - extent.reset(); - } else { - tmp_file_header_.free_page_nums_ -= extent.get_page_nums(); - } - const int64_t cur_time = ObTimeUtility::fast_current_time(); - alloc_time_ = cur_time; - ATOMIC_STORE(&access_time_, alloc_time_ + 1); - } - return ret; -} - -int ObTmpMacroBlock::alloc(const uint8_t page_nums, ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - uint8_t start_page_id = extent.get_start_page_id(); - uint8_t alloced_page_nums = 0; - SpinWLockGuard guard(lock_); - const bool sealed = is_sealed(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else if (OB_UNLIKELY(!is_memory() || sealed)) { - ret = OB_STATE_NOT_MATCH; - STORAGE_LOG(WARN, "the block is not in memory", K(ret), K(ATOMIC_LOAD(&block_status_)), K(sealed)); - } else if (OB_FAIL(page_buddy_.alloc(page_nums, start_page_id, alloced_page_nums))) { - STORAGE_LOG(WARN, "Fail to allocate the tmp extent", K(ret), K_(tmp_file_header), K_(page_buddy)); - } else { - extent.set_block_id(tmp_file_header_.block_id_); - extent.set_page_nums(alloced_page_nums); - extent.set_start_page_id(start_page_id); - extent.alloced(); - if (OB_FAIL(using_extents_.push_back(&extent))) { - STORAGE_LOG(WARN, "Fail to push back into using_extexts", K(ret)); - page_buddy_.free(extent.get_start_page_id(), extent.get_page_nums()); - extent.reset(); - } else { - tmp_file_header_.free_page_nums_ -= alloced_page_nums; - } - const int64_t cur_time = ObTimeUtility::fast_current_time(); - if (0 == alloc_time_) { - alloc_time_ = cur_time; - } - if (OB_UNLIKELY(0 == ATOMIC_LOAD(&access_time_))) { - ATOMIC_STORE(&access_time_, - alloc_time_ + 60 * 1000000L * int64_t(alloced_page_nums) / get_max_cont_page_nums()); - } else { - ATOMIC_STORE(&access_time_, cur_time); - } - } - return ret; -} - -int ObTmpMacroBlock::free(ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else { - SpinWLockGuard guard(lock_); - page_buddy_.free(extent.get_start_page_id(), extent.get_page_nums()); - for (int64_t i = using_extents_.count() - 1; i >= 0; --i) { - if (&extent == using_extents_.at(i)) { - using_extents_.remove(i); - break; - } - } - tmp_file_header_.free_page_nums_ += extent.get_page_nums(); - if (tmp_file_header_.free_page_nums_ == ObTmpFilePageBuddy::MAX_PAGE_NUMS) { - alloc_time_ = 0; - ATOMIC_STORE(&access_time_, 0); - } - } - return ret; -} - -int ObTmpMacroBlock::free(const int32_t start_page_id, const int32_t page_nums) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlock has not been inited", K(ret)); - } else { - page_buddy_.free(start_page_id, page_nums); - tmp_file_header_.free_page_nums_ += page_nums; - } - return ret; -} - -int64_t ObTmpMacroBlock::get_used_page_nums() const -{ - SpinRLockGuard guard(lock_); - int64_t used_page_nums = 0; - for (int64_t i = using_extents_.count() - 1; i >= 0; --i) { - used_page_nums += std::ceil((1.0 * using_extents_.at(i)->get_offset()) / DEFAULT_PAGE_SIZE); - } - return used_page_nums; -} - -void ObTmpMacroBlock::set_io_desc(const common::ObIOFlag &io_desc) -{ - io_desc_ = io_desc; -} - -int ObTmpMacroBlock::check_and_set_status(const BlockStatus old_block_status, const BlockStatus block_status) -{ - int ret = OB_SUCCESS; - if (old_block_status != ATOMIC_VCAS(&block_status_, old_block_status, block_status)) { - ret = OB_STATE_NOT_MATCH; - } - return ret; -} - -ObTmpTenantMacroBlockManager::ObTmpTenantMacroBlockManager() - : allocator_(), - blocks_(), - is_inited_(false) -{ -} - -ObTmpTenantMacroBlockManager::~ObTmpTenantMacroBlockManager() -{ - destroy(); -} - -int ObTmpTenantMacroBlockManager::init(const uint64_t tenant_id, common::ObIAllocator &allocator) -{ - int ret = OB_SUCCESS; - ObMemAttr attr(tenant_id, ObModIds::OB_TMP_BLOCK_MAP); - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has been inited", K(ret)); - } else if (OB_FAIL(blocks_.create(MBLK_HASH_BUCKET_NUM, attr, attr))) { - STORAGE_LOG(WARN, "Fail to create tmp macro block map, ", K(ret)); - } else { - allocator_ = &allocator; - is_inited_ = true; - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -int ObTmpTenantMacroBlockManager::alloc_macro_block(const int64_t dir_id, const uint64_t tenant_id, - ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - void *block_buf = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); - } else if (OB_ISNULL(block_buf = allocator_->alloc(sizeof(ObTmpMacroBlock)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "fail to alloc a buf", K(ret)); - } else if (OB_ISNULL(t_mblk = new (block_buf) ObTmpMacroBlock())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to new a ObTmpMacroBlock", K(ret)); - } else if (OB_FAIL(t_mblk->init(OB_TMP_FILE_STORE.get_next_blk_id(), dir_id, tenant_id, *allocator_))) { - STORAGE_LOG(WARN, "fail to init tmp block", K(ret)); - } else if (OB_FAIL(blocks_.set_refactored(t_mblk->get_block_id(), t_mblk))) { - STORAGE_LOG(WARN, "fail to set tmp macro block map", K(ret), K(t_mblk)); - } - if (OB_FAIL(ret)) { - if (NULL != t_mblk) { - int tmp_ret = OB_SUCCESS; - if (OB_SUCCESS != (tmp_ret = blocks_.erase_refactored(t_mblk->get_block_id()))) { - STORAGE_LOG(WARN, "fail to erase from tmp macro block map", K(tmp_ret), K(t_mblk)); - } - t_mblk->~ObTmpMacroBlock(); - allocator_->free(block_buf); - } - } - return ret; -} - -int ObTmpTenantMacroBlockManager::get_macro_block(const int64_t block_id, ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id)); - } else if (OB_FAIL(blocks_.get_refactored(block_id, t_mblk))) { - STORAGE_LOG(WARN, "fail to get tmp macro block", K(ret), K(block_id)); - } - return ret; -} - -int ObTmpTenantMacroBlockManager::free_macro_block(const int64_t block_id) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id)); - } else if (OB_FAIL(blocks_.erase_refactored(block_id))) { - STORAGE_LOG(WARN, "fail to erase tmp macro block", K(ret)); - } - return ret; -} - -int ObTmpTenantMacroBlockManager::get_disk_macro_block_count(int64_t &count) const -{ - int ret = OB_SUCCESS; - - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); - } else { - count = blocks_.size(); - } - - return ret; -} - -int ObTmpTenantMacroBlockManager::get_disk_macro_block_list( - common::ObIArray ¯o_id_list) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpMacroBlockManager has not been inited", K(ret)); - } else { - TmpMacroBlockMap::iterator iter; - ObTmpMacroBlock *tmp = NULL; - for (iter = blocks_.begin(); OB_SUCC(ret) && iter != blocks_.end(); ++iter) { - if (OB_ISNULL(tmp = iter->second)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to iterate tmp macro block map", K(ret)); - } else if (tmp->is_disked() && tmp->get_macro_block_id().is_valid() && - OB_FAIL(macro_id_list.push_back(tmp->get_macro_block_id()))) { - STORAGE_LOG(WARN, "fail to push back macro block id", K(ret), K(tmp->get_macro_block_id())); - } - } - } - - return ret; -} - -void ObTmpTenantMacroBlockManager::print_block_usage() -{ - int64_t disk_count = 0; - int64_t disk_fragment = 0; - int64_t mem_count = 0; - int64_t mem_fragment = 0; - TmpMacroBlockMap::iterator iter; - ObTmpMacroBlock *tmp = NULL; - for (iter = blocks_.begin(); iter != blocks_.end(); ++iter) { - tmp = iter->second; - if (tmp->is_disked()) { - disk_count++; - disk_fragment += tmp->get_free_page_nums(); - } else { - mem_count++; - mem_fragment += tmp->get_free_page_nums(); - } - } - double disk_fragment_ratio = 0; - if (0 != disk_count) { - disk_fragment_ratio = disk_fragment * 1.0 / (disk_count * ObTmpFilePageBuddy::MAX_PAGE_NUMS); - } - double mem_fragment_ratio = 0; - if (0 != mem_count) { - mem_fragment_ratio = mem_fragment * 1.0 / (mem_count * ObTmpFilePageBuddy::MAX_PAGE_NUMS); - } - STORAGE_LOG(INFO, "the block usage for temporary files", - K(disk_count), K(disk_fragment), K(disk_fragment_ratio), - K(mem_count), K(mem_fragment), K(mem_fragment_ratio)); -} - -void ObTmpTenantMacroBlockManager::destroy() -{ - TmpMacroBlockMap::iterator iter; - ObTmpMacroBlock *tmp = NULL; - for (iter = blocks_.begin(); iter != blocks_.end(); ++iter) { - if (OB_NOT_NULL(tmp = iter->second)) { - tmp->~ObTmpMacroBlock(); - allocator_->free(tmp); - } - } - blocks_.destroy(); - allocator_ = NULL; - is_inited_ = false; -} - -ObTmpTenantFileStore::ObTmpTenantFileStore() - : is_inited_(false), - page_cache_num_(0), - block_cache_num_(0), - ref_cnt_(0), - page_cache_(NULL), - lock_(), - allocator_(), - io_allocator_(), - tmp_block_manager_(), - tmp_mem_block_manager_(*this), - last_access_tenant_config_ts_(0), - last_meta_mem_limit_(TOTAL_LIMIT) -{ -} - -ObTmpTenantFileStore::~ObTmpTenantFileStore() -{ - destroy(); -} - -void ObTmpTenantFileStore::inc_ref() -{ - ATOMIC_INC(&ref_cnt_); -} - -int64_t ObTmpTenantFileStore::dec_ref() -{ - int ret = OB_SUCCESS; - const int64_t tmp_ref = ATOMIC_SAF(&ref_cnt_, 1); - if (tmp_ref < 0) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(ERROR, "bug: ref_cnt < 0", K(ret), K(tmp_ref), K(lbt())); - ob_abort(); - } - return tmp_ref; -} - -int ObTmpTenantFileStore::init(const uint64_t tenant_id) -{ - int ret = OB_SUCCESS; - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(allocator_.init(BLOCK_SIZE, ObModIds::OB_TMP_BLOCK_MANAGER, tenant_id, get_memory_limit(tenant_id)))) { - STORAGE_LOG(WARN, "fail to init allocator", K(ret)); - } else if (OB_FAIL(io_allocator_.init( - lib::ObMallocAllocator::get_instance(), - OB_MALLOC_MIDDLE_BLOCK_SIZE, - ObMemAttr(tenant_id, ObModIds::OB_TMP_PAGE_CACHE, ObCtxIds::DEFAULT_CTX_ID)))) { - STORAGE_LOG(WARN, "Fail to init io allocator, ", K(ret)); - } else if (OB_ISNULL(page_cache_ = &ObTmpPageCache::get_instance())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to get the page cache", K(ret)); - } else if (OB_FAIL(tmp_block_manager_.init(tenant_id, allocator_))) { - STORAGE_LOG(WARN, "fail to init the block manager for ObTmpFileStore", K(ret)); - } else if (OB_FAIL(tmp_mem_block_manager_.init(tenant_id, allocator_))) { - STORAGE_LOG(WARN, "fail to init memory block manager", K(ret)); - } else { - is_inited_ = true; - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -void ObTmpTenantFileStore::refresh_memory_limit(const uint64_t tenant_id) -{ - const int64_t old_limit = ATOMIC_LOAD(&last_meta_mem_limit_); - const int64_t new_limit = get_memory_limit(tenant_id); - if (old_limit != new_limit) { - allocator_.set_total_limit(new_limit); - ObTaskController::get().allow_next_syslog(); - STORAGE_LOG(INFO, "succeed to refresh temporary file meta memory limit", K(tenant_id), K(old_limit), K(new_limit)); - } -} - -int64_t ObTmpTenantFileStore::get_memory_limit(const uint64_t tenant_id) -{ - const int64_t last_access_ts = ATOMIC_LOAD(&last_access_tenant_config_ts_); - int64_t memory_limit = TOTAL_LIMIT; - if (last_access_ts > 0 && common::ObClockGenerator::getClock() - last_access_ts < REFRESH_CONFIG_INTERVAL) { - memory_limit = ATOMIC_LOAD(&last_meta_mem_limit_); - } else { - omt::ObTenantConfigGuard config(TENANT_CONF(tenant_id)); - const int64_t tenant_mem_limit = lib::get_tenant_memory_limit(tenant_id); - if (!config.is_valid() || 0 == tenant_mem_limit || INT64_MAX == tenant_mem_limit) { - COMMON_LOG(INFO, "failed to get tenant config", K(tenant_id), K(tenant_mem_limit)); - } else { - const int64_t limit_percentage_config = config->_temporary_file_meta_memory_limit_percentage; - const int64_t limit_percentage = 0 == limit_percentage_config ? 70 : limit_percentage_config; - memory_limit = tenant_mem_limit * limit_percentage / 100; - if (OB_UNLIKELY(memory_limit <= 0)) { - STORAGE_LOG(INFO, "memory limit isn't more than 0", K(memory_limit)); - memory_limit = ATOMIC_LOAD(&last_meta_mem_limit_); - } else { - ATOMIC_STORE(&last_meta_mem_limit_, memory_limit); - ATOMIC_STORE(&last_access_tenant_config_ts_, common::ObClockGenerator::getClock()); - } - } - } - return memory_limit; -} - -void ObTmpTenantFileStore::destroy() -{ - tmp_mem_block_manager_.destroy(); - tmp_block_manager_.destroy(); - if (NULL != page_cache_) { - page_cache_ = NULL; - } - allocator_.destroy(); - io_allocator_.reset(); - last_access_tenant_config_ts_ = 0; - last_meta_mem_limit_ = TOTAL_LIMIT; - is_inited_ = false; - STORAGE_LOG(INFO, "cache num when destroy", - K(ATOMIC_LOAD(&page_cache_num_)), K(ATOMIC_LOAD(&block_cache_num_))); - page_cache_num_ = 0; - block_cache_num_ = 0; -} - -int ObTmpTenantFileStore::alloc(const int64_t dir_id, const uint64_t tenant_id, const int64_t alloc_size, - ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - const int64_t block_size = ObTmpFileStore::get_block_size(); - // In buddy allocation, if free space in one block isn't powers of 2, need upper align. - int64_t max_order = std::ceil(std::log(ObTmpFilePageBuddy::MAX_PAGE_NUMS) / std::log(2)); - int64_t origin_max_cont_page_nums = std::pow(2, max_order - 1); - int64_t max_cont_size_per_block = origin_max_cont_page_nums * ObTmpMacroBlock::get_default_page_size(); - ObTmpMacroBlock *t_mblk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (alloc_size <= 0 || alloc_size > block_size) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(alloc_size), K(block_size)); - } else { - const int64_t timeout_ts = THIS_WORKER.get_timeout_ts(); - ret = OB_STATE_NOT_MATCH; - while (OB_STATE_NOT_MATCH == ret || OB_SIZE_OVERFLOW == ret) { - if (OB_UNLIKELY(timeout_ts <= ObTimeUtility::current_time())) { - ret = OB_TIMEOUT; - STORAGE_LOG(WARN, "it's timeout", K(ret), K(timeout_ts), K(ObTimeUtility::current_time())); - } else if (OB_FAIL(tmp_mem_block_manager_.cleanup())) { - if (OB_STATE_NOT_MATCH != ret) { - STORAGE_LOG(WARN, "fail to try wash tmp macro block", K(ret), K(dir_id), K(tenant_id)); - } - } - SpinWLockGuard guard(lock_); - if (OB_SUCC(ret)) { - if (alloc_size > max_cont_size_per_block) { - 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)); - } - } 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)); - } else if (OB_FAIL(t_mblk->alloc_all_pages(extent))) { - STORAGE_LOG(WARN, "Fail to allocate the tmp extent", K(ret), K(t_mblk->get_block_id())); - } else if (alloc_size < block_size) { - const int64_t nums = std::ceil(alloc_size * 1.0 / ObTmpMacroBlock::get_default_page_size()); - if (OB_FAIL(free_extent(t_mblk->get_block_id(), nums, - ObTmpFilePageBuddy::MAX_PAGE_NUMS - nums))) { - STORAGE_LOG(WARN, "fail to free pages", K(ret), K(t_mblk->get_block_id())); - } else { - extent.set_page_nums(nums); - } - } - } 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))) { - 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)); - } else { - const int64_t page_nums = std::ceil(alloc_size * 1.0 / ObTmpMacroBlock::get_default_page_size()); - if (OB_FAIL(t_mblk->alloc(page_nums, extent))){ - STORAGE_LOG(WARN, "fail to alloc tmp extent", K(ret)); - } else if (OB_FAIL(tmp_mem_block_manager_.refresh_dir_to_blk_map(dir_id, t_mblk))) { - STORAGE_LOG(WARN, "fail to refresh dir_to_blk_map", K(ret), K(*t_mblk)); - } - } - } - } - } - } - } - if (OB_FAIL(ret)) { - STORAGE_LOG(WARN, "fail to alloc tmp extent", K(ret)); - if (OB_ALLOCATE_MEMORY_FAILED == ret) { - STORAGE_LOG(WARN, "alloc memory failed", K(ret), K(ATOMIC_LOAD(&block_cache_num_)), K(ATOMIC_LOAD(&page_cache_num_))); - } - } - return ret; -} - -int ObTmpTenantFileStore::free(ObTmpFileExtent *extent) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(free_extent(extent))) { - STORAGE_LOG(WARN, "fail to free the extent", K(ret), K(*extent)); - } - return ret; -} - -int ObTmpTenantFileStore::free(const int64_t block_id, const int32_t start_page_id, - const int32_t page_nums) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(free_extent(block_id, start_page_id, page_nums))) { - STORAGE_LOG(WARN, "fail to free the extent", K(ret), K(block_id), K(start_page_id), - K(page_nums)); - } - return ret; -} - -int ObTmpTenantFileStore::free_extent(ObTmpFileExtent *extent) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *t_mblk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_ISNULL(extent) || OB_UNLIKELY(!extent->is_valid())) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(*extent)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(extent->get_block_id(), t_mblk))) { - STORAGE_LOG(WARN, "fail to get tmp macro block", K(ret)); - } else if (OB_ISNULL(t_mblk)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "block is null", K(ret)); - } else if (OB_FAIL(t_mblk->free(*extent))) { - STORAGE_LOG(WARN, "fail to free extent", K(ret)); - } else { - extent->reset(); - if (OB_SUCC(ret) && t_mblk->is_empty()) { - if (OB_FAIL(free_macro_block(t_mblk))) { - STORAGE_LOG(WARN, "fail to free tmp macro block", K(ret)); - } - } - } - return ret; -} - -int ObTmpTenantFileStore::free_extent(const int64_t block_id, const int32_t start_page_id, - const int32_t page_nums) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *t_mblk = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_UNLIKELY(start_page_id < 0 || page_nums < 0 || block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id), K(start_page_id), K(page_nums)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(block_id, t_mblk))) { - STORAGE_LOG(WARN, "fail to get tmp block", K(ret), K(block_id), K(start_page_id), K(page_nums)); - } else if (OB_ISNULL(t_mblk)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "block is null", K(ret)); - } else if (OB_FAIL(t_mblk->free(start_page_id, page_nums))) { - STORAGE_LOG(WARN, "fail to free extent", K(ret)); - } else { - if (OB_SUCC(ret) && t_mblk->is_empty()) { - if (OB_FAIL(free_macro_block(t_mblk))) { - STORAGE_LOG(WARN, "fail to free tmp macro block", K(ret)); - } - } - } - return ret; -} - -int ObTmpTenantFileStore::free_macro_block(ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_ISNULL(t_mblk)) { - ret = OB_INVALID_ARGUMENT; - 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 { - 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; -} - -int ObTmpTenantFileStore::alloc_macro_block(const int64_t dir_id, const uint64_t tenant_id, - ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - t_mblk = nullptr; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - 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(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)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "block is null", K(ret)); - } else { - ObTmpBlockCacheKey key(t_mblk->get_block_id(), tenant_id); - if (OB_FAIL(tmp_mem_block_manager_.alloc_buf(key, t_mblk->get_handle()))) { - STORAGE_LOG(WARN, "fail to alloc block cache buf", K(ret)); - } else { - t_mblk->set_buffer(t_mblk->get_handle().value_->get_buffer()); - if (OB_FAIL(tmp_mem_block_manager_.add_macro_block(t_mblk))) { - STORAGE_LOG(WARN, "fail to put meta into block cache", K(ret), K(t_mblk)); - } - inc_block_cache_num(1); - if (OB_FAIL(ret)) { - tmp_mem_block_manager_.free_macro_block(t_mblk->get_block_id()); - t_mblk->give_back_buf_into_cache(); - dec_block_cache_num(1); - } - } - if (OB_FAIL(ret) && OB_NOT_NULL(t_mblk)) { - tmp_block_manager_.free_macro_block(t_mblk->get_block_id()); - t_mblk->~ObTmpMacroBlock(); - allocator_.free(t_mblk); - t_mblk = nullptr; - } - } - - return ret; -} - -int ObTmpTenantFileStore::read(ObTmpBlockIOInfo &io_info, ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpBlockValueHandle tb_handle; - ObTmpMacroBlock *block = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (!handle.is_valid()) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(handle)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(io_info.block_id_, block))) { - STORAGE_LOG(WARN, "fail to get block from tmp block manager", K(ret), K_(io_info.block_id)); - } else if (OB_ISNULL(block)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "the block is NULL", K(ret), K_(io_info.block_id)); - } else { - if (OB_SUCC(block->get_block_cache_handle(tb_handle))) { - ObTmpFileIOHandle::ObBlockCacheHandle block_handle( - tb_handle, - io_info.buf_, - io_info.offset_, - io_info.size_); - OB_TMP_FILE_STORE.inc_block_cache_num(io_info.tenant_id_, 1); - if (OB_FAIL(handle.get_block_cache_handles().push_back(block_handle))) { - STORAGE_LOG(WARN, "Fail to push back into block_handles", K(ret), K(block_handle)); - } - } else if (OB_SUCC(read_page(block, io_info, handle))) { - //nothing to do. - } else { - STORAGE_LOG(WARN, "fail to read", K(ret)); - } - - if (OB_SUCC(ret)) { - block->set_io_desc(io_info.io_desc_); - } - } - return ret; -} - -int ObTmpTenantFileStore::read_page(ObTmpMacroBlock *block, ObTmpBlockIOInfo &io_info, - ObTmpFileIOHandle &handle) -{ - int ret = OB_SUCCESS; - int64_t page_start_id = io_info.offset_ / ObTmpMacroBlock::get_default_page_size(); - int64_t offset = io_info.offset_ % ObTmpMacroBlock::get_default_page_size(); - int64_t remain_size = io_info.size_; - int64_t size = std::min(ObTmpMacroBlock::get_default_page_size() - offset, remain_size); - int32_t page_nums = 0; - common::ObIArray *page_io_infos = nullptr; - - void *buf = - ob_malloc(sizeof(common::ObSEArray), - ObMemAttr(MTL_ID(), "TmpReadPage")); - if (OB_ISNULL(buf)) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "fail to alloc a buf", K(ret)); - } else { - page_io_infos = new (buf) common::ObSEArray(); - do { - ObTmpPageCacheKey key(io_info.block_id_, page_start_id, io_info.tenant_id_); - ObTmpPageValueHandle p_handle; - if (OB_SUCC(page_cache_->get_page(key, p_handle))) { - ObTmpFileIOHandle::ObPageCacheHandle page_handle( - p_handle, - io_info.buf_ + ObTmpMacroBlock::calculate_offset(page_start_id, offset) - io_info.offset_, - offset, - size); - inc_page_cache_num(1); - if (OB_FAIL(handle.get_page_cache_handles().push_back(page_handle))) { - STORAGE_LOG(WARN, "Fail to push back into page_handles", K(ret)); - } - } else if (OB_ENTRY_NOT_EXIST == ret) { - ret = OB_SUCCESS; - // accumulate page io info. - ObTmpPageIOInfo page_io_info; - page_io_info.key_ = key; - page_io_info.offset_ = offset; - page_io_info.size_ = size; - if (OB_FAIL(page_io_infos->push_back(page_io_info))) { - STORAGE_LOG(WARN, "Fail to push back into page_io_infos", K(ret), K(page_io_info)); - } - } else { - STORAGE_LOG(WARN, "fail to get page from page cache", K(ret)); - } - page_nums++; - page_start_id++; - offset = 0; - remain_size -= size; - size = std::min(ObTmpMacroBlock::get_default_page_size(), remain_size); - } while (OB_SUCC(ret) && size > 0); - } - - if (OB_SUCC(ret)) { - if (page_io_infos->count() > DEFAULT_PAGE_IO_MERGE_RATIO * page_nums) { - // merge multi page io into one. - ObMacroBlockHandle mb_handle; - ObTmpBlockIOInfo info(io_info); - const int64_t p_offset = common::lower_align(io_info.offset_, ObTmpMacroBlock::get_default_page_size()); - // just skip header and padding. - info.offset_ = p_offset + ObTmpMacroBlock::get_header_padding(); - info.size_ = page_nums * ObTmpMacroBlock::get_default_page_size(); - info.macro_block_id_ = block->get_macro_block_id(); - if (handle.is_disable_page_cache()) { - if (OB_FAIL(page_cache_->direct_read(info, mb_handle, io_allocator_))) { - STORAGE_LOG(WARN, "fail to direct read multi page", K(ret)); - } - } else { - if (OB_FAIL(page_cache_->prefetch(info, *page_io_infos, mb_handle, io_allocator_))) { - STORAGE_LOG(WARN, "fail to prefetch multi tmp page", K(ret)); - } - } - if (OB_SUCC(ret)) { - ObTmpFileIOHandle::ObIOReadHandle read_handle(mb_handle, io_info.buf_, - io_info.offset_ - p_offset, io_info.size_); - if (OB_FAIL(handle.get_io_handles().push_back(read_handle))) { - STORAGE_LOG(WARN, "Fail to push back into read_handles", K(ret)); - } - } - } else { - // just do io, page by page. - for (int i = 0; OB_SUCC(ret) && i < page_io_infos->count(); i++) { - ObMacroBlockHandle mb_handle; - ObTmpBlockIOInfo info(io_info); - info.offset_ = page_io_infos->at(i).key_.get_page_id() * ObTmpMacroBlock::get_default_page_size(); - // just skip header and padding. - info.offset_ += ObTmpMacroBlock::get_header_padding(); - info.size_ = ObTmpMacroBlock::get_default_page_size(); - info.macro_block_id_ = block->get_macro_block_id(); - if (handle.is_disable_page_cache()) { - if (OB_FAIL(page_cache_->direct_read(info, mb_handle, io_allocator_))) { - STORAGE_LOG(WARN, "fail to direct read tmp page", K(ret)); - } - } else { - if (OB_FAIL(page_cache_->prefetch(page_io_infos->at(i).key_, info, mb_handle, io_allocator_))) { - STORAGE_LOG(WARN, "fail to prefetch tmp page", K(ret)); - } - } - if (OB_SUCC(ret)) { - char *buf = io_info.buf_ + ObTmpMacroBlock::calculate_offset( - page_io_infos->at(i).key_.get_page_id(), page_io_infos->at(i).offset_) - io_info.offset_; - ObTmpFileIOHandle::ObIOReadHandle read_handle(mb_handle, buf, page_io_infos->at(i).offset_, - page_io_infos->at(i).size_); - if (OB_FAIL(handle.get_io_handles().push_back(read_handle))) { - STORAGE_LOG(WARN, "Fail to push back into read_handles", K(ret)); - } - } - } - } - } - if (OB_NOT_NULL(page_io_infos)) { - page_io_infos->destroy(); - ob_free(page_io_infos); - } - return ret; -} - -int ObTmpTenantFileStore::write(const ObTmpBlockIOInfo &io_info) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *block = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(io_info.block_id_, block))) { - STORAGE_LOG(WARN, "fail to get block from tmp block manager", K(ret), K_(io_info.block_id)); - } else if (OB_ISNULL(block)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "the block is NULL", K(ret), K_(io_info.block_id)); - } else if (block->is_memory()) { - block->set_io_desc(io_info.io_desc_); - MEMCPY(block->get_buffer() + io_info.offset_, io_info.buf_, io_info.size_); - } else { - // The washing and disked block shouldn't write data, Otherwise, it will cause data corrupt. - ret = OB_NOT_SUPPORTED; - STORAGE_LOG(WARN, "block status is not correct", K(ret), K(io_info)); - } - return ret; -} - -int ObTmpTenantFileStore::wash_block(const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle) -{ - int ret = OB_SUCCESS; - SpinWLockGuard guard(lock_); - ObTmpMacroBlock *block = NULL; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret), K(block_id)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(block_id, block))) { - STORAGE_LOG(WARN, "fail to get block from tmp block manager", K(ret), K(block_id)); - } else if (OB_ISNULL(block)) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "the block is NULL", K(ret), K(block_id)); - } else if (OB_FAIL(tmp_mem_block_manager_.wash_block(block_id, handle))) { - STORAGE_LOG(WARN, "wash block failed", K(ret), K(block_id)); - } - return ret; -} - -int ObTmpTenantFileStore::sync_block(const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *blk = NULL; - SpinRLockGuard guard(lock_); - bool is_closed = false; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret), K(block_id)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(block_id, blk))) { - STORAGE_LOG(WARN, "fail to get macro block", K(ret), K(block_id)); - } else if (OB_FAIL(blk->is_extents_closed(is_closed))) { - STORAGE_LOG(WARN, "check block closed failed", K(ret), K(block_id)); - } else if (!is_closed) { - //do nothing - } else if (OB_FAIL(tmp_mem_block_manager_.wash_block(block_id, handle))) { - STORAGE_LOG(WARN, "wash block failed", K(ret), K(block_id)); - } else { - STORAGE_LOG(DEBUG, "succeed to sync block", K(block_id)); - } - return ret; -} - -int ObTmpTenantFileStore::wait_write_finish(const int64_t block_id, const int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - ObTmpMacroBlock *blk = NULL; - bool is_closed = false; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret), K(block_id)); - } else if (OB_UNLIKELY(block_id <= 0)) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(block_id)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(block_id, blk))) { - STORAGE_LOG(WARN, "fail to get macro block", K(ret), K(block_id)); - } else if (OB_ISNULL(blk)) { - ret = OB_ERR_NULL_VALUE; - STORAGE_LOG(WARN, "macro block is NULL", K(ret), K(block_id)); - } else if (!blk->is_washing()) { - // block has not been washed, nothing todo. - } else if (blk->is_washing() && - OB_FAIL(tmp_mem_block_manager_.wait_write_finish(block_id, timeout_ms))){ - STORAGE_LOG(WARN, "wait write finish failed", K(ret), K(block_id)); - } - return ret; -} - -int ObTmpTenantFileStore::get_disk_macro_block_count(int64_t &count) const -{ - int ret = OB_SUCCESS; - SpinRLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(tmp_block_manager_.get_disk_macro_block_count(count))) { - STORAGE_LOG(WARN, "fail to get disk macro block count from tmp_block_manager_", K(ret)); - } - return ret; -} - -int ObTmpTenantFileStore::get_disk_macro_block_list(common::ObIArray ¯o_id_list) -{ - int ret = OB_SUCCESS; - SpinRLockGuard guard(lock_); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(tmp_block_manager_.get_disk_macro_block_list(macro_id_list))) { - STORAGE_LOG(WARN, "fail to get disk macro block list from tmp_block_manager_", K(ret)); - } - return ret; -} - -int ObTmpTenantFileStore::get_macro_block(const int64_t block_id, ObTmpMacroBlock *&t_mblk) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpTenantFileStore has not been inited", K(ret)); - } else if (OB_FAIL(tmp_block_manager_.get_macro_block(block_id, t_mblk))) { - STORAGE_LOG(WARN, "ObTmpTenantFileStore get macro block failed", K(ret)); - } - - return ret; -} - -ObTmpTenantFileStoreHandle::ObTmpTenantFileStoreHandle() - : tenant_store_(), allocator_() -{ -} - -ObTmpTenantFileStoreHandle::~ObTmpTenantFileStoreHandle() -{ - reset(); -} -void ObTmpTenantFileStoreHandle::set_tenant_store(ObTmpTenantFileStore *tenant_store, - common::ObConcurrentFIFOAllocator *allocator) -{ - if (OB_NOT_NULL(tenant_store)) { - reset(); - tenant_store->inc_ref(); - tenant_store_ = tenant_store; - allocator_ = allocator; - } -} - -ObTmpTenantFileStoreHandle& -ObTmpTenantFileStoreHandle::operator=(const ObTmpTenantFileStoreHandle &other) -{ - if (&other != this) { - set_tenant_store(other.tenant_store_, other.allocator_); - } - return *this; -} - -bool ObTmpTenantFileStoreHandle::is_empty() const -{ - return NULL == tenant_store_; -} - -bool ObTmpTenantFileStoreHandle::is_valid() const -{ - return NULL != tenant_store_; -} - -void ObTmpTenantFileStoreHandle::reset() -{ - if (OB_NOT_NULL(tenant_store_)) { - int64_t tmp_ref = tenant_store_->dec_ref(); - if (0 == tmp_ref) { - tenant_store_->~ObTmpTenantFileStore(); - allocator_->free(tenant_store_); - } - tenant_store_ = NULL; - } -} - -ObTmpFileStore &ObTmpFileStore::get_instance() -{ - static ObTmpFileStore instance; - return instance; -} - -ObTmpFileStore::ObTmpFileStore() - : next_blk_id_(0), - tenant_file_stores_(), - lock_(), - is_inited_(false), - allocator_() -{ -} - -ObTmpFileStore::~ObTmpFileStore() -{ -} - -int ObTmpFileStore::init() -{ - int ret = OB_SUCCESS; - ObMemAttr attr = SET_USE_500(ObModIds::OB_TMP_FILE_STORE_MAP); - if (IS_INIT) { - ret = OB_INIT_TWICE; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); - } else if (OB_FAIL(allocator_.init(TOTAL_LIMIT, HOLD_LIMIT, BLOCK_SIZE))) { - STORAGE_LOG(WARN, "fail to init allocator", K(ret)); - } else if (OB_FAIL(ObTmpPageCache::get_instance().init("tmp_page_cache", - TMP_FILE_PAGE_CACHE_PRIORITY))) { - STORAGE_LOG(WARN, "Fail to init tmp page cache, ", K(ret)); - } else if (OB_FAIL(ObTmpBlockCache::get_instance().init("tmp_block_cache", - TMP_FILE_BLOCK_CACHE_PRIORITY))) { - STORAGE_LOG(WARN, "Fail to init tmp tenant block cache, ", K(ret)); - } else if (OB_FAIL(tenant_file_stores_.create(STORE_HASH_BUCKET_NUM, - attr, attr))) { - STORAGE_LOG(WARN, "Fail to create tmp tenant file store map, ", K(ret)); - } else { - allocator_.set_attr(attr); - is_inited_ = true; - } - if (!is_inited_) { - destroy(); - } - return ret; -} - -int ObTmpFileStore::alloc(const int64_t dir_id, const uint64_t tenant_id, const int64_t size, - ObTmpFileExtent &extent) -{ - int ret = OB_SUCCESS; - DISABLE_SQL_MEMLEAK_GUARD; - 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 if (OB_FAIL(store_handle.get_tenant_store()->alloc(dir_id, tenant_id, size, extent))) { - STORAGE_LOG(WARN, "fail to allocate extents", K(ret), K(tenant_id), K(dir_id), K(size), - K(extent)); - } - return ret; -} - -int ObTmpFileStore::read(const uint64_t tenant_id, ObTmpBlockIOInfo &io_info, - ObTmpFileIOHandle &handle) -{ - 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), K(tenant_id), - K(io_info), K(handle)); - } else if (OB_FAIL(store_handle.get_tenant_store()->read(io_info, handle))) { - STORAGE_LOG(WARN, "fail to read the extent", K(ret), K(tenant_id), K(io_info), K(handle)); - } - return ret; -} - -int ObTmpFileStore::write(const uint64_t tenant_id, const ObTmpBlockIOInfo &io_info) -{ - 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), K(io_info)); - } else if (OB_FAIL(store_handle.get_tenant_store()->write(io_info))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret), K(tenant_id), K(io_info)); - } - return ret; -} - -int ObTmpFileStore::wash_block(const uint64_t tenant_id, const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle) -{ - 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), K(block_id)); - } else if (OB_FAIL(store_handle.get_tenant_store()->wash_block(block_id, handle))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret), K(tenant_id), K(block_id)); - } - return ret; -} - -int ObTmpFileStore::wait_write_finish(const uint64_t tenant_id, const int64_t block_id, const int64_t timeout_ms) -{ - 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 if (OB_FAIL(store_handle.get_tenant_store()->wait_write_finish(block_id, timeout_ms))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret), K(tenant_id)); - } - return ret; -} - -int ObTmpFileStore::sync_block(const uint64_t tenant_id, const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle) -{ - 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 if (OB_FAIL(store_handle.get_tenant_store()->sync_block(block_id, handle))) { - STORAGE_LOG(WARN, "fail to write the extent", K(ret), K(tenant_id)); - } - return ret; -} - -int ObTmpFileStore::inc_block_cache_num(const uint64_t tenant_id, const 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 { - store_handle.get_tenant_store()->inc_block_cache_num(num); - } - return ret; -} - -int ObTmpFileStore::dec_block_cache_num(const uint64_t tenant_id, const 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 { - store_handle.get_tenant_store()->dec_block_cache_num(num); - } - return ret; -} - -int ObTmpFileStore::inc_page_cache_num(const uint64_t tenant_id, const 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 { - store_handle.get_tenant_store()->inc_page_cache_num(num); - } - return ret; -} - -int ObTmpFileStore::dec_page_cache_num(const uint64_t tenant_id, const 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 { - store_handle.get_tenant_store()->dec_page_cache_num(num); - } - return ret; -} - -int ObTmpFileStore::free(const uint64_t tenant_id, ObTmpFileExtent *extent) -{ - 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), K(*extent)); - } else if (OB_FAIL(store_handle.get_tenant_store()->free(extent))) { - STORAGE_LOG(WARN, "fail to free extents", K(ret), K(tenant_id), K(*extent)); - } - return ret; -} - -int ObTmpFileStore::free(const uint64_t tenant_id, const int64_t block_id, - const int32_t start_page_id, const int32_t page_nums) -{ - 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 if (OB_FAIL(store_handle.get_tenant_store()->free(block_id, start_page_id, page_nums))) { - STORAGE_LOG(WARN, "fail to free", K(ret), K(tenant_id), K(block_id), K(start_page_id), - K(page_nums)); - } - return ret; -} - -int ObTmpFileStore::free_tenant_file_store(const uint64_t tenant_id) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(tenant_file_stores_.erase_refactored(tenant_id))) { - if (OB_HASH_NOT_EXIST == ret) { - ret = OB_ENTRY_NOT_EXIST; - } else { - STORAGE_LOG(WARN, "fail to erase tmp tenant file store", K(ret), K(tenant_id)); - } - } - return ret; -} - -int ObTmpFileStore::get_macro_block(const int64_t tenant_id, const int64_t block_id, ObTmpMacroBlock *&t_mblk) -{ - 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 if (OB_FAIL(store_handle.get_tenant_store()->get_macro_block(block_id, t_mblk))) { - STORAGE_LOG(WARN, "fail to free", K(ret), K(tenant_id), K(block_id)); - } - return ret; -} - -int ObTmpFileStore::get_macro_block_list(common::ObIArray ¯o_id_list) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); - } else { - macro_id_list.reset(); - TenantFileStoreMap::iterator iter; - ObTmpTenantFileStore *tmp = NULL; - for (iter = tenant_file_stores_.begin(); OB_SUCC(ret) && iter != tenant_file_stores_.end(); - ++iter) { - if (OB_ISNULL(tmp = iter->second.get_tenant_store())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to iterate tmp tenant file store", K(ret)); - } else if (OB_FAIL(tmp->get_disk_macro_block_list(macro_id_list))){ - STORAGE_LOG(WARN, "fail to get list of tenant macro block in disk", K(ret)); - } - } - } - return ret; -} - -int ObTmpFileStore::get_macro_block_list(ObIArray &tmp_block_cnt_pairs) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); - } else { - tmp_block_cnt_pairs.reset(); - TenantFileStoreMap::iterator iter; - ObTmpTenantFileStore *tmp = NULL; - for (iter = tenant_file_stores_.begin(); OB_SUCC(ret) && iter != tenant_file_stores_.end(); - ++iter) { - int64_t macro_id_count = 0; - TenantTmpBlockCntPair pair; - if (OB_ISNULL(tmp = iter->second.get_tenant_store())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to iterate tmp tenant file store", K(ret)); - } else if (OB_FAIL(tmp->get_disk_macro_block_count(macro_id_count))){ - STORAGE_LOG(WARN, "fail to get list of tenant macro block in disk", K(ret)); - } else if (OB_FAIL(pair.init(iter->first, macro_id_count))) { - STORAGE_LOG(WARN, "fail to init tenant tmp block count pair", K(ret), "tenant id", - iter->first, "macro block count", macro_id_count); - } else if (OB_FAIL(tmp_block_cnt_pairs.push_back(pair))) { - STORAGE_LOG(WARN, "fail to push back tmp_block_cnt_pairs", K(ret), K(pair)); - } - } - } - return ret; -} - -int ObTmpFileStore::get_all_tenant_id(common::ObIArray &tenant_ids) -{ - int ret = OB_SUCCESS; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret)); - } else { - tenant_ids.reset(); - TenantFileStoreMap::iterator iter; - SpinRLockGuard guard(lock_); - for (iter = tenant_file_stores_.begin(); OB_SUCC(ret) && iter != tenant_file_stores_.end(); - ++iter) { - if (OB_ISNULL(iter->second.get_tenant_store())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to iterate tmp tenant file store", K(ret)); - } else if (OB_FAIL(tenant_ids.push_back(iter->first))) { - STORAGE_LOG(WARN, "fail to push back tmp_block_cnt_pairs", K(ret)); - } - } - } - return ret; -} - -int ObTmpFileStore::get_store(const uint64_t tenant_id, ObTmpTenantFileStoreHandle &handle) -{ - int ret = OB_SUCCESS; - DISABLE_SQL_MEMLEAK_GUARD; - void *buf = NULL; - handle.reset(); - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret), K(tenant_id)); - } else { - SpinRLockGuard guard(lock_); - if (OB_FAIL(tenant_file_stores_.get_refactored(tenant_id, handle))) { - if (OB_HASH_NOT_EXIST == ret) { - STORAGE_LOG(DEBUG, "ObTmpFileStore get tenant store failed", K(ret), K(tenant_id)); - } else { - STORAGE_LOG(WARN, "ObTmpFileStore get tenant store failed", K(ret), K(tenant_id)); - } - } - } - - if (OB_UNLIKELY(OB_HASH_NOT_EXIST == ret)) { - SpinWLockGuard guard(lock_); - if (OB_FAIL(tenant_file_stores_.get_refactored(tenant_id, handle))) { - if (OB_HASH_NOT_EXIST == ret) { - ObTmpTenantFileStore *store = NULL; - if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObTmpTenantFileStore)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - STORAGE_LOG(WARN, "fail to alloc a buf", K(ret), K(tenant_id)); - } else if (OB_ISNULL(store = new (buf) ObTmpTenantFileStore())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "fail to new a ObTmpTenantFileStore", K(ret), K(tenant_id)); - } else if (OB_FAIL(store->init(tenant_id))) { - store->~ObTmpTenantFileStore(); - allocator_.free(store); - store = NULL; - STORAGE_LOG(WARN, "fail to init ObTmpTenantFileStore", K(ret), K(tenant_id)); - } else if (FALSE_IT(handle.set_tenant_store(store, &allocator_))) { - } else if (OB_FAIL(tenant_file_stores_.set_refactored(tenant_id, handle))) { - STORAGE_LOG(WARN, "fail to set tenant_file_stores_", K(ret), K(tenant_id)); - } - } else { - STORAGE_LOG(WARN, "fail to get tmp tenant file store", K(ret), K(tenant_id)); - } - } - } - if (OB_SUCC(ret)) { - if (OB_UNLIKELY(!handle.is_valid())) { - ret = OB_ERR_UNEXPECTED; - STORAGE_LOG(WARN, "unexpected error, invalid tenant file store handle", K(ret), K(handle)); - } - } - return ret; -} - -int64_t ObTmpFileStore::get_next_blk_id() -{ - int64_t next_blk_id = -1; - int64_t old_val = ATOMIC_LOAD(&next_blk_id_); - int64_t new_val = 0; - bool finish = false; - while (!finish) { - new_val = (old_val + 1) % INT64_MAX; - next_blk_id = new_val; - finish = (old_val == (new_val = ATOMIC_VCAS(&next_blk_id_, old_val, new_val))); - old_val = new_val; - } - return next_blk_id; -} - -int ObTmpFileStore::get_tenant_extent_allocator(const int64_t tenant_id, common::ObIAllocator *&allocator) -{ - int ret = OB_SUCCESS; - ObTmpTenantFileStoreHandle store_handle; - if (IS_NOT_INIT) { - ret = OB_NOT_INIT; - STORAGE_LOG(WARN, "ObTmpFileStore has not been inited", K(ret), K(tenant_id)); - } else 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 { - allocator = &(store_handle.get_tenant_store()->get_extent_allocator()); - } - return ret; -} - -void ObTmpFileStore::destroy() -{ - ObTmpPageCache::get_instance().destroy(); - tenant_file_stores_.destroy(); - allocator_.destroy(); - ObTmpBlockCache::get_instance().destroy(); - is_inited_ = false; -} - -} // end namespace blocksstable -} // end namespace oceanbase diff --git a/src/storage/blocksstable/ob_tmp_file_store.h b/src/storage/blocksstable/ob_tmp_file_store.h deleted file mode 100644 index c0455158eb..0000000000 --- a/src/storage/blocksstable/ob_tmp_file_store.h +++ /dev/null @@ -1,393 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_STORE_H_ -#define OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_STORE_H_ - -#include "storage/blocksstable/ob_macro_block_common_header.h" -#include "storage/blocksstable/ob_macro_block_handle.h" -#include "storage/blocksstable/ob_block_manager.h" -#include "storage/blocksstable/ob_tmp_file_cache.h" - -namespace oceanbase -{ -namespace blocksstable -{ - -class ObTmpFile; -class ObTmpFileExtent; -class ObTmpFileIOHandle; -class ObTmpTenantMacroBlockManager; -struct ObTmpBlockValueHandle; -class ObTmpTenantBlockCache; -class ObTmpPageCache; - -typedef common::ObSEArray ExtentArray; - -struct ObTmpFileArea final -{ -public: - ObTmpFileArea(const uint8_t &start_page_id, const uint8_t &page_nums) - : start_page_id_(start_page_id), page_nums_(page_nums), next_(NULL) {} - ~ObTmpFileArea() - { - start_page_id_ = 0; - page_nums_ = 0; - next_ = NULL; - } - TO_STRING_KV(K_(start_page_id), K_(page_nums)); - uint8_t start_page_id_; - uint8_t page_nums_; - ObTmpFileArea *next_; -} __attribute__((packed, __aligned__(1)));; - -class ObTmpFilePageBuddy final -{ -public: - ObTmpFilePageBuddy(); - ~ObTmpFilePageBuddy(); - int init(common::ObIAllocator &allocator); - void destroy(); - int alloc_all_pages(); - int alloc(const uint8_t page_nums, uint8_t &start_page_id, uint8_t &alloced_page_nums); - void free(const int32_t start_page_id, const int32_t page_nums); - OB_INLINE uint8_t get_max_cont_page_nums() const { return max_cont_page_nums_; } - bool is_inited() { return is_inited_; } - bool is_empty() const; - int64_t to_string(char* buf, const int64_t buf_len) const; - - static const uint8_t MAX_PAGE_NUMS = 252; // 2^MAX_ORDER - 2^MIN_ORDER, uint8_t is only for < 256 - -private: - void free_align(const int32_t start_page_id, const int32_t page_nums, ObTmpFileArea *&area); - ObTmpFileArea *find_buddy(const int32_t page_nums, const int32_t start_page_id); - -private: - static const uint8_t MIN_ORDER = 2; - static const uint8_t MAX_ORDER = 8; - bool is_inited_; - uint8_t max_cont_page_nums_; - ObTmpFileArea *buf_; - common::ObIAllocator *allocator_; - ObTmpFileArea *free_area_[ObTmpFilePageBuddy::MAX_ORDER]; - DISALLOW_COPY_AND_ASSIGN(ObTmpFilePageBuddy); -}; - -struct ObTmpFileMacroBlockHeader final -{ -public: - ObTmpFileMacroBlockHeader(); - ~ObTmpFileMacroBlockHeader() = default; - bool is_valid() const; - int serialize(char *buf, const int64_t buf_len, int64_t &pos) const; - int deserialize(const char *buf, const int64_t data_len, int64_t &pos); - static int64_t get_serialize_size() { return sizeof(ObTmpFileMacroBlockHeader); } - void reset(); - - TO_STRING_KV(K_(version), K_(magic), K_(block_id), K_(dir_id), K_(tenant_id), K_(free_page_nums)); -private: - static const int32_t TMP_FILE_MACRO_BLOCK_HEADER_VERSION = 1; - static const int32_t TMP_FILE_MACRO_BLOCK_HEADER_MAGIC = 20720; -public: - int32_t version_; - int32_t magic_; - int64_t block_id_; - int64_t dir_id_; - uint64_t tenant_id_; - int64_t free_page_nums_; -}; - -struct ObTmpBlockIOInfo final -{ -public: - ObTmpBlockIOInfo() - : block_id_(0), offset_(0), size_(0), io_timeout_ms_(DEFAULT_IO_WAIT_TIME_MS), tenant_id_(0), - buf_(NULL), io_desc_(), macro_block_id_() {} - ObTmpBlockIOInfo(const int64_t block_id, const int64_t offset, const int64_t size, - const uint64_t tenant_id, const MacroBlockId macro_block_id, char *buf, - const common::ObIOFlag io_desc) - : block_id_(block_id), offset_(offset), size_(size), tenant_id_(tenant_id), - buf_(buf), io_desc_(io_desc), macro_block_id_(macro_block_id) {} - TO_STRING_KV(K_(block_id), K_(offset), K_(size), K_(io_timeout_ms), K_(tenant_id), K_(macro_block_id), KP_(buf), - K_(io_desc)); - int64_t block_id_; - int64_t offset_; - int64_t size_; - int64_t io_timeout_ms_; - uint64_t tenant_id_; - char *buf_; - common::ObIOFlag io_desc_; - MacroBlockId macro_block_id_; -}; - -class ObTmpMacroBlock final -{ -public: - enum BlockStatus: uint8_t { - MEMORY = 0, - WASHING, - DISKED, - MAX, - }; - ObTmpMacroBlock(); - ~ObTmpMacroBlock(); - int init(const int64_t block_id, const int64_t dir_id, const uint64_t tenant_id, - common::ObIAllocator &allocator); - void destroy(); - int alloc_all_pages(ObTmpFileExtent &extent); - int alloc(const uint8_t page_nums, ObTmpFileExtent &extent); - int free(ObTmpFileExtent &extent); - int free(const int32_t start_page_id, const int32_t page_nums); - OB_INLINE void set_buffer(char *buf) { buffer_ = buf; } - OB_INLINE char *get_buffer() { return buffer_; } - OB_INLINE uint8_t get_max_cont_page_nums() const { return page_buddy_.get_max_cont_page_nums(); } - OB_INLINE uint8_t get_free_page_nums() const { return tmp_file_header_.free_page_nums_; } - int64_t get_used_page_nums() const; - int get_block_cache_handle(ObTmpBlockValueHandle &handle); - int get_wash_io_info(ObTmpBlockIOInfo &info); - 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 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; } - OB_INLINE bool is_inited() const { return is_inited_; } - static int64_t get_default_page_size() { return DEFAULT_PAGE_SIZE; } - static int64_t calculate_offset(const int64_t page_start_id, const int64_t offset) - { - return page_start_id * ObTmpMacroBlock::DEFAULT_PAGE_SIZE + offset; - } - static int64_t get_header_padding() - { - return 4 * DEFAULT_PAGE_SIZE; - } - OB_INLINE int64_t get_block_id() const { return tmp_file_header_.block_id_; } - OB_INLINE const ObTmpFileMacroBlockHeader &get_tmp_block_header() const { return tmp_file_header_; } - OB_INLINE uint64_t get_tenant_id() const { return tmp_file_header_.tenant_id_; } - OB_INLINE int64_t get_dir_id() const { return tmp_file_header_.dir_id_; } - OB_INLINE const MacroBlockId& get_macro_block_id() const { return macro_block_handle_.get_macro_id(); } - OB_INLINE ObMacroBlockHandle &get_macro_block_handle() { return macro_block_handle_; } - OB_INLINE int64_t get_alloc_time() const { return alloc_time_; } - OB_INLINE int64_t get_access_time() const { return ATOMIC_LOAD(&access_time_); } - OB_INLINE double get_wash_score(int64_t cur_time) const { - if (get_used_page_nums() == ObTmpFilePageBuddy::MAX_PAGE_NUMS) { - return INT64_MAX; - } - return (double) get_used_page_nums() * (cur_time - get_alloc_time()) / (get_access_time() - get_alloc_time()); - } - common::ObIArray &get_extents() { return using_extents_; } - ObTmpBlockValueHandle &get_handle() { return handle_; } - bool is_empty() const { return page_buddy_.is_empty(); } - int seal(bool &is_sealed); - int is_extents_closed(bool &is_extents_closed); - 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), - K_(io_desc), K_(block_status), K_(is_inited), K_(alloc_time), K_(access_time)); -private: - bool is_sealed() const { return ATOMIC_LOAD(&is_sealed_); } -private: - static const int64_t DEFAULT_PAGE_SIZE; - char *buffer_; - ObTmpFilePageBuddy page_buddy_; - ObTmpBlockValueHandle handle_; - ExtentArray using_extents_; - ObMacroBlockHandle macro_block_handle_; - ObTmpFileMacroBlockHeader tmp_file_header_; - common::ObIOFlag io_desc_; - common::SpinRWLock lock_; - BlockStatus block_status_; - bool is_sealed_; - bool is_inited_; - int64_t alloc_time_; - int64_t access_time_; - DISALLOW_COPY_AND_ASSIGN(ObTmpMacroBlock); -}; - -class ObTmpTenantMacroBlockManager final -{ -public: - ObTmpTenantMacroBlockManager(); - ~ObTmpTenantMacroBlockManager(); - int init(const uint64_t tenant_id, common::ObIAllocator &allocator); - void destroy(); - int alloc_macro_block(const int64_t dir_id, const uint64_t tenant_id, ObTmpMacroBlock *&t_mblk); - int free_macro_block(const int64_t block_id); - int get_macro_block(const int64_t block_id, ObTmpMacroBlock *&t_mblk); - int get_disk_macro_block_count(int64_t &count) const; - int get_disk_macro_block_list(common::ObIArray ¯o_id_list); - void print_block_usage(); - -private: - static const uint64_t MBLK_HASH_BUCKET_NUM = 10243L; - typedef common::hash::ObHashMap - TmpMacroBlockMap; - common::ObIAllocator *allocator_; - TmpMacroBlockMap blocks_; // all of block meta. - bool is_inited_; - DISALLOW_COPY_AND_ASSIGN(ObTmpTenantMacroBlockManager); -}; - -class ObTmpTenantFileStore final -{ -public: - ObTmpTenantFileStore(); - ~ObTmpTenantFileStore(); - int init(const uint64_t tenant_id); - void destroy(); - int alloc(const int64_t dir_id, const uint64_t tenant_id, const int64_t alloc_size, - ObTmpFileExtent &extent); - int free(ObTmpFileExtent *extent); - int free(const int64_t block_id, const int32_t start_page_id, const int32_t page_nums); - int read(ObTmpBlockIOInfo &io_info, ObTmpFileIOHandle &handle); - int write(const ObTmpBlockIOInfo &io_info); - int wash_block(const int64_t block_id, ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle); - void refresh_memory_limit(const uint64_t tenant_id); - int sync_block(const int64_t block_id, ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle); - int wait_write_finish(const int64_t block_id, const int64_t timeout_ms); - int get_disk_macro_block_count(int64_t &count) const; - int get_disk_macro_block_list(common::ObIArray ¯o_id_list); - int get_macro_block(const int64_t block_id, ObTmpMacroBlock *&t_mblk); - // use io_allocator_ to allocate tenant extent memory. - common::ObIAllocator &get_extent_allocator() { return allocator_; } - void print_block_usage() { tmp_block_manager_.print_block_usage(); } - OB_INLINE void inc_page_cache_num(const int64_t num) { - ATOMIC_FAA(&page_cache_num_, num); - }; - OB_INLINE void dec_page_cache_num(const int64_t num) { - ATOMIC_FAS(&page_cache_num_, num); - }; - OB_INLINE void inc_block_cache_num(const int64_t num) { - ATOMIC_FAA(&block_cache_num_, num); - }; - 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(); - -private: - int read_page(ObTmpMacroBlock *block, ObTmpBlockIOInfo &io_info, ObTmpFileIOHandle &handle); - int free_extent(ObTmpFileExtent *extent); - int free_extent(const int64_t block_id, const int32_t start_page_id, const int32_t page_nums); - int free_macro_block(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); - int wait_write_io_finish_if_need(); - -private: - static const uint64_t IO_LIMIT = 4 * 1024L * 1024L * 1024L; - static const uint64_t TOTAL_LIMIT = 15 * 1024L * 1024L * 1024L; - static const uint64_t HOLD_LIMIT = 8 * 1024L * 1024L; - static const uint64_t REFRESH_CONFIG_INTERVAL = 5 * 60 * 1000 * 1000L; // 5min - static const uint64_t BLOCK_SIZE = common::OB_MALLOC_MIDDLE_BLOCK_SIZE; - static constexpr double DEFAULT_PAGE_IO_MERGE_RATIO = 0.5; - - bool is_inited_; - int64_t page_cache_num_; - int64_t block_cache_num_; - volatile int64_t ref_cnt_; - ObTmpPageCache *page_cache_; - common::SpinRWLock lock_; - common::ObConcurrentFIFOAllocator allocator_; - common::ObFIFOAllocator io_allocator_; - ObTmpTenantMacroBlockManager tmp_block_manager_; - ObTmpTenantMemBlockManager tmp_mem_block_manager_; - int64_t last_access_tenant_config_ts_; - int64_t last_meta_mem_limit_; - - DISALLOW_COPY_AND_ASSIGN(ObTmpTenantFileStore); -}; - -struct ObTmpTenantFileStoreHandle final -{ -public: - ObTmpTenantFileStoreHandle(); - ~ObTmpTenantFileStoreHandle(); - ObTmpTenantFileStoreHandle(const ObTmpTenantFileStoreHandle &other); - ObTmpTenantFileStoreHandle &operator=(const ObTmpTenantFileStoreHandle &other); - void set_tenant_store(ObTmpTenantFileStore *store, common::ObConcurrentFIFOAllocator *allocator); - bool is_empty() const; - bool is_valid() const; - void reset(); - OB_INLINE ObTmpTenantFileStore* get_tenant_store() const { return tenant_store_; } - TO_STRING_KV(KP_(tenant_store), KP_(allocator)); -private: - ObTmpTenantFileStore *tenant_store_; - common::ObConcurrentFIFOAllocator *allocator_; -}; - -class ObTmpFileStore final -{ -public: - typedef common::hash::HashMapPair TenantTmpBlockCntPair; - - static ObTmpFileStore &get_instance(); - int init(); - void destroy(); - - int alloc(const int64_t dir_id, const uint64_t tenant_id, const int64_t size, - ObTmpFileExtent &extent); - int read(const uint64_t tenant_id, ObTmpBlockIOInfo &io_info, ObTmpFileIOHandle &handle); - int write(const uint64_t tenant_id, const ObTmpBlockIOInfo &io_info); - int wash_block(const uint64_t tenant_id, const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle); - int sync_block(const uint64_t tenant_id, const int64_t block_id, - ObTmpTenantMemBlockManager::ObIOWaitInfoHandle &handle); - int wait_write_finish(const uint64_t tenant_id, const int64_t block_id, const int64_t timeout_ms); - int free(const uint64_t tenant_id, ObTmpFileExtent *extent); - int free(const uint64_t tenant_id, const int64_t block_id, const int32_t start_page_id, - const int32_t page_nums); - int free_tenant_file_store(const uint64_t tenant_id); - int get_macro_block(const int64_t tenant_id, const int64_t block_id, ObTmpMacroBlock *&t_mblk); - int get_macro_block_list(common::ObIArray ¯o_id_list); - int get_macro_block_list(common::ObIArray &tmp_block_cnt_pairs); - int get_all_tenant_id(common::ObIArray &tenant_ids); - int64_t get_next_blk_id(); - int get_tenant_extent_allocator(const int64_t tenant_id, common::ObIAllocator *&allocator); - - static int64_t get_block_size() - { - return ObTmpFilePageBuddy::MAX_PAGE_NUMS * ObTmpMacroBlock::get_default_page_size(); - } - int inc_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 dec_block_cache_num(const uint64_t tenant_id, const int64_t num); -private: - ObTmpFileStore(); - ~ObTmpFileStore(); - int get_store(const uint64_t tenant_id, ObTmpTenantFileStoreHandle &handle); - -private: - static const uint64_t STORE_HASH_BUCKET_NUM = 1543L; - static const int64_t TOTAL_LIMIT = 512L * 1024L * 1024L; - static const int64_t HOLD_LIMIT = 8 * 1024L * 1024L; - static const int64_t BLOCK_SIZE = common::OB_MALLOC_NORMAL_BLOCK_SIZE; - static const int TMP_FILE_PAGE_CACHE_PRIORITY = 1; - static const int TMP_FILE_BLOCK_CACHE_PRIORITY = 1; - typedef common::hash::ObHashMap TenantFileStoreMap; - int64_t next_blk_id_; - TenantFileStoreMap tenant_file_stores_; - common::SpinRWLock lock_; - bool is_inited_; - common::ObConcurrentFIFOAllocator allocator_; -}; - -#define OB_TMP_FILE_STORE (::oceanbase::blocksstable::ObTmpFileStore::get_instance()) - -} // end namespace blocksstable -} // end namespace oceanbase -#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_STORE_H_ diff --git a/src/storage/direct_load/ob_direct_load_tmp_file.cpp b/src/storage/direct_load/ob_direct_load_tmp_file.cpp index af2cc280b6..ec64853cd3 100644 --- a/src/storage/direct_load/ob_direct_load_tmp_file.cpp +++ b/src/storage/direct_load/ob_direct_load_tmp_file.cpp @@ -181,9 +181,9 @@ ObDirectLoadTmpFileIOHandle::~ObDirectLoadTmpFileIOHandle() void ObDirectLoadTmpFileIOHandle::reset() { tmp_file_ = nullptr; + file_io_handle_.reset(); file_handle_.reset(); io_info_.reset(); - file_io_handle_.reset(); is_cancel_ = false; } @@ -207,7 +207,6 @@ int ObDirectLoadTmpFileIOHandle::open(const ObDirectLoadTmpFileHandle &file_hand LOG_WARN("fail to assign file handle", KR(ret)); } else { tmp_file_ = tmp_file; - io_info_.tenant_id_ = MTL_ID(); io_info_.dir_id_ = tmp_file_->get_file_id().dir_id_; io_info_.fd_ = tmp_file_->get_file_id().fd_; io_info_.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); @@ -424,7 +423,7 @@ int ObDirectLoadTmpFileIOHandle::wait() while (OB_SUCC(ret)) { if (OB_FAIL(check_status())) { LOG_WARN("fail to check status", KR(ret)); - } else if (OB_FAIL(file_io_handle_.wait())) { + } else if (file_io_handle_.is_valid() && OB_FAIL(file_io_handle_.wait())) { LOG_WARN("fail to wait io finish", KR(ret)); if (OB_LIKELY(is_retry_err(ret))) { if (++retry_cnt <= MAX_RETRY_CNT) { @@ -441,34 +440,6 @@ int ObDirectLoadTmpFileIOHandle::wait() return ret; } -int ObDirectLoadTmpFileIOHandle::seek(const ObDirectLoadTmpFileHandle &file_handle, int64_t offset, - int whence) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!file_handle.is_valid())) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid args", KR(ret), K(file_handle)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.seek(file_handle.get_file()->get_file_id().fd_, - offset, whence))) { - LOG_WARN("fail to seek tmp file", KR(ret), K(file_handle), K(offset), K(whence)); - } - return ret; -} - -int ObDirectLoadTmpFileIOHandle::sync(const ObDirectLoadTmpFileHandle &file_handle, - int64_t timeout_ms) -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!file_handle.is_valid())) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid args", KR(ret), K(file_handle)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(file_handle.get_file()->get_file_id().fd_, - timeout_ms))) { - LOG_WARN("fail to sync tmp file", KR(ret), K(file_handle)); - } - return ret; -} - /** * ObDirectLoadTmpFileManager */ diff --git a/src/storage/direct_load/ob_direct_load_tmp_file.h b/src/storage/direct_load/ob_direct_load_tmp_file.h index 95d2037163..b6e180c301 100644 --- a/src/storage/direct_load/ob_direct_load_tmp_file.h +++ b/src/storage/direct_load/ob_direct_load_tmp_file.h @@ -12,7 +12,7 @@ #pragma once #include "observer/table_load/ob_table_load_object_allocator.h" -#include "storage/blocksstable/ob_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" namespace oceanbase { @@ -110,8 +110,6 @@ public: int aio_write(char *buf, int64_t size); int wait(); OB_INLINE void cancel() { is_cancel_ = true; } - static int seek(const ObDirectLoadTmpFileHandle &file_handle, int64_t offset, int whence); - static int sync(const ObDirectLoadTmpFileHandle &file_handle, int64_t timeout_ms); static bool is_retry_err(int ret_code) { return OB_TIMEOUT == ret_code; } TO_STRING_KV(K_(file_handle), K_(io_info)); private: @@ -119,8 +117,8 @@ private: private: ObDirectLoadTmpFileHandle file_handle_; ObDirectLoadTmpFile *tmp_file_; - blocksstable::ObTmpFileIOInfo io_info_; - blocksstable::ObTmpFileIOHandle file_io_handle_; + tmp_file::ObTmpFileIOInfo io_info_; + tmp_file::ObTmpFileIOHandle file_io_handle_; bool is_cancel_; DISABLE_COPY_ASSIGN(ObDirectLoadTmpFileIOHandle); }; diff --git a/src/storage/ob_disk_usage_reporter.cpp b/src/storage/ob_disk_usage_reporter.cpp index 69c4037eec..19dd2d092a 100644 --- a/src/storage/ob_disk_usage_reporter.cpp +++ b/src/storage/ob_disk_usage_reporter.cpp @@ -13,7 +13,7 @@ #include "storage/ob_disk_usage_reporter.h" #include "share/ob_disk_usage_table_operator.h" -#include "storage/blocksstable/ob_tmp_file_store.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" #include "observer/omt/ob_multi_tenant.h" #include "share/rc/ob_tenant_base.h" @@ -33,6 +33,7 @@ namespace oceanbase using namespace common; using namespace share; using namespace logservice; +using namespace tmp_file; namespace storage { @@ -399,19 +400,31 @@ int ObDiskUsageReportTask::count_tenant_tmp() int ret = OB_SUCCESS; ObDiskUsageReportKey report_key; int64_t macro_block_cnt = 0; - common::ObArray tenant_block_cnt_pairs; - if (OB_FAIL(OB_TMP_FILE_STORE.get_macro_block_list(tenant_block_cnt_pairs))) { - STORAGE_LOG(WARN, "failed to get tenant tmp macro block list", K(ret)); + common::ObArray tenant_ids; + + if (OB_FAIL(GCTX.omt_->get_mtl_tenant_ids(tenant_ids))) { + STORAGE_LOG(WARN, "fail to get_mtl_tenant_ids", KR(ret)); } else { report_key.file_type_ = ObDiskReportFileType::TENANT_TMP_DATA; - for (int64_t i = 0; OB_SUCC(ret) && i < tenant_block_cnt_pairs.count(); ++i) { - report_key.tenant_id_ = tenant_block_cnt_pairs.at(i).first; - macro_block_cnt = tenant_block_cnt_pairs.at(i).second; - int64_t tenant_tmp_size = macro_block_cnt * common::OB_DEFAULT_MACRO_BLOCK_SIZE; - if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(tenant_tmp_size, tenant_tmp_size), 1))) { - STORAGE_LOG(WARN, "failed to set tenant tmp usage into result map", K(ret), K(report_key), K(macro_block_cnt)); + for (int64_t i = 0; OB_SUCC(ret) && i < tenant_ids.size(); i++) { + if (GCTX.omt_->is_available_tenant(tenant_ids.at(i))) { + MTL_SWITCH(tenant_ids.at(i)) { + ObTenantTmpFileManager* tmp_file_manager = MTL(ObTenantTmpFileManager*); + report_key.tenant_id_ = tenant_ids.at(i); + if (OB_ISNULL(tmp_file_manager)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected null", KR(ret)); + } else if (OB_FAIL(tmp_file_manager->get_macro_block_count(macro_block_cnt))) { + STORAGE_LOG(WARN, "fail to get_macro_block_count", KR(ret)); + } else { + int64_t tenant_tmp_size = macro_block_cnt * common::OB_DEFAULT_MACRO_BLOCK_SIZE; + if (OB_FAIL(result_map_.set_refactored(report_key, std::make_pair(tenant_tmp_size, tenant_tmp_size), 1))) { + STORAGE_LOG(WARN, "failed to set tenant tmp usage into result map", K(ret), K(report_key), K(macro_block_cnt)); + } + } + } // end MTL_SWITCH } - } + } // end for } return ret; } diff --git a/src/storage/ob_parallel_external_sort.h b/src/storage/ob_parallel_external_sort.h index 0779cad5ab..f89bcda05b 100644 --- a/src/storage/ob_parallel_external_sort.h +++ b/src/storage/ob_parallel_external_sort.h @@ -22,7 +22,7 @@ #include "share/io/ob_io_manager.h" #include "share/scheduler/ob_tenant_dag_scheduler.h" #include "blocksstable/ob_block_sstable_struct.h" -#include "blocksstable/ob_tmp_file.h" +#include "tmp_file/ob_tmp_file_manager.h" #include "share/config/ob_server_config.h" @@ -194,7 +194,7 @@ private: ObMacroBufferWriter macro_buffer_writer_; bool has_sample_item_; T sample_item_; - blocksstable::ObTmpFileIOHandle file_io_handle_; + tmp_file::ObTmpFileIOHandle file_io_handle_; int64_t fd_; int64_t dir_id_; uint64_t tenant_id_; @@ -308,16 +308,13 @@ int ObFragmentWriterV2::flush_buffer() STORAGE_LOG(WARN, "ObFragmentWriterV2 has not been inited", K(ret)); } else if (OB_FAIL(ObExternalSortConstant::get_io_timeout_ms(expire_timestamp_, timeout_ms))) { STORAGE_LOG(WARN, "fail to get io timeout ms", K(ret), K(expire_timestamp_)); - } else if (OB_FAIL(file_io_handle_.wait())) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret)); } else if (OB_FAIL(macro_buffer_writer_.serialize_header())) { STORAGE_LOG(WARN, "fail to serialize header", K(ret)); } else { - blocksstable::ObTmpFileIOInfo io_info; + tmp_file::ObTmpFileIOInfo io_info; io_info.fd_ = fd_; io_info.dir_id_ = dir_id_; io_info.size_ = buf_size_; - io_info.tenant_id_ = tenant_id_; io_info.buf_ = buf_; io_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_INDEX_BUILD_WRITE); io_info.io_timeout_ms_ = timeout_ms; @@ -343,16 +340,6 @@ int ObFragmentWriterV2::sync() STORAGE_LOG(WARN, "fail to flush buffer", K(ret)); } } - if (OB_SUCC(ret)) { - int64_t timeout_ms = 0; - if (OB_FAIL(file_io_handle_.wait())) { - STORAGE_LOG(WARN, "fail to wait io finish", K(ret)); - } else if (OB_FAIL(ObExternalSortConstant::get_io_timeout_ms(expire_timestamp_, timeout_ms))) { - STORAGE_LOG(WARN, "fail to get io timeout ms", K(ret), K(expire_timestamp_)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(fd_, timeout_ms))) { - STORAGE_LOG(WARN, "fail to sync macro file", K(ret)); - } - } } return ret; } @@ -472,9 +459,9 @@ private: int64_t fd_; int64_t dir_id_; T curr_item_; - blocksstable::ObTmpFileIOHandle file_io_handles_[MAX_HANDLE_COUNT]; + tmp_file::ObTmpFileIOHandle file_io_handles_[MAX_HANDLE_COUNT]; int64_t handle_cursor_; - char *buf_; + char *buf_[MAX_HANDLE_COUNT]; uint64_t tenant_id_; bool is_prefetch_end_; int64_t buf_size_; @@ -487,9 +474,12 @@ ObFragmentReaderV2::ObFragmentReaderV2() allocator_(common::ObNewModIds::OB_ASYNC_EXTERNAL_SORTER, common::OB_MALLOC_BIG_BLOCK_SIZE), sample_allocator_(common::ObNewModIds::OB_ASYNC_EXTERNAL_SORTER, OB_MALLOC_NORMAL_BLOCK_SIZE), macro_buffer_reader_(), fd_(-1), dir_id_(-1), curr_item_(), - file_io_handles_(), handle_cursor_(-1), buf_(NULL), tenant_id_(common::OB_INVALID_ID), + file_io_handles_(), handle_cursor_(-1), buf_(), tenant_id_(common::OB_INVALID_ID), is_prefetch_end_(false), buf_size_(0), is_first_prefetch_(true) { + for (int64_t i = 0; i < MAX_HANDLE_COUNT; ++i) { + buf_[i] = nullptr; + } } template @@ -558,23 +548,24 @@ int ObFragmentReaderV2::prefetch() ret = OB_NOT_INIT; STORAGE_LOG(WARN, "ObFragmentReaderV2 has not been inited", K(ret)); } else { - if (nullptr == buf_) { - if (OB_ISNULL(buf_ = static_cast(allocator_.alloc(buf_size_)))) { + int64_t handle_index = handle_cursor_ % MAX_HANDLE_COUNT; + if (nullptr == buf_[handle_index]) { + if (OB_ISNULL(buf_[handle_index] = static_cast(allocator_.alloc(buf_size_)))) { ret = common::OB_ALLOCATE_MEMORY_FAILED; STORAGE_LOG(WARN, "fail to allocate memory", K(ret)); } } if (OB_SUCC(ret)) { - blocksstable::ObTmpFileIOInfo io_info; + tmp_file::ObTmpFileIOInfo io_info; io_info.fd_ = fd_; io_info.dir_id_ = dir_id_; io_info.size_ = buf_size_; - io_info.tenant_id_ = tenant_id_; - io_info.buf_ = buf_; + io_info.buf_ = buf_[handle_index]; + io_info.disable_page_cache_ = true; io_info.io_desc_.set_wait_event(ObWaitEventIds::DB_FILE_INDEX_BUILD_READ); if (OB_FAIL(ObExternalSortConstant::get_io_timeout_ms(expire_timestamp_, io_info.io_timeout_ms_))) { STORAGE_LOG(WARN, "fail to get io timeout ms", K(ret), K(expire_timestamp_), K(io_info.io_timeout_ms_)); - } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.aio_read(io_info, file_io_handles_[handle_cursor_ % MAX_HANDLE_COUNT]))) { + } else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.aio_read(io_info, file_io_handles_[handle_index]))) { if (common::OB_ITER_END != ret) { STORAGE_LOG(WARN, "fail to do aio read from macro file", K(ret), K(fd_), K(io_info)); } else { @@ -676,7 +667,9 @@ void ObFragmentReaderV2::reset() file_io_handles_[i].reset(); } handle_cursor_ = 0; - buf_ = NULL; + for (int64_t i = 0; i < MAX_HANDLE_COUNT; ++i) { + buf_[i] = nullptr; + } tenant_id_ = common::OB_INVALID_ID; is_prefetch_end_ = false; buf_size_ = 0; @@ -688,6 +681,9 @@ int ObFragmentReaderV2::clean_up() { int ret = common::OB_SUCCESS; if (is_inited_) { + for (int64_t i = 0; i < MAX_HANDLE_COUNT; ++i) { + file_io_handles_[i].reset(); + } if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.remove(fd_))) { STORAGE_LOG(WARN, "fail to remove macro file", K(ret)); } diff --git a/src/storage/tmp_file/ob_shared_nothing_tmp_file.cpp b/src/storage/tmp_file/ob_shared_nothing_tmp_file.cpp new file mode 100644 index 0000000000..b0a050810b --- /dev/null +++ b/src/storage/tmp_file/ob_shared_nothing_tmp_file.cpp @@ -0,0 +1,2717 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_eviction_manager.h" +#include "storage/tmp_file/ob_tmp_file_flush_priority_manager.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "storage/tmp_file/ob_tmp_file_flush_manager.h" +#include "storage/tmp_file/ob_tmp_file_io_ctx.h" +#include "storage/blocksstable/ob_block_manager.h" +#include "lib/hash/ob_hashmap.h" +#include "lib/random/ob_random.h" +#include "storage/tmp_file/ob_tmp_file_page_cache_controller.h" + +namespace oceanbase +{ +namespace tmp_file +{ +ObTmpFileHandle::ObTmpFileHandle(ObSharedNothingTmpFile *tmp_file) + : ptr_(tmp_file) +{ + if (ptr_ != nullptr) { + ptr_->inc_ref_cnt(); + } +} + +ObTmpFileHandle::ObTmpFileHandle(const ObTmpFileHandle &handle) + : ptr_(nullptr) +{ + operator=(handle); +} + +ObTmpFileHandle & ObTmpFileHandle::operator=(const ObTmpFileHandle &other) +{ + if (other.get() != ptr_) { + reset(); + ptr_ = other.get(); + if (ptr_ != nullptr) { + ptr_->inc_ref_cnt(); + } + } + return *this; +} + +void ObTmpFileHandle::reset() +{ + if (ptr_ != nullptr) { + ptr_->dec_ref_cnt(); + if (ptr_->get_ref_cnt() == 0) { + ptr_->~ObSharedNothingTmpFile(); + } + ptr_ = nullptr; + } +} + +int ObTmpFileHandle::init(ObSharedNothingTmpFile *tmp_file) +{ + int ret = OB_SUCCESS; + + if (is_inited()) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret), KP(ptr_)); + } else if (OB_ISNULL(tmp_file)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(tmp_file)); + } else { + ptr_ = tmp_file; + ptr_->inc_ref_cnt(); + } + + return ret; +} + +void ObSharedNothingTmpFile::InnerFlushContext::reset() +{ + meta_finished_continuous_flush_info_num_ = 0; + data_finished_continuous_flush_info_num_ = 0; + meta_flush_infos_.reset(); + data_flush_infos_.reset(); + flush_seq_ = ObTmpFileGlobal::INVALID_FLUSH_SEQUENCE; +} + +int ObSharedNothingTmpFile::InnerFlushContext::update_finished_continuous_flush_info_num(const bool is_meta, const int64_t end_pos) +{ + int ret = OB_SUCCESS; + if (is_meta) { + if (OB_UNLIKELY(end_pos < meta_finished_continuous_flush_info_num_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected flush info num", K(ret), K(end_pos), K(meta_finished_continuous_flush_info_num_)); + } else { + meta_finished_continuous_flush_info_num_ = end_pos; + } + } else { + if (OB_UNLIKELY(end_pos < data_finished_continuous_flush_info_num_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected flush info num", K(ret), K(end_pos), K(data_finished_continuous_flush_info_num_)); + } else { + data_finished_continuous_flush_info_num_ = end_pos; + } + } + return ret; +} + +ObSharedNothingTmpFile::InnerFlushInfo::InnerFlushInfo() + : update_meta_data_done_(false), + flush_data_page_disk_begin_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flush_data_page_num_(-1), + flush_virtual_page_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID), + file_size_(0), + flush_meta_page_array_() +{ + flush_meta_page_array_.set_attr(ObMemAttr(MTL_ID(), "TFFlushMetaArr")); +} + +void ObSharedNothingTmpFile::InnerFlushInfo::reset() +{ + update_meta_data_done_ = false; + flush_data_page_disk_begin_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flush_data_page_num_ = -1; + flush_virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + file_size_ = 0; + flush_meta_page_array_.reset(); +} + +int ObSharedNothingTmpFile::InnerFlushInfo::init_by_tmp_file_flush_info(const ObTmpFileFlushInfo& flush_info) +{ + int ret = OB_SUCCESS; + flush_data_page_disk_begin_id_ = flush_info.flush_data_page_disk_begin_id_; + flush_data_page_num_ = flush_info.flush_data_page_num_; + flush_virtual_page_id_ = flush_info.flush_virtual_page_id_; + file_size_ = flush_info.file_size_; + if (!flush_info.flush_meta_page_array_.empty() + && OB_FAIL(flush_meta_page_array_.assign(flush_info.flush_meta_page_array_))) { + LOG_WARN("fail to assign flush_meta_page_array_", KR(ret), K(flush_info)); + } + return ret; +} + +ObSharedNothingTmpFile::ObSharedNothingTmpFile() + : tmp_file_block_manager_(nullptr), + callback_allocator_(nullptr), + page_cache_controller_(nullptr), + flush_prio_mgr_(nullptr), + eviction_mgr_(nullptr), + wbp_(nullptr), + page_idx_cache_(), + is_inited_(false), + is_deleting_(false), + is_in_data_eviction_list_(false), + is_in_meta_eviction_list_(false), + data_page_flush_level_(-1), + meta_page_flush_level_(-1), + tenant_id_(OB_INVALID_TENANT_ID), + dir_id_(ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID), + fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), + ref_cnt_(0), + truncated_offset_(0), + read_offset_(0), + file_size_(0), + flushed_data_page_num_(0), + write_back_data_page_num_(0), + cached_page_nums_(0), + begin_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + begin_page_virtual_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID), + flushed_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flushed_page_virtual_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID), + end_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + meta_tree_(), + data_flush_node_(*this), + meta_flush_node_(*this), + data_eviction_node_(*this), + meta_eviction_node_(*this), + meta_lock_(common::ObLatchIds::TMP_FILE_LOCK), + last_page_lock_(common::ObLatchIds::TMP_FILE_LOCK), + multi_write_lock_(common::ObLatchIds::TMP_FILE_LOCK), + truncate_lock_(common::ObLatchIds::TMP_FILE_LOCK), + inner_flush_ctx_() +{ +} + +ObSharedNothingTmpFile::~ObSharedNothingTmpFile() +{ + destroy(); +} + +int ObSharedNothingTmpFile::init(const uint64_t tenant_id, const int64_t fd, const int64_t dir_id, + ObTmpFileBlockManager *block_manager, + ObIAllocator *callback_allocator, + ObIAllocator *wbp_index_cache_allocator, + ObIAllocator *wbp_index_cache_bkt_allocator, + ObTmpFilePageCacheController *pc_ctrl) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret), KPC(this)); + } else if (!is_valid_tenant_id(tenant_id) || ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID == dir_id || + OB_ISNULL(block_manager) || OB_ISNULL(callback_allocator) || + OB_ISNULL(wbp_index_cache_allocator) || OB_ISNULL(wbp_index_cache_bkt_allocator) || + OB_ISNULL(pc_ctrl)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(fd), K(dir_id), + KP(block_manager), KP(callback_allocator), KP(pc_ctrl), + KP(wbp_index_cache_allocator), KP(wbp_index_cache_bkt_allocator)); + } else if (OB_FAIL(page_idx_cache_.init(fd, &pc_ctrl->get_write_buffer_pool(), + wbp_index_cache_allocator, wbp_index_cache_bkt_allocator))) { + LOG_WARN("fail to init page idx array", KR(ret), K(fd)); + } else if (OB_FAIL(meta_tree_.init(fd, &pc_ctrl->get_write_buffer_pool(), callback_allocator))) { + LOG_WARN("fail to init meta tree", KR(ret), K(fd)); + } else { + is_inited_ = true; + tmp_file_block_manager_ = block_manager; + callback_allocator_ = callback_allocator; + page_cache_controller_ = pc_ctrl; + eviction_mgr_ = &pc_ctrl->get_eviction_manager(); + flush_prio_mgr_ = &pc_ctrl->get_flush_priority_mgr(); + wbp_ = &pc_ctrl->get_write_buffer_pool(); + tenant_id_ = tenant_id; + dir_id_ = dir_id; + fd_ = fd; + } + + LOG_INFO("tmp file init over", KR(ret), K(fd), K(dir_id)); + return ret; +} + +int ObSharedNothingTmpFile::destroy() +{ + int ret = OB_SUCCESS; + int64_t fd_backup = fd_; + + LOG_INFO("tmp file destroy start", KR(ret), K(fd_), KPC(this)); + + if (cached_page_nums_ > 0) { + uint32_t cur_page_id = begin_page_id_; + while (OB_SUCC(ret) && cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("begin page virtual id is invalid", KR(ret), K(fd_), K(begin_page_virtual_id_)); + } else if (OB_FAIL(wbp_->free_page(fd_, cur_page_id, ObTmpFilePageUniqKey(begin_page_virtual_id_), next_page_id))) { + LOG_WARN("fail to free page", KR(ret), K(fd_), K(cur_page_id), K(begin_page_virtual_id_)); + } else { + cur_page_id = next_page_id; + begin_page_virtual_id_ += 1; + } + } + } + + LOG_INFO("tmp file destroy, free wbp page phase over", KR(ret), K(fd_), KPC(this)); + + if (FAILEDx(meta_tree_.clear(truncated_offset_, file_size_))) { + LOG_WARN("fail to clear", KR(ret), K(fd_), K(truncated_offset_), K(file_size_)); + } + + LOG_INFO("tmp file destroy, meta_tree_ clear phase over", KR(ret), K(fd_), KPC(this)); + + if (OB_SUCC(ret)) { + reset(); + } + + LOG_INFO("tmp file destroy over", KR(ret), "fd", fd_backup); + return ret; +} + +void ObSharedNothingTmpFile::reset() +{ + tmp_file_block_manager_ = nullptr; + callback_allocator_ = nullptr; + page_cache_controller_ = nullptr; + flush_prio_mgr_ = nullptr; + eviction_mgr_ = nullptr; + wbp_ = nullptr; + page_idx_cache_.destroy(); + is_inited_ = false; + is_deleting_ = false; + is_in_data_eviction_list_ = false; + is_in_meta_eviction_list_ = false; + data_page_flush_level_ = -1; + meta_page_flush_level_ = -1; + tenant_id_ = OB_INVALID_TENANT_ID; + dir_id_ = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + ref_cnt_ = 0; + truncated_offset_ = 0; + read_offset_ = 0; + file_size_ = 0; + flushed_data_page_num_ = 0; + write_back_data_page_num_ = 0; + cached_page_nums_ = 0; + begin_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + begin_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + flushed_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flushed_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + meta_tree_.reset(); + data_flush_node_.unlink(); + meta_flush_node_.unlink(); + data_eviction_node_.unlink(); + meta_eviction_node_.unlink(); + inner_flush_ctx_.reset(); +} + +bool ObSharedNothingTmpFile::is_deleting() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return is_deleting_; +} + +bool ObSharedNothingTmpFile::can_remove() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return is_deleting_ && get_ref_cnt() == 1; +} + +int ObSharedNothingTmpFile::delete_file() +{ + int ret = OB_SUCCESS; + common::TCRWLock::WLockGuard guard(meta_lock_); + if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("the file is deleting", KR(ret), KPC(this)); + } else if (OB_FAIL(eviction_mgr_->remove_file(*this))) { + LOG_WARN("fail to remove file from eviction manager", KR(ret), KPC(this)); + } else if (OB_FAIL(flush_prio_mgr_->remove_file(*this))) { + LOG_WARN("fail to remove file from flush priority manager", KR(ret),KPC(this)); + } else { + // read, write, truncate, flush and evict function will fail when is_deleting_ == true. + is_deleting_ = true; + } + + LOG_INFO("tmp file delete", KR(ret), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::aio_pread(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObSharedNothingTmpFile has not been inited", KR(ret), K(tenant_id_), KPC(this)); + } else { + common::TCRWLock::RLockGuard guard(meta_lock_); + if (io_ctx.get_read_offset_in_file() < 0) { + io_ctx.set_read_offset_in_file(read_offset_); + } + + LOG_DEBUG("start to inner read tmp file", K(fd_), K(io_ctx.get_read_offset_in_file()), + K(io_ctx.get_todo_size()), K(io_ctx.get_done_size()), KPC(this)); + if (OB_FAIL(ret)) { + } else if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to read a deleting file", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(io_ctx.get_read_offset_in_file() >= file_size_)) { + ret = OB_ITER_END; + LOG_WARN("iter end", KR(ret), K(fd_), K(file_size_), K(io_ctx)); + } else if (io_ctx.get_read_offset_in_file() < truncated_offset_ && + OB_FAIL(inner_read_truncated_part_(io_ctx))) { + LOG_WARN("fail to read truncated part", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(io_ctx.get_todo_size() == 0)) { + // do nothing + } else { + // Iterate to read disk data. + int64_t wbp_begin_offset = cal_wbp_begin_offset_(); + if (OB_UNLIKELY(wbp_begin_offset < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected wbp begin offset", KR(ret), K(fd_), K(wbp_begin_offset), K(io_ctx)); + } else if (io_ctx.get_read_offset_in_file() < wbp_begin_offset) { + const int64_t expected_read_disk_size = MIN(io_ctx.get_todo_size(), + wbp_begin_offset - io_ctx.get_read_offset_in_file()); + + if (OB_UNLIKELY(expected_read_disk_size < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected read disk size", KR(ret), K(fd_), K(expected_read_disk_size), K(wbp_begin_offset), K(io_ctx)); + } else if (expected_read_disk_size == 0) { + // do nothing + } else if (OB_FAIL(inner_read_from_disk_(expected_read_disk_size, io_ctx))) { + LOG_WARN("fail to read tmp file from disk", KR(ret), K(fd_), K(expected_read_disk_size), + K(wbp_begin_offset), K(io_ctx)); + } else { + LOG_DEBUG("finish disk read", K(fd_), K(io_ctx.get_read_offset_in_file()), + K(io_ctx.get_todo_size()), + K(io_ctx.get_done_size()), + K(wbp_begin_offset), K(expected_read_disk_size)); + } + } + + // Iterate to read memory data (in write buffer pool). + if (OB_SUCC(ret) && io_ctx.get_todo_size() > 0) { + if (OB_UNLIKELY(0 == cached_page_nums_)) { + ret = OB_ITER_END; + LOG_WARN("iter end", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_FAIL(inner_read_from_wbp_(io_ctx))) { + LOG_WARN("fail to read tmp file from wbp", KR(ret), K(fd_), K(io_ctx)); + } else { + LOG_DEBUG("finish wbp read", K(fd_), K(io_ctx.get_read_offset_in_file()), + K(io_ctx.get_todo_size()), + K(io_ctx.get_done_size())); + } + } + } + + LOG_DEBUG("inner read finish once", KR(ret), K(fd_), + K(io_ctx.get_read_offset_in_file()), + K(io_ctx.get_todo_size()), + K(io_ctx.get_done_size()), KPC(this)); + } + return ret; +} + +int ObSharedNothingTmpFile::inner_read_truncated_part_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(io_ctx.get_read_offset_in_file() >= truncated_offset_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("read offset should be less than truncated offset", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid io_ctx", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(io_ctx.get_todo_size() == 0)) { + // do nothing + } else { + int64_t read_size = MIN(truncated_offset_ - io_ctx.get_read_offset_in_file(), + io_ctx.get_todo_size()); + char *read_buf = io_ctx.get_todo_buffer(); + if (OB_UNLIKELY(!io_ctx.check_buf_range_valid(read_buf, read_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf range", KR(ret), K(fd_), K(read_buf), K(read_size), K(io_ctx)); + } else if (FALSE_IT(MEMSET(read_buf, 0, read_size))) { + } else if (OB_FAIL(io_ctx.update_data_size(read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(read_size)); + } else if (OB_UNLIKELY(io_ctx.get_todo_size() > 0 && + truncated_offset_ == file_size_)) { + ret = OB_ITER_END; + LOG_WARN("iter end", KR(ret), K(fd_), K(file_size_), K(truncated_offset_), K(io_ctx)); + } + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_read_from_disk_(const int64_t expected_read_disk_size, + ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + common::ObArray data_items; + if (OB_FAIL(meta_tree_.search_data_items(io_ctx.get_read_offset_in_file(), + expected_read_disk_size, data_items))) { + LOG_WARN("fail to search data items", KR(ret), K(fd_), K(expected_read_disk_size), K(io_ctx)); + } else if (OB_UNLIKELY(data_items.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("no data item found", KR(ret), K(fd_), K(expected_read_disk_size), K(io_ctx)); + } + + // Iterate to read each block. + int64_t remain_read_size = expected_read_disk_size; + int64_t actual_read_size = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < data_items.count() && 0 < remain_read_size; i++) { + const int64_t block_index = data_items[i].block_index_; + const int64_t read_start_virtual_page_id = get_page_virtual_id_from_offset_(io_ctx.get_read_offset_in_file(), + false /*is_open_interval*/); + const int64_t start_page_id_in_data_item = read_start_virtual_page_id - data_items[i].virtual_page_id_; + const int64_t begin_offset_in_block = + (data_items[i].physical_page_id_ + start_page_id_in_data_item) * ObTmpFileGlobal::PAGE_SIZE; + const int64_t end_offset_in_block = + (data_items[i].physical_page_id_ + data_items[i].physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + const int64_t begin_read_offset_in_block = begin_offset_in_block + + (0 == i? + get_page_offset_from_file_or_block_offset_(io_ctx.get_read_offset_in_file()) : 0); + const int64_t end_read_offset_in_block = (data_items.count() - 1 == i? + begin_read_offset_in_block + remain_read_size : + end_offset_in_block); + int64_t actual_block_read_size = 0; + + ObTmpBlockValueHandle block_value_handle; + if (OB_SUCC(ObTmpBlockCache::get_instance().get_block(ObTmpBlockCacheKey(block_index, MTL_ID()), + block_value_handle))) { + LOG_DEBUG("hit block cache", K(block_index), K(fd_), K(io_ctx)); + char *read_buf = io_ctx.get_todo_buffer(); + const int64_t read_size = end_read_offset_in_block - begin_read_offset_in_block; + ObTmpFileIOCtx::ObBlockCacheHandle block_cache_handle(block_value_handle, read_buf, + begin_read_offset_in_block, + read_size); + if (OB_FAIL(io_ctx.get_block_cache_handles().push_back(block_cache_handle))) { + LOG_WARN("Fail to push back into block_handles", KR(ret), K(fd_)); + } else if (OB_FAIL(io_ctx.update_data_size(read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(read_size)); + } else { + remain_read_size -= read_size; + actual_read_size += read_size; + LOG_DEBUG("succ to read data from cached block", + KR(ret), K(fd_), K(block_index), K(begin_offset_in_block), K(end_offset_in_block), + K(begin_read_offset_in_block), K(end_read_offset_in_block), + K(remain_read_size), K(read_size), K(actual_read_size), + K(data_items[i]), K(io_ctx)); + } + } else if (OB_ENTRY_NOT_EXIST != ret) { + LOG_WARN("fail to get block", KR(ret), K(fd_), K(block_index)); + } else { // not hit block cache, read page from disk. + ret = OB_SUCCESS; + if (io_ctx.is_disable_page_cache()) { + if (OB_FAIL(inner_seq_read_from_block_(block_index, + begin_read_offset_in_block, + end_read_offset_in_block, + io_ctx, actual_block_read_size))) { + LOG_WARN("fail to seq read from block", + KR(ret), K(fd_), K(block_index), K(begin_offset_in_block), K(end_offset_in_block), + K(begin_read_offset_in_block), K(end_read_offset_in_block), + K(remain_read_size), K(actual_block_read_size), K(actual_read_size), + K(data_items[i]), K(io_ctx)); + } else { + remain_read_size -= actual_block_read_size; + actual_read_size += actual_block_read_size; + LOG_DEBUG("succ to seq read from block", + KR(ret), K(fd_), K(block_index), K(begin_offset_in_block), K(end_offset_in_block), + K(begin_read_offset_in_block), K(end_read_offset_in_block), + K(remain_read_size), K(actual_block_read_size), K(actual_read_size), + K(data_items[i]), K(io_ctx)); + } + } else { + if (OB_FAIL(inner_rand_read_from_block_(block_index, + begin_read_offset_in_block, + end_read_offset_in_block, + io_ctx, actual_block_read_size))) { + LOG_WARN("fail to rand read from block", + KR(ret), K(fd_), K(block_index), K(begin_offset_in_block), K(end_offset_in_block), + K(begin_read_offset_in_block), K(end_read_offset_in_block), + K(remain_read_size), K(actual_block_read_size), K(actual_read_size), + K(data_items[i]), K(io_ctx)); + } else { + remain_read_size -= actual_block_read_size; + actual_read_size += actual_block_read_size; + LOG_DEBUG("succ to rand read from block", + KR(ret), K(fd_), K(block_index), K(begin_offset_in_block), K(end_offset_in_block), + K(begin_read_offset_in_block), K(end_read_offset_in_block), + K(remain_read_size), K(actual_block_read_size), K(actual_read_size), + K(data_items[i]), K(io_ctx)); + } + } + } + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_seq_read_from_block_(const int64_t block_index, + const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx, + int64_t &actual_read_size) +{ + int ret = OB_SUCCESS; + const int64_t expected_read_size = end_read_offset_in_block - begin_read_offset_in_block; + ObTmpFileBlockHandle block_handle; + actual_read_size = 0; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX == block_index || + expected_read_size <= 0 || expected_read_size > OB_DEFAULT_MACRO_BLOCK_SIZE)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(block_index), + K(begin_read_offset_in_block), + K(end_read_offset_in_block)); + } else if (OB_FAIL(tmp_file_block_manager_->get_tmp_file_block_handle(block_index, block_handle))) { + LOG_WARN("fail to get tmp file block_handle", KR(ret), K(fd_), K(block_index)); + } else if (OB_ISNULL(block_handle.get()) || OB_UNLIKELY(!block_handle.get()->get_macro_block_id().is_valid())) { + LOG_WARN("fail to get tmp file block_handle", KR(ret), K(fd_), K(block_handle)); + } else { + char *read_buf = io_ctx.get_todo_buffer(); + ObTmpFileIOCtx::ObIOReadHandle io_read_handle(read_buf, + 0 /*offset_in_src_data_buf_*/, + expected_read_size, block_handle); + if (OB_FAIL(io_ctx.get_io_handles().push_back(io_read_handle))) { + LOG_WARN("Fail to push back into io_handles", KR(ret), K(fd_)); + } else if (OB_FAIL(ObTmpPageCache::get_instance().direct_read( + block_handle.get()->get_macro_block_id(), expected_read_size, begin_read_offset_in_block, + io_ctx.get_io_flag(), io_ctx.get_io_timeout_ms(), *callback_allocator_, + io_ctx.get_io_handles().at(io_ctx.get_io_handles().count()-1).handle_))) { + LOG_WARN("fail to cached_read", KR(ret), K(fd_), K(block_handle), K(expected_read_size), + K(begin_read_offset_in_block), K(io_ctx)); + } + } + + // Update read offset and read size. + if (OB_FAIL(ret)) { + } else if (OB_FAIL(io_ctx.update_data_size(expected_read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(expected_read_size)); + } else { + actual_read_size = expected_read_size; + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_rand_read_from_block_(const int64_t block_index, + const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx, + int64_t &actual_read_size) +{ + int ret = OB_SUCCESS; + const int64_t begin_page_idx_in_block = get_page_id_in_block_(begin_read_offset_in_block); + const int64_t end_page_idx_in_block = get_page_id_in_block_(end_read_offset_in_block - 1); // -1 for changing open interval to close interval + ObTmpFileBlockPageBitmap bitmap; + ObTmpFileBlockPageBitmapIterator iterator; + ObArray page_value_handles; + + if (OB_FAIL(collect_pages_in_block_(block_index, begin_page_idx_in_block, end_page_idx_in_block, + bitmap, page_value_handles))) { + LOG_WARN("fail to collect pages in block", KR(ret), K(fd_), K(block_index), + K(begin_page_idx_in_block), + K(end_page_idx_in_block)); + } else if (OB_FAIL(iterator.init(&bitmap, begin_page_idx_in_block, end_page_idx_in_block))) { + LOG_WARN("fail to init iterator", KR(ret), K(fd_), K(block_index), + K(begin_page_idx_in_block), K(end_page_idx_in_block)); + } else { + int64_t has_read_cached_page_num = 0; + while (OB_SUCC(ret) && iterator.has_next()) { + bool is_in_cache = false; + int64_t begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t begin_read_offset = -1; + int64_t end_read_offset = -1; + if (OB_FAIL(iterator.next_range(is_in_cache, begin_page_id, end_page_id))) { + LOG_WARN("fail to next range", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(begin_page_id > end_page_id || begin_page_id < 0 || end_page_id < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid range", KR(ret), K(fd_), K(begin_page_id), K(end_page_id)); + } else { + begin_read_offset = begin_page_id == begin_page_idx_in_block ? + begin_read_offset_in_block : + get_page_begin_offset_by_virtual_id_(begin_page_id); + end_read_offset = end_page_id == end_page_idx_in_block ? + end_read_offset_in_block : + get_page_end_offset_by_virtual_id_(end_page_id); + } + + if (OB_FAIL(ret)) { + } else if (is_in_cache) { + if (OB_FAIL(inner_read_continuous_cached_pages_(begin_read_offset, end_read_offset, + page_value_handles, has_read_cached_page_num, + io_ctx))) { + LOG_WARN("fail to inner read continuous cached pages", KR(ret), K(fd_), K(begin_read_offset), + K(end_read_offset), K(io_ctx)); + } else { + has_read_cached_page_num += (end_page_id - begin_page_id + 1); + } + } else { + if (OB_FAIL(inner_read_continuous_uncached_pages_(block_index, begin_read_offset, + end_read_offset, io_ctx))) { + LOG_WARN("fail to inner read continuous uncached pages", KR(ret), K(fd_), K(block_index), + K(begin_read_offset), + K(end_read_offset), + K(io_ctx)); + } + } + + if (OB_SUCC(ret)) { + const int64_t read_size = end_read_offset - begin_read_offset; + actual_read_size += read_size; + } + } + } + + return ret; +} + +int ObSharedNothingTmpFile::collect_pages_in_block_(const int64_t block_index, + const int64_t begin_page_idx_in_block, + const int64_t end_page_idx_in_block, + ObTmpFileBlockPageBitmap &bitmap, + ObIArray &page_value_handles) +{ + int ret = OB_SUCCESS; + bitmap.reset(); + page_value_handles.reset(); + + for (int64_t page_idx_in_block = begin_page_idx_in_block; + OB_SUCC(ret) && page_idx_in_block <= end_page_idx_in_block; + page_idx_in_block++) { + ObTmpPageCacheKey key(block_index, page_idx_in_block, tenant_id_); + ObTmpPageValueHandle p_handle; + if (OB_SUCC(ObTmpPageCache::get_instance().get_page(key, p_handle))) { + if (OB_FAIL(page_value_handles.push_back(p_handle))) { + LOG_WARN("fail to push back", KR(ret), K(fd_), K(key)); + } else if (OB_FAIL(bitmap.set_bitmap(page_idx_in_block, true))) { + LOG_WARN("fail to set bitmap", KR(ret), K(fd_), K(key)); + } + } else if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + if (OB_FAIL(bitmap.set_bitmap(page_idx_in_block, false))) { + LOG_WARN("fail to set bitmap", KR(ret), K(fd_), K(key)); + } + } else { + LOG_WARN("fail to get page from cache", KR(ret), K(fd_), K(key)); + } + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_read_continuous_uncached_pages_(const int64_t block_index, + const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + ObArray page_keys; + const int64_t begin_page_idx = get_page_id_in_block_(begin_read_offset_in_block); + const int64_t end_page_idx = get_page_id_in_block_(end_read_offset_in_block - 1); // -1 to change open interval to close interval + const int64_t block_read_begin_offset = get_page_begin_offset_by_file_or_block_offset_(begin_read_offset_in_block); + const int64_t block_read_end_offset = get_page_end_offset_by_file_or_block_offset_(end_read_offset_in_block); + const int64_t block_read_size = block_read_end_offset - block_read_begin_offset; // read and cached completed pages from disk + const int64_t usr_read_begin_offset = begin_read_offset_in_block; + const int64_t usr_read_end_offset = end_read_offset_in_block; + // from loaded disk block buf, from "offset_in_block_buf" read "usr_read_size" size data to user's read buf + const int64_t offset_in_block_buf = usr_read_begin_offset - block_read_begin_offset; + const int64_t usr_read_size = block_read_size - + (usr_read_begin_offset - block_read_begin_offset) - + (block_read_end_offset - usr_read_end_offset); + + for (int64_t page_id = begin_page_idx; OB_SUCC(ret) && page_id <= end_page_idx; page_id++) { + ObTmpPageCacheKey key(block_index, page_id, tenant_id_); + if (OB_FAIL(page_keys.push_back(key))) { + LOG_WARN("fail to push back", KR(ret), K(fd_), K(key)); + } + } + + ObTmpFileBlockHandle block_handle; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(tmp_file_block_manager_->get_tmp_file_block_handle(block_index, block_handle))) { + LOG_WARN("fail to get tmp file block_handle", KR(ret), K(fd_), K(block_index)); + } else { + char *user_read_buf = io_ctx.get_todo_buffer(); + ObTmpFileIOCtx::ObIOReadHandle io_read_handle(user_read_buf, + offset_in_block_buf, + usr_read_size, block_handle); + if (OB_FAIL(io_ctx.get_io_handles().push_back(io_read_handle))) { + LOG_WARN("Fail to push back into io_handles", KR(ret), K(fd_)); + } else if (OB_FAIL(ObTmpPageCache::get_instance().cached_read(page_keys, + block_handle.get()->get_macro_block_id(), block_read_begin_offset, + io_ctx.get_io_flag(), io_ctx.get_io_timeout_ms(), *callback_allocator_, + io_ctx.get_io_handles().at(io_ctx.get_io_handles().count()-1).handle_))) { + LOG_WARN("fail to cached_read", KR(ret), K(fd_), K(block_handle), + K(block_read_begin_offset), K(io_ctx)); + } else if (OB_FAIL(io_ctx.update_data_size(usr_read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(usr_read_size)); + } + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_read_continuous_cached_pages_(const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + const ObArray &page_value_handles, + const int64_t start_array_idx, + ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + const int64_t begin_page_idx = get_page_id_in_block_(begin_read_offset_in_block); + const int64_t end_page_idx = get_page_id_in_block_(end_read_offset_in_block - 1); // -1 for changing open interval to close interval + const int64_t iter_array_cnt = end_page_idx - begin_page_idx + 1; + + if (OB_UNLIKELY(start_array_idx < 0 || start_array_idx + iter_array_cnt > page_value_handles.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid array idx", KR(ret), K(fd_), K(start_array_idx), K(iter_array_cnt), + K(begin_page_idx), K(end_page_idx), K(page_value_handles.count())); + } + + int64_t read_offset = begin_read_offset_in_block; + int64_t cur_array_idx = start_array_idx; + for (int64_t cached_page_idx = begin_page_idx; OB_SUCC(ret) && cached_page_idx <= end_page_idx; cached_page_idx++) { + ObTmpPageValueHandle p_handle = page_value_handles.at(cur_array_idx++); + char *read_buf = io_ctx.get_todo_buffer(); + const int64_t cur_page_end_offset = get_page_end_offset_by_virtual_id_(cached_page_idx); + const int64_t read_size = MIN(cur_page_end_offset, end_read_offset_in_block) - read_offset; + const int64_t read_offset_in_page = get_page_offset_from_file_or_block_offset_(read_offset); + ObTmpFileIOCtx::ObPageCacheHandle page_handle(p_handle, read_buf, + read_offset_in_page, + read_size); + if (OB_FAIL(io_ctx.get_page_cache_handles().push_back(page_handle))) { + LOG_WARN("Fail to push back into page_handles", KR(ret), K(fd_)); + } else if (OB_FAIL(io_ctx.update_data_size(read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(read_size)); + } else { + read_offset += read_size; + } + } // end for + + return ret; +} + +int ObSharedNothingTmpFile::inner_read_from_wbp_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + const int64_t begin_read_page_virtual_id = get_page_virtual_id_from_offset_(io_ctx.get_read_offset_in_file(), + false /*is_open_interval*/); + uint32_t begin_read_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t wbp_begin_offset = cal_wbp_begin_offset_(); + + if (OB_UNLIKELY(wbp_begin_offset < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected wbp begin offset", KR(ret), K(fd_), K(wbp_begin_offset), K(io_ctx)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_read_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected begin read page virtual id", KR(ret), K(fd_), K(begin_read_page_virtual_id), K(io_ctx)); + } else if (io_ctx.get_read_offset_in_file() < wbp_begin_offset + ObTmpFileGlobal::PAGE_SIZE) { + begin_read_page_id = begin_page_id_; + } else if (OB_FAIL(page_idx_cache_.binary_search(begin_read_page_virtual_id, begin_read_page_id))) { + LOG_WARN("fail to find page index in array", KR(ret), K(fd_), K(io_ctx)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_read_page_id && + OB_FAIL(wbp_->get_page_id_by_virtual_id(fd_, begin_read_page_virtual_id, begin_page_id_, begin_read_page_id))) { + LOG_WARN("fail to get page id by virtual id", KR(ret), K(fd_), K(begin_read_page_virtual_id), K(begin_page_id_)); + } else if (OB_UNLIKELY(begin_read_page_id == ObTmpFileGlobal::INVALID_PAGE_ID)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid page index", KR(ret), K(fd_), K(begin_read_page_id)); + } + + uint32_t curr_page_id = begin_read_page_id; + int64_t curr_page_virtual_id = begin_read_page_virtual_id; + while (OB_SUCC(ret) && io_ctx.get_todo_size() > 0) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *data_page = nullptr; + if (OB_FAIL(wbp_->read_page(fd_, curr_page_id, ObTmpFilePageUniqKey(curr_page_virtual_id), data_page, next_page_id))) { + LOG_WARN("fail to fetch page", KR(ret), K(fd_), K(curr_page_id), K(curr_page_virtual_id), K(io_ctx)); + } else if (OB_ISNULL(data_page)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("data page is null", KR(ret), K(fd_), K(curr_page_id)); + } else { + const int64_t read_offset_in_page = get_page_offset_from_file_or_block_offset_(io_ctx.get_read_offset_in_file()); + const int64_t read_size = MIN3(ObTmpFileGlobal::PAGE_SIZE - read_offset_in_page, + io_ctx.get_todo_size(), + file_size_ - io_ctx.get_read_offset_in_file()); + char *read_buf = io_ctx.get_todo_buffer(); + if (OB_UNLIKELY(!io_ctx.check_buf_range_valid(read_buf, read_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf range", KR(ret), K(fd_), K(read_buf), K(read_size), K(io_ctx)); + } else if (OB_FAIL(io_ctx.update_data_size(read_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(read_size)); + } else { + MEMCPY(read_buf, data_page + read_offset_in_page, read_size); + curr_page_id = next_page_id; + curr_page_virtual_id += 1; + } + } + + if (OB_SUCC(ret) && io_ctx.get_todo_size() > 0 && io_ctx.get_read_offset_in_file() == file_size_) { + ret = OB_ITER_END; + LOG_WARN("iter end", KR(ret), K(fd_), K(io_ctx)); + } + } + + return ret; +} + +// attention: +// in order to avoid blocking reading, flushing and evicting pages, +// write operation only adds write lock for meta_lock when it try to update meta data. +// due to the write operation is appending write, if the meta data doesn't been modified, +// all operation could not write or read the new appending data. +int ObSharedNothingTmpFile::aio_write(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + LOG_DEBUG("aio write start", K(fd_), K(file_size_), K(io_ctx)); + + ObSpinLockGuard guard(multi_write_lock_); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObSharedNothingTmpFile has not been inited", KR(ret), K(tenant_id_), KPC(this)); + } else if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(is_deleting_)) { + // this check is just a hint. + // although is_deleting_ == false, it might be set as true in the processing of inner_write(). + // we will check is_deleting_ again when try to update meta data in the inner_write() + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to write a deleting file", KR(ret), K(fd_)); + } else { + while (OB_SUCC(ret) && io_ctx.get_todo_size() > 0) { + if (OB_FAIL(inner_write_(io_ctx))) { + if (OB_ALLOCATE_TMP_FILE_PAGE_FAILED == ret) { + ret = OB_SUCCESS; + if (TC_REACH_COUNT_INTERVAL(10)) { + LOG_INFO("alloc mem failed, try to evict pages", K(fd_), K(file_size_), K(io_ctx)); + } + if (OB_FAIL(page_cache_controller_->invoke_swap_and_wait( + MIN(io_ctx.get_todo_size(), ObTmpFileGlobal::TMP_FILE_WRITE_BATCH_PAGE_NUM * ObTmpFileGlobal::PAGE_SIZE), + io_ctx.get_io_timeout_ms()))) { + LOG_WARN("fail to invoke swap and wait", KR(ret), K(io_ctx), K(fd_)); + } + } else { + LOG_WARN("fail to inner write", KR(ret), K(fd_), K(io_ctx), KPC(this)); + } + } + } // end while + } + + if (OB_SUCC(ret)) { + LOG_DEBUG("aio write finish", KR(ret), K(fd_), K(file_size_), K(io_ctx)); + } else { + LOG_DEBUG("aio write failed", KR(ret), K(fd_), K(file_size_), K(io_ctx), KPC(this)); + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_write_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + if (has_unfinished_page_()) { + if (OB_FAIL(inner_fill_tail_page_(io_ctx))) { + LOG_WARN("fail to fill tail page", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_UNLIKELY(has_unfinished_page_() && io_ctx.get_todo_size() > 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("tmp file page is unaligned", KR(ret), K(fd_)); + } + } + + // each batch at most write TMP_FILE_WRITE_BATCH_PAGE_NUM pages + while (OB_SUCC(ret) && io_ctx.get_todo_size() > 0) { + if (OB_FAIL(inner_write_continuous_pages_(io_ctx))) { + LOG_WARN("fail to write continuous pages", KR(ret), K(fd_), K(io_ctx)); + } + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_fill_tail_page_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + ObSpinLockGuard last_page_lock_guard(last_page_lock_); + const bool is_in_disk = (0 == cached_page_nums_); + + if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(io_ctx)); + } else if (is_in_disk) { + if (OB_FAIL(load_disk_tail_page_and_rewrite_(io_ctx))) { + LOG_WARN("fail to load disk tail page and rewrite", KR(ret),K(fd_), K(io_ctx)); + } + } else { + if (OB_FAIL(append_write_memory_tail_page_(io_ctx))) { + LOG_WARN("fail to append write memory tail page", KR(ret), K(fd_), K(io_ctx)); + } + } + + return ret; +} + +int ObSharedNothingTmpFile::load_disk_tail_page_and_rewrite_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + const int64_t has_written_size = get_page_offset_from_file_or_block_offset_(file_size_); + const int64_t begin_page_virtual_id = get_page_virtual_id_from_offset_(file_size_, + true /*is_open_interval*/); + const int64_t need_write_size = MIN(ObTmpFileGlobal::PAGE_SIZE - has_written_size, io_ctx.get_todo_size()); + char *write_buff = io_ctx.get_todo_buffer(); + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileDataItem data_item; + bool block_meta_tree_flushing = false; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("begin page virtual id is invalid", KR(ret), K(fd_), K(begin_page_virtual_id), K(file_size_)); + } else if (OB_UNLIKELY(has_written_size + need_write_size > ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("need write size is invalid", KR(ret), K(fd_), K(has_written_size), K(need_write_size)); + } else if (OB_ISNULL(write_buff)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("write buffer is null", KR(ret), K(fd_), K(write_buff)); + } else if (OB_FAIL(meta_tree_.prepare_for_write_tail(data_item))) { + LOG_WARN("fail to prepare for write tail", KR(ret), K(fd_)); + } else { + block_meta_tree_flushing = true; + } + + if (OB_SUCC(ret)) { + blocksstable::MacroBlockId macro_block_id; + int64_t last_page_begin_offset_in_block = + (data_item.physical_page_id_ + data_item.physical_page_num_ - 1) + * ObTmpFileGlobal::PAGE_SIZE; + char *page_buf = nullptr; + if (OB_FAIL(tmp_file_block_manager_->get_macro_block_id(data_item.block_index_, macro_block_id))) { + LOG_WARN("fail to get macro block id", KR(ret), K(fd_), K(data_item.block_index_)); + } else if (OB_UNLIKELY(!macro_block_id.is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("macro block id is invalid", KR(ret), K(fd_), K(data_item.block_index_)); + } else if (OB_FAIL(wbp_->alloc_page(fd_, ObTmpFilePageUniqKey(begin_page_virtual_id), new_page_id, page_buf))) { + LOG_WARN("fail to alloc page", KR(ret), K(fd_), K(begin_page_virtual_id), + K(new_page_id), KP(page_buf)); + } else { + // load last unfilled page from disk + blocksstable::ObMacroBlockHandle mb_handle; + blocksstable::ObMacroBlockReadInfo info; + info.io_desc_ = io_ctx.get_io_flag(); + info.macro_block_id_ = macro_block_id; + info.size_ = has_written_size; + info.offset_ = last_page_begin_offset_in_block; + info.buf_ = page_buf; + info.io_callback_ = nullptr; + info.io_timeout_ms_ = io_ctx.get_io_timeout_ms(); + + if (OB_FAIL(mb_handle.async_read(info))) { + LOG_ERROR("fail to async write block", KR(ret), K(fd_), K(info)); + } else if (OB_FAIL(mb_handle.wait())) { + LOG_WARN("fail to wait", KR(ret), K(fd_), K(info)); + } else if (mb_handle.get_data_size() < has_written_size) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to read expected size", KR(ret), K(fd_), K(info), K(has_written_size)); + } else if (OB_UNLIKELY(!io_ctx.check_buf_range_valid(write_buff, need_write_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf range", KR(ret), K(fd_), K(write_buff), K(need_write_size), K(io_ctx)); + } else { + // fill last page in memory + MEMCPY(page_buf + has_written_size, write_buff, need_write_size); + } + + if (OB_FAIL(ret)) { + int tmp_ret = OB_SUCCESS; + uint32_t unused_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_TMP_FAIL(wbp_->free_page(fd_, new_page_id, ObTmpFilePageUniqKey(begin_page_virtual_id), unused_page_id))) { + LOG_WARN("fail to free page", KR(tmp_ret), K(fd_), K(begin_page_virtual_id), K(new_page_id)); + } + new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + } else if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, ObTmpFilePageUniqKey(begin_page_virtual_id)))) { + LOG_WARN("fail to notify dirty", KR(ret), K(fd_), K(new_page_id)); + } + } + } + + // update meta data + if (OB_SUCC(ret)) { + common::TCRWLock::WLockGuard guard(meta_lock_); + if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file is deleting", KR(ret), K(fd_)); + } else if (OB_FAIL(meta_tree_.finish_write_tail(data_item, true /*release_tail_in_disk*/))) { + LOG_WARN("fail to finish write tail page", KR(ret), K(fd_)); + } else if (OB_FAIL(io_ctx.update_data_size(need_write_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(need_write_size)); + } else if (OB_FAIL(page_idx_cache_.push(new_page_id))) { + LOG_WARN("fail to push back page idx array", KR(ret), K(fd_), K(new_page_id)); + } else { + cached_page_nums_ = 1; + file_size_ += need_write_size; + begin_page_id_ = new_page_id; + begin_page_virtual_id_ = begin_page_virtual_id; + end_page_id_ = new_page_id; + } + + if (FAILEDx(insert_or_update_data_flush_node_())) { + LOG_WARN("fail to insert or update flush data list", KR(ret), K(fd_), KPC(this)); + } else if (OB_FAIL(insert_or_update_meta_flush_node_())) { + LOG_WARN("fail to insert or update flush meta list", KR(ret), K(fd_), KPC(this)); + } + + LOG_DEBUG("load_disk_tail_page_and_rewrite_ end", KR(ret), K(fd_), K(end_page_id_), KPC(this)); + } + + if (OB_FAIL(ret) && block_meta_tree_flushing) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(meta_tree_.finish_write_tail(data_item, false /*release_tail_in_disk*/))) { + LOG_WARN("fail to modify items after tail load", KR(tmp_ret), K(fd_)); + } + } + return ret; +} + +// attention: +// last_page_lock_ could promise that there are not truncating task, +// evicting task or flush generator task for the last page. +// however, it might happen that append write for the last page is processing after +// flush generator task has been processed over. +// thus, this function should consider the concurrence problem between append write and callback of flush task. +int ObSharedNothingTmpFile::append_write_memory_tail_page_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + char *page_buff = nullptr; + const int64_t has_written_size = get_page_offset_from_file_or_block_offset_(file_size_); + const int64_t need_write_size = MIN(ObTmpFileGlobal::PAGE_SIZE - has_written_size, + io_ctx.get_todo_size()); + const int64_t end_page_virtual_id = get_page_virtual_id_from_offset_(file_size_, + true /*is_open_interval*/); + char *write_buff = io_ctx.get_todo_buffer(); + uint32_t unused_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileDataItem rightest_data_item; + + if (OB_UNLIKELY(has_written_size + need_write_size > ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("need write size is invalid", KR(ret), K(fd_), K(has_written_size), K(need_write_size)); + } else if (OB_ISNULL(write_buff)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("write buffer is null", KR(ret), K(fd_), K(write_buff)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == end_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("end page virtual id is invalid", KR(ret), K(fd_), K(end_page_virtual_id), K(file_size_)); + } else if (OB_FAIL(wbp_->read_page(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id), + page_buff, unused_page_id))) { + LOG_WARN("fail to fetch page", KR(ret), K(fd_), K(end_page_id_), K(end_page_virtual_id)); + } else if (OB_ISNULL(page_buff)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page buff is null", KR(ret), K(fd_), K(end_page_id_)); + } else if ((wbp_->is_write_back(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id)) || + wbp_->is_cached(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id))) && + OB_FAIL(meta_tree_.prepare_for_write_tail(rightest_data_item))) { + LOG_WARN("fail to prepare for write tail", KR(ret), K(fd_), + K(end_page_id_), K(end_page_virtual_id), K(rightest_data_item)); + } else if (OB_UNLIKELY(!io_ctx.check_buf_range_valid(write_buff, need_write_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf range", KR(ret), K(fd_), K(write_buff), K(need_write_size), K(io_ctx)); + } else { + MEMCPY(page_buff + has_written_size, write_buff, need_write_size); + } + + // update meta data + if (OB_SUCC(ret)) { + common::TCRWLock::WLockGuard guard(meta_lock_); + const bool is_cached = wbp_->is_cached(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id)); + const bool is_write_back = wbp_->is_write_back(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id)); + if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file is deleting", KR(ret), K(fd_)); + } else if (is_cached || is_write_back) { + // due to the appending writing for the last page which is flushing or flushed into disk, + // the page carbon in memory and disk will be different. + // thus, we need to rollback the flush status of the last page, + if (OB_FAIL(wbp_->notify_dirty(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id)))) { + LOG_WARN("fail to notify dirty", KR(ret), K(fd_), K(end_page_id_), K(end_page_virtual_id)); + } else if (is_cached) { + // for the last page, if the status of flushed_page_id_ page is not cached, + // we will treat this page as a non-flushed page + flushed_data_page_num_--; + } else if (is_write_back) { + write_back_data_page_num_--; + } + + // modify meta tree + if (OB_FAIL(ret)) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(meta_tree_.finish_write_tail(rightest_data_item, false /*release_tail_in_disk*/))) { + LOG_WARN("fail to modify finish write tail page", KR(tmp_ret), K(fd_)); + } + } else { + // add file into flush list if file is not flushing, + // because we produce 1 dirty data page and 1 dirty meta page after writing tail page + if (OB_FAIL(meta_tree_.finish_write_tail(rightest_data_item, true /*release_tail_in_disk*/))) { + LOG_WARN("fail to finish write tail page", KR(ret), K(fd_)); + } else if (OB_FAIL(insert_or_update_data_flush_node_())) { + LOG_WARN("fail to insert or update flush data list", KR(ret), K(fd_), KPC(this)); + } else if (OB_FAIL(insert_or_update_meta_flush_node_())) { + LOG_WARN("fail to insert or update flush meta list", KR(ret), K(fd_), KPC(this)); + } + } + } + + if (FAILEDx(io_ctx.update_data_size(need_write_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(need_write_size)); + } else { + file_size_ += need_write_size; + } + + LOG_DEBUG("append_write_memory_tail_page_ end", KR(ret), K(fd_), K(end_page_id_), KPC(this)); + } + + return ret; +} + +int ObSharedNothingTmpFile::inner_write_continuous_pages_(ObTmpFileIOCtx &io_ctx) +{ + int ret = OB_SUCCESS; + bool is_alloc_failed = false; + int64_t write_size = 0; + ObArray page_entry_idxs; + + // write pages + if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(io_ctx)); + } else if (OB_FAIL(alloc_and_write_pages_(io_ctx, page_entry_idxs, write_size))) { + if (OB_ALLOCATE_TMP_FILE_PAGE_FAILED == ret) { + // this error code will return to caller after modifing meta data of file based on written data pages + ret = OB_SUCCESS; + is_alloc_failed = true; + if (TC_REACH_COUNT_INTERVAL(10)) { + LOG_INFO("fail to alloc memory", K(tenant_id_), K(fd_), K(write_size), + K(page_entry_idxs), K(io_ctx)); + } + } else { + LOG_WARN("fail to batch write pages", KR(ret), K(fd_), K(io_ctx)); + } + } + + // Update meta data. + if (OB_FAIL(ret)) { + } else if (OB_UNLIKELY(page_entry_idxs.empty() || write_size <= 0)) { + if (!is_alloc_failed) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page entry idxs is empty", KR(ret), K(fd_), K(page_entry_idxs), K(write_size)); + } else { + // do nothing, no need to update meta data + } + } else { + common::TCRWLock::WLockGuard guard(meta_lock_); + const int64_t end_page_virtual_id = cached_page_nums_ == 0 ? + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID : + get_page_virtual_id_from_offset_(file_size_, true /*is_open_interval*/); + if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file is deleting", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID != end_page_id_ && + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == end_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("end page virtual id is invalid", KR(ret), K(fd_), K(end_page_virtual_id), K(file_size_)); + } else if (OB_UNLIKELY((ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_ && + ObTmpFileGlobal::INVALID_PAGE_ID != end_page_id_) || + (ObTmpFileGlobal::INVALID_PAGE_ID != begin_page_id_ && + ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("begin or end page id is invalid", KR(ret), K(fd_), + K(begin_page_id_), + K(end_page_id_)); + } else if (OB_FAIL(io_ctx.update_data_size(write_size))) { + LOG_WARN("fail to update data size", KR(ret), K(fd_), K(io_ctx)); + } else { + if (ObTmpFileGlobal::INVALID_PAGE_ID != end_page_id_ && + OB_FAIL(wbp_->link_page(fd_, page_entry_idxs.at(0), end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id)))) { + LOG_WARN("fail to link page", KR(ret), K(fd_), K(page_entry_idxs.at(0)), + K(end_page_id_), K(end_page_virtual_id)); + } else if (FALSE_IT(end_page_id_ = page_entry_idxs.at(page_entry_idxs.count() - 1))) { + } else { + if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_) { + begin_page_id_ = page_entry_idxs.at(0); + begin_page_virtual_id_ = get_page_virtual_id_from_offset_(file_size_, false /*is_open_interval*/); + } + file_size_ += write_size; + cached_page_nums_ += page_entry_idxs.count(); + } + + for (int64_t i = 0; i < page_entry_idxs.count() && OB_SUCC(ret); i++) { + if (OB_FAIL(page_idx_cache_.push(page_entry_idxs[i]))) { + LOG_WARN("fail to push page idx array", KR(ret), K(fd_)); + } + } // TODO: we can ignore page_idx_cache_ allocate memory fail to continue writing in the future. + if (FAILEDx(insert_or_update_data_flush_node_())) { + LOG_WARN("fail to insert or update data flush list", KR(ret), K(fd_), KPC(this)); + } + } + + if (OB_FAIL(ret)) { + int tmp_ret = OB_SUCCESS; + uint32_t unused_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t free_page_virtual_id = file_size_ == 0 ? 0 : end_page_virtual_id + 1; + for (int64_t i = 0; i < page_entry_idxs.count() && OB_LIKELY(OB_SUCCESS == tmp_ret); i++) { + if (OB_TMP_FAIL(tmp_ret = wbp_->free_page(fd_, page_entry_idxs[i], ObTmpFilePageUniqKey(free_page_virtual_id), unused_page_id))) { + LOG_WARN("fail to free page", KR(tmp_ret), K(fd_), K(i), K(free_page_virtual_id), K(page_entry_idxs[i])); + } else { + free_page_virtual_id += 1; + } + } + } + } // end update meta data. + + // reset allocation failure status + ret = is_alloc_failed && OB_SUCC(ret) ? OB_ALLOCATE_TMP_FILE_PAGE_FAILED : ret; + return ret; +} + +int ObSharedNothingTmpFile::alloc_and_write_pages_(const ObTmpFileIOCtx &io_ctx, + ObIArray &alloced_page_id, + int64_t &actual_write_size) +{ + int ret = OB_SUCCESS; + int64_t write_size = 0; + int64_t new_page_virtual_id = get_page_virtual_id_from_offset_(file_size_, false /*is_open_interval*/); + uint32_t previous_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + alloced_page_id.reset(); + actual_write_size = 0; + + if (OB_UNLIKELY(has_unfinished_page_())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("the file has unfinished page", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == new_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("new page virtual id is invalid", KR(ret), K(fd_), K(file_size_), K(new_page_virtual_id)); + } else if (OB_UNLIKELY(!io_ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(io_ctx)); + } else { + const int64_t expected_write_size = io_ctx.get_todo_size(); + while (OB_SUCC(ret) && actual_write_size < expected_write_size && + alloced_page_id.count() < ObTmpFileGlobal::TMP_FILE_WRITE_BATCH_PAGE_NUM) { + char *page_buf = nullptr; + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_->alloc_page(fd_, ObTmpFilePageUniqKey(new_page_virtual_id), new_page_id, page_buf))) { + LOG_WARN("fail to alloc_page", KR(ret), K(fd_), K(new_page_virtual_id), + K(actual_write_size), K(expected_write_size)); + } else if (OB_ISNULL(page_buf)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page_buf is null", KR(ret), K(fd_), KP(page_buf)); + } else if (OB_UNLIKELY(new_page_id == ObTmpFileGlobal::INVALID_PAGE_ID)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid page id", KR(ret), K(fd_), K(new_page_id)); + } else { + int64_t write_size = common::min(ObTmpFileGlobal::PAGE_SIZE, + expected_write_size - actual_write_size); + const char *write_buf = io_ctx.get_todo_buffer() + actual_write_size; + + if (OB_UNLIKELY(!io_ctx.check_buf_range_valid(write_buf, write_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf range", KR(ret), K(fd_), K(write_buf), K(write_size), K(io_ctx)); + } else if (FALSE_IT(MEMCPY(page_buf, write_buf, write_size))) { + } else if (FALSE_IT(actual_write_size += write_size)) { + } else if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, ObTmpFilePageUniqKey(new_page_virtual_id)))) { + LOG_WARN("fail to notify dirty", KR(ret), K(fd_), K(new_page_id), K(new_page_virtual_id)); + } else if (previous_page_id != ObTmpFileGlobal::INVALID_PAGE_ID && + OB_FAIL(wbp_->link_page(fd_, new_page_id, previous_page_id, + ObTmpFilePageUniqKey(new_page_virtual_id - 1)))) { + LOG_WARN("fail to link page", KR(ret), K(fd_), K(new_page_id), K(previous_page_id), + K(new_page_virtual_id)); + } else if (OB_FAIL(alloced_page_id.push_back(new_page_id))) { + LOG_WARN("wbp fail to push back page id", KR(ret), K(fd_), K(new_page_id)); + } else { + previous_page_id = new_page_id; + new_page_virtual_id += 1; + } + } + + if (OB_FAIL(ret) && OB_NOT_NULL(page_buf)) { + int tmp_ret = OB_SUCCESS; + uint32_t tmp_pid = ObTmpFileGlobal::INVALID_PAGE_ID; // unused + if (OB_TMP_FAIL(wbp_->free_page(fd_, new_page_id, ObTmpFilePageUniqKey(new_page_virtual_id), tmp_pid))) { + LOG_WARN("fail to free page", KR(tmp_ret), K(fd_), K(new_page_id)); + } + } + } // end while + } + LOG_DEBUG("alloc_and_write", KR(ret), K(fd_), K(io_ctx), K(file_size_), + K(end_page_id_), K(actual_write_size), K(alloced_page_id)); + return ret; +} + +// Attention!!!! +// in order to prevent concurrency problems of eviction list +// from the operation from eviction manager and flush manager, +// before eviction manager calls this function, +// it removes the file's node from its' eviction list, +// but still keeps the file's `is_in_data_eviction_list_` be true. +// when this function run over, +// if `remain_flushed_page_num` > 0, this function will reinsert node into the list of eviction manager again; +// otherwise, this function will set `is_in_data_eviction_list_` be false. +int ObSharedNothingTmpFile::evict_data_pages(const int64_t expected_evict_page_num, + int64_t &actual_evict_page_num, + int64_t &remain_flushed_page_num) +{ + int ret = OB_SUCCESS; + bool lock_last_page = false; + common::TCRWLock::WLockGuard meta_lock_guard(meta_lock_); + const int64_t end_page_virtual_id = get_page_virtual_id_from_offset_(file_size_, true /*is_open_interval*/); + actual_evict_page_num = 0; + remain_flushed_page_num = 0; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObSharedNothingTmpFile has not been inited", KR(ret), K(tenant_id_), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(is_deleting_)) { + // actual_evict_page_num = 0; + // remain_flushed_page_num = 0; + is_in_data_eviction_list_ = false; + LOG_INFO("try to evict data pages when file is deleting", K(fd_), K(tenant_id_), KPC(this)); + } else if (OB_UNLIKELY(expected_evict_page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(expected_evict_page_num)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == end_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid end page virtual id", KR(ret), K(fd_), K(end_page_virtual_id), K(file_size_), KPC(this)); + } else if (OB_UNLIKELY(0 == flushed_data_page_num_)) { + is_in_data_eviction_list_ = false; + LOG_INFO("tmp file has no flushed data pages need to be evicted", KR(ret), K(fd_), K(flushed_data_page_num_), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == flushed_page_id_ || + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == flushed_page_virtual_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid flush status", KR(ret), K(fd_), K(flushed_page_id_), K(flushed_page_virtual_id_), KPC(this)); + } else if (OB_UNLIKELY(!is_in_data_eviction_list_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN( "the file is not in data eviction list", K(fd_), K(is_in_data_eviction_list_), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid begin page virtual id", KR(ret), K(fd_), K(begin_page_virtual_id_), KPC(this)); + } else { + bool need_to_evict_last_page = false; + if (flushed_data_page_num_ == cached_page_nums_ && expected_evict_page_num >= flushed_data_page_num_ && + wbp_->is_cached(fd_, end_page_id_, ObTmpFilePageUniqKey(end_page_virtual_id))) { + need_to_evict_last_page = true; + // last page should be evicted, try to lock the last page. + if (OB_FAIL(last_page_lock_.trylock())) { + // fail to get the lock of last data page, it will not be evicted. + ret = OB_SUCCESS; + } else { + lock_last_page = true; + } + } + + int64_t remain_evict_page_num = 0; + if (lock_last_page) { + remain_evict_page_num = flushed_data_page_num_; + } else if (need_to_evict_last_page) { // need_to_evict_last_page && !lock_last_page + remain_evict_page_num = flushed_data_page_num_ - 1; + } else { + remain_evict_page_num = MIN(flushed_data_page_num_, expected_evict_page_num); + } + + const int64_t evict_end_virtual_id = begin_page_virtual_id_ + remain_evict_page_num; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(page_idx_cache_.truncate(evict_end_virtual_id))) { + LOG_WARN("fail to truncate page idx cache", KR(ret), K(fd_), K(evict_end_virtual_id), KPC(this)); + } + + // evict data pages + const bool evict_last_page = lock_last_page; + while (OB_SUCC(ret) && remain_evict_page_num > 0) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t first_cached_page_idx = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (OB_UNLIKELY(!wbp_->is_cached(fd_, begin_page_id_, ObTmpFilePageUniqKey(begin_page_virtual_id_)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("the page is not cached", KR(ret), K(fd_), K(begin_page_id_), K(begin_page_virtual_id_), KPC(this)); + } else if (OB_FAIL(wbp_->free_page(fd_, begin_page_id_, ObTmpFilePageUniqKey(begin_page_virtual_id_), next_page_id))) { + LOG_WARN("fail to free page", KR(ret), K(fd_), K(begin_page_id_), K(begin_page_virtual_id_), K(next_page_id), KPC(this)); + } else { + if (begin_page_id_ == flushed_page_id_) { + flushed_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flushed_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } + begin_page_id_ = next_page_id; + if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_) { + end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + begin_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } else { + begin_page_virtual_id_ += 1; + } + actual_evict_page_num++; + remain_evict_page_num--; + flushed_data_page_num_--; + cached_page_nums_--; + } + } // end while + + if (OB_FAIL(ret)) { + } else if (flushed_data_page_num_ > 0) { + if (OB_FAIL(eviction_mgr_->add_file(false/*is_meta*/, *this))) { + LOG_WARN("fail to add file to eviction mgr", KR(ret), KPC(this)); + } + } else { + is_in_data_eviction_list_ = false; + LOG_DEBUG("all data pages of file have been evicted", KPC(this)); + } + + if (OB_SUCC(ret)) { + remain_flushed_page_num = flushed_data_page_num_; + } + + if (lock_last_page) { + last_page_lock_.unlock(); + } + + LOG_DEBUG("evict data page of tmp file over", KR(ret), K(fd_), K(is_deleting_), + K(expected_evict_page_num), K(actual_evict_page_num), + K(remain_flushed_page_num), + K(begin_page_id_), K(begin_page_virtual_id_), + K(flushed_page_id_), K(flushed_page_virtual_id_), KPC(this)); + } + + return ret; +} + +// Attention!!!! +// in order to prevent concurrency problems of eviction list +// from the operation from eviction manager and flush manager, +// before eviction manager calls this function, +// it removes the file's node from its' eviction list, +// but still keeps the file's `is_in_meta_eviction_list_` be true. +// when this function run over, +// if `remain_flushed_page_num` > 0, this function will reinsert node into the list of eviction manager again; +// otherwise, this function will set `is_in_meta_eviction_list_` be false. +int ObSharedNothingTmpFile::evict_meta_pages(const int64_t expected_evict_page_num, + int64_t &actual_evict_page_num) +{ + int ret = OB_SUCCESS; + // TODO: wanyue.wy, wuyuefei.wyf + // if meta_tree_ could protect the consistency by itself, this lock could be removed + common::TCRWLock::WLockGuard meta_lock_guard(meta_lock_); + actual_evict_page_num = 0; + int64_t total_need_evict_page_num = 0; + int64_t total_need_evict_rightmost_page_num = 0; + int64_t remain_need_evict_page_num = 0; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObSharedNothingTmpFile has not been inited", KR(ret), K(tenant_id_), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(is_deleting_)) { + // actual_evict_page_num = 0; + is_in_meta_eviction_list_ = false; + LOG_INFO("try to evict data pages when file is deleting", K(fd_), K(tenant_id_), KPC(this)); + } else if (OB_UNLIKELY(expected_evict_page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(expected_evict_page_num)); + } else if (OB_UNLIKELY(!is_in_meta_eviction_list_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN( "the file is not in meta eviction list", K(fd_), K(is_in_meta_eviction_list_), KPC(this)); + } else if (OB_FAIL(meta_tree_.get_need_evict_page_num(total_need_evict_page_num, total_need_evict_rightmost_page_num))) { + LOG_WARN( "fail to get need evict page num", KR(ret), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(total_need_evict_page_num <= 0)) { + is_in_meta_eviction_list_ = false; + LOG_INFO("meta tree has no flushed pages need to be evicted", K(fd_), K(total_need_evict_page_num), KPC(this)); + } else if (OB_FAIL(meta_tree_.evict_meta_pages(expected_evict_page_num, + ObTmpFileTreeEvictType::FULL, actual_evict_page_num))) { + LOG_WARN("fail to evict meta pages", KR(ret), K(fd_), K(expected_evict_page_num), KPC(this)); + } else { + remain_need_evict_page_num = total_need_evict_page_num - actual_evict_page_num; + if (remain_need_evict_page_num > 0) { + if (OB_FAIL(eviction_mgr_->add_file(true/*is_meta*/, *this))) { + LOG_WARN("fail to add file to eviction mgr", KR(ret), K(fd_), KPC(this)); + } + } else { + is_in_meta_eviction_list_ = false; + LOG_DEBUG("all meta pages are evicted", KPC(this)); + } + LOG_DEBUG("evict meta page of tmp file over", KR(ret), K(fd_), K(is_deleting_), + K(expected_evict_page_num), K(actual_evict_page_num), K(remain_need_evict_page_num), KPC(this)); + } + + return ret; +} + +// Attention!! +// 1. currently truncate() only gc the memory resource for data page. it doesn't free disk space. +// thus, it will not modify meta tree and not affect flushing mechanism +// 2. if truncate_offset is not the begin or end offset of page, +// we will fill zero from begin_offset to truncate_offset for truncated page in wbp. +// 3. truncate_offset is a open interval number, which means the offset before than it need to be truncated +int ObSharedNothingTmpFile::truncate(const int64_t truncate_offset) +{ + int ret = OB_SUCCESS; + SpinWLockGuard truncate_lock_guard(truncate_lock_); + ObSpinLockGuard last_page_lock_guard(last_page_lock_); + common::TCRWLock::WLockGuard guard(meta_lock_); + int64_t wbp_begin_offset = cal_wbp_begin_offset_(); + LOG_INFO("start to truncate a temporary file", KR(ret), K(fd_), K(truncate_offset), K(wbp_begin_offset), KPC(this)); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObSharedNothingTmpFile has not been inited", KR(ret), K(tenant_id_), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(is_deleting_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to truncate a deleting file", KR(ret), K(tenant_id_), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(wbp_begin_offset < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected wbp begin offset", KR(ret), K(fd_), K(wbp_begin_offset), K(truncate_offset), KPC(this)); + } else if (OB_UNLIKELY(truncate_offset <= 0 || truncate_offset > file_size_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid truncate_offset", KR(ret), K(fd_), K(truncate_offset), K(file_size_), KPC(this)); + } else if (OB_UNLIKELY(truncate_offset <= truncated_offset_)) { + // do nothing + } else { + // release disk space between truncated_offset_ and truncate_offset + if (OB_FAIL(meta_tree_.truncate(truncated_offset_, truncate_offset))) { + LOG_WARN("fail to truncate meta tree", KR(ret), K(fd_), K(truncated_offset_), K(truncate_offset), KPC(this)); + } + + if (FAILEDx(inner_truncate_(truncate_offset, wbp_begin_offset))) { + LOG_WARN("fail to truncate data page", KR(ret), K(fd_), K(truncate_offset), KPC(this)); + } else { + truncated_offset_ = truncate_offset; + } + } + + LOG_INFO("truncate a temporary file over", KR(ret), K(truncate_offset), K(wbp_begin_offset), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::inner_truncate_(const int64_t truncate_offset, const int64_t wbp_begin_offset) +{ + int ret = OB_SUCCESS; + uint32_t truncate_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (OB_UNLIKELY(wbp_begin_offset < 0 || truncate_offset <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid offset", KR(ret), K(fd_), K(wbp_begin_offset), K(truncate_offset)); + } else if (truncate_offset > wbp_begin_offset) { + const int64_t truncate_page_virtual_id = get_page_virtual_id_from_offset_(truncate_offset, + true /*is_open_interval*/); + if (OB_FAIL(page_idx_cache_.binary_search(truncate_page_virtual_id, truncate_page_id))) { + LOG_WARN("fail to find page index in array", KR(ret), K(fd_), K(truncate_page_virtual_id)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID != truncate_page_id) { + // the page index of truncate_offset is in the range of cached page index. + // truncate all page indexes whose offset is smaller than truncate_offset. + if (OB_FAIL(page_idx_cache_.truncate(truncate_page_virtual_id+1))) { + LOG_WARN("fail to truncate page idx cache", KR(ret), K(fd_), K(truncate_page_virtual_id)); + } + } else { // ObTmpFileGlobal::INVALID_PAGE_ID == truncate_page_id + // the page index of truncate_offset is smaller than the smallest cached page index. + // we need to find truncate_page_id by iterating wbp_. + if (OB_FAIL(wbp_->get_page_id_by_virtual_id(fd_, truncate_page_virtual_id, begin_page_id_, truncate_page_id))) { + LOG_WARN("fail to get page id by virtual id", KR(ret), K(fd_), K(truncate_offset), K(begin_page_id_)); + } + } + + if (OB_SUCC(ret)) { + // truncate complete pages except the last page + while(OB_SUCC(ret) && begin_page_id_ != truncate_page_id) { + if (OB_FAIL(truncate_the_first_wbp_page_())) { + LOG_WARN("fail to truncate first wbp page", KR(ret), K(fd_), + K(begin_page_id_), K(truncate_page_id), K(end_page_id_)); + } + } + + // truncate the last page + if (OB_SUCC(ret)) { + const int64_t truncate_offset_in_page = get_page_offset_from_file_or_block_offset_(truncate_offset); + if (OB_UNLIKELY(truncate_page_virtual_id != begin_page_virtual_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected begin page virtual id", KR(ret), K(fd_), K(truncate_page_virtual_id), K(begin_page_virtual_id_)); + } else if (0 == truncate_offset_in_page) { + if (OB_FAIL(truncate_the_first_wbp_page_())) { + LOG_WARN("fail to truncate first wbp page", KR(ret), K(fd_), + K(begin_page_id_), K(truncate_offset_in_page), K(end_page_id_)); + } + } else { // truncate_offset_in_page is in the middle of page, fill zero before the truncate_offset of page + if (OB_FAIL(wbp_->truncate_page(fd_, truncate_page_id, + ObTmpFilePageUniqKey(truncate_page_virtual_id), + truncate_offset_in_page))) { + LOG_WARN("fail to truncate page", KR(ret), K(fd_), K(truncate_page_id), K(truncate_offset_in_page)); + } else { + LOG_INFO("truncate part of page", KR(ret), K(truncate_page_id), K(truncate_offset), + K(truncate_offset_in_page), K(wbp_begin_offset), KPC(this)); + } + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFile::truncate_the_first_wbp_page_() +{ + int ret = OB_SUCCESS; + bool is_flushed_page = false; + bool is_write_back_page = false; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_ || + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("begin_page_id_ is already INVALID", KR(ret), K(fd_), K(begin_page_id_), + K(begin_page_virtual_id_)); + } else if (wbp_->is_cached(fd_, begin_page_id_, ObTmpFilePageUniqKey(begin_page_virtual_id_))) { + // [begin_page_id_, flushed_page_id_] has been flushed + is_flushed_page = true; + } else if (wbp_->is_write_back(fd_, begin_page_id_, ObTmpFilePageUniqKey(begin_page_virtual_id_))) { + is_write_back_page = true; + } + + if (FAILEDx(wbp_->free_page(fd_, begin_page_id_, ObTmpFilePageUniqKey(begin_page_virtual_id_), next_page_id))) { + LOG_WARN("fail to free page", KR(ret), K(fd_), K(begin_page_id_), K(begin_page_virtual_id_)); + } else { + if (is_flushed_page) { + if (flushed_data_page_num_ <= 0) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("flushed_data_page_num_ is unexpected", KR(ret), KPC(this)); + } else { + flushed_data_page_num_--; + if (0 == flushed_data_page_num_) { + if (OB_UNLIKELY(flushed_page_id_ != begin_page_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("flushed_page_id_ or flushed_data_page_num_ is unexpected", KR(ret), KPC(this)); + } else { + flushed_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flushed_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + LOG_INFO("all flushed page has been truncated", KPC(this)); + } + } + } + } else if (is_write_back_page) { + write_back_data_page_num_--; + } + + if (OB_SUCC(ret)) { + cached_page_nums_--; + begin_page_id_ = next_page_id; + if (ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_) { + end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + begin_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } else { + begin_page_virtual_id_ += 1; + } + } + } + return ret; +} + +void ObSharedNothingTmpFile::update_read_offset(int64_t read_offset) +{ + common::TCRWLock::WLockGuard guard(meta_lock_); + if (read_offset > read_offset_) { + read_offset_ = read_offset; + } +} + +void ObSharedNothingTmpFile::get_dirty_meta_page_num_with_lock(int64_t &non_rightmost_dirty_page_num, int64_t &rightmost_dirty_page_num) +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + get_dirty_meta_page_num(non_rightmost_dirty_page_num, rightmost_dirty_page_num); +} + +void ObSharedNothingTmpFile::get_dirty_meta_page_num(int64_t &non_rightmost_dirty_page_num, int64_t &rightmost_dirty_page_num) const +{ + int ret = OB_SUCCESS; + int64_t total_need_flush_page_num = 0; + if (OB_FAIL(meta_tree_.get_need_flush_page_num(total_need_flush_page_num, rightmost_dirty_page_num))) { + non_rightmost_dirty_page_num = 1; + rightmost_dirty_page_num = 1; + LOG_WARN("fail to get need flush page num", KR(ret), KPC(this)); + } else { + non_rightmost_dirty_page_num = total_need_flush_page_num - rightmost_dirty_page_num; + } +} + +int64_t ObSharedNothingTmpFile::get_dirty_data_page_size_with_lock() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return get_dirty_data_page_size(); +} + +int64_t ObSharedNothingTmpFile::get_dirty_data_page_size() const +{ + int ret = OB_SUCCESS; + + int64_t dirty_size = 0; + // cached_page_nums == flushed_data_page_num + dirty_data_page_num + if (0 == cached_page_nums_ || flushed_data_page_num_ == cached_page_nums_) { + dirty_size = 0; + } else { + if (0 == file_size_ % ObTmpFileGlobal::PAGE_SIZE) { + dirty_size = + (cached_page_nums_ - flushed_data_page_num_ - write_back_data_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + } else { + dirty_size = + (cached_page_nums_ - flushed_data_page_num_ - write_back_data_page_num_ - 1) * ObTmpFileGlobal::PAGE_SIZE + + file_size_ % ObTmpFileGlobal::PAGE_SIZE; + } + } + + if (dirty_size < 0) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("dirty_size is unexpected", KR(ret), K(dirty_size), KPC(this)); + dirty_size = 1; + } + return dirty_size; +} + +int64_t ObSharedNothingTmpFile::get_file_size() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return file_size_; +} + +int64_t ObSharedNothingTmpFile::cal_wbp_begin_offset() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return cal_wbp_begin_offset_(); +} + +int64_t ObSharedNothingTmpFile::cal_wbp_begin_offset_() const +{ + int ret = OB_SUCCESS; + int64_t res = -1; + if (0 == cached_page_nums_) { + res = file_size_; + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_ || + begin_page_virtual_id_ * ObTmpFileGlobal::PAGE_SIZE != + get_page_end_offset_by_file_or_block_offset_(file_size_) - + cached_page_nums_ * ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("begin_page_offset_in_file_ is unexpected", KR(ret), + K(begin_page_virtual_id_), K(file_size_), K(cached_page_nums_), KPC(this)); + } else { + res = begin_page_virtual_id_ * ObTmpFileGlobal::PAGE_SIZE; + } + + return res; +} + +int ObSharedNothingTmpFile::remove_flush_node(const bool is_meta) +{ + int ret = OB_SUCCESS; + common::TCRWLock::WLockGuard guard(meta_lock_); + if (OB_FAIL(flush_prio_mgr_->remove_file(is_meta, *this))) { + LOG_WARN("fail to remove flush node", KR(ret), K(is_meta), KPC(this)); + } + return ret; +} + +int ObSharedNothingTmpFile::reinsert_flush_node(const bool is_meta) +{ + int ret = OB_SUCCESS; + common::TCRWLock::WLockGuard guard(meta_lock_); + if (OB_FAIL(reinsert_flush_node_(is_meta))) { + LOG_WARN("fail to reinsert flush node", KR(ret), K(is_meta), KPC(this)); + } + + return ret; +} + +int ObSharedNothingTmpFile::reinsert_flush_node_(const bool is_meta) +{ + int ret = OB_SUCCESS; + ObTmpFileNode &flush_node = is_meta ? meta_flush_node_ : data_flush_node_; + + if (OB_UNLIKELY(nullptr != flush_node.get_next())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("flush node should not have next", KR(ret), K(fd_), K(is_meta)); + } else if (OB_UNLIKELY(is_deleting_)) { + // do nothing + } else if (is_meta) { + int64_t non_rightmost_dirty_page_num = 0; + int64_t rightmost_dirty_page_num = 0; + get_dirty_meta_page_num(non_rightmost_dirty_page_num, rightmost_dirty_page_num); + if (0 == non_rightmost_dirty_page_num && 0 == rightmost_dirty_page_num) { + // no need to reinsert + meta_page_flush_level_ = -1; + } else if (OB_FAIL(flush_prio_mgr_->insert_meta_flush_list(*this, non_rightmost_dirty_page_num, + rightmost_dirty_page_num))) { + LOG_WARN("fail to insert meta flush list", KR(ret), K(fd_), + K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } + } else { + int64_t dirty_page_size = get_dirty_data_page_size(); + if (0 == dirty_page_size) { + // no need to reinsert + data_page_flush_level_ = -1; + } else if (OB_FAIL(flush_prio_mgr_->insert_data_flush_list(*this, dirty_page_size))) { + LOG_WARN("fail to insert data flush list", KR(ret), K(fd_), K(dirty_page_size)); + } + } + + return ret; +} + +bool ObSharedNothingTmpFile::is_in_meta_flush_list() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return OB_NOT_NULL(meta_flush_node_.get_next()); +} + +bool ObSharedNothingTmpFile::is_flushing() +{ + common::TCRWLock::RLockGuard guard(meta_lock_); + return inner_flush_ctx_.is_flushing(); +} + +// ATTENTION! need to be protected by meta_lock_ +int ObSharedNothingTmpFile::insert_or_update_data_flush_node_() +{ + int ret = OB_SUCCESS; + if (!inner_flush_ctx_.is_flushing()) { + int64_t dirty_data_page_size = get_dirty_data_page_size(); + if (data_page_flush_level_ >= 0 && OB_ISNULL(data_flush_node_.get_next())) { + // flush task mgr is processing this file. + // it will add this file to according flush list in the end + } else if (0 == dirty_data_page_size) { + // do nothing + } else if (data_page_flush_level_ < 0) { + if (OB_FAIL(flush_prio_mgr_->insert_data_flush_list(*this, dirty_data_page_size))) { + LOG_WARN("fail to get list idx", KR(ret), K(dirty_data_page_size), KPC(this)); + } + } else if (OB_FAIL(flush_prio_mgr_->update_data_flush_list(*this, dirty_data_page_size))) { + LOG_WARN("fail to update flush list", KR(ret), K(dirty_data_page_size), KPC(this)); + } + } + return ret; +} + +// ATTENTION! call this need to be protected it in meta_lock_ +int ObSharedNothingTmpFile::insert_or_update_meta_flush_node_() +{ + int ret = OB_SUCCESS; + if (!inner_flush_ctx_.is_flushing()) { + int64_t non_rightmost_dirty_page_num = 0; + int64_t rightmost_dirty_page_num = 0; + get_dirty_meta_page_num(non_rightmost_dirty_page_num, rightmost_dirty_page_num); + if (meta_page_flush_level_ >= 0 && OB_ISNULL(meta_flush_node_.get_next())) { + // flush task mgr is processing this file. + // it will add this file to according flush list in the end + } else if (0 == non_rightmost_dirty_page_num + rightmost_dirty_page_num) { + // do nothing + } else if (meta_page_flush_level_ < 0) { + if (OB_FAIL(flush_prio_mgr_->insert_meta_flush_list(*this, non_rightmost_dirty_page_num, + rightmost_dirty_page_num))) { + LOG_WARN("fail to get list idx", KR(ret), + K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num), KPC(this)); + } + } else if (OB_FAIL(flush_prio_mgr_->update_meta_flush_list(*this, non_rightmost_dirty_page_num, + rightmost_dirty_page_num))) { + LOG_WARN("fail to update flush list", KR(ret), + K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num), KPC(this)); + } + } + return ret; +} + +int ObSharedNothingTmpFile::cal_end_position_( + ObArray &flush_infos, + const int64_t start_pos, + int64_t &end_pos, + int64_t &flushed_data_page_num) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(start_pos >= flush_infos.count() || start_pos < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid flushed info num", + KR(ret), K(start_pos), K(flush_infos.count()), K(inner_flush_ctx_), KPC(this)); + } else if (OB_UNLIKELY(flush_infos[start_pos].has_data() && flush_infos[start_pos].has_meta())) { + // we require that each flush info only represents one type of data or meta + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid flush info", KR(ret), K(start_pos), K(flush_infos[start_pos]), KPC(this)); + } else if (flush_infos[start_pos].update_meta_data_done_) { + for (end_pos = start_pos; OB_SUCC(ret) && end_pos < flush_infos.count(); end_pos++) { + if (OB_UNLIKELY(flush_infos[end_pos].has_data() && flush_infos[end_pos].has_meta()) || + OB_UNLIKELY(!flush_infos[end_pos].has_data() && !flush_infos[end_pos].has_meta())) { + ret = OB_ERR_UNEXPECTED; // flush_info contains one and only one type of pages + LOG_WARN("invalid flush info", KR(ret), K(flush_infos[end_pos]), KPC(this)); + } else if (flush_infos[end_pos].update_meta_data_done_) { + if (flush_infos[end_pos].has_data()) { + flushed_data_page_num += flush_infos[end_pos].flush_data_page_num_; + } + } else { + break; + } + } // end for + } + return ret; +} + +int ObSharedNothingTmpFile::update_meta_after_flush(const int64_t info_idx, const bool is_meta, bool &reset_ctx) +{ + int ret = OB_SUCCESS; + reset_ctx = false; + common::TCRWLock::WLockGuard guard(meta_lock_); + int64_t start_pos = is_meta ? inner_flush_ctx_.meta_finished_continuous_flush_info_num_ + : inner_flush_ctx_.data_finished_continuous_flush_info_num_; + int64_t end_pos = start_pos; + ObArray &flush_infos_ = is_meta ? inner_flush_ctx_.meta_flush_infos_ + : inner_flush_ctx_.data_flush_infos_; + int64_t flushed_data_page_num = 0; + if (OB_UNLIKELY(info_idx < 0 || info_idx >= flush_infos_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid idx", KR(ret), K(info_idx), K(is_meta), KPC(this)); + } else if (FALSE_IT(flush_infos_[info_idx].update_meta_data_done_ = true)) { + } else if (OB_FAIL(cal_end_position_(flush_infos_, start_pos, end_pos, flushed_data_page_num))) { + LOG_WARN("fail to cal end position for update after flush", KR(ret), K(info_idx), KPC(this)); + } else if (start_pos < end_pos) { // have new continuous finished flush infos + if (is_meta && OB_FAIL(update_meta_tree_after_flush_(start_pos, end_pos))) { + // meta tree may be updated before file meta infos if data_flush_info IO hang or + // data_flush_info could not insert data items + LOG_WARN("fail to update meta tree", KR(ret), K(start_pos), K(end_pos), KPC(this)); + } else if (!is_meta && OB_FAIL(update_file_meta_after_flush_(start_pos, end_pos, flushed_data_page_num))) { + LOG_WARN("fail to update file meta", KR(ret), K(start_pos), K(end_pos), K(flushed_data_page_num), KPC(this)); + } else { + reset_ctx = true; + } + LOG_DEBUG("update meta based a set of flush info over", KR(ret), K(info_idx), K(start_pos), + K(end_pos), K(inner_flush_ctx_), KPC(this)); + } + + if (OB_FAIL(inner_flush_ctx_.update_finished_continuous_flush_info_num(is_meta, end_pos))) { + LOG_WARN("fail to update finished continuous flush info num", KR(ret), K(start_pos), K(end_pos), KPC(this)); + } else if (inner_flush_ctx_.is_all_finished()) { + if (OB_ISNULL(data_flush_node_.get_next()) && OB_FAIL(reinsert_flush_node_(false /*is_meta*/))) { + LOG_ERROR("fail to reinsert data flush node", KR(ret), K(is_meta), K(inner_flush_ctx_), KPC(this)); + } else if (OB_ISNULL(meta_flush_node_.get_next()) && OB_FAIL(reinsert_flush_node_(true /*is_meta*/))) { + LOG_ERROR("fail to reinsert meta flush node", KR(ret), K(is_meta), K(inner_flush_ctx_), KPC(this)); + } else { + inner_flush_ctx_.reset(); + } + } + + LOG_DEBUG("update_meta_after_flush finish", KR(ret), K(info_idx), K(is_meta), K(inner_flush_ctx_), KPC(this)); + return ret; +} + +// 1. skip truncated flush infos +// 2. abort updating file meta for tail data page if it is appending written after generating flushing task +int ObSharedNothingTmpFile::remove_useless_page_in_data_flush_infos_(const int64_t start_pos, + const int64_t end_pos, + const int64_t flushed_data_page_num, + int64_t &new_start_pos, + int64_t &new_flushed_data_page_num) +{ + int ret = OB_SUCCESS; + ObArray &flush_infos_ = inner_flush_ctx_.data_flush_infos_; + new_start_pos = start_pos; + new_flushed_data_page_num = flushed_data_page_num; + + if (OB_UNLIKELY(start_pos >= end_pos || end_pos < 1 || flushed_data_page_num <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid param", KR(ret), K(fd_), K(start_pos), K(end_pos), K(flushed_data_page_num)); + } else if (start_pos >= flush_infos_.count() || end_pos > flush_infos_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid param", KR(ret), K(fd_), K(start_pos), K(end_pos), K(flush_infos_), KPC(this)); + } else { + // skip truncated flush infos + for (int64_t i = start_pos; OB_SUCC(ret) && i < end_pos; i++) { + int64_t flushed_start_offset = get_page_begin_offset_by_virtual_id_(flush_infos_[start_pos].flush_virtual_page_id_); + int64_t flushed_end_offset = flushed_start_offset + + flush_infos_[i].flush_data_page_num_ * ObTmpFileGlobal::PAGE_SIZE; + if (truncated_offset_ <= flushed_start_offset) { + // the following flushing pages are not affected by truncate_offset + break; + } else if (truncated_offset_ < flushed_end_offset) { + // although a page is truncated partly, it will also be flushed whole page + int64_t flush_info_flushed_size = flushed_end_offset - truncated_offset_; + int64_t flush_info_data_page_num = + common::upper_align(flush_info_flushed_size, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + uint32_t flush_info_virtual_page_id = + common::lower_align(truncated_offset_, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + new_flushed_data_page_num -= (flush_infos_[i].flush_data_page_num_ - flush_info_data_page_num); + + LOG_INFO("due to truncating, modify flush info", KR(ret), K(fd_), + K(start_pos), K(end_pos), K(flushed_data_page_num), K(i), + K(new_start_pos), K(new_flushed_data_page_num), + K(truncated_offset_), K(flushed_start_offset), K(flushed_end_offset), K(flush_infos_[i]), + K(flush_info_data_page_num), K(flush_info_virtual_page_id)); + flush_infos_[i].flush_data_page_num_ = flush_info_data_page_num; + flush_infos_[i].flush_virtual_page_id_ = flush_info_virtual_page_id; + break; + } else { + // all pages of this flush_info have been truncated, abort it + new_start_pos++; + new_flushed_data_page_num -= flush_infos_[i].flush_data_page_num_; + LOG_INFO("due to truncating, abort flush info", KR(ret), K(fd_), + K(start_pos), K(end_pos), K(flushed_data_page_num), K(i), + K(new_start_pos), K(new_flushed_data_page_num), + K(truncated_offset_), K(flushed_start_offset), K(flushed_end_offset), K(flush_infos_[i])); + } + } // end for + + // abort updating file meta for tail data page + if (OB_SUCC(ret) && new_flushed_data_page_num > 0 && new_start_pos < end_pos) { + if (flush_infos_[end_pos - 1].file_size_ > 0 && flush_infos_[end_pos - 1].file_size_ != file_size_) { + // the last page has been written after flush task has been generated. + // we will discard the flushed last page when update meta data + // (append write has correct the meta data of the last page, here only need to ignore it) + + int64_t discard_page_virtual_id = flush_infos_[end_pos - 1].flush_virtual_page_id_ + + flush_infos_[end_pos - 1].flush_data_page_num_ - 1; + uint32_t discard_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == discard_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("discard page virtual id is invalid", KR(ret), K(fd_), + K(discard_page_virtual_id), K(flush_infos_[end_pos - 1])); + } else if (OB_FAIL(get_physical_page_id_in_wbp(discard_page_virtual_id, discard_page_id))) { + LOG_WARN("fail to get physical page id in wbp", KR(ret), K(fd_), K(discard_page_virtual_id)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == discard_page_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("discard page id is invalid", KR(ret), K(fd_), K(discard_page_id)); + } else if (OB_UNLIKELY(!wbp_->is_dirty(fd_, discard_page_id, ObTmpFilePageUniqKey(discard_page_virtual_id)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("discard page is not dirty", KR(ret), K(fd_), K(discard_page_id), K(discard_page_virtual_id)); + } else { + new_flushed_data_page_num -= 1; + flush_infos_[end_pos - 1].flush_data_page_num_ -= 1; + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFile::update_file_meta_after_flush_(const int64_t start_pos, + const int64_t end_pos, + const int64_t flushed_data_page_num) +{ + int ret = OB_SUCCESS; + ObArray &flush_infos_ = inner_flush_ctx_.data_flush_infos_; + LOG_DEBUG("update_file_meta_after_flush start", + KR(ret), K(start_pos), K(end_pos), K(flushed_data_page_num), KPC(this)); + int64_t new_start_pos = start_pos; + int64_t new_flushed_data_page_num = flushed_data_page_num; + + if (OB_UNLIKELY(start_pos >= end_pos || end_pos < 1 || flushed_data_page_num <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid param", KR(ret), K(fd_), K(end_pos), K(flushed_data_page_num)); + } else if (start_pos >= flush_infos_.count() || end_pos > flush_infos_.count()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid param", KR(ret), K(start_pos), K(end_pos), K(flush_infos_), KPC(this)); + } else if (OB_FAIL(remove_useless_page_in_data_flush_infos_(start_pos, end_pos, flushed_data_page_num, + new_start_pos, new_flushed_data_page_num))) { + LOG_WARN("fail to remove useless page in flush infos", KR(ret), K(fd_), K(start_pos), K(end_pos), + K(flushed_data_page_num), K(end_pos)); + } else if (0 == new_flushed_data_page_num) { + // do nothing + } else if (OB_UNLIKELY(new_flushed_data_page_num < 0 || new_start_pos >= end_pos)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid param", KR(ret), K(fd_), K(start_pos), K(end_pos), K(flushed_data_page_num), + K(new_start_pos), K(new_flushed_data_page_num)); + } else { // exist multiple continuous pages have been flushed over + uint32_t last_flushed_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t cur_flush_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t cur_flush_page_virtual_id = flush_infos_[new_start_pos].flush_virtual_page_id_; + if (OB_FAIL(get_physical_page_id_in_wbp(flush_infos_[new_start_pos].flush_virtual_page_id_, cur_flush_page_id))) { + LOG_WARN("fail to get physical page id in wbp", + KR(ret),K(fd_), K(start_pos), K(end_pos), K(flushed_data_page_num), + K(new_start_pos), K(new_flushed_data_page_num), + K(flush_infos_[new_start_pos])); + } else { + int64_t write_back_succ_data_page_num = 0; + + // update each flush info + for (int64_t i = new_start_pos; OB_SUCC(ret) && i < end_pos; ++i) { + const InnerFlushInfo& flush_info = flush_infos_[i]; + const int64_t cur_flushed_data_page_num = flush_info.flush_data_page_num_; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == cur_flush_page_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid next flush page id", KR(ret), K(fd_), K(cur_flush_page_id), + K(begin_page_id_), K(flushed_page_id_), K(end_page_id_)); + } else { + int64_t cur_page_virtual_id_in_flush_info = cur_flush_page_virtual_id; + // update each page of flush info + for (int64_t j = 0; OB_SUCC(ret) && j < cur_flushed_data_page_num; ++j) { + if (OB_FAIL(wbp_->notify_write_back_succ(fd_, cur_flush_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id_in_flush_info)))) { + LOG_WARN("fail to mark page as clean", KR(ret), K(fd_), + K(flushed_page_id_), K(cur_page_virtual_id_in_flush_info), K(cur_flush_page_id), + K(flushed_data_page_num), K(new_flushed_data_page_num)); + } else if (FALSE_IT(last_flushed_page_id = cur_flush_page_id)) { + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_flush_page_id, + ObTmpFilePageUniqKey(cur_page_virtual_id_in_flush_info), cur_flush_page_id))) { + LOG_WARN("fail to get next page id", KR(ret), K(fd_), K(cur_flush_page_id), + K(cur_page_virtual_id_in_flush_info)); + } else { + cur_page_virtual_id_in_flush_info += 1; + } + } // end for + if (OB_SUCC(ret)) { + write_back_succ_data_page_num += cur_flushed_data_page_num; + cur_flush_page_virtual_id += cur_flushed_data_page_num; + } + } + } // end for + + // update file meta + if (OB_SUCC(ret)) { + if (write_back_succ_data_page_num != new_flushed_data_page_num) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("write_back_succ_data_page_num not correct", + KR(ret), K(fd_), K(write_back_succ_data_page_num), + K(flushed_data_page_num), K(new_flushed_data_page_num), KPC(this)); + } else { + flushed_page_id_ = last_flushed_page_id; + flushed_page_virtual_id_ = cur_flush_page_virtual_id - 1; + flushed_data_page_num_ += new_flushed_data_page_num; + write_back_data_page_num_ -= new_flushed_data_page_num; + if (write_back_data_page_num_ < 0) { + int tmp_ret = OB_ERR_UNEXPECTED; + LOG_WARN("write_back_data_page_num_ is unexpected", KR(tmp_ret), K(fd_), + K(write_back_data_page_num_), K(flushed_data_page_num), K(new_flushed_data_page_num), KPC(this)); + write_back_data_page_num_ = 0; + } + + if (!is_deleting_ && !is_in_data_eviction_list_ && OB_ISNULL(data_eviction_node_.get_next())) { + if (OB_FAIL(eviction_mgr_->add_file(false/*is_meta*/, *this))) { + LOG_WARN("fail to insert into eviction list", KR(ret), K(fd_)); + } else { + is_in_data_eviction_list_ = true; + } + } + } + } // update file meta over + } + } // handle continuous success flush infos over + + LOG_DEBUG("update_file_meta_after_flush finish", KR(ret), K(fd_), + K(start_pos), K(end_pos), K(flushed_data_page_num), + K(new_start_pos), K(new_flushed_data_page_num), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::update_meta_tree_after_flush_(const int64_t start_pos, const int64_t end_pos) +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + + ObArray &flush_infos_ = inner_flush_ctx_.meta_flush_infos_; + for (int64_t i = start_pos; i < end_pos; i++) { + // ATTENTION! need to alloc memory inside, caller must retry update meta data if alloc fail + if (OB_FAIL(meta_tree_.update_after_flush(flush_infos_[i].flush_meta_page_array_))) { + LOG_ERROR("fail to update meta items", KR(ret), K(fd_), K(flush_infos_[i]), KPC(this)); + } else { + LOG_INFO("succ to update meta items", KR(ret), K(fd_), K(flush_infos_[i]), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + if (!is_deleting_ && !is_in_meta_eviction_list_ && OB_ISNULL(meta_eviction_node_.get_next())) { + int64_t total_need_evict_page_num = 1; + int64_t total_need_evict_rightmost_page_num = 1; + if (OB_TMP_FAIL(meta_tree_.get_need_evict_page_num(total_need_evict_page_num, + total_need_evict_rightmost_page_num))) { + LOG_ERROR("fail to get_need_evict_page_num", KR(tmp_ret), + K(total_need_evict_page_num), K(total_need_evict_rightmost_page_num), KPC(this)); + } + + if (total_need_evict_page_num > 0) { + if (OB_FAIL(eviction_mgr_->add_file(true/*is_meta*/, *this))) { + LOG_WARN("fail to insert into eviction list", KR(ret), K(fd_)); + } else { + is_in_meta_eviction_list_ = true; + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFile::generate_data_flush_info( + ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &info, + ObTmpFileDataFlushContext &data_flush_context, + const int64_t flush_sequence, + const bool need_flush_tail) +{ + int ret = OB_SUCCESS; + info.reset(); + + if (!truncate_lock_.try_rdlock()) { + ret = OB_ITER_END; + LOG_WARN("fail to get truncate lock", KR(ret), K(fd_), KPC(this)); + } else { + common::TCRWLock::RLockGuard guard(meta_lock_); + uint32_t copy_begin_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t copy_begin_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + + uint32_t copy_end_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(cal_next_flush_page_id_from_flush_ctx_or_file_(data_flush_context, + copy_begin_page_id, + copy_begin_page_virtual_id))) { + LOG_WARN("fail to calculate next_flush_page_id", KR(ret), + K(flush_task), K(info), K(data_flush_context), KPC(this)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID == copy_begin_page_id) { + ret = OB_ITER_END; + LOG_DEBUG("no more data to flush", KR(ret), K(fd_), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == copy_begin_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("next_flush_page_virtual_id is invalid", KR(ret), K(fd_), K(copy_begin_page_id), + K(copy_begin_page_virtual_id), K(flush_task), K(info), K(data_flush_context), + K(flush_sequence), K(need_flush_tail), KPC(this)); + } else if (OB_FAIL(get_flush_end_page_id_(copy_end_page_id, need_flush_tail))) { + LOG_WARN("fail to get_flush_end_page_id_", KR(ret), + K(flush_task), K(info), K(data_flush_context), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_FLUSH_SEQUENCE != inner_flush_ctx_.flush_seq_ + && flush_sequence != inner_flush_ctx_.flush_seq_ + && flush_sequence != flush_task.get_flush_seq())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("flush sequence not match", + KR(ret), K(inner_flush_ctx_.flush_seq_), K(flush_task), KPC(this)); + } else if (OB_FAIL(copy_flush_data_from_wbp_(flush_task, info, data_flush_context, + copy_begin_page_id, copy_begin_page_virtual_id, + copy_end_page_id, + flush_sequence, need_flush_tail))) { + LOG_WARN("fail to copy flush data from wbp", KR(ret), + K(flush_task), K(info), K(data_flush_context), KPC(this)); + } + if (OB_FAIL(ret)) { + truncate_lock_.unlock(); + } + LOG_DEBUG("generate flush data info end", KR(ret), K(fd_), + K(data_flush_context), K(info), K(flush_task), KPC(this)); + } + return ret; +} + +int ObSharedNothingTmpFile::copy_flush_data_from_wbp_( + ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &info, + ObTmpFileDataFlushContext &data_flush_context, + const uint32_t copy_begin_page_id, + const int64_t copy_begin_page_virtual_id, + const uint32_t copy_end_page_id, + const int64_t flush_sequence, + const bool need_flush_tail) +{ + int ret = OB_SUCCESS; + char *buf = flush_task.get_data_buf(); + int64_t write_offset = flush_task.get_total_page_num() * ObTmpFileGlobal::PAGE_SIZE; + + bool has_last_page_lock = false; + int64_t next_disk_page_id = flush_task.get_next_free_page_id(); + int64_t flushing_page_num = 0; + int64_t origin_info_num = inner_flush_ctx_.data_flush_infos_.size(); + int64_t cur_page_file_offset = -1; + uint32_t cur_flushed_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t cur_page_id = copy_begin_page_id; + int64_t cur_page_virtual_id = copy_begin_page_virtual_id; + + if (OB_ISNULL(buf) || OB_UNLIKELY(OB_SERVER_BLOCK_MGR.get_macro_block_size() <= write_offset)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf or write_offset", KR(ret), KP(buf), K(write_offset), K(flush_task), KPC(this)); + } else if (OB_FAIL(inner_flush_ctx_.data_flush_infos_.push_back(InnerFlushInfo()))) { + LOG_WARN("fail to push back empty flush info", KR(ret), K(fd_), K(info), K(flush_task), KPC(this)); + } + while (OB_SUCC(ret) && cur_page_id != copy_end_page_id && write_offset < OB_SERVER_BLOCK_MGR.get_macro_block_size()) { + if (need_flush_tail && cur_page_id == end_page_id_ && file_size_ % ObTmpFileGlobal::PAGE_SIZE != 0) { + if (OB_SUCC(last_page_lock_.trylock())) { + has_last_page_lock = true; + } else { + LOG_WARN("fail to get last page lock", KR(ret), K(fd_)); + ret = OB_SUCCESS; // ignore error to continue flushing the copied data + break; + } + } + char *page_buf = nullptr; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + bool original_state_is_dirty = wbp_->is_dirty(fd_, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)); + if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id), page_buf, next_page_id))) { + LOG_WARN("fail to read page", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_FAIL(wbp_->notify_write_back(fd_, cur_page_id, ObTmpFilePageUniqKey(cur_page_virtual_id)))) { + LOG_WARN("fail to notify write back", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_UNLIKELY(!flush_task.check_buf_range_valid(buf, ObTmpFileGlobal::PAGE_SIZE))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buffer range", KR(ret), K(fd_), K(write_offset), KP(buf)); + } else { + // ObTmpPageCacheKey cache_key(flush_task.get_block_index(), + // write_offset / ObTmpFileGlobal::PAGE_SIZE, tenant_id_); + // ObTmpPageCacheValue cache_value(page_buf); + // ObTmpPageCache::get_instance().try_put_page_to_cache(cache_key, cache_value); + + MEMCPY(buf + write_offset, page_buf, ObTmpFileGlobal::PAGE_SIZE); + write_offset += ObTmpFileGlobal::PAGE_SIZE; + flushing_page_num += 1; + cur_flushed_page_id = cur_page_id; + cur_page_id = next_page_id; + cur_page_virtual_id += 1; + if (original_state_is_dirty) { + write_back_data_page_num_++; + } + } + } + + if (OB_SUCC(ret) && 0 == flushing_page_num) { + ret = OB_ITER_END; + } + if (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID == cur_flushed_page_id) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("cur_flushed_page_id is unexpected", KR(ret), K(cur_flushed_page_id), KPC(this)); + } + + if (OB_SUCC(ret)) { + int64_t flush_info_idx = inner_flush_ctx_.data_flush_infos_.size() - 1; + // set flush_info in flush_task + info.flush_data_page_disk_begin_id_ = next_disk_page_id; + info.flush_data_page_num_ = flushing_page_num; + info.batch_flush_idx_ = flush_info_idx; + info.flush_virtual_page_id_ = copy_begin_page_virtual_id; + info.has_last_page_lock_ = has_last_page_lock; + // record file_size to check if the last page is appended while flushing + if (has_last_page_lock) { + info.file_size_ = file_size_; + } + + info.fd_ = fd_; + // set flush_info in file inner_flush_ctx + if (OB_FAIL(info.file_handle_.init(this))) { + LOG_WARN("fail to init tmp file handle", KR(ret), K(fd_), K(flush_task), KPC(this)); + } else if (OB_FAIL(inner_flush_ctx_.data_flush_infos_.at(flush_info_idx).init_by_tmp_file_flush_info(info))) { + LOG_WARN("fail to init_by_tmp_file_flush_info", KR(ret), K(fd_), K(flush_task), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + // maintain flushed_page_id recorded in flush_mgr + data_flush_context.set_flushed_page_id(cur_flushed_page_id); + data_flush_context.set_flushed_page_virtual_id(cur_page_virtual_id - 1); + data_flush_context.set_next_flush_page_id(cur_page_id); + data_flush_context.set_next_flush_page_virtual_id(cur_page_virtual_id); + data_flush_context.set_is_valid(true); + if (has_last_page_lock) { + data_flush_context.set_has_flushed_last_partially_written_page(true); + } + flush_task.set_data_length(write_offset); + + inner_flush_ctx_.flush_seq_ = flush_sequence; + } else { + LOG_WARN("fail to generate data flush info", KR(ret), K(fd_), K(need_flush_tail), + K(flush_sequence), K(data_flush_context), K(info), K(flush_task), KPC(this)); + if (inner_flush_ctx_.data_flush_infos_.size() == origin_info_num + 1) { + inner_flush_ctx_.data_flush_infos_.pop_back(); + } + if (has_last_page_lock) { + last_page_lock_.unlock(); + } + } + return ret; +} + +int ObSharedNothingTmpFile::generate_meta_flush_info( + ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &info, + ObTmpFileTreeFlushContext &meta_flush_context, + const int64_t flush_sequence, + const bool need_flush_tail) +{ + int ret = OB_SUCCESS; + info.reset(); + + common::TCRWLock::RLockGuard guard(meta_lock_); + ObArray &flush_infos_ = inner_flush_ctx_.meta_flush_infos_; + int64_t origin_info_num = flush_infos_.size(); + + const int64_t block_index = flush_task.get_block_index(); + char *buf = flush_task.get_data_buf(); + int64_t write_offset = flush_task.get_total_page_num() * ObTmpFileGlobal::PAGE_SIZE; + + ObTmpFileTreeEvictType flush_type = need_flush_tail ? + ObTmpFileTreeEvictType::FULL : ObTmpFileTreeEvictType::MAJOR; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_FLUSH_SEQUENCE != inner_flush_ctx_.flush_seq_ + && flush_sequence != inner_flush_ctx_.flush_seq_ + && flush_sequence != flush_task.get_flush_seq())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("flush sequence not match", KR(ret), K(flush_sequence), K(inner_flush_ctx_.flush_seq_), + K(flush_task), KPC(this)); + } else if (OB_ISNULL(buf) || OB_UNLIKELY(OB_SERVER_BLOCK_MGR.get_macro_block_size() <= write_offset)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid buf or write_offset", KR(ret), KP(buf), K(write_offset), K(flush_task), KPC(this)); + } else if (OB_FAIL(flush_infos_.push_back(InnerFlushInfo()))) { + LOG_WARN("fail to push back empty flush info", KR(ret), K(fd_), K(info), K(flush_task), KPC(this)); + } else if (OB_FAIL(meta_tree_.flush_meta_pages_for_block(block_index, flush_type, buf, write_offset, + meta_flush_context, info.flush_meta_page_array_))) { + LOG_WARN("fail to flush meta pages for block", KR(ret), K(fd_), K(flush_task), K(meta_flush_context), KPC(this)); + } else if (0 == info.flush_meta_page_array_.count()) { + ret = OB_ITER_END; + } + + if (OB_SUCC(ret)) { + int64_t flush_info_idx = flush_infos_.size() - 1; + info.batch_flush_idx_ = flush_info_idx; + + info.fd_ = fd_; + // set flush_info in flush_task + if (OB_FAIL(info.file_handle_.init(this))) { + LOG_WARN("fail to init tmp file handle", KR(ret), K(fd_), K(flush_task), KPC(this)); + } else if (OB_FAIL(inner_flush_ctx_.meta_flush_infos_.at(flush_info_idx).init_by_tmp_file_flush_info(info))) { + LOG_WARN("fail to init_by_tmp_file_flush_info", KR(ret), K(fd_), K(flush_task), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + flush_task.set_data_length(write_offset); + inner_flush_ctx_.flush_seq_ = flush_sequence; + } else { + LOG_WARN("fail to generate meta flush info", KR(ret), K(fd_), K(flush_task), + K(meta_flush_context), K(flush_sequence), K(need_flush_tail)); + if (flush_infos_.size() == origin_info_num + 1) { + flush_infos_.pop_back(); + } + } + + LOG_INFO("generate flush meta info end", KR(ret), K(fd_), K(need_flush_tail), + K(inner_flush_ctx_), K(meta_flush_context), K(info), K(flush_task), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::insert_meta_tree_item(const ObTmpFileFlushInfo &info, int64_t block_index) +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + + common::TCRWLock::WLockGuard guard(meta_lock_); + if (info.has_data()) { + ObSharedNothingTmpFileDataItem data_item; + data_item.block_index_ = block_index; + data_item.physical_page_id_ = info.flush_data_page_disk_begin_id_; + data_item.physical_page_num_ = info.flush_data_page_num_; + data_item.virtual_page_id_ = info.flush_virtual_page_id_; + ObArray data_items; + + if (OB_FAIL(meta_tree_.prepare_for_insert_items())) { + LOG_WARN("fail to prepare for insert items", KR(ret), K(info), K(block_index), KPC(this)); + } else if (OB_FAIL(data_items.push_back(data_item))) { + LOG_WARN("fail to push back data item", KR(ret), K(info), K(block_index), KPC(this)); + } else if (OB_FAIL(meta_tree_.insert_items(data_items))) { + LOG_WARN("fail to insert data items", KR(ret), K(info), K(block_index), KPC(this)); + } + + if (OB_SUCC(ret) && info.has_last_page_lock_) { + last_page_lock_.unlock(); + } + + if (OB_SUCC(ret)) { + truncate_lock_.unlock(); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("flush info does not contain data info", KR(ret), K(info), KPC(this)); + } + + if (!is_deleting_ && !is_in_meta_eviction_list_ && OB_ISNULL(meta_eviction_node_.get_next())) { + int64_t total_need_evict_page_num = 1; + int64_t total_need_evict_rightmost_page_num = 1; + if (OB_TMP_FAIL(meta_tree_.get_need_evict_page_num(total_need_evict_page_num, + total_need_evict_rightmost_page_num))) { + LOG_ERROR("fail to get_need_evict_page_num", KR(tmp_ret), + K(total_need_evict_page_num), K(total_need_evict_rightmost_page_num), KPC(this)); + } + + if (total_need_evict_page_num > 0) { + if (OB_TMP_FAIL(eviction_mgr_->add_file(true/*is_meta*/, *this))) { + LOG_WARN("fail to insert into eviction list", KR(ret), K(fd_), KPC(this)); + } else { + is_in_meta_eviction_list_ = true; + } + } + } + + // reinsert meta flush node during flushing to allow meta pages to be flushed if + // insert_meta_tree_item need tp allocate new meta pages + if (!is_deleting_ && OB_ISNULL(meta_flush_node_.get_next())) { + if (OB_TMP_FAIL(reinsert_flush_node_(true/*is_meta*/))) { + LOG_WARN("fail to reinsert flush node", KR(ret), K(fd_), K(info), K(block_index), KPC(this)); + } + } + + LOG_DEBUG("insert_meta_tree_item end", KR(ret), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::cal_next_flush_page_id_from_flush_ctx_or_file_( + const ObTmpFileDataFlushContext &data_flush_context, + uint32_t &next_flush_page_id, + int64_t &next_flush_page_virtual_id) +{ + int ret = OB_SUCCESS; + if (data_flush_context.is_valid()) { + // use next_flush_page_id from data_flush_ctx + if (data_flush_context.has_flushed_last_partially_written_page()) { + next_flush_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + next_flush_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } else { + next_flush_page_id = data_flush_context.get_next_flush_page_id(); + next_flush_page_virtual_id = data_flush_context.get_next_flush_page_virtual_id(); + int64_t truncate_page_virtual_id = get_page_virtual_id_from_offset_(truncated_offset_, + false /*is_open_interval*/); + if (ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID != truncate_page_virtual_id && + truncate_page_virtual_id > data_flush_context.get_flushed_page_virtual_id()) { + if (OB_FAIL(get_next_flush_page_id_(next_flush_page_id, next_flush_page_virtual_id))) { + LOG_ERROR("origin next flush page has been truncated, fail to get_next_flush_page_id_", + KR(ret), K(data_flush_context), KPC(this)); + } else { + LOG_INFO("origin next flush page has been truncated", + KR(ret), K(next_flush_page_id), K(next_flush_page_virtual_id), K(data_flush_context), KPC(this)); + } + } + + if (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID != truncate_page_virtual_id && + truncate_page_virtual_id <= data_flush_context.get_flushed_page_virtual_id()) { + uint32_t last_flushed_page_id = data_flush_context.get_flushed_page_id(); + int64_t last_flushed_page_virtual_id = data_flush_context.get_flushed_page_virtual_id(); + if (ObTmpFileGlobal::INVALID_PAGE_ID != last_flushed_page_id) { + if (!wbp_->is_write_back(fd_, last_flushed_page_id, ObTmpFilePageUniqKey(last_flushed_page_virtual_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("last flushed page state not match", + KR(ret), K(last_flushed_page_id), K(data_flush_context), KPC(this)); + } + } + } + } + } else { + // cal next_flush_page_id from file meta when doing flush for the first time + if (OB_FAIL(get_next_flush_page_id_(next_flush_page_id, next_flush_page_virtual_id))) { + LOG_WARN("fail to get_next_flush_page_id_", KR(ret), K(fd_), K(data_flush_context)); + } + } + + if (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID != next_flush_page_id) { + if (!wbp_->is_dirty(fd_, next_flush_page_id, ObTmpFilePageUniqKey(next_flush_page_virtual_id)) && + !wbp_->is_write_back(fd_, next_flush_page_id, ObTmpFilePageUniqKey(next_flush_page_virtual_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("next_flush_page_id state not validate", + KR(ret), K(next_flush_page_id), K(data_flush_context), KPC(this)); + } + } + + return ret; +} + +// output next page id after flushed_page_id_ in normal case,or output flushed_page_id for write operation appending last page +int ObSharedNothingTmpFile::get_next_flush_page_id_(uint32_t& next_flush_page_id, int64_t& next_flush_page_virtual_id) const +{ + int ret = OB_SUCCESS; + next_flush_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + next_flush_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + if (ObTmpFileGlobal::INVALID_PAGE_ID == flushed_page_id_) { + // all pages are dirty + if (OB_UNLIKELY((ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_ && + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID != begin_page_virtual_id_) || + (ObTmpFileGlobal::INVALID_PAGE_ID != begin_page_id_ && + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_) )) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("begin_page_id_ and begin_page_virtual_id_ not match", KR(ret), K(begin_page_id_), K(begin_page_virtual_id_), KPC(this)); + } else { + next_flush_page_id = begin_page_id_; + next_flush_page_virtual_id = begin_page_virtual_id_; + } + } else if (wbp_->is_dirty(fd_, flushed_page_id_, ObTmpFilePageUniqKey(flushed_page_virtual_id_))) { + // start from flushed_page_id_ if flushed_page_id_ is dirty caused by appending write + if (OB_UNLIKELY((ObTmpFileGlobal::INVALID_PAGE_ID == flushed_page_id_ && + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID != flushed_page_virtual_id_) || + (ObTmpFileGlobal::INVALID_PAGE_ID != flushed_page_id_ && + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == flushed_page_virtual_id_) )) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("flushed_page_id_ and flushed_page_virtual_id_ not match", + KR(ret), K(flushed_page_id_), K(flushed_page_virtual_id_), KPC(this)); + } else { + next_flush_page_id = flushed_page_id_; + next_flush_page_virtual_id = flushed_page_virtual_id_; + } + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, flushed_page_id_, ObTmpFilePageUniqKey(flushed_page_virtual_id_), next_flush_page_id))){ + // start from the next page, could return INVALID_PAGE_ID if flushed_page_id_ == end_page_id_ + LOG_WARN("fail to get next page id", KR(ret), K(fd_), K(begin_page_id_), K(flushed_page_id_), K(end_page_id_)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID == next_flush_page_id) { + next_flush_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } else { + next_flush_page_virtual_id = flushed_page_virtual_id_ + 1; + } + + LOG_DEBUG("get_next_flush_page_id_", KR(ret), K(fd_), K(begin_page_id_), K(flushed_page_id_), KPC(this)); + return ret; +} + +int ObSharedNothingTmpFile::get_physical_page_id_in_wbp(const int64_t virtual_page_id, uint32_t& page_id) const +{ + int ret = OB_SUCCESS; + int64_t end_page_virtual_id = cached_page_nums_ == 0 ? + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID : + get_page_virtual_id_from_offset_(file_size_, true /*is_open_interval*/); + page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == virtual_page_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd_), K(virtual_page_id)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == begin_page_virtual_id_ || + ObTmpFileGlobal::INVALID_PAGE_ID == begin_page_id_ || + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == end_page_virtual_id || + ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("no pages exist in wbp", KR(ret), K(begin_page_id_), K(begin_page_virtual_id_), + K(end_page_id_), K(end_page_virtual_id), KPC(this)); + } else if (OB_UNLIKELY(virtual_page_id < begin_page_virtual_id_ || virtual_page_id > end_page_virtual_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("the page doesn't exist in wbp", KR(ret), K(virtual_page_id), K(end_page_virtual_id), K(begin_page_virtual_id_), KPC(this)); + } else if (virtual_page_id == begin_page_virtual_id_) { + page_id = begin_page_id_; + } else if (virtual_page_id < flushed_page_virtual_id_) { + if (OB_FAIL(wbp_->get_page_id_by_virtual_id(fd_, virtual_page_id, begin_page_id_, page_id))) { + LOG_WARN("fail to get page id by virtual id", KR(ret), K(virtual_page_id), K(begin_page_id_), KPC(this)); + } + } else if (virtual_page_id == flushed_page_virtual_id_) { + page_id = flushed_page_id_; + } else if (virtual_page_id == end_page_virtual_id) { + page_id = end_page_id_; + } else { // virtual_page_id < end_page_virtual_id + if (OB_FAIL(wbp_->get_page_id_by_virtual_id(fd_, virtual_page_id, flushed_page_id_, page_id))) { + LOG_WARN("fail to get page id by virtual id", KR(ret), K(virtual_page_id), K(flushed_page_id_), KPC(this)); + } + } + + return ret; +} + +// output the last page to be flushed +int ObSharedNothingTmpFile::get_flush_end_page_id_(uint32_t& flush_end_page_id, const bool need_flush_tail) const +{ + int ret = OB_SUCCESS; + const int64_t INVALID_PAGE_ID = ObTmpFileGlobal::INVALID_PAGE_ID; + flush_end_page_id = INVALID_PAGE_ID; + if (file_size_ % ObTmpFileGlobal::PAGE_SIZE == 0) { + flush_end_page_id = INVALID_PAGE_ID; + } else { // determined whether to flush last page based on flag if last page is not full + flush_end_page_id = need_flush_tail ? INVALID_PAGE_ID : end_page_id_; + } + + LOG_DEBUG("get_flush_end_page_id_", K(fd_), K(need_flush_tail), K(flush_end_page_id), KPC(this)); + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_shared_nothing_tmp_file.h b/src/storage/tmp_file/ob_shared_nothing_tmp_file.h new file mode 100644 index 0000000000..03d05e1b93 --- /dev/null +++ b/src/storage/tmp_file/ob_shared_nothing_tmp_file.h @@ -0,0 +1,364 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_SHARE_NOTHING_TMP_FILE_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_SHARE_NOTHING_TMP_FILE_H_ + +#include "storage/tmp_file/ob_tmp_file_cache.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/blocksstable/ob_macro_block_id.h" +#include "lib/lock/ob_spin_rwlock.h" +#include "lib/lock/ob_spin_lock.h" +#include "lib/lock/ob_tc_rwlock.h" +#include "storage/tmp_file/ob_tmp_file_meta_tree.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileIOCtx; +class ObTenantTmpFileManager; +class ObTmpWriteBufferPool; +class ObTmpFileBlockPageBitmap; +class ObTmpFileBlockManager; +class ObTmpFileEvictionManager; +class ObTmpFileFlushPriorityManager; +class ObTmpFileFlushInfo; +class ObTmpFileFlushTask; +class ObTmpFilePageCacheController; +class ObTmpFileFlushManager; +class ObTmpFileTreeFlushContext; +class ObTmpFileDataFlushContext; + +class ObSharedNothingTmpFile final +{ +public: + struct ObTmpFileNode : public ObDLinkBase + { + ObTmpFileNode(ObSharedNothingTmpFile &file) : file_(file) {} + ObSharedNothingTmpFile &file_; + }; + + struct InnerFlushInfo + { + public: + InnerFlushInfo(); + ~InnerFlushInfo() { reset(); } + void reset(); + bool has_data() const { return flush_data_page_num_ > 0; } + bool has_meta() const { return !flush_meta_page_array_.empty(); } + int init_by_tmp_file_flush_info(const ObTmpFileFlushInfo& flush_info); + TO_STRING_KV(K(flush_data_page_disk_begin_id_), K(flush_data_page_num_), K(flush_virtual_page_id_), K(file_size_), + K(flush_meta_page_array_)); + public: + bool update_meta_data_done_; + // information for updating data + int64_t flush_data_page_disk_begin_id_; // record begin page id in the macro block,for updating meta tree item + int64_t flush_data_page_num_; + int64_t flush_virtual_page_id_; // record virtual_page_id while copying data, pass to meta tree while inserting items + int64_t file_size_; // if file_size > 0, it means the last page is in flushing + // information for updating meta tree + ObArray flush_meta_page_array_; + }; + + struct InnerFlushContext + { + InnerFlushContext() + : flush_seq_(ObTmpFileGlobal::INVALID_FLUSH_SEQUENCE), + data_finished_continuous_flush_info_num_(0), + meta_finished_continuous_flush_info_num_(0), + data_flush_infos_(), + meta_flush_infos_() + { + data_flush_infos_.set_attr(ObMemAttr(MTL_ID(), "TmpFileFInfo")); + meta_flush_infos_.set_attr(ObMemAttr(MTL_ID(), "TmpFileFInfo")); + } + void reset(); + int update_finished_continuous_flush_info_num(const bool is_meta, const int64_t end_pos); + bool is_all_finished() const + { + return data_flush_infos_.size() == data_finished_continuous_flush_info_num_ && + meta_flush_infos_.size() == meta_finished_continuous_flush_info_num_; + } + bool is_flushing() const + { + return !data_flush_infos_.empty() || !meta_flush_infos_.empty(); + } + TO_STRING_KV(K(flush_seq_), K(data_flush_infos_.size()), K(meta_flush_infos_.size()), + K(data_finished_continuous_flush_info_num_), + K(meta_finished_continuous_flush_info_num_)); + int64_t flush_seq_; + int64_t data_finished_continuous_flush_info_num_; + int64_t meta_finished_continuous_flush_info_num_; + ObArray data_flush_infos_; + ObArray meta_flush_infos_; + }; +public: + ObSharedNothingTmpFile(); + ~ObSharedNothingTmpFile(); + int init(const uint64_t tenant_id, const int64_t fd, const int64_t dir_id, + ObTmpFileBlockManager *block_manager, + ObIAllocator *callback_allocator, + ObIAllocator *wbp_index_cache_allocator, + ObIAllocator *wbp_index_cache_bkt_allocator, + ObTmpFilePageCacheController *page_cache_controller); + int destroy(); + void reset(); + bool can_remove(); + bool is_deleting(); + int delete_file(); + +// XXX Currently, K(tmp_file) is used to print the ObSharedNothingTmpFile structure without holding +// the file lock. Before adding the print field, make sure it is thread-safe. + TO_STRING_KV(K(is_inited_), K(is_deleting_), + K(tenant_id_), K(dir_id_), K(fd_), + K(ref_cnt_), K(truncated_offset_), K(read_offset_), + K(file_size_), K(flushed_data_page_num_), K(write_back_data_page_num_), + K(cached_page_nums_), + K(begin_page_id_), K(begin_page_virtual_id_), + K(flushed_page_id_), K(flushed_page_virtual_id_), + K(end_page_id_), + K(data_page_flush_level_), K(meta_page_flush_level_), + KP(data_flush_node_.get_next()), + KP(meta_flush_node_.get_next()), + K(is_in_data_eviction_list_), K(is_in_meta_eviction_list_), + KP(data_eviction_node_.get_next()), + KP(meta_eviction_node_.get_next())); +// XXX Currently, K(tmp_file) is used to print the ObSharedNothingTmpFile structure without holding +// the file lock. Before adding the print field, make sure it is thread-safe. + +public: + int aio_pread(ObTmpFileIOCtx &io_ctx); + int aio_write(ObTmpFileIOCtx &io_ctx); + + int evict_data_pages(const int64_t expected_evict_page_num, + int64_t &actual_evict_page_num, + int64_t &remain_flushed_page_num); + int evict_meta_pages(const int64_t expected_evict_page_num, + int64_t &actual_evict_page_num); + // truncate offset is open interval + int truncate(const int64_t truncate_offset); + +public: + int update_meta_after_flush(const int64_t info_idx, const bool is_meta, bool &reset_ctx); + int generate_data_flush_info(ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &info, + ObTmpFileDataFlushContext &data_flush_context, + const int64_t flush_sequence, + const bool need_flush_tail); + int generate_meta_flush_info(ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &info, + ObTmpFileTreeFlushContext &meta_flush_context, + const int64_t flush_sequence, + const bool need_flush_tail); + int insert_meta_tree_item(const ObTmpFileFlushInfo &info, int64_t block_index); +public: + int remove_flush_node(const bool is_meta); + int reinsert_flush_node(const bool is_meta); + int insert_or_update_data_flush_node_(); + int insert_or_update_meta_flush_node_(); + bool is_in_meta_flush_list(); + bool is_flushing(); + OB_INLINE ObTmpFileNode &get_data_flush_node() { return data_flush_node_; } + OB_INLINE ObTmpFileNode &get_meta_flush_node() { return meta_flush_node_; } + OB_INLINE ObTmpFileNode &get_data_eviction_node() { return data_eviction_node_; } + OB_INLINE ObTmpFileNode &get_meta_eviction_node() { return meta_eviction_node_; } + OB_INLINE const ObTmpFileNode &get_data_eviction_node() const { return data_eviction_node_; } + OB_INLINE const ObTmpFileNode &get_meta_eviction_node() const { return meta_eviction_node_; } + + OB_INLINE int64_t get_fd() const { return fd_; } + OB_INLINE int64_t get_dir_id() const { return dir_id_; } + OB_INLINE void inc_ref_cnt() { ATOMIC_INC(&ref_cnt_); } + OB_INLINE void dec_ref_cnt() { ATOMIC_AAF(&ref_cnt_, -1); } + OB_INLINE int64_t get_ref_cnt() const { return ATOMIC_LOAD(&ref_cnt_); } + void update_read_offset(int64_t read_offset); + int64_t get_file_size(); + + OB_INLINE void set_data_page_flush_level(int64_t data_page_flush_level) + { + data_page_flush_level_ = data_page_flush_level; + } + OB_INLINE int64_t get_data_page_flush_level() const { return data_page_flush_level_; } + OB_INLINE void set_meta_page_flush_level(int64_t meta_page_flush_level) + { + meta_page_flush_level_ = meta_page_flush_level; + } + OB_INLINE int64_t get_meta_page_flush_level() const { return meta_page_flush_level_; } + int64_t get_dirty_data_page_size() const; + int64_t get_dirty_data_page_size_with_lock(); + void get_dirty_meta_page_num(int64_t &non_rightmost_dirty_page_num, int64_t &rightmost_dirty_page_num) const; + void get_dirty_meta_page_num_with_lock(int64_t &non_rightmost_dirty_page_num, int64_t &rightmost_dirty_page_num); + int64_t cal_wbp_begin_offset(); + +private: + int inner_read_truncated_part_(ObTmpFileIOCtx &io_ctx); + int inner_read_from_wbp_(ObTmpFileIOCtx &io_ctx); + int inner_read_from_disk_(const int64_t expected_read_disk_size, ObTmpFileIOCtx &io_ctx); + int inner_seq_read_from_block_(const int64_t block_index, + const int64_t begin_read_offset_in_block, const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx, int64_t &actual_read_size); + int inner_rand_read_from_block_(const int64_t block_index, + const int64_t begin_read_offset_in_block, const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx, int64_t &actual_read_size); + int collect_pages_in_block_(const int64_t block_index, + const int64_t begin_page_idx_in_block, + const int64_t end_page_idx_in_block, + ObTmpFileBlockPageBitmap &bitmap, + ObIArray &page_value_handles); + int inner_read_continuous_cached_pages_(const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + const ObArray &page_value_handles, + const int64_t start_array_idx, + ObTmpFileIOCtx &io_ctx); + int inner_read_continuous_uncached_pages_(const int64_t block_index, + const int64_t begin_read_offset_in_block, + const int64_t end_read_offset_in_block, + ObTmpFileIOCtx &io_ctx); + int inner_truncate_(const int64_t truncate_offset, const int64_t wbp_begin_offset); +private: + int inner_write_(ObTmpFileIOCtx &io_ctx); + int inner_fill_tail_page_(ObTmpFileIOCtx &io_ctx); + int load_disk_tail_page_and_rewrite_(ObTmpFileIOCtx &io_ctx); + int append_write_memory_tail_page_(ObTmpFileIOCtx &io_ctx); + int inner_write_continuous_pages_(ObTmpFileIOCtx &io_ctx); + int alloc_and_write_pages_(const ObTmpFileIOCtx &io_ctx, + ObIArray &alloced_page_id, + int64_t &actual_write_size); + int truncate_the_first_wbp_page_(); + + int copy_flush_data_from_wbp_(ObTmpFileFlushTask &flush_task, ObTmpFileFlushInfo &info, + ObTmpFileDataFlushContext &data_flush_context, + const uint32_t copy_begin_page_id, const int64_t copy_begin_page_virtual_id, + const uint32_t copy_end_page_id, const int64_t flush_sequence, const bool need_flush_tail); + int cal_end_position_(ObArray &flush_infos, + const int64_t start_pos, + int64_t &end_pos, + int64_t &flushed_data_page_num); + int cal_next_flush_page_id_from_flush_ctx_or_file_(const ObTmpFileDataFlushContext &data_flush_context, + uint32_t &next_flush_page_id, + int64_t &next_flush_page_virtual_id); + int get_next_flush_page_id_(uint32_t& next_flush_page_id, int64_t& next_flush_page_virtual_id) const; + int get_flush_end_page_id_(uint32_t& flush_end_page_id, const bool need_flush_tail) const; + int get_physical_page_id_in_wbp(const int64_t virtual_page_id, uint32_t& page_id) const ; + int remove_useless_page_in_data_flush_infos_(const int64_t start_pos, + const int64_t end_pos, + const int64_t flushed_data_page_num, + int64_t &new_start_pos, + int64_t &new_flushed_data_page_num); + int update_file_meta_after_flush_(const int64_t start_pos, + const int64_t end_pos, + const int64_t flushed_data_page_num); + int update_meta_tree_after_flush_(const int64_t start_pos, const int64_t end_pos); +private: + int reinsert_flush_node_(const bool is_meta); + OB_INLINE bool has_unfinished_page_() const { return file_size_ % ObTmpFileGlobal::PAGE_SIZE != 0; } + + int64_t cal_wbp_begin_offset_() const; + OB_INLINE int64_t get_page_begin_offset_by_file_or_block_offset_(const int64_t offset) const + { + return common::lower_align(offset, ObTmpFileGlobal::PAGE_SIZE); + } + OB_INLINE int64_t get_page_end_offset_by_file_or_block_offset_(const int64_t offset) const + { + return common::upper_align(offset, ObTmpFileGlobal::PAGE_SIZE); + } + OB_INLINE int64_t get_page_begin_offset_by_virtual_id_(const int64_t virtual_page_id) const + { + return virtual_page_id * ObTmpFileGlobal::PAGE_SIZE; + } + OB_INLINE int64_t get_page_end_offset_by_virtual_id_(const int64_t virtual_page_id) const + { + return (virtual_page_id + 1) * ObTmpFileGlobal::PAGE_SIZE; + } + OB_INLINE int64_t get_page_virtual_id_from_offset_(const int64_t page_offset_in_file, const bool is_open_interval) const + { + return is_open_interval ? + common::upper_align(page_offset_in_file, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE - 1 : + page_offset_in_file / ObTmpFileGlobal::PAGE_SIZE; + } + OB_INLINE int64_t get_page_id_in_block_(const int64_t page_offset_in_block) const + { + return page_offset_in_block / ObTmpFileGlobal::PAGE_SIZE; + } + OB_INLINE int64_t get_page_offset_from_file_or_block_offset_(const int64_t offset) const + { + return offset % ObTmpFileGlobal::PAGE_SIZE; + } + +private: + ObTmpFileBlockManager *tmp_file_block_manager_; + ObIAllocator *callback_allocator_; + ObTmpFilePageCacheController *page_cache_controller_; + ObTmpFileFlushPriorityManager *flush_prio_mgr_; + ObTmpFileEvictionManager *eviction_mgr_; + ObTmpWriteBufferPool *wbp_; + ObTmpFileWBPIndexCache page_idx_cache_; + bool is_inited_; + bool is_deleting_; + bool is_in_data_eviction_list_; + bool is_in_meta_eviction_list_; + int64_t data_page_flush_level_; + int64_t meta_page_flush_level_; + uint64_t tenant_id_; + int64_t dir_id_; + int64_t fd_; + int64_t ref_cnt_; + int64_t truncated_offset_; // read data befor truncated_offset will be set as 0 + int64_t read_offset_; // read offset is on the entire file + int64_t file_size_; // has written size of this file + int64_t flushed_data_page_num_; // equal to the page num between [begin_page_id_, flushed_page_id_] in wbp + int64_t write_back_data_page_num_; + int64_t cached_page_nums_; // page nums in write buffer pool + uint32_t begin_page_id_; // the first page index in write buffer pool + int64_t begin_page_virtual_id_; + uint32_t flushed_page_id_; // the last flushed page index in write buffer pool. + // specifically, if flushed_page_id_ == end_page_id_, + // it means the last page has been flushed. However, + // the last page might be appending written after flushing. + // thus, in this case, whether "flushed_page_id_" page in wbp is flushed + // needs to be checked with page status (with some get functions). + int64_t flushed_page_virtual_id_; + uint32_t end_page_id_; // the last page index in write buffer pool + ObSharedNothingTmpFileMetaTree meta_tree_; + ObTmpFileNode data_flush_node_; + ObTmpFileNode meta_flush_node_; + ObTmpFileNode data_eviction_node_; + ObTmpFileNode meta_eviction_node_; + common::TCRWLock meta_lock_; // handle conflicts between writing and reading meta tree and meta data of file + ObSpinLock last_page_lock_; // handle conflicts between writing and evicting for last page + ObSpinLock multi_write_lock_; // handle conflicts between multiple writes + SpinRWLock truncate_lock_; // handle conflicts between truncate and flushing + InnerFlushContext inner_flush_ctx_; // file-level flush context +}; + +class ObTmpFileHandle final +{ +public: + ObTmpFileHandle() : ptr_(nullptr) {} + ObTmpFileHandle(ObSharedNothingTmpFile *tmp_file); + ObTmpFileHandle(const ObTmpFileHandle &handle); + ObTmpFileHandle & operator=(const ObTmpFileHandle &other); + ~ObTmpFileHandle() { reset(); } + OB_INLINE ObSharedNothingTmpFile * get() const {return ptr_; } + bool is_inited() { return nullptr != ptr_; } + void reset(); + int init(ObSharedNothingTmpFile *tmp_file); + TO_STRING_KV(KP(ptr_)); +private: + ObSharedNothingTmpFile *ptr_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase + +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_SHARE_NOTHING_TMP_FILE_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_block_manager.cpp b/src/storage/tmp_file/ob_tmp_file_block_manager.cpp new file mode 100644 index 0000000000..7fe4d4ada3 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_block_manager.cpp @@ -0,0 +1,922 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_block_manager.h" +namespace oceanbase +{ +using namespace storage; +using namespace share; + +namespace tmp_file +{ +//-----------------------ObTmpFileBlockPageBitmapIterator----------------------- +int ObTmpFileBlockPageBitmapIterator::init(const ObTmpFileBlockPageBitmap *bitmap, + const int64_t start_idx, int64_t end_idx) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_ISNULL(bitmap) || + OB_UNLIKELY(start_idx < 0 || end_idx < 0 || start_idx > end_idx || + end_idx >= ObTmpFileBlockPageBitmap::get_capacity())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KPC(bitmap), K(start_idx), K(end_idx)); + } else { + is_inited_ = true; + bitmap_ = bitmap; + cur_idx_ = start_idx; + end_idx_ = end_idx; + } + return ret; +} + +void ObTmpFileBlockPageBitmapIterator::reset() +{ + bitmap_ = nullptr; + is_inited_ = false; + cur_idx_ = OB_INVALID_INDEX; + end_idx_ = OB_INVALID_INDEX; +} + +int ObTmpFileBlockPageBitmapIterator::next_range(bool &value, int64_t &start_page_id, int64_t &end_page_id) +{ + int ret = OB_SUCCESS; + start_page_id = OB_INVALID_INDEX; + end_page_id = OB_INVALID_INDEX; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (has_next()) { + start_page_id = cur_idx_; + if (OB_FAIL(bitmap_->get_value(cur_idx_++, value))) { + LOG_WARN("fail to get value", KR(ret), K(cur_idx_)); + } else { + while(cur_idx_ <= end_idx_ && OB_SUCC(ret)) { + bool cur_value = false; + if (OB_FAIL(bitmap_->get_value(cur_idx_, cur_value))) { + LOG_WARN("fail to get value", KR(ret), K(cur_idx_)); + } else if (cur_value != value) { + break; + } else { + cur_idx_ += 1; + } + } + end_page_id = cur_idx_ - 1; + } + } else { + ret = OB_ITER_END; + LOG_WARN("iter end", KR(ret)); + } + return ret; +} + +//---------------------------ObTmpFileBlockPageBitmap--------------------------- +int ObTmpFileBlockPageBitmap::get_value(const int64_t offset, bool &value) const +{ + int ret = OB_SUCCESS; + const int64_t capacity = PAGE_CAPACITY; + + if (OB_UNLIKELY(capacity <= offset || offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument", KR(ret), K(capacity), K(offset)); + } else { + value = (bitmap_[offset / 8] & (1 << (offset % 8))) != 0; + } + return ret; +} + +int ObTmpFileBlockPageBitmap::set_bitmap(const int64_t offset, const bool value) +{ + int ret = OB_SUCCESS; + const int64_t capacity = PAGE_CAPACITY; + if (OB_UNLIKELY(capacity <= offset || offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument", KR(ret), K(capacity), K(offset)); + } else if (value) { + bitmap_[offset / 8] |= (1 <<(offset % 8)); + } else { + bitmap_[offset / 8] &= ~(1 <<(offset % 8)); + } + return ret; +} + +int ObTmpFileBlockPageBitmap::set_bitmap_batch(const int64_t offset, const int64_t count, const bool value) +{ + int ret = OB_SUCCESS; + const int64_t capacity = PAGE_CAPACITY; + uint8_t start_byte_pos = offset / 8; + uint8_t end_byte_pos = (offset + count - 1) / 8; + uint8_t start_bit_pos = offset % 8; + uint8_t end_bit_pos = (offset + count - 1) % 8; + if (OB_UNLIKELY(capacity < (offset + count) || count <= 0 || offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument", KR(ret), K(capacity), K(offset), K(count)); + } else if (start_byte_pos == end_byte_pos) { + // only one byte + uint8_t mask = ((1 << (end_bit_pos + 1)) - 1) & ~((1 << start_bit_pos) - 1); + if (value) { + bitmap_[start_byte_pos] |= mask; + } else { + bitmap_[start_byte_pos] &= ~mask; + } + } else { + uint8_t start_mask = ~((1 << start_bit_pos) - 1); + uint8_t end_mask = (1 << (end_bit_pos + 1)) - 1; + if (value) { + bitmap_[start_byte_pos] |= start_mask; + bitmap_[end_byte_pos] |= end_mask; + } else { + bitmap_[start_byte_pos] &= ~start_mask; + bitmap_[end_byte_pos] &= ~end_mask; + } + for (uint8_t i = start_byte_pos + 1; i < end_byte_pos; ++i) { + bitmap_[i] = value ? (1 << 8) - 1 : 0; + } + } + + return ret; +} + +int ObTmpFileBlockPageBitmap::is_all_true(const int64_t start, const int64_t end, bool &is_all_true) const +{ + int ret = OB_SUCCESS; + const int64_t capacity = PAGE_CAPACITY; + const uint8_t start_byte_pos = start / 8; + const uint8_t end_byte_pos = end / 8; + const uint8_t start_bit_pos = start % 8; + const uint8_t end_bit_pos = end % 8; + is_all_true = false; + + if (OB_UNLIKELY(start > end || start < 0 || end >= capacity)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument", KR(ret), K(capacity), K(start), K(end)); + } else if (start_byte_pos == end_byte_pos) { + // only one byte + uint8_t mask = ((1 << (end_bit_pos + 1)) - 1) & ~((1 << start_bit_pos) - 1); + is_all_true = (bitmap_[start_byte_pos] & mask) == mask; + } else { + uint8_t start_mask = ~((1 << start_bit_pos) - 1); + uint8_t end_mask = (1 << (end_bit_pos + 1)) - 1; + + if ((bitmap_[start_byte_pos] & start_mask) != start_mask || + (bitmap_[end_byte_pos] & end_mask) != end_mask) { + is_all_true = false; + } else { + is_all_true = true; + for (int64_t i = start_byte_pos + 1; is_all_true && i < end_byte_pos; ++i) { + if (bitmap_[i] != 0XFF) { + is_all_true = false; + } + } + } + } + + return ret; +} + +int ObTmpFileBlockPageBitmap::is_all_false(const int64_t start, const int64_t end, bool &is_all_false) const +{ + int ret = OB_SUCCESS; + const int64_t capacity = PAGE_CAPACITY; + const uint8_t start_byte_pos = start / 8; + const uint8_t end_byte_pos = end / 8; + const uint8_t start_bit_pos = start % 8; + const uint8_t end_bit_pos = end % 8; + is_all_false = false; + + if (OB_UNLIKELY(start > end || start < 0 || end >= capacity)) { + is_all_false = false; + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument", KR(ret), K(capacity), K(start), K(end)); + } else if (start_byte_pos == end_byte_pos) { + // only one byte + uint8_t mask = ((1 << (end_bit_pos + 1)) - 1) & ~((1 << start_bit_pos) - 1); + is_all_false = (bitmap_[start_byte_pos] & mask) == 0; + } else { + uint8_t start_mask = ~((1 << start_bit_pos) - 1); + uint8_t end_mask = (1 << (end_bit_pos + 1)) - 1; + + if ((bitmap_[start_byte_pos] & start_mask) != 0 || + (bitmap_[end_byte_pos] & end_mask) != 0) { + is_all_false = false; + } else { + is_all_false = true; + for (int64_t i = start_byte_pos + 1; is_all_false && i < end_byte_pos; ++i) { + if (bitmap_[i] != 0) { + is_all_false = false; + } + } + } + } + + return ret; +} + +int ObTmpFileBlockPageBitmap::is_all_true(bool &b_ret) const +{ + return is_all_true(0, PAGE_CAPACITY - 1, b_ret); +} + +int ObTmpFileBlockPageBitmap::is_all_false(bool &b_ret) const +{ + return is_all_false(0, PAGE_CAPACITY - 1, b_ret); +} + +int64_t ObTmpFileBlockPageBitmap::to_string(char* buf, const int64_t buf_len) const +{ + int64_t pos = 0; + J_OBJ_START(); + for (int i = 0; i < PAGE_BYTE_CAPACITY / sizeof(uint64_t); ++i) { + int64_t offset = i * sizeof(uint64_t); + if (i > 0) { + J_COMMA(); + } + char bitmap_id[11]; + snprintf(bitmap_id, sizeof(bitmap_id), "bitmap_[%d]", i); + J_KV(bitmap_id, *(reinterpret_cast(bitmap_ + offset))); + } + J_OBJ_END(); + return pos; +} +//-----------------------ObTmpFileBlock----------------------- +ObTmpFileBlock::~ObTmpFileBlock() +{ + reset(); +} + +int ObTmpFileBlock::reset() +{ + int ret = OB_SUCCESS; + if (is_valid() && BlockState::ON_DISK == block_state_ && OB_FAIL(OB_SERVER_BLOCK_MGR.dec_ref(macro_block_id_))) { + LOG_ERROR("failed to dec macro block ref cnt", KR(ret), KPC(this)); + } else { + block_state_ = BlockState::INVALID; + page_bitmap_.reset(); + block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + macro_block_id_.reset(); + ref_cnt_ = 0; + } + return ret; +} + +int ObTmpFileBlock::init_block(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + bool use_released_pages = false; + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX == block_index)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid block_index", KR(ret), K(block_index), K(begin_page_id), K(page_num)); + } else if (OB_UNLIKELY(page_num <= 0 || begin_page_id < 0 || + begin_page_id + page_num > ObTmpFileGlobal::BLOCK_PAGE_NUMS)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(block_index), K(begin_page_id), K(page_num)); + } else if (OB_UNLIKELY(block_index_ != ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block has been inited", KR(ret), K(block_index_)); + } else if (block_state_ != BlockState::INVALID) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("attempt to reinit a block not in invalid state", KR(ret), KPC(this)); + } else if (OB_FAIL(page_bitmap_.is_all_false(begin_page_id, begin_page_id + page_num - 1, use_released_pages))) { + LOG_WARN("fail to check whether the pages are all false", KR(ret), K(begin_page_id), K(page_num), K(page_bitmap_)); + } else if (OB_UNLIKELY(!use_released_pages)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to use a allocated page", KR(ret), K(block_index), K(begin_page_id), K(page_num), + K(page_bitmap_), K(use_released_pages)); + } else if (OB_FAIL(page_bitmap_.set_bitmap_batch(begin_page_id, page_num, true))) { + LOG_WARN("fail to set bitmap", KR(ret), K(begin_page_id), K(page_num), K(page_bitmap_)); + } else { + block_index_ = block_index; + block_state_ = BlockState::IN_MEMORY; + LOG_DEBUG("init block successfully", K(block_index), K(begin_page_id), K(page_num), KPC(this)); + } + return ret; +} + +int ObTmpFileBlock::write_back_start() +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (block_state_ != BlockState::IN_MEMORY && block_state_ != BlockState::WRITE_BACK) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("write back a block not in in_memory state and not in write back state", + KR(ret), KPC(this)); + } else { + block_state_ = BlockState::WRITE_BACK; + LOG_DEBUG("switch tmp file block to write back state", KPC(this)); + } + return ret; +} + +int ObTmpFileBlock::write_back_failed() +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (block_state_ != BlockState::WRITE_BACK) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("notify a block write_back_failed, but the block not in write back state", + KR(ret), KPC(this)); + } else { + block_state_ = BlockState::IN_MEMORY; + LOG_DEBUG("switch tmp file block to in memory state", KPC(this)); + } + return ret; +} + +int ObTmpFileBlock::write_back_succ(blocksstable::MacroBlockId macro_block_id) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (OB_UNLIKELY(!macro_block_id.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(macro_block_id)); + } else if (block_state_ != BlockState::WRITE_BACK) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("attempt to set macro_block_id for a block not in write_back state", KR(ret), KPC(this)); + } else if (OB_FAIL(OB_SERVER_BLOCK_MGR.inc_ref(macro_block_id))) { + LOG_ERROR("failed to dec macro block ref cnt", KR(ret), K(macro_block_id), KPC(this)); + } else { + macro_block_id_ = macro_block_id; + block_state_ = BlockState::ON_DISK; + LOG_DEBUG("switch tmp file block to on disk state", KPC(this)); + } + return ret; +} + +int ObTmpFileBlock::release_pages(const int64_t begin_page_id, const int64_t page_num) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + bool release_allocated_page = false; + if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is invalid", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(page_num <= 0 || begin_page_id < 0 || + begin_page_id + page_num > ObTmpFileGlobal::BLOCK_PAGE_NUMS)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(block_index_), K(begin_page_id), K(page_num)); + } else if (OB_FAIL(page_bitmap_.is_all_true(begin_page_id, begin_page_id + page_num - 1, release_allocated_page))) { + LOG_WARN("fail to check whether the pages are all true", KR(ret), K(block_index_), K(macro_block_id_), + K(begin_page_id), K(page_num), K(page_bitmap_)); + } else if (OB_UNLIKELY(!release_allocated_page)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to release a released page", KR(ret), K(block_index_), K(macro_block_id_), + K(begin_page_id), K(page_num), + K(page_bitmap_), K(release_allocated_page)); + } else if (OB_FAIL(page_bitmap_.set_bitmap_batch(begin_page_id, page_num, false))) { + LOG_WARN("fail to set bitmap", KR(ret), K(begin_page_id), K(page_num), K(page_bitmap_)); + } + return ret; +} + +int ObTmpFileBlock::get_page_usage(int64_t &page_num) const +{ + int ret = OB_SUCCESS; + SpinRLockGuard guard(lock_); + page_num = 0; + if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is invalid", KR(ret), KPC(this)); + } else { + for (int i = 0; OB_SUCC(ret) && i < ObTmpFileBlockPageBitmap::get_capacity(); ++i) { + bool value = false; + if (OB_FAIL(page_bitmap_.get_value(i, value))) { + LOG_WARN("fail to get value from bitmap", KR(ret), K(i), KPC(this)); + } else if (value) { + page_num++; + } + } // end for + } + return ret; +} + +int ObTmpFileBlock::inc_ref_cnt() +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + + if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is invalid", KR(ret), KPC(this)); + } else { + ref_cnt_++; + } + + return ret; +} + +int ObTmpFileBlock::dec_ref_cnt(int64_t &ref_cnt) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + + if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is invalid", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(ref_cnt_ <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid ref cnt", KR(ret), KPC(this)); + } else { + ref_cnt_ -= 1; + ref_cnt = ref_cnt_; + } + + return ret; +} + +bool ObTmpFileBlock::on_disk() const +{ + return BlockState::ON_DISK == block_state_; +} + +int ObTmpFileBlock::can_remove(bool &can_remove) const +{ + int ret = OB_SUCCESS; + SpinRLockGuard guard(lock_); + + can_remove = false; + bool is_all_page_released = false; + if (OB_FAIL(page_bitmap_.is_all_false(is_all_page_released))) { + LOG_WARN("fail to check whether the pages are all false", KR(ret), KPC(this)); + } else { + can_remove = is_all_page_released && BlockState::WRITE_BACK != block_state_; + } + return ret; +} + +blocksstable::MacroBlockId ObTmpFileBlock::get_macro_block_id() const +{ + SpinRLockGuard guard(lock_); + return macro_block_id_; +} + +int64_t ObTmpFileBlock::get_block_index() const +{ + SpinRLockGuard guard(lock_); + return block_index_; +} +//-----------------------ObTmpFileBlockHandle----------------------- + +ObTmpFileBlockHandle::ObTmpFileBlockHandle(ObTmpFileBlock *block, ObTmpFileBlockManager *tmp_file_blk_mgr) + : ptr_(nullptr), tmp_file_blk_mgr_(nullptr) +{ + if (nullptr != block && nullptr != tmp_file_blk_mgr) { + ptr_ = block; + tmp_file_blk_mgr_ = tmp_file_blk_mgr; + ptr_->inc_ref_cnt(); + } +} + +ObTmpFileBlockHandle::ObTmpFileBlockHandle(const ObTmpFileBlockHandle &handle) + : ptr_(nullptr), tmp_file_blk_mgr_(nullptr) +{ + operator=(handle); +} + +ObTmpFileBlockHandle & ObTmpFileBlockHandle::operator=(const ObTmpFileBlockHandle &other) +{ + if (other.ptr_ != ptr_) { + reset(); + ptr_ = other.ptr_; + tmp_file_blk_mgr_ = other.tmp_file_blk_mgr_; + if (is_inited()) { + ptr_->inc_ref_cnt(); + } + } + return *this; +} + +void ObTmpFileBlockHandle::reset() +{ + int ret = OB_SUCCESS; + if (is_inited()) { + int64_t cur_ref_cnt = -1; + int64_t block_index = ptr_->get_block_index(); + if (OB_FAIL(ptr_->dec_ref_cnt(cur_ref_cnt))) { + LOG_ERROR("fail to dec block ref cnt", KR(ret), KPC(ptr_)); + } else if (0 == cur_ref_cnt) { + ptr_->reset(); + tmp_file_blk_mgr_->get_block_allocator().free(ptr_); + } + ptr_ = nullptr; + } +} + +int ObTmpFileBlockHandle::init(ObTmpFileBlock *block, ObTmpFileBlockManager *tmp_file_blk_mgr) +{ + int ret = OB_SUCCESS; + + if (is_inited()) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret), KP(ptr_)); + } else if (OB_ISNULL(block) || OB_ISNULL(tmp_file_blk_mgr)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(block), KP(tmp_file_blk_mgr)); + } else if (OB_FAIL(block->inc_ref_cnt())) { + LOG_WARN("fail to inc block ref cnt", KR(ret), KPC(block)); + } else { + ptr_ = block; + tmp_file_blk_mgr_ = tmp_file_blk_mgr; + } + + return ret; +} + +//-----------------------ObTmpFileBlockManager----------------------- +ObTmpFileBlockManager::ObTmpFileBlockManager() : + is_inited_(false), + tenant_id_(OB_INVALID_TENANT_ID), + used_page_num_(0), + physical_block_num_(0), + block_index_generator_(0), + block_map_(), + block_allocator_(), + stat_lock_(), + map_lock_() + {} + +ObTmpFileBlockManager::~ObTmpFileBlockManager() +{ + destroy(); +} + +void ObTmpFileBlockManager::destroy() +{ + is_inited_ = false; + tenant_id_ = OB_INVALID_TENANT_ID; + used_page_num_ = 0; + physical_block_num_ = 0; + block_index_generator_ = 0; + block_map_.destroy(); + block_allocator_.reset(); +} + +int ObTmpFileBlockManager::init(const uint64_t tenant_id, const int64_t block_mem_limit) +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileBlockManager init twice", KR(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments", KR(ret), K(tenant_id)); + } else if (OB_FAIL(block_map_.init("TmpFileBlkMgr", tenant_id))) { + LOG_WARN("fail to init tenant temporary file block manager", KR(ret), K(tenant_id)); + } else if (OB_FAIL(block_allocator_.init(common::OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObModIds::OB_TMP_BLOCK_MANAGER, tenant_id_, + block_mem_limit))) { + LOG_WARN("fail to init temporary file block allocator", KR(ret), K(tenant_id_), K(block_mem_limit)); + } else { + block_allocator_.set_attr(ObMemAttr(tenant_id, "TmpFileBlk")); + tenant_id_ = tenant_id; + is_inited_ = true; + } + + return ret; +} + +int ObTmpFileBlockManager::create_tmp_file_block(const int64_t begin_page_id, const int64_t page_num, + int64_t &block_index) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + ObTmpFileBlock* blk = nullptr; + const uint64_t blk_size = sizeof(ObTmpFileBlock); + void *buf = nullptr; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_ISNULL(buf = block_allocator_.alloc(blk_size, lib::ObMemAttr(tenant_id_, "TmpFileBlk")))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory for tmp file block", KR(ret), K(blk_size)); + } else if (FALSE_IT(blk = new (buf) ObTmpFileBlock())) { + } else if (FALSE_IT(block_index = ATOMIC_AAF(&block_index_generator_, 1))) { + } else if (OB_FAIL(blk->init_block(block_index, begin_page_id, page_num))) { + LOG_WARN("fail to init tmp file block", KR(ret), K(block_index)); + } else if (OB_FAIL(handle.init(blk, this))) { + LOG_WARN("fail to init tmp file block handle", KR(ret), K(block_index)); + } else { + SharedLockGuard guard(map_lock_); + if (OB_FAIL(block_map_.insert(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to insert tmp file block into map", KR(ret), K(block_index)); + } + LOG_DEBUG("create tmp file block succ", KR(ret), K(block_index)); + } + + if (OB_FAIL(ret) && OB_NOT_NULL(blk)) { + blk->~ObTmpFileBlock(); + block_allocator_.free(blk); + blk = nullptr; + } + + return ret; +} + +int ObTmpFileBlockManager::write_back_start(const int64_t block_index) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + ObTmpFileBlock *blk = nullptr; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(blk = handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is null", KR(ret), K(block_index)); + } else if (OB_FAIL(blk->write_back_start())) { + LOG_WARN("fail to write back start", KR(ret), K(block_index)); + } + + return ret; +} + +int ObTmpFileBlockManager::write_back_failed(const int64_t block_index) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + ObTmpFileBlock *blk = nullptr; + bool can_remove = false; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(blk = handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is null", KR(ret), K(block_index)); + } else if (OB_FAIL(blk->write_back_failed())) { + LOG_WARN("fail to notify block write_back_failed", KR(ret), K(handle)); + } + + if (FAILEDx(handle.get()->can_remove(can_remove))) { + LOG_WARN("check block can remove failed", KR(ret), K(handle)); + } else if (can_remove) { + if (OB_FAIL(remove_tmp_file_block_(block_index))) { + LOG_ERROR("fail to remove tmp file block", KR(ret), K(handle)); + } + } + return ret; +} + +int ObTmpFileBlockManager::write_back_succ(const int64_t block_index, const blocksstable::MacroBlockId macro_block_id) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + ObTmpFileBlock *blk = nullptr; + bool can_remove = false; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(blk = handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file block is null", KR(ret), K(block_index)); + } else if (OB_FAIL(blk->write_back_succ(macro_block_id))) { + LOG_WARN("fail to notify write_back_succ", KR(ret), K(block_index), K(macro_block_id), K(handle)); + } else { + SpinWLockGuard guard(stat_lock_); + int64_t used_page_num = 0; + if (OB_FAIL(blk->get_page_usage(used_page_num))) { + LOG_WARN("fail to get page usage", KR(ret), K(handle)); + } else { + used_page_num_ += used_page_num; + physical_block_num_ += 1; + } + } + + if (FAILEDx(handle.get()->can_remove(can_remove))) { + LOG_WARN("check block can remove failed", KR(ret), K(handle)); + } else if (can_remove) { + if (OB_FAIL(remove_tmp_file_block_(block_index))) { + LOG_ERROR("fail to remove tmp file block", KR(ret), K(handle)); + } + } + return ret; +} + +// if a block is released all pages in this function, +// when ObTmpFileBlockHandle destructed, the handle will remove itself from block_map +int ObTmpFileBlockManager::release_tmp_file_page(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + bool can_remove = false; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, block should not be null", KR(ret), K(block_index)); + } else if (OB_FAIL(handle.get()->release_pages(begin_page_id, page_num))) { + LOG_WARN("fail to release pages", KR(ret), K(begin_page_id), K(page_num), K(handle)); + } else if (handle.get()->on_disk()) { + SpinWLockGuard guard(stat_lock_); + used_page_num_ -= page_num; + } + + LOG_DEBUG("release_tmp_file_page", KR(ret), K(block_index), K(begin_page_id), K(page_num), K(handle)); + + if (FAILEDx(handle.get()->can_remove(can_remove))) { + LOG_WARN("check block can remove failed", KR(ret), K(handle)); + } else if (can_remove) { + if (OB_FAIL(remove_tmp_file_block_(block_index))) { + LOG_ERROR("fail to remove tmp file block", KR(ret), K(handle)); + } + } + + return ret; +} + +// only called by ObTmpFileBlockHandle::reset() +int ObTmpFileBlockManager::remove_tmp_file_block_(const int64_t block_index) +{ + int ret = OB_SUCCESS; + ObTmpFileBlock* blk = nullptr; + ObTmpFileBlockHandle handle; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else { + SharedLockGuard guard(map_lock_); + if (OB_FAIL(block_map_.erase(ObTmpFileBlockKey(block_index), handle))) { + if (ret != OB_ENTRY_NOT_EXIST) { + LOG_WARN("fail to erase tmp file block", KR(ret), K(block_index)); + } else { + LOG_DEBUG("erase tmp file block succ", KR(ret), K(block_index)); + ret = OB_SUCCESS; + } + } else if (OB_UNLIKELY(nullptr == (blk = handle.get()))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(block_index)); + } else { + LOG_DEBUG("erase tmp file block from map succ", KR(ret), K(handle)); + } + } + + if (OB_SUCC(ret) && OB_NOT_NULL(blk) && blk->on_disk()) { + SpinWLockGuard guard(stat_lock_); + int64_t used_page_num = 0; + if (OB_FAIL(blk->get_page_usage(used_page_num))) { + LOG_WARN("fail to get page usage", KR(ret), K(block_index), K(handle)); + } else { + used_page_num_ -= used_page_num; + physical_block_num_ -= 1; + } + } + + return ret; +} + +int ObTmpFileBlockManager::get_block_usage_stat(int64_t &used_page_num, int64_t ¯o_block_count) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else { + SpinRLockGuard guard(stat_lock_); + used_page_num = used_page_num_; + macro_block_count = physical_block_num_; + } + + return ret; +} + +void ObTmpFileBlockManager::print_block_usage() +{ + int ret = OB_SUCCESS; + int64_t used_page_num = 0; + int64_t block_num = 0; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(get_block_usage_stat(used_page_num, block_num))) { + LOG_WARN("fail to get block usage stat", KR(ret)); + } else if (OB_UNLIKELY(0 == block_num)) { + LOG_INFO("temporary file module use no blocks"); + } else { + int64_t occupied_page_num = block_num * ObTmpFileGlobal::BLOCK_PAGE_NUMS; + double disk_fragment_ratio = static_cast(used_page_num) / static_cast(occupied_page_num); + LOG_INFO("the block usage for temporary files", + K(used_page_num), K(occupied_page_num), K(disk_fragment_ratio)); + } +} + +int ObTmpFileBlockManager::get_macro_block_count(int64_t ¯o_block_count) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else { + SpinRLockGuard guard(stat_lock_); + macro_block_count = physical_block_num_; + } + return ret; +} + +int ObTmpFileBlockManager::get_macro_block_list(common::ObIArray ¯o_id_list) +{ + int ret = OB_SUCCESS; + ExclusiveLockGuard guard(map_lock_); + CollectMacroBlockIdFunctor func(macro_id_list); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.for_each(func))) { + LOG_WARN("fail to collect macro block ids", KR(ret)); + } + + return ret; +} + +bool ObTmpFileBlockManager::CollectMacroBlockIdFunctor::operator()(const ObTmpFileBlockKey &block_index, const ObTmpFileBlockHandle &handle) +{ + int ret = OB_SUCCESS; + + if (OB_ISNULL(handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("handle should not be null", KR(ret), K(block_index)); + } else if (!handle.get()->is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("handle should not be invalid", KR(ret), K(block_index), K(handle)); + } else if (!handle.get()->on_disk()) { + // do nothing + } else if (OB_FAIL(macro_id_list_.push_back(handle.get()->get_macro_block_id()))) { + LOG_WARN("failed to push back", KR(ret), K(block_index), K(handle)); + } + return OB_SUCCESS == ret; +} + +int ObTmpFileBlockManager::get_macro_block_id(const int64_t block_index, blocksstable::MacroBlockId ¯o_block_id) +{ + int ret = OB_SUCCESS; + ObTmpFileBlockHandle handle; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, block should not be null", KR(ret), K(block_index)); + } else if (OB_UNLIKELY(!handle.get()->is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, block should not be invalid", KR(ret), K(block_index), K(handle)); + } else { + macro_block_id = handle.get()->get_macro_block_id(); + } + + return ret; +} + +int ObTmpFileBlockManager::get_tmp_file_block_handle(const int64_t block_index, ObTmpFileBlockHandle &handle) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTmpFileBlockManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(block_map_.get(ObTmpFileBlockKey(block_index), handle))) { + LOG_WARN("fail to get tmp file block", KR(ret), K(block_index)); + } else if (OB_ISNULL(handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, block should not be null", KR(ret), K(block_index)); + } else if (OB_UNLIKELY(!handle.get()->is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, block should not be invalid", KR(ret), K(block_index), K(handle)); + } + + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_block_manager.h b/src/storage/tmp_file/ob_tmp_file_block_manager.h new file mode 100644 index 0000000000..593ec8f9a6 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_block_manager.h @@ -0,0 +1,221 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_BLOCK_MANAGER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_BLOCK_MANAGER_H_ + +#include "storage/blocksstable/ob_macro_block_id.h" +#include "storage/blocksstable/ob_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "lib/hash/ob_linear_hash_map.h" +#include "lib/lock/ob_spin_rwlock.h" +#include "lib/allocator/page_arena.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileBlockPageBitmap; +class ObTmpFileBlockManager; + +struct ObTmpFileBlockKey final +{ + explicit ObTmpFileBlockKey(const int64_t block_index) : block_index_(block_index) {} + OB_INLINE int hash(uint64_t &hash_val) const + { + hash_val = murmurhash(&block_index_, sizeof(int64_t), 0); + return OB_SUCCESS; + } + OB_INLINE bool operator==(const ObTmpFileBlockKey &other) const + { + return block_index_ == other.block_index_; + } + TO_STRING_KV(K(block_index_)); + int64_t block_index_; +}; + +// search every set of continuous existence or non-existence pages in the range of [start_idx, end_idx_] +class ObTmpFileBlockPageBitmapIterator +{ +public: + ObTmpFileBlockPageBitmapIterator() : + bitmap_(nullptr), is_inited_(false), cur_idx_(OB_INVALID_INDEX), end_idx_(OB_INVALID_INDEX) {} + ~ObTmpFileBlockPageBitmapIterator() { reset(); } + int init(const ObTmpFileBlockPageBitmap *bitmap, const int64_t start_idx, int64_t end_idx); + void reset(); + int next_range(bool &value, int64_t &start_page_id, int64_t &end_page_id); + OB_INLINE bool has_next() const { return is_inited_ && cur_idx_ <= end_idx_; } +private: + const ObTmpFileBlockPageBitmap *bitmap_; + bool is_inited_; + int64_t cur_idx_; + int64_t end_idx_; +}; + +class ObTmpFileBlockPageBitmap +{ +public: + ObTmpFileBlockPageBitmap() { reset(); } + ~ObTmpFileBlockPageBitmap() { reset(); } + OB_INLINE void reset() { MEMSET(bitmap_, 0, PAGE_CAPACITY / 8); } + int get_value(const int64_t offset, bool &value) const; + int set_bitmap(const int64_t offset, const bool value); + int set_bitmap_batch(const int64_t offset, const int64_t count, const bool value); + // [start, end] + int is_all_true(const int64_t start, const int64_t end, bool &is_all_true) const; + int is_all_false(const int64_t start, const int64_t end, bool &is_all_false) const; + int is_all_true(bool &b_ret) const; + int is_all_false(bool &b_ret) const; + OB_INLINE static int64_t get_capacity() { return PAGE_CAPACITY; } + int64_t to_string(char* buf, const int64_t buf_len) const; +private: + static constexpr int64_t PAGE_CAPACITY = ObTmpFileGlobal::BLOCK_PAGE_NUMS; + // upper_align(PAGE_CAPACITY/8, 8) + static constexpr int64_t PAGE_BYTE_CAPACITY = (PAGE_CAPACITY / 8 + 7) & ~7; + +private: + uint8_t bitmap_[PAGE_BYTE_CAPACITY]; +}; + +// ObTmpFileBlock records the usage of a macro block. +// each macro block could be referenced by one or more tmp files of a tenant. +class ObTmpFileBlock final +{ + enum BlockState { + INVALID = -1, + IN_MEMORY = 0, + WRITE_BACK = 1, + ON_DISK = 2, + }; +public: + ObTmpFileBlock(): block_state_(BlockState::INVALID), lock_(common::ObLatchIds::TMP_FILE_LOCK), + page_bitmap_(), block_index_(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX), + macro_block_id_(), ref_cnt_(0) {} + ~ObTmpFileBlock(); + + OB_INLINE bool is_valid() const { + bool b_ret = false; + + if (block_state_ < BlockState::INVALID || block_state_ > BlockState::ON_DISK) { + b_ret = false; + } else { + b_ret = block_index_ != ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + if (BlockState::ON_DISK == block_state_) { + b_ret = b_ret && macro_block_id_.is_valid(); + } + } + return b_ret; + } + + int reset(); + int init_block(const int64_t block_index, const int64_t begin_page_id, const int64_t page_num); + int write_back_start(); + int write_back_failed(); + int write_back_succ(const blocksstable::MacroBlockId macro_block_id); + int release_pages(const int64_t begin_page_id, const int64_t page_num); + int get_page_usage(int64_t &page_num) const; + int inc_ref_cnt(); + int dec_ref_cnt(int64_t &ref_cnt); + bool on_disk() const; + int can_remove(bool &can_remove) const; + blocksstable::MacroBlockId get_macro_block_id() const; + int64_t get_block_index() const; + + TO_STRING_KV(K(block_index_), K(macro_block_id_), + K(block_state_), K(page_bitmap_), + K(ref_cnt_)); + +private: + BlockState block_state_; + common::SpinRWLock lock_; + ObTmpFileBlockPageBitmap page_bitmap_; // records the page usage of this block + int64_t block_index_; + blocksstable::MacroBlockId macro_block_id_; + int64_t ref_cnt_; + DISALLOW_COPY_AND_ASSIGN(ObTmpFileBlock); +}; + +class ObTmpFileBlockHandle final +{ +public: + ObTmpFileBlockHandle() : ptr_(nullptr) {} + ObTmpFileBlockHandle(ObTmpFileBlock *block, ObTmpFileBlockManager *tmp_file_blk_mgr); + ObTmpFileBlockHandle(const ObTmpFileBlockHandle &handle); + ObTmpFileBlockHandle & operator=(const ObTmpFileBlockHandle &other); + ~ObTmpFileBlockHandle() { reset(); } + OB_INLINE ObTmpFileBlock * get() const {return ptr_; } + bool is_inited() { return nullptr != ptr_ && nullptr != tmp_file_blk_mgr_; } + void reset(); + int init(ObTmpFileBlock *block, ObTmpFileBlockManager *tmp_file_blk_mgr); + TO_STRING_KV(KPC(ptr_)); +private: + ObTmpFileBlock *ptr_; + ObTmpFileBlockManager *tmp_file_blk_mgr_; +}; + +class ObTmpFileBlockManager final +{ +public: + ObTmpFileBlockManager(); + ~ObTmpFileBlockManager(); + int init(const uint64_t tenant_id, const int64_t block_mem_limit); + void destroy(); + int create_tmp_file_block(const int64_t begin_page_id, const int64_t page_num, + int64_t &block_index); + + int write_back_start(const int64_t block_index); + int write_back_failed(const int64_t block_index); + int write_back_succ(const int64_t block_index, const blocksstable::MacroBlockId macro_block_id); + int release_tmp_file_page(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num); + int get_macro_block_list(common::ObIArray ¯o_id_list); + int get_macro_block_count(int64_t ¯o_block_count); + int get_tmp_file_block_handle(const int64_t block_index, ObTmpFileBlockHandle &handle); + int get_macro_block_id(const int64_t block_index, blocksstable::MacroBlockId ¯o_block_id); + int get_block_usage_stat(int64_t &used_page_num, int64_t ¯o_block_count); + void print_block_usage(); + OB_INLINE common::ObConcurrentFIFOAllocator &get_block_allocator() { return block_allocator_; } + TO_STRING_KV(K(is_inited_), K(tenant_id_), K(used_page_num_), + K(physical_block_num_), K(block_index_generator_)); +private: + int remove_tmp_file_block_(const int64_t block_index); +private: + typedef common::ObLinearHashMap ObTmpFileBlockMap; + typedef SpinWLockGuard ExclusiveLockGuard; + typedef SpinRLockGuard SharedLockGuard; +private: + class CollectMacroBlockIdFunctor final + { + public: + CollectMacroBlockIdFunctor(common::ObIArray ¯o_id_list) + : macro_id_list_(macro_id_list) {} + bool operator()(const ObTmpFileBlockKey &block_index, const ObTmpFileBlockHandle &block); + + private: + common::ObIArray ¯o_id_list_; + }; +private: + bool is_inited_; + uint64_t tenant_id_; + uint64_t used_page_num_; + uint64_t physical_block_num_; + uint64_t block_index_generator_; + ObTmpFileBlockMap block_map_; + common::ObConcurrentFIFOAllocator block_allocator_; + common::SpinRWLock stat_lock_; // to protect the consistency of used_page_num_ and physical_block_num_ + common::SpinRWLock map_lock_; // to protect the for_each operation of block_map_ + DISALLOW_COPY_AND_ASSIGN(ObTmpFileBlockManager); +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_BLOCK_MANAGER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_cache.cpp b/src/storage/tmp_file/ob_tmp_file_cache.cpp new file mode 100644 index 0000000000..3e4d08d523 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_cache.cpp @@ -0,0 +1,674 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include "observer/omt/ob_tenant_config_mgr.h" +#include "lib/stat/ob_diagnose_info.h" +#include "common/ob_smart_var.h" +#include "storage/ob_file_system_router.h" +#include "share/ob_task_define.h" +#include "ob_tmp_file_cache.h" +#include "storage/blocksstable/ob_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" + +using namespace oceanbase::storage; +using namespace oceanbase::share; + +namespace oceanbase +{ +namespace tmp_file +{ +/* -------------------------- ObTmpBlockCacheKey --------------------------- */ +ObTmpBlockCacheKey::ObTmpBlockCacheKey() + : block_id_(-1), tenant_id_(OB_INVALID_TENANT_ID) +{ +} + +ObTmpBlockCacheKey::ObTmpBlockCacheKey(const int64_t block_id, const uint64_t tenant_id) + : block_id_(block_id), tenant_id_(tenant_id) +{ +} + +ObTmpBlockCacheKey::~ObTmpBlockCacheKey() +{ +} + +bool ObTmpBlockCacheKey::operator ==(const ObIKVCacheKey &other) const +{ + const ObTmpBlockCacheKey &other_key = reinterpret_cast (other); + return block_id_ == other_key.block_id_ && tenant_id_ == other_key.tenant_id_; +} + +uint64_t ObTmpBlockCacheKey::get_tenant_id() const +{ + return tenant_id_; +} + +uint64_t ObTmpBlockCacheKey::hash() const +{ + return murmurhash(this, sizeof(ObTmpBlockCacheKey), 0); +} + +int64_t ObTmpBlockCacheKey::size() const +{ + return sizeof(*this); +} + +int ObTmpBlockCacheKey::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid argument, ", KR(ret)); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid tmp block cache key, ", KPC(this), KR(ret)); + } else { + key = new (buf) ObTmpBlockCacheKey(block_id_, tenant_id_); + } + return ret; +} + +bool ObTmpBlockCacheKey::is_valid() const +{ + return OB_LIKELY(block_id_ >= 0 && tenant_id_ > 0 && size() > 0); +} + +/* -------------------------- ObTmpBlockCacheValue --------------------------- */ +ObTmpBlockCacheValue::ObTmpBlockCacheValue(char *buf) + : buf_(buf), size_(OB_DEFAULT_MACRO_BLOCK_SIZE) +{ +} + +ObTmpBlockCacheValue::~ObTmpBlockCacheValue() +{ +} + +int64_t ObTmpBlockCacheValue::size() const +{ + return sizeof(*this) + size_; +} + +int ObTmpBlockCacheValue::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), KP(buf), K(buf_len), + "request_size", size()); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid tmp block cache value", KR(ret)); + } else { + ObTmpBlockCacheValue *blk_cache_value = new (buf) ObTmpBlockCacheValue(buf + sizeof(*this)); + MEMCPY(buf + sizeof(*this), buf_, size() - sizeof(*this)); + blk_cache_value->size_ = size_; + value = blk_cache_value; + } + return ret; +} + +/* -------------------------- ObTmpBlockCache --------------------------- */ + +ObTmpBlockCache &ObTmpBlockCache::get_instance() +{ + static ObTmpBlockCache instance; + return instance; +} + +int ObTmpBlockCache::init(const char *cache_name, const int64_t priority) +{ + int ret = OB_SUCCESS; + if (OB_FAIL((common::ObKVCache::init( + cache_name, priority)))) { + STORAGE_LOG(WARN, "Fail to init kv cache, ", KR(ret)); + } + return ret; +} + +void ObTmpBlockCache::destroy() +{ + common::ObKVCache::destroy(); +} + +int ObTmpBlockCache::get_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle) +{ + int ret = OB_SUCCESS; + const ObTmpBlockCacheValue *value = NULL; + if (OB_UNLIKELY(!key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), K(key)); + } else if (OB_FAIL(get(key, value, handle.handle_))) { + if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { + STORAGE_LOG(WARN, "fail to get key from block cache", KR(ret), K(key)); + } else { + EVENT_INC(ObStatEventIds::TMP_BLOCK_CACHE_MISS); + } + } else { + if (OB_ISNULL(value)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected error, the value must not be NULL", KR(ret)); + } else { + handle.value_ = const_cast(value); + EVENT_INC(ObStatEventIds::TMP_BLOCK_CACHE_HIT); + } + } + return ret; +} + +int ObTmpBlockCache::put_block(ObKVCacheInstHandle &inst_handle, + ObKVCachePair *&kvpair, + ObTmpBlockValueHandle &block_handle) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!inst_handle.is_valid() || nullptr == kvpair || !block_handle.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(inst_handle), KP(kvpair), K(block_handle)); + } else if (OB_FAIL(put_kvpair(inst_handle, kvpair, block_handle.handle_, false/*overwrite*/))) { + STORAGE_LOG(WARN, "fail to put tmp block to block cache", KR(ret)); + } + + return ret; +} + +int ObTmpBlockCache::prealloc_block(const ObTmpBlockCacheKey &key, ObKVCacheInstHandle &inst_handle, + ObKVCachePair *&kvpair, + ObTmpBlockValueHandle &block_handle) +{ + int ret = OB_SUCCESS; + inst_handle.reset(); + kvpair = nullptr; + block_handle.reset(); + if (OB_UNLIKELY(!key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(key)); + } else if (OB_FAIL(alloc(key.get_tenant_id(), key.size(), + sizeof(ObTmpBlockCacheValue) + OB_DEFAULT_MACRO_BLOCK_SIZE, + kvpair, block_handle.handle_, inst_handle))) { + STORAGE_LOG(WARN, "failed to alloc kvcache buf", KR(ret), K(key)); + } else if (OB_FAIL(key.deep_copy(reinterpret_cast(kvpair->key_), + key.size(), kvpair->key_))) { + STORAGE_LOG(WARN, "failed to deep copy key", KR(ret), K(key)); + } else { + char *buf = reinterpret_cast(kvpair->value_); + block_handle.value_ = new (buf) ObTmpBlockCacheValue(buf + sizeof(ObTmpBlockCacheValue)); + } + + if (OB_FAIL(ret)) { + block_handle.reset(); + inst_handle.reset(); + kvpair = NULL; + } + return ret; +} + +/* -------------------------- ObTmpPageCacheKey --------------------------- */ +ObTmpPageCacheKey::ObTmpPageCacheKey() + : block_id_(-1), page_id_(-1), tenant_id_(OB_INVALID_TENANT_ID) +{ +} + +ObTmpPageCacheKey::ObTmpPageCacheKey(const int64_t block_id, const int64_t page_id, + const uint64_t tenant_id) + : block_id_(block_id), page_id_(page_id), tenant_id_(tenant_id) +{ +} + +ObTmpPageCacheKey::~ObTmpPageCacheKey() +{ +} + +bool ObTmpPageCacheKey::operator ==(const ObIKVCacheKey &other) const +{ + const ObTmpPageCacheKey &other_key = reinterpret_cast (other); + return block_id_ == other_key.block_id_ + && page_id_ == other_key.page_id_ + && tenant_id_ == other_key.tenant_id_; +} + +uint64_t ObTmpPageCacheKey::get_tenant_id() const +{ + return tenant_id_; +} + +uint64_t ObTmpPageCacheKey::hash() const +{ + return murmurhash(this, sizeof(ObTmpPageCacheKey), 0); +} + +int64_t ObTmpPageCacheKey::size() const +{ + return sizeof(*this); +} + +int ObTmpPageCacheKey::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid argument, ", KR(ret)); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid tmp page cache key, ", KPC(this), KR(ret)); + } else { + key = new (buf) ObTmpPageCacheKey(block_id_, page_id_, tenant_id_); + } + return ret; +} + +bool ObTmpPageCacheKey::is_valid() const +{ + return OB_LIKELY(block_id_ >= 0 && page_id_ >= 0 && tenant_id_ > 0 && size() > 0); +} + +/* -------------------------- ObTmpPageCacheValue --------------------------- */ +ObTmpPageCacheValue::ObTmpPageCacheValue(char *buf) + : buf_(buf), size_(ObTmpFileGlobal::PAGE_SIZE) +{ +} + +ObTmpPageCacheValue::~ObTmpPageCacheValue() +{ +} + +int64_t ObTmpPageCacheValue::size() const +{ + return sizeof(*this) + size_; +} + +int ObTmpPageCacheValue::deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(buf) || OB_UNLIKELY(buf_len < size())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), KP(buf), K(buf_len), + "request_size", size()); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid tmp page cache value", KR(ret)); + } else { + ObTmpPageCacheValue *page_cache_value = new (buf) ObTmpPageCacheValue(buf + sizeof(*this)); + MEMCPY(buf + sizeof(*this), buf_, size() - sizeof(*this)); + page_cache_value->size_ = size_; + value = page_cache_value; + } + return ret; +} + +/* -------------------------- ObTmpPageCache --------------------------- */ + +ObTmpPageCache &ObTmpPageCache::get_instance() +{ + static ObTmpPageCache instance; + return instance; +} + +int ObTmpPageCache::init(const char *cache_name, const int64_t priority) +{ + int ret = OB_SUCCESS; + if (OB_FAIL((common::ObKVCache::init( + cache_name, priority)))) { + STORAGE_LOG(WARN, "Fail to init kv cache, ", KR(ret)); + } + return ret; +} + +void ObTmpPageCache::destroy() +{ + common::ObKVCache::destroy(); +} + +int ObTmpPageCache::get_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle) +{ + int ret = OB_SUCCESS; + const ObTmpPageCacheValue *value = NULL; + if (OB_UNLIKELY(!key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), K(key)); + } else if (OB_FAIL(get(key, value, handle.handle_))) { + if (OB_UNLIKELY(OB_ENTRY_NOT_EXIST != ret)) { + STORAGE_LOG(WARN, "fail to get key from page cache", KR(ret), K(key)); + } else { + EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_MISS); + } + } else { + if (OB_ISNULL(value)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected error, the value must not be NULL", KR(ret)); + } else { + handle.value_ = const_cast(value); + EVENT_INC(ObStatEventIds::TMP_PAGE_CACHE_HIT); + } + } + return ret; +} + +void ObTmpPageCache::try_put_page_to_cache(const ObTmpPageCacheKey &key, + const ObTmpPageCacheValue &value) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!key.is_valid() || !value.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), K(key), K(value)); + } else if (OB_FAIL(put(key, value, true/*overwrite*/))) { + STORAGE_LOG(WARN, "fail to put tmp page into cache", KR(ret), K(key), K(value)); + } +} + +int ObTmpPageCache::load_page(const ObTmpPageCacheKey &key, + ObIAllocator *callback_allocator, + ObTmpPageValueHandle &p_handle) +{ + int ret = OB_SUCCESS; + ObKVCacheInstHandle inst_handle; + ObKVCachePair *kvpair = NULL; + p_handle.reset(); + if (OB_UNLIKELY(!key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(key)); + } else if (OB_ISNULL(callback_allocator)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "callback_allocator is unexpected nullptr", KR(ret), K(key)); + } else if (OB_FAIL(alloc(key.get_tenant_id(), key.size(), + sizeof(ObTmpPageCacheValue) + ObTmpFileGlobal::PAGE_SIZE, + kvpair, p_handle.handle_, inst_handle))) { + STORAGE_LOG(WARN, "failed to alloc kvcache buf", KR(ret), K(key)); + } else if (OB_FAIL(key.deep_copy(reinterpret_cast(kvpair->key_), + key.size(), kvpair->key_))) { + STORAGE_LOG(WARN, "failed to deep copy key", KR(ret), K(key)); + } else { + char *buf = reinterpret_cast(kvpair->value_); + p_handle.value_ = new (buf) ObTmpPageCacheValue(buf + sizeof(ObTmpPageCacheValue)); + } + if (OB_SUCC(ret)) { + ObTmpFileBlockManager &block_manager = MTL(ObTenantTmpFileManager*)->get_tmp_file_block_manager(); + blocksstable::ObMacroBlockHandle mb_handle; + blocksstable::MacroBlockId macro_block_id; + //TODO: io_desc and io_timeout_ms value settings + common::ObIOFlag io_desc; + io_desc.set_wait_event(ObWaitEventIds::DB_FILE_DATA_READ); + int64_t io_timeout_ms = 10 * 1000; // 10s + if (OB_FAIL(block_manager.get_macro_block_id(key.get_block_id(), macro_block_id))) { + STORAGE_LOG(WARN, "failed to get macro block id", KR(ret), K(key)); + } else if (OB_FAIL(direct_read(macro_block_id, ObTmpFileGlobal::PAGE_SIZE, + key.get_page_id() * ObTmpFileGlobal::PAGE_SIZE, + io_desc, io_timeout_ms, *callback_allocator, mb_handle))) { + STORAGE_LOG(WARN, "failed to alloc kvcache buf", KR(ret), K(key)); + } else if (OB_FAIL(mb_handle.wait())) { + STORAGE_LOG(WARN, "fail to do handle read wait", KR(ret), K(key)); + } else { + MEMCPY(p_handle.value_->get_buffer(), mb_handle.get_buffer(), ObTmpFileGlobal::PAGE_SIZE); + } + } + if (FAILEDx(put_kvpair(inst_handle, kvpair, p_handle.handle_, false/*overwrite*/))) { + if (OB_ENTRY_EXIST == ret) { + ret = OB_SUCCESS; + } else { + STORAGE_LOG(WARN, "fail to put tmp page to page cache", KR(ret), K(key)); + } + } + if (OB_FAIL(ret)) { + p_handle.reset(); + inst_handle.reset(); + kvpair = NULL; + } + return ret; +} + +// only read pages from disk +int ObTmpPageCache::direct_read(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + common::ObIAllocator &callback_allocator, + blocksstable::ObMacroBlockHandle &mb_handle) +{ + int ret = OB_SUCCESS; + void *buf = nullptr; + ObTmpDirectReadPageIOCallback *callback = nullptr; + if (OB_UNLIKELY(!macro_block_id.is_valid() || read_size <= 0 || begin_offset_in_block < 0 || + (begin_offset_in_block + read_size > OB_DEFAULT_MACRO_BLOCK_SIZE) || + !io_desc.is_valid() || io_timeout_ms < 0)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid arguments", KR(ret), K(macro_block_id), K(read_size), + K(begin_offset_in_block), K(io_desc), K(io_timeout_ms)); + } else if (OB_ISNULL(buf = callback_allocator.alloc(sizeof(ObTmpDirectReadPageIOCallback)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "allocate callback memory failed", KR(ret)); + } else { + callback = new (buf) ObTmpDirectReadPageIOCallback; + callback->cache_ = this; + callback->allocator_ = &callback_allocator; + if (OB_FAIL(inner_read_io_(macro_block_id, read_size, begin_offset_in_block, + io_desc, io_timeout_ms, callback, mb_handle))) { + STORAGE_LOG(WARN, "fail to inner read io", KR(ret), K(macro_block_id), K(read_size), + K(begin_offset_in_block), K(io_desc), K(io_timeout_ms)); + } + // There is no need to handle error cases (freeing the memory of the + // callback) because inner_read_io_ will handle error cases and free the + // memory of the callback. + } + return ret; +} + +// read pages from disk and put them into kv_cache +int ObTmpPageCache::cached_read(const common::ObIArray &page_keys, + const blocksstable::MacroBlockId macro_block_id, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + common::ObIAllocator &callback_allocator, + blocksstable::ObMacroBlockHandle &mb_handle) +{ + int ret = OB_SUCCESS; + void *buf = nullptr; + ObTmpCachedReadPageIOCallback *callback = nullptr; + const int64_t read_size = page_keys.count() * ObTmpFileGlobal::PAGE_SIZE; + if (OB_UNLIKELY(page_keys.count() <= 0)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid arguments", KR(ret), K(page_keys.count())); + } else if (OB_UNLIKELY(!macro_block_id.is_valid() || begin_offset_in_block < 0 || + !io_desc.is_valid() || io_timeout_ms < 0)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "Invalid arguments", KR(ret), K(macro_block_id), K(begin_offset_in_block), + K(io_desc), K(io_timeout_ms)); + } else if (OB_UNLIKELY(begin_offset_in_block % ObTmpFileGlobal::PAGE_SIZE != 0)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "cached_read should read completed pages", KR(ret), K(begin_offset_in_block)); + } else if (OB_UNLIKELY(begin_offset_in_block + read_size > OB_DEFAULT_MACRO_BLOCK_SIZE)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid range", KR(ret), K(read_size), K(begin_offset_in_block)); + } else if (OB_ISNULL(buf = callback_allocator.alloc(sizeof(ObTmpCachedReadPageIOCallback)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "allocate callback memory failed", KR(ret), K(macro_block_id)); + } else { + callback = new (buf) ObTmpCachedReadPageIOCallback; + callback->cache_ = this; + callback->allocator_ = &callback_allocator; + if (OB_FAIL(callback->page_keys_.assign(page_keys))) { + STORAGE_LOG(WARN, "fail to assign page keys", KR(ret), K(page_keys.count())); + } else if (OB_FAIL(inner_read_io_(macro_block_id, read_size, begin_offset_in_block, + io_desc, io_timeout_ms, callback, mb_handle))) { + STORAGE_LOG(WARN, "fail to inner read io", KR(ret), K(macro_block_id), K(read_size), + K(begin_offset_in_block), K(io_desc), K(io_timeout_ms)); + } + // There is no need to handle error cases (freeing the memory of the + // callback) because inner_read_io_ will handle error cases and free the + // memory of the callback. + } + return ret; +} + +// TODO: wanyue.wy +// refactor logic of callback. +// currently, the buffer of callback is freed in different places. +// if async_read_() is failed, it will be released in here; +// otherwise, it will be released after ObTmpFileIOCtx::do_read_wait_() has been called. +// (callback is bound with ObIOResult of io_handle, do_read_wait_() will call reset() of io_handle to +// destroy callback and its data buf) +// we need to refactor it by removing callback allocator and directly copy read io buf to user buf +// rather than using a callback buf +int ObTmpPageCache::inner_read_io_(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + ObITmpPageIOCallback *callback, + blocksstable::ObMacroBlockHandle &handle) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(async_read_(macro_block_id, read_size, begin_offset_in_block, + io_desc, io_timeout_ms, callback, handle))) { + STORAGE_LOG(WARN, "fail to read tmp page from io", KR(ret), K(macro_block_id), K(read_size), + K(begin_offset_in_block), K(io_desc), K(io_timeout_ms), KP(callback)); + } + + // if read successful, callback will be freed after user calls ObTmpFileIOHandle::wait() + // for copying data from callback's buf to user's buf + // thus, here just free memory of the failed cases + if (OB_FAIL(ret) && OB_NOT_NULL(callback) && OB_NOT_NULL(callback->get_allocator())) { + common::ObIAllocator *allocator = callback->get_allocator(); + callback->~ObITmpPageIOCallback(); + allocator->free(callback); + } + + return ret; +} + +int ObTmpPageCache::async_read_(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + ObITmpPageIOCallback *callback, + blocksstable::ObMacroBlockHandle &handle) +{ + int ret = OB_SUCCESS; + blocksstable::ObMacroBlockReadInfo read_info; + read_info.macro_block_id_ = macro_block_id; + read_info.size_ = read_size; + read_info.offset_ = begin_offset_in_block; + read_info.io_desc_ = io_desc; + read_info.io_timeout_ms_ = io_timeout_ms; + read_info.io_callback_ = callback; + read_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); + read_info.io_desc_.set_sys_module_id(ObIOModule::TMP_TENANT_MEM_BLOCK_IO); + + if (OB_FAIL(handle.async_read(read_info))) { + STORAGE_LOG(WARN, "fail to async read block", KR(ret), K(read_info)); + } + return ret; +} + +ObTmpPageCache::ObITmpPageIOCallback::ObITmpPageIOCallback() + : cache_(NULL), allocator_(NULL), data_buf_(NULL) +{ + static_assert(sizeof(*this) <= CALLBACK_BUF_SIZE, "IOCallback buf size not enough"); +} + +ObTmpPageCache::ObITmpPageIOCallback::~ObITmpPageIOCallback() +{ + if (NULL != allocator_ && NULL != data_buf_) { + allocator_->free(data_buf_); + data_buf_ = NULL; + } + allocator_ = NULL; +} + +int ObTmpPageCache::ObITmpPageIOCallback::alloc_data_buf(const char *io_data_buffer, const int64_t data_size) +{ + int ret = alloc_and_copy_data(io_data_buffer, data_size, allocator_, data_buf_); + return ret; +} + +int ObTmpPageCache::ObITmpPageIOCallback::process_page( + const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!key.is_valid() || !value.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid arguments", KR(ret), K(key), K(value)); + } else if (OB_FAIL(cache_->put(key, value, true/*overwrite*/))) { + STORAGE_LOG(WARN, "fail to put tmp page into cache", KR(ret), K(key), K(value)); + } + return ret; +} + +ObTmpPageCache::ObTmpCachedReadPageIOCallback::ObTmpCachedReadPageIOCallback() + : page_keys_() +{ + static_assert(sizeof(*this) <= CALLBACK_BUF_SIZE, "IOCallback buf size not enough"); + page_keys_.set_attr(ObMemAttr(MTL_ID(), "TFCacheRead")); +} + +ObTmpPageCache::ObTmpCachedReadPageIOCallback::~ObTmpCachedReadPageIOCallback() +{ + page_keys_.reset(); +} + +int ObTmpPageCache::ObTmpCachedReadPageIOCallback::inner_process(const char *data_buffer, const int64_t size) +{ + int ret = OB_SUCCESS; + ObTimeGuard time_guard("TmpCachedReadPage_Callback_Process", 100000); //100ms + if (OB_ISNULL(cache_) || OB_ISNULL(allocator_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "Invalid tmp page cache callback allocator", KR(ret), KP(cache_), KP(allocator_)); + } else if (OB_UNLIKELY(size <= 0 || data_buffer == nullptr)) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid data buffer size", KR(ret), K(size), KP(data_buffer)); + } else if (OB_UNLIKELY(page_keys_.count() * ObTmpFileGlobal::PAGE_SIZE != size)) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid data buffer size", KR(ret), K(size), K(page_keys_.count())); + } else if (OB_FAIL(alloc_data_buf(data_buffer, size))) { + STORAGE_LOG(WARN, "Fail to allocate memory, ", KR(ret), K(size)); + } else if (FALSE_IT(time_guard.click("alloc_data_buf"))) { + } else { + for (int32_t i = 0; OB_SUCC(ret) && i < page_keys_.count(); i++) { + ObTmpPageCacheValue value(nullptr); + value.set_buffer(data_buf_ + i * ObTmpFileGlobal::PAGE_SIZE); + if (OB_FAIL(process_page(page_keys_.at(i), value))) { + STORAGE_LOG(WARN, "fail to process tmp page cache in callback", KR(ret)); + } + } + time_guard.click("process_page"); + } + if (OB_FAIL(ret) && NULL != allocator_ && NULL != data_buf_) { + allocator_->free(data_buf_); + data_buf_ = NULL; + } + return ret; +} + +int ObTmpPageCache::ObTmpDirectReadPageIOCallback::inner_process(const char *data_buffer, const int64_t size) +{ + int ret = OB_SUCCESS; + ObTimeGuard time_guard("ObTmpDirectReadPageIOCallback", 100000); //100ms + if (OB_ISNULL(cache_) || OB_ISNULL(allocator_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "Invalid tmp page cache callback allocator", KR(ret), KP(cache_), KP(allocator_)); + } else if (OB_UNLIKELY(size <= 0 || data_buffer == nullptr)) { + ret = OB_INVALID_DATA; + STORAGE_LOG(WARN, "invalid data buffer size", KR(ret), K(size), KP(data_buffer)); + } else if (OB_FAIL(alloc_data_buf(data_buffer, size))) { + STORAGE_LOG(WARN, "Fail to allocate memory, ", KR(ret), K(size)); + } else if (FALSE_IT(time_guard.click("alloc_data_buf"))) { + } + if (OB_FAIL(ret) && NULL != allocator_ && NULL != data_buf_) { + allocator_->free(data_buf_); + data_buf_ = NULL; + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_cache.h b/src/storage/tmp_file/ob_tmp_file_cache.h new file mode 100644 index 0000000000..bece9b6abb --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_cache.h @@ -0,0 +1,251 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_CACHE_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_CACHE_H_ + +#include "lib/hash/ob_hashmap.h" +#include "lib/queue/ob_link_queue.h" +#include "share/io/ob_io_manager.h" +#include "share/cache/ob_kv_storecache.h" +#include "storage/ob_i_store.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpBlockCacheKey final : public common::ObIKVCacheKey +{ +public: + ObTmpBlockCacheKey(); + ObTmpBlockCacheKey(const int64_t block_id, const uint64_t tenant_id); + ~ObTmpBlockCacheKey(); + bool operator ==(const ObIKVCacheKey &other) const override; + uint64_t get_tenant_id() const override; + uint64_t hash() const override; + int64_t size() const override; + int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const override; + bool is_valid() const; + int64_t get_block_id() const { return block_id_; } + TO_STRING_KV(K(block_id_), K(tenant_id_)); + +private: + int64_t block_id_; + uint64_t tenant_id_; +}; + +class ObTmpBlockCacheValue final : public common::ObIKVCacheValue +{ +public: + explicit ObTmpBlockCacheValue(char *buf); + ~ObTmpBlockCacheValue(); + int64_t size() const override; + int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const override; + bool is_valid() const { return NULL != buf_ && size() > 0; } + char *get_buffer() { return buf_; } + void set_buffer(char *buf) { buf_ = buf;} + TO_STRING_KV(KP(buf_), K(size_)); + +private: + char *buf_; + int64_t size_; + DISALLOW_COPY_AND_ASSIGN(ObTmpBlockCacheValue); +}; + +struct ObTmpBlockValueHandle final +{ +public: + ObTmpBlockValueHandle() : value_(NULL), handle_() {} + ~ObTmpBlockValueHandle() = default; + bool is_valid() const { return NULL != value_ && handle_.is_valid(); } + void reset() + { + handle_.reset(); + value_ = NULL; + } + TO_STRING_KV(KP(value_), K(handle_)); + ObTmpBlockCacheValue *value_; + common::ObKVCacheHandle handle_; +}; + +class ObTmpBlockCache final : public common::ObKVCache +{ +public: + typedef common::ObKVCache BasePageCache; + static ObTmpBlockCache &get_instance(); + int init(const char *cache_name, const int64_t priority); + void destroy(); + int get_block(const ObTmpBlockCacheKey &key, ObTmpBlockValueHandle &handle); + int put_block(ObKVCacheInstHandle &inst_handle, + ObKVCachePair *&kvpair, + ObTmpBlockValueHandle &block_handle); + int prealloc_block(const ObTmpBlockCacheKey &key, + ObKVCacheInstHandle &inst_handle, + ObKVCachePair *&kvpair, + ObTmpBlockValueHandle &block_handle); +private: + ObTmpBlockCache() {} + ~ObTmpBlockCache() {} + +private: + DISALLOW_COPY_AND_ASSIGN(ObTmpBlockCache); +}; + +class ObTmpPageCacheKey final : public common::ObIKVCacheKey +{ +public: + ObTmpPageCacheKey(); + ObTmpPageCacheKey(const int64_t block_id, const int64_t page_id, const uint64_t tenant_id); + ~ObTmpPageCacheKey(); + bool operator ==(const ObIKVCacheKey &other) const override; + uint64_t get_tenant_id() const override; + uint64_t hash() const override; + int64_t size() const override; + int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheKey *&key) const override; + bool is_valid() const; + int64_t get_page_id() const { return page_id_; } + int64_t get_block_id() const { return block_id_; } + TO_STRING_KV(K(block_id_), K(page_id_), K(tenant_id_)); + +private: + int64_t block_id_; + int64_t page_id_; + uint64_t tenant_id_; +}; + +class ObTmpPageCacheValue final : public common::ObIKVCacheValue +{ +public: + explicit ObTmpPageCacheValue(char *buf); + ~ObTmpPageCacheValue(); + int64_t size() const override; + int deep_copy(char *buf, const int64_t buf_len, ObIKVCacheValue *&value) const override; + bool is_valid() const { return NULL != buf_ && size() > 0; } + char *get_buffer() { return buf_; } + void set_buffer(char *buf) { buf_ = buf;} + TO_STRING_KV(KP(buf_), K(size_)); + +private: + char *buf_; + int64_t size_; + DISALLOW_COPY_AND_ASSIGN(ObTmpPageCacheValue); +}; + +struct ObTmpPageValueHandle final +{ +public: + ObTmpPageValueHandle() : value_(NULL), handle_() {} + ~ObTmpPageValueHandle() = default; + void reset() + { + handle_.reset(); + value_ = NULL; + } + TO_STRING_KV(KP(value_), K(handle_)); + ObTmpPageCacheValue *value_; + common::ObKVCacheHandle handle_; +}; + +class ObTmpPageCache final : public common::ObKVCache +{ +public: + typedef common::ObKVCache BasePageCache; + static ObTmpPageCache &get_instance(); + int init(const char *cache_name, const int64_t priority); + int direct_read(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + common::ObIAllocator &callback_allocator, + blocksstable::ObMacroBlockHandle &mb_handle); + // multi page cached_read + int cached_read(const common::ObIArray &page_keys, + const blocksstable::MacroBlockId macro_block_id, + const int64_t begin_offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + common::ObIAllocator &callback_allocator, + blocksstable::ObMacroBlockHandle &mb_handle); + int get_page(const ObTmpPageCacheKey &key, ObTmpPageValueHandle &handle); + int load_page(const ObTmpPageCacheKey &key, + ObIAllocator *callback_allocator, + ObTmpPageValueHandle &p_handle); + void try_put_page_to_cache(const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value); + void destroy(); +public: + class ObITmpPageIOCallback : public common::ObIOCallback + { + public: + ObITmpPageIOCallback(); + virtual ~ObITmpPageIOCallback(); + virtual int alloc_data_buf(const char *io_data_buffer, const int64_t data_size) override; + const char *get_data() override { return data_buf_; } + protected: + friend class ObTmpPageCache; + virtual int process_page(const ObTmpPageCacheKey &key, const ObTmpPageCacheValue &value); + virtual ObIAllocator *get_allocator() { return allocator_; } + + protected: + BasePageCache *cache_; + common::ObIAllocator *allocator_; + char *data_buf_; // actual data buffer + }; + + class ObTmpCachedReadPageIOCallback final : public ObITmpPageIOCallback + { + public: + ObTmpCachedReadPageIOCallback(); + ~ObTmpCachedReadPageIOCallback(); + int64_t size() const override { return sizeof(*this); } + int inner_process(const char *data_buffer, const int64_t size) override; + TO_STRING_KV("callback_type:", "ObTmpCachedReadPageIOCallback", KP(data_buf_)); + DISALLOW_COPY_AND_ASSIGN(ObTmpCachedReadPageIOCallback); + private: + friend class ObTmpPageCache; + common::ObArray page_keys_; + }; + class ObTmpDirectReadPageIOCallback final : public ObITmpPageIOCallback + { + public: + ObTmpDirectReadPageIOCallback() {} + ~ObTmpDirectReadPageIOCallback() {} + int64_t size() const override { return sizeof(*this); } + int inner_process(const char *data_buffer, const int64_t size) override; + TO_STRING_KV("callback_type:", "ObTmpDirectReadPageIOCallback", KP(data_buf_)); + DISALLOW_COPY_AND_ASSIGN(ObTmpDirectReadPageIOCallback); + }; +private: + ObTmpPageCache() {} + ~ObTmpPageCache() {} + int inner_read_io_(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + ObITmpPageIOCallback *callback, + blocksstable::ObMacroBlockHandle &handle); + int async_read_(const blocksstable::MacroBlockId macro_block_id, + const int64_t read_size, + const int64_t offset_in_block, + const common::ObIOFlag io_desc, + const int64_t io_timeout_ms, + ObITmpPageIOCallback *callback, + blocksstable::ObMacroBlockHandle &handle); + +private: + DISALLOW_COPY_AND_ASSIGN(ObTmpPageCache); +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_CACHE_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_eviction_manager.cpp b/src/storage/tmp_file/ob_tmp_file_eviction_manager.cpp new file mode 100644 index 0000000000..7e43812e74 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_eviction_manager.cpp @@ -0,0 +1,218 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_eviction_manager.h" + +namespace oceanbase +{ +using namespace storage; +using namespace share; + +namespace tmp_file +{ +void ObTmpFileEvictionManager::destroy() +{ + { + ObSpinLockGuard guard(meta_list_lock_); + file_meta_eviction_list_.reset(); + } + { + ObSpinLockGuard guard(data_list_lock_); + file_data_eviction_list_.reset(); + } +} + +int64_t ObTmpFileEvictionManager::get_file_size() +{ + int64_t file_size = 0; + { + ObSpinLockGuard guard(meta_list_lock_); + file_size += file_meta_eviction_list_.get_size(); + } + { + ObSpinLockGuard guard(data_list_lock_); + file_size += file_data_eviction_list_.get_size(); + } + return file_size; +} + +int ObTmpFileEvictionManager::add_file(const bool is_meta, ObSharedNothingTmpFile &file) +{ + int ret = OB_SUCCESS; + ObSpinLock &lock = is_meta ? meta_list_lock_ : data_list_lock_; + TmpFileEvictionList &eviction_list = is_meta ? file_meta_eviction_list_ : file_data_eviction_list_; + ObSharedNothingTmpFile::ObTmpFileNode &eviction_node = is_meta ? file.get_meta_eviction_node() : file.get_data_eviction_node(); + + ObSpinLockGuard guard(lock); + file.inc_ref_cnt(); + if (OB_UNLIKELY(!eviction_list.add_last(&eviction_node))) { + file.dec_ref_cnt(); + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to add node", KR(ret)); + } + + return ret; +} + +int ObTmpFileEvictionManager::remove_file(ObSharedNothingTmpFile &file) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remove_file(true, file))) { + LOG_WARN("fail to remove file from meta list", KR(ret), K(file)); + } else if (OB_FAIL(remove_file(false, file))) { + LOG_WARN("fail to remove file from data list", KR(ret), K(file)); + } + return ret; +} + +int ObTmpFileEvictionManager::remove_file(const bool is_meta, ObSharedNothingTmpFile &file) +{ + int ret = OB_SUCCESS; + ObSpinLock &lock = is_meta ? meta_list_lock_ : data_list_lock_; + TmpFileEvictionList &eviction_list = is_meta ? file_meta_eviction_list_ : file_data_eviction_list_; + ObSharedNothingTmpFile::ObTmpFileNode &eviction_node = is_meta ? file.get_meta_eviction_node() : file.get_data_eviction_node(); + ObSpinLockGuard guard(lock); + if (OB_NOT_NULL(eviction_node.get_next())) { + if (OB_UNLIKELY(!eviction_list.remove(&eviction_node))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to remove node", KR(ret), K(file)); + } else { + file.dec_ref_cnt(); + } + } + + return ret; +} + +int ObTmpFileEvictionManager::evict(const int64_t expected_evict_page_num, int64_t &actual_evict_page_num) +{ + int ret = OB_SUCCESS; + int64_t remain_evict_page_num = expected_evict_page_num; + actual_evict_page_num = 0; + + int64_t actual_evict_data_page_num = 0; + if (OB_FAIL(evict_file_from_list_(false/*is_meta*/, remain_evict_page_num, actual_evict_data_page_num))) { + LOG_WARN("fail to evict file from list", KR(ret), K(remain_evict_page_num), K(actual_evict_data_page_num)); + } else { + remain_evict_page_num -= actual_evict_data_page_num; + actual_evict_page_num += actual_evict_data_page_num; + } + LOG_DEBUG("evict data pages over", KR(ret), K(expected_evict_page_num), K(remain_evict_page_num), + K(actual_evict_page_num), K(actual_evict_data_page_num)); + + int64_t actual_evict_meta_page_num = 0; + if (FAILEDx(evict_file_from_list_(true/*is_meta*/, remain_evict_page_num, actual_evict_meta_page_num))) { + LOG_WARN("fail to evict file from list", KR(ret), K(remain_evict_page_num), K(actual_evict_meta_page_num)); + } else { + remain_evict_page_num -= actual_evict_meta_page_num; + actual_evict_page_num += actual_evict_meta_page_num; + } + LOG_DEBUG("evict meta pages over", KR(ret), K(expected_evict_page_num), K(remain_evict_page_num), + K(actual_evict_page_num), K(actual_evict_meta_page_num)); + + return ret; +} + +int ObTmpFileEvictionManager::evict_file_from_list_(const bool &is_meta, + const int64_t expected_evict_page_num, + int64_t &actual_evict_page_num) +{ + int ret = OB_SUCCESS; + bool is_empty_list = false; + int64_t remain_evict_page_num = expected_evict_page_num; + actual_evict_page_num = 0; + + // attention: + // in order to avoid repeated inserting file node into list by flush manager, + // even thought evict manager has popped file from list, + // we also keep the file's `is_in_meta_eviction_list_` or `is_in_data_eviction_list_` be true. + // in evict_page() of file, if `remain_flushed_file_page_num` > 0, it will reinsert file node + // into eviction list; otherwise, it will set `is_in_meta_eviction_list_` or `is_in_data_eviction_list_` be false + + int64_t list_cnt = + is_meta ? file_meta_eviction_list_.get_size() : file_data_eviction_list_.get_size(); + + while(OB_SUCC(ret) && remain_evict_page_num > 0 && !is_empty_list && list_cnt-- > 0) { + ObTmpFileHandle file_handle; + int64_t actual_evict_file_page_num = 0; + int64_t remain_flushed_file_page_num = 0; + if (OB_FAIL(pop_file_from_list_(is_meta, file_handle))) { + if (OB_EMPTY_RESULT == ret) { + ret = OB_SUCCESS; + is_empty_list = true; + LOG_DEBUG("no tmp file in list", K(is_meta), K(expected_evict_page_num), + K(remain_evict_page_num), K(actual_evict_page_num)); + } else { + LOG_WARN("fail to pop file from list", KR(ret), K(is_meta)); + } + } else if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file handle is invalid", KR(ret), K(file_handle)); + } else if (is_meta) { + if (OB_FAIL(file_handle.get()->evict_meta_pages(remain_evict_page_num, + actual_evict_file_page_num))) { + LOG_WARN("fail to evict meta pages", KR(ret), K(file_handle), K(remain_evict_page_num), + K(actual_evict_file_page_num)); + } + } else { + if (OB_FAIL(file_handle.get()->evict_data_pages(remain_evict_page_num, + actual_evict_file_page_num, + remain_flushed_file_page_num))) { + LOG_WARN("fail to evict data pages", KR(ret), K(file_handle), K(remain_evict_page_num), + K(actual_evict_page_num), K(remain_flushed_file_page_num)); + } else if (OB_UNLIKELY(remain_evict_page_num > actual_evict_file_page_num && remain_flushed_file_page_num > 1)) { + // we allow to not evict the last data page + ret = OB_ERR_UNEXPECTED; + LOG_WARN("evict_data_pages unexpected finishes before expected pages are eliminated", + KR(ret), K(is_meta), K(file_handle), K(remain_evict_page_num), + K(actual_evict_page_num), K(actual_evict_file_page_num), + K(remain_flushed_file_page_num)); + } + } + + if (OB_SUCC(ret)) { + remain_evict_page_num -= actual_evict_file_page_num; + actual_evict_page_num += actual_evict_file_page_num; + } + } + + return ret; +} + +int ObTmpFileEvictionManager::pop_file_from_list_(const bool &is_meta, ObTmpFileHandle &file_handle) +{ + int ret = OB_SUCCESS; + file_handle.reset(); + ObSharedNothingTmpFile *file = nullptr; + ObSpinLock &lock = is_meta ? meta_list_lock_ : data_list_lock_; + TmpFileEvictionList &eviction_list = is_meta ? file_meta_eviction_list_ : file_data_eviction_list_; + + ObSpinLockGuard guard(lock); + if (eviction_list.is_empty()) { + ret = OB_EMPTY_RESULT; + LOG_DEBUG("eviction_list is empty", K(is_meta)); + } else if (OB_ISNULL(file = &eviction_list.remove_first()->file_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file is null", KR(ret)); + } else if (OB_FAIL(file_handle.init(file))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to init file", KR(ret), KP(file)); + } else { + file->dec_ref_cnt(); + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_eviction_manager.h b/src/storage/tmp_file/ob_tmp_file_eviction_manager.h new file mode 100644 index 0000000000..6b72de2490 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_eviction_manager.h @@ -0,0 +1,56 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_ELIMINATION_MANAGER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_ELIMINATION_MANAGER_H_ + +#include "storage/blocksstable/ob_macro_block_id.h" +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" +#include "lib/allocator/ob_fifo_allocator.h" +#include "lib/list/ob_dlist.h" +#include "lib/lock/ob_spin_lock.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileEvictionManager +{ +public: + typedef ObSharedNothingTmpFile::ObTmpFileNode TmpFileNode; + typedef common::ObDList TmpFileEvictionList; + +public: + ObTmpFileEvictionManager() : data_list_lock_(), file_data_eviction_list_(), + meta_list_lock_(), file_meta_eviction_list_() {} + ~ObTmpFileEvictionManager() { destroy(); } + void destroy(); + int64_t get_file_size(); + int add_file(const bool is_meta, ObSharedNothingTmpFile &file); + int remove_file(ObSharedNothingTmpFile &file); + int remove_file(const bool is_meta, ObSharedNothingTmpFile &file); + int evict(const int64_t expected_evict_page_num, int64_t &actual_evict_page_num); + +private: + int evict_file_from_list_(const bool &is_meta, const int64_t expected_evict_page_num, int64_t &actual_evict_page_num); + int pop_file_from_list_(const bool &is_meta, ObTmpFileHandle &file_handle); +private: + ObSpinLock data_list_lock_; + TmpFileEvictionList file_data_eviction_list_; + ObSpinLock meta_list_lock_; + TmpFileEvictionList file_meta_eviction_list_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase + +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_ELIMINATION_MANAGER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_flush_ctx.cpp b/src/storage/tmp_file/ob_tmp_file_flush_ctx.cpp new file mode 100644 index 0000000000..b2c759e3c8 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_ctx.cpp @@ -0,0 +1,321 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "share/ob_errno.h" // KR +#include "storage/tmp_file/ob_tmp_file_flush_ctx.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +ObTmpFileFlushInfo::ObTmpFileFlushInfo() + : fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), + batch_flush_idx_(0), + has_last_page_lock_(false), + insert_meta_tree_done_(false), + update_meta_data_done_(false), + file_handle_(), + flush_data_page_disk_begin_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flush_data_page_num_(-1), + flush_virtual_page_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID), + file_size_(0), + flush_meta_page_array_() +{ + flush_meta_page_array_.set_attr(ObMemAttr(MTL_ID(), "TFFlushMetaArr")); +} + +void ObTmpFileFlushInfo::reset() +{ + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + batch_flush_idx_ = 0; + has_last_page_lock_ = false; + insert_meta_tree_done_ = false; + update_meta_data_done_ = false; + file_handle_.reset(); + flush_data_page_disk_begin_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flush_data_page_num_ = -1; + flush_virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + file_size_ = 0; + flush_meta_page_array_.reset(); +} + +// -------------- ObTmpFileBatchFlushContext --------------- // + +int ObTmpFileBatchFlushContext::init() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileBatchFlushContext init twice", KR(ret)); + } else if (OB_FAIL(file_ctx_hash_.create(256, ObMemAttr(MTL_ID(), "TFileFLCtx")))) { + LOG_WARN("failed to create hash map", K(ret)); + } else { + flush_failed_array_.set_attr(ObMemAttr(MTL_ID(), "TFFlushFailArr")); + state_ = FlushCtxState::FSM_F1; + is_inited_ = true; + } + return ret; +} + +int ObTmpFileBatchFlushContext::prepare_flush_ctx( + const int64_t expect_flush_size, + ObTmpFileFlushPriorityManager *prio_mgr, + ObTmpFileFlushMonitor *flush_monitor) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(expect_flush_size <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(expect_flush_size)); + } else if (OB_ISNULL(prio_mgr) || OB_ISNULL(flush_monitor)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(prio_mgr), KP(flush_monitor)); + } else if (OB_FAIL(iter_.init(prio_mgr))) { + LOG_WARN("failed to init iterator", KR(ret), K(*this)); + } else if (OB_FAIL(flush_failed_array_.reserve(MAX_COPY_FAIL_COUNT))) { + LOG_WARN("fail to reserve flush filed array", KR(ret), K(*this)); + } else { + expect_flush_size_ = expect_flush_size; + flush_monitor_ptr_ = flush_monitor; + } + return ret; +} + +int ObTmpFileBatchFlushContext::clear_flush_ctx(ObTmpFileFlushPriorityManager &prio_mgr) +{ + int ret = OB_SUCCESS; + // insert the remaining files in the iterator back into the flush priority manager + if (OB_FAIL(iter_.reset())) { + LOG_ERROR("failed to reset flush iterator", KR(ret), KPC(this)); + } + + // insert the files that failed to copy data back into the flush queue. after this step, + // call files taken out by the iterator for this round + // (excluding files with no dirty pages) will appear in the flush priority manager. + for (int64_t i = 0; OB_SUCC(ret) && i < flush_failed_array_.count(); i++) { + const ObTmpFileFlushFailRecord &record = flush_failed_array_.at(i); + const ObTmpFileHandle &file_handle = record.file_handle_; + if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("file handle is nullptr", KR(ret)); + } else { + ObSharedNothingTmpFile &file = *file_handle.get(); + file.reinsert_flush_node(record.is_meta_); + } + } + + if (OB_FAIL(ret)) { + } else if (flush_seq_ctx_.send_io_succ_cnt_ == flush_seq_ctx_.create_flush_task_cnt_) { + LOG_DEBUG("reset flush_seq_ctx_", KPC(this)); + flush_seq_ctx_.send_io_succ_cnt_ = 0; + flush_seq_ctx_.create_flush_task_cnt_ = 0; + flush_seq_ctx_.flush_sequence_ += 1; + // remove the files that have already been successfully flushed(recorded in file_ctx_hash_) + // from flush priority manager, ensuring that these files do not trigger flushing + // before the completion of this round of flush IO. + RemoveFileOp remove_op(prio_mgr); + if (OB_FAIL(file_ctx_hash_.foreach_refactored(remove_op))) { + LOG_ERROR("fail to erase file ctx from hash", KR(ret), KPC(this)); + } else { + file_ctx_hash_.clear(); + } + } else if (flush_seq_ctx_.send_io_succ_cnt_ < flush_seq_ctx_.create_flush_task_cnt_) { + // likely to occur when write buffer pool is full and need to fast flush meta page + LOG_WARN("flush_seq_ctx_ could not increase flush sequence", KPC(this)); + } else if (OB_UNLIKELY(flush_seq_ctx_.send_io_succ_cnt_ > flush_seq_ctx_.create_flush_task_cnt_)) { + LOG_ERROR("unexpected flush_seq_ctx_", KPC(this)); + } + + fail_too_many_ = false; + expect_flush_size_ = 0; + actual_flush_size_ = 0; + iter_.destroy(); + flush_failed_array_.reset(); + state_ = FlushCtxState::FSM_F1; + return ret; +} + +int ObTmpFileBatchFlushContext::RemoveFileOp::operator () (hash::HashMapPair &kv) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle &file_handle = kv.second.file_handle_; + if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file handle is nullptr", KR(ret)); + } else { + // remove all flush nodes for this file to prevent repeated flushing when flushing only data page or only meta page, + // this avoids triggering a data page flush while waiting for I/O on meta pages, which could lead to new data items + // being inserted to the flushing meta page, causing the memory meta page to become inconsistent with the page on disk. + ObSharedNothingTmpFile &file = *file_handle.get(); + if (file.is_flushing()) { + if (OB_FAIL(file.remove_flush_node(false/*is_meta*/))) { + LOG_ERROR("fail to remove file data node from flush priority mgr", KR(ret), K(file)); + } else if (OB_FAIL(file.remove_flush_node(true/*is_meta*/))) { + LOG_ERROR("fail to remove file meta node from flush priority mgr", KR(ret), K(file)); + } else { + LOG_DEBUG("succ to remove flush node from flush priority mgr", K(file)); + } + } + } + return ret; +} + +void ObTmpFileBatchFlushContext::destroy() +{ + is_inited_ = false; + fail_too_many_ = false; + expect_flush_size_ = 0; + actual_flush_size_ = 0; + flush_monitor_ptr_ = nullptr; + state_ = FlushCtxState::FSM_F1; + iter_.destroy(); + flush_failed_array_.reset(); + file_ctx_hash_.destroy(); +} + +void ObTmpFileBatchFlushContext::update_ctx_by_flush_task(const ObTmpFileFlushTask &flush_task) +{ + actual_flush_size_ += flush_task.get_data_length(); + + if (ObTmpFileFlushTask::TFFT_WAIT == flush_task.get_state() || flush_task.get_data_length() == 0) { + // we will free task if its data_length == 0, therefore we count it as succ here + flush_seq_ctx_.send_io_succ_cnt_ += 1; + } +} + +void ObTmpFileBatchFlushContext::record_flush_stage() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_monitor_ptr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("flush monitor is null", KR(ret)); + } else { + flush_monitor_ptr_->record_flush_stage(state_); + } +} + +void ObTmpFileBatchFlushContext::record_flush_task(const int64_t data_length) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_monitor_ptr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("flush monitor is null", KR(ret)); + } else { + flush_monitor_ptr_->record_flush_task(data_length); + } +} + +// -------------- ObTmpFileFlushTask --------------- // + +ObTmpFileFlushTask::ObTmpFileFlushTask(ObIAllocator &task_allocator) + : inst_handle_(), + kvpair_(nullptr), + block_handle_(), + ret_code_(OB_SUCCESS), + data_length_(0), + block_index_(-1), + flush_seq_(-1), + create_ts_(-1), + is_io_finished_(false), + fast_flush_tree_page_(false), + task_state_(ObTmpFileFlushTaskState::TFFT_INITED), + tmp_file_block_handle_(), + handle_(), + flush_infos_(), + task_allocator_(task_allocator) +{ + flush_infos_.set_attr(ObMemAttr(MTL_ID(), "TFFlushInfos")); +} + +void ObTmpFileFlushTask::destroy() +{ + block_handle_.reset(); + inst_handle_.reset(); + kvpair_ = nullptr; + ret_code_ = OB_SUCCESS; + data_length_ = 0; + block_index_ = -1; + flush_seq_ = -1; + create_ts_ = -1; + is_io_finished_ = false; + fast_flush_tree_page_ = false; + task_state_ = ObTmpFileFlushTaskState::TFFT_INITED; + flush_infos_.reset(); +} + +int ObTmpFileFlushTask::prealloc_block_buf() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("prealloc block buf twice", KR(ret), KPC(this)); + } else if (OB_FAIL(ObTmpBlockCache::get_instance().prealloc_block( + ObTmpBlockCacheKey(block_index_, MTL_ID()), inst_handle_, kvpair_, block_handle_))) { + LOG_WARN("fail to prealloc block", KR(ret), K(block_index_), K(MTL_ID())); + } + return ret; +} + +int ObTmpFileFlushTask::write_one_block() +{ + int ret = OB_SUCCESS; + handle_.reset(); + + blocksstable::ObMacroBlockWriteInfo write_info; + write_info.io_desc_.set_wait_event(2); // TODO: 检查是否需要用临时文件自己的event + write_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); + write_info.io_desc_.set_sys_module_id(ObIOModule::TMP_TENANT_MEM_BLOCK_IO); + write_info.buffer_ = get_data_buf(); + write_info.size_ = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + write_info.offset_ = 0; + + if (FAILEDx(blocksstable::ObBlockManager::async_write_block(write_info, handle_))) { + LOG_ERROR("fail to async write block", KR(ret), K(write_info)); + } else if (OB_FAIL(OB_SERVER_BLOCK_MGR.update_write_time(handle_.get_macro_id(), + true/*update_to_max_time)*/))){ // update to max time to skip bad block inspect + LOG_WARN("failed to update write time", KR(ret), K(handle_)); + } + return ret; +} + +// async wait, return OB_EAGAIN immediately if IO is not finished +int ObTmpFileFlushTask::wait_macro_block_handle() +{ + int ret = OB_SUCCESS; + int64_t wait_timeout_ms = 0; // timeout == 0 for async wait + if (OB_FAIL(handle_.wait(wait_timeout_ms))) { + if (OB_EAGAIN == ret) { + // do nothing + } else { + atomic_set_ret_code(ret); + atomic_set_io_finished(true); + LOG_WARN("fail to wait macro block handle", KR(ret), KPC(this)); + ret = OB_SUCCESS; + } + } else { + atomic_set_ret_code(OB_SUCCESS); + atomic_set_io_finished(true); + LOG_DEBUG("macro block handle io finished", KR(ret), KPC(this)); + } + return ret; +} + +int64_t ObTmpFileFlushTask::get_total_page_num() const +{ + const int64_t PAGE_SIZE = ObTmpFileGlobal::PAGE_SIZE; + return upper_align(data_length_, PAGE_SIZE) / PAGE_SIZE; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_flush_ctx.h b/src/storage/tmp_file/ob_tmp_file_flush_ctx.h new file mode 100644 index 0000000000..ad7a8c8ca3 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_ctx.h @@ -0,0 +1,305 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_CTX_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_CTX_H_ + +#include "lib/queue/ob_link_queue.h" +#include "lib/utility/ob_print_utils.h" +#include "storage/tmp_file/ob_tmp_file_block_manager.h" +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_flush_list_iterator.h" +#include "storage/tmp_file/ob_tmp_file_meta_tree.h" +#include "storage/tmp_file/ob_tmp_file_thread_job.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileFlushTG; + +struct ObTmpFileDataFlushContext +{ +public: + ObTmpFileDataFlushContext () + : is_valid_(false), + has_flushed_last_partially_written_page_(false), + flushed_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flushed_page_virtual_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID), + next_flush_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + next_flush_page_virtual_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID) {} + + ~ObTmpFileDataFlushContext () { reset(); } + void reset() { + is_valid_ = false; + has_flushed_last_partially_written_page_ = false; + flushed_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flushed_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + next_flush_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + next_flush_page_virtual_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } + OB_INLINE bool is_valid() const { return is_valid_; } + OB_INLINE void set_is_valid(const bool is_valid) { is_valid_ = is_valid; } + OB_INLINE bool has_flushed_last_partially_written_page() const { + return has_flushed_last_partially_written_page_; + } + OB_INLINE void set_has_flushed_last_partially_written_page(bool has_flushed_last_partially_written_page) { + has_flushed_last_partially_written_page_ = has_flushed_last_partially_written_page; + } + OB_INLINE uint32_t get_flushed_page_id() const { return flushed_page_id_; } + OB_INLINE void set_flushed_page_id(const uint32_t flushed_page_id) { flushed_page_id_ = flushed_page_id; } + OB_INLINE int64_t get_flushed_page_virtual_id() const { + return flushed_page_virtual_id_; + } + OB_INLINE void set_flushed_page_virtual_id(int64_t virtual_page_id) { + flushed_page_virtual_id_ = virtual_page_id; + } + OB_INLINE uint32_t get_next_flush_page_id() const { return next_flush_page_id_; } + OB_INLINE void set_next_flush_page_id(const uint32_t next_flush_page_id) { + next_flush_page_id_ = next_flush_page_id; + } + OB_INLINE void set_next_flush_page_virtual_id(const int64_t virtual_page_id) { + next_flush_page_virtual_id_ = virtual_page_id; + } + OB_INLINE int64_t get_next_flush_page_virtual_id() const { return next_flush_page_virtual_id_; } + TO_STRING_KV(K(is_valid_), K(has_flushed_last_partially_written_page_), K(flushed_page_id_), + K(flushed_page_virtual_id_), + K(next_flush_page_id_), K(next_flush_page_virtual_id_)); +public: + bool is_valid_; + bool has_flushed_last_partially_written_page_; + uint32_t flushed_page_id_; + int64_t flushed_page_virtual_id_; + uint32_t next_flush_page_id_; + int64_t next_flush_page_virtual_id_; +}; + +struct ObTmpFileSingleFlushContext +{ +public: + ObTmpFileSingleFlushContext() : file_handle_(), data_ctx_(), meta_ctx_() {} + ObTmpFileSingleFlushContext(ObTmpFileHandle file_handle) : file_handle_(file_handle), data_ctx_(), meta_ctx_() {} + TO_STRING_KV(K(file_handle_), K(data_ctx_), K(meta_ctx_)); +public: + ObTmpFileHandle file_handle_; + ObTmpFileDataFlushContext data_ctx_; + ObTmpFileTreeFlushContext meta_ctx_; +}; + +// The ObTmpFileFlushManager maintains multiple file flushing contexts during a round of flushing +class ObTmpFileBatchFlushContext +{ +public: + ObTmpFileBatchFlushContext() + : is_inited_(), + fail_too_many_(false), + expect_flush_size_(0), + actual_flush_size_(0), + flush_seq_ctx_(), + flush_monitor_ptr_(nullptr), + state_(ObTmpFileGlobal::FlushCtxState::FSM_F1), + iter_(), + file_ctx_hash_(), + flush_failed_array_() + { + } + ~ObTmpFileBatchFlushContext() { destroy(); } + int init(); + int prepare_flush_ctx(const int64_t expect_flush_size, + ObTmpFileFlushPriorityManager *prio_mgr, + ObTmpFileFlushMonitor *flush_monitor); + int clear_flush_ctx(ObTmpFileFlushPriorityManager &prio_mgr); + void destroy(); + void record_flush_stage(); + void record_flush_task(const int64_t data_length); + void update_ctx_by_flush_task(const ObTmpFileFlushTask &flush_task); +public: + static const int64_t MAX_COPY_FAIL_COUNT = 512; + typedef hash::ObHashMap ObTmpFileFlushCtxHash; + struct ObTmpFileFlushFailRecord + { + public: + ObTmpFileFlushFailRecord() : is_meta_(false), file_handle_() {} + ObTmpFileFlushFailRecord(bool is_meta, ObTmpFileHandle file_handle) : is_meta_(is_meta), file_handle_(file_handle) {} + TO_STRING_KV(K(is_meta_), K(file_handle_)); + public: + bool is_meta_; + ObTmpFileHandle file_handle_; + }; + struct FlushSequenceContext + { + FlushSequenceContext() : create_flush_task_cnt_(0), send_io_succ_cnt_(0), flush_sequence_(0) {} + int64_t create_flush_task_cnt_; // created flush task number in one round + int64_t send_io_succ_cnt_; + int64_t flush_sequence_; // increate flush seq only when all task in this round send io succ + TO_STRING_KV(K(create_flush_task_cnt_), K(send_io_succ_cnt_), K(flush_sequence_)); + }; + struct RemoveFileOp + { + public: + RemoveFileOp(ObTmpFileFlushPriorityManager &flush_priority_mgr) : flush_priority_mgr_(flush_priority_mgr) {} + int operator () (hash::HashMapPair &kv); + private: + ObTmpFileFlushPriorityManager &flush_priority_mgr_; + }; + bool can_clear_flush_ctx() const + { + return flush_seq_ctx_.send_io_succ_cnt_ == flush_seq_ctx_.create_flush_task_cnt_ && + flush_seq_ctx_.send_io_succ_cnt_ > 0; + } + OB_INLINE void set_fail_too_many(const bool fail_too_many) { fail_too_many_ = fail_too_many; } + OB_INLINE bool is_fail_too_many() { return fail_too_many_; } + OB_INLINE ObTmpFileFlushListIterator &get_flush_list_iterator() { return iter_; } + OB_INLINE void set_expect_flush_size(const int64_t flush_size) { expect_flush_size_ = flush_size; } + OB_INLINE int64_t get_expect_flush_size() { return expect_flush_size_; } + OB_INLINE void add_actual_flush_size(const int64_t flush_size) { actual_flush_size_ += flush_size; } + OB_INLINE int64_t get_actual_flush_size() { return actual_flush_size_; } + OB_INLINE bool is_inited() { return is_inited_; } + OB_INLINE void set_state(const FlushCtxState state) { state_ = state; } + OB_INLINE FlushCtxState get_state() { return state_; } + OB_INLINE ObTmpFileFlushCtxHash &get_file_ctx_hash() { return file_ctx_hash_; } + OB_INLINE ObArray &get_flush_failed_array() { return flush_failed_array_; } + OB_INLINE ObTmpFileFlushMonitor *get_flush_monitor() { return flush_monitor_ptr_; } + OB_INLINE void inc_create_flush_task_cnt() { ++flush_seq_ctx_.create_flush_task_cnt_; } + OB_INLINE int64_t get_flush_sequence() { return flush_seq_ctx_.flush_sequence_; } + TO_STRING_KV(K(is_inited_), K(fail_too_many_), K(expect_flush_size_), K(actual_flush_size_), K(flush_seq_ctx_), K(state_), K(iter_)); +private: + bool is_inited_; + bool fail_too_many_; // indicate wether the number of file copy failures exceeds MAX_COPY_FAIL_COUNT in one round. + int64_t expect_flush_size_; + int64_t actual_flush_size_; + FlushSequenceContext flush_seq_ctx_; + ObTmpFileFlushMonitor *flush_monitor_ptr_; + FlushCtxState state_; + ObTmpFileFlushListIterator iter_; + ObTmpFileFlushCtxHash file_ctx_hash_; // maintain the data/meta flushing offset for each file in one round + ObArray flush_failed_array_; // record files that failed to copy data/meta pages, re-insert them into flush priority mgr + // after the iteration ends to prevent these files from appearing multiple times in flush iterator. +}; + +// represent up to 2MB flush information of a file, +// if a macro block contains multiple tmp files, it will include multiple flush infos +struct ObTmpFileFlushInfo +{ +public: + ObTmpFileFlushInfo(); + ~ObTmpFileFlushInfo() { reset(); } + void reset(); + bool has_data() const { return flush_data_page_num_ > 0; } + bool has_meta() const { return !flush_meta_page_array_.empty(); } + TO_STRING_KV(K(fd_), K(batch_flush_idx_), K(has_last_page_lock_), K(insert_meta_tree_done_), K(update_meta_data_done_), + K(flush_data_page_disk_begin_id_), K(flush_data_page_num_), K(flush_virtual_page_id_), K(file_size_), + K(flush_meta_page_array_), KP(file_handle_.get())); +public: + int64_t fd_; + int64_t batch_flush_idx_; // during one round of flushing, multiple FlushInfo may be generated. + // records the sequence number to which this info belongs (starting from 0). + bool has_last_page_lock_; // indicate the last page is in flushing and holds last_page_lock_ in file + bool insert_meta_tree_done_; // indicate the insertion of the corresponding data item into the meta tree is completed + bool update_meta_data_done_; // indicate the file metadata or metadata tree update is completed + ObTmpFileHandle file_handle_; + + // information for updating data + int64_t flush_data_page_disk_begin_id_; // record begin page id in the macro block, for updating meta tree item + int64_t flush_data_page_num_; + int64_t flush_virtual_page_id_; // record virtual_page_id while copying data, pass to meta tree while inserting items + int64_t file_size_; // if file_size > 0, it means the last page is in flushing + // information for updating meta tree + ObArray flush_meta_page_array_; +}; + +// Each ObTmpFileFlushTask corresponds to a flushing macro block, which can be exclusively used by one file +// or shared by multiple files based on observer config. The task internally maintains the state machine, +// and the ObTmpFileFlushManager continuously advances each task to TFFT_FINISH terminal state. +// Each task that are higher than TFFT_FILL_BLOCK_BUF will be retried if errors occurred. +struct ObTmpFileFlushTask : public common::ObSpLinkQueue::Link +{ +public: + ObTmpFileFlushTask(ObIAllocator &task_allocator); + ~ObTmpFileFlushTask() { destroy(); } + enum ObTmpFileFlushTaskState + { + TFFT_INITED = 0, + TFFT_ALLOC_BLOCK_BUF = 1, + TFFT_CREATE_BLOCK_INDEX = 2, + TFFT_FILL_BLOCK_BUF = 3, + TFFT_INSERT_META_TREE = 4, + TFFT_ASYNC_WRITE = 5, + TFFT_WAIT = 6, + TFFT_FINISH = 7, + }; +public: + void destroy(); + int prealloc_block_buf(); + int write_one_block(); + int wait_macro_block_handle(); + int64_t get_total_page_num() const; + OB_INLINE ObKVCacheInstHandle& get_inst_handle() { return inst_handle_; } + OB_INLINE ObKVCachePair*& get_kvpair() { return kvpair_; } + OB_INLINE ObTmpBlockValueHandle& get_block_handle() { return block_handle_; } + OB_INLINE bool is_valid() const { return OB_NOT_NULL(get_data_buf()); } + OB_INLINE bool is_full() const { return data_length_ == OB_SERVER_BLOCK_MGR.get_macro_block_size(); } + OB_INLINE char *get_data_buf() const { return block_handle_.value_ == nullptr ? nullptr : block_handle_.value_->get_buffer(); } + OB_INLINE void atomic_set_ret_code(int ret_code) { ATOMIC_SET(&ret_code_, ret_code); } + OB_INLINE int atomic_get_ret_code() const { return ATOMIC_LOAD(&ret_code_); } + OB_INLINE void set_data_length(const int64_t len) { data_length_ = len; } + OB_INLINE int64_t get_data_length() const { return data_length_; } + OB_INLINE void set_block_index(const int64_t block_index) { block_index_ = block_index; } + OB_INLINE int64_t get_block_index() const { return block_index_; } + OB_INLINE void set_flush_seq(const int64_t flush_seq) { flush_seq_ = flush_seq; } + OB_INLINE int64_t get_flush_seq() const { return flush_seq_; } + OB_INLINE void set_create_ts(const int64_t create_ts) { create_ts_ = create_ts; } + OB_INLINE int64_t get_create_ts() const { return create_ts_; } + OB_INLINE void atomic_set_io_finished(const bool is_finished) { ATOMIC_SET(&is_io_finished_, is_finished); } + OB_INLINE bool atomic_get_io_finished() const { return ATOMIC_LOAD(&is_io_finished_); } + OB_INLINE void set_is_fast_flush_tree(const bool is_fast_flush_tree) { fast_flush_tree_page_ = is_fast_flush_tree; } + OB_INLINE bool get_is_fast_flush_tree() const { return fast_flush_tree_page_; } + OB_INLINE void set_state(const ObTmpFileFlushTaskState state) { task_state_ = state; } + OB_INLINE ObTmpFileFlushTaskState get_state() const { return task_state_; } + OB_INLINE void set_tmp_file_block_handle(const ObTmpFileBlockHandle &tfb_handle) { tmp_file_block_handle_ = tfb_handle; } + OB_INLINE ObTmpFileBlockHandle &get_tmp_file_block_handle() { return tmp_file_block_handle_; } + OB_INLINE void set_macro_block_handle(const blocksstable::ObMacroBlockHandle &handle) { handle_ = handle; } + OB_INLINE blocksstable::ObMacroBlockHandle &get_macro_block_handle() { return handle_; } + OB_INLINE ObArray &get_flush_infos() { return flush_infos_; } + OB_INLINE int64_t get_next_free_page_id() { return get_total_page_num(); } + OB_INLINE bool check_buf_range_valid(const char* buffer, const int64_t length) const + { + return buffer != nullptr && get_data_buf() != nullptr && + buffer >= get_data_buf() && buffer + length <= get_data_buf() + OB_SERVER_BLOCK_MGR.get_macro_block_size(); + } + TO_STRING_KV(KP(this), KP(kvpair_), K(ret_code_), K(data_length_), + K(block_index_), K(flush_seq_), K(create_ts_), K(is_io_finished_), + K(fast_flush_tree_page_), K(task_state_), K(tmp_file_block_handle_), K(flush_infos_)); +private: + ObKVCacheInstHandle inst_handle_; + ObKVCachePair *kvpair_; + ObTmpBlockValueHandle block_handle_; + int ret_code_; + int64_t data_length_; // data length (including padding to make length upper align to page size) + int64_t block_index_; // tmp file block logical index in ObTmpFileBlockManager + int64_t flush_seq_; // flush sequence, for verification purpose + int64_t create_ts_; + bool is_io_finished_; + bool fast_flush_tree_page_; // indicate the task requires fast flush tree pages + ObTmpFileFlushTaskState task_state_; + ObTmpFileBlockHandle tmp_file_block_handle_;// hold a reference to the corresponding tmp file block to prevent it from being released + blocksstable::ObMacroBlockHandle handle_; + ObArray flush_infos_; // multi file flush into one block if size > 0 + ObIAllocator &task_allocator_; // ref to ObTmpFilePageCacheController::task_allocator_, used to free data_buf_ +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_CTX_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.cpp b/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.cpp new file mode 100644 index 0000000000..7ca0b8eb18 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.cpp @@ -0,0 +1,776 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "lib/utility/ob_sort.h" +#include "storage/tmp_file/ob_tmp_file_flush_list_iterator.h" + +namespace oceanbase +{ +namespace tmp_file +{ +ObTmpFileFlushListIterator::ObTmpFileFlushListIterator() : + is_inited_(false), files_(), dirs_(), cur_caching_list_is_meta_(false), + cur_caching_list_idx_(ObTmpFileFlushPriorityManager::FileList::L1), + cur_iter_dir_idx_(-1), cur_iter_file_idx_(-1), + cached_file_num_(0), cached_dir_num_(0) +{} + +ObTmpFileFlushListIterator::~ObTmpFileFlushListIterator() +{ + destroy(); +} + +int ObTmpFileFlushListIterator::init(ObTmpFileFlushPriorityManager *prio_mgr) +{ + int ret = OB_SUCCESS; + if (is_inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_ISNULL(prio_mgr)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(prio_mgr)); + } else if (FALSE_IT(files_.set_attr(ObMemAttr(MTL_ID(), "TFFlushIterFile")))) { + } else if (OB_FAIL(files_.prepare_allocate(MAX_CACHE_NUM))) { + LOG_WARN("fail to prepare allocate", KR(ret)); + } else if (FALSE_IT(dirs_.set_attr(ObMemAttr(MTL_ID(), "TFFlushIterDir")))) { + } else if (OB_FAIL(dirs_.prepare_allocate(MAX_CACHE_NUM))) { + LOG_WARN("fail to prepare allocate", KR(ret)); + } else { + is_inited_ = true; + prio_mgr_ = prio_mgr; + } + return ret; +} + +int ObTmpFileFlushListIterator::clear() +{ + int ret = OB_SUCCESS; + + // reinsert unused cached file into flush list + FlushCtxState cur_stage = cal_current_flush_stage_(); + if (cur_stage < FlushCtxState::FSM_F1 || cur_stage >= FlushCtxState::FSM_FINISHED) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("unexpected flush stage", KR(ret), K(cur_stage)); + } else if (0 == cached_file_num_) { + // no need to reinsert files, do nothing + } else if (FlushCtxState::FSM_F1 == cur_stage) { + if (OB_FAIL(reinsert_files_into_flush_list_(cur_iter_file_idx_, cached_file_num_ - 1))){ + LOG_ERROR("fail to reinsert files into flush list", KR(ret), K(cur_iter_file_idx_), K(cached_file_num_)); + } + } else { + for (int64_t i = cur_iter_dir_idx_; OB_SUCC(ret) && (i >= 0 && i < cached_dir_num_); i++) { + if (OB_UNLIKELY(!dirs_[i].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("uninitialized dir is unexpected", KR(ret), K(dirs_[i])); + } else { + int64_t start_file_idx = dirs_[i].start_file_idx_; + int64_t end_file_idx = dirs_[i].end_file_idx_; + if (i == cur_iter_dir_idx_) { + start_file_idx = cur_iter_file_idx_; + if (start_file_idx < dirs_[i].start_file_idx_ || start_file_idx > dirs_[i].end_file_idx_) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("start file idx not in cur dir range", KR(ret), K(start_file_idx), K(dirs_[i])); + } + } + if (FAILEDx(reinsert_files_into_flush_list_(start_file_idx, end_file_idx))) { + LOG_ERROR("fail to reinsert files into flush list", + KR(ret), K(i), K(dirs_[i]), K(start_file_idx), K(end_file_idx)); + } + } + } + } + + if (OB_SUCC(ret)) { + cur_caching_list_is_meta_ = false; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L1; + cur_iter_dir_idx_ = -1; + cur_iter_file_idx_ = -1; + cached_file_num_ = 0; + cached_dir_num_ = 0; + for (int64_t i = 0; i < files_.count(); ++i) { + files_[i].reset(); + } + for (int64_t i = 0; i < dirs_.count(); ++i) { + dirs_[i].reset(); + } + } + + return ret; +} + +int ObTmpFileFlushListIterator::reset() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + if (OB_FAIL(clear())) { + LOG_WARN("fail to clear", KR(ret)); + } + is_inited_ = false; + } + return ret; +} + +void ObTmpFileFlushListIterator::destroy() +{ + reset(); +} + +int ObTmpFileFlushListIterator::reinsert_files_into_flush_list_(const int64_t start_file_idx, + const int64_t end_file_idx) +{ + int ret = OB_SUCCESS; + for (int64_t i = start_file_idx; OB_SUCC(ret) && (i >= 0 && i <= end_file_idx); i++) { + if (OB_UNLIKELY(i < 0 || i >= cached_file_num_ || cached_file_num_ > files_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid file idx", KR(ret), K(i), K(cached_file_num_), K(files_)); + } else if (OB_UNLIKELY(!files_[i].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("uninitialized file is unexpected", KR(ret), K(i), K(files_[i])); + } else { + ObTmpFileHandle &file_handle = files_[i].file_handle_; + if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret)); + } else if (files_[i].is_meta_ && file_handle.get()->is_in_meta_flush_list()) { + // do nothing, because meta flush node may be re-inserted after + // tmp file insert meta tree item; do not handle data flush node here + // because data node will not be re-inserted during flushing procedure + } else if (OB_FAIL(file_handle.get()->reinsert_flush_node(files_[i].is_meta_))) { + LOG_WARN("fail to reinsert flush node", KR(ret), K(files_[i])); + } + } + } + return ret; +} + +int ObTmpFileFlushListIterator::next(const FlushCtxState iter_stage, bool &is_meta, ObTmpFileHandle &file_handle) +{ + int ret = OB_SUCCESS; + FlushCtxState cur_stage = FlushCtxState::FSM_FINISHED; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (OB_UNLIKELY(dirs_.count() != MAX_CACHE_NUM || + cached_dir_num_ > MAX_CACHE_NUM || + cached_dir_num_ < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(files_.count()), K(cached_dir_num_)); + } else if (OB_UNLIKELY(files_.count() != MAX_CACHE_NUM || + cached_file_num_ > MAX_CACHE_NUM || + cached_file_num_ < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(files_.count()), K(cached_file_num_)); + } else if (OB_UNLIKELY(FlushCtxState::FSM_FINISHED <= iter_stage || iter_stage < FlushCtxState::FSM_F1)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid iter_stage", KR(ret), K(iter_stage)); + } else if (FALSE_IT(cur_stage = cal_current_flush_stage_())) { + } else if (OB_UNLIKELY(FlushCtxState::FSM_FINISHED <= cur_stage || FlushCtxState::FSM_F1 > cur_stage)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_stage)); + } else if (OB_UNLIKELY(cur_stage > iter_stage)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid iter_stage", KR(ret), K(iter_stage), K(cur_stage)); + } else if (cur_stage < iter_stage || + (FlushCtxState::FSM_F1 == cur_stage && cur_iter_file_idx_ == cached_file_num_) || + (FlushCtxState::FSM_F1 < cur_stage && cur_iter_dir_idx_ == cached_dir_num_)) { + // code will run here when: + // 1. expected iterating flush stage is over than current flush stage; + // 2. all cached file has been iterated + if (OB_FAIL(clear())) { + LOG_WARN("fail to clear", KR(ret)); + } + } + + if (OB_FAIL(ret)) { + } else if (0 == cached_file_num_ && OB_FAIL(cache_files_(iter_stage))) { + if (OB_ITER_END == ret) { + LOG_DEBUG("fail to cache files", KR(ret)); + } else { + LOG_WARN("fail to cache files", KR(ret)); + } + } else if (OB_FAIL(check_cur_idx_status_())) { + LOG_WARN("fail to check cur idx status", KR(ret)); + } else { + is_meta = files_[cur_iter_file_idx_].is_meta_; + file_handle = files_[cur_iter_file_idx_].file_handle_; + if (ObTmpFileFlushPriorityManager::FileList::L1 == cur_caching_list_idx_) { + if (OB_FAIL(advance_big_file_idx_())) { + LOG_WARN("fail to advance big file idx", KR(ret)); + } + } else if (OB_FAIL(advance_small_file_idx_())) { + LOG_WARN("fail to advance small file idx", KR(ret)); + } + } + LOG_DEBUG("try to get next file", KR(ret), K(iter_stage), K(cur_stage), K(is_meta), + K(cur_iter_file_idx_), K(cached_file_num_), K(file_handle)); + return ret; +} + +int ObTmpFileFlushListIterator::cache_files_(const FlushCtxState iter_stage) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushPriorityManager::FileList end_list_idx = ObTmpFileFlushPriorityManager::FileList::MAX; + const int64_t target_cache_file_num = iter_stage == FlushCtxState::FSM_F1 ? + BIG_FILE_CACHE_NUM : MAX_CACHE_NUM; + ObArray file_handles; + + if (OB_UNLIKELY(0 != cached_dir_num_ || 0 != cached_file_num_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid cached num", KR(ret), K(cached_dir_num_), K(cached_file_num_)); + } else if (OB_FAIL(init_caching_list_with_flush_stage_(iter_stage))) { + LOG_WARN("fail to init caching list", KR(ret), K(iter_stage)); + } else if (OB_FAIL(acquire_final_list_of_flush_stage_(iter_stage, end_list_idx))) { + LOG_WARN("fail to acquire final list of flush stage", KR(ret), K(iter_stage)); + } else if (OB_FAIL(file_handles.prepare_allocate_and_keep_count(target_cache_file_num))) { + LOG_WARN("fail to prepare allocate", KR(ret), K(target_cache_file_num)); + } else { // pop enough files from priority manager for caching + int64_t remain_cache_file_num = target_cache_file_num; + bool cache_over = false; + while (OB_SUCC(ret) && !cache_over && remain_cache_file_num > 0) { + int64_t actual_cache_file_num = 0; + if (OB_FAIL(prio_mgr_->popN_from_file_list(cur_caching_list_is_meta_, cur_caching_list_idx_, + remain_cache_file_num, actual_cache_file_num, + file_handles))) { + LOG_WARN("fail to pop N from file list", KR(ret), K(cur_caching_list_idx_)); + } else if (FALSE_IT(remain_cache_file_num -= actual_cache_file_num)) { + } else if (OB_UNLIKELY(remain_cache_file_num < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected cache file num", KR(ret), K(target_cache_file_num), + K(remain_cache_file_num), K(actual_cache_file_num)); + } else if (0 == remain_cache_file_num) { // cache enough files + cache_over = true; + } else { // remain_cache_file_num > 0 + if (cur_caching_list_idx_ == end_list_idx) { // has reached the last list of this flush stage + if (file_handles.empty()) { + ret = OB_ITER_END; + LOG_DEBUG("iter end in current flush stage", KR(ret), K(iter_stage), + K(cur_caching_list_idx_), K(remain_cache_file_num)); + } else { // cache files successful, but the num is not enough + cache_over = true; + } + } else if (OB_FAIL(advance_caching_list_idx_())) { // pop files of the next list of this flush stage + LOG_WARN("fail to advance caching list idx", KR(ret)); + } + } + } // end while + } + + if (OB_SUCC(ret)) { + if (ObTmpFileFlushPriorityManager::FileList::L1 == cur_caching_list_idx_) { + if (OB_FAIL(cache_big_files_(file_handles))) { + LOG_WARN("fail to cache big files", KR(ret)); + } + } else if (OB_FAIL(cache_small_files_(file_handles))) { + LOG_WARN("fail to cache big files", KR(ret)); + } + } + + if (OB_FAIL(ret)) { + int tmp_ret = OB_SUCCESS; + for (int64_t i = 0; OB_LIKELY(OB_SUCCESS == tmp_ret) && i < file_handles.count(); ++i) { + if (OB_ISNULL(file_handles[i].get())) { + // could not happen, just skip + } else if (OB_TMP_FAIL(file_handles[i].get()->reinsert_flush_node(cur_caching_list_is_meta_))) { + LOG_WARN("fail to reinsert flush node", KR(tmp_ret), K(i), K(file_handles[i])); + } + } + } + return ret; +} + +int ObTmpFileFlushListIterator::cache_big_files_(const ObArray &file_handles) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(build_file_wrappers_(file_handles))) { + LOG_WARN("fail to build file wrappers", KR(ret)); + } else { + cur_iter_file_idx_ = 0; + cur_iter_dir_idx_ = -1; // no need to aggregate files into dir + cached_dir_num_ = 0; + } + return ret; +} + +int ObTmpFileFlushListIterator::cache_small_files_(const ObArray &file_handles) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(build_file_wrappers_(file_handles))) { + LOG_WARN("fail to build file wrappers", KR(ret)); + } else if (OB_FAIL(build_dir_wrappers_())) { + LOG_WARN("fail to build dir wrappers", KR(ret)); + } else { + cur_iter_file_idx_ = dirs_[0].start_file_idx_; + cur_iter_dir_idx_ = 0; + } + return ret; +} + +int ObTmpFileFlushListIterator::build_file_wrappers_(const ObArray &file_handles) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(file_handles.empty())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(file_handles.empty())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < file_handles.count(); ++i) { + if (i >= files_.count()) { + ret = OB_ERROR_OUT_OF_RANGE; + LOG_ERROR("index is out of range", KR(ret), K(i), K(files_)); + } else if (OB_FAIL(files_[i].init(cur_caching_list_is_meta_, file_handles[i]))) { + LOG_WARN("fail to init tmp file handle wrapper", KR(ret), K(i), K(file_handles[i]), K(files_[i])); + + for (int64_t j = 0; j < i; ++j) { + files_[j].reset(); + } + } + } + + if (OB_SUCC(ret)) { + lib::ob_sort(files_.begin(), files_.begin() + file_handles.count()); + cached_file_num_ = file_handles.count(); + } + } + return ret; +} + +int ObTmpFileFlushListIterator::build_dir_wrappers_() +{ + int ret = OB_SUCCESS; + int cached_dir_num = 0; + + if (OB_UNLIKELY(cached_file_num_ <= 0 || cached_file_num_ > files_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid cached file num", KR(ret), K(cached_file_num_)); + } else { // we assume that files_ has been initialized and sorted + int64_t dir_id = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + int64_t start_file_idx = -1; + int64_t end_file_idx = -1; + int64_t page_num = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < cached_file_num_; ++i) { + ObSharedNothingTmpFile *file = nullptr; + int64_t file_dirty_page_num = 0; + if (OB_UNLIKELY(!files_[i].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp file wrapper is not inited", KR(ret), K(files_[i])); + } else if (OB_ISNULL(file = files_[i].file_handle_.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file ptr is null", K(ret), KP(file)); + } else if (OB_FAIL(get_flushing_file_dirty_page_num_(*file, file_dirty_page_num))) { + LOG_WARN("fail to get flushing file dirty page num", KR(ret)); + } else if (file->get_dir_id() != dir_id) { + if (0 != i) { + end_file_idx = i - 1; + if (cached_dir_num >= dirs_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("cached_dir_num is equal or bigger than dirs_.count", + KR(ret), K(cached_dir_num), K(dirs_)); + } else if (OB_FAIL(dirs_[cached_dir_num].init(cur_caching_list_is_meta_, page_num, + start_file_idx, end_file_idx))) { + LOG_WARN("fail to init tmp file dir wrapper", KR(ret), K(i), K(dirs_[cached_dir_num]), K(page_num), + K(start_file_idx), K(end_file_idx)); + } else { + cached_dir_num++; + } + } + + if (OB_SUCC(ret)) { + dir_id = file->get_dir_id(); + start_file_idx = i; + page_num = file_dirty_page_num; + } + } else { + page_num += file_dirty_page_num; + } + + if (OB_SUCC(ret) && cached_file_num_ - 1 == i) { + end_file_idx = i; + if (cached_dir_num >= dirs_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("cached_dir_num is equal or bigger than dirs_.count", + KR(ret), K(cached_dir_num), K(dirs_)); + } else if (OB_FAIL(dirs_[cached_dir_num].init(cur_caching_list_is_meta_, page_num, + start_file_idx, end_file_idx))) { + LOG_WARN("fail to init tmp file dir wrapper", KR(ret), K(page_num), + K(start_file_idx), K(end_file_idx)); + } else { + cached_dir_num++; + } + } + } // end for + + } + + if (OB_FAIL(ret)) { + for (int64_t i = 0; i < cached_dir_num; i++) { + dirs_[i].reset(); + } + for (int64_t i = 0; i < cached_file_num_; i++) { + files_[i].reset(); + cached_file_num_ = 0; + } + } else { + lib::ob_sort(dirs_.begin(), dirs_.begin() + cached_dir_num); + cached_dir_num_ = cached_dir_num; + } + + return ret; +} + +int ObTmpFileFlushListIterator::get_flushing_file_dirty_page_num_(const ObSharedNothingTmpFile &file, int64_t &page_num) +{ + int ret = OB_SUCCESS; + FlushCtxState cur_stage = cal_current_flush_stage_(); + if (OB_UNLIKELY(cur_stage < FlushCtxState::FSM_F1 || cur_stage > FlushCtxState::FSM_F5)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected flush stage", KR(ret), K(cur_stage)); + } else if (cur_stage <= FlushCtxState::FSM_F3) { + ObSharedNothingTmpFile &mutable_file_ref = const_cast(file); + int64_t dirty_page_size = mutable_file_ref.get_dirty_data_page_size_with_lock(); + page_num = upper_align(dirty_page_size, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + } else { + int64_t non_rightmost_dirty_page_num = 0; + int64_t rightmost_dirty_page_num = 0; + ObSharedNothingTmpFile &mutable_file_ref = const_cast(file); + mutable_file_ref.get_dirty_meta_page_num_with_lock(non_rightmost_dirty_page_num, rightmost_dirty_page_num); + if (cur_stage == FlushCtxState::FSM_F4) { + page_num = non_rightmost_dirty_page_num; + } else if (cur_stage == FlushCtxState::FSM_F5) { + page_num = rightmost_dirty_page_num; + } + } + return ret; +} + +int ObTmpFileFlushListIterator::advance_caching_list_idx_() +{ + int ret = OB_SUCCESS; + if (!cur_caching_list_is_meta_ && ObTmpFileFlushPriorityManager::FileList::L5 == cur_caching_list_idx_) { + cur_caching_list_is_meta_ = true; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L1; + } else { + switch(cur_caching_list_idx_) { + case ObTmpFileFlushPriorityManager::FileList::L1: + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L2; + break; + case ObTmpFileFlushPriorityManager::FileList::L2: + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L3; + break; + case ObTmpFileFlushPriorityManager::FileList::L3: + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L4; + break; + case ObTmpFileFlushPriorityManager::FileList::L4: + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L5; + break; + default: + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_caching_list_idx_)); + break; + } + } + + return ret; +} + +int ObTmpFileFlushListIterator::check_cur_idx_status_() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(cur_iter_file_idx_ < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_file_idx_)); + } else if (ObTmpFileFlushPriorityManager::FileList::L1 == cur_caching_list_idx_) { + // the file in L1 list will not be flushed with an aggregating dir. + // thus, it is no need to check dir + if (OB_UNLIKELY(cur_iter_file_idx_ >= cached_file_num_ || cached_file_num_ > files_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_file_idx_), K(cached_file_num_)); + } + } else if (OB_UNLIKELY(cur_iter_dir_idx_ < 0 || cur_iter_dir_idx_ >= cached_dir_num_ || + cached_dir_num_ > dirs_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_dir_idx_), K(cached_dir_num_), K(dirs_)); + } else if (OB_UNLIKELY(!dirs_[cur_iter_dir_idx_].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_dir_idx_), K(dirs_[cur_iter_dir_idx_])); + } else if (OB_UNLIKELY(cur_iter_file_idx_ < dirs_[cur_iter_dir_idx_].start_file_idx_ || + cur_iter_file_idx_ > dirs_[cur_iter_dir_idx_].end_file_idx_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_dir_idx_), K(cur_iter_file_idx_), + K(dirs_[cur_iter_dir_idx_])); + } + + if (OB_SUCC(ret) && OB_UNLIKELY(!files_[cur_iter_file_idx_].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_file_idx_), K(files_[cur_iter_file_idx_])); + } + + return ret; +} + +int ObTmpFileFlushListIterator::advance_big_file_idx_() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(ObTmpFileFlushPriorityManager::FileList::L1 != cur_caching_list_idx_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid caching list idx", KR(ret), K(cur_caching_list_idx_)); + } else if (OB_UNLIKELY(cur_iter_file_idx_ < 0 || cur_iter_file_idx_ >= cached_file_num_ || + cached_file_num_ > files_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid file idx", KR(ret), K(cur_iter_file_idx_), K(cached_file_num_), K(files_)); + } else { + files_[cur_iter_file_idx_].reset(); + cur_iter_file_idx_++; + } + return ret; +} + +int ObTmpFileFlushListIterator::advance_small_file_idx_() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(cur_caching_list_idx_ <= ObTmpFileFlushPriorityManager::FileList::L1 || + cur_caching_list_idx_ > ObTmpFileFlushPriorityManager::FileList::L5)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid caching list idx", KR(ret), K(cur_caching_list_idx_)); + } else if (OB_UNLIKELY(cur_iter_dir_idx_ < 0 || cur_iter_dir_idx_ >= cached_dir_num_ || + cached_dir_num_ > dirs_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid dir idx", KR(ret), K(cur_iter_dir_idx_), K(cached_dir_num_), K(dirs_)); + } else if (OB_UNLIKELY(cur_iter_file_idx_ < 0 || cur_iter_file_idx_ >= cached_file_num_ || + cached_file_num_ > files_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_file_idx_), K(cached_file_num_)); + } else if (OB_UNLIKELY(cur_iter_file_idx_ < dirs_[cur_iter_dir_idx_].start_file_idx_ || + cur_iter_file_idx_ > dirs_[cur_iter_dir_idx_].end_file_idx_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid file idx", KR(ret), K(cur_iter_file_idx_), K(dirs_[cur_iter_dir_idx_])); + } else { + files_[cur_iter_file_idx_].reset(); + cur_iter_file_idx_++; + if (cur_iter_file_idx_ > dirs_[cur_iter_dir_idx_].end_file_idx_) { + if (OB_FAIL(advance_dir_idx_())) { + LOG_WARN("fail to advance dir idx", KR(ret)); + } else if (cur_iter_dir_idx_ < cached_dir_num_) { + if (OB_UNLIKELY(!dirs_[cur_iter_dir_idx_].is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(cur_iter_dir_idx_), K(dirs_[cur_iter_dir_idx_])); + } else { + cur_iter_file_idx_ = dirs_[cur_iter_dir_idx_].start_file_idx_; + } + } else { + // iter end + } + } + } + return ret; +} + +int ObTmpFileFlushListIterator::advance_dir_idx_() +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(ObTmpFileFlushPriorityManager::FileList::L1 == cur_caching_list_idx_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid caching list idx", KR(ret), K(cur_caching_list_idx_)); + } else if (OB_UNLIKELY(cur_iter_dir_idx_ < 0 || cur_iter_dir_idx_ >= cached_dir_num_ || + cached_dir_num_ > dirs_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid dir idx", KR(ret), K(cur_iter_dir_idx_), K(cached_dir_num_), K(dirs_)); + } else { + dirs_[cur_iter_dir_idx_].reset(); + cur_iter_dir_idx_++; + } + + return ret; +} + +FlushCtxState ObTmpFileFlushListIterator::cal_current_flush_stage_() +{ + FlushCtxState stage = FlushCtxState::FSM_FINISHED; + if (OB_UNLIKELY(cur_caching_list_idx_ < ObTmpFileFlushPriorityManager::FileList::L1 || + cur_caching_list_idx_ > ObTmpFileFlushPriorityManager::FileList::L5)) { + // stage = FlushCtxState::FSM_FINISHED; + } else if (!cur_caching_list_is_meta_) { + if (cur_caching_list_idx_ == ObTmpFileFlushPriorityManager::FileList::L1) { + stage = FlushCtxState::FSM_F1; + } else if (cur_caching_list_idx_ <= ObTmpFileFlushPriorityManager::FileList::L4) { + stage = FlushCtxState::FSM_F2; + } else if (cur_caching_list_idx_ == ObTmpFileFlushPriorityManager::FileList::L5) { + stage = FlushCtxState::FSM_F3; + } + } else { + if (cur_caching_list_idx_ <= ObTmpFileFlushPriorityManager::FileList::L4) { + stage = FlushCtxState::FSM_F4; + } else if (cur_caching_list_idx_ == ObTmpFileFlushPriorityManager::FileList::L5) { + stage = FlushCtxState::FSM_F5; + } + } + + return stage; +} + +int ObTmpFileFlushListIterator::init_caching_list_with_flush_stage_(const FlushCtxState iter_stage) +{ + int ret = OB_SUCCESS; + if (cal_current_flush_stage_() == iter_stage) { + // no need to change caching list + } else if (iter_stage == FlushCtxState::FSM_F1) { + cur_caching_list_is_meta_ = false; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L1; + } else if (iter_stage == FlushCtxState::FSM_F2) { + cur_caching_list_is_meta_ = false; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L2; + } else if (iter_stage == FlushCtxState::FSM_F3) { + cur_caching_list_is_meta_ = false; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L5; + } else if (iter_stage == FlushCtxState::FSM_F4) { + cur_caching_list_is_meta_ = true; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L1; + } else if (iter_stage == FlushCtxState::FSM_F5) { + cur_caching_list_is_meta_ = true; + cur_caching_list_idx_ = ObTmpFileFlushPriorityManager::FileList::L5; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid stage", KR(ret), K(iter_stage)); + } + + return ret; +} + +int ObTmpFileFlushListIterator::acquire_final_list_of_flush_stage_(const FlushCtxState iter_stage, + ObTmpFileFlushPriorityManager::FileList &list_idx) +{ + int ret = OB_SUCCESS; + switch(iter_stage) { + case FlushCtxState::FSM_F1: + list_idx = ObTmpFileFlushPriorityManager::FileList::L1; + break; + case FlushCtxState::FSM_F2: + list_idx = ObTmpFileFlushPriorityManager::FileList::L4; + break; + case FlushCtxState::FSM_F3: + list_idx = ObTmpFileFlushPriorityManager::FileList::L5; + break; + case FlushCtxState::FSM_F4: + list_idx = ObTmpFileFlushPriorityManager::FileList::L4; + break; + case FlushCtxState::FSM_F5: + list_idx = ObTmpFileFlushPriorityManager::FileList::L5; + break; + default: + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid stage", KR(ret), K(iter_stage)); + } + return ret; +} + +int ObTmpFileFlushListIterator::ObFlushingTmpFileWrapper::init(const bool is_meta, const ObTmpFileHandle &file_handle) +{ + int ret = OB_SUCCESS; + if (is_inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_ISNULL(file_handle.get())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(file_handle.get())); + } else { + is_meta_ = is_meta; + file_handle_ = file_handle; + is_inited_ = true; + } + return ret; +} + +void ObTmpFileFlushListIterator::ObFlushingTmpFileWrapper::reset() +{ + if (IS_INIT) { + is_inited_ = false; + is_meta_ = false; + file_handle_.reset(); + } +} + +bool ObTmpFileFlushListIterator::ObFlushingTmpFileWrapper::operator <(const ObFlushingTmpFileWrapper &other) +{ + int ret = OB_SUCCESS; + bool b_ret = false; + if (OB_UNLIKELY(!other.is_inited_ || !is_inited_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status", KR(ret), K(other), K(*this)); + } else if (OB_ISNULL(other.file_handle_.get()) || OB_ISNULL(file_handle_.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to compare file handle with nullptr", KR(ret), K(file_handle_), K(other.file_handle_)); + } else if (!is_meta_ && other.is_meta_) { + b_ret = true; + } else if (is_meta_ && !other.is_meta_) { + b_ret = false; + } else if (file_handle_.get()->get_dir_id() < other.file_handle_.get()->get_dir_id()) { + b_ret = true; + } else if (file_handle_.get()->get_dir_id() > other.file_handle_.get()->get_dir_id()) { + b_ret = false; + } else if (is_meta_) { + b_ret = file_handle_.get()->get_meta_page_flush_level() < other.file_handle_.get()->get_meta_page_flush_level(); + } else { + b_ret = file_handle_.get()->get_data_page_flush_level() < other.file_handle_.get()->get_data_page_flush_level(); + } + return b_ret; +} + +int ObTmpFileFlushListIterator::ObFlushingTmpFileDirWrapper::init(const bool is_meta, const int64_t page_num, + const int64_t start_file_idx, + const int64_t end_file_idx) +{ + int ret = OB_SUCCESS; + if (is_inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_UNLIKELY(page_num < 0 || start_file_idx < 0 || start_file_idx > end_file_idx)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(page_num), K(start_file_idx), K(end_file_idx)); + } else { + is_inited_ = true; + is_meta_ = is_meta; + page_num_ = page_num; + start_file_idx_ = start_file_idx; + end_file_idx_ = end_file_idx; + } + return ret; +} + +void ObTmpFileFlushListIterator::ObFlushingTmpFileDirWrapper::reset() +{ + if (IS_INIT) { + is_meta_ = false; + page_num_ = 0; + start_file_idx_ = -1; + end_file_idx_ = -1; + is_inited_ = false; + } +} + +bool ObTmpFileFlushListIterator::ObFlushingTmpFileDirWrapper::operator <(const ObFlushingTmpFileDirWrapper &other) +{ + int ret = OB_SUCCESS; + bool b_ret = false; + if (OB_UNLIKELY(!other.is_inited_ || !is_inited_)) { + LOG_WARN("unexpected status", K(other), K(*this)); + } else if (!is_meta_ && other.is_meta_) { + b_ret = true; + } else if (is_meta_ && !other.is_meta_) { + b_ret = false; + } else { + b_ret = page_num_ > other.page_num_; + } + return b_ret; +} +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.h b/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.h new file mode 100644 index 0000000000..28b017d9da --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_list_iterator.h @@ -0,0 +1,105 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_LIST_ITERATOR_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_LIST_ITERATOR_H_ + +#include "storage/tmp_file/ob_tmp_file_flush_priority_manager.h" +#include "storage/tmp_file/ob_tmp_file_global.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +typedef ObTmpFileGlobal::FlushCtxState FlushCtxState; + +class ObTmpFileFlushListIterator +{ +public: + ObTmpFileFlushListIterator(); + ~ObTmpFileFlushListIterator(); + + int init(ObTmpFileFlushPriorityManager *prio_mgr); + int clear(); + int reset(); + void destroy(); + int next(const FlushCtxState iter_stage, bool &is_meta, ObTmpFileHandle &file_handle); + + TO_STRING_KV(K(is_inited_), K(cur_caching_list_idx_), K(cur_caching_list_is_meta_), + K(cur_iter_dir_idx_), K(cur_iter_file_idx_), K(cached_file_num_), K(cached_dir_num_)) +private: + int reinsert_files_into_flush_list_(const int64_t start_file_idx, const int64_t end_file_idx); + FlushCtxState cal_current_flush_stage_(); + int init_caching_list_with_flush_stage_(const FlushCtxState iter_stage); + int acquire_final_list_of_flush_stage_(const FlushCtxState iter_stage, + ObTmpFileFlushPriorityManager::FileList &list_idx); + int cache_files_(const FlushCtxState iter_stage); + int build_file_wrappers_(const ObArray &file_handles); + int build_dir_wrappers_(); + int cache_big_files_(const ObArray &file_handles); + int cache_small_files_(const ObArray &file_handles); + int get_flushing_file_dirty_page_num_(const ObSharedNothingTmpFile &file, int64_t &page_num); + int check_cur_idx_status_(); + int advance_big_file_idx_(); + int advance_small_file_idx_(); + int advance_dir_idx_(); + int advance_caching_list_idx_(); +private: + struct ObFlushingTmpFileWrapper + { + ObFlushingTmpFileWrapper() : is_inited_(false), is_meta_(false), file_handle_() {} + ~ObFlushingTmpFileWrapper() { reset(); }; + int init(const bool is_meta, const ObTmpFileHandle &file_handle); + void reset(); + bool operator <(const ObFlushingTmpFileWrapper &other); + TO_STRING_KV(K(is_inited_), K(is_meta_), K(file_handle_)); + + bool is_inited_; + bool is_meta_; + ObTmpFileHandle file_handle_; + }; + + struct ObFlushingTmpFileDirWrapper + { + ObFlushingTmpFileDirWrapper() : is_inited_(false), is_meta_(false), page_num_(0), + start_file_idx_(-1), end_file_idx_(-1) {} + ~ObFlushingTmpFileDirWrapper() { reset(); }; + int init(const bool is_meta, const int64_t page_num, const int64_t start_file_idx, const int64_t end_file_idx); + void reset(); + bool operator <(const ObFlushingTmpFileDirWrapper &other); + TO_STRING_KV(K(is_inited_), K(is_meta_), K(page_num_), K(start_file_idx_), K(end_file_idx_)); + + bool is_inited_; + bool is_meta_; + int64_t page_num_; + int64_t start_file_idx_; + int64_t end_file_idx_; + }; +private: + static constexpr int64_t MAX_CACHE_NUM = 256; + static constexpr int64_t BIG_FILE_CACHE_NUM = 8; + bool is_inited_; + ObTmpFileFlushPriorityManager *prio_mgr_; + ObArray files_; + ObArray dirs_; + bool cur_caching_list_is_meta_; + ObTmpFileFlushPriorityManager::FileList cur_caching_list_idx_; + int64_t cur_iter_dir_idx_; + int64_t cur_iter_file_idx_; + int64_t cached_file_num_; + int64_t cached_dir_num_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_LIST_ITERATOR_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_flush_manager.cpp b/src/storage/tmp_file/ob_tmp_file_flush_manager.cpp new file mode 100644 index 0000000000..d98eecf5f7 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_manager.cpp @@ -0,0 +1,969 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_flush_manager.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" +#include "storage/blocksstable/ob_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_flush_list_iterator.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +ObTmpFileFlushManager::ObTmpFileFlushManager(ObTmpFilePageCacheController &pc_ctrl) + : is_inited_(false), + flush_ctx_(), + pc_ctrl_(pc_ctrl), + tmp_file_block_mgr_(pc_ctrl.get_tmp_file_block_manager()), + task_allocator_(pc_ctrl.get_task_allocator()), + write_buffer_pool_(pc_ctrl.get_write_buffer_pool()), + evict_mgr_(pc_ctrl.get_eviction_manager()), + flush_priority_mgr_(pc_ctrl.get_flush_priority_mgr()) +{ +} + +int ObTmpFileFlushManager::init() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObTmpFileFlushManager inited twice", KR(ret)); + } else if (OB_FAIL(flush_ctx_.init())) { + STORAGE_LOG(WARN, "fail to init flush ctx", KR(ret)); + } else { + is_inited_ = true; + } + return ret; +} + +void ObTmpFileFlushManager::destroy() +{ + is_inited_ = false; + flush_ctx_.destroy(); +} + +int ObTmpFileFlushManager::alloc_flush_task(ObTmpFileFlushTask *&flush_task) +{ + int ret = OB_SUCCESS; + flush_task = nullptr; + + const int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + void *task_buf = nullptr; + if (OB_ISNULL(task_buf = task_allocator_.alloc(sizeof(ObTmpFileFlushTask)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "fail to allocate memory for flush callback", KR(ret)); + } else { + flush_task = new (task_buf) ObTmpFileFlushTask(task_allocator_); + } + return ret; +} + +int ObTmpFileFlushManager::free_flush_task(ObTmpFileFlushTask *flush_task) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task ptr is null", KR(ret)); + } else { + LOG_DEBUG("free flush task", KPC(flush_task)); + flush_task->~ObTmpFileFlushTask(); + task_allocator_.free(flush_task); + } + return ret; +} + +int ObTmpFileFlushManager::notify_write_back_failed(ObTmpFileFlushTask *flush_task) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task ptr is null", KR(ret)); + } else if (OB_FAIL(tmp_file_block_mgr_.write_back_failed(flush_task->get_block_index()))) { + STORAGE_LOG(ERROR, "fail to notify tmp file block write back failed", KR(ret), KPC(flush_task)); + } + return ret; +} + +// release whole block +int ObTmpFileFlushManager::free_tmp_file_block(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + int64_t block_index = flush_task.get_block_index(); + if (ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX == block_index) { + // do nothing + } else if (OB_FAIL(tmp_file_block_mgr_.release_tmp_file_page(block_index, + 0/*begin_page_id*/, + ObTmpFileGlobal::BLOCK_PAGE_NUMS))) { + STORAGE_LOG(WARN, "fail to remove tmp file block", KR(ret), K(block_index), K(flush_task)); + } + return ret; +} + +void ObTmpFileFlushManager::init_flush_level_() +{ + int64_t dirty_page_percentage = write_buffer_pool_.get_dirty_page_percentage(); + if (pc_ctrl_.is_flush_all_data()) { // only for unittest + flush_ctx_.set_state(FlushCtxState::FSM_F1); + } else { + if (FLUSH_WATERMARK_F1 <= dirty_page_percentage) { + flush_ctx_.set_state(FlushCtxState::FSM_F1); + } else if (FLUSH_WATERMARK_F2 <= dirty_page_percentage) { + flush_ctx_.set_state(FlushCtxState::FSM_F2); + } else if (FLUSH_WATERMARK_F3 <= dirty_page_percentage) { + flush_ctx_.set_state(FlushCtxState::FSM_F3); + } else if (FLUSH_WATERMARK_F4 <= dirty_page_percentage) { + flush_ctx_.set_state(FlushCtxState::FSM_F4); + } else if (FLUSH_WATERMARK_F5 <= dirty_page_percentage) { + flush_ctx_.set_state(FlushCtxState::FSM_F5); + } else { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } + } +} + +void ObTmpFileFlushManager::advance_flush_level_(const int ret_code) +{ + int64_t dirty_page_percentage = write_buffer_pool_.get_dirty_page_percentage(); + + if (pc_ctrl_.is_flush_all_data()) { // only for unittest + if (OB_SUCCESS == ret_code) { + // continue to flush to OB_ITER_END, do nothing + } else if (OB_ITER_END == ret_code) { + inner_advance_flush_level_without_checking_watermark_(); + } else { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } + } else { + if (OB_SUCCESS == ret_code) { + // if lower than low_watermark or reach expect_flush_size, terminate flush process + if (get_low_watermark_(flush_ctx_.get_state()) >= dirty_page_percentage + || flush_ctx_.get_actual_flush_size() >= flush_ctx_.get_expect_flush_size()) { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } + } else if (OB_ITER_END == ret_code) { + inner_advance_flush_level_(); + } else { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } + } + + LOG_DEBUG("advance_flush_level_", K(flush_ctx_)); +} + +int64_t ObTmpFileFlushManager::get_low_watermark_(FlushCtxState state) +{ + int ret = OB_SUCCESS; + int64_t low_watermark = 100; + switch(state) { + case FlushCtxState::FSM_F1: + low_watermark = FLUSH_LOW_WATERMARK_F1; + break; + case FlushCtxState::FSM_F2: + low_watermark = FLUSH_LOW_WATERMARK_F2; + break; + case FlushCtxState::FSM_F3: + low_watermark = FLUSH_LOW_WATERMARK_F3; + break; + case FlushCtxState::FSM_F4: + low_watermark = FLUSH_LOW_WATERMARK_F4; + break; + case FlushCtxState::FSM_F5: + low_watermark = FLUSH_LOW_WATERMARK_F5; + break; + case FlushCtxState::FSM_FINISHED: + break; + default: + STORAGE_LOG(WARN, "unexpected flush state", K(state), K(flush_ctx_)); + break; + } + return low_watermark; +} + +void ObTmpFileFlushManager::inner_advance_flush_level_() +{ + if (flush_ctx_.get_actual_flush_size() >= flush_ctx_.get_expect_flush_size()) { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } + + int64_t dirty_page_percentage = write_buffer_pool_.get_dirty_page_percentage(); + if (FLUSH_WATERMARK_F2 <= dirty_page_percentage && flush_ctx_.get_state() < FlushCtxState::FSM_F2) { + flush_ctx_.set_state(FlushCtxState::FSM_F2); + } else if (FLUSH_WATERMARK_F3 <= dirty_page_percentage && flush_ctx_.get_state() < FlushCtxState::FSM_F3) { + flush_ctx_.set_state(FlushCtxState::FSM_F3); + } else if (FLUSH_WATERMARK_F4 <= dirty_page_percentage && flush_ctx_.get_state() < FlushCtxState::FSM_F4) { + flush_ctx_.set_state(FlushCtxState::FSM_F4); + } else if (FLUSH_WATERMARK_F5 <= dirty_page_percentage && flush_ctx_.get_state() < FlushCtxState::FSM_F5) { + flush_ctx_.set_state(FlushCtxState::FSM_F5); + } else { + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + } +} + +// only for unittest +void ObTmpFileFlushManager::inner_advance_flush_level_without_checking_watermark_() +{ + switch(flush_ctx_.get_state()) { + case FlushCtxState::FSM_F1: + flush_ctx_.set_state(FlushCtxState::FSM_F2); + break; + case FlushCtxState::FSM_F2: + flush_ctx_.set_state(FlushCtxState::FSM_F3); + break; + case FlushCtxState::FSM_F3: + flush_ctx_.set_state(FlushCtxState::FSM_F4); + break; + case FlushCtxState::FSM_F4: + flush_ctx_.set_state(FlushCtxState::FSM_F5); + break; + case FlushCtxState::FSM_F5: + flush_ctx_.set_state(FlushCtxState::FSM_FINISHED); + break; + default: + break; + } +} + +// Generate a set of flush tasks based on the flushing level to flush dirty pages, +// and attempt to advance the task status to TFFT_WAIT (waiting for asynchronous IO). +// When OB_ALLOCATE_TMP_FILE_PAGE_FAILED error occurs during flushing, +// a special flush task is generated to flush meta pages and terminate the current round of flushing. +// Setting fast_flush_meta to true by the caller also triggers this process. +int ObTmpFileFlushManager::flush(ObSpLinkQueue &flushing_queue, + ObTmpFileFlushMonitor &flush_monitor, + const int64_t expect_flush_size, + const bool is_flush_meta_tree) +{ + int ret = OB_SUCCESS; + bool fast_flush_meta = is_flush_meta_tree; + + init_flush_level_(); + + if (FlushCtxState::FSM_FINISHED == flush_ctx_.get_state() && !fast_flush_meta) { + ret = OB_SUCCESS; + } else if (OB_FAIL(flush_ctx_.prepare_flush_ctx(expect_flush_size, &flush_priority_mgr_, &flush_monitor))) { + STORAGE_LOG(WARN, "fail to prepare flush iterator", KR(ret), K(flush_ctx_)); + } else { + while (OB_SUCC(ret) && !flush_ctx_.is_fail_too_many() + && (FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() || fast_flush_meta)) { + ObTmpFileFlushTask *flush_task = nullptr; + if (OB_FAIL(handle_alloc_flush_task_(fast_flush_meta, flush_task))) { + STORAGE_LOG(WARN, "fail to alloc flush task", KR(ret), K(flush_ctx_)); + } else { + flush_ctx_.inc_create_flush_task_cnt(); + flushing_queue.push(flush_task); + STORAGE_LOG(DEBUG, "create new flush task", K(fast_flush_meta), KPC(flush_task), K(flush_ctx_)); + + FlushState state = ObTmpFileFlushTask::TFFT_INITED; + FlushState next_state = state; + do { + next_state = state = flush_task->get_state(); + if (OB_FAIL(drive_flush_task_prepare_(*flush_task, state, next_state))) { + STORAGE_LOG(WARN, "fail to drive flush task prepare", KR(ret), K(flush_ctx_)); + } else if (flush_task->get_state() >= next_state) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected flush state after drive task succ", KR(ret), K(flush_ctx_), K(state)); + } else if (OB_FAIL(advance_status_(*flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to advance status", + KR(ret), K(flush_ctx_), K(flush_task->get_state()), K(state), K(next_state)); + } + } while (OB_SUCC(ret) && FlushState::TFFT_WAIT != next_state); + + STORAGE_LOG(DEBUG, "drive flush task finished", KR(ret), K(fast_flush_meta), KPC(flush_task), K(flush_ctx_)); + flush_ctx_.update_ctx_by_flush_task(*flush_task); + if (ObTmpFileFlushTask::TFFT_FILL_BLOCK_BUF < flush_task->get_state()) { + flush_ctx_.record_flush_task(flush_task->get_data_length()); // maintain statistics + } + if (flush_task->get_is_fast_flush_tree()) { + if (OB_FAIL(ret)) { + STORAGE_LOG(ERROR, "fail to execute fast_flush_tree_page flush task to TFFT_WAIT", KR(ret), KPC(flush_task)); + } + break; // generate only one fast_flush_tree_page_ task to avoid excessive flushing of the meta + } + if (OB_ALLOCATE_TMP_FILE_PAGE_FAILED == ret){ + if (flush_task->get_state() == FlushState::TFFT_INSERT_META_TREE) { + STORAGE_LOG(WARN, "fail to insert meta tree, generating fast_flush_meta task", KR(ret)); + fast_flush_meta = true; // set this flag generate fast_flush_tree_page_ task in the next loop + ret = OB_SUCCESS; + } else { + STORAGE_LOG(ERROR, "flush task is not in TFFT_INSERT_META_TREE state", KPC(flush_task)); + } + } + } + } + + if (!flushing_queue.is_empty()) { + STORAGE_LOG(DEBUG, "ObTmpFileFlushManager flush finish", KR(ret), K(fast_flush_meta), K(flush_ctx_)); + } + if (OB_FAIL(ret) && !flushing_queue.is_empty()) { + ret = OB_SUCCESS; // ignore error if generate at least 1 task + } + flush_ctx_.clear_flush_ctx(flush_priority_mgr_); + } + return ret; +} + +// skip flush level, copy meta tree pages directly +int ObTmpFileFlushManager::fast_fill_block_buf_with_meta_(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + + if (OB_FAIL(inner_fill_block_buf_(flush_task, FlushCtxState::FSM_F4, true/*is_meta*/, false/*flush_tail*/))) { + LOG_WARN("fail to fast generate flush meta task in flush policy f4", KR(ret), K(flush_task)); + } + + if (flush_task.get_data_length() < FAST_FLUSH_TREE_PAGE_NUM * ObTmpFileGlobal::PAGE_SIZE) { // ignore error code and try F5 + if (OB_FAIL(inner_fill_block_buf_(flush_task, FlushCtxState::FSM_F5, true/*is_meta*/, true/*flush_tail*/))) { + LOG_WARN("fail to fast generate flush meta task in flush policy f5", KR(ret), K(flush_task)); + } + } + + if (OB_FAIL(ret)) { + if (OB_ITER_END == ret && flush_task.get_flush_infos().size() != 0) { + // ignore OB_ITER_END error code to continue to next stage + LOG_INFO("fast_fill_block_buf_with_meta_ iterator reach end, ignore error code to continue", + KR(ret), K(flush_task), K(flush_ctx_)); + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to fast fill block buf with meta", KR(ret), K(flush_task), K(flush_ctx_)); + } + } + return ret; +} + +int ObTmpFileFlushManager::fill_block_buf_(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + switch(flush_ctx_.get_state()) { + case FlushCtxState::FSM_F1: + if (!flush_task.is_full() && FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() + && OB_FAIL(inner_fill_block_buf_(flush_task, flush_ctx_.get_state(), + false/*is_meta*/, false/*flush_tail*/))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate flush data task in flush policy f1", KR(ret), K(flush_task)); + } + } + advance_flush_level_(ret); + // go through + case FlushCtxState::FSM_F2: + if (!flush_task.is_full() && FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() + && OB_FAIL(inner_fill_block_buf_(flush_task, flush_ctx_.get_state(), + false/*is_meta*/, false/*flush_tail*/))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate flush data task in flush policy f2", KR(ret), K(flush_task)); + } + } + advance_flush_level_(ret); + // go through + case FlushCtxState::FSM_F3: + if (!flush_task.is_full() && FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() + && OB_FAIL(inner_fill_block_buf_(flush_task, flush_ctx_.get_state(), + false/*is_meta*/, true/*flush_tail*/))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate flush data task in flush policy f3", KR(ret), K(flush_task)); + } + } + advance_flush_level_(ret); + // break here to forbid flush data pages and meta pages in the same flush task + // to prevent 1 flush task contains all of meta page that can be flushed, + // but is stuck in TFFT_INSERT_META_TREE state and new task has no meta pages to flush + break; + case FlushCtxState::FSM_F4: + if (!flush_task.is_full() && FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() + && OB_FAIL(inner_fill_block_buf_(flush_task, flush_ctx_.get_state(), + true/*is_meta*/, false/*flush_tail*/))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate flush meta task in flush policy f4", KR(ret), K(flush_task)); + } + } + advance_flush_level_(ret); + // go through + case FlushCtxState::FSM_F5: + if (!flush_task.is_full() && FlushCtxState::FSM_FINISHED != flush_ctx_.get_state() + && OB_FAIL(inner_fill_block_buf_(flush_task, flush_ctx_.get_state(), + true/*is_meta*/, true/*flush_tail*/))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to generate flush meta task in flush policy f5", KR(ret), K(flush_task)); + } + } + advance_flush_level_(ret); + // go through + case FlushCtxState::FSM_FINISHED: + // do nothing + break; + default: + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected flush state", KR(ret), K(flush_ctx_)); + break; + } + if (OB_ITER_END == ret && flush_task.get_flush_infos().size() != 0) { + // ignore OB_ITER_END error code to continue TFFT_INSERT_META_TREE state + STORAGE_LOG(DEBUG, "fill_block_buf_ iterator reach end, ignore error code to continue", + KR(ret), K(flush_task), K(flush_ctx_)); + ret = OB_SUCCESS; + } + return ret; +} + +int ObTmpFileFlushManager::inner_fill_block_buf_( + ObTmpFileFlushTask &flush_task, + const FlushCtxState flush_stage, + const bool is_meta, + const bool flush_tail) +{ + int ret = OB_SUCCESS; + const int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + bool fail_too_many = false; + + ObArray &flush_failed_array = flush_ctx_.get_flush_failed_array(); + ObTmpFileFlushListIterator &iter = flush_ctx_.get_flush_list_iterator(); + bool tmp_is_meta = false; + ObTmpFileHandle file_handle; + while (OB_SUCC(ret) && !fail_too_many && !flush_task.is_full()) { + if (OB_FAIL(iter.next(flush_stage, tmp_is_meta, file_handle))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to get file from iterator", + KR(ret), K(flush_stage), K(is_meta), K(flush_tail), K(flush_ctx_)); + } + } else if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "file handle is nullptr", KR(ret)); + } else { + ObSharedNothingTmpFile &file = *file_handle.get(); + if (file.is_deleting()) { + STORAGE_LOG(INFO, "file is deleted while generating flush task", K(file)); + } else { + STORAGE_LOG(DEBUG, "try to copy data from file", K(file.get_fd()), K(is_meta), K(flush_tail), K(file)); + int64_t last_idx = -1; + bool copy_flush_info_fail = false; + ObArray &flush_infos = flush_task.get_flush_infos(); + const int64_t origin_info_cnt = flush_infos.count(); + ObTmpFileSingleFlushContext file_flush_ctx(file_handle); + // push back first to prevent array resizing or + // hash map allocate node failure AFTER copying the file data + if (FAILEDx(get_or_create_file_in_ctx_(file.get_fd(), file_flush_ctx))) { + STORAGE_LOG(WARN, "fail to get or create file in file flush ctx", KR(ret), K(file)); + } else if (OB_FAIL(flush_infos.push_back(ObTmpFileFlushInfo()))) { + STORAGE_LOG(WARN, "fail to insert flush info", KR(ret), K(file), K(flush_task)); + } else if (FALSE_IT(last_idx = flush_infos.size() - 1)) { + } else if (!is_meta && OB_FAIL(file.generate_data_flush_info(flush_task, flush_infos.at(last_idx), + file_flush_ctx.data_ctx_, flush_ctx_.get_flush_sequence(), flush_tail))) { + STORAGE_LOG(WARN, "fail to generate flush data info", KR(ret), K(flush_task), + K(flush_stage), K(is_meta), K(flush_tail), K(file)); + copy_flush_info_fail = true; + } else if (is_meta && OB_FAIL(file.generate_meta_flush_info(flush_task, flush_infos.at(last_idx), + file_flush_ctx.meta_ctx_, flush_ctx_.get_flush_sequence(), flush_tail))) { + STORAGE_LOG(WARN, "fail to generate flush meta info", KR(ret), K(flush_task), + K(flush_stage), K(is_meta), K(flush_tail), K(file)); + copy_flush_info_fail = true; + } else { + if (!(flush_tail || (is_meta && file_flush_ctx.meta_ctx_.is_meta_reach_end_))) { + file.reinsert_flush_node(is_meta); + } + UpdateFlushCtx update_op(file_flush_ctx); + if (OB_FAIL(flush_ctx_.get_file_ctx_hash().set_or_update(file.get_fd(), file_flush_ctx, update_op))) { + // if update fails, the copy offset will be incorrect when the file is flushed a second time in the same round + STORAGE_LOG(ERROR, "fail to set flush ctx after copying data", KR(ret), K(file)); + copy_flush_info_fail = true; + } + flush_ctx_.record_flush_stage(); + } + + if (OB_FAIL(ret)) { + int tmp_ret = OB_SUCCESS; + ObTmpFileBatchFlushContext::ObTmpFileFlushFailRecord record(is_meta, ObTmpFileHandle(&file)); + if (OB_TMP_FAIL(flush_failed_array.push_back(record))) { + // array is pre-allocated to MAX_COPY_FAIL_COUNT, + // file could not be flush afterwards if push_back failed + LOG_ERROR("fail to push back flush failed array", KR(tmp_ret), K(file.get_fd())); + } else if (!flush_task.get_is_fast_flush_tree() + && flush_failed_array.size() >= ObTmpFileBatchFlushContext::MAX_COPY_FAIL_COUNT - 8) { + fail_too_many = true; + LOG_WARN("inner_fill_block_buf_ fail too many times", K(file.get_fd()), K(flush_failed_array.size())); + } else if (flush_task.get_is_fast_flush_tree() + && flush_failed_array.size() >= ObTmpFileBatchFlushContext::MAX_COPY_FAIL_COUNT) { + fail_too_many = true; + LOG_WARN("inner_fill_block_buf_ fail too many times", K(file.get_fd()), K(flush_failed_array.size())); + flush_ctx_.set_fail_too_many(true); // set this flag to exit flush + } + + if (flush_infos.size() > origin_info_cnt) { + flush_infos.pop_back(); + } + if (!file_flush_ctx.data_ctx_.is_valid() && !file_flush_ctx.meta_ctx_.is_valid()) { + // clear pre-created files to ensure that file_ctx_hash_ only records files that were able to send IO + if (OB_TMP_FAIL(flush_ctx_.get_file_ctx_hash().erase_refactored(file.get_fd()))) { + if (OB_HASH_NOT_EXIST != tmp_ret) { + STORAGE_LOG(ERROR, "fail to erase file ctx from hash", KR(tmp_ret), K(file.get_fd())); + } + } + } + + if (!copy_flush_info_fail) { + STORAGE_LOG(WARN, "inner fill block buffer fail, try next file", KR(ret), K(file.get_fd())); + ret = OB_SUCCESS; // ignore error code if fail before copying data + } + } + } + } + file_handle.reset(); + } // end while + + return ret; +} + +void ObTmpFileFlushManager::UpdateFlushCtx::operator() (hash::HashMapPair &pair) +{ + int64_t fd = pair.first; + ObTmpFileDataFlushContext &data_ctx = pair.second.data_ctx_; + ObTmpFileTreeFlushContext &meta_ctx = pair.second.meta_ctx_; + if (input_data_ctx_.is_valid()) { + data_ctx = input_data_ctx_; + } + if (input_meta_ctx_.is_valid()) { + meta_ctx = input_meta_ctx_; + } + STORAGE_LOG(DEBUG, "UpdateFlushCtx after fill block data", K(fd), K(data_ctx), K(meta_ctx)); +} + +int ObTmpFileFlushManager::get_or_create_file_in_ctx_(const int64_t fd, ObTmpFileSingleFlushContext &file_flush_ctx) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(flush_ctx_.get_file_ctx_hash().get_refactored(fd, file_flush_ctx))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + if (OB_FAIL(flush_ctx_.get_file_ctx_hash().set_refactored(fd, file_flush_ctx))) { + STORAGE_LOG(WARN, "fail to insert file flush context", KR(ret), K(fd)); + } + } else { + STORAGE_LOG(WARN, "fail to get file flush context", KR(ret), K(fd)); + } + } + return ret; +} + +// iterate all file to insert data items into its meta tree +int ObTmpFileFlushManager::insert_items_into_meta_tree_(ObTmpFileFlushTask &flush_task, + const int64_t logic_block_index) +{ + int ret = OB_SUCCESS; + ObArray &flush_infos = flush_task.get_flush_infos(); + for (int64_t i = 0; OB_SUCC(ret) && i < flush_infos.count(); ++i) { + ObTmpFileFlushInfo &flush_info = flush_infos.at(i); + ObSharedNothingTmpFile *file = flush_info.file_handle_.get(); + if (OB_ISNULL(file)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "file is nullptr", KR(ret), K(flush_info)); + } else { + if (flush_info.has_data() && !flush_info.insert_meta_tree_done_) { + if (OB_FAIL(file->insert_meta_tree_item(flush_info, logic_block_index))) { + STORAGE_LOG(WARN, "fail to insert meta tree item", KR(ret), K(flush_info), K(logic_block_index), KP(&flush_task)); + // flushing data pages may generate new meta pages. ff there is not enough space allocated for the meta pages, + // it will result in the failure of the current data flushing. therefore, we need to evict some pages to free up space + if (OB_ALLOCATE_TMP_FILE_PAGE_FAILED == ret) { + if (OB_FAIL(evict_pages_and_retry_insert_(flush_task, flush_info, logic_block_index))) { + STORAGE_LOG(WARN, "fail to evict pages and retry insert meta tree item", + KR(ret), K(flush_info), K(logic_block_index)); + } + } + } else { + flush_info.insert_meta_tree_done_ = true; + } + } + } + } + return ret; +} + +// If eviction or insertion fails, retry the operation on the next round +int ObTmpFileFlushManager::evict_pages_and_retry_insert_(ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &flush_info, + const int64_t logic_block_index) +{ + int ret = OB_SUCCESS; + int64_t expect_evict_page = FAST_FLUSH_TREE_PAGE_NUM; + int64_t actual_evict_page = 0; + ObSharedNothingTmpFile *file = flush_info.file_handle_.get(); + if (OB_ISNULL(file)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "file ptr is null", KR(ret), K(flush_task), K(logic_block_index)); + } else if (OB_FAIL(evict_mgr_.evict(expect_evict_page, actual_evict_page))) { + STORAGE_LOG(WARN, "fail to evict meta pages while flushing data pages", + KR(ret), K(flush_task), K(logic_block_index)); + } else if (OB_FAIL(file->insert_meta_tree_item(flush_info, logic_block_index))) { + STORAGE_LOG(WARN, "fail to insert meta tree item again, retry later", + KR(ret), K(flush_info), K(logic_block_index)); + } else { + flush_info.insert_meta_tree_done_ = true; + } + return ret; +} + +int ObTmpFileFlushManager::drive_flush_task_prepare_(ObTmpFileFlushTask &flush_task, + const FlushState state, + FlushState &next_state) +{ + int ret = OB_SUCCESS; + next_state = state; + switch (state) { + case FlushState::TFFT_CREATE_BLOCK_INDEX: + if (OB_FAIL(handle_create_block_index_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task create block index", KR(ret), K(flush_task)); + } + break; + case FlushState::TFFT_FILL_BLOCK_BUF: + if (OB_FAIL(handle_fill_block_buf_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task fill block", KR(ret), K(flush_task)); + } + break; + case FlushState::TFFT_INSERT_META_TREE: + if (OB_FAIL(handle_insert_meta_tree_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task insert meta tree", KR(ret), K(flush_task)); + } + break; + case FlushState::TFFT_ASYNC_WRITE: + if (OB_FAIL(handle_async_write_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task async write", KR(ret), K(flush_task)); + } + break; + default: + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected state in drive_flush_task_prepare_", K(state)); + break; + } + return ret; +} + +int ObTmpFileFlushManager::drive_flush_task_retry_( + ObTmpFileFlushTask &flush_task, + const FlushState state, + FlushState &next_state) +{ + int ret = OB_SUCCESS; + next_state = state; + switch (state) { + case FlushState::TFFT_INSERT_META_TREE: + if (OB_FAIL(handle_insert_meta_tree_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task insert meta tree", KR(ret), K(flush_task)); + } + break; + case FlushState::TFFT_ASYNC_WRITE: + if (OB_FAIL(handle_async_write_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle flush task async write", KR(ret), K(flush_task)); + } + break; + default: + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected state in drive_flush_task_retry_", K(state), K(flush_task)); + break; + } + return ret; +} + +int ObTmpFileFlushManager::drive_flush_task_wait_to_finish_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + ObTmpFileFlushTask::ObTmpFileFlushTaskState state = flush_task.get_state(); + switch (state) { + case FlushState::TFFT_WAIT: + if (OB_FAIL(handle_wait_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to handle wait", KR(ret), K(flush_task)); + } + break; + default: + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected state in drive_flush_task_wait_to_finish_", K(state)); + break; + } + return ret; +} + +int ObTmpFileFlushManager::retry(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + + FlushState state = flush_task.get_state(); + FlushState next_state = state; + do { + next_state = state = flush_task.get_state(); + if (OB_FAIL(drive_flush_task_retry_(flush_task, state, next_state))) { + STORAGE_LOG(WARN, "fail to drive flush state machine", KR(ret), K(flush_task)); + } else if (flush_task.get_state() >= next_state) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected flush state after drive task succ", KR(ret), K(state), K(flush_task)); + } else if (OB_FAIL(advance_status_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to advance status", KR(ret), K(state), K(next_state), K(flush_task)); + } + } while (OB_SUCC(ret) && FlushState::TFFT_WAIT != next_state); + + flush_ctx_.update_ctx_by_flush_task(flush_task); + if (flush_ctx_.can_clear_flush_ctx()) { + flush_ctx_.clear_flush_ctx(flush_priority_mgr_); + } + return ret; +} + +int ObTmpFileFlushManager::io_finished(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + FlushState next_state = FlushState::TFFT_INITED; + if (OB_FAIL(drive_flush_task_wait_to_finish_(flush_task, next_state))) { + STORAGE_LOG(WARN, "fail to drive flush state machine to FINISHED", KR(ret), K(flush_task)); + } else if (flush_task.get_state() < next_state && OB_FAIL(advance_status_(flush_task, next_state))) { + // if the task encounters an IO error, its status will silently revert to TFFT_ASYNC_WRITE; do not verify status here. + STORAGE_LOG(WARN, "fail to advance status", KR(ret), K(flush_task.get_state()), K(next_state)); + } + return ret; +} + +int ObTmpFileFlushManager::update_file_meta_after_flush(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + if (FlushState::TFFT_FINISH != flush_task.get_state()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected flush state after drive task succ", KR(ret), K(flush_task)); + } else if (OB_FAIL(handle_finish_(flush_task))) { + STORAGE_LOG(WARN, "fail to update file meta after flush", KR(ret), K(flush_task)); + } + return ret; +} + +int ObTmpFileFlushManager::advance_status_(ObTmpFileFlushTask &flush_task, const FlushState &state) +{ + int ret = OB_SUCCESS; + if (flush_task.get_state() >= state) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "unexpected state in advance_status_", K(state), K(flush_task)); + } else { + flush_task.set_state(state); + STORAGE_LOG(DEBUG, "advance flush task status succ", K(state), K(flush_task)); + } + return ret; +} + +int ObTmpFileFlushManager::handle_alloc_flush_task_(const bool fast_flush_meta, ObTmpFileFlushTask *&flush_task) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(alloc_flush_task(flush_task))) { + STORAGE_LOG(WARN, "fail to alloc flush callback", KR(ret)); + } else if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush cb is null", KR(ret)); + } else { + flush_task->set_state(FlushState::TFFT_CREATE_BLOCK_INDEX); + flush_task->set_create_ts(ObTimeUtil::current_time()); + flush_task->set_flush_seq(flush_ctx_.get_flush_sequence()); + flush_task->set_is_fast_flush_tree(fast_flush_meta); + } + return ret; +} + +int ObTmpFileFlushManager::handle_create_block_index_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + int64_t block_index = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + ObTmpFileBlockHandle tmp_file_block_handle; + // occupy the whole tmp file block because we don't know how many pages we will use before TFFT_FILL_BLOCK_BUF, + // and we still create block first because copy meta tree pages needs to provide tmp file block's block_index, + // unused pages will be released after TFFT_FILL_BLOCK_BUF succ + if (OB_FAIL(tmp_file_block_mgr_.create_tmp_file_block(0/*begin_page_id*/, + ObTmpFileGlobal::BLOCK_PAGE_NUMS, + block_index))) { + STORAGE_LOG(WARN, "fail to create tmp file block", KR(ret), K(flush_task)); + } else if (OB_FAIL(tmp_file_block_mgr_.get_tmp_file_block_handle(block_index, tmp_file_block_handle))) { + // keep a tmp file block handle in flush task to prevent rollback operations + // caused by "append writes last not full page during the flush process" release tmp file block. + // for example, a tmp file block will be unexpected released if a flush task + // only contains "not full pages" and all there pages are rollback by append writes + STORAGE_LOG(WARN, "fail to get tmp file block handle", KR(ret), K(block_index), K(flush_task)); + } else { + flush_task.set_tmp_file_block_handle(tmp_file_block_handle); + flush_task.set_block_index(block_index); + next_state = FlushState::TFFT_FILL_BLOCK_BUF; + } + return ret; +} + +int ObTmpFileFlushManager::handle_fill_block_buf_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + if (OB_FAIL(flush_task.prealloc_block_buf())) { + STORAGE_LOG(WARN, "fail to prealloc block buf", KR(ret), K(flush_task)); + } else if (flush_task.get_is_fast_flush_tree()) { // skip flush level, copy meta tree pages directly + if (OB_FAIL(fast_fill_block_buf_with_meta_(flush_task))) { + STORAGE_LOG(WARN, "fail to fill block buffer with meta", KR(ret), K(flush_task)); + } + } else { + if (OB_FAIL(fill_block_buf_(flush_task))) { + if (OB_ITER_END != ret) { + STORAGE_LOG(WARN, "fail to fill block buf", KR(ret), K(flush_task)); + } + } else if (0 == flush_task.get_data_length()) { + ret = OB_ITER_END; + } + } + + if (OB_FAIL(ret)){ + STORAGE_LOG(WARN, "fail to fill block buf, skip release page", KR(ret)); + } else if (OB_UNLIKELY(flush_task.get_flush_infos().empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "flush infos is empty", KR(ret), K(flush_task)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < flush_task.get_flush_infos().count(); ++i) { + if (OB_UNLIKELY(flush_task.get_flush_infos().at(i).has_data() && + flush_task.get_flush_infos().at(i).has_meta())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "flush info has both data and meta", KR(ret), K(flush_task)); + } else if (OB_UNLIKELY((flush_task.get_flush_infos().at(0).has_data() != + flush_task.get_flush_infos().at(i).has_data()) || + (flush_task.get_flush_infos().at(0).has_meta() != + flush_task.get_flush_infos().at(i).has_meta()))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "flush infos mixed storage meta and data pages", KR(ret), K(flush_task)); + } + } + } + + if (OB_SUCC(ret)) { + const bool is_whole_data_page = flush_task.get_flush_infos().at(0).has_data(); + if (is_whole_data_page && + OB_TMP_FAIL(ObTmpBlockCache::get_instance().put_block(flush_task.get_inst_handle(), + flush_task.get_kvpair(), + flush_task.get_block_handle()))) { + STORAGE_LOG(WARN, "fail to put block into block cache", KR(tmp_ret), K(flush_task)); + } + + int64_t used_page_num = flush_task.get_total_page_num(); + int64_t unused_page_id = used_page_num; + int64_t unused_page_num = ObTmpFileGlobal::BLOCK_PAGE_NUMS - used_page_num; + int64_t block_index = flush_task.get_block_index(); + bool need_release_page = unused_page_num > 0; + + if (OB_FAIL(tmp_file_block_mgr_.write_back_start(block_index))) { + STORAGE_LOG(ERROR, "fail to notify tmp file block write back start", KR(ret), K(block_index)); + } else if (need_release_page && OB_FAIL(tmp_file_block_mgr_.release_tmp_file_page( + block_index, unused_page_id, unused_page_num))) { + STORAGE_LOG(ERROR, "fail to release tmp file page", + KR(ret), K(unused_page_id), K(unused_page_num), K(used_page_num), K(block_index), K(flush_task)); + } else { + next_state = FlushState::TFFT_INSERT_META_TREE; + } + } + + return ret; +} + +int ObTmpFileFlushManager::handle_insert_meta_tree_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(insert_items_into_meta_tree_(flush_task, flush_task.get_block_index()))) { + STORAGE_LOG(WARN, "fail to insert meta tree", KR(ret), K(flush_task)); + } else { + next_state = FlushState::TFFT_ASYNC_WRITE; + } + return ret; +} + +int ObTmpFileFlushManager::handle_async_write_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(flush_task.write_one_block())) { + STORAGE_LOG(WARN, "fail to async write blocks", KR(ret), K(flush_task)); + } else { + next_state = FlushState::TFFT_WAIT; + } + return ret; +} + +int ObTmpFileFlushManager::handle_wait_(ObTmpFileFlushTask &flush_task, FlushState &next_state) +{ + int ret = OB_SUCCESS; + int task_ret_code = flush_task.atomic_get_ret_code(); + if (OB_SUCCESS != task_ret_code) { + // rollback the status to TFFT_ASYNC_WRITE if IO failed, and re-send the I/O in the retry process. + STORAGE_LOG(INFO, "flush_task io fail, retry it later", KR(task_ret_code), K(flush_task)); + flush_task.set_state(FlushState::TFFT_ASYNC_WRITE); + } else if (OB_FAIL(tmp_file_block_mgr_.write_back_succ(flush_task.get_block_index(), + flush_task.get_macro_block_handle().get_macro_id()))) { + STORAGE_LOG(WARN, "fail to notify tmp file block write back succ", KR(ret), K(flush_task)); + } else { + next_state = FlushState::TFFT_FINISH; + } + return ret; +} + +// Update file meta after flush task IO complete, ensures reentrancy +int ObTmpFileFlushManager::handle_finish_(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + + if (OB_FAIL(update_meta_data_after_flush_for_files_(flush_task))) { + STORAGE_LOG(WARN, "fail to update meta data after flush for files", KR(ret), K(flush_task)); + } else { + STORAGE_LOG(DEBUG, "flush task finish successfully", K(flush_task)); + } + + return ret; +} + +void ObTmpFileFlushManager::ResetFlushCtxOp::operator() (hash::HashMapPair &pair) +{ + if (is_meta_) { + pair.second.meta_ctx_.reset(); + } else { + pair.second.data_ctx_.reset(); + } +} + +int ObTmpFileFlushManager::update_meta_data_after_flush_for_files_(ObTmpFileFlushTask &flush_task) +{ + int ret = OB_SUCCESS; + ObArray &flush_infos = flush_task.get_flush_infos(); + for (int64_t i = 0; OB_SUCC(ret) && i < flush_infos.size(); ++i) { + ObTmpFileFlushInfo &flush_info = flush_infos.at(i); + ObSharedNothingTmpFile *file = flush_info.file_handle_.get(); + bool is_meta = false; + bool reset_ctx = false; + if (!flush_info.update_meta_data_done_) { + if (OB_ISNULL(file)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "tmp file ptr is null", KR(ret), K(i), K(flush_task)); + } else if ((OB_UNLIKELY(flush_info.has_data() && flush_info.has_meta()) || + OB_UNLIKELY(!flush_info.has_data() && !flush_info.has_meta()))) { + ret = OB_ERR_UNEXPECTED; // expect one flush_info only contains one type of pages + STORAGE_LOG(ERROR, "flush info is not valid", KR(ret), K(flush_info)); + } else if (FALSE_IT(is_meta = flush_info.has_meta())) { + } else if (OB_FAIL(file->update_meta_after_flush(flush_info.batch_flush_idx_, is_meta, reset_ctx))){ + STORAGE_LOG(WARN, "fail to update meta data", KR(ret), K(is_meta), K(flush_info)); + } else { + int tmp_ret = OB_SUCCESS; + // reset data/meta flush ctx after file update meta complete to prevent + // flush use stale flushed_page_id to copy data after the page is evicted. + // use empty ctx here because we have inserted ctx for every file when flushing begins. + ResetFlushCtxOp reset_op(is_meta); + ObTmpFileSingleFlushContext empty_ctx; + if (flush_task.get_flush_seq() == flush_ctx_.get_flush_sequence() && + OB_TMP_FAIL(flush_ctx_.get_file_ctx_hash().set_or_update(file->get_fd(), empty_ctx, reset_op))) { + STORAGE_LOG(WARN, "fail to clean file ctx from hash", KR(tmp_ret), K(file)); + } + flush_info.update_meta_data_done_ = true; + } + } + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_flush_manager.h b/src/storage/tmp_file/ob_tmp_file_flush_manager.h new file mode 100644 index 0000000000..87e1383ebb --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_manager.h @@ -0,0 +1,127 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_TASK_MANAGER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_TASK_MANAGER_H_ + +#include "storage/tmp_file/ob_tmp_file_eviction_manager.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_flush_ctx.h" +#include "storage/tmp_file/ob_tmp_file_flush_priority_manager.h" +#include "storage/tmp_file/ob_tmp_file_thread_job.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +// flush manager generates flush tasks according to dirty page watermark +class ObTmpFileFlushManager +{ +public: + static const int32_t FLUSH_WATERMARK_F1 = 50; // percentage + static const int32_t FLUSH_WATERMARK_F2 = 70; + static const int32_t FLUSH_WATERMARK_F3 = 75; + static const int32_t FLUSH_WATERMARK_F4 = 80; + static const int32_t FLUSH_WATERMARK_F5 = 90; + static const int32_t FLUSH_LOW_WATERMARK_F1 = 25; + static const int32_t FLUSH_LOW_WATERMARK_F2 = 60; + static const int32_t FLUSH_LOW_WATERMARK_F3 = 65; + static const int32_t FLUSH_LOW_WATERMARK_F4 = 70; + static const int32_t FLUSH_LOW_WATERMARK_F5 = 80; + static const int64_t FAST_FLUSH_TREE_PAGE_NUM = 32; + struct UpdateFlushCtx + { + public: + UpdateFlushCtx(ObTmpFileSingleFlushContext &file_ctx) + : input_data_ctx_(file_ctx.data_ctx_), + input_meta_ctx_(file_ctx.meta_ctx_) {} + void operator() (hash::HashMapPair &pair); + private: + ObTmpFileDataFlushContext &input_data_ctx_; + ObTmpFileTreeFlushContext &input_meta_ctx_; + }; + struct ResetFlushCtxOp + { + public: + public: + ResetFlushCtxOp(const bool is_meta) : is_meta_(is_meta) {} + void operator() (hash::HashMapPair &pair); + private: + bool is_meta_; + }; +public: + typedef ObTmpFileGlobal::FlushCtxState FlushCtxState; + typedef common::ObDList ObTmpFileFlushList; + typedef ObTmpFileFlushTask::ObTmpFileFlushTaskState FlushState; + ObTmpFileFlushManager(ObTmpFilePageCacheController &pc_ctrl); + ~ObTmpFileFlushManager() {} + int init(); + void destroy(); +public: + int free_tmp_file_block(ObTmpFileFlushTask &flush_task); + int alloc_flush_task(ObTmpFileFlushTask *&flush_task); + int free_flush_task(ObTmpFileFlushTask *flush_task); + int notify_write_back_failed(ObTmpFileFlushTask *flush_task); + int flush(ObSpLinkQueue &flushing_queue, + ObTmpFileFlushMonitor &flush_monitor, + const int64_t expect_flush_size, + const bool is_flush_meta_tree); + int retry(ObTmpFileFlushTask &flush_task); + int io_finished(ObTmpFileFlushTask &flush_task); + int update_file_meta_after_flush(ObTmpFileFlushTask &flush_task); +private: + int fill_block_buf_(ObTmpFileFlushTask &flush_task); + int fast_fill_block_buf_with_meta_(ObTmpFileFlushTask &flush_task); + int inner_fill_block_buf_(ObTmpFileFlushTask &flush_task, + const FlushCtxState flush_stage, + const bool is_meta, + const bool flush_tail); + int insert_items_into_meta_tree_(ObTmpFileFlushTask &flush_task, + const int64_t logic_block_index); + void init_flush_level_(); + void advance_flush_level_(const int ret_code); + void inner_advance_flush_level_(); + void inner_advance_flush_level_without_checking_watermark_(); + int64_t get_low_watermark_(FlushCtxState state); + int advance_status_(ObTmpFileFlushTask &flush_task, const FlushState &state); + int drive_flush_task_prepare_(ObTmpFileFlushTask &flush_task, const FlushState state, FlushState &next_state); + int drive_flush_task_retry_(ObTmpFileFlushTask &flush_task, const FlushState state, FlushState &next_state); + int drive_flush_task_wait_to_finish_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_alloc_flush_task_(const bool fast_flush_meta, ObTmpFileFlushTask *&flush_task); + int handle_create_block_index_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_fill_block_buf_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_insert_meta_tree_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_async_write_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_wait_(ObTmpFileFlushTask &flush_task, FlushState &next_state); + int handle_finish_(ObTmpFileFlushTask &flush_task); +private: + int update_meta_data_after_flush_for_files_(ObTmpFileFlushTask &flush_task); + int get_or_create_file_in_ctx_(const int64_t fd, ObTmpFileSingleFlushContext &file_flush_ctx); + int evict_pages_and_retry_insert_(ObTmpFileFlushTask &flush_task, + ObTmpFileFlushInfo &flush_info, + const int64_t logic_block_index); + DISALLOW_COPY_AND_ASSIGN(ObTmpFileFlushManager); +private: + bool is_inited_; + ObTmpFileBatchFlushContext flush_ctx_; + ObTmpFilePageCacheController &pc_ctrl_; + ObTmpFileBlockManager &tmp_file_block_mgr_; + ObIAllocator &task_allocator_; // ref to ObTmpFilePageCacheController::task_allocator_ + ObTmpWriteBufferPool &write_buffer_pool_; + ObTmpFileEvictionManager &evict_mgr_; + ObTmpFileFlushPriorityManager &flush_priority_mgr_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_TASK_MANAGER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.cpp b/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.cpp new file mode 100644 index 0000000000..5c4567138f --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.cpp @@ -0,0 +1,354 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_flush_priority_manager.h" + +namespace oceanbase +{ +namespace tmp_file +{ +//----------------------- ObTmpFileFlushPriorityManager -------------------// +int ObTmpFileFlushPriorityManager::init() +{ + int ret = OB_SUCCESS; + STATIC_ASSERT(ARRAYSIZEOF(data_flush_lists_) == (int64_t)FileList::MAX, + "data_flush_lists_ size mismatch enum FileList count"); + STATIC_ASSERT(ARRAYSIZEOF(data_list_locks_) == (int64_t)FileList::MAX, + "data_list_locks_ size mismatch enum FileList count"); + STATIC_ASSERT(ARRAYSIZEOF(meta_flush_lists_) == (int64_t)FileList::MAX, + "meta_flush_lists_ size mismatch enum FileList count"); + STATIC_ASSERT(ARRAYSIZEOF(meta_list_locks_) == (int64_t)FileList::MAX, + "meta_list_locks_ size mismatch enum FileList count"); + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileFlushPriorityManager inited twice", KR(ret)); + } else { + is_inited_ = true; + LOG_INFO("ObTmpFileFlushPriorityManager init succ", K(is_inited_)); + } + return ret; +} + +void ObTmpFileFlushPriorityManager::destroy() +{ + is_inited_ = false; + for (int64_t i = 0; i < FileList::MAX; i++) { + ObSpinLockGuard guard(data_list_locks_[i]); + data_flush_lists_[i].reset(); + } + for (int64_t i = 0; i < FileList::MAX; i++) { + ObSpinLockGuard guard(meta_list_locks_[i]); + meta_flush_lists_[i].reset(); + } +} + +int64_t ObTmpFileFlushPriorityManager::get_file_size() +{ + int64_t size = 0; + is_inited_ = false; + for (int64_t i = 0; i < FileList::MAX; i++) { + ObSpinLockGuard guard(data_list_locks_[i]); + size += data_flush_lists_[i].get_size(); + } + for (int64_t i = 0; i < FileList::MAX; i++) { + ObSpinLockGuard guard(meta_list_locks_[i]); + size += meta_flush_lists_[i].get_size(); + } + return size; +} + +// attention: +// call this function with protection of ObSharedNothingTmpFile's meta_lock +int ObTmpFileFlushPriorityManager::insert_data_flush_list(ObSharedNothingTmpFile &file, const int64_t dirty_page_size) +{ + int ret = OB_SUCCESS; + FileList flush_idx = FileList::MAX; + + if (OB_FAIL(get_data_list_idx_(dirty_page_size, flush_idx))) { + LOG_WARN("fail to get data list idx", KR(ret), K(dirty_page_size)); + } else if (OB_FAIL(insert_flush_list_(false/*is_meta*/, file, flush_idx))) { + LOG_WARN("fail to insert data flush list", KR(ret), K(file), K(dirty_page_size)); + } else { + LOG_DEBUG("insert_data_flush_list succ", K(file), K(dirty_page_size)); + } + + return ret; +} + +// attention: +// call this function with protection of ObSharedNothingTmpFile's meta_lock +int ObTmpFileFlushPriorityManager::insert_meta_flush_list(ObSharedNothingTmpFile &file, + const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num) +{ + int ret = OB_SUCCESS; + FileList flush_idx = FileList::MAX; + + if (OB_FAIL(get_meta_list_idx_(non_rightmost_dirty_page_num, rightmost_dirty_page_num, flush_idx))) { + LOG_WARN("fail to get meta list idx", KR(ret), K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } else if (OB_FAIL(insert_flush_list_(true/*is_meta*/, file, flush_idx))) { + LOG_WARN("fail to insert meta flush list", KR(ret), K(file), K(flush_idx), + K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } else { + LOG_DEBUG("insert_meta_flush_list succ", K(file), K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } + + return ret; +} + +int ObTmpFileFlushPriorityManager::insert_flush_list_(const bool is_meta, ObSharedNothingTmpFile &file, + const FileList flush_idx) +{ + int ret = OB_SUCCESS; + ObSharedNothingTmpFile::ObTmpFileNode &flush_node = is_meta ? file.get_meta_flush_node() : file.get_data_flush_node(); + ObSpinLock* locks = is_meta ? meta_list_locks_ : data_list_locks_; + ObTmpFileFlushList *flush_lists = is_meta ? meta_flush_lists_ : data_flush_lists_; + + if (OB_UNLIKELY(flush_node.get_next() != nullptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file already in flush list", KR(ret), K(flush_node)); + } else if (flush_idx < FileList::L1 || flush_idx >= FileList::MAX){ + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid flush list idx", KR(ret), K(flush_idx)); + } else { + ObSpinLockGuard guard(locks[flush_idx]); + file.inc_ref_cnt(); + if (OB_UNLIKELY(!flush_lists[flush_idx].add_last(&flush_node))) { + file.dec_ref_cnt(); + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to add node to list", KR(ret), K(flush_idx), K(flush_node)); + } else if (is_meta) { + file.set_meta_page_flush_level(flush_idx); + } else { + file.set_data_page_flush_level(flush_idx); + } + } + + return ret; +} + +// attention: +// call this function with protection of ObSharedNothingTmpFile's meta_lock +int ObTmpFileFlushPriorityManager::update_data_flush_list(ObSharedNothingTmpFile &file, const int64_t dirty_page_size) +{ + int ret = OB_SUCCESS; + FileList new_flush_idx = FileList::MAX; + + if (OB_FAIL(get_data_list_idx_(dirty_page_size, new_flush_idx))) { + LOG_WARN("fail to get data list idx", KR(ret), K(dirty_page_size)); + } else if (OB_FAIL(update_flush_list_(false/*is_meta*/, file, new_flush_idx))) { + LOG_WARN("fail to update data flush list", KR(ret), K(file), K(dirty_page_size)); + } else { + LOG_DEBUG("update_data_flush_list succ", K(file), K(dirty_page_size)); + } + return ret; +} + +// attention: +// call this function with protection of ObSharedNothingTmpFile's meta_lock +int ObTmpFileFlushPriorityManager::update_meta_flush_list(ObSharedNothingTmpFile &file, + const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num) +{ + int ret = OB_SUCCESS; + FileList new_flush_idx = FileList::MAX; + + if (OB_FAIL(get_meta_list_idx_(non_rightmost_dirty_page_num, rightmost_dirty_page_num, new_flush_idx))) { + LOG_WARN("fail to get meta list idx", KR(ret), K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } else if (OB_FAIL(update_flush_list_(true/*is_meta*/, file, new_flush_idx))) { + LOG_WARN("fail to update meta flush list", KR(ret), K(file), K(new_flush_idx), + K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } else { + LOG_DEBUG("update_meta_flush_list succ", K(file), K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } + return ret; +} + +int ObTmpFileFlushPriorityManager::update_flush_list_(const bool is_meta, ObSharedNothingTmpFile &file, + const FileList new_flush_idx) +{ + int ret = OB_SUCCESS; + ObSharedNothingTmpFile::ObTmpFileNode &flush_node = is_meta ? file.get_meta_flush_node() : file.get_data_flush_node(); + ObSpinLock* locks = is_meta ? meta_list_locks_ : data_list_locks_; + ObTmpFileFlushList *flush_lists = is_meta ? meta_flush_lists_ : data_flush_lists_; + int64_t cur_flush_idx = is_meta ? file.get_meta_page_flush_level() : file.get_data_page_flush_level(); + + if (cur_flush_idx < FileList::L1 || cur_flush_idx >= FileList::MAX){ + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid flush list idx", KR(ret), K(new_flush_idx)); + } else if (new_flush_idx == cur_flush_idx) { + // no need to update + } else { // need to move file into a new flush list + bool is_in_flushing = false; + { + ObSpinLockGuard guard(locks[cur_flush_idx]); + if (OB_ISNULL(flush_node.get_next())) { + // before we lock the list, flush task mgr has popped the node from list and is operating it, do nothing + is_in_flushing = true; + } else if (OB_UNLIKELY(!flush_lists[cur_flush_idx].remove(&flush_node))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to remove node from old list", KR(ret), K(cur_flush_idx)); + } + } + + if (OB_FAIL(ret)) { + } else if (is_in_flushing) { + // the node will be added into flush list again by flush task mgr + // do nothing + } else { + ObSpinLockGuard guard(locks[new_flush_idx]); + if (OB_UNLIKELY(!flush_lists[new_flush_idx].add_last(&flush_node))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to add node to new list", KR(ret), K(new_flush_idx)); + } else { + if (is_meta) { + file.set_meta_page_flush_level(new_flush_idx); + } else { + file.set_data_page_flush_level(new_flush_idx); + } + } + } + } + return ret; +} + +int ObTmpFileFlushPriorityManager::remove_file(ObSharedNothingTmpFile &file) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remove_file(true /*is_meta*/, file))) { + LOG_WARN("fail to remove file from meta flush list", KR(ret)); + } else if (OB_FAIL(remove_file(false /*is_meta*/, file))) { + LOG_WARN("fail to remove file from data flush list", KR(ret)); + } + return ret; +} + +// attention: +// call this function with protection of ObSharedNothingTmpFile's meta_lock +int ObTmpFileFlushPriorityManager::remove_file(const bool is_meta, ObSharedNothingTmpFile &file) +{ + int ret = OB_SUCCESS; + int64_t flush_idx = is_meta ? file.get_meta_page_flush_level() : file.get_data_page_flush_level(); + ObSharedNothingTmpFile::ObTmpFileNode &flush_node = is_meta ? file.get_meta_flush_node() : file.get_data_flush_node(); + if (FileList::INVALID == flush_idx) { + // file doesn't exist in the flushing list + // do nothing + } else if (flush_idx < FileList::L1 || flush_idx >= FileList::MAX){ + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid flush list idx", KR(ret), K(flush_idx)); + } else { + ObSpinLock* locks = is_meta ? meta_list_locks_ : data_list_locks_; + ObTmpFileFlushList *flush_lists = is_meta ? meta_flush_lists_ : data_flush_lists_; + ObSpinLockGuard guard(locks[flush_idx]); + if (OB_ISNULL(flush_node.get_next())) { + // node has not been inserted, do nothing + } else if (OB_ISNULL(flush_lists[flush_idx].remove(&flush_node))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to remove node from list", KR(ret), K(flush_idx)); + } else { + file.dec_ref_cnt(); + if (is_meta) { + file.set_meta_page_flush_level(-1); + } else { + file.set_data_page_flush_level(-1); + } + LOG_DEBUG("remove file succ", K(file), K(is_meta)); + } + } + return ret; +} + +int ObTmpFileFlushPriorityManager::popN_from_file_list(const bool is_meta, const int64_t list_idx, + const int64_t expected_count, int64_t &actual_count, + ObArray &file_handles) +{ + int ret = OB_SUCCESS; + ObSpinLock* locks = is_meta ? meta_list_locks_ : data_list_locks_; + ObTmpFileFlushList *flush_lists = is_meta ? meta_flush_lists_ : data_flush_lists_; + actual_count = 0; + + if (OB_UNLIKELY(list_idx < FileList::L1 || list_idx >= FileList::MAX)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(list_idx)); + } else { + ObSpinLockGuard guard(locks[list_idx]); + while (OB_SUCC(ret) && !flush_lists[list_idx].is_empty() && actual_count < expected_count) { + ObSharedNothingTmpFile *file = nullptr; + if (OB_ISNULL(file = &flush_lists[list_idx].remove_first()->file_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("file is null", KR(ret)); + } else if (OB_FAIL(file_handles.push_back(file))) { + LOG_WARN("fail to push back", KR(ret), KP(file)); + int tmp_ret = OB_SUCCESS; + ObSharedNothingTmpFile::ObTmpFileNode &node = is_meta ? file->get_meta_flush_node() : file->get_data_flush_node(); + if (OB_UNLIKELY(!flush_lists[list_idx].add_last(&node))) { + tmp_ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to add node to list", KR(tmp_ret), K(list_idx), KP(file)); + } + } else { + LOG_DEBUG("pop file succ", KPC(file), K(is_meta)); + file->dec_ref_cnt(); // ref_cnt of flush list + actual_count++; + } + } // end while + } + + return ret; +} + +int ObTmpFileFlushPriorityManager::get_meta_list_idx_(const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num, FileList &idx) +{ + int ret = OB_SUCCESS; + idx = FileList::MAX; + if (non_rightmost_dirty_page_num + rightmost_dirty_page_num <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(non_rightmost_dirty_page_num), K(rightmost_dirty_page_num)); + } else if (non_rightmost_dirty_page_num == 0) { // all dirty pages is the rightmost page + idx = FileList::L5; + } else if (non_rightmost_dirty_page_num < 16) { // (0KB, 128KB) + idx = FileList::L4; + } else if (non_rightmost_dirty_page_num < 128) { // [128KB, 1MB) + idx = FileList::L3; + } else if (non_rightmost_dirty_page_num < 256) { // [1MB, 2MB) + idx = FileList::L2; + } else { // [2MB, INFINITE) + idx = FileList::L1; + } + return ret; +} + +int ObTmpFileFlushPriorityManager::get_data_list_idx_(const int64_t dirty_page_size, FileList &idx) +{ + int ret = OB_SUCCESS; + idx = FileList::MAX; + if (dirty_page_size <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(dirty_page_size)); + } else if (dirty_page_size < (1 << 13)) { // page_size 8KB, (0, 8KB) + idx = FileList::L5; + } else if (dirty_page_size < (1 << 17)) { // [8KB, 128KB) + idx = FileList::L4; + } else if (dirty_page_size < (1 << 20)) { // [128KB, 1MB) + idx = FileList::L3; + } else if (dirty_page_size < (1 << 21)) { // [1MB, 2MB) + idx = FileList::L2; + } else { // [2MB, INFINITE) + idx = FileList::L1; + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.h b/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.h new file mode 100644 index 0000000000..47f9e3600a --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_flush_priority_manager.h @@ -0,0 +1,82 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_PRIORITY_MANAGER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_PRIORITY_MANAGER_H_ + +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileFlushPriorityManager +{ +public: + ObTmpFileFlushPriorityManager() + : is_inited_(false), + data_flush_lists_(), + data_list_locks_(), + meta_flush_lists_(), + meta_list_locks_() + {} + ~ObTmpFileFlushPriorityManager() {} + int init(); + void destroy(); + +private: + enum FileList { + INVALID = -1, + L1 = 0, // [2MB, INFINITE) + L2, // [1MB, 2MB) + L3, // [128KB, 1MB) + L4, // data_list: [8KB, 128KB); meta_list: (0KB, 128KB) + L5, // data_list: (0, 8KB); meta_list: 0KB + MAX + }; + typedef common::ObDList ObTmpFileFlushList; + friend class ObTmpFileFlushListIterator; + +public: + int insert_data_flush_list(ObSharedNothingTmpFile &file, const int64_t dirty_page_size); + int insert_meta_flush_list(ObSharedNothingTmpFile &file, const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num); + int update_data_flush_list(ObSharedNothingTmpFile &file, const int64_t dirty_page_size); + int update_meta_flush_list(ObSharedNothingTmpFile &file, const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num); + int remove_file(ObSharedNothingTmpFile &file); + int remove_file(const bool is_meta, ObSharedNothingTmpFile &file); + int popN_from_file_list(const bool is_meta, const int64_t list_idx, + const int64_t expected_count, int64_t &actual_count, + ObArray &file_handles); + int64_t get_file_size(); +private: + int get_meta_list_idx_(const int64_t non_rightmost_dirty_page_num, + const int64_t rightmost_dirty_page_num, FileList &idx); + int get_data_list_idx_(const int64_t dirty_page_size, FileList &idx); + int insert_flush_list_(const bool is_meta, ObSharedNothingTmpFile &file, + const FileList flush_idx); + int update_flush_list_(const bool is_meta, ObSharedNothingTmpFile &file, + const FileList new_flush_idx); + +private: + bool is_inited_; + ObTmpFileFlushList data_flush_lists_[FileList::MAX]; + ObSpinLock data_list_locks_[FileList::MAX]; + ObTmpFileFlushList meta_flush_lists_[FileList::MAX]; + ObSpinLock meta_list_locks_[FileList::MAX]; +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_FLUSH_PRIORITY_MANAGER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_global.cpp b/src/storage/tmp_file/ob_tmp_file_global.cpp new file mode 100644 index 0000000000..f2427f94a8 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_global.cpp @@ -0,0 +1,30 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_global.h" + +namespace oceanbase +{ +namespace tmp_file +{ +const int64_t ObTmpFileGlobal::INVALID_TMP_FILE_FD = -1; +const int64_t ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID = -1; +const int64_t ObTmpFileGlobal::TMP_FILE_READ_BATCH_SIZE = 8 * 1024 * 1024; // 8MB +const int64_t ObTmpFileGlobal::TMP_FILE_WRITE_BATCH_PAGE_NUM = 16; +const int64_t ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX = -1; +const uint32_t ObTmpFileGlobal::INVALID_PAGE_ID = UINT32_MAX; +const int64_t ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID = -1; + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_global.h b/src/storage/tmp_file/ob_tmp_file_global.h new file mode 100644 index 0000000000..f0dcb69d9a --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_global.h @@ -0,0 +1,57 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_GLOBAL_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_GLOBAL_H_ +#include "deps/oblib/src/lib/ob_define.h" + +namespace oceanbase +{ +namespace tmp_file +{ +struct ObTmpFileGlobal final +{ + // TMP_FILE + static const int64_t INVALID_TMP_FILE_FD; + static const int64_t INVALID_TMP_FILE_DIR_ID; + + static const int64_t PAGE_SIZE = 8 * 1024; // 8KB + static const int64_t BLOCK_PAGE_NUMS = + OB_DEFAULT_MACRO_BLOCK_SIZE / PAGE_SIZE; // 256 pages per macro block + + static const int64_t TMP_FILE_READ_BATCH_SIZE; + static const int64_t TMP_FILE_WRITE_BATCH_PAGE_NUM; + + // TMP_FILE_BLOCK + static const int64_t INVALID_TMP_FILE_BLOCK_INDEX; + + // TMP_FILE_WRITE_BUFFER + static const uint32_t INVALID_PAGE_ID; + static const int64_t INVALID_VIRTUAL_PAGE_ID; + + // TMP_FILE_FLUSH_STAGE + enum FlushCtxState + { + FSM_F1 = 0, // flush data list L1 + FSM_F2 = 1, // flush data list L2 & L3 & L4 + FSM_F3 = 2, // flush data list L5 + FSM_F4 = 3, // flush meta list non-rightmost pages + FSM_F5 = 4, // flush meta list rightmost pages + FSM_FINISHED = 5 + }; + static const int64_t INVALID_FLUSH_SEQUENCE = -1; +}; + + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_GLOBAL_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_io_ctx.cpp b/src/storage/tmp_file/ob_tmp_file_io_ctx.cpp new file mode 100644 index 0000000000..19ed94c5bd --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_io_ctx.cpp @@ -0,0 +1,497 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_io_ctx.h" + +namespace oceanbase +{ +using namespace storage; +using namespace share; + +namespace tmp_file +{ +ObTmpFileIOCtx::ObTmpFileIOCtx(): + is_inited_(false), + is_read_(false), + fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), + dir_id_(ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID), + buf_(nullptr), + buf_size_(-1), + done_size_(-1), + todo_size_(-1), + read_offset_in_file_(-1), + disable_page_cache_(false), + io_flag_(), + io_timeout_ms_(DEFAULT_IO_WAIT_TIME_MS), + io_handles_(), + page_cache_handles_() +{ + io_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_IO_HDL")); + page_cache_handles_.set_attr(ObMemAttr(MTL_ID(), "TMP_PCACHE_HDL")); +} + +ObTmpFileIOCtx::~ObTmpFileIOCtx() +{ + reset(); +} + +int ObTmpFileIOCtx::init(const int64_t fd, const int64_t dir_id, + const bool is_read, + const common::ObIOFlag io_flag, + const int64_t io_timeout_ms, + const bool disable_page_cache) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_ERROR("ObTmpFileIOCtx init twice", KR(ret)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID == dir_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(dir_id)); + } else if (OB_UNLIKELY(!io_flag.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_flag)); + } else if (OB_UNLIKELY(io_timeout_ms < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_timeout_ms)); + } else { + fd_ = fd; + dir_id_ = dir_id; + is_read_ = is_read; + io_flag_ = io_flag; + io_timeout_ms_ = io_timeout_ms; + disable_page_cache_ = disable_page_cache; + is_inited_ = true; + } + return ret; +} + +void ObTmpFileIOCtx::reuse() +{ + buf_ = nullptr; + buf_size_ = -1; + done_size_ = -1; + todo_size_ = -1; + read_offset_in_file_ = -1; + for (int32_t i = 0; i < io_handles_.count(); i++) { + io_handles_.at(i).handle_.reset(); + } + for (int32_t i = 0; i < page_cache_handles_.count(); i++) { + page_cache_handles_.at(i).page_handle_.reset(); + } + io_handles_.reset(); + page_cache_handles_.reset(); +} + +void ObTmpFileIOCtx::reset() +{ + reuse(); + is_inited_ = false; + is_read_ = false; + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + dir_id_ = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + disable_page_cache_ = false; + io_flag_.reset(); + io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; +} + +bool ObTmpFileIOCtx::is_valid() const +{ + return is_inited_ && + fd_ != ObTmpFileGlobal::INVALID_TMP_FILE_FD && + dir_id_ != ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID && + nullptr != buf_ && buf_size_ > 0 && done_size_ >= 0 && todo_size_ >= 0 && + (is_read_ ? read_offset_in_file_ >= 0 : true) && + io_timeout_ms_ >= 0 && io_flag_.is_valid(); +} + +int ObTmpFileIOCtx::prepare_read(char *read_buf, const int64_t read_size) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", KR(ret)); + } else if (OB_ISNULL(read_buf)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(read_buf)); + } else if (OB_UNLIKELY(read_size <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(read_size)); + } else { + is_read_ = true; + buf_ = read_buf; + buf_size_ = read_size; + done_size_ = 0; + todo_size_ = read_size; + read_offset_in_file_ = -1; + } + return ret; +} + +int ObTmpFileIOCtx::prepare_read(char *read_buf, const int64_t read_size, const int64_t read_offset) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(read_offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(read_offset)); + } else if (OB_FAIL(prepare_read(read_buf, read_size))) { + LOG_WARN("failed to prepare read", KR(ret), KP(read_buf), K(read_size)); + } else { + read_offset_in_file_ = read_offset; + } + return ret; +} + +int ObTmpFileIOCtx::prepare_write(char *write_buf, const int64_t write_size) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", KR(ret)); + } else if (OB_ISNULL(write_buf)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(write_buf)); + } else if (OB_UNLIKELY(write_size <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(write_size)); + } else { + is_read_ = false; + buf_ = write_buf; + buf_size_ = write_size; + done_size_ = 0; + todo_size_ = write_size; + read_offset_in_file_ = -1; + } + return ret; +} + +int ObTmpFileIOCtx::update_data_size(const int64_t size) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", KR(ret)); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid ctx", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(size > todo_size_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(size), K(todo_size_)); + } else if (OB_UNLIKELY(is_read_ && read_offset_in_file_ < 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("read offset is invalid", KR(ret), K(read_offset_in_file_)); + } else { + if (is_read_) { + read_offset_in_file_ += size; + } + done_size_ += size; + todo_size_ -= size; + } + return ret; +} + +int ObTmpFileIOCtx::wait() +{ + int ret = OB_SUCCESS; + + const int64_t timeout_ms = std::max(GCONF._data_storage_io_timeout / 1000, io_timeout_ms_); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", KR(ret)); + } else if (timeout_ms < 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to wait, invalid argument, timeout must be positive", KR(ret), K(timeout_ms)); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid ctx", KR(ret), KPC(this)); + } else if (!is_read_) { + // due to tmp file always writes data in buffer, + // there are no asynchronous io tasks need to wait + // do nothing + } else if (OB_FAIL(wait_read_finish_(timeout_ms))) { + STORAGE_LOG(WARN, "wait read finish failed", KR(ret), K(timeout_ms), K(is_read_)); + } + + return ret; +} + +int ObTmpFileIOCtx::wait_read_finish_(const int64_t timeout_ms) +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(!is_read_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("the handle is prepared for writing, not allowed to wait read finish", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(buf_size_ != done_size_ + todo_size_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("done_size_ + todo_size_ is not equal to buf size", KR(ret), KPC(this)); + } else if (OB_FAIL(do_read_wait_())) { + LOG_WARN("fail to wait tmp file io", KR(ret), K(fd_), K(timeout_ms)); + } + + return ret; +} + +int ObTmpFileIOCtx::do_read_wait_() +{ + int ret = OB_SUCCESS; + + for (int64_t i = 0; OB_SUCC(ret) && i < page_cache_handles_.count(); ++i) { + ObPageCacheHandle &page_cache_handle = page_cache_handles_.at(i); + if (OB_UNLIKELY(!page_cache_handle.is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page cache handle is not valid", KR(ret), K(page_cache_handle)); + } else { + const char * page_buf = page_cache_handle.page_handle_.value_->get_buffer(); + const int64_t offset_in_page = page_cache_handle.offset_in_src_data_buf_; + const int64_t read_size = page_cache_handle.read_size_; + char * read_buf = page_cache_handle.dest_user_read_buf_; + if (OB_UNLIKELY(!check_buf_range_valid(read_buf, read_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid range", KR(ret), K(read_buf), K(read_size), K(buf_size_)); + } else if (OB_UNLIKELY(offset_in_page + read_size > ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("read size is over than page range", KR(ret), KPC(this), K(offset_in_page), K(read_size)); + } else { + MEMCPY(read_buf, page_buf + offset_in_page, read_size); + page_cache_handle.page_handle_.reset(); + } + } + } + if (OB_SUCC(ret)) { + page_cache_handles_.reset(); + } + + for (int64_t i = 0; OB_SUCC(ret) && i < block_cache_handles_.count(); ++i) { + ObBlockCacheHandle &block_cache_handle = block_cache_handles_.at(i); + if (OB_UNLIKELY(!block_cache_handle.is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("block cache handle is not valid", KR(ret), K(block_cache_handle)); + } else { + const char * block_buf = block_cache_handle.block_handle_.value_->get_buffer(); + const int64_t offset_in_block = block_cache_handle.offset_in_src_data_buf_; + const int64_t read_size = block_cache_handle.read_size_; + char * read_buf = block_cache_handle.dest_user_read_buf_; + if (OB_UNLIKELY(!check_buf_range_valid(read_buf, read_size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid range", KR(ret), K(read_buf), K(read_size), K(buf_size_)); + } else if (OB_UNLIKELY(offset_in_block + read_size > OB_DEFAULT_MACRO_BLOCK_SIZE)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("read size is over than macro block range", KR(ret), KPC(this), K(offset_in_block), K(read_size)); + } else { + MEMCPY(read_buf, block_buf + offset_in_block, read_size); + block_cache_handle.block_handle_.reset(); + } + } + } + if (OB_SUCC(ret)) { + block_cache_handles_.reset(); + } + + // Wait read io finish. + for (int64_t i = 0; OB_SUCC(ret) && i < io_handles_.count(); ++i) { + ObIOReadHandle &io_handle = io_handles_.at(i); + if (OB_UNLIKELY(!io_handle.is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("io handle is not valid", KR(ret), K(io_handle)); + } else if (OB_FAIL(io_handle.handle_.wait())) { + LOG_WARN("fail to do object handle read wait", KR(ret)); + } else { + const char * data_buf = io_handle.handle_.get_buffer(); + const int64_t offset = io_handle.offset_in_src_data_buf_; + const int64_t size = io_handle.read_size_; + char * read_buf = io_handle.dest_user_read_buf_; + if (OB_UNLIKELY(!check_buf_range_valid(read_buf, size))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid range", KR(ret), K(read_buf), K(size), K(buf_size_)); + } else { + MEMCPY(read_buf, data_buf + offset, size); + io_handle.handle_.reset(); + } + } + } + + if (OB_SUCC(ret)) { + io_handles_.reset(); + } + + return ret; +} + +ObTmpFileIOCtx::ObIReadHandle::ObIReadHandle() + : dest_user_read_buf_(nullptr), offset_in_src_data_buf_(0), read_size_(0) +{ +} + +ObTmpFileIOCtx::ObIReadHandle::ObIReadHandle( + char *dest_user_read_buf, const int64_t offset_in_src_data_buf, const int64_t read_size) + : dest_user_read_buf_(dest_user_read_buf), offset_in_src_data_buf_(offset_in_src_data_buf), read_size_(read_size) +{ +} + +ObTmpFileIOCtx::ObIReadHandle::~ObIReadHandle() +{ +} + +ObTmpFileIOCtx::ObIReadHandle::ObIReadHandle(const ObTmpFileIOCtx::ObIReadHandle &other) +{ + *this = other; +} + +ObTmpFileIOCtx::ObIReadHandle &ObTmpFileIOCtx::ObIReadHandle::operator=( + const ObTmpFileIOCtx::ObIReadHandle &other) +{ + if (&other != this) { + offset_in_src_data_buf_ = other.offset_in_src_data_buf_; + dest_user_read_buf_ = other.dest_user_read_buf_; + read_size_ = other.read_size_; + } + return *this; +} + +ObTmpFileIOCtx::ObIOReadHandle::ObIOReadHandle() + : ObIReadHandle(), handle_() +{ +} + +ObTmpFileIOCtx::ObIOReadHandle::ObIOReadHandle( + char *dest_user_read_buf, const int64_t offset_in_src_data_buf, const int64_t read_size, + ObTmpFileBlockHandle block_handle) + : ObIReadHandle(dest_user_read_buf, offset_in_src_data_buf, read_size), handle_(), block_handle_(block_handle) +{ +} + +ObTmpFileIOCtx::ObIOReadHandle::~ObIOReadHandle() +{ +} + +ObTmpFileIOCtx::ObIOReadHandle::ObIOReadHandle(const ObTmpFileIOCtx::ObIOReadHandle &other) +{ + *this = other; +} + +ObTmpFileIOCtx::ObIOReadHandle &ObTmpFileIOCtx::ObIOReadHandle::operator=( + const ObTmpFileIOCtx::ObIOReadHandle &other) +{ + if (&other != this) { + handle_ = other.handle_; + block_handle_ = other.block_handle_; + ObIReadHandle::operator=(other); + } + return *this; +} + +bool ObTmpFileIOCtx::ObIOReadHandle::is_valid() +{ + bool bret = false; + if (OB_NOT_NULL(dest_user_read_buf_) && offset_in_src_data_buf_ >= 0 && + read_size_ >= 0 && + read_size_ <= OB_DEFAULT_MACRO_BLOCK_SIZE && + handle_.is_valid() && block_handle_.is_inited()) { + bret = true; + } + return bret; +} + +ObTmpFileIOCtx::ObBlockCacheHandle::ObBlockCacheHandle() + : ObIReadHandle(), block_handle_() +{ +} + +ObTmpFileIOCtx::ObBlockCacheHandle::ObBlockCacheHandle(const ObTmpBlockValueHandle &block_handle, + char *dest_user_read_buf, const int64_t offset_in_src_data_buf, const int64_t read_size) + : ObIReadHandle(dest_user_read_buf, offset_in_src_data_buf, read_size), block_handle_(block_handle) +{ +} + +ObTmpFileIOCtx::ObBlockCacheHandle::~ObBlockCacheHandle() +{ +} + +ObTmpFileIOCtx::ObBlockCacheHandle::ObBlockCacheHandle( + const ObTmpFileIOCtx::ObBlockCacheHandle &other) +{ + *this = other; +} + +ObTmpFileIOCtx::ObBlockCacheHandle &ObTmpFileIOCtx::ObBlockCacheHandle::operator=( + const ObTmpFileIOCtx::ObBlockCacheHandle &other) +{ + if (&other != this) { + block_handle_ = other.block_handle_; + ObIReadHandle::operator=(other); + } + return *this; +} + +bool ObTmpFileIOCtx::ObBlockCacheHandle::is_valid() +{ + bool bret = false; + if (OB_NOT_NULL(dest_user_read_buf_) && offset_in_src_data_buf_ >= 0 && + read_size_ >= 0 && + read_size_ <= OB_DEFAULT_MACRO_BLOCK_SIZE && + OB_NOT_NULL(block_handle_.value_) && + block_handle_.handle_.is_valid()) { + bret = true; + } + return bret; +} + +ObTmpFileIOCtx::ObPageCacheHandle::ObPageCacheHandle() + : ObIReadHandle(), page_handle_() +{ +} + +ObTmpFileIOCtx::ObPageCacheHandle::ObPageCacheHandle(const ObTmpPageValueHandle &page_handle, + char *dest_user_read_buf, const int64_t offset_in_src_data_buf, const int64_t read_size) + : ObIReadHandle(dest_user_read_buf, offset_in_src_data_buf, read_size), page_handle_(page_handle) +{ +} + +ObTmpFileIOCtx::ObPageCacheHandle::~ObPageCacheHandle() +{ +} + +ObTmpFileIOCtx::ObPageCacheHandle::ObPageCacheHandle( + const ObTmpFileIOCtx::ObPageCacheHandle &other) +{ + *this = other; +} + +ObTmpFileIOCtx::ObPageCacheHandle &ObTmpFileIOCtx::ObPageCacheHandle::operator=( + const ObTmpFileIOCtx::ObPageCacheHandle &other) +{ + if (&other != this) { + page_handle_ = other.page_handle_; + ObIReadHandle::operator=(other); + } + return *this; +} + +bool ObTmpFileIOCtx::ObPageCacheHandle::is_valid() +{ + bool bret = false; + if (OB_NOT_NULL(dest_user_read_buf_) && offset_in_src_data_buf_ >= 0 && + read_size_ >= 0 && + read_size_ <= ObTmpFileGlobal::PAGE_SIZE && + OB_NOT_NULL(page_handle_.value_) && + page_handle_.handle_.is_valid()) { + bret = true; + } + return bret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_io_ctx.h b/src/storage/tmp_file/ob_tmp_file_io_ctx.h new file mode 100644 index 0000000000..bd985c8f17 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_io_ctx.h @@ -0,0 +1,170 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_CTX_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_CTX_H_ + +#include "storage/blocksstable/ob_macro_block_handle.h" +#include "storage/tmp_file/ob_tmp_file_cache.h" +#include "storage/tmp_file/ob_tmp_file_block_manager.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpFileIOCtx +{ +public: + ObTmpFileIOCtx(); + ~ObTmpFileIOCtx(); + + int init(const int64_t fd, const int64_t dir_id, + const bool is_read, + const common::ObIOFlag io_flag, + const int64_t io_timeout_ms, + const bool disable_page_cache); + void reuse(); + void reset(); + bool is_valid() const; + int prepare_read(char *read_buf, const int64_t read_size); + int prepare_read(char *read_buf, const int64_t read_size, const int64_t read_offset); + int prepare_write(char *write_buf, const int64_t write_size); + int update_data_size(const int64_t size); + int wait(); + +public: + OB_INLINE bool check_buf_range_valid(const char* buffer, const int64_t length) const + { + return buffer != nullptr && buffer >= buf_ && buffer + length <= buf_ + buf_size_; + } + OB_INLINE int64_t get_fd() const { return fd_; } + OB_INLINE int64_t get_dir_id() const { return dir_id_; } + OB_INLINE bool is_read() const { return is_read_; } + OB_INLINE char *get_buffer() { return buf_; } + OB_INLINE char *get_buffer() const { return buf_; } + OB_INLINE char *get_todo_buffer() { return buf_ + done_size_; } + OB_INLINE char *get_todo_buffer() const { return buf_ + done_size_; } + OB_INLINE int64_t get_done_size() const { return done_size_; } + OB_INLINE int64_t get_todo_size() const { return todo_size_; } + OB_INLINE int64_t get_read_offset_in_file() const { return read_offset_in_file_; } + OB_INLINE void set_read_offset_in_file(const int64_t offset) { read_offset_in_file_ = offset; } + OB_INLINE bool is_disable_page_cache() const { return disable_page_cache_; } + OB_INLINE common::ObIOFlag get_io_flag() const { return io_flag_; } + OB_INLINE int64_t get_io_timeout_ms() const { return io_timeout_ms_; } + + TO_STRING_KV(K(is_inited_), K(is_read_), + K(fd_), K(dir_id_), KP(buf_), + K(buf_size_), K(done_size_), K(todo_size_), + K(read_offset_in_file_), + K(disable_page_cache_), + K(io_flag_), K(io_timeout_ms_)); + +public: + struct ObIReadHandle + { + ObIReadHandle(); + ObIReadHandle(char *dest_user_read_buf, + const int64_t offset_in_src_data_buf, + const int64_t read_size); + ~ObIReadHandle(); + ObIReadHandle(const ObIReadHandle &other); + ObIReadHandle &operator=(const ObIReadHandle &other); + virtual bool is_valid() = 0; + TO_STRING_KV(KP(dest_user_read_buf_), K(offset_in_src_data_buf_), K(read_size_)); + + char *dest_user_read_buf_; // user buf + int64_t offset_in_src_data_buf_; + int64_t read_size_; + }; + + struct ObIOReadHandle final : public ObIReadHandle + { + ObIOReadHandle(); + ObIOReadHandle(char *dest_user_read_buf, + const int64_t offset_in_src_data_buf, const int64_t read_size, + ObTmpFileBlockHandle block_handle); + ~ObIOReadHandle(); + ObIOReadHandle(const ObIOReadHandle &other); + ObIOReadHandle &operator=(const ObIOReadHandle &other); + virtual bool is_valid() override; + INHERIT_TO_STRING_KV("ObIReadHandle", ObIReadHandle, K(handle_), K(block_handle_)); + blocksstable::ObMacroBlockHandle handle_; + ObTmpFileBlockHandle block_handle_; + }; + + struct ObBlockCacheHandle final : public ObIReadHandle + { + ObBlockCacheHandle(); + ObBlockCacheHandle(const ObTmpBlockValueHandle &block_handle_, char *dest_user_read_buf, + const int64_t offset_in_src_data_buf, + const int64_t read_size); + ~ObBlockCacheHandle(); + ObBlockCacheHandle(const ObBlockCacheHandle &other); + ObBlockCacheHandle &operator=(const ObBlockCacheHandle &other); + virtual bool is_valid() override; + INHERIT_TO_STRING_KV("ObIReadHandle", ObIReadHandle, K(block_handle_)); + ObTmpBlockValueHandle block_handle_; + }; + + struct ObPageCacheHandle final : public ObIReadHandle + { + ObPageCacheHandle(); + ObPageCacheHandle(const ObTmpPageValueHandle &page_handle, char *dest_user_read_buf, + const int64_t offset_in_src_data_buf, + const int64_t read_size); + ~ObPageCacheHandle(); + ObPageCacheHandle(const ObPageCacheHandle &other); + ObPageCacheHandle &operator=(const ObPageCacheHandle &other); + virtual bool is_valid() override; + INHERIT_TO_STRING_KV("ObIReadHandle", ObIReadHandle, K(page_handle_)); + ObTmpPageValueHandle page_handle_; + }; + + common::ObIArray &get_io_handles() + { + return io_handles_; + } + common::ObIArray &get_page_cache_handles() + { + return page_cache_handles_; + } + common::ObIArray &get_block_cache_handles() + { + return block_cache_handles_; + } + +private: + int wait_read_finish_(const int64_t timeout_ms); + int do_read_wait_(); + +private: + bool is_inited_; + bool is_read_; + int64_t fd_; + int64_t dir_id_; + char *buf_; + int64_t buf_size_; + int64_t done_size_; + int64_t todo_size_; + int64_t read_offset_in_file_; + bool disable_page_cache_; + common::ObIOFlag io_flag_; + int64_t io_timeout_ms_; + common::ObSEArray io_handles_; + common::ObSEArray page_cache_handles_; + common::ObSEArray block_cache_handles_; + DISALLOW_COPY_AND_ASSIGN(ObTmpFileIOCtx); +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_CTX_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_io_define.cpp b/src/storage/tmp_file/ob_tmp_file_io_define.cpp new file mode 100644 index 0000000000..de904c1b9b --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_io_define.cpp @@ -0,0 +1,236 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_io_define.h" + +namespace oceanbase +{ +using namespace storage; +using namespace share; + +namespace tmp_file +{ +/* -------------------------- ObTmpFileIOInfo --------------------------- */ + +ObTmpFileIOInfo::ObTmpFileIOInfo() + : fd_(0), dir_id_(0), buf_(nullptr), size_(0), + disable_page_cache_(false), + io_desc_(), io_timeout_ms_(DEFAULT_IO_WAIT_TIME_MS) +{} + +ObTmpFileIOInfo::~ObTmpFileIOInfo() +{ + reset(); +} + +void ObTmpFileIOInfo::reset() +{ + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + dir_id_ = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + size_ = 0; + io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; + buf_ = nullptr; + io_desc_.reset(); + disable_page_cache_ = false; +} + +bool ObTmpFileIOInfo::is_valid() const +{ + return fd_ != ObTmpFileGlobal::INVALID_TMP_FILE_FD && + dir_id_ != ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID && + size_ > 0 && + nullptr != buf_ && io_desc_.is_valid() && io_timeout_ms_ >= 0; +} + +/* -------------------------- ObTmpFileIOHandle --------------------------- */ + +ObTmpFileIOHandle::ObTmpFileIOHandle() + : is_inited_(false), + tmp_file_handle_(), + ctx_(), + buf_(nullptr), + update_offset_in_file_(false), + buf_size_(-1), + done_size_(-1), + read_offset_in_file_(-1) +{ +} + +ObTmpFileIOHandle::~ObTmpFileIOHandle() +{ + reset(); +} + +void ObTmpFileIOHandle::reset() +{ + is_inited_ = false; + ctx_.reset(); + tmp_file_handle_.reset(); + buf_ = nullptr; + update_offset_in_file_ = false; + buf_size_ = -1; + done_size_ = -1; + read_offset_in_file_ = -1; +} + +int ObTmpFileIOHandle::init_write(const ObTmpFileIOInfo &io_info, ObTmpFileHandle &tmp_file_handle) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileIOHandle has been inited twice", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(!io_info.is_valid()) || OB_ISNULL(tmp_file_handle.get())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(ctx_.init(io_info.fd_, io_info.dir_id_, false /*is_read*/, io_info.io_desc_, + io_info.io_timeout_ms_, io_info.disable_page_cache_))) { + LOG_WARN("failed to init io handle context", KR(ret), K(io_info)); + } else if (OB_FAIL(ctx_.prepare_write(io_info.buf_, io_info.size_))) { + LOG_WARN("fail to prepare write context", KR(ret), KPC(this)); + } else { + is_inited_ = true; + tmp_file_handle_ = tmp_file_handle; + buf_ = io_info.buf_; + buf_size_ = io_info.size_; + done_size_ = 0; + } + + return ret; +} + +int ObTmpFileIOHandle::init_pread(const ObTmpFileIOInfo &io_info, const int64_t read_offset, ObTmpFileHandle &tmp_file_handle) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileIOHandle has been inited twice", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(!io_info.is_valid()) || OB_ISNULL(tmp_file_handle.get())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_info)); + } else if (OB_UNLIKELY(read_offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(read_offset)); + } else if (OB_FAIL(ctx_.init(io_info.fd_, io_info.dir_id_, true /*is_read*/, io_info.io_desc_, + io_info.io_timeout_ms_, io_info.disable_page_cache_))) { + LOG_WARN("failed to init io handle context", KR(ret), K(io_info)); + } else if (OB_FAIL(ctx_.prepare_read(io_info.buf_, MIN(io_info.size_, ObTmpFileGlobal::TMP_FILE_READ_BATCH_SIZE), read_offset))) { + LOG_WARN("fail to prepare read context", KR(ret), KPC(this), K(read_offset)); + } else { + is_inited_ = true; + tmp_file_handle_ = tmp_file_handle; + buf_ = io_info.buf_; + buf_size_ = io_info.size_; + done_size_ = 0; + read_offset_in_file_ = read_offset; + } + + return ret; +} + +int ObTmpFileIOHandle::init_read(const ObTmpFileIOInfo &io_info, ObTmpFileHandle &tmp_file_handle) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTmpFileIOHandle has been inited twice", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(!io_info.is_valid()) || OB_ISNULL(tmp_file_handle.get())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(ctx_.init(io_info.fd_, io_info.dir_id_, true /*is_read*/, io_info.io_desc_, + io_info.io_timeout_ms_, io_info.disable_page_cache_))) { + LOG_WARN("failed to init io handle context", KR(ret), K(io_info)); + } else if (OB_FAIL(ctx_.prepare_read(io_info.buf_, MIN(io_info.size_, ObTmpFileGlobal::TMP_FILE_READ_BATCH_SIZE)))) { + LOG_WARN("fail to prepare read context", KR(ret), KPC(this)); + } else { + is_inited_ = true; + tmp_file_handle_ = tmp_file_handle; + buf_ = io_info.buf_; + buf_size_ = io_info.size_; + done_size_ = 0; + read_offset_in_file_ = -1; + update_offset_in_file_ = true; + } + + return ret; +} + +bool ObTmpFileIOHandle::is_valid() const +{ + return is_inited_ && + nullptr != buf_ && + done_size_ >= 0 && buf_size_ > 0 && + buf_size_ >= done_size_; +} + +int ObTmpFileIOHandle::wait() +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", KR(ret)); + } else if (OB_UNLIKELY(!is_valid())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid handle", KR(ret), KPC(this)); + } else if (is_finished() || !ctx_.is_read()) { + // do nothing + } else if (OB_FAIL(ctx_.wait())) { + LOG_WARN("fail to wait tmp file io", KR(ret), K(ctx_), KPC(this)); + } else if (OB_FAIL(handle_finished_ctx_(ctx_))) { + LOG_WARN("fail to handle finished ctx", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(done_size_ > buf_size_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("done size is larger than total todo size", KR(ret), KPC(this)); + } else { + while (OB_SUCC(ret) && !is_finished()) { + if (OB_FAIL(ctx_.prepare_read(buf_ + done_size_, + MIN(buf_size_ - done_size_, + ObTmpFileGlobal::TMP_FILE_READ_BATCH_SIZE), + read_offset_in_file_))) { + LOG_WARN("fail to generate read ctx", KR(ret), KPC(this)); + } else if (OB_FAIL(tmp_file_handle_.get()->aio_pread(ctx_))) { + LOG_WARN("fail to continue read once batch", KR(ret), K(ctx_)); + } else if (OB_FAIL(ctx_.wait())) { + LOG_WARN("fail to wait tmp file io", KR(ret), K(ctx_)); + } else if (OB_FAIL(handle_finished_ctx_(ctx_))) { + LOG_WARN("fail to handle finished ctx", KR(ret), KPC(this)); + } + } // end while + } + + if (update_offset_in_file_ && (OB_SUCC(ret) || OB_ITER_END == ret)) { + tmp_file_handle_.get()->update_read_offset(read_offset_in_file_); + } + return ret; +} + +int ObTmpFileIOHandle::handle_finished_ctx_(ObTmpFileIOCtx &ctx) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!ctx.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(ctx)); + } else { + if (ctx_.is_read()) { + read_offset_in_file_ = ctx.get_read_offset_in_file(); + } + done_size_ += ctx.get_done_size(); + ctx.reuse(); + } + + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_io_define.h b/src/storage/tmp_file/ob_tmp_file_io_define.h new file mode 100644 index 0000000000..e17daecac7 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_io_define.h @@ -0,0 +1,81 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_DEFINE_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_DEFINE_H_ + +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_io_ctx.h" + +namespace oceanbase +{ +namespace tmp_file +{ +struct ObTmpFileIOInfo final +{ + ObTmpFileIOInfo(); + ~ObTmpFileIOInfo(); + void reset(); + bool is_valid() const; + TO_STRING_KV(K(fd_), K(dir_id_), KP(buf_), K(size_), K(disable_page_cache_), + K(io_timeout_ms_), K(io_desc_)); + + int64_t fd_; + int64_t dir_id_; + char *buf_; + int64_t size_; + bool disable_page_cache_; + common::ObIOFlag io_desc_; + int64_t io_timeout_ms_; +}; + +class ObTmpFileIOHandle final +{ +public: + ObTmpFileIOHandle(); + ~ObTmpFileIOHandle(); + int init_write(const ObTmpFileIOInfo &io_info, ObTmpFileHandle &tmp_file_handle); + int init_read(const ObTmpFileIOInfo &io_info, ObTmpFileHandle &tmp_file_handle); + int init_pread(const ObTmpFileIOInfo &io_info, const int64_t read_offset, ObTmpFileHandle &tmp_file_handle); + int wait(); + void reset(); + bool is_valid() const; + + TO_STRING_KV(K(is_inited_), K(tmp_file_handle_), K(ctx_), + KP(buf_), K(update_offset_in_file_), + K(buf_size_), K(done_size_), + K(read_offset_in_file_)); +public: + OB_INLINE char *get_buffer() { return buf_; } + OB_INLINE int64_t get_done_size() const { return done_size_; } + OB_INLINE int64_t get_buf_size() const { return buf_size_; } + OB_INLINE ObTmpFileIOCtx &get_io_ctx() { return ctx_; } + OB_INLINE bool is_finished() const { return done_size_ == buf_size_; } +private: + int handle_finished_ctx_(ObTmpFileIOCtx &ctx); + +private: + bool is_inited_; + ObTmpFileHandle tmp_file_handle_; + ObTmpFileIOCtx ctx_; + char *buf_; + bool update_offset_in_file_; + int64_t buf_size_; // excepted total read or write size + int64_t done_size_; // has finished read or write size + int64_t read_offset_in_file_; // records the beginning read offset for current read ctx + + DISALLOW_COPY_AND_ASSIGN(ObTmpFileIOHandle); +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_IO_DEFINE_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_manager.cpp b/src/storage/tmp_file/ob_tmp_file_manager.cpp new file mode 100644 index 0000000000..69f4109a96 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_manager.cpp @@ -0,0 +1,530 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_manager.h" +#include "storage/tmp_file/ob_tmp_file_cache.h" + +namespace oceanbase +{ +namespace tmp_file +{ +ObTenantTmpFileManager::ObTenantTmpFileManager() + : is_inited_(false), + tenant_id_(OB_INVALID_TENANT_ID), + last_access_tenant_config_ts_(-1), + last_meta_mem_limit_(META_DEFAULT_LIMIT), + tmp_file_allocator_(), + callback_allocator_(), + wbp_index_cache_allocator_(), + wbp_index_cache_bucket_allocator_(), + files_(), + tmp_file_block_manager_(), + page_cache_controller_(tmp_file_block_manager_), + current_fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), + current_dir_id_(ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID) +{ +} + +ObTenantTmpFileManager::~ObTenantTmpFileManager() +{ + destroy(); +} + +int ObTenantTmpFileManager::mtl_init(ObTenantTmpFileManager *&manager) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(manager)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to mtl init tmp file manager, null pointer argument", KR(ret), KP(manager)); + } else if (OB_FAIL(manager->init())) { + LOG_WARN("fail to init ObTenantTmpFileManager", KR(ret)); + } + return ret; +} + +ObTenantTmpFileManager &ObTenantTmpFileManager::get_instance() +{ + int ret = OB_SUCCESS; + ObTenantTmpFileManager *tmp_file_manager = MTL(tmp_file::ObTenantTmpFileManager *); + if (OB_ISNULL(tmp_file_manager)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("ObTenantTmpFileManager is null, please check whether MTL module is normal", KR(ret)); + ob_abort(); + } + + return *tmp_file_manager; +} + +int ObTenantTmpFileManager::init() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("ObTenantTmpFileManager init twice", KR(ret), K(is_inited_)); + } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id_ = MTL_ID()))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid tenant id", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(files_.init("TmpFileMap", tenant_id_))) { + LOG_WARN("fail to init tmp files map", KR(ret)); + } else if (FALSE_IT(refresh_meta_memory_limit())) { + } else if (OB_FAIL(tmp_file_block_manager_.init(tenant_id_, last_meta_mem_limit_))) { + LOG_WARN("fail to init tenant tmp file block manager", KR(ret)); + } else if (OB_FAIL(tmp_file_allocator_.init(common::OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObModIds::OB_TMP_FILE_MANAGER, tenant_id_, + last_meta_mem_limit_))) { + LOG_WARN("fail to init tmp file allocator", KR(ret), K(tenant_id_), K(last_meta_mem_limit_)); + } else if (OB_FAIL(callback_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(tenant_id_, "TmpFileCallback", ObCtxIds::DEFAULT_CTX_ID)))) { + LOG_WARN("fail to init callback allocator", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(wbp_index_cache_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_NORMAL_BLOCK_SIZE, + ObMemAttr(tenant_id_, "TmpFileIndCache", + ObCtxIds::DEFAULT_CTX_ID)))) { + LOG_WARN("fail to init wbp index cache allocator", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(wbp_index_cache_bucket_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(tenant_id_, "TmpFileIndCBkt", + ObCtxIds::DEFAULT_CTX_ID)))) { + LOG_WARN("fail to init wbp index cache bucket allocator", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(page_cache_controller_.init(*this))) { + LOG_WARN("fail to init page cache controller", KR(ret)); + } else { + is_inited_ = true; + LOG_INFO("ObTenantTmpFileManager init successful", K(tenant_id_), KP(this)); + } + return ret; +} + +int ObTenantTmpFileManager::start() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(page_cache_controller_.start())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to start page cache controller background threads", KR(ret)); + } else { + LOG_INFO("ObTenantTmpFileManager start successful", K(tenant_id_), KP(this)); + } + return ret; +} + +void ObTenantTmpFileManager::stop() +{ + page_cache_controller_.stop(); + LOG_INFO("ObTenantTmpFileManager stop successful", K(tenant_id_), KP(this)); +} + +void ObTenantTmpFileManager::wait() +{ + page_cache_controller_.wait(); + LOG_INFO("ObTenantTmpFileManager wait successful", K(tenant_id_), KP(this)); +} + +void ObTenantTmpFileManager::destroy() +{ + last_access_tenant_config_ts_ = -1; + last_meta_mem_limit_ = META_DEFAULT_LIMIT; + page_cache_controller_.destroy(); + files_.destroy(); + tmp_file_block_manager_.destroy(); + tmp_file_allocator_.reset(); + callback_allocator_.reset(); + wbp_index_cache_allocator_.reset(); + wbp_index_cache_bucket_allocator_.reset(); + is_inited_ = false; + current_fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + current_dir_id_ = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + + LOG_INFO("ObTenantTmpFileManager destroy", K(tenant_id_), KP(this)); +} + +int ObTenantTmpFileManager::alloc_dir(int64_t &dir_id) +{ + int ret = OB_SUCCESS; + dir_id = ObTmpFileGlobal::INVALID_TMP_FILE_DIR_ID; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else { + dir_id = ATOMIC_AAF(¤t_dir_id_, 1); + } + + LOG_DEBUG("alloc dir over", KR(ret), K(dir_id), K(lbt())); + return ret; +} + +int ObTenantTmpFileManager::open(int64_t &fd, const int64_t &dir_id) +{ + int ret = OB_SUCCESS; + fd = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + void *buf = nullptr; + ObSharedNothingTmpFile *tmp_file = nullptr; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_ISNULL(buf = tmp_file_allocator_.alloc(sizeof(ObSharedNothingTmpFile), + lib::ObMemAttr(tenant_id_, "SNTmpFile")))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory for tmp file", + KR(ret), K(tenant_id_), K(sizeof(ObSharedNothingTmpFile))); + } else if (FALSE_IT(tmp_file = new (buf) ObSharedNothingTmpFile())) { + } else if (FALSE_IT(fd = ATOMIC_AAF(¤t_fd_, 1))) { + } else if (OB_FAIL(tmp_file->init(tenant_id_, fd, dir_id, + &tmp_file_block_manager_, &callback_allocator_, + &wbp_index_cache_allocator_, &wbp_index_cache_bucket_allocator_, + &page_cache_controller_))) { + LOG_WARN("fail to init tmp file", KR(ret), K(fd), K(dir_id)); + } else if (OB_FAIL(files_.insert(ObTmpFileKey(fd), tmp_file))) { + LOG_WARN("fail to set refactored to tmp file map", KR(ret), K(fd), KP(tmp_file)); + } + + if (OB_FAIL(ret) && OB_NOT_NULL(tmp_file)) { + tmp_file->~ObSharedNothingTmpFile(); + tmp_file_allocator_.free(tmp_file); + tmp_file = nullptr; + } + + LOG_INFO("open a tmp file over", KR(ret), K(fd), K(dir_id), KP(tmp_file), K(lbt())); + return ret; +} + +int ObTenantTmpFileManager::remove(const int64_t fd) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + int64_t start_remove_ts = ObTimeUtility::current_time(); + LOG_INFO("remove a tmp file start", KR(ret), K(start_remove_ts), K(fd), K(lbt())); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(files_.erase(ObTmpFileKey(fd), tmp_file_handle))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + LOG_INFO("erase non-exist tmp file", K(fd), K(lbt())); + } else { + LOG_WARN("fail to erase tmp file", KR(ret), K(fd), K(lbt())); + } + } else if (OB_ISNULL(tmp_file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), KP(tmp_file_handle.get()), K(fd)); + } else if (OB_FAIL(tmp_file_handle.get()->delete_file())) { + LOG_WARN("fail to delete tmp file", KR(ret), K(fd), K(lbt())); + } else { + ObSharedNothingTmpFile *tmp_file = tmp_file_handle.get(); + + int64_t LOG_WARN_TIMEOUT = 10 * 1000 * 1000; + int64_t LOG_ERROR_TIMEOUT = 60 * 1000 * 1000; + while(!tmp_file->can_remove()) + { + if (start_remove_ts + LOG_ERROR_TIMEOUT < ObTimeUtility::current_time()) { + LOG_ERROR("wait thread release reference too long", + K(start_remove_ts), KP(tmp_file), KPC(tmp_file), K(lbt())); + sleep(10); // 10s + } else if (start_remove_ts + LOG_WARN_TIMEOUT < ObTimeUtility::current_time()) { + LOG_WARN("wait thread release reference too long", + K(start_remove_ts), KP(tmp_file), KPC(tmp_file), K(lbt())); + sleep(2); // 2s + } else { + usleep(100 * 1000); // 100ms + } + } + tmp_file_handle.reset(); + tmp_file_allocator_.free(tmp_file); + } + + LOG_INFO("remove a tmp file over", KR(ret), K(start_remove_ts), K(fd), K(lbt())); + return ret; +} + +void ObTenantTmpFileManager::refresh_meta_memory_limit() +{ + int ret = OB_SUCCESS; + const int64_t last_access_ts = ATOMIC_LOAD(&last_access_tenant_config_ts_); + int64_t mem_limit = META_DEFAULT_LIMIT; + + if (last_access_ts < 0 || common::ObClockGenerator::getClock() - last_access_ts > REFRESH_CONFIG_INTERVAL) { + omt::ObTenantConfigGuard config(TENANT_CONF(tenant_id_)); + const int64_t tenant_mem_limit = lib::get_tenant_memory_limit(tenant_id_); + if (!config.is_valid() || 0 == tenant_mem_limit || INT64_MAX == tenant_mem_limit) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get tenant config or tenant memory", KR(ret), K(tenant_id_), K(tenant_mem_limit)); + } else { + const int64_t last_memory_limit = ATOMIC_LOAD(&last_meta_mem_limit_); + const int64_t limit_percentage_config = config->_temporary_file_meta_memory_limit_percentage; + const int64_t limit_percentage = 0 == limit_percentage_config ? 70 : limit_percentage_config; + mem_limit = tenant_mem_limit * limit_percentage / 100; + if (OB_UNLIKELY(mem_limit <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("memory limit isn't more than 0", KR(ret), K(mem_limit)); + } else if (FALSE_IT(ATOMIC_STORE(&last_access_tenant_config_ts_, common::ObClockGenerator::getClock()))) { + } else if (mem_limit != last_memory_limit) { + ATOMIC_STORE(&last_meta_mem_limit_, mem_limit); + tmp_file_allocator_.set_total_limit(mem_limit); + tmp_file_block_manager_.get_block_allocator().set_total_limit(mem_limit); + share::ObTaskController::get().allow_next_syslog(); + LOG_INFO("succeed to refresh tmp file meta memory limit", + K(tenant_id_), K(last_memory_limit), K(mem_limit), KP(this)); + } + } + } +} + +int ObTenantTmpFileManager::aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + io_handle.reset(); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (!io_info.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to aio read, invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(get_tmp_file(io_info.fd_, tmp_file_handle))) { + LOG_WARN("fail to get tmp file io handle", KR(ret), K(io_info)); + } else if (OB_FAIL(io_handle.init_read(io_info, tmp_file_handle))) { + LOG_WARN("fail to init io handle", KR(ret), K(io_info), K(tmp_file_handle)); + } else if (OB_FAIL(tmp_file_handle.get()->aio_pread(io_handle.get_io_ctx()))) { + LOG_WARN("fail to aio pread", KR(ret), K(io_info)); + } + + LOG_DEBUG("aio_read a tmp file over", KR(ret), K(io_info), K(io_handle), KPC(tmp_file_handle.get())); + return ret; +} + +int ObTenantTmpFileManager::aio_pread(const ObTmpFileIOInfo &io_info, + const int64_t offset, + ObTmpFileIOHandle &io_handle) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + io_handle.reset(); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (!io_info.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to aio read, invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(get_tmp_file(io_info.fd_, tmp_file_handle))) { + LOG_WARN("fail to get tmp file io handle", KR(ret), K(io_info)); + } else if (OB_FAIL(io_handle.init_pread(io_info, offset, tmp_file_handle))) { + LOG_WARN("fail to init io handle", KR(ret), K(io_info), K(tmp_file_handle)); + } else if (OB_FAIL(tmp_file_handle.get()->aio_pread(io_handle.get_io_ctx()))) { + LOG_WARN("fail to aio pread", KR(ret), K(io_info)); + } + + LOG_DEBUG("aio_pread a tmp file over", KR(ret), K(io_info), K(offset), K(io_handle), KPC(tmp_file_handle.get())); + return ret; +} + +int ObTenantTmpFileManager::read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + io_handle.reset(); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (!io_info.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to aio read, invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(get_tmp_file(io_info.fd_, tmp_file_handle))) { + LOG_WARN("fail to get tmp file io handle", KR(ret), K(io_info)); + } else if (OB_FAIL(io_handle.init_read(io_info, tmp_file_handle))) { + LOG_WARN("fail to init io handle", KR(ret), K(io_info), K(tmp_file_handle)); + } else if (OB_FAIL(tmp_file_handle.get()->aio_pread(io_handle.get_io_ctx()))) { + LOG_WARN("fail to aio pread", KR(ret), K(io_info)); + } + + if (OB_SUCC(ret) || OB_ITER_END == ret) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(io_handle.wait())) { + LOG_WARN("fail to wait", KR(tmp_ret), K(io_info)); + } + ret = OB_SUCCESS == ret ? tmp_ret : ret; + } + + LOG_DEBUG("read a tmp file over", KR(ret), K(io_info), K(io_handle), KPC(tmp_file_handle.get())); + return ret; +} + +int ObTenantTmpFileManager::pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &io_handle) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + io_handle.reset(); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (!io_info.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to aio read, invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(get_tmp_file(io_info.fd_, tmp_file_handle))) { + LOG_WARN("fail to get tmp file io handle", KR(ret), K(io_info)); + } else if (OB_FAIL(io_handle.init_pread(io_info, offset, tmp_file_handle))) { + LOG_WARN("fail to init io handle", KR(ret), K(io_info), K(tmp_file_handle)); + } else if (OB_FAIL(tmp_file_handle.get()->aio_pread(io_handle.get_io_ctx()))) { + LOG_WARN("fail to aio pread", KR(ret), K(io_info)); + } + + if (OB_SUCC(ret) || OB_ITER_END == ret) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(io_handle.wait())) { + LOG_WARN("fail to wait", KR(tmp_ret), K(io_info)); + } + ret = OB_SUCCESS == ret ? tmp_ret : ret; + } + + LOG_DEBUG("pread a tmp file over", KR(ret), K(io_info), K(offset), K(io_handle), KPC(tmp_file_handle.get())); + return ret; +} + +int ObTenantTmpFileManager::aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + io_handle.reset(); + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (!io_info.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(io_info)); + } else if (OB_FAIL(get_tmp_file(io_info.fd_, tmp_file_handle))) { + LOG_WARN("fail to get tmp file io handle", KR(ret), K(io_info)); + } else if (OB_FAIL(io_handle.init_write(io_info, tmp_file_handle))) { + LOG_WARN("fail to init io handle", KR(ret), K(io_info), K(tmp_file_handle)); + } else if (OB_FAIL(tmp_file_handle.get()->aio_write(io_handle.get_io_ctx()))) { + LOG_WARN("fail to aio write", KR(ret), K(io_info)); + } + + LOG_DEBUG("aio_write a tmp file over", KR(ret), K(io_info), K(io_handle), KPC(tmp_file_handle.get())); + return ret; +} + +// tmp file is always buffer writing, there are no io tasks need to be waited +int ObTenantTmpFileManager::write(const ObTmpFileIOInfo &io_info) +{ + int ret = OB_SUCCESS; + ObTmpFileIOHandle io_handle; + + if (OB_FAIL(aio_write(io_info, io_handle))) { + LOG_WARN("fail to aio write", KR(ret), K(io_info)); + } + + LOG_DEBUG("write a tmp file over", KR(ret), K(io_info), K(io_handle)); + return ret; +} + +// attention: +// currently truncate() only release memory, but not release disk space. +// we will support to release disk space in future. +int ObTenantTmpFileManager::truncate(const int64_t fd, const int64_t offset) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_UNLIKELY(offset < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(offset)); + } else if (OB_FAIL(get_tmp_file(fd, tmp_file_handle))) { + LOG_WARN("fail to get tmp file handle", KR(ret), K(fd)); + } else if (OB_FAIL(tmp_file_handle.get()->truncate(offset))) { + LOG_WARN("fail to truncate", KR(ret), K(fd), K(offset)); + } else { + LOG_INFO("truncate a tmp file over", KR(ret), K(fd), K(offset)); + } + return ret; +} + +// Get tmp file and increase its refcnt +int ObTenantTmpFileManager::get_tmp_file(const int64_t fd, ObTmpFileHandle &file_handle) +{ + int ret = OB_SUCCESS; + + if (OB_FAIL(files_.get(ObTmpFileKey(fd), file_handle))) { + if (OB_HASH_NOT_EXIST == ret) { + LOG_WARN("tmp file does not exist", KR(ret), K(fd)); + } else { + LOG_WARN("fail to get tmp file", KR(ret), K(fd)); + } + } else if (OB_ISNULL(file_handle.get())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get invalid tmp file pointer", KR(ret), K(fd), KP(file_handle.get())); + } + + return ret; +} + +int ObTenantTmpFileManager::get_tmp_file_size(const int64_t fd, int64_t &size) +{ + int ret = OB_SUCCESS; + ObTmpFileHandle tmp_file_handle; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(get_tmp_file(fd, tmp_file_handle))) { + LOG_WARN("fail to get tmp file handle", KR(ret), K(fd)); + } else { + size = tmp_file_handle.get()->get_file_size(); + } + + LOG_DEBUG("get tmp file size", KR(ret), K(fd), K(size)); + return ret; +} + +int ObTenantTmpFileManager::get_macro_block_list(common::ObIArray ¯o_id_list) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(tmp_file_block_manager_.get_macro_block_list(macro_id_list))) { + LOG_WARN("fail to get macro block id list", KR(ret)); + } + + LOG_INFO("get tmp file macro block list", KR(ret), K(macro_id_list.count())); + return ret; +} + +int ObTenantTmpFileManager::get_macro_block_count(int64_t ¯o_block_count) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObTenantTmpFileManager has not been inited", KR(ret), K(tenant_id_)); + } else if (OB_FAIL(tmp_file_block_manager_.get_macro_block_count(macro_block_count))) { + LOG_WARN("fail to get macro block id count", KR(ret)); + } + + LOG_INFO("get tmp file macro block count", KR(ret), K(macro_block_count)); + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_manager.h b/src/storage/tmp_file/ob_tmp_file_manager.h new file mode 100644 index 0000000000..2d93532887 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_manager.h @@ -0,0 +1,111 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_MANAGER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_MANAGER_H_ + +#include "lib/hash/ob_linear_hash_map.h" +#include "lib/container/ob_array.h" +#include "lib/lock/ob_spin_rwlock.h" +#include "storage/blocksstable/ob_macro_block_id.h" +#include "storage/tmp_file/ob_shared_nothing_tmp_file.h" +#include "storage/tmp_file/ob_tmp_file_io_define.h" +#include "storage/tmp_file/ob_tmp_file_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_eviction_manager.h" +#include "storage/tmp_file/ob_tmp_file_page_cache_controller.h" + +namespace oceanbase +{ +namespace tmp_file +{ +struct ObTmpFileKey final +{ + explicit ObTmpFileKey(const int64_t fd) : fd_(fd) {} + OB_INLINE int hash(uint64_t &hash_val) const + { + hash_val = murmurhash(&fd_, sizeof(int64_t), 0); + return OB_SUCCESS; + } + OB_INLINE bool operator==(const ObTmpFileKey &other) const { return fd_ == other.fd_; } + TO_STRING_KV(K(fd_)); + int64_t fd_; +}; + +class ObTenantTmpFileManager final +{ +public: + typedef common::ObLinearHashMap TmpFileMap; +public: + ObTenantTmpFileManager(); + ~ObTenantTmpFileManager(); + static int mtl_init(ObTenantTmpFileManager *&manager); + static ObTenantTmpFileManager &get_instance(); + int init(); + int start(); + void stop(); + void wait(); + void destroy(); + + int alloc_dir(int64_t &dir_id); + int open(int64_t &fd, const int64_t &dir_id); + int remove(const int64_t fd); + + void refresh_meta_memory_limit(); + +public: + int aio_read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle); + int aio_pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &io_handle); + int read(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle); + int pread(const ObTmpFileIOInfo &io_info, const int64_t offset, ObTmpFileIOHandle &io_handle); + // NOTE: + // only support append write. + int aio_write(const ObTmpFileIOInfo &io_info, ObTmpFileIOHandle &io_handle); + // NOTE: + // only support append write. + int write(const ObTmpFileIOInfo &io_info); + int truncate(const int64_t fd, const int64_t offset); + +public: + int get_tmp_file(const int64_t fd, ObTmpFileHandle &file_handle); + int get_tmp_file_size(const int64_t fd, int64_t &size); + int get_macro_block_list(common::ObIArray ¯o_id_list); + int get_macro_block_count(int64_t ¯o_block_count); + OB_INLINE ObIAllocator * get_callback_allocator() { return &callback_allocator_; } + OB_INLINE ObTmpFileBlockManager &get_tmp_file_block_manager() { return tmp_file_block_manager_; } + OB_INLINE ObTmpFilePageCacheController &get_page_cache_controller() { return page_cache_controller_; } + +private: + static const int64_t REFRESH_CONFIG_INTERVAL = 5 * 60 * 1000 * 1000L; // 5min + static const int64_t META_DEFAULT_LIMIT = 15 * 1024L * 1024L * 1024L; + +private: + bool is_inited_; + uint64_t tenant_id_; + int64_t last_access_tenant_config_ts_; + int64_t last_meta_mem_limit_; + common::ObConcurrentFIFOAllocator tmp_file_allocator_; + common::ObFIFOAllocator callback_allocator_; + common::ObFIFOAllocator wbp_index_cache_allocator_; + common::ObFIFOAllocator wbp_index_cache_bucket_allocator_; + TmpFileMap files_; + ObTmpFileBlockManager tmp_file_block_manager_; + ObTmpFilePageCacheController page_cache_controller_; + + int64_t current_fd_; + int64_t current_dir_id_; +}; + +#define FILE_MANAGER_INSTANCE_V2 (::oceanbase::tmp_file::ObTenantTmpFileManager::get_instance()) +} // end namespace tmp_file +} // end namespace oceanbase + +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_MANAGER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_meta_tree.cpp b/src/storage/tmp_file/ob_tmp_file_meta_tree.cpp new file mode 100644 index 0000000000..b49a598683 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_meta_tree.cpp @@ -0,0 +1,3139 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include "ob_tmp_file_meta_tree.h" +#include "storage/tmp_file/ob_tmp_file_cache.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" + +namespace oceanbase +{ +using namespace storage; +using namespace share; + +namespace tmp_file +{ + +const uint16_t ObSharedNothingTmpFileMetaTree::PAGE_MAGIC_NUM = 0xa12f; +const int16_t ObSharedNothingTmpFileMetaTree::PAGE_HEADER_SIZE = 16; //16B +int16_t ObSharedNothingTmpFileMetaTree::MAX_DATA_ITEM_ARRAY_COUNT = 16; + +//only for unittest +int16_t ObSharedNothingTmpFileMetaTree::MAX_PAGE_ITEM_COUNT = INT16_MAX; + +ObSharedNothingTmpFileMetaTree::ObSharedNothingTmpFileMetaTree() : + is_inited_(false), + fd_(-1), + tree_epoch_(0), + root_item_(), + data_item_array_(), + level_page_range_array_(), + is_writing_(false), + lock_(common::ObLatchIds::TMP_FILE_LOCK), + last_truncate_leaf_info_(), + released_offset_(0), + stat_info_() +{ + STATIC_ASSERT(sizeof(struct ObSharedNothingTmpFileMetaItem) == 24, "size of tree meta item is mismatch"); + STATIC_ASSERT(sizeof(struct ObSharedNothingTmpFileDataItem) == 24, "size of tree data item is mismatch"); + STATIC_ASSERT(sizeof(struct ObSharedNothingTmpFileTreePageHeader) == 16, "size of tree page header is mismatch"); +} + +ObSharedNothingTmpFileMetaTree::~ObSharedNothingTmpFileMetaTree() +{ + reset(); +} + +void ObSharedNothingTmpFileMetaTree::reset() +{ + is_inited_ = false; + fd_ = -1; + wbp_ = NULL; + tree_epoch_ = 0, + root_item_.reset(); + data_item_array_.reset(); + level_page_range_array_.reset(); + is_writing_ = false; + last_truncate_leaf_info_.reset(); + released_offset_ = 0; + stat_info_.reset(); +} + +int ObSharedNothingTmpFileMetaTree::init(const int64_t fd, + ObTmpWriteBufferPool *wbp, + ObIAllocator *callback_allocator) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(wbp)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(wbp)); + } else { + data_item_array_.set_attr(ObMemAttr(MTL_ID(), "TFDataItemArr")); + level_page_range_array_.set_attr(ObMemAttr(MTL_ID(), "TFTreeLevelArr")); + fd_ = fd; + wbp_ = wbp; + callback_allocator_ = callback_allocator; + is_inited_ = true; + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::prepare_for_insert_items() +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (!root_item_.is_valid()) { + //do nothing + } else { + ObSharedNothingTmpFileMetaItem page_info; + if (OB_FAIL(get_rightmost_leaf_page_for_write_(page_info))) { + STORAGE_LOG(WARN, "fail to get rightmost leaf page for write", KR(ret), KPC(this)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::get_rightmost_leaf_page_for_write_( + ObSharedNothingTmpFileMetaItem &page_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_writing_ + || level_page_range_array_.empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected is_writing_ or level_array_", KR(ret), K(fd_), K(is_writing_), K(level_page_range_array_)); + } else { + is_writing_ = true; //need to be protected by lock_ + ObSharedNothingTmpFileMetaItem meta_item = root_item_; + uint32_t parent_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileMetaItem next_level_meta_item; + while (OB_SUCC(ret) + && 0 < meta_item.page_level_) { + next_level_meta_item.reset(); + if (OB_FAIL(get_last_item_of_internal_page_(parent_page_id, meta_item, next_level_meta_item))) { + STORAGE_LOG(WARN, "fail to get last item of internal page", KR(ret), K(fd_), K(parent_page_id), K(meta_item)); + } else { + parent_page_id = meta_item.buffer_page_id_; + meta_item = next_level_meta_item; + } + } + if (OB_SUCC(ret)) { + if (OB_UNLIKELY(0 != meta_item.page_level_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_level_", KR(ret), K(meta_item)); + } else if (OB_FAIL(cache_page_for_write_(parent_page_id, meta_item))) { + STORAGE_LOG(WARN, "fail to cache page for write", KR(ret), K(fd_), K(parent_page_id), K(meta_item)); + } else { + page_info = meta_item; + } + } + + //there is no need to remove the successfully loaded rightmost page from the write cache + if (OB_FAIL(ret)) { + is_writing_ = false; + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::get_last_item_of_internal_page_( + const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info, + ObSharedNothingTmpFileMetaItem &last_meta_item) +{ + int ret = OB_SUCCESS; + ObSharedNothingTmpFileTreePageHeader page_header; + if (OB_FAIL(cache_page_for_write_(parent_page_id, page_info))) { + STORAGE_LOG(WARN, "fail to cache page for write", KR(ret), K(fd_), K(parent_page_id), K(page_info)); + } else { + char* page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = level_page_range_array_[page_info.page_level_].evicted_page_num_ + + level_page_range_array_[page_info.page_level_].cached_page_num_ - 1; + ObTmpFilePageUniqKey page_key(page_info.page_level_, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, page_info.buffer_page_id_, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_info), K(page_key), K(level_page_range_array_)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (OB_UNLIKELY(0 >= page_header.item_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(page_header)); + } else if (OB_FAIL(read_item_(page_buff, page_header.item_num_ - 1, last_meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(page_header)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::insert_items( + const ObIArray &data_items) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (OB_UNLIKELY(data_items.empty())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(data_items)); + } else if (OB_UNLIKELY(!data_items.at(0).is_valid() || + released_offset_ > data_items.at(0).virtual_page_id_ * ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(data_items.at(0)), KPC(this)); + } else { + int64_t write_count = 0; + const int64_t data_item_count = data_items.count(); + ObSEArray level_origin_page_write_counts; + ObSEArray, 2> level_new_pages; + if (!root_item_.is_valid() && OB_FAIL(try_to_insert_items_to_array_(data_items, level_new_pages))) { + STORAGE_LOG(WARN, "fail to try to insert items to array", KR(ret), K(data_items), KPC(this)); + } else if (root_item_.is_valid()) { + //we set is_writing_ in previous step, + // so, we need not to worry about end pages of each level will be evicted. + if (OB_UNLIKELY(!is_writing_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected is_writing_", KR(ret), KPC(this)); + } else if (OB_FAIL(try_to_fill_rightmost_leaf_page_(data_items, level_new_pages, write_count, + level_origin_page_write_counts))) { + STORAGE_LOG(WARN, "fail to try to fill rightmost leaf page", KR(ret), K(data_items), K(level_new_pages), KPC(this)); + } else if (write_count < data_item_count) { + ObSEArray meta_items; + ObSharedNothingTmpFileMetaItem meta_item; + if (level_page_range_array_.count() <= 1) { + //there is only a root page + if (OB_FAIL(meta_items.push_back(root_item_))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(root_item_)); + } + } + //alloc new leaf pages + while (OB_SUCC(ret) + && write_count < data_item_count) { + meta_item.reset(); + if (OB_FAIL(add_new_page_and_fill_items_at_leaf_(data_items, write_count, meta_item, level_new_pages))) { + STORAGE_LOG(WARN, "fail to add new page and fill items at leaf", KR(ret), + K(data_items), K(write_count), K(level_new_pages), KPC(this)); + } else if (OB_FAIL(meta_items.push_back(meta_item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_item)); + } + } + //cascade to modify the internal pages + if (FAILEDx(cascade_modification_at_internal_(meta_items, + level_origin_page_write_counts, level_new_pages))) { + STORAGE_LOG(WARN, "fail to cascade modification at internal", KR(ret), K(meta_items), + K(level_origin_page_write_counts), K(level_new_pages), KPC(this)); + } + } + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(finish_insert_(ret, level_origin_page_write_counts, level_new_pages))) { + STORAGE_LOG(WARN, "fail to finish insert", KR(tmp_ret), KR(ret), + K(level_origin_page_write_counts), K(level_new_pages), KPC(this)); + } + if (OB_SUCC(ret)) { + ret = tmp_ret; + } + if (OB_SUCC(ret)) { + is_writing_ = false; + } + } + } + if (OB_SUCC(ret)) { + ARRAY_FOREACH_N(data_items, i, cnt) { + stat_info_.all_type_page_flush_cnt_ += data_items.at(i).physical_page_num_; + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::try_to_insert_items_to_array_( + const ObIArray &data_items, + ObIArray> &level_new_pages) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(data_items.empty() + || !level_page_range_array_.empty() + || MAX_DATA_ITEM_ARRAY_COUNT > MAX_PAGE_ITEM_COUNT + || is_writing_)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(data_items), K(level_page_range_array_), + K(MAX_DATA_ITEM_ARRAY_COUNT), K(MAX_PAGE_ITEM_COUNT), K(is_writing_)); + } else if (!data_item_array_.empty()) { + const ObSharedNothingTmpFileDataItem &last_item = data_item_array_.at(data_item_array_.count() - 1); + if (OB_UNLIKELY(data_items.at(0).virtual_page_id_ != last_item.virtual_page_id_ + last_item.physical_page_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data_items or data_item_array_", KR(ret), K(fd_), K(data_items), K(last_item)); + } + } + if (OB_SUCC(ret)) { + bool need_build_tree = false; + int16_t push_cnt = 0; + if (data_item_array_.count() + data_items.count() <= MAX_DATA_ITEM_ARRAY_COUNT) { + ARRAY_FOREACH_N(data_items, i, cnt) { + if (OB_UNLIKELY(!data_items.at(i).is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(data_items.at(i))); + } else if (OB_FAIL(data_item_array_.push_back(data_items.at(i)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(data_items.at(i))); + if (OB_ALLOCATE_MEMORY_FAILED == ret) { + need_build_tree = true; + } + } else { + push_cnt++; + } + } + if (OB_FAIL(ret)) { + for (int16_t i = 0; i < push_cnt; i++) { + data_item_array_.pop_back(); + } + } + } else { + need_build_tree = true; + } + if (true == need_build_tree) { + ret = OB_SUCCESS; + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buff = NULL; + int16_t count = 0; + ObTmpFilePageUniqKey leaf_page_offset(0/*tree level*/, 0/*level page index*/); + if (OB_FAIL(wbp_->alloc_page(fd_, leaf_page_offset, new_page_id, new_page_buff))) { + STORAGE_LOG(WARN, "fail to alloc page from write cache", KR(ret), K(fd_), K(leaf_page_offset)); + } else if (OB_FAIL(init_page_header_(new_page_buff, 0 /*level*/))) { + STORAGE_LOG(WARN, "fail to init page header", KR(ret), K(fd_), KP(new_page_buff)); + } else if (!data_item_array_.empty() && + OB_FAIL(write_items_(new_page_buff, data_item_array_, 0/*begin_index*/, count))) { + STORAGE_LOG(WARN, "fail to write items", KR(ret), K(fd_), K(data_item_array_), KP(new_page_buff)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, leaf_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(new_page_id), K(leaf_page_offset)); + } else { + is_writing_ = true; //must be protected by lock + ObSEArray new_pages; + if (OB_FAIL(new_pages.push_back(new_page_id))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_page_id)); + } else if (OB_FAIL(level_new_pages.push_back(new_pages))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_pages)); + } else { + root_item_.page_level_ = 0; + root_item_.buffer_page_id_ = new_page_id; + root_item_.virtual_page_id_ = data_item_array_.empty() ? + data_items.at(0).virtual_page_id_ : data_item_array_.at(0).virtual_page_id_; + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileDataItem first_item; + ObSharedNothingTmpFileDataItem last_item; + read_page_simple_content_(new_page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(INFO, "dump tree page", KR(ret), K(fd_), K(new_page_id), K(leaf_page_offset), + KP(new_page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::try_to_fill_rightmost_leaf_page_( + const ObIArray &data_items, + const ObIArray> &level_new_pages, + int64_t &write_count, + ObIArray &level_origin_page_write_counts) +{ + int ret = OB_SUCCESS; + write_count = 0; + if (OB_UNLIKELY(data_items.empty() || !data_items.at(0).is_valid() + || (level_new_pages.empty() && level_page_range_array_.empty()))) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(data_items), K(level_new_pages), K(level_page_range_array_)); + } else { + const int16_t MAX_PAGE_DATA_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(data_items.at(0))); + uint32_t page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = -1; + int16_t count = 0; + ObSharedNothingTmpFileTreePageHeader page_header; + char *leaf_page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (!level_new_pages.empty()) { + page_id = level_new_pages.at(0/*level*/).at(0); + level_page_index = 0; + } else { + page_id = level_page_range_array_.at(0/*level*/).end_page_id_; + level_page_index = level_page_range_array_.at(0).evicted_page_num_ + level_page_range_array_.at(0).cached_page_num_ - 1; + } + ObTmpFilePageUniqKey leaf_page_offset(0, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, page_id, leaf_page_offset, leaf_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_id), K(leaf_page_offset), + K(level_new_pages), K(level_page_range_array_)); + } else if (OB_FAIL(read_page_header_(leaf_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read header", KR(ret), K(fd_), KP(leaf_page_buff)); + } else if (0 == page_header.item_num_) { + //maybe the item have been cleared (corresponds to an unfilled data page) + //or maybe this is a newly allocated meta page + //so "0 == item_num" + //but we do not need to rewrite page info (change virtual_page_id) + STORAGE_LOG(INFO, "item_num is 0", KR(ret), K(fd_), KP(leaf_page_buff), K(page_id)); + } else { + //check last data item + ObSharedNothingTmpFileDataItem origin_last_item; + if (OB_FAIL(read_item_(leaf_page_buff, page_header.item_num_ - 1, origin_last_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(leaf_page_buff), K(page_header)); + } else if (OB_UNLIKELY(data_items.at(0).virtual_page_id_ != origin_last_item.virtual_page_id_ + origin_last_item.physical_page_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data_items or origin_last_item", KR(ret), K(fd_), K(data_items), K(origin_last_item)); + } + } + if (OB_SUCC(ret) && page_header.item_num_ < MAX_PAGE_DATA_ITEM_NUM) { + if (OB_FAIL(write_items_(leaf_page_buff, data_items, 0, count))) { + STORAGE_LOG(WARN, "fail to write items", KR(ret), K(fd_), KP(leaf_page_buff), K(data_items)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, page_id, leaf_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(page_id), K(leaf_page_offset)); + } else { + write_count += count; + } + if (OB_SUCC(ret)) { + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileDataItem first_item; + ObSharedNothingTmpFileDataItem last_item; + read_page_simple_content_(leaf_page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(DEBUG, "dump tree page", KR(ret), K(fd_), K(page_id), K(leaf_page_offset), + KP(leaf_page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + } + if (OB_SUCC(ret) && !level_page_range_array_.empty() + && OB_FAIL(level_origin_page_write_counts.push_back(count))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(count)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::add_new_page_and_fill_items_at_leaf_( + const ObIArray &data_items, + int64_t &write_count, + ObSharedNothingTmpFileMetaItem &meta_item, + ObIArray> &level_new_pages) +{ + int ret = OB_SUCCESS; + meta_item.reset(); + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buff = NULL; + int16_t count = 0; + int32_t level_page_index = 0; + if (!level_page_range_array_.empty()) { + level_page_index += (level_page_range_array_.at(0).evicted_page_num_ + + level_page_range_array_.at(0).cached_page_num_); + } + if (!level_new_pages.empty()) { + level_page_index += level_new_pages.at(0).count(); + } + ObTmpFilePageUniqKey leaf_page_offset(0, level_page_index); + if (OB_UNLIKELY(!level_new_pages.empty() && 1 != level_new_pages.count())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(level_new_pages)); + } else if (OB_UNLIKELY(data_items.count() <= write_count || !data_items.at(write_count).is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(data_items), K(write_count)); + } else if (OB_FAIL(wbp_->alloc_page(fd_, leaf_page_offset, new_page_id, new_page_buff))) { + STORAGE_LOG(WARN, "fail to alloc page from write cache", KR(ret), K(fd_), K(leaf_page_offset), + K(level_new_pages), K(level_page_range_array_)); + } else if (OB_FAIL(init_page_header_(new_page_buff, 0 /*level*/))) { + STORAGE_LOG(WARN, "fail to init page header", KR(ret), K(fd_), KP(new_page_buff)); + } else if (OB_FAIL(write_items_(new_page_buff, data_items, write_count/*begin_index*/, count))) { + STORAGE_LOG(WARN, "fail to write items", KR(ret), K(fd_), KP(new_page_buff), K(data_items), K(write_count)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, leaf_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(new_page_id), K(leaf_page_offset)); + } else { + meta_item.page_level_ = 0; + meta_item.buffer_page_id_ = new_page_id; + meta_item.virtual_page_id_ = data_items.at(write_count).virtual_page_id_; + write_count += count; + if (level_new_pages.empty()) { + ObSEArray new_pages; + if (OB_FAIL(level_new_pages.push_back(new_pages))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_pages)); + } + } + if (FAILEDx(level_new_pages.at(0).push_back(new_page_id))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_page_id)); + } + } + if (OB_SUCC(ret)) { + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileDataItem first_item; + ObSharedNothingTmpFileDataItem last_item; + read_page_simple_content_(new_page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(INFO, "dump tree page", KR(ret), K(fd_), K(new_page_id), K(leaf_page_offset), + KP(new_page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::cascade_modification_at_internal_( + const ObIArray &new_leaf_page_infos, + ObIArray &level_origin_page_write_counts, + ObIArray> &level_new_pages) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(new_leaf_page_infos.empty() + || level_new_pages.empty())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(new_leaf_page_infos), K(level_new_pages)); + } else { + ObSEArray prev_meta_items; + ObSEArray meta_items; + ObSharedNothingTmpFileMetaItem meta_item; + const int16_t origin_level_count = level_page_range_array_.count(); + int16_t cur_level = 1; + //TODO: time limitation in while loop + while (OB_SUCC(ret)) { + int64_t write_count = 0; + prev_meta_items.reset(); + if (cur_level > origin_level_count && meta_items.count() == 1) { + //we need to change root_item_ + break; + } else if (cur_level == 1 && OB_FAIL(prev_meta_items.assign(new_leaf_page_infos))) { + STORAGE_LOG(WARN, "fail to assign", KR(ret), K(fd_), K(new_leaf_page_infos)); + } else if (cur_level > 1 && OB_FAIL(prev_meta_items.assign(meta_items))) { + STORAGE_LOG(WARN, "fail to assign", KR(ret), K(fd_), K(meta_items)); + } else if (cur_level < origin_level_count && + OB_FAIL(try_to_fill_rightmost_internal_page_(prev_meta_items, cur_level, write_count, + level_origin_page_write_counts))) { + STORAGE_LOG(WARN, "fail to try to fill rightmost internal page", KR(ret), K(fd_), K(prev_meta_items), + K(cur_level), K(level_origin_page_write_counts)); + } else { + int64_t meta_item_count = prev_meta_items.count(); + if (write_count < meta_item_count) { + meta_items.reset(); + if (origin_level_count - 1 == cur_level) { + if (OB_FAIL(meta_items.push_back(root_item_))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(root_item_)); + } + } + while (OB_SUCC(ret) && write_count < meta_item_count) { + meta_item.reset(); + if (OB_FAIL(add_new_page_and_fill_items_at_internal_(prev_meta_items, cur_level, write_count, + meta_item, level_new_pages))) { + STORAGE_LOG(WARN, "fail to add new page and fill items at internal", KR(ret), K(fd_), + K(prev_meta_items), K(cur_level), K(write_count)); + } else if (OB_FAIL(meta_items.push_back(meta_item))) { + STORAGE_LOG(WARN, "fail to push_back", KR(ret), K(fd_), K(meta_item)); + } + } + } else { + //cur level does not alloc new pages + break; + } + } + cur_level++; + } + if (OB_SUCC(ret)) { + if (OB_UNLIKELY(origin_level_count == level_new_pages.count())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_new_pages", KR(ret), K(fd_), K(level_new_pages), K(origin_level_count)); + } else if (origin_level_count < level_new_pages.count()) { + if (OB_UNLIKELY(1 != meta_items.count() + || 1 != level_new_pages.at(level_new_pages.count() - 1).count())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_items or level_new_pages", KR(ret), K(fd_), K(meta_items), + K(level_new_pages), K(origin_level_count)); + } else { + root_item_ = meta_items.at(0); + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::try_to_fill_rightmost_internal_page_( + const ObIArray &meta_items, + const int16_t page_level, + int64_t &write_count, + ObIArray &level_origin_page_write_counts) +{ + int ret = OB_SUCCESS; + write_count = 0; + //only a writing thread, array count will not change + if (OB_UNLIKELY(meta_items.empty() + || page_level < 1 + || page_level >= level_page_range_array_.count())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(meta_items), K(page_level), K(level_page_range_array_)); + } else { + const int16_t MAX_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(meta_items.at(0))); + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *internal_page_buff = NULL; + ObSharedNothingTmpFileTreePageHeader page_header; + int16_t count = 0; + uint32_t page_id = level_page_range_array_[page_level].end_page_id_; + int32_t level_page_index = level_page_range_array_[page_level].evicted_page_num_ + + level_page_range_array_[page_level].cached_page_num_ - 1; + ObTmpFilePageUniqKey internal_page_offset(page_level, level_page_index); + //only a writing thread. + if (OB_FAIL(wbp_->read_page(fd_, page_id, internal_page_offset, internal_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_id), K(internal_page_offset)); + } else if (OB_FAIL(read_page_header_(internal_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(internal_page_buff)); + } else if (page_header.item_num_ < MAX_PAGE_META_ITEM_NUM) { + if (page_header.item_num_ <= 0) { + //the rightmost page in internal level must has items + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(fd_), K(page_header)); + } else if (OB_FAIL(write_items_(internal_page_buff, meta_items, 0, count))) { + STORAGE_LOG(WARN, "fail to write items", KR(ret), K(fd_), KP(internal_page_buff), K(meta_items)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, page_id, internal_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(page_id), K(internal_page_offset)); + } else { + write_count += count; + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileMetaItem first_item; + ObSharedNothingTmpFileMetaItem last_item; + read_page_simple_content_(internal_page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(INFO, "dump tree page", KR(ret), K(fd_), K(page_id), K(internal_page_offset), + KP(internal_page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + } + if (FAILEDx(level_origin_page_write_counts.push_back(count))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(count)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::add_new_page_and_fill_items_at_internal_( + const ObIArray &meta_items, + const int16_t page_level, + int64_t &write_count, + ObSharedNothingTmpFileMetaItem &meta_item, + ObIArray> &level_new_pages) +{ + int ret = OB_SUCCESS; + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buff = NULL; + int16_t count = 0; + int32_t level_page_index = 0; + if (page_level < level_page_range_array_.count()) { + level_page_index += (level_page_range_array_.at(page_level).evicted_page_num_ + + level_page_range_array_.at(page_level).cached_page_num_); + } + if (page_level < level_new_pages.count()) { + level_page_index += level_new_pages.at(page_level).count(); + } + ObTmpFilePageUniqKey internal_page_offset(page_level, level_page_index); + if (OB_UNLIKELY(meta_items.count() <= write_count || !meta_items.at(write_count).is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(meta_items), K(write_count)); + } else if (OB_FAIL(wbp_->alloc_page(fd_, internal_page_offset, new_page_id, new_page_buff))) { + STORAGE_LOG(WARN, "fail to alloc page from write cache", KR(ret), K(fd_), K(internal_page_offset)); + } else if (OB_FAIL(init_page_header_(new_page_buff, page_level))) { + STORAGE_LOG(WARN, "fail to init page header", KR(ret), K(fd_), KP(new_page_buff)); + } else if (OB_FAIL(write_items_(new_page_buff, meta_items, write_count/*begin_index*/, count))) { + STORAGE_LOG(WARN, "fail to write items", KR(ret), K(fd_), KP(new_page_buff), K(meta_items), K(write_count)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, internal_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(new_page_id), K(internal_page_offset)); + } else { + meta_item.page_level_ = page_level; + meta_item.buffer_page_id_ = new_page_id; + meta_item.virtual_page_id_ = meta_items.at(write_count).virtual_page_id_; + write_count += count; + if (OB_UNLIKELY(level_new_pages.count() != page_level + && level_new_pages.count() != page_level + 1)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_new_pages", KR(ret), K(fd_), K(page_level), K(level_new_pages)); + } else if (level_new_pages.count() == page_level) { + ObSEArray new_pages; + if (OB_FAIL(level_new_pages.push_back(new_pages))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_pages)); + } + } + if (FAILEDx(level_new_pages.at(page_level).push_back(new_page_id))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_page_id)); + } + } + if (OB_SUCC(ret)) { + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileMetaItem first_item; + ObSharedNothingTmpFileMetaItem last_item; + read_page_simple_content_(new_page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(INFO, "dump tree page", KR(ret), K(fd_), K(new_page_id), K(internal_page_offset), + KP(new_page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::finish_insert_( + const int return_ret, + ObIArray &level_origin_page_write_counts, + ObIArray> &level_new_pages) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(level_origin_page_write_counts.count() > level_page_range_array_.count())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected new record level page infos", KR(ret), K(fd_), + K(level_origin_page_write_counts), K(level_page_range_array_)); + } else if (OB_SUCCESS == return_ret) { + ARRAY_FOREACH_N(level_new_pages, level, level_cnt) { + ARRAY_FOREACH_N(level_new_pages.at(level), i, new_page_cnt) { + const uint32_t new_page_id = level_new_pages.at(level).at(i); + if (0 == i && level_page_range_array_.count() <= level) { + if (OB_FAIL(level_page_range_array_.push_back( + LevelPageRangeInfo(new_page_id, ObTmpFileGlobal::INVALID_PAGE_ID, new_page_id, 1, 0, 0)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(new_page_id)); + } + } else if (OB_UNLIKELY(level_page_range_array_.count() <= level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), K(fd_), K(level_page_range_array_), K(level)); + } else { + uint32_t end_page_id_in_array = level_page_range_array_[level].end_page_id_; + int32_t level_prev_page_index = level_page_range_array_[level].evicted_page_num_ + + level_page_range_array_[level].cached_page_num_ - 1; + if (ObTmpFileGlobal::INVALID_PAGE_ID != end_page_id_in_array + && OB_FAIL(wbp_->link_page(fd_, new_page_id, end_page_id_in_array, + ObTmpFilePageUniqKey(level, level_prev_page_index)))) { + STORAGE_LOG(WARN, "fail to link page in write cache", KR(ret), K(fd_), K(level_page_range_array_), + K(new_page_id), K(level)); + } else { + if (ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id_in_array) { + level_page_range_array_[level].start_page_id_ = new_page_id; + } + level_page_range_array_[level].end_page_id_ = new_page_id; + level_page_range_array_[level].cached_page_num_++; + } + } + } + } + if (OB_SUCC(ret) && !data_item_array_.empty()) { + data_item_array_.reset(); + } + } else { //fail + //rollback + ARRAY_FOREACH_N(level_origin_page_write_counts, i, cnt) { + const int16_t remove_cnt = level_origin_page_write_counts.at(i); + if (0 != remove_cnt) { + char *page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t end_page_id = level_page_range_array_[i].end_page_id_; + int32_t level_page_index = level_page_range_array_[i].evicted_page_num_ + + level_page_range_array_[i].cached_page_num_ - 1; + ObTmpFilePageUniqKey page_key(i, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, end_page_id, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(end_page_id), K(page_key)); + } else if (OB_FAIL(remove_page_item_from_tail_(page_buff, remove_cnt))) { + STORAGE_LOG(WARN, "fail to remove page item from tail", KR(ret), K(fd_), KP(page_buff), K(remove_cnt)); + } + } + } + ARRAY_FOREACH_N(level_new_pages, level, level_cnt) { + int32_t level_page_index = 0; + if (level_page_range_array_.count() > level) { + level_page_index += (level_page_range_array_[level].evicted_page_num_ + + level_page_range_array_[level].cached_page_num_); + } + ARRAY_FOREACH_N(level_new_pages.at(level), i, new_page_cnt) { + uint32_t unused_next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + const uint32_t new_page_id = level_new_pages.at(level).at(i); + ObTmpFilePageUniqKey page_key(level, level_page_index + i); + if (OB_FAIL(wbp_->free_page(fd_, new_page_id, page_key, unused_next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page in write cache", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } + } + } + if (OB_SUCC(ret)) { + if (level_page_range_array_.empty()) { + root_item_.reset(); + } + is_writing_ = false; + } + STORAGE_LOG(INFO, "fail to insert, finish rollback to before", KR(ret), K(return_ret), KPC(this)); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::search_data_items( + const int64_t start_offset, + const int64_t read_size, + ObIArray &data_items) +{ + int ret = OB_SUCCESS; + data_items.reset(); + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + const int64_t end_offset = start_offset + read_size; + int64_t offset = start_offset; + SpinRLockGuard guard(lock_); + if (OB_UNLIKELY(start_offset < released_offset_)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(start_offset), KPC(this)); + } else if (!root_item_.is_valid()) { + //read from data_item_array_ + if (OB_FAIL(search_data_items_from_array_(end_offset, offset, data_items))) { + STORAGE_LOG(WARN, "fail to get data items from array", KR(ret), K(end_offset), K(offset), KPC(this)); + } + } else { + const int64_t target_virtual_page_id = start_offset / ObTmpFileGlobal::PAGE_SIZE; + ObSharedNothingTmpFileMetaItem meta_item = root_item_; + ObSharedNothingTmpFileMetaItem next_level_meta_item; + //read from meta tree + //root page index must be 0 + int32_t level_page_index = 0; + //we use a array to simulate a stack + //TODO: 初始值的设置 + ObSEArray search_path; + while (OB_SUCC(ret) + && 0 < meta_item.page_level_) { + next_level_meta_item.reset(); + int16_t item_index = -1; + ObSharedNothingTmpFileTreePageHeader page_header; + char *page_buff = NULL; + ObTmpPageValueHandle p_handle; + if (OB_FAIL(get_page_(meta_item, level_page_index, page_buff, p_handle))) { + STORAGE_LOG(WARN, "fail to get page", KR(ret), K(meta_item), K(level_page_index), KPC(this)); + } else if (OB_FAIL(read_item_(page_buff, target_virtual_page_id, item_index, next_level_meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), KP(page_buff), + K(target_virtual_page_id), K(meta_item), K(level_page_index), KPC(this)); + } else if (OB_FAIL(search_path.push_back(BacktraceNode(meta_item /*page info*/, + level_page_index, /*page index in level*/ + item_index /*item index on the page*/)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_item), K(level_page_index), K(item_index)); + } else { + meta_item = next_level_meta_item; + level_page_index = level_page_index * FULL_PAGE_META_ITEM_NUM + item_index; + } + p_handle.reset(); + } + if (OB_SUCC(ret)) { + if (0 == meta_item.page_level_) { + if (OB_FAIL(get_items_of_leaf_page_(meta_item, level_page_index, end_offset, + true /*need read from specified item index*/, offset, data_items))) { + STORAGE_LOG(WARN, "fail to get items of leaf page", KR(ret), K(meta_item), + K(level_page_index), K(end_offset), K(offset), KPC(this)); + } else if (offset < end_offset + && OB_FAIL(backtrace_search_data_items_(end_offset, offset, search_path, data_items))) { + STORAGE_LOG(WARN, "fail to backtrace search data items", KR(ret), + K(end_offset), K(offset), K(search_path), KPC(this)); + } + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page type", KR(ret), K(meta_item), KPC(this)); + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::search_data_items_from_array_( + const int64_t end_offset, + int64_t &cur_offset, + common::ObIArray &data_items) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(cur_offset < 0 + || cur_offset >= end_offset + || data_item_array_.empty() + || data_item_array_.count() > MAX_DATA_ITEM_ARRAY_COUNT)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(cur_offset), K(end_offset), K(data_item_array_)); + } else { + const int64_t target_virtual_page_id = cur_offset / ObTmpFileGlobal::PAGE_SIZE; + int16_t index = -1; + ARRAY_FOREACH_N(data_item_array_, i, cnt) { + const ObSharedNothingTmpFileDataItem &data_item = data_item_array_.at(i); + if (OB_UNLIKELY(!data_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data_item", KR(ret), K(fd_), K(i), K(data_item)); + } else if (data_item.virtual_page_id_ <= target_virtual_page_id) { + index = i; + } else { + break; + } + } + if (OB_UNLIKELY(0 > index)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected index", KR(ret), K(fd_), K(index), K(cur_offset), K(data_item_array_)); + } else { + for (int16_t i = index; OB_SUCC(ret) && i < data_item_array_.count() && cur_offset < end_offset; i++) { + const ObSharedNothingTmpFileDataItem &data_item = data_item_array_.at(i); + if (OB_UNLIKELY(i > index && cur_offset != data_item.virtual_page_id_ * ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected virtual_page_id", KR(ret), K(fd_), K(cur_offset), K(i), K(index), K(data_item)); + } else if (OB_FAIL(data_items.push_back(data_item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(data_item)); + } else { + cur_offset = (data_item.virtual_page_id_ + data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + } + } + } + if (OB_SUCC(ret) && cur_offset < end_offset) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected offset", KR(ret), K(fd_), K(cur_offset), K(end_offset)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::get_items_of_leaf_page_( + const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int64_t end_offset, + const bool need_find_index, + int64_t &cur_offset, + common::ObIArray &data_items) +{ + int ret = OB_SUCCESS; + char *page_buff = NULL; + int16_t item_index = -1; + int64_t tmp_offset = -1; + const int64_t target_virtual_page_id = cur_offset / ObTmpFileGlobal::PAGE_SIZE; + ObSharedNothingTmpFileDataItem data_item; + ObSharedNothingTmpFileTreePageHeader page_header; + ObTmpPageValueHandle p_handle; + if (OB_FAIL(get_page_(page_info, level_page_index, page_buff, p_handle))) { + STORAGE_LOG(WARN, "fail to get page", KR(ret), K(fd_), K(page_info), K(level_page_index)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else { + if (!need_find_index) { + if (OB_UNLIKELY(0 != cur_offset % ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected cur_offset", KR(ret), K(fd_), K(cur_offset)); + } else { + //read from beginning of the page + item_index = 0; + } + } else { + //get the specified item_index based on target_virtual_page_id + if (OB_FAIL(read_item_(page_buff, target_virtual_page_id, item_index, data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(target_virtual_page_id)); + } else if (FALSE_IT(tmp_offset = (data_item.virtual_page_id_ + data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE)) { + } else if (cur_offset >= tmp_offset) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected offset", KR(ret), K(fd_), K(cur_offset), K(tmp_offset), K(data_item)); + } else if (OB_FAIL(data_items.push_back(data_item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(data_item)); + } else { + cur_offset = tmp_offset; + item_index++; + } + } + if (OB_SUCC(ret)) { + while (OB_SUCC(ret) + && cur_offset < end_offset + && item_index < page_header.item_num_) { + data_item.reset(); + if (OB_FAIL(read_item_(page_buff, item_index, data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index), K(page_info), K(level_page_index)); + } else if (OB_UNLIKELY(cur_offset != data_item.virtual_page_id_ * ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + //print page content + //NOTE: control print frequence + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObArray items; + read_page_content_(page_buff, tmp_page_header, items); + STORAGE_LOG(WARN, "unexpected virtual_page_id, dump tree page", KR(ret), K(fd_), KP(page_buff), K(cur_offset), + K(item_index), K(data_item), K(target_virtual_page_id), K(tmp_page_header), K(items)); + } else if (OB_FAIL(data_items.push_back(data_item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(data_item)); + } else { + cur_offset = (data_item.virtual_page_id_ + data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + item_index++; + } + } + } + if (OB_FAIL(ret)) { + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObSharedNothingTmpFileDataItem first_item; + ObSharedNothingTmpFileDataItem last_item; + read_page_simple_content_(page_buff, page_header, first_item, last_item); + STORAGE_LOG(WARN, "dump tree page", KR(ret), K(fd_), K(page_info), K(level_page_index), + KP(page_buff), K(tmp_page_header), K(first_item), K(last_item)); + } + } + p_handle.reset(); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::backtrace_search_data_items_( + const int64_t end_offset, + int64_t &offset, + ObIArray &search_path, + ObIArray &data_items) +{ + int ret = OB_SUCCESS; + //TODO: check last node in search_path must be level_1 page info + if (OB_UNLIKELY(search_path.empty() + || data_items.empty() + || offset >= end_offset)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(search_path), K(data_items), + K(end_offset), K(offset)); + } else { + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + ObSEArray meta_items; + while (OB_SUCC(ret) + && !search_path.empty() + && offset < end_offset) { + meta_items.reset(); + bool reach_last_item = false; + int64_t last_node_index = search_path.count() - 1; + //meta_item points to a page + const ObSharedNothingTmpFileMetaItem &page_info = search_path.at(last_node_index).page_meta_info_; + //the page index in its level + const int32_t level_page_index = search_path.at(last_node_index).level_page_index_; + //current pos need to be processed on this page. + int16_t cur_item_index = search_path.at(last_node_index).prev_item_index_ + 1; + if (OB_FAIL(get_items_of_internal_page_(page_info, level_page_index, cur_item_index, + end_offset, reach_last_item, meta_items))) { + STORAGE_LOG(WARN, "fail to get items of internal page", KR(ret), K(fd_), K(page_info), + K(level_page_index), K(cur_item_index), K(end_offset)); + } else if (1 == page_info.page_level_) { + //process level_0 + int64_t i = 0; + const int64_t cnt = meta_items.count(); + const int32_t start_leaf_level_page_index = level_page_index * FULL_PAGE_META_ITEM_NUM + cur_item_index; + for (; OB_SUCC(ret) && i < cnt && offset < end_offset; i++) { + const ObSharedNothingTmpFileMetaItem &leaf_page_info = meta_items.at(i); + bool need_lock = ((i + 1 == cnt) && reach_last_item); + if (OB_FAIL(get_items_of_leaf_page_(leaf_page_info, start_leaf_level_page_index + i, end_offset, + false /*need read from specified item index*/, offset, data_items))) { + STORAGE_LOG(WARN, "fail to get items of leaf page", KR(ret), K(fd_), K(leaf_page_info), + K(start_leaf_level_page_index + i), K(end_offset), K(need_lock), K(offset)); + } + } + if (OB_SUCC(ret)) { + if (OB_UNLIKELY(cnt != i || (!reach_last_item && offset < end_offset))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_items", KR(ret), K(fd_), K(meta_items), K(i), + K(reach_last_item), K(offset), K(end_offset)); + } else { + search_path.pop_back(); + } + } + } else if (page_info.page_level_ > 1) { + if (meta_items.count() == 1) { + const int32_t child_level_page_index = level_page_index * FULL_PAGE_META_ITEM_NUM + cur_item_index; + search_path.at(last_node_index).prev_item_index_ = cur_item_index; + if (OB_FAIL(search_path.push_back(BacktraceNode(meta_items.at(0), child_level_page_index, -1)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_items.at(0)), K(child_level_page_index)); + } + } else if (meta_items.empty()) { + search_path.pop_back(); + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_items", KR(ret), K(fd_), K(meta_items)); + } + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page info", KR(ret), K(fd_), K(page_info)); + } + } + if (OB_SUCC(ret) && end_offset > offset) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected offset", KR(ret), K(fd_), K(end_offset), K(offset)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::get_items_of_internal_page_( + const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int16_t item_index, + const int64_t end_offset, + bool &reach_last_item, + ObIArray &meta_items) +{ + int ret = OB_SUCCESS; + reach_last_item = false; + meta_items.reset(); + int16_t cur_item_index = item_index; + ObSharedNothingTmpFileTreePageHeader page_header; + ObSharedNothingTmpFileMetaItem meta_item; + char *page_buff = NULL; + ObTmpPageValueHandle p_handle; + if (OB_FAIL(get_page_(page_info, level_page_index, page_buff, p_handle))) { + STORAGE_LOG(WARN, "fail to get page", KR(ret), K(fd_), K(page_info), K(level_page_index)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (OB_UNLIKELY(0 >= page_header.item_num_ + || item_index > page_header.item_num_ + || item_index < 0)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_header", KR(ret), K(fd_), K(page_header), K(item_index)); + } else { + int64_t end_virtual_page_id = upper_align(end_offset, ObTmpFileGlobal::PAGE_SIZE) / ObTmpFileGlobal::PAGE_SIZE; + while (OB_SUCC(ret) + && cur_item_index < page_header.item_num_) { + meta_item.reset(); + if (OB_FAIL(read_item_(page_buff, cur_item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(cur_item_index)); + } else if (meta_item.virtual_page_id_ >= end_virtual_page_id) { + break; + } else if (OB_FAIL(meta_items.push_back(meta_item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_item)); + } else if (FALSE_IT(cur_item_index++)) { + } else if (1 != page_info.page_level_) { + break; + } + } + } + p_handle.reset(); + if (OB_SUCC(ret) && cur_item_index == page_header.item_num_) { + reach_last_item = true; + } + return ret; +} + +//TODO: how to check whether block_index is valid +int ObSharedNothingTmpFileMetaTree::flush_meta_pages_for_block( + const int64_t block_index, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeFlushContext &flush_context, + ObIArray &tree_io_array) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX == block_index + || ObTmpFileTreeEvictType::INVALID == flush_type + || NULL == block_buff + || 0 != write_offset % ObTmpFileGlobal::PAGE_SIZE + || level_page_range_array_.count() >= INT16_MAX)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(block_index), K(flush_type), + KP(block_buff), K(write_offset), KPC(this)); + } else if (!root_item_.is_valid()) { + if (!level_page_range_array_.empty()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), KPC(this)); + } else { + flush_context.is_meta_reach_end_ = true; + } + } else { + int16_t level = 0; + bool need_use_context = false; + bool need_flush = true; + const int16_t max_level = level_page_range_array_.count() - 1; + ObTmpFileTreeIOInfo tree_io_info; + if (flush_context.is_valid()) { + if (flush_context.tree_epoch_ != tree_epoch_) { + STORAGE_LOG(INFO, "the tree_epoch_ in flush context is not equal to current tree_epoch_", + K(fd_), K(flush_context), K(tree_epoch_)); + need_flush = false; + flush_context.is_meta_reach_end_ = true; + } else { + //flush context means: in this round of flushing, + // the meta tree has already flushed some pages in previous blocks, + // so we need "flush context" to avoid repeated flushing. + level = flush_context.last_flush_level_; + need_use_context = true; + if (last_truncate_leaf_info_.is_valid()) { + if (OB_UNLIKELY(max_level < level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_index_arr", KR(ret), K(level), KPC(this)); + } else if (flush_context.last_flush_page_index_in_level_ < level_page_range_array_.at(level).evicted_page_num_) { + need_flush = false; + flush_context.is_meta_reach_end_ = true; + } + } + } + } + if (OB_SUCC(ret) && need_flush) { + while (OB_SUCC(ret) + && level <= max_level + && write_offset < OB_DEFAULT_MACRO_BLOCK_SIZE) + { + uint32_t flush_start_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = -1; + if (need_use_context) { + level_page_index = flush_context.last_flush_page_index_in_level_; + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (wbp_->is_dirty(fd_, flush_context.last_flush_page_id_, page_key)) { + //do nothing, we just skip this level. + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, flush_context.last_flush_page_id_, + page_key, flush_start_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(flush_context), K(page_key)); + } else { + level_page_index++; + } + //only used the first time + need_use_context = false; + } else { + uint32_t flushed_page_id = level_page_range_array_[level].flushed_end_page_id_; + level_page_index = level_page_range_array_[level].flushed_page_num_; + if (ObTmpFileGlobal::INVALID_PAGE_ID == flushed_page_id) { + flush_start_page_id = level_page_range_array_[level].start_page_id_; + } else { + level_page_index--; + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (wbp_->is_dirty(fd_, flushed_page_id, page_key)) { + //flushed page -> write cache + //we only write rightmost page, so pages before flushed_end_page_id_ can not be dirty. + flush_start_page_id = flushed_page_id; + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, flushed_page_id, page_key, flush_start_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(flushed_page_id), K(page_key)); + } else { + level_page_index++; + } + } + } + if (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID != flush_start_page_id) { + tree_io_info.reset(); + tree_io_info.tree_epoch_ = tree_epoch_; + tree_io_info.block_index_ = block_index; + if (0 == level && OB_FAIL(flush_leaf_pages_(flush_start_page_id, level_page_index, flush_type, + block_buff, write_offset, tree_io_info))) { + STORAGE_LOG(WARN, "fail to flush leaf pages", KR(ret), K(flush_start_page_id), + K(level_page_index), K(flush_type), KP(block_buff), K(write_offset), KPC(this)); + } else if (0 < level && OB_FAIL(flush_internal_pages_(flush_start_page_id, level, level_page_index, + flush_type, block_buff, write_offset, tree_io_info))) { + STORAGE_LOG(WARN, "fail to flush internal pages", KR(ret), K(flush_start_page_id), + K(level), K(level_page_index), K(flush_type), KP(block_buff), K(write_offset), KPC(this)); + } else if (0 == tree_io_info.flush_nums_) { + //do nothing + STORAGE_LOG(INFO, "no meta page flush in this level", KR(ret), K(fd_), K(level), K(level_page_range_array_)); + } else if (OB_UNLIKELY(!tree_io_info.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected tree_io_info", KR(ret), K(tree_io_info), KPC(this)); + //set the page flush information on the parent node, so that we can flush parent pages in advance. + } else if (OB_FAIL(modify_meta_items_at_parent_level_(tree_io_info, level_page_index))) { + STORAGE_LOG(WARN, "fail to modify meta items at parent level", KR(ret), + K(tree_io_info), K(level_page_index), KPC(this)); + } else if (OB_FAIL(tree_io_array.push_back(tree_io_info))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(tree_io_info)); + } else { + flush_context.tree_epoch_ = tree_epoch_; + flush_context.last_flush_level_ = level; + flush_context.last_flush_page_id_ = tree_io_info.flush_end_page_id_; + flush_context.last_flush_page_index_in_level_ = level_page_index + tree_io_info.flush_nums_ - 1; + stat_info_.meta_page_flushing_cnt_ += tree_io_info.flush_nums_; + stat_info_.all_type_page_flush_cnt_ += tree_io_info.flush_nums_; + } + } + if (OB_SUCC(ret) && max_level == level) { + flush_context.is_meta_reach_end_ = true; + } + level++; + } + } + } + STORAGE_LOG(INFO, "finish flush meta pages for block", KR(ret), K(fd_), K(tree_io_array)); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::flush_leaf_pages_( + const uint32_t flush_start_page_id, + const int32_t start_page_index_in_level, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeIOInfo &tree_io_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == flush_start_page_id + || 0 > start_page_index_in_level + || 0 != write_offset % ObTmpFileGlobal::PAGE_SIZE + || level_page_range_array_.empty())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(flush_start_page_id), K(start_page_index_in_level), + K(write_offset), K(level_page_range_array_)); + } else { + //TODO: start page id in block is from 0 + tree_io_info.physical_start_page_id_ = write_offset / ObTmpFileGlobal::PAGE_SIZE; + tree_io_info.page_level_ = 0; + uint32_t cur_page_id = flush_start_page_id; + const uint32_t end_page_id = level_page_range_array_[0].end_page_id_; + int32_t page_index_in_level = start_page_index_in_level; + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id + && write_offset + ObTmpFileGlobal::PAGE_SIZE <= OB_DEFAULT_MACRO_BLOCK_SIZE + && (ObTmpFileTreeEvictType::FULL == flush_type || end_page_id != cur_page_id)) { + char *page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(0, page_index_in_level); + if (OB_UNLIKELY(!wbp_->is_dirty(fd_, cur_page_id, page_key))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page state", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + //change page state to write back + } else if (OB_FAIL(wbp_->notify_write_back(fd_, cur_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify write back for meta", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + ObTmpPageCacheKey cache_key(tree_io_info.block_index_, + write_offset / ObTmpFileGlobal::PAGE_SIZE, MTL_ID()); + ObTmpPageCacheValue cache_value(page_buff); + MEMCPY(block_buff + write_offset, page_buff, ObTmpFileGlobal::PAGE_SIZE); + if (OB_FAIL(calc_and_set_page_checksum_(block_buff + write_offset))) { + STORAGE_LOG(WARN, "fail to calc and set page checksum", KR(ret), K(fd_), KP(block_buff + write_offset)); + } else { + ObTmpPageCacheValue cache_value(block_buff + write_offset); + ObTmpPageCache::get_instance().try_put_page_to_cache(cache_key, cache_value); + write_offset += ObTmpFileGlobal::PAGE_SIZE; + if (flush_start_page_id == cur_page_id) { + tree_io_info.flush_start_page_id_ = cur_page_id; + tree_io_info.flush_start_level_page_index_ = page_index_in_level; + } + tree_io_info.flush_end_page_id_ = cur_page_id; + tree_io_info.flush_nums_++; + cur_page_id = next_page_id; + page_index_in_level++; + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::calc_and_set_page_checksum_(char* page_buff) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + page_header.checksum_ = ob_crc64(page_buff + PAGE_HEADER_SIZE, ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE); + MEMCPY(page_buff, &page_header, PAGE_HEADER_SIZE); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::flush_internal_pages_( + const uint32_t flush_start_page_id, + const int16_t level, + const int32_t start_page_index_in_level, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeIOInfo &tree_io_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == flush_start_page_id + || level >= level_page_range_array_.count() + || level < 1 + || 0 > start_page_index_in_level + || ObTmpFileTreeEvictType::INVALID == flush_type + || NULL == block_buff + || 0 != write_offset % ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(flush_start_page_id), K(level), K(start_page_index_in_level), + K(flush_type), KP(block_buff), K(level_page_range_array_), K(write_offset)); + } else { + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + //TODO: start page id in block is from 0 + tree_io_info.physical_start_page_id_ = write_offset / ObTmpFileGlobal::PAGE_SIZE; + tree_io_info.page_level_ = level; + uint32_t cur_page_id = flush_start_page_id; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + ObSharedNothingTmpFileMetaItem last_item; + int32_t page_index_in_level = start_page_index_in_level; + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id + && write_offset + ObTmpFileGlobal::PAGE_SIZE <= OB_DEFAULT_MACRO_BLOCK_SIZE + && (ObTmpFileTreeEvictType::FULL == flush_type || end_page_id != cur_page_id)) { + char * page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(level, page_index_in_level); + if (OB_UNLIKELY(!wbp_->is_dirty(fd_, cur_page_id, page_key))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page state", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + //check whether the page is satisfied for flush + last_item.reset(); + ObSharedNothingTmpFileTreePageHeader page_header; + int32_t rightmost_child_page_index = -1; + if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (page_header.item_num_ <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(fd_), K(page_header)); + } else if (OB_FAIL(read_item_(page_buff, page_header.item_num_ - 1/*item_index*/, last_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(page_header)); + } else if (FALSE_IT(rightmost_child_page_index = page_index_in_level * FULL_PAGE_META_ITEM_NUM + page_header.item_num_ - 1)) { + } else if (!is_page_flushed(last_item) + || (is_page_in_write_cache(last_item) + && wbp_->is_dirty(fd_, last_item.buffer_page_id_, ObTmpFilePageUniqKey(level - 1, rightmost_child_page_index)))) { + //we can be sure that the following pages in this level will not satisfy + break; + //change page state to write back + } else if (OB_FAIL(wbp_->notify_write_back(fd_, cur_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify write back for meta", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + ObTmpPageCacheKey cache_key(tree_io_info.block_index_, + write_offset / ObTmpFileGlobal::PAGE_SIZE, MTL_ID()); + MEMCPY(block_buff + write_offset, page_buff, ObTmpFileGlobal::PAGE_SIZE); + if (OB_FAIL(modify_child_pages_location(block_buff + write_offset))) { + STORAGE_LOG(WARN, "fail to modify child pages location", KR(ret), K(fd_), KP(block_buff + write_offset)); + } else if (OB_FAIL(calc_and_set_page_checksum_(block_buff + write_offset))) { + STORAGE_LOG(WARN, "fail to calc and set page checksum", KR(ret), K(fd_), KP(block_buff + write_offset)); + } else { + ObTmpPageCacheValue cache_value(block_buff + write_offset); + ObTmpPageCache::get_instance().try_put_page_to_cache(cache_key, cache_value); + write_offset += ObTmpFileGlobal::PAGE_SIZE; + if (flush_start_page_id == cur_page_id) { + tree_io_info.flush_start_page_id_ = cur_page_id; + tree_io_info.flush_start_level_page_index_ = page_index_in_level; + } + tree_io_info.flush_end_page_id_ = cur_page_id; + tree_io_info.flush_nums_++; + cur_page_id = next_page_id; + page_index_in_level++; + } + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::modify_child_pages_location( + char *page_buff) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header; + if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else { + int16_t item_index = 0; + ObSharedNothingTmpFileMetaItem meta_item; + while (OB_SUCC(ret) + && item_index < page_header.item_num_) { + meta_item.reset(); + if (OB_FAIL(read_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID == meta_item.buffer_page_id_) { + //do nothing + } else { + meta_item.buffer_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(rewrite_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } + } + item_index++; + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::modify_meta_items_at_parent_level_( + const ObTmpFileTreeIOInfo &tree_io, + const int32_t start_page_index_in_level) +{ + int ret = OB_SUCCESS; + const uint32_t cur_level = tree_io.page_level_ + 1; + if (OB_UNLIKELY(cur_level > level_page_range_array_.count() + || !tree_io.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(tree_io), K(level_page_range_array_)); + } else if (cur_level == level_page_range_array_.count()) { + //this must be the root page being flushed + if (OB_UNLIKELY(1 != tree_io.flush_nums_ + || tree_io.flush_start_page_id_ != tree_io.flush_end_page_id_ + || tree_io.flush_start_page_id_ != root_item_.buffer_page_id_ + || 0 != start_page_index_in_level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected tree_io", KR(ret), K(fd_), K(tree_io), K(root_item_), K(start_page_index_in_level)); + //NOTE: It doesn't matter if we release this page early, because under wlock. + } else if (is_page_flushed(root_item_) + && OB_FAIL(release_tmp_file_page_(root_item_.block_index_, root_item_.physical_page_id_, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(root_item_)); + } else { + root_item_.block_index_ = tree_io.block_index_; + root_item_.physical_page_id_ = tree_io.physical_start_page_id_; + } + } else { + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + int32_t child_level_page_index = start_page_index_in_level; + uint32_t physical_page_id = tree_io.physical_start_page_id_; + ObSharedNothingTmpFileMetaItem meta_item; + bool is_end = false; + bool has_find = false; + uint32_t last_flushed_page_id = level_page_range_array_.at(cur_level).flushed_end_page_id_; + //the child page of last meta item on last_flushed_page may be flushed again. + uint32_t cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t cur_level_page_index = level_page_range_array_.at(cur_level).flushed_page_num_; + if (ObTmpFileGlobal::INVALID_PAGE_ID == last_flushed_page_id) { + cur_page_id = level_page_range_array_.at(cur_level).start_page_id_; + } else { + cur_page_id = last_flushed_page_id; + cur_level_page_index--; + } + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id + && !is_end) { + int16_t item_index = 0; + char *page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileTreePageHeader page_header; + ObTmpFilePageUniqKey page_key(cur_level, cur_level_page_index); + //this upper layer page must be in write cache + if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (OB_UNLIKELY(0 >= page_header.item_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_header", KR(ret), K(fd_), K(page_header)); + } else { + while (OB_SUCC(ret) + && item_index < page_header.item_num_ + && !is_end) { + meta_item.reset(); + if (OB_FAIL(read_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } else { + //TODO: flush_start_page_id_ -> buffer_start_page_id_ + if (has_find || cur_level_page_index * FULL_PAGE_META_ITEM_NUM + item_index == child_level_page_index) { + if (OB_UNLIKELY((!has_find && tree_io.flush_start_page_id_ != meta_item.buffer_page_id_) + || cur_level_page_index * FULL_PAGE_META_ITEM_NUM + item_index != child_level_page_index)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_item", KR(ret), K(fd_), K(tree_io), K(meta_item), + K(child_level_page_index), K(cur_page_id), K(cur_level_page_index), K(item_index)); + } else { + has_find = true; + if (is_page_flushed(meta_item) + && OB_FAIL(release_tmp_file_page_(meta_item.block_index_, meta_item.physical_page_id_, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(meta_item), K(item_index), K(page_header)); + } else { + meta_item.block_index_ = tree_io.block_index_; + meta_item.physical_page_id_ = physical_page_id; + physical_page_id++; + child_level_page_index++; + if (OB_FAIL(rewrite_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), K(fd_), KP(page_buff), K(item_index), K(meta_item)); + } else if (tree_io.flush_end_page_id_ == meta_item.buffer_page_id_) { + is_end = true; + } + } + } + } + item_index++; + } + } + } + if (OB_SUCC(ret) && has_find && OB_FAIL(wbp_->notify_dirty(fd_, cur_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } + cur_page_id = next_page_id; + cur_level_page_index++; + } + if (OB_SUCC(ret)) { + if (OB_UNLIKELY(!is_end + || physical_page_id - tree_io.physical_start_page_id_ != tree_io.flush_nums_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected is_end or flush_nums_", KR(ret), K(fd_), + K(is_end), K(physical_page_id), K(tree_io)); + } + } + } + return ret; +} + +//TODO: 或许可以传进来ObIArray* > &tree_io_arrays二维数组 +// check tree io seq +int ObSharedNothingTmpFileMetaTree::update_after_flush( + const common::ObIArray &tree_io_array) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (OB_UNLIKELY(tree_io_array.empty())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(tree_io_array), K(root_item_)); + } else { + ARRAY_FOREACH_N(tree_io_array, i, cnt) { + bool tree_io_is_empty = false; + const ObTmpFileTreeIOInfo &tree_io = tree_io_array.at(i); + const int16_t origin_tree_flush_num = tree_io.flush_nums_; + bool end_page_flush_again = false; + if (OB_UNLIKELY(!tree_io.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(tree_io), KPC(this)); + } else if (tree_io.tree_epoch_ != tree_epoch_) { + STORAGE_LOG(INFO, "the tree_epoch_ in tree_io is not equal to current tree_epoch_", + K(fd_), K(tree_io), K(tree_epoch_)); + } else if (tree_io.page_level_ >= level_page_range_array_.count()) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(tree_io), KPC(this)); + } else { + const uint32_t flushed_end_page_id_in_array = level_page_range_array_[tree_io.page_level_].flushed_end_page_id_; + const uint32_t start_page_id_in_array = level_page_range_array_[tree_io.page_level_].start_page_id_; + uint32_t next_page_id_in_array = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = level_page_range_array_[tree_io.page_level_].flushed_page_num_; + if (ObTmpFileGlobal::INVALID_PAGE_ID != flushed_end_page_id_in_array) { + level_page_index--; + ObTmpFilePageUniqKey page_key(tree_io.page_level_, level_page_index); + if (OB_FAIL(wbp_->get_next_page_id(fd_, flushed_end_page_id_in_array, page_key, next_page_id_in_array))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(flushed_end_page_id_in_array), K(page_key)); + } else if (OB_UNLIKELY(flushed_end_page_id_in_array != tree_io.flush_start_page_id_ + && next_page_id_in_array != tree_io.flush_start_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected tree_io", KR(ret), K(tree_io), + K(flushed_end_page_id_in_array), K(next_page_id_in_array), KPC(this)); + } else if (flushed_end_page_id_in_array == tree_io.flush_start_page_id_) { + end_page_flush_again = true; + } else { + level_page_index++; + } + } else { + if (level_page_index > tree_io.flush_start_level_page_index_) { + int64_t truncated_num = level_page_index - tree_io.flush_start_level_page_index_; + if (truncated_num >= tree_io.flush_nums_) { + tree_io_is_empty = true; + } else { + ObTmpFileTreeIOInfo& tree_io_mutable_ref = const_cast(tree_io); + tree_io_mutable_ref.flush_start_page_id_ = start_page_id_in_array; + tree_io_mutable_ref.flush_nums_ -= truncated_num; + } + } else if (OB_UNLIKELY(start_page_id_in_array != tree_io.flush_start_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected tree_io", KR(ret), K(tree_io), KPC(this)); + } + } + if (OB_SUCC(ret) && !tree_io_is_empty) { + uint32_t cur_page_id = tree_io.flush_start_page_id_; + int16_t num = 0; + //change page state from writeback to cached + while (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + ObTmpFilePageUniqKey page_key(tree_io.page_level_, level_page_index); + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (cur_page_id == tree_io.flush_end_page_id_ && wbp_->is_dirty(fd_, cur_page_id, page_key)) { + //do nothing + STORAGE_LOG(INFO, "page is dirty again, do not change page status", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_FAIL(wbp_->notify_write_back_succ(fd_, cur_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify write back succ for meta", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } + if (OB_SUCC(ret)) { + num++; + if (OB_UNLIKELY(num > tree_io.flush_nums_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected num", KR(ret), K(tree_io), K(num), K(cur_page_id), KPC(this)); + } else if (cur_page_id == tree_io.flush_end_page_id_) { + break; + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_page_id, page_key, next_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(cur_page_id), K(tree_io), K(page_key)); + } else { + cur_page_id = next_page_id; + level_page_index++; + } + } + } + if (OB_SUCC(ret)) { + if (OB_UNLIKELY(tree_io.flush_nums_ != num + || cur_page_id != tree_io.flush_end_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page level or flush num", KR(ret), K(tree_io), K(num), K(cur_page_id), KPC(this)); + } else { + level_page_range_array_[tree_io.page_level_].flushed_end_page_id_ = tree_io.flush_end_page_id_; + if (end_page_flush_again) { + num--; + } + level_page_range_array_[tree_io.page_level_].flushed_page_num_ += num; + } + } + } + } + if (OB_SUCC(ret)) { + stat_info_.meta_page_flushing_cnt_ -= origin_tree_flush_num; + } + } + } + STORAGE_LOG(INFO, "finish update after flush", KR(ret), K(fd_), K(level_page_range_array_)); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::prepare_for_write_tail( + ObSharedNothingTmpFileDataItem &last_data_item) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (!root_item_.is_valid()) { + if (OB_UNLIKELY(data_item_array_.empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data_item_array_ count", KR(ret), K(fd_), K(data_item_array_)); + } else { + last_data_item = data_item_array_.at(data_item_array_.count() - 1); + if (OB_UNLIKELY(0 >= last_data_item.physical_page_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected physical_page_num", KR(ret), K(last_data_item), KPC(this)); + } + } + } else { + ObSharedNothingTmpFileTreePageHeader page_header; + ObSharedNothingTmpFileMetaItem page_info; + if (OB_UNLIKELY(level_page_range_array_.empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), KPC(this)); + } else if (OB_FAIL(get_rightmost_leaf_page_for_write_(page_info))) { + STORAGE_LOG(WARN, "fail to get rightmost leaf page for write", KR(ret), KPC(this)); + } else { + //we don't need to worry about the rightmost leaf page being evicted, + // because we set is_writing_ = true. + char *leaf_page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t leaf_level_page_index = level_page_range_array_.at(0).evicted_page_num_ + + level_page_range_array_.at(0).cached_page_num_ - 1; + if (OB_FAIL(wbp_->read_page(fd_, page_info.buffer_page_id_, ObTmpFilePageUniqKey(0, leaf_level_page_index), + leaf_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_info), K(leaf_level_page_index)); + } else if (OB_FAIL(read_page_header_(leaf_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), KP(leaf_page_buff), KPC(this)); + } else if (OB_UNLIKELY(0 >= page_header.item_num_)) { + //There is no concurrent writing + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), KP(leaf_page_buff), K(page_header), KPC(this)); + } else if (OB_FAIL(read_item_(leaf_page_buff, page_header.item_num_ - 1, last_data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), KP(leaf_page_buff), K(page_header), KPC(this)); + } else if (OB_UNLIKELY(0 >= last_data_item.physical_page_num_)) { + ret = OB_ERR_UNEXPECTED; + //print page content + //NOTE: control print frequence + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObArray items; + read_page_content_(leaf_page_buff, tmp_page_header, items); + STORAGE_LOG(WARN, "unexpected physical_page_num, dump tree page", KR(ret), K(last_data_item), KP(leaf_page_buff), + K(page_info), K(leaf_level_page_index), K(tmp_page_header), K(items), KPC(this)); + } + } + } + return ret; +} + +//After the tail is written +int ObSharedNothingTmpFileMetaTree::finish_write_tail( + const ObSharedNothingTmpFileDataItem &last_data_item, + const bool release_tail_in_disk) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (!root_item_.is_valid()) { + if (OB_UNLIKELY(data_item_array_.empty() + || last_data_item != data_item_array_[data_item_array_.count() - 1])) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data_item_array_ or last_data_item", KR(ret), K(last_data_item), KPC(this)); + } else if (release_tail_in_disk) { + if (0 == --data_item_array_[data_item_array_.count() - 1].physical_page_num_) { + data_item_array_.pop_back(); + } + } + } else { + if (OB_UNLIKELY(level_page_range_array_.empty() + || !is_writing_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_array_ or is_writing_", KR(ret), KPC(this)); + } else if (release_tail_in_disk) { + char *leaf_page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t page_id = level_page_range_array_[0].end_page_id_; + int32_t level_page_index = level_page_range_array_.at(0).evicted_page_num_ + + level_page_range_array_.at(0).cached_page_num_ - 1; + ObSharedNothingTmpFileTreePageHeader page_header; + ObSharedNothingTmpFileDataItem data_item; + ObTmpFilePageUniqKey page_key(0, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, page_id, page_key, leaf_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_id), K(page_key)); + } else if (OB_FAIL(read_page_header_(leaf_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), KP(leaf_page_buff), KPC(this)); + } else if (OB_FAIL(read_item_(leaf_page_buff, page_header.item_num_ - 1, data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), KP(leaf_page_buff), K(page_header), KPC(this)); + } else if (OB_UNLIKELY(last_data_item != data_item)) { + ret = OB_ERR_UNEXPECTED; + //print page content + //NOTE: control print frequence + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObArray items; + read_page_content_(leaf_page_buff, tmp_page_header, items); + STORAGE_LOG(WARN, "unexpected virtual_page_id, dump tree page", KR(ret), KP(leaf_page_buff), K(page_id), K(page_key), + K(last_data_item), K(data_item), K(tmp_page_header), K(items), KPC(this)); + } else if (FALSE_IT(data_item.physical_page_num_--)) { + } else if (0 == data_item.physical_page_num_) { + if (OB_FAIL(remove_page_item_from_tail_(leaf_page_buff, 1/*remove_num*/))) { + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), KP(leaf_page_buff), KPC(this)); + } + } else if (OB_FAIL(rewrite_item_(leaf_page_buff, page_header.item_num_ - 1, data_item))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), KP(leaf_page_buff), + K(page_header), K(data_item), KPC(this)); + } + if (FAILEDx(wbp_->notify_dirty(fd_, page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(page_id), K(page_key)); + } + } + if (OB_SUCC(ret)) { + is_writing_ = false; + } + } + if (OB_SUCC(ret) && release_tail_in_disk) { + if (OB_FAIL(release_tmp_file_page_(last_data_item.block_index_, + last_data_item.physical_page_id_ + last_data_item.physical_page_num_ - 1, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(last_data_item)); + } + } + STORAGE_LOG(INFO, "finish write tail", KR(ret), K(fd_), K(release_tail_in_disk)); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::evict_meta_pages( + const int64_t expected_page_num, + const ObTmpFileTreeEvictType flush_type, + int64_t &actual_evict_page_num) +{ + int ret = OB_SUCCESS; + actual_evict_page_num = 0; + if (OB_UNLIKELY(0 >= expected_page_num + || ObTmpFileTreeEvictType::INVALID == flush_type)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(expected_page_num), K(flush_type)); + } else { + SpinWLockGuard guard(lock_); + if (!root_item_.is_valid()) { + //do nothing + } else { + ObSEArray evict_pages; + ARRAY_FOREACH_X(level_page_range_array_, level, level_cnt, + OB_SUCC(ret) && actual_evict_page_num < expected_page_num) { + evict_pages.reset(); + uint32_t next_page_id = level_page_range_array_[level].start_page_id_; + const uint32_t end_evict_page = level_page_range_array_[level].flushed_end_page_id_; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + const int32_t start_level_page_index = level_page_range_array_.at(level).evicted_page_num_; + uint32_t cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = start_level_page_index; + if (ObTmpFileGlobal::INVALID_PAGE_ID == end_evict_page) { + //no pages need to be evicted at this level + continue; + } + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != next_page_id + && actual_evict_page_num < expected_page_num + && (!is_writing_ || end_page_id != next_page_id) //not evict end page if is writing + && (ObTmpFileTreeEvictType::FULL == flush_type || end_page_id != next_page_id)) { + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (OB_UNLIKELY(end_evict_page != next_page_id && !wbp_->is_cached(fd_, next_page_id, page_key))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected next_page_id", KR(ret), K(next_page_id), KPC(this)); + } else if (end_evict_page == next_page_id && !wbp_->is_cached(fd_, next_page_id, page_key)) { + break; + } else if (level > 0) { + //check whether the page is satisfied for evict + char * page_buff = NULL; + ObSharedNothingTmpFileMetaItem last_item; + uint32_t unused_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_->read_page(fd_, next_page_id, page_key, page_buff, unused_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(next_page_id), K(page_key)); + } else { + last_item.reset(); + ObSharedNothingTmpFileTreePageHeader page_header; + if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (page_header.item_num_ <= 0) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(page_header), KPC(this)); + } else if (OB_FAIL(read_item_(page_buff, page_header.item_num_ - 1/*item_index*/, last_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), KP(page_buff), K(page_header), KPC(this)); + } else if (is_page_in_write_cache(last_item)) { + if (OB_UNLIKELY(end_evict_page != next_page_id)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected next_page", KR(ret), K(next_page_id), K(last_item), KPC(this)); + } else { + break; + } + } + } + } + if (OB_SUCC(ret)) { + cur_page_id = next_page_id; + next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(evict_pages.push_back(cur_page_id))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(cur_page_id)); + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_page_id, page_key, next_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + actual_evict_page_num++; + level_page_index++; + if (end_evict_page == cur_page_id) { + break; + } + } + } + } + if (OB_SUCC(ret) && !evict_pages.empty()) { + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == cur_page_id)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected cur_page_id", KR(ret), K(cur_page_id), K(level), K(level_page_index), KPC(this)); + } else if (OB_FAIL(modify_meta_items_during_evict_(evict_pages, level + 1, start_level_page_index))) { + STORAGE_LOG(WARN, "fail to modify meta items during evict", KR(ret), K(evict_pages), + K(level), K(start_level_page_index), KPC(this)); + } else { + uint32_t unused_next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ARRAY_FOREACH_N(evict_pages, i, cnt) { + //change page state from flushed/cached to evicted/invalid + //TODO: check whether page state is cached/clean. + if (OB_FAIL(wbp_->free_page(fd_, evict_pages.at(i), + ObTmpFilePageUniqKey(level, start_level_page_index + i), unused_next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page in write cache", KR(ret), K(fd_), K(evict_pages.at(i)), + K(level), K(start_level_page_index + i)); + } + } + } + if (OB_SUCC(ret)) { + if (end_page_id == cur_page_id) { + //all pages in this level are evicted + level_page_range_array_[level].start_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + level_page_range_array_[level].end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + level_page_range_array_[level].flushed_end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == next_page_id)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected next_page_id", KR(ret), K(next_page_id), K(cur_page_id), KPC(this)); + } else if (end_evict_page == cur_page_id) { + //flushed_end_page is not equal to end_page, + // and pages are evicted to flushed_end_page(including flushed_end_page) + level_page_range_array_[level].start_page_id_ = next_page_id; + level_page_range_array_[level].flushed_end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } else { + //pages are not evicted to flushed_end_page + level_page_range_array_[level].start_page_id_ = next_page_id; + } + if (OB_SUCC(ret)) { + level_page_range_array_[level].cached_page_num_ -= evict_pages.count(); + level_page_range_array_[level].evicted_page_num_ += evict_pages.count(); + } + } + } + } + } + } + STORAGE_LOG(INFO, "finish evict meta pages", KR(ret), K(fd_), K(level_page_range_array_)); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::modify_meta_items_during_evict_( + const ObIArray &evict_pages, + const int16_t level, + const int32_t start_page_index_in_level) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(evict_pages.empty() + || level > level_page_range_array_.count())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected evict_pages", KR(ret), K(fd_), K(evict_pages), + K(level), K(level_page_range_array_)); + } else { + char *page_buff = NULL; + const int64_t level_count = level_page_range_array_.count(); + const int64_t evict_page_count = evict_pages.count(); + int64_t array_index = 0; + bool has_find = false; + if (level < level_count) { + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + uint32_t cur_page_id = level_page_range_array_[level].start_page_id_; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + int32_t level_page_index = level_page_range_array_[level].evicted_page_num_; + int32_t evict_page_index_in_level = start_page_index_in_level; + ObSharedNothingTmpFileMetaItem meta_item; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == cur_page_id + || ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id + || 0 > evict_page_index_in_level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_id", KR(ret), K(fd_), K(level_page_range_array_), + K(level), K(evict_page_index_in_level)); + } else { + while(OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id + && array_index < evict_page_count) { + ObSharedNothingTmpFileTreePageHeader page_header; + char *page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, ObTmpFilePageUniqKey(level, level_page_index), page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(cur_page_id), K(level), K(level_page_index)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else { + int16_t item_index = 0; + while (OB_SUCC(ret) + && item_index < page_header.item_num_ + && array_index < evict_page_count) { + meta_item.reset(); + if (OB_FAIL(read_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } else { + if (has_find || level_page_index * FULL_PAGE_META_ITEM_NUM + item_index == evict_page_index_in_level) { + if (OB_UNLIKELY(meta_item.buffer_page_id_ != evict_pages.at(array_index) + || level_page_index * FULL_PAGE_META_ITEM_NUM + item_index != evict_page_index_in_level + || !is_page_in_write_cache(meta_item) + || !is_page_flushed(meta_item))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_item", KR(ret), K(fd_), K(evict_pages), K(array_index), K(meta_item), + K(evict_page_index_in_level), K(cur_page_id), K(level_page_index), K(item_index)); + } else { + has_find = true; + meta_item.buffer_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + array_index++; + evict_page_index_in_level++; + if (OB_FAIL(rewrite_item_(page_buff, item_index, meta_item))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } + } + } + } + item_index++; + } + } + //TODO: Is it necessary to mark it as dirty? Maybe not. + cur_page_id = next_page_id; + level_page_index++; + } + if (OB_SUCC(ret) && evict_page_count != array_index) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected array_index", KR(ret), K(fd_), K(array_index), K(evict_pages), K(level_page_range_array_)); + } + } + } else { //level == level_count + if (OB_UNLIKELY(!is_page_in_write_cache(root_item_) + || !is_page_flushed(root_item_) + || 1 != evict_page_count + || evict_pages.at(0) != root_item_.buffer_page_id_ + || 0 != start_page_index_in_level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected root_item_ or evict_pages", KR(ret), K(fd_), K(root_item_), + K(evict_pages), K(start_page_index_in_level)); + } else { + root_item_.buffer_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::clear( + const int64_t last_truncate_offset, + const int64_t total_file_size) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + int64_t end_truncate_offset = upper_align(total_file_size, ObTmpFileGlobal::PAGE_SIZE); + + if (OB_UNLIKELY(last_truncate_offset < released_offset_ + || last_truncate_offset > total_file_size + || is_writing_ + || 0 < stat_info_.meta_page_flushing_cnt_)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(last_truncate_offset), K(total_file_size), KPC(this)); + } else if (OB_FAIL(truncate_(end_truncate_offset))) { + STORAGE_LOG(WARN, "fail to truncate_", KR(ret), K(last_truncate_offset), K(end_truncate_offset), K(total_file_size), KPC(this)); + } else if (OB_UNLIKELY(root_item_.is_valid() + || !level_page_range_array_.empty() + || !data_item_array_.empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected root_item_ or array_", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(stat_info_.all_type_page_flush_cnt_ != stat_info_.all_type_flush_page_released_cnt_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected stat_info_", KR(ret), KPC(this)); + } + return ret; +} + +//clear leaf page or internal page in meta tree +int ObSharedNothingTmpFileMetaTree::release_meta_page_( + const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t page_index_in_level) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(level_page_range_array_.count() <= page_info.page_level_ + || 0 > page_index_in_level)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_ or page_info", KR(ret), K(fd_), K(level_page_range_array_), + K(page_info), K(page_index_in_level)); + } else if (is_page_in_write_cache(page_info)) { + const uint32_t start_page_id_in_array = level_page_range_array_.at(page_info.page_level_).start_page_id_; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + if (OB_FAIL(wbp_->free_page(fd_, page_info.buffer_page_id_, + ObTmpFilePageUniqKey(page_info.page_level_, page_index_in_level), next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page in write cache", KR(ret), K(fd_), K(page_info), K(page_index_in_level)); + } else if (OB_UNLIKELY(start_page_id_in_array != page_info.buffer_page_id_)) { + //NOTE: pages must be released sequentially (from front to back in array) + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), K(fd_), K(level_page_range_array_), K(page_info)); + } else { + level_page_range_array_.at(page_info.page_level_).start_page_id_ = next_page_id; + if (start_page_id_in_array == level_page_range_array_.at(page_info.page_level_).end_page_id_) { + //next_page_id must be invalid + level_page_range_array_.at(page_info.page_level_).end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } + if (ObTmpFileGlobal::INVALID_PAGE_ID == level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_) { + level_page_range_array_.at(page_info.page_level_).flushed_page_num_ += 1; + } + if (start_page_id_in_array == level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_) { + level_page_range_array_.at(page_info.page_level_).flushed_end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + } + level_page_range_array_.at(page_info.page_level_).cached_page_num_ -= 1; + level_page_range_array_.at(page_info.page_level_).evicted_page_num_ += 1; + } + } + if (OB_SUCC(ret) && is_page_flushed(page_info)) { + if (OB_FAIL(release_tmp_file_page_(page_info.block_index_, page_info.physical_page_id_, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(page_info)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::release_tmp_file_page_( + const int64_t block_index, const int64_t begin_page_id, const int64_t page_num) +{ + int ret = OB_SUCCESS; + + // XXX 最终需要消除对 MTL 的依赖; + ObTmpFileBlockManager &block_manager = MTL(ObTenantTmpFileManager*)->get_tmp_file_block_manager(); + if (OB_FAIL(block_manager.release_tmp_file_page(block_index, begin_page_id, page_num))) { + STORAGE_LOG(WARN, "fail to release tmp file page", + KR(ret), K(fd_), K(block_index), K(begin_page_id), K(page_num)); + } else { + stat_info_.all_type_flush_page_released_cnt_ += page_num; + } + + return ret; +} + +int ObSharedNothingTmpFileMetaTree::truncate( + const int64_t last_truncate_offset, + const int64_t end_truncate_offset) +{ + int ret = OB_SUCCESS; + SpinWLockGuard guard(lock_); + if (OB_UNLIKELY(last_truncate_offset < released_offset_ + || 0 != released_offset_ % ObTmpFileGlobal::PAGE_SIZE + || last_truncate_offset > end_truncate_offset + || is_writing_)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(last_truncate_offset), K(end_truncate_offset), KPC(this)); + } else if (OB_FAIL(truncate_(end_truncate_offset))) { + STORAGE_LOG(WARN, "fail to truncate_", KR(ret), K(last_truncate_offset), K(end_truncate_offset), KPC(this)); + } else if (!root_item_.is_valid()) { + //NOTE: end_truncate_offset >= max offset in tree. + // released_offset_ = end_truncate_offset; + } + STORAGE_LOG(INFO, "finish truncate array or meta tree", KR(ret), K(fd_), K(released_offset_), K(last_truncate_offset), K(end_truncate_offset)); + return ret; +} + +int ObSharedNothingTmpFileMetaTree::truncate_( + const int64_t end_truncate_offset) +{ + int ret = OB_SUCCESS; + if (!root_item_.is_valid()) { + if (OB_FAIL(truncate_array_(end_truncate_offset))) { + STORAGE_LOG(WARN, "fail to truncate array", KR(ret), K(fd_), K(end_truncate_offset)); + } + } else { + //TODO: 可以将FULL_PAGE_META_ITEM_NUM作为类的成员吗? + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + ObArray> item_index_arr; + if (OB_FAIL(calculate_truncate_index_path_(item_index_arr))) { + STORAGE_LOG(WARN, "fail to calculate truncate index path", KR(ret), K(fd_), K(item_index_arr)); + } else if (OB_UNLIKELY(item_index_arr.empty())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_index_arr", KR(ret), K(fd_), K(item_index_arr)); + } else { + ObSharedNothingTmpFileMetaItem meta_item = root_item_; + ObSharedNothingTmpFileMetaItem next_level_meta_item; + ObSEArray truncate_path; + while (OB_SUCC(ret) + && 0 < meta_item.page_level_) { + next_level_meta_item.reset(); + int16_t item_index = -1; + int32_t level_page_index = -1; + ObSharedNothingTmpFileTreePageHeader page_header; + char *page_buff = NULL; + ObTmpPageValueHandle p_handle; + if (OB_UNLIKELY(item_index_arr.count() <= meta_item.page_level_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_index_arr", KR(ret), K(fd_), K(item_index_arr), K(meta_item)); + } else if (FALSE_IT(level_page_index = item_index_arr.at(meta_item.page_level_).first)) { + } else if (FALSE_IT(item_index = item_index_arr.at(meta_item.page_level_).second)) { + } else if (OB_FAIL(get_page_(meta_item, level_page_index, page_buff, p_handle))) { + STORAGE_LOG(WARN, "fail to get page", KR(ret), K(fd_), K(meta_item)); + } else if (OB_FAIL(read_item_(page_buff, item_index, next_level_meta_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } else if (OB_FAIL(truncate_path.push_back(BacktraceNode(meta_item /*page info*/, + level_page_index, /*page index in its level*/ + item_index /*item index on the page*/)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_item)); + } else { + meta_item = next_level_meta_item; + } + p_handle.reset(); + } + if (OB_SUCC(ret)) { + bool release_last_item = false; + if (0 == meta_item.page_level_) { + if (OB_FAIL(release_items_of_leaf_page_(meta_item, item_index_arr.at(0).first, end_truncate_offset, + item_index_arr.at(0).second, release_last_item))) { + STORAGE_LOG(WARN, "fail to release items of leaf page", KR(ret), K(fd_), K(meta_item), + K(end_truncate_offset), K(item_index_arr)); + } else if (release_last_item) { + if (OB_FAIL(release_meta_page_(meta_item, item_index_arr.at(0).first))) { + STORAGE_LOG(WARN, "fail to release meta page", KR(ret), K(fd_), K(meta_item), K(item_index_arr.at(0).first)); + //even release_offset == end_offset,We won't end here, because the upper-level pages may need to be released + } else if (OB_FAIL(backtrace_truncate_tree_(end_truncate_offset, truncate_path))) { + STORAGE_LOG(WARN, "fail to backtrace truncate tree", KR(ret), K(fd_), K(end_truncate_offset), K(truncate_path)); + } + } + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page type", KR(ret), K(fd_), K(meta_item)); + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::truncate_array_( + const int64_t end_offset) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(end_offset < 0 + || data_item_array_.count() > MAX_DATA_ITEM_ARRAY_COUNT)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(end_offset), K(data_item_array_)); + } else if (data_item_array_.empty()) { + STORAGE_LOG(INFO, "no data to truncate", KR(ret), K(fd_), K(data_item_array_), K(root_item_)); + } else { + const ObSharedNothingTmpFileDataItem &last_item = data_item_array_.at(data_item_array_.count() - 1); + if (OB_UNLIKELY(!last_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected last item", KR(ret), K(fd_), K(last_item)); + } else if (end_offset >= (last_item.virtual_page_id_ + last_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE) { + //clear all items + ARRAY_FOREACH_N(data_item_array_, i, cnt) { + const ObSharedNothingTmpFileDataItem &item = data_item_array_.at(i); + if (OB_FAIL(release_tmp_file_page_(item.block_index_, + item.physical_page_id_, item.physical_page_num_))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(item), K(i), K(cnt)); + } else if (i + 1 == cnt) { + released_offset_ = (item.virtual_page_id_ + item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + } + } + if (OB_SUCC(ret)) { + data_item_array_.reset(); + } + } else { + //clear some items + const int64_t target_virtual_page_id = end_offset / ObTmpFileGlobal::PAGE_SIZE; + int16_t index = 0; + ARRAY_FOREACH_N(data_item_array_, i, cnt) { + const ObSharedNothingTmpFileDataItem &data_item = data_item_array_.at(i); + if (OB_UNLIKELY(!data_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data item", KR(ret), K(fd_), K(data_item)); + } else if (data_item.virtual_page_id_ <= target_virtual_page_id) { + index = i; + } else { + break; + } + } + if (OB_SUCC(ret)) { + if (0 == index) { + //do nothing + } else { + //release items before "index" + int16_t array_cnt = data_item_array_.count(); + for (int16_t i = 0; OB_SUCC(ret) && i < index; i++) { + ObSharedNothingTmpFileDataItem &data_item = data_item_array_.at(i); + if (OB_UNLIKELY(!data_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data item", KR(ret), K(fd_), K(data_item)); + } else if (OB_FAIL(release_tmp_file_page_(data_item.block_index_, + data_item.physical_page_id_, + data_item.physical_page_num_))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(data_item), K(i), K(index)); + } else if (i + 1 == index) { + released_offset_ = (data_item.virtual_page_id_ + data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + } else { + data_item.reset(); + } + } + if (OB_SUCC(ret)) { + //move items + for (int16_t i = index; i < array_cnt; i++) { + data_item_array_.at(i - index) = data_item_array_.at(i); + } + for (int16_t i = 0; i < index; i++) { + data_item_array_.pop_back(); + } + } + } + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::calculate_truncate_index_path_( + ObIArray> &item_index_arr) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!root_item_.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected root_item_", KR(ret), K(fd_), K(root_item_)); + } else { + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + int32_t child_level_page_index = -1; + ARRAY_FOREACH_N(level_page_range_array_, i, cnt) { + int16_t item_index = -1; + if (!last_truncate_leaf_info_.is_valid()) { + child_level_page_index = 0; + item_index = 0; + } else { + int32_t level_page_index = -1; + int32_t level_page_num = level_page_range_array_.at(i).evicted_page_num_ + level_page_range_array_.at(i).cached_page_num_; + if (0 == i) { + level_page_index = last_truncate_leaf_info_.page_index_in_leaf_level_; + item_index = last_truncate_leaf_info_.item_index_in_page_ + 1; + if (last_truncate_leaf_info_.release_to_end_in_page_) { + level_page_index++; + item_index = 0; + } + } else { //internal level + item_index = child_level_page_index % FULL_PAGE_META_ITEM_NUM; + level_page_index = child_level_page_index / FULL_PAGE_META_ITEM_NUM; + } + if (level_page_index >= level_page_num) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "no data to truncate but root_item_ is valid", KR(ret), K(fd_), + K(i), K(level_page_index), K(level_page_num), K(level_page_range_array_)); + } else { + child_level_page_index = level_page_index; + } + } + if (FAILEDx(item_index_arr.push_back(std::make_pair(child_level_page_index, item_index)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(child_level_page_index), K(item_index)); + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::release_items_of_leaf_page_( + const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int64_t end_offset, + const int16_t begin_release_index, + bool &release_last_item) +{ + int ret = OB_SUCCESS; + release_last_item = false; + char *page_buff = NULL; + int16_t item_index = -1; + int16_t end_release_index = -1; + int64_t tmp_release_offset = released_offset_; + ObSharedNothingTmpFileDataItem data_item; + ObSharedNothingTmpFileTreePageHeader page_header; + ObTmpPageValueHandle p_handle; + if (OB_UNLIKELY(0 != tmp_release_offset % ObTmpFileGlobal::PAGE_SIZE + || 0 > begin_release_index + || PAGE_HEADER_SIZE + (begin_release_index + 1) * sizeof(ObSharedNothingTmpFileDataItem) > ObTmpFileGlobal::PAGE_SIZE + || begin_release_index >= MAX_PAGE_ITEM_COUNT)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected release_offset or begin_release_index", KR(ret), K(fd_), K(tmp_release_offset), K(begin_release_index)); + } else if (OB_FAIL(get_page_(page_info, level_page_index, page_buff, p_handle))) { + STORAGE_LOG(WARN, "fail to get page", KR(ret), K(fd_), K(page_info), K(level_page_index)); + } else if (OB_FAIL(read_page_header_(page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(page_buff)); + } else if (0 == page_header.item_num_ || begin_release_index == page_header.item_num_) { + //maybe the item have been cleared (corresponds to an unfilled data page) + release_last_item = true; + } else { + //get end_release_index + int64_t target_virtual_page_id = end_offset / ObTmpFileGlobal::PAGE_SIZE; + item_index = -1; + data_item.reset(); + if (OB_FAIL(read_item_(page_buff, target_virtual_page_id, item_index, data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(target_virtual_page_id)); + } else if (OB_UNLIKELY(!data_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data item", KR(ret), K(fd_), K(data_item)); + } else if (target_virtual_page_id >= data_item.virtual_page_id_ + data_item.physical_page_num_) { + if (OB_UNLIKELY(item_index + 1 != page_header.item_num_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_index", KR(ret), K(fd_), K(item_index)); + } else { + end_release_index = item_index; + release_last_item = true; + } + } else { + end_release_index = item_index - 1; + } + if (OB_SUCC(ret)) { + item_index = begin_release_index; + while (OB_SUCC(ret) + && item_index <= end_release_index) { + data_item.reset(); + if (OB_FAIL(read_item_(page_buff, item_index, data_item))) { + STORAGE_LOG(WARN, "fail to read item", KR(ret), K(fd_), KP(page_buff), K(item_index)); + } else if (OB_UNLIKELY(!data_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected data item", KR(ret), K(fd_), K(data_item)); + } else if (OB_FAIL(release_tmp_file_page_(data_item.block_index_, + data_item.physical_page_id_, + data_item.physical_page_num_))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(data_item), + K(begin_release_index), K(item_index), K(end_release_index)); + } else { + item_index++; + } + } + } + if (OB_SUCC(ret) && begin_release_index <= end_release_index) { + tmp_release_offset = (data_item.virtual_page_id_ + data_item.physical_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + if (OB_UNLIKELY(tmp_release_offset > end_offset + || tmp_release_offset <= released_offset_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected release_offset", KR(ret), K(fd_), K(tmp_release_offset), K(end_offset), K(released_offset_)); + } else { + released_offset_ = tmp_release_offset; + last_truncate_leaf_info_.page_index_in_leaf_level_ = level_page_index; + last_truncate_leaf_info_.item_index_in_page_ = end_release_index; + last_truncate_leaf_info_.release_to_end_in_page_ = release_last_item; + } + } + if (OB_FAIL(ret)) { + //print page content + //NOTE: control print frequence + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ObArray items; + read_page_content_(page_buff, tmp_page_header, items); + STORAGE_LOG(WARN, "fail to release leaf page items, dump tree page", KR(ret), K(fd_), KP(page_buff), K(page_info), + K(level_page_index), K(begin_release_index), K(end_release_index), K(end_offset), K(tmp_page_header), K(items)); + } + p_handle.reset(); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::backtrace_truncate_tree_( + const int64_t end_offset, + ObIArray &search_path) +{ + int ret = OB_SUCCESS; + const int16_t FULL_PAGE_META_ITEM_NUM = + MIN(MAX_PAGE_ITEM_COUNT, (ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE) / sizeof(ObSharedNothingTmpFileMetaItem)); + ObSEArray meta_items; + bool need_finish = false; + while (OB_SUCC(ret) + && !search_path.empty() + && !need_finish) { + meta_items.reset(); + bool reach_last_item = false; + int64_t last_node_index = search_path.count() - 1; + //meta_item points to a page + const ObSharedNothingTmpFileMetaItem &page_info = search_path.at(last_node_index).page_meta_info_; + //the page index in its level + const int32_t level_page_index = search_path.at(last_node_index).level_page_index_; + //current pos need to be processed on this page. + int16_t cur_item_index = search_path.at(last_node_index).prev_item_index_ + 1; + if (OB_FAIL(get_items_of_internal_page_(page_info, level_page_index, cur_item_index, + end_offset, reach_last_item, meta_items))) { + STORAGE_LOG(WARN, "fail to get items of internal page", KR(ret), K(fd_), K(page_info), + K(level_page_index), K(cur_item_index), K(end_offset)); + } else if (1 == page_info.page_level_) { + int16_t release_page_cnt = 0; + const int32_t start_leaf_level_page_index = level_page_index * FULL_PAGE_META_ITEM_NUM + cur_item_index; + ARRAY_FOREACH_X(meta_items, i, cnt, OB_SUCC(ret) && !need_finish) { + bool release_last_item = false; + if (OB_FAIL(release_items_of_leaf_page_(meta_items.at(i), start_leaf_level_page_index + i, end_offset, + 0 /*release from start of the page*/, release_last_item))) { + STORAGE_LOG(WARN, "fail to release items of leaf page", KR(ret), K(fd_), K(meta_items.at(i)), + K(start_leaf_level_page_index + i), K(end_offset)); + } else if (release_last_item) { + if (OB_FAIL(release_meta_page_(meta_items.at(i), start_leaf_level_page_index + i))) { + STORAGE_LOG(WARN, "fail to release meta page", KR(ret), K(fd_), K(meta_items.at(i)), + K(start_leaf_level_page_index + i)); + } else { + release_page_cnt++; + } + } else { + need_finish = true; + } + } + if (OB_SUCC(ret)) { + if (!reach_last_item) { + need_finish = true; + } else if (reach_last_item && (meta_items.empty() || release_page_cnt == meta_items.count())) { + if (OB_FAIL(release_meta_page_(page_info, level_page_index))) { + STORAGE_LOG(WARN, "fail to release meta page", KR(ret), K(fd_), K(page_info), K(level_page_index)); + } else { + search_path.pop_back(); + } + } + } + } else if (1 < page_info.page_level_) { + if (meta_items.count() == 1) { + const int32_t child_level_page_index = level_page_index * FULL_PAGE_META_ITEM_NUM + cur_item_index; + search_path.at(last_node_index).prev_item_index_ = cur_item_index; + if (OB_FAIL(search_path.push_back(BacktraceNode(meta_items.at(0), child_level_page_index, -1)))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(meta_items.at(0)), K(child_level_page_index)); + } + } else if (meta_items.empty()) { + if (reach_last_item) { + if (OB_FAIL(release_meta_page_(page_info, level_page_index))) { + STORAGE_LOG(WARN, "fail to release meta page", KR(ret), K(fd_), K(page_info), K(level_page_index)); + } else { + search_path.pop_back(); + } + } else { + need_finish = true; + } + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected meta_items", KR(ret), K(fd_), K(meta_items)); + } + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page info", KR(ret), K(fd_), K(page_info)); + } + } + if (OB_SUCC(ret) && !need_finish) { + //all pages are released + if (OB_FAIL(check_tree_is_empty_())) { + STORAGE_LOG(WARN, "unexpected, tree is not empty", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(stat_info_.all_type_page_flush_cnt_ != stat_info_.all_type_flush_page_released_cnt_)) { + //TODO: do not throw errors in the future + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected stat_info_", KR(ret), K(fd_), K(stat_info_)); + } else { + ++tree_epoch_; + root_item_.reset(); + level_page_range_array_.reset(); + last_truncate_leaf_info_.reset(); + STORAGE_LOG(INFO, "meta tree pages are all truncated", KR(ret), K(fd_), K(tree_epoch_)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::check_tree_is_empty_() +{ + int ret = OB_SUCCESS; + ARRAY_FOREACH_N(level_page_range_array_, i, cnt) { + if (OB_UNLIKELY(0 != level_page_range_array_.at(i).cached_page_num_ + || ObTmpFileGlobal::INVALID_PAGE_ID != level_page_range_array_.at(i).start_page_id_ + || ObTmpFileGlobal::INVALID_PAGE_ID != level_page_range_array_.at(i).end_page_id_ + || ObTmpFileGlobal::INVALID_PAGE_ID != level_page_range_array_.at(i).flushed_end_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected level_page_range_array_", KR(ret), K(fd_), K(level_page_range_array_), K(i)); + } + } + return ret; +} + +//total_need_flush_rightmost_page_num will not consider "is_writing_ = true" or "ObTmpFileTreeEvictType" +int ObSharedNothingTmpFileMetaTree::get_need_flush_page_num( + int64_t &total_need_flush_page_num, + int64_t &total_need_flush_rightmost_page_num) const +{ + int ret = OB_SUCCESS; + total_need_flush_page_num = 0; + total_need_flush_rightmost_page_num = 0; + SpinRLockGuard guard(lock_); + ARRAY_FOREACH_N(level_page_range_array_, level, level_cnt) { + const uint32_t start_page_id = level_page_range_array_[level].start_page_id_; + const uint32_t flushed_end_page_id = level_page_range_array_[level].flushed_end_page_id_; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + if (ObTmpFileGlobal::INVALID_PAGE_ID != start_page_id) { + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected end_page_id", KR(ret), K(fd_), K(level_page_range_array_), K(level)); + } else { + uint32_t cur_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int32_t level_page_index = level_page_range_array_[level].flushed_page_num_; + if (ObTmpFileGlobal::INVALID_PAGE_ID == flushed_end_page_id) { + cur_page_id = start_page_id; + } else { + cur_page_id = flushed_end_page_id; + level_page_index--; + } + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (wbp_->is_dirty(fd_, cur_page_id, page_key)) { + total_need_flush_page_num++; + if (end_page_id == cur_page_id) { + total_need_flush_rightmost_page_num++; + break; + } + } + if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_page_id, page_key, next_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + cur_page_id = next_page_id; + level_page_index++; + } + } + if (OB_SUCC(ret) && end_page_id != cur_page_id && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected cur_page_id", KR(ret), K(fd_), K(cur_page_id), + K(level_page_range_array_), K(level)); + } + } + } + } + return ret; +} + +//total_need_evict_rightmost_page_num will not consider "is_writing_ = true" or "ObTmpFileTreeEvictType" +int ObSharedNothingTmpFileMetaTree::get_need_evict_page_num( + int64_t &total_need_evict_page_num, + int64_t &total_need_evict_rightmost_page_num) const +{ + int ret = OB_SUCCESS; + total_need_evict_page_num = 0; + total_need_evict_rightmost_page_num = 0; + SpinRLockGuard guard(lock_); + ARRAY_FOREACH_N(level_page_range_array_, level, level_cnt) { + const uint32_t start_page_id = level_page_range_array_[level].start_page_id_; + const uint32_t flushed_end_page_id = level_page_range_array_[level].flushed_end_page_id_; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + if (ObTmpFileGlobal::INVALID_PAGE_ID != flushed_end_page_id) { + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == start_page_id + || ObTmpFileGlobal::INVALID_PAGE_ID == end_page_id)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected start_page_id or end_page_id", KR(ret), K(fd_), K(level_page_range_array_), K(level)); + } else { + uint32_t cur_page_id = start_page_id; + int32_t level_page_index = level_page_range_array_[level].evicted_page_num_; + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (OB_UNLIKELY(flushed_end_page_id != cur_page_id && !wbp_->is_cached(fd_, cur_page_id, page_key))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected cur_page_id", KR(ret), K(fd_), K(cur_page_id), K(flushed_end_page_id)); + } else if (!wbp_->is_cached(fd_, cur_page_id, page_key)) { + break; + } else if (FALSE_IT(total_need_evict_page_num++)) { + } else if (end_page_id == cur_page_id && FALSE_IT(total_need_evict_rightmost_page_num++)) { + } else if (flushed_end_page_id == cur_page_id) { + break; + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_page_id, page_key, next_page_id))) { + STORAGE_LOG(WARN, "fail to get next meta page id", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + cur_page_id = next_page_id; + level_page_index++; + } + } + if (OB_SUCC(ret) && flushed_end_page_id != cur_page_id) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected cur_page_id", KR(ret), K(fd_), K(cur_page_id), + K(level_page_range_array_), K(level)); + } + } + } + } + return ret; +} + +//NOTE: 这个页被取出来时,并没有加锁,到时候可能需要为write_cache的读写单独弄一把锁 +int ObSharedNothingTmpFileMetaTree::get_page_( + const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + char *&page_buff, + ObTmpPageValueHandle &p_handle) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!page_info.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(page_info)); + } else { + if (!is_page_in_write_cache(page_info)) { + ObTmpPageCacheKey key(page_info.block_index_, page_info.physical_page_id_, MTL_ID()); + if (OB_SUCC(ObTmpPageCache::get_instance().get_page(key, p_handle))) { + page_buff = p_handle.value_->get_buffer(); + } else if (OB_ENTRY_NOT_EXIST != ret) { + STORAGE_LOG(WARN, "fail to read from read_cache", KR(ret), K(fd_), K(key)); + } else { + ret = OB_SUCCESS; + if (OB_FAIL(ObTmpPageCache::get_instance().load_page(key, callback_allocator_, p_handle))) { + STORAGE_LOG(WARN, "fail to load page from disk", KR(ret), K(fd_), K(key)); + } else { + page_buff = p_handle.value_->get_buffer(); + } + } + if (FAILEDx(check_page_(page_buff))) { + STORAGE_LOG(WARN, "the page is invalid or corrupted", KR(ret), K(fd_), KP(page_buff)); + } + } else { + //still in write cache + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(page_info.page_level_, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, page_info.buffer_page_id_, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(page_info), K(page_key)); + } + } + if (OB_SUCC(ret) && OB_ISNULL(page_buff)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_buff", KR(ret), K(fd_), KP(page_buff)); + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::check_page_(const char* const page_buff) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + const uint64_t checksum = ob_crc64(page_buff + PAGE_HEADER_SIZE, ObTmpFileGlobal::PAGE_SIZE - PAGE_HEADER_SIZE); + if (OB_UNLIKELY(PAGE_MAGIC_NUM != page_header.magic_number_ + || checksum != page_header.checksum_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "checksum or magic_number is not equal, page is invalid", KR(ret), K(fd_), + KP(page_buff), K(page_header), K(checksum), K(PAGE_MAGIC_NUM)); + } + } + return ret; +} + +//get page and put into write cache (if not exists) +//put page_id into level_page_range_array_ +int ObSharedNothingTmpFileMetaTree::cache_page_for_write_( + const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!page_info.is_valid() + || page_info.page_level_ >= level_page_range_array_.count() + || (ObTmpFileGlobal::INVALID_PAGE_ID != parent_page_id + && page_info.page_level_ + 1 >= level_page_range_array_.count()))) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), K(page_info), K(level_page_range_array_), K(parent_page_id)); + } else { + if (!is_page_in_write_cache(page_info)) { + uint32_t new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + char *new_page_buff = NULL; + int32_t level_page_index = level_page_range_array_[page_info.page_level_].evicted_page_num_ - 1; + ObTmpFilePageUniqKey page_key(page_info.page_level_, level_page_index); + if (OB_UNLIKELY(!is_page_flushed(page_info) + || 0 < level_page_range_array_[page_info.page_level_].cached_page_num_ + || 0 > level_page_index + || ObTmpFileGlobal::INVALID_PAGE_ID != level_page_range_array_[page_info.page_level_].end_page_id_)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_info", KR(ret), K(fd_), K(page_info), K(level_page_range_array_)); + } else if (OB_FAIL(wbp_->alloc_page(fd_, page_key, new_page_id, new_page_buff))) { + STORAGE_LOG(WARN, "fail to alloc meta page", KR(ret), K(fd_), K(page_info), K(level_page_range_array_)); + } else if (OB_ISNULL(new_page_buff)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected null page buff", KR(ret), K(fd_), KP(new_page_buff)); + } else if (OB_FAIL(wbp_->notify_load(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load for meta", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } else { + ObTmpPageValueHandle p_handle; + ObTmpPageCacheKey key(page_info.block_index_, page_info.physical_page_id_, MTL_ID()); + if (OB_SUCC(ObTmpPageCache::get_instance().get_page(key, p_handle))) { + MEMCPY(new_page_buff, p_handle.value_->get_buffer(), ObTmpFileGlobal::PAGE_SIZE); + } else if (OB_ENTRY_NOT_EXIST != ret) { + STORAGE_LOG(WARN, "fail to read from read_cache", KR(ret), K(fd_), K(key)); + } else { + ret = OB_SUCCESS; + p_handle.reset(); + if (OB_FAIL(ObTmpPageCache::get_instance().load_page(key, callback_allocator_, p_handle))) { + STORAGE_LOG(WARN, "fail to load page from disk", KR(ret), K(fd_), K(key)); + } else { + MEMCPY(new_page_buff, p_handle.value_->get_buffer(), ObTmpFileGlobal::PAGE_SIZE); + } + } + if (FAILEDx(check_page_(new_page_buff))) { + STORAGE_LOG(WARN, "the page is invalid or corrupted", KR(ret), K(fd_), KP(new_page_buff)); + } + if (OB_SUCC(ret)) { + //change page state to cached + if (OB_FAIL(wbp_->notify_load_succ(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load succ for meta", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } + } else { + int tmp_ret = OB_SUCCESS; + //change page state to invalid + if (OB_TMP_FAIL(wbp_->notify_load_fail(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify load fail for meta", KR(tmp_ret), K(fd_), K(new_page_id), K(page_key)); + } + } + p_handle.reset(); + } + if (OB_SUCC(ret)) { + int64_t origin_block_index = page_info.block_index_; + int16_t origin_physical_page_id = page_info.physical_page_id_; + page_info.buffer_page_id_ = new_page_id; + page_info.block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + page_info.physical_page_id_ = -1; + if (ObTmpFileGlobal::INVALID_PAGE_ID == parent_page_id) { + root_item_ = page_info; + } else { + char *parent_page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObSharedNothingTmpFileTreePageHeader page_header; + int32_t parent_level_page_index = level_page_range_array_[page_info.page_level_ + 1].evicted_page_num_ + + level_page_range_array_[page_info.page_level_ + 1].cached_page_num_ - 1; + ObTmpFilePageUniqKey parent_page_offset(page_info.page_level_ + 1, parent_level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, parent_page_id, parent_page_offset, parent_page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(parent_page_id), K(parent_page_offset)); + } else if (OB_FAIL(read_page_header_(parent_page_buff, page_header))) { + STORAGE_LOG(WARN, "fail to read page header", KR(ret), K(fd_), KP(parent_page_buff)); + } else if (OB_FAIL(rewrite_item_(parent_page_buff, page_header.item_num_ - 1, page_info))) { + STORAGE_LOG(WARN, "fail to rewrite item", KR(ret), K(fd_), K(page_header), K(page_info), KP(parent_page_buff)); + } else if (OB_FAIL(wbp_->notify_dirty(fd_, parent_page_id, parent_page_offset))) { + STORAGE_LOG(WARN, "fail to notify dirty for meta", KR(ret), K(fd_), K(parent_page_id), K(parent_page_offset)); + } + } + if (OB_SUCC(ret)) { + int16_t page_level = page_info.page_level_; + if (OB_FAIL(wbp_->notify_dirty(fd_, new_page_id, page_key))) { + STORAGE_LOG(WARN, "fail to notify dirty", KR(ret), K(fd_), K(new_page_id), K(page_key)); + } else if (OB_FAIL(release_tmp_file_page_(origin_block_index, origin_physical_page_id, 1))) { + STORAGE_LOG(WARN, "fail to release tmp file page", KR(ret), K(fd_), K(origin_block_index), K(origin_physical_page_id)); + } else { + level_page_range_array_[page_level].start_page_id_ = new_page_id; + level_page_range_array_[page_level].end_page_id_ = new_page_id; + level_page_range_array_[page_level].cached_page_num_++; + level_page_range_array_[page_level].evicted_page_num_--; + level_page_range_array_[page_level].flushed_page_num_--; + } + } + } else if (ObTmpFileGlobal::INVALID_PAGE_ID != new_page_id) { //fail + uint32_t unused_next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(wbp_->free_page(fd_, new_page_id, page_key, unused_next_page_id))) { + STORAGE_LOG(WARN, "fail to free meta page", KR(tmp_ret), K(fd_), K(new_page_id), K(page_key)); + } + } + STORAGE_LOG(INFO, "load page to write cache", KR(ret), K(fd_), K(page_info), K(page_key)); + } else { + //still in write cache + if (!is_page_in_write_cache(page_info)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected page_info", KR(ret), K(fd_), K(page_info)); + } + } + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::init_page_header_( + char* page_buff, + const int16_t page_level) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header; + page_header.item_num_ = 0; + page_header.page_level_ = page_level; + page_header.magic_number_ = PAGE_MAGIC_NUM; + MEMCPY(page_buff, &page_header, PAGE_HEADER_SIZE); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::read_page_header_( + const char* page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(NULL == page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + } + return ret; +} + +int ObSharedNothingTmpFileMetaTree::remove_page_item_from_tail_( + char* page_buff, + const int16_t remove_item_num) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(NULL == page_buff + || 0 > remove_item_num)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff), K(remove_item_num)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + page_header.item_num_ -= remove_item_num; + if (OB_UNLIKELY(page_header.item_num_ < 0)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(fd_), K(page_header)); + } else { + MEMCPY(page_buff, &page_header, PAGE_HEADER_SIZE); + } + } + return ret; +} + +template +int ObSharedNothingTmpFileMetaTree::read_item_( + const char* page_buff, + const int64_t target_virtual_page_id, + int16_t &item_index, + ItemType &item) +{ + int ret = OB_SUCCESS; + item_index = -1; + item.reset(); + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + int16_t item_num = page_header.item_num_; + if (OB_UNLIKELY(NULL == page_buff + || target_virtual_page_id < 0)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff), K(target_virtual_page_id)); + } else if (OB_UNLIKELY(PAGE_HEADER_SIZE + item_num * sizeof(item) > ObTmpFileGlobal::PAGE_SIZE + || item_num > MAX_PAGE_ITEM_COUNT)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num", KR(ret), K(fd_), K(page_header)); + } else { + int16_t left = 0; + int16_t right = item_num - 1; + const char *items_buff = page_buff + PAGE_HEADER_SIZE; + //find the last index less than or equal to the target_virtual_page_id + while (left <= right) { + int16_t mid = (left + right) / 2; + ItemType mid_item = *((ItemType *)(items_buff + mid * sizeof(item))); + if (OB_UNLIKELY(!mid_item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected mid item", KR(ret), K(fd_), K(mid_item)); + } else if (mid_item.virtual_page_id_ <= target_virtual_page_id) { + left = mid + 1; + } else { + right = mid - 1; + } + } + if (OB_UNLIKELY(right < 0)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected dichotomy result", KR(ret), K(fd_), K(right)); + } else { + item_index = right; + item = *((ItemType *)(items_buff + item_index * sizeof(item))); + if (OB_UNLIKELY(!item.is_valid())) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item", KR(ret), K(fd_), K(item)); + } else if (item.virtual_page_id_ > target_virtual_page_id) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected virtual_page_id", KR(ret), K(fd_), K(item), K(target_virtual_page_id)); + } + } + } + if (OB_FAIL(ret)) { + //print page content + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + ItemType first_item; + ItemType last_item; + read_page_simple_content_(page_buff, tmp_page_header, first_item, last_item); + STORAGE_LOG(WARN, "fail to read item, dump tree page", KR(ret), K(fd_), KP(page_buff), K(target_virtual_page_id), + K(tmp_page_header), K(first_item), K(last_item)); + } + return ret; +} + +// XXX 增加 page 物理范围校验; +template +int ObSharedNothingTmpFileMetaTree::read_item_( + const char* page_buff, + const int16_t item_index, + ItemType &item) +{ + int ret = OB_SUCCESS; + item.reset(); + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + if (OB_UNLIKELY(NULL == page_buff + || item_index < 0 + || item_index >= page_header.item_num_ + || PAGE_HEADER_SIZE + page_header.item_num_ * sizeof(item) > ObTmpFileGlobal::PAGE_SIZE + || PAGE_HEADER_SIZE + (item_index + 1) * sizeof(item) > ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff), K(item_index), K(page_header)); + } else { + const char *items_buff = page_buff + PAGE_HEADER_SIZE; + item = *((ItemType *)(items_buff + item_index * sizeof(item))); + } + return ret; +} + +template +int ObSharedNothingTmpFileMetaTree::rewrite_item_( + char* page_buff, + const int16_t item_index, + const ItemType &item) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(NULL == page_buff + || item_index < 0 + || PAGE_HEADER_SIZE + (item_index + 1) * sizeof(item) > ObTmpFileGlobal::PAGE_SIZE + || !item.is_valid())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff), K(item_index), K(item)); + } else { + char *items_buff = page_buff + PAGE_HEADER_SIZE; + MEMCPY(items_buff + item_index * sizeof(item), &item, sizeof(item)); + } + return ret; +} + +template +int ObSharedNothingTmpFileMetaTree::write_items_( + char* page_buff, + const ObIArray &items, + const int64_t begin_index, + int16_t &count) +{ + int ret = OB_SUCCESS; + count = 0; + if (OB_UNLIKELY(NULL == page_buff + || items.empty() + || begin_index < 0 + || begin_index >= items.count())) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff), K(items), K(begin_index)); + } else { + ObSharedNothingTmpFileTreePageHeader page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + char *items_buff = page_buff + PAGE_HEADER_SIZE; + int16_t item_index = page_header.item_num_; + for (int64_t i = begin_index; i < items.count() + && PAGE_HEADER_SIZE + (item_index + 1) * sizeof(items.at(i)) <= ObTmpFileGlobal::PAGE_SIZE && item_index < MAX_PAGE_ITEM_COUNT; i++) { + MEMCPY(items_buff + item_index * sizeof(items.at(i)), &items.at(i), sizeof(items.at(i))); + item_index++; + count++; + } + page_header.item_num_ = item_index; + MEMCPY(page_buff, &page_header, PAGE_HEADER_SIZE); + } + return ret; +} + +void ObSharedNothingTmpFileMetaTree::print_meta_tree_overview_info() +{ + SpinRLockGuard guard(lock_); + STORAGE_LOG(INFO, "dump meta tree", KPC(this)); +} + +void ObSharedNothingTmpFileMetaTree::print_meta_tree_total_info() +{ + int ret = OB_SUCCESS; + ObArray data_items; + ObArray meta_items; + SpinRLockGuard guard(lock_); + STORAGE_LOG(INFO, "dump meta tree", KPC(this)); + ARRAY_FOREACH_N(level_page_range_array_, level, level_cnt) { + const uint32_t start_page_id = level_page_range_array_[level].start_page_id_; + const uint32_t end_page_id = level_page_range_array_[level].end_page_id_; + if (ObTmpFileGlobal::INVALID_PAGE_ID != start_page_id) { + uint32_t cur_page_id = start_page_id; + int32_t level_page_index = level_page_range_array_[level].evicted_page_num_; + while (OB_SUCC(ret) + && ObTmpFileGlobal::INVALID_PAGE_ID != cur_page_id) { + char *page_buff = NULL; + uint32_t next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageUniqKey page_key(level, level_page_index); + if (OB_FAIL(wbp_->read_page(fd_, cur_page_id, page_key, page_buff, next_page_id))) { + STORAGE_LOG(WARN, "fail to read from write cache", KR(ret), K(fd_), K(cur_page_id), K(page_key)); + } else { + ObSharedNothingTmpFileTreePageHeader tmp_page_header; + if (0 == level) { + data_items.reset(); + read_page_content_(page_buff, tmp_page_header, data_items); + STORAGE_LOG(INFO, "dump cached leaf page", KR(ret), K(fd_), KP(page_buff), K(cur_page_id), K(page_key), K(tmp_page_header), K(data_items)); + } else { + meta_items.reset(); + read_page_content_(page_buff, tmp_page_header, meta_items); + STORAGE_LOG(INFO, "dump cached internal page", KR(ret), K(fd_), KP(page_buff), K(cur_page_id), K(page_key), K(tmp_page_header), K(meta_items)); + } + } + cur_page_id = next_page_id; + level_page_index++; + } + } + } +} + +template +void ObSharedNothingTmpFileMetaTree::read_page_content_( + const char *page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header, + ObIArray &items) +{ + int ret = OB_SUCCESS; + items.reset(); + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + const char *items_buff = page_buff + PAGE_HEADER_SIZE; + int16_t index = 0; + page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + while (OB_SUCC(ret) + && index < page_header.item_num_ + && PAGE_HEADER_SIZE + (index + 1) * sizeof(ItemType) <= ObTmpFileGlobal::PAGE_SIZE) { + ItemType item = *((ItemType *)(items_buff + index * sizeof(ItemType))); + if (OB_FAIL(items.push_back(item))) { + STORAGE_LOG(WARN, "fail to push back", KR(ret), K(fd_), K(item)); + } else { + index++; + } + } + } +} + +template +void ObSharedNothingTmpFileMetaTree::read_page_simple_content_( + const char *page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header, + ItemType &first_item, + ItemType &last_item) +{ + int ret = OB_SUCCESS; + first_item.reset(); + last_item.reset(); + if (OB_ISNULL(page_buff)) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "invalid argument", KR(ret), K(fd_), KP(page_buff)); + } else { + const char *items_buff = page_buff + PAGE_HEADER_SIZE; + page_header = *((ObSharedNothingTmpFileTreePageHeader *)(page_buff)); + if (OB_UNLIKELY(0 > page_header.item_num_ + || PAGE_HEADER_SIZE + page_header.item_num_ * sizeof(ItemType) > ObTmpFileGlobal::PAGE_SIZE)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unexpected item_num_", KR(ret), K(fd_), KP(page_buff), K(page_header)); + } else if (0 == page_header.item_num_) { + } else { + first_item = *((ItemType *)(items_buff)); + last_item = *((ItemType *)(items_buff + (page_header.item_num_ - 1) * sizeof(ItemType))); + } + } +} + +} +} diff --git a/src/storage/tmp_file/ob_tmp_file_meta_tree.h b/src/storage/tmp_file/ob_tmp_file_meta_tree.h new file mode 100644 index 0000000000..4abc7edad5 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_meta_tree.h @@ -0,0 +1,537 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_META_TREE_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_META_TREE_H_ + +#include "deps/oblib/src/lib/container/ob_se_array.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "storage/tmp_file/ob_tmp_file_global.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +class ObTmpPageValueHandle; + +struct ObSharedNothingTmpFileMetaItem +{ +public: + ObSharedNothingTmpFileMetaItem() : + page_level_(-1), + physical_page_id_(-1), + buffer_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + block_index_(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX), + virtual_page_id_(-1) {} + bool is_valid() const + { + return page_level_ >= 0 + && virtual_page_id_ >= 0; + } + void reset() + { + page_level_ = -1; + physical_page_id_ = -1; + buffer_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } + int16_t page_level_; // child page level + int16_t physical_page_id_; // child page 在宏块内的page id + uint32_t buffer_page_id_; // child page 在写缓存中的page id + int64_t block_index_; // child page 对应的宏块block index + int64_t virtual_page_id_; // child page 对应的文件逻辑页号最小值(即索引) + TO_STRING_KV(K(page_level_), K(buffer_page_id_), K(physical_page_id_), + K(block_index_), K(virtual_page_id_)); +}; // 24B + +struct ObSharedNothingTmpFileDataItem +{ + ObSharedNothingTmpFileDataItem() : + block_index_(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX), + physical_page_id_(-1), + physical_page_num_(0), + virtual_page_id_(-1) {} + bool operator!=(const ObSharedNothingTmpFileDataItem &other) const + { + return other.block_index_ != block_index_ + || other.physical_page_id_ != physical_page_id_ + || other.physical_page_num_ != physical_page_num_ + || other.virtual_page_id_ != virtual_page_id_; + } + bool is_valid() const + { + return ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX != block_index_ + && physical_page_id_ >= 0 + && physical_page_num_ > 0 //data item must be cleaned up if physical_page_num_ = 0 + && virtual_page_id_ >= 0; + } + void reset() + { + block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + physical_page_id_ = -1; + physical_page_num_ = 0; + virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + } + int64_t block_index_; // 刷盘data extent所在的宏块block index + int16_t physical_page_id_; // 刷盘data extent在宏块内的起始page id + int16_t physical_page_num_; // 刷盘data extent在宏块内的连续page数量 + int64_t virtual_page_id_; // 刷盘data extent在整个文件中逻辑页号(偏移位置) + TO_STRING_KV(K(block_index_), K(physical_page_id_), + K(physical_page_num_), K(virtual_page_id_)); +}; // 24B + +struct ObSharedNothingTmpFileTreePageHeader +{ + ObSharedNothingTmpFileTreePageHeader() : + page_level_(-1), + item_num_(0), + magic_number_(0), + checksum_(0) {} + int16_t page_level_; + int16_t item_num_; + uint32_t magic_number_; + uint64_t checksum_; + TO_STRING_KV(K(page_level_), K(item_num_), K(magic_number_), K(checksum_)); +}; + +struct ObTmpFileTreeIOInfo +{ + ObTmpFileTreeIOInfo() :tree_epoch_(-1), block_index_(ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX), + flush_start_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flush_start_level_page_index_(-1), + flush_end_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + physical_start_page_id_(-1), + flush_nums_(0), + page_level_(-1) {} + void reset() + { + tree_epoch_ = -1; + block_index_ = ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX; + flush_start_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + flush_start_level_page_index_ = -1; + flush_end_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + physical_start_page_id_ = -1; + flush_nums_ = 0; + page_level_ = -1; + } + bool is_valid() const + { + return 0 <= tree_epoch_ + && ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX != block_index_ + && ObTmpFileGlobal::INVALID_PAGE_ID != flush_start_page_id_ + && 0 <= flush_start_level_page_index_ + && ObTmpFileGlobal::INVALID_PAGE_ID != flush_end_page_id_ + && physical_start_page_id_ >= 0 + && flush_nums_ > 0 + && page_level_ >= 0; + } + int64_t tree_epoch_; + int64_t block_index_; //刷脏页所在的block index + uint32_t flush_start_page_id_; //刷脏起始页在写缓存中的page_id + int32_t flush_start_level_page_index_; //刷脏起始页在其所在层的 page_index + uint32_t flush_end_page_id_; //刷脏最后一个页在写缓存中的page_id + int16_t physical_start_page_id_; //这批页在block中的start page id(物理偏移) + int16_t flush_nums_; //刷脏的页数 + int16_t page_level_; //刷脏的页所属的元数据树层次 + TO_STRING_KV(K(tree_epoch_), K(block_index_), K(flush_start_page_id_), + K(flush_start_level_page_index_), K(flush_end_page_id_), + K(physical_start_page_id_), K(flush_nums_), K(page_level_)); +}; + +struct ObTmpFileTreeFlushContext +{ + ObTmpFileTreeFlushContext() : tree_epoch_(-1), + is_meta_reach_end_(false), + last_flush_level_(-1), + last_flush_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + last_flush_page_index_in_level_(-1) {} + void reset() + { + tree_epoch_ = -1; + is_meta_reach_end_ = false; + last_flush_level_ = -1; + last_flush_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + last_flush_page_index_in_level_ = -1; + } + bool is_valid() const + { + return tree_epoch_ >= 0 + && last_flush_level_ >= 0 + && ObTmpFileGlobal::INVALID_PAGE_ID != last_flush_page_id_ + && last_flush_page_index_in_level_ >= 0; + } + int64_t tree_epoch_; + bool is_meta_reach_end_; + int16_t last_flush_level_; + uint32_t last_flush_page_id_; + int32_t last_flush_page_index_in_level_; + TO_STRING_KV(K(tree_epoch_), K(is_meta_reach_end_), K(last_flush_level_), K(last_flush_page_id_), + K(last_flush_page_index_in_level_)); +}; + +enum ObTmpFileTreeEvictType : int16_t +{ + INVALID = -1, + FULL = 0, //flush all pages + MAJOR, //flush all pages except rightmost page of each level +}; + +class ObSharedNothingTmpFileMetaTree +{ +public: + struct LevelPageRangeInfo + { + LevelPageRangeInfo() : start_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + flushed_end_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + end_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + cached_page_num_(0), + flushed_page_num_(0), + evicted_page_num_(0) {} + LevelPageRangeInfo(const uint32_t start_page_id, + const uint32_t flushed_page_id, + const uint32_t end_page_id, + const int32_t cached_page_num, + const int32_t flushed_page_num, + const int32_t evicted_page_num) : + start_page_id_(start_page_id), + flushed_end_page_id_(flushed_page_id), + end_page_id_(end_page_id), + cached_page_num_(cached_page_num), + flushed_page_num_(flushed_page_num), + evicted_page_num_(evicted_page_num) {} + uint32_t start_page_id_; + uint32_t flushed_end_page_id_; + uint32_t end_page_id_; + //Rules: + // cached_page_num_ + evicted_page_num_ = level total page num + // flushed_page_num_ >= evicted_page_num_ + // flushed_pages must contail evicted_pages, but maybe some cached_pages. + // cached_pages -> [start_page_id_, end_page_id_] + // flushed_pages -> [start_page_id_, flushed_end_page_id_] or none(depnds on whether flushed_end_page_id_ is valid or not) + // They do not contain duplicate pages, even if the page is cached/flushed/evicted multiple times. + int32_t cached_page_num_; + int32_t flushed_page_num_; + int32_t evicted_page_num_; + TO_STRING_KV(K(start_page_id_), K(flushed_end_page_id_), K(end_page_id_), + K(cached_page_num_), K(flushed_page_num_), K(evicted_page_num_)); + }; + + struct BacktraceNode + { + BacktraceNode() : page_meta_info_(), + level_page_index_(-1), + prev_item_index_(-1) {} + BacktraceNode(const ObSharedNothingTmpFileMetaItem &item, + const int32_t level_page_index, + const int16_t prev_item_index) + : page_meta_info_(item), + level_page_index_(level_page_index), + prev_item_index_(prev_item_index) {} + bool is_valid() const + { + return page_meta_info_.is_valid() + && 0 <= level_page_index_; + } + //corresponding to a page (represent meta info of the page) + ObSharedNothingTmpFileMetaItem page_meta_info_; + //the page index in its level + int32_t level_page_index_; + //prev processed item index on the page + int16_t prev_item_index_; + TO_STRING_KV(K(page_meta_info_), K(level_page_index_), K(prev_item_index_)); + }; + + struct LastTruncateLeafInfo + { + LastTruncateLeafInfo() : page_index_in_leaf_level_(-1), + item_index_in_page_(-1), + release_to_end_in_page_(false) {} + bool is_valid() const + { + return 0 <= page_index_in_leaf_level_ + && 0 <= item_index_in_page_; + } + void reset() + { + page_index_in_leaf_level_ = -1; + item_index_in_page_ = -1; + release_to_end_in_page_ = false; + } + int32_t page_index_in_leaf_level_; + int16_t item_index_in_page_; + bool release_to_end_in_page_; + TO_STRING_KV(K(page_index_in_leaf_level_), K(item_index_in_page_), K(release_to_end_in_page_)); + }; + + struct StatInfo + { + StatInfo() : meta_page_flushing_cnt_(0), + all_type_page_flush_cnt_(0), + all_type_flush_page_released_cnt_(0) {} + void reset() + { + meta_page_flushing_cnt_ = 0; + all_type_page_flush_cnt_ = 0; + all_type_flush_page_released_cnt_ = 0; + } + int64_t meta_page_flushing_cnt_; + //can contain the same page if page is flushed again + // contain total pages(meta and data) + int64_t all_type_page_flush_cnt_; + int64_t all_type_flush_page_released_cnt_; + TO_STRING_KV(K(meta_page_flushing_cnt_), K(all_type_page_flush_cnt_), K(all_type_flush_page_released_cnt_)); + }; + +public: + ObSharedNothingTmpFileMetaTree(); + ~ObSharedNothingTmpFileMetaTree(); + void reset(); +public: + int init(const int64_t fd, ObTmpWriteBufferPool *wbp, ObIAllocator *callback_allocator); + + //append write: We always write the rightmost page of the leaf layer + //It happens after a tmp file write request: + // data pages of a write request is persisted on disk + int prepare_for_insert_items(); + int insert_items(const common::ObIArray &data_items); + + //It happens when there is a tmp file read request + int search_data_items(const int64_t offset, + const int64_t read_size, + common::ObIArray &data_items); + + //It happens when there is a meta tree pages flush request: + // tmp file data and meta pages are aggregated to flush + int flush_meta_pages_for_block(const int64_t block_index, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeFlushContext &flush_context, + common::ObIArray &meta_io_array); + + //It happens after meta tree pages flush: + // meta pages is persisted on disk successfully + int update_after_flush(const common::ObIArray &meta_io_array); + + //It happens when there is a tmp file write request: + // a tmp file page that is not full need to be continued writing. + //last_data_item is the page info of the unfilled data page. + int prepare_for_write_tail(ObSharedNothingTmpFileDataItem &last_data_item); + // After the tail is written (the tail page exists in disk) + int finish_write_tail(const ObSharedNothingTmpFileDataItem &last_data_item, + const bool release_tail_in_disk); + + //It happens when there is a tmp file eviction + int evict_meta_pages(const int64_t expected_page_num, + const ObTmpFileTreeEvictType flush_type, + int64_t &actual_evict_page_num); + + //it happens when there is a tmp file clearing + int clear(const int64_t last_truncate_offset, const int64_t total_file_size); + + //it happens when there is a need to truncate a tmp file + int truncate(const int64_t last_truncate_offset, const int64_t end_truncate_offset); + + //get the num of tree pages that need to be flushed + int get_need_flush_page_num(int64_t &total_need_flush_page_num, + int64_t &total_need_flush_rightmost_page_num) const; + //get the num of tree pages that need to be evicted + int get_need_evict_page_num(int64_t &total_need_evict_page_num, + int64_t &total_need_evict_rightmost_page_num) const; + //NOTE: the following function is called externally. + // It needs to be called internally without holding lock_ (avoid deadlock). + void print_meta_tree_overview_info(); + //NOTE: need control print frequency. + void print_meta_tree_total_info(); + +private: + int modify_meta_items_at_parent_level_(const ObTmpFileTreeIOInfo &meta_io, + const int32_t start_page_index_in_level); + int truncate_array_(const int64_t end_offset); + int search_data_items_from_array_(const int64_t end_offset, + int64_t &cur_offset, + common::ObIArray &data_items); + int get_items_of_internal_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int16_t cur_item_index, + const int64_t end_offset, + bool &is_last_item, + ObIArray &meta_items); + int backtrace_truncate_tree_(const int64_t end_offset, + ObIArray &search_path); + int backtrace_search_data_items_(const int64_t end_offset, + int64_t &offset, + ObIArray &search_path, + ObIArray &data_items); + int finish_insert_(const int return_ret, + ObIArray &level_origin_page_write_counts, + ObIArray> &level_new_pages); + int try_to_insert_items_to_array_(const ObIArray &data_items, + ObIArray> &level_new_pages); + int release_items_of_leaf_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int64_t end_offset, + const int16_t begin_release_index, + bool &release_last_item); + int get_items_of_leaf_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + const int64_t end_offset, + const bool need_find_index, + int64_t &cur_offset, + common::ObIArray &data_items); + virtual int release_meta_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t page_index_in_level); + virtual int release_tmp_file_page_(const int64_t block_index, + const int64_t begin_page_id, const int64_t page_num); + int get_rightmost_leaf_page_for_write_(ObSharedNothingTmpFileMetaItem &page_info); + int get_last_item_of_internal_page_(const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info, + ObSharedNothingTmpFileMetaItem &last_meta_item); + int try_to_fill_rightmost_internal_page_(const ObIArray &meta_items, + const int16_t page_level, + int64_t &write_count, + ObIArray &level_origin_page_write_counts); + int try_to_fill_rightmost_leaf_page_(const ObIArray &data_items, + const ObIArray> &level_new_pages, + int64_t &write_count, + ObIArray &level_origin_page_write_counts); + int add_new_page_and_fill_items_at_leaf_(const ObIArray &data_items, + int64_t &write_count, + ObSharedNothingTmpFileMetaItem &meta_item, + ObIArray> &level_new_pages); + int add_new_page_and_fill_items_at_internal_(const ObIArray &meta_items, + const int16_t page_level, + int64_t &write_count, + ObSharedNothingTmpFileMetaItem &meta_item, + ObIArray> &level_new_pages); + int cascade_modification_at_internal_(const ObIArray &new_leaf_page_infos, + ObIArray &level_origin_page_write_counts, + ObIArray> &level_new_pages); + int modify_child_pages_location(char *page_buff); + int modify_meta_items_during_evict_(const ObIArray &evict_pages, + const int16_t level, + const int32_t start_page_index_in_level); + int flush_leaf_pages_(const uint32_t flush_start_page_id, + const int32_t start_page_index_in_level, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeIOInfo &meta_io_info); + int calc_and_set_page_checksum_(char* page_buff); + int flush_internal_pages_(const uint32_t flush_start_page_id, + const int16_t level, + const int32_t start_page_index_in_level, + const ObTmpFileTreeEvictType flush_type, + char *block_buff, + int64_t &write_offset, + ObTmpFileTreeIOInfo &meta_io_info); + int get_page_(const ObSharedNothingTmpFileMetaItem &page_info, + const int32_t level_page_index, + char *&page_buff, + ObTmpPageValueHandle &p_handle); + int check_page_(const char* const page_buff); + virtual int cache_page_for_write_(const uint32_t parent_page_id, + ObSharedNothingTmpFileMetaItem &page_info); + int init_page_header_(char* page_buff, + const int16_t page_level); + int read_page_header_(const char* page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header); + int remove_page_item_from_tail_(char* page_buff, + const int16_t remove_item_num); + int truncate_(const int64_t end_offset); + int calculate_truncate_index_path_(ObIArray> &item_index_arr); + template + int read_item_(const char* page_buff, + const int64_t target_virtual_page_id, + int16_t &item_index, + ItemType &item); + template + int read_item_(const char* page_buff, + const int16_t item_index, + ItemType &item); + template + int rewrite_item_(char* page_buff, + const int16_t item_index, + const ItemType &item) __attribute__((noinline)); + template + int write_items_(char* page_buff, + const ObIArray &items, + const int64_t begin_index, + int16_t &write_count); + int check_tree_is_empty_(); + template + void read_page_content_(const char *page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header, + ObIArray &data_items); + template + void read_page_simple_content_(const char *page_buff, + ObSharedNothingTmpFileTreePageHeader &page_header, + ItemType &first_item, + ItemType &last_item); + inline bool is_page_in_write_cache(const ObSharedNothingTmpFileMetaItem &meta_item) + { + return ObTmpFileGlobal::INVALID_PAGE_ID != meta_item.buffer_page_id_; + } + inline bool is_page_flushed(const ObSharedNothingTmpFileMetaItem &meta_item) + { + return ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX != meta_item.block_index_ + && 0 <= meta_item.physical_page_id_; + } + +private: + //only for unittest + static void set_max_array_item_cnt(const int16_t cnt) + { + MAX_DATA_ITEM_ARRAY_COUNT = cnt; + } + static void set_max_page_item_cnt(const int16_t cnt) + { + MAX_PAGE_ITEM_COUNT = cnt; + } +private: + static const uint16_t PAGE_MAGIC_NUM; + static const int16_t PAGE_HEADER_SIZE; + static int16_t MAX_DATA_ITEM_ARRAY_COUNT; + //only for unittest + static int16_t MAX_PAGE_ITEM_COUNT; + +private: + bool is_inited_; + int64_t fd_; + ObTmpWriteBufferPool *wbp_; + ObIAllocator *callback_allocator_; + int64_t tree_epoch_; + ObSharedNothingTmpFileMetaItem root_item_; + //When the tmp file writes less data, we can use an array instead of a tree to store metadata + common::ObSEArray data_item_array_; + //level page ranges in write cache + common::ObSEArray level_page_range_array_; + //Only writing the rightmost pages + //Used to prevent the rightmost pages from being evicted + bool is_writing_; + common::SpinRWLock lock_; + LastTruncateLeafInfo last_truncate_leaf_info_; + int64_t released_offset_; +public: + StatInfo stat_info_; + //NOTE: without protection of lock_, we do not recommend using K(meta_tree_) externally. + TO_STRING_KV(K(fd_), K(tree_epoch_), K(root_item_), + K(data_item_array_), K(level_page_range_array_), K(is_writing_), + K(last_truncate_leaf_info_), K(released_offset_), K(stat_info_)); +}; + + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_FILE_META_TREE_H_ \ No newline at end of file diff --git a/src/storage/tmp_file/ob_tmp_file_page_cache_controller.cpp b/src/storage/tmp_file/ob_tmp_file_page_cache_controller.cpp new file mode 100644 index 0000000000..802c1633e6 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_page_cache_controller.cpp @@ -0,0 +1,161 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_page_cache_controller.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +int ObTmpFilePageCacheController::init(ObTenantTmpFileManager &file_mgr) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObTmpFilePageCacheController init twice"); + } else if (OB_FAIL(task_allocator_.init(lib::ObMallocAllocator::get_instance(), + OB_MALLOC_MIDDLE_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileCtl", ObCtxIds::DEFAULT_CTX_ID)))) { + STORAGE_LOG(WARN, "fail to init task allocator", KR(ret)); + } else if (OB_FAIL(flush_mgr_.init())) { + STORAGE_LOG(WARN, "fail to init flush task mgr", KR(ret)); + } else if (OB_FAIL(flush_priority_mgr_.init())) { + STORAGE_LOG(WARN, "fail to init flush priority mgr", KR(ret)); + } else if (OB_FAIL(write_buffer_pool_.init())) { + STORAGE_LOG(WARN, "fail to init write buffer pool", KR(ret)); + } else if (OB_FAIL(flush_tg_.init())) { + STORAGE_LOG(WARN, "fail to init flush thread", KR(ret)); + } else if (OB_FAIL(swap_tg_.init(file_mgr))) { + STORAGE_LOG(WARN, "fail to init swap thread", KR(ret)); + } else { + flush_all_data_ = false; + is_inited_ = true; + } + return ret; +} + +int ObTmpFilePageCacheController::start() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + STORAGE_LOG(WARN, "tmp file page cache controller is not inited"); + } else if (OB_FAIL(swap_tg_.start())) { + STORAGE_LOG(WARN, "fail to start swap thread", KR(ret)); + } + return ret; +} + +void ObTmpFilePageCacheController::stop() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + STORAGE_LOG(WARN, "tmp file page cache controller is not inited"); + } else { + // stop background threads should follow the order 'swap' -> 'flush' because 'swap' holds ref to 'flush' + swap_tg_.stop(); + } +} + +void ObTmpFilePageCacheController::wait() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + STORAGE_LOG(WARN, "tmp file page cache controller is not inited"); + } else { + swap_tg_.wait(); + } +} + +void ObTmpFilePageCacheController::destroy() +{ + swap_tg_.destroy(); + flush_tg_.destroy(); + task_allocator_.reset(); + write_buffer_pool_.destroy(); + flush_mgr_.destroy(); + evict_mgr_.destroy(); + flush_priority_mgr_.destroy(); + flush_all_data_ = false; + is_inited_ = false; +} + +int ObTmpFilePageCacheController::swap_job_enqueue_(ObTmpFileSwapJob *swap_job) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(swap_job)){ + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap job is null", KR(ret)); + } else if (!swap_job->is_valid()) { + ret = OB_INVALID_ARGUMENT; + STORAGE_LOG(WARN, "swap job is not valid", KR(ret), KPC(swap_job)); + } else if (OB_FAIL(swap_tg_.swap_job_enqueue(swap_job))) { + STORAGE_LOG(WARN, "fail to enqueue swap job", KR(ret), KP(swap_job)); + } + return ret; +} + +int ObTmpFilePageCacheController::free_swap_job_(ObTmpFileSwapJob *swap_job) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(swap_job)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap job is null", KR(ret)); + } else if (swap_job->is_inited() && !swap_job->is_finished()) { + ret = OB_EAGAIN; + STORAGE_LOG(ERROR, "swap job is not finished", KR(ret), KPC(swap_job)); + } else { + swap_job->~ObTmpFileSwapJob(); + task_allocator_.free(swap_job); + } + return ret; +} + +int ObTmpFilePageCacheController::invoke_swap_and_wait(int64_t expect_swap_size, int64_t timeout_ms) +{ + int ret = OB_SUCCESS; + + int64_t mem_limit = write_buffer_pool_.get_memory_limit(); + expect_swap_size = min(expect_swap_size, static_cast(0.2 * mem_limit)); + + void *task_buf = nullptr; + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_ISNULL(task_buf = task_allocator_.alloc(sizeof(ObTmpFileSwapJob)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + STORAGE_LOG(WARN, "fail to allocate memory for swap job", KR(ret)); + } else if (FALSE_IT(swap_job = new (task_buf) ObTmpFileSwapJob())) { + } else if (OB_FAIL(swap_job->init(expect_swap_size, timeout_ms))) { + STORAGE_LOG(WARN, "fail to init sync swap job", KR(ret), KPC(swap_job)); + } else if (OB_FAIL(swap_job_enqueue_(swap_job))) { + STORAGE_LOG(WARN, "fail to enqueue swap job", KR(ret), KPC(swap_job)); + } else { + swap_tg_.notify_doing_swap(); + if (OB_FAIL(swap_job->wait_swap_complete())) { + STORAGE_LOG(WARN, "fail to wait for swap job complete timeout", KR(ret)); + } + } + + if (OB_NOT_NULL(swap_job)) { + // reset swap job to set is_finished to false in case of failure to push into queue: + // otherwise job is not finished, but it will not be executed, so it will never become finished. + swap_job->reset(); + if (OB_FAIL(free_swap_job_(swap_job))) { + STORAGE_LOG(ERROR, "fail to free swap job", KR(ret)); + } + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_page_cache_controller.h b/src/storage/tmp_file/ob_tmp_file_page_cache_controller.h new file mode 100644 index 0000000000..93b10f108b --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_page_cache_controller.h @@ -0,0 +1,78 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_PAGE_CACHE_CONTROLLER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_PAGE_CACHE_CONTROLLER_H_ + +#include "storage/tmp_file/ob_tmp_file_thread_wrapper.h" +#include "storage/tmp_file/ob_tmp_file_flush_manager.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +class ObTmpFilePageCacheController +{ +public: + ObTmpFilePageCacheController(ObTmpFileBlockManager &tmp_file_block_manager) + : is_inited_(false), + flush_all_data_(false), + tmp_file_block_manager_(tmp_file_block_manager), + task_allocator_(), + write_buffer_pool_(), + flush_priority_mgr_(), + evict_mgr_(), + flush_mgr_(*this), + flush_tg_(write_buffer_pool_, flush_mgr_, task_allocator_, tmp_file_block_manager_), + swap_tg_(write_buffer_pool_, evict_mgr_, flush_tg_) + { + } + ~ObTmpFilePageCacheController() {} +public: + static const int64_t FLUSH_FAST_INTERVAL = 5; // 5ms + static const int64_t FLUSH_INTERVAL = 1000; // 1s + static const int64_t SWAP_FAST_INTERVAL = 5; // 5ms + static const int64_t SWAP_INTERVAL = 1000; // 2s + int init(ObTenantTmpFileManager &file_mgr); + int start(); + void stop(); + void wait(); + void destroy(); + ObIAllocator &get_task_allocator() { return task_allocator_; } + ObTmpWriteBufferPool &get_write_buffer_pool() { return write_buffer_pool_; } + ObTmpFileFlushManager &get_flush_task_mgr() { return flush_mgr_; } + ObTmpFileEvictionManager &get_eviction_manager() { return evict_mgr_; } + ObTmpFileFlushPriorityManager &get_flush_priority_mgr() { return flush_priority_mgr_; } + ObTmpFileBlockManager &get_tmp_file_block_manager() { return tmp_file_block_manager_; } + OB_INLINE bool is_flush_all_data() { return ATOMIC_LOAD(&flush_all_data_); } + int invoke_swap_and_wait(int64_t expect_swap_size, int64_t timeout_ms = ObTmpFileSwapJob::DEFAULT_TIMEOUT_MS); +private: + int swap_job_enqueue_(ObTmpFileSwapJob *swap_job); + int free_swap_job_(ObTmpFileSwapJob *swap_job); + DISALLOW_COPY_AND_ASSIGN(ObTmpFilePageCacheController); +private: + bool is_inited_; + bool flush_all_data_; // unit test only + ObTmpFileBlockManager &tmp_file_block_manager_; // ref to ObTmpFileBlockManager + ObFIFOAllocator task_allocator_; // used by flush_mgr_ to allocate flush tasks + ObTmpWriteBufferPool write_buffer_pool_; + ObTmpFileFlushPriorityManager flush_priority_mgr_; + ObTmpFileEvictionManager evict_mgr_; // maintain evict lists and evict pages from write buffer pool + ObTmpFileFlushManager flush_mgr_; // maintain flush lists and generate flush tasks + ObTmpFileFlushTG flush_tg_; // flush thread + ObTmpFileSwapTG swap_tg_; // swap thread +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_PAGE_CACHE_CONTROLLER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_thread_job.cpp b/src/storage/tmp_file/ob_tmp_file_thread_job.cpp new file mode 100644 index 0000000000..7c7d3c2916 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_thread_job.cpp @@ -0,0 +1,193 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "share/ob_errno.h" // KR +#include "storage/tmp_file/ob_tmp_file_thread_job.h" +#include "lib/utility/utility.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +int ObTmpFileSwapJob::init(int64_t expect_swap_size, uint32_t timeout_ms) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObTmpFileSwapJob init twice", KR(ret)); + } else if (timeout_ms <= 0) { + ret = OB_INVALID_ARGUMENT; + } else if (OB_FAIL(swap_cond_.init(ObWaitEventIds::NULL_EVENT))) { + STORAGE_LOG(WARN, "ObTmpFileSwapJob init cond failed", KR(ret)); + } else { + is_inited_ = true; + is_finished_ = false; + expect_swap_size_ = expect_swap_size; + timeout_ms_ = timeout_ms; + create_ts_ = ObTimeUtility::current_time(); + abs_timeout_ts_ = ObTimeUtility::current_time() + timeout_ms_ * 1000; + } + return ret; +} + +void ObTmpFileSwapJob::reset() +{ + is_inited_ = false; + is_finished_ = false; + expect_swap_size_ = 0; + timeout_ms_ = DEFAULT_TIMEOUT_MS; + create_ts_ = 0; + abs_timeout_ts_ = 0; + swap_cond_.destroy(); +} + +// waits for swap job to finish, loop inside until is_finished_ is true +int ObTmpFileSwapJob::wait_swap_complete() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapJob not init", KR(ret)); + } else { + bool is_finished = false; + ObThreadCondGuard guard(swap_cond_); + while (false == (is_finished = ATOMIC_LOAD(&is_finished_))) { + if (OB_FAIL(swap_cond_.wait())) { + if (OB_TIMEOUT == ret) { + STORAGE_LOG(WARN, "fail to wait swap job complete", KR(ret), K(is_finished), K(timeout_ms_)); + ret = OB_SUCCESS; + } else { + STORAGE_LOG(ERROR, "fail to wait swap job thread cond", KR(ret), K(is_finished), KPC(this)); + } + } + } + } + return ret; +} + +// set swap job is_finished, wake up threads that invoke swap job +int ObTmpFileSwapJob::signal_swap_complete() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapJob not init", KR(ret)); + } else { + ObThreadCondGuard guard(swap_cond_); + ATOMIC_SET(&is_finished_, true); + if (OB_FAIL(swap_cond_.signal())) { + STORAGE_LOG(WARN, "ObTmpFileSwapJob signal swap complete failed", KR(ret)); + } + } + return ret; +} + +void ObTmpFileFlushMonitor::reset() +{ + flush_task_cnt_ = 0; + total_flush_data_length_ = 0; + max_flush_data_length_ = -1; + min_flush_data_length_ = INT64_MAX; + f1_cnt_ = 0; + f2_cnt_ = 0; + f3_cnt_ = 0; + f4_cnt_ = 0; + f5_cnt_ = 0; +} + +void ObTmpFileFlushMonitor::print_statistics() +{ + int64_t flush_task_cnt = flush_task_cnt_; + int64_t avg_flush_data_len = total_flush_data_length_ / max(flush_task_cnt, 1); + int64_t max_flush_data_len = max_flush_data_length_; + int64_t min_flush_data_len = min_flush_data_length_ == INT64_MAX ? -1 : min_flush_data_length_; + int64_t f1_cnt = f1_cnt_; + int64_t f2_cnt = f2_cnt_; + int64_t f3_cnt = f3_cnt_; + int64_t f4_cnt = f4_cnt_; + int64_t f5_cnt = f5_cnt_; + STORAGE_LOG(INFO, "tmp file flush statistics", K(flush_task_cnt), K(avg_flush_data_len), + K(max_flush_data_len), K(min_flush_data_len), K(f1_cnt), K(f2_cnt), K(f3_cnt), K(f4_cnt), K(f5_cnt)); + reset(); +} + +void ObTmpFileFlushMonitor::record_flush_stage(const ObTmpFileGlobal::FlushCtxState flush_stage) +{ + switch(flush_stage) { + case ObTmpFileGlobal::FSM_F1: + f1_cnt_++; + break; + case ObTmpFileGlobal::FSM_F2: + f2_cnt_++; + break; + case ObTmpFileGlobal::FSM_F3: + f3_cnt_++; + break; + case ObTmpFileGlobal::FSM_F4: + f4_cnt_++; + break; + case ObTmpFileGlobal::FSM_F5: + f5_cnt_++; + break; + default: + break; + } +} + +void ObTmpFileFlushMonitor::record_flush_task(const int64_t data_length) +{ + flush_task_cnt_ += 1; + total_flush_data_length_ += data_length; + if (data_length > 0 && data_length > max_flush_data_length_) { + max_flush_data_length_ = data_length; + } + if (data_length > 0 && data_length < min_flush_data_length_) { + min_flush_data_length_ = data_length; + } +} + +void ObTmpFileSwapMonitor::reset() +{ + swap_task_cnt_ = 0; + swap_total_response_time_= 0; + swap_max_response_time_ = -1; + swap_min_response_time_ = INT64_MAX; +} + +void ObTmpFileSwapMonitor::print_statistics() +{ + int64_t swap_task_cnt = swap_task_cnt_; + int64_t avg_swap_response_time = swap_total_response_time_ / max(swap_task_cnt, 1); + int64_t max_swap_response_time = swap_max_response_time_; + int64_t min_swap_response_time = swap_min_response_time_ == INT64_MAX ? -1 : swap_min_response_time_; + STORAGE_LOG(INFO, "tmp file swap statistics", K(swap_task_cnt), + K(avg_swap_response_time), K(max_swap_response_time), K(min_swap_response_time)); + reset(); +} + +void ObTmpFileSwapMonitor::record_swap_response_time(const int64_t response_time) +{ + swap_task_cnt_ += 1; + swap_total_response_time_ += response_time; + if (response_time > swap_max_response_time_) { + swap_max_response_time_ = response_time; + } + if (response_time < swap_min_response_time_) { + swap_min_response_time_ = response_time; + } +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_thread_job.h b/src/storage/tmp_file/ob_tmp_file_thread_job.h new file mode 100644 index 0000000000..24a92914b1 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_thread_job.h @@ -0,0 +1,111 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_JOB_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_JOB_H_ + +#include "lib/lock/ob_thread_cond.h" +#include "lib/queue/ob_link_queue.h" +#include "lib/utility/ob_print_utils.h" +#include "storage/tmp_file/ob_tmp_file_global.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +class ObTmpFileSwapJob : public ObSpLinkQueue::Link +{ +public: + static const uint32_t DEFAULT_TIMEOUT_MS = 10 * 1000; + ObTmpFileSwapJob() + : is_inited_(false), + is_finished_(false), + timeout_ms_(DEFAULT_TIMEOUT_MS), + create_ts_(0), + abs_timeout_ts_(0), + expect_swap_size_(0), + swap_cond_() {} + ~ObTmpFileSwapJob() { reset(); } + int init(int64_t expect_swap_size, uint32_t timeout_ms = DEFAULT_TIMEOUT_MS); + void reset(); + int wait_swap_complete(); + int signal_swap_complete(); + OB_INLINE int64_t get_create_ts() const { return create_ts_; } + OB_INLINE int64_t get_abs_timeout_ts() const { return abs_timeout_ts_; } + OB_INLINE int64_t get_expect_swap_size() const { return expect_swap_size_; } + OB_INLINE bool is_valid() { return ATOMIC_LOAD(&is_inited_) && swap_cond_.is_inited(); } + OB_INLINE bool is_finished() const { return ATOMIC_LOAD(&is_finished_); } + OB_INLINE bool is_inited() const { return ATOMIC_LOAD(&is_inited_); } + TO_STRING_KV(KP(this), K(is_inited_), K(is_finished_), K(create_ts_), K(timeout_ms_), K(abs_timeout_ts_), K(expect_swap_size_)); +private: + bool is_inited_; + bool is_finished_; + uint32_t timeout_ms_; + int64_t create_ts_; + int64_t abs_timeout_ts_; + int64_t expect_swap_size_; // in bytes + ObThreadCond swap_cond_; +}; + +// record statistics for flush tasks in flushTG thread +class ObTmpFileFlushMonitor +{ +public: + ObTmpFileFlushMonitor() + : flush_task_cnt_(0), + total_flush_data_length_(0), + max_flush_data_length_(-1), + min_flush_data_length_(INT64_MAX), + f1_cnt_(0), + f2_cnt_(0), + f3_cnt_(0), + f4_cnt_(0), + f5_cnt_(0) {} + void reset(); + void print_statistics(); + void record_flush_stage(const ObTmpFileGlobal::FlushCtxState flush_stage); + void record_flush_task(const int64_t data_length); +private: + int64_t flush_task_cnt_; + int64_t total_flush_data_length_; + int64_t max_flush_data_length_; + int64_t min_flush_data_length_; + int64_t f1_cnt_; + int64_t f2_cnt_; + int64_t f3_cnt_; + int64_t f4_cnt_; + int64_t f5_cnt_; +}; + +// record statistics for swap tasks in swapTG thread +class ObTmpFileSwapMonitor +{ +public: + ObTmpFileSwapMonitor() + : swap_task_cnt_(0), + swap_total_response_time_(0), + swap_max_response_time_(-1), + swap_min_response_time_(INT64_MAX) {} + void reset(); + void print_statistics(); + void record_swap_response_time(const int64_t response_time); +private: + int64_t swap_task_cnt_; + int64_t swap_total_response_time_; + int64_t swap_max_response_time_; + int64_t swap_min_response_time_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_JOB_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_thread_wrapper.cpp b/src/storage/tmp_file/ob_tmp_file_thread_wrapper.cpp new file mode 100644 index 0000000000..d08c739e9b --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_thread_wrapper.cpp @@ -0,0 +1,1023 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_thread_wrapper.h" +#include "share/ob_thread_mgr.h" +#include "storage/blocksstable/ob_block_manager.h" +#include "storage/tmp_file/ob_tmp_file_page_cache_controller.h" +#include "storage/tmp_file/ob_tmp_file_manager.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +ObTmpFileFlushTG::ObTmpFileFlushTG( + ObTmpWriteBufferPool &wbp, + ObTmpFileFlushManager &flush_mgr, + ObIAllocator &allocator, + ObTmpFileBlockManager &tmp_file_block_mgr) + : is_inited_(false), + mode_(RUNNING_MODE::INVALID), + last_flush_timestamp_(0), + flush_io_finished_round_(0), + flushing_block_num_(0), + is_fast_flush_meta_(false), + fast_flush_meta_task_cnt_(0), + wait_list_size_(0), + retry_list_size_(0), + finished_list_size_(0), + wait_list_(), + retry_list_(), + finished_list_(), + flush_monitor_(), + flush_mgr_(flush_mgr), + wbp_(wbp), + tmp_file_block_mgr_(tmp_file_block_mgr), + normal_loop_cnt_(0), + normal_idle_loop_cnt_(0), + fast_loop_cnt_(0), + fast_idle_loop_cnt_(0) +{ +} + +int ObTmpFileFlushTG::init() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObTmpFileSwapTG init twice"); + } else { + is_inited_ = true; + mode_ = RUNNING_MODE::NORMAL; + last_flush_timestamp_ = 0; + flush_io_finished_round_ = 0; + flushing_block_num_ = 0; + + fast_flush_meta_task_cnt_ = 0; + wait_list_size_ = 0; + retry_list_size_ = 0; + finished_list_size_ = 0; + + normal_loop_cnt_ = 0; + normal_idle_loop_cnt_ = 0; + fast_loop_cnt_ = 0; + fast_idle_loop_cnt_ = 0; + } + return ret; +} + +void ObTmpFileFlushTG::destroy() +{ + if (IS_INIT) { + clean_up_lists(); + mode_ = RUNNING_MODE::INVALID; + last_flush_timestamp_ = 0; + flush_io_finished_round_ = 0; + flushing_block_num_ = 0; + + is_fast_flush_meta_ = false; + fast_flush_meta_task_cnt_ = 0; + wait_list_size_ = 0; + retry_list_size_ = 0; + finished_list_size_ = 0; + + normal_loop_cnt_ = 0; + normal_idle_loop_cnt_ = 0; + fast_loop_cnt_ = 0; + fast_idle_loop_cnt_ = 0; + + is_inited_ = false; + } +} + +void ObTmpFileFlushTG::clean_up_lists() +{ + int ret = OB_SUCCESS; + while (!retry_list_.is_empty()) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_retry_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is null", KR(ret)); + } else { + STORAGE_LOG(INFO, "free flush task in retry_list_", KPC(flush_task)); + flush_mgr_.free_flush_task(flush_task); + } + } + + while (!finished_list_.is_empty()) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_finished_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is null", KR(ret)); + } else { + STORAGE_LOG(INFO, "free flush task in finished_list_", KPC(flush_task)); + flush_mgr_.free_flush_task(flush_task); + } + } + + while (!wait_list_.is_empty()) { // clean up tasks after IO complete + STORAGE_LOG(WARN, "wait_list_ is not empty after flush thread stop", K(wait_list_size_)); + for (int64_t cnt = 0; cnt < wait_list_size_ && !wait_list_.is_empty(); ++cnt) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_wait_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is null", K(cnt), K(wait_list_size_)); + } else if (OB_FAIL(flush_task->wait_macro_block_handle())) { + if (OB_EAGAIN == ret) { + push_wait_list_(flush_task); + ret = OB_SUCCESS; + } else { + STORAGE_LOG(ERROR, "unexpected error in waiting flush task finished", KR(ret), KPC(this)); + } + } else if (!flush_task->atomic_get_io_finished()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected flush task state", KR(ret), KPC(flush_task)); + } else if (OB_FAIL(flush_mgr_.notify_write_back_failed(flush_task))) { + STORAGE_LOG(ERROR, "fail to notify_write_back_failed", KR(ret), KPC(flush_task)); + } else { + STORAGE_LOG(DEBUG, "free flush task in wait_list_", KPC(flush_task)); + flush_mgr_.free_flush_task(flush_task); + } + } + usleep(10 * 1000); // 10ms + } +} + +void ObTmpFileFlushTG::set_running_mode(const RUNNING_MODE mode) +{ + mode_ = mode; +} + +void ObTmpFileFlushTG::notify_doing_flush() +{ + last_flush_timestamp_ = 0; +} + +void ObTmpFileFlushTG::signal_io_finish() +{ + ++flush_io_finished_round_; +} + +int64_t ObTmpFileFlushTG::get_flush_io_finished_round() +{ + return flush_io_finished_round_; +} + +int64_t ObTmpFileFlushTG::cal_idle_time() +{ + int64_t idle_time = 0; + int64_t dirty_page_percentage = wbp_.get_dirty_page_percentage(); + if (!wait_list_.is_empty() || !retry_list_.is_empty() || !finished_list_.is_empty() + || ObTmpFileFlushManager::FLUSH_WATERMARK_F1 <= dirty_page_percentage) { + idle_time = ObTmpFilePageCacheController::FLUSH_FAST_INTERVAL; + } else if (RUNNING_MODE::FAST == mode_) { + int64_t flushing_block_num = ATOMIC_LOAD(&flushing_block_num_); + if (flushing_block_num >= get_flushing_block_num_threshold_()) { + idle_time = ObTmpFilePageCacheController::FLUSH_FAST_INTERVAL; + } else { + idle_time = 0; + } + } else { + idle_time = ObTmpFilePageCacheController::FLUSH_INTERVAL; + } + return idle_time; +} + +int ObTmpFileFlushTG::try_work() +{ + int ret = OB_SUCCESS; + + int64_t cur_time = ObTimeUtility::current_monotonic_time(); + if (0 == last_flush_timestamp_ || cur_time - last_flush_timestamp_ >= cal_idle_time() * 1000) { + if (OB_FAIL(do_work_())) { + STORAGE_LOG(WARN, "fail do flush", KR(ret), KPC(this)); + } + last_flush_timestamp_ = ObTimeUtility::current_monotonic_time(); + } + + return ret; +} + +int ObTmpFileFlushTG::do_work_() +{ + int ret = OB_SUCCESS; + + if (is_fast_flush_meta_) { + check_flush_task_io_finished_(); + } else { + if (RUNNING_MODE::FAST == mode_) { + flush_fast_(); + } else if (RUNNING_MODE::NORMAL == mode_) { + flush_normal_(); + } else { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "unknown running mode", KR(ret), K(mode_)); + } + } + + if (TC_REACH_TIME_INTERVAL(1 * 1000 * 1000)) { + tmp_file_block_mgr_.print_block_usage(); + flush_monitor_.print_statistics(); + wbp_.print_statistics(); + STORAGE_LOG(INFO, "ObTmpFileFlushTG information", KPC(this)); + normal_loop_cnt_ = 0; + normal_idle_loop_cnt_ = 0; + fast_loop_cnt_ = 0; + fast_idle_loop_cnt_ = 0; + } + return ret; +} + +// 1. check wait_list_ for IO complete task; 2. retry old task if exists; 3. build new task if needed +void ObTmpFileFlushTG::flush_fast_() +{ + int ret = OB_SUCCESS; + int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + int64_t flush_size = min(get_fast_flush_size_(), get_flushing_block_num_threshold_() * BLOCK_SIZE); + if (OB_FAIL(check_flush_task_io_finished_())) { + STORAGE_LOG(WARN, "fail to check flush task io finished", KR(ret)); + } + if (OB_FAIL(retry_task_())) { + STORAGE_LOG(WARN, "fail to retry task", KR(ret)); + } + if (flush_size > 0) { + if (OB_FAIL(wash_(flush_size, RUNNING_MODE::FAST))) { + STORAGE_LOG(WARN, "fail to flush fast", KR(ret), KPC(this), K(flush_size)); + } + } else { + STORAGE_LOG(DEBUG, "current expect flush size is 0, skip flush", K(flush_size), K(this)); + } +} + +void ObTmpFileFlushTG::flush_normal_() +{ + int ret = OB_SUCCESS; + int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + int64_t normal_flush_size = max(0, (MAX_FLUSHING_BLOCK_NUM - ATOMIC_LOAD(&flushing_block_num_)) * BLOCK_SIZE); + if (OB_FAIL(check_flush_task_io_finished_())) { + STORAGE_LOG(WARN, "fail to check flush task io finished", KR(ret)); + } + if (OB_FAIL(retry_task_())) { + STORAGE_LOG(WARN, "fail to retry task", KR(ret)); + } + if (normal_flush_size > 0) { + if (OB_FAIL(wash_(normal_flush_size, RUNNING_MODE::NORMAL))) { + STORAGE_LOG(WARN, "fail to flush normal", KR(ret), KPC(this)); + } + } else { + STORAGE_LOG(DEBUG, "current expect flush size is 0, skip flush", K(normal_flush_size), K(this)); + } +} + +int ObTmpFileFlushTG::handle_generated_flush_tasks_(ObSpLinkQueue &flushing_list, int64_t &task_num) +{ + int ret = OB_SUCCESS; + task_num = 0; + while (!flushing_list.is_empty()) { // ignore error code to handle all flush tasks + task_num += 1; + ObSpLinkQueue::Link *link = nullptr; + flushing_list.pop(link); + if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is null", KR(ret)); + } else { + ObTmpFileFlushTask *flush_task = static_cast(link); + FlushState state = flush_task->get_state(); + bool need_release_resource = false; + STORAGE_LOG(DEBUG, "check flush task state", KPC(flush_task)); + if (FlushState::TFFT_WAIT == state) { + ATOMIC_INC(&flushing_block_num_); + push_wait_list_(flush_task); + if (flush_task->get_is_fast_flush_tree()) { + // after setting this flag, the thread will only process tasks that are already in wait_list_ and finished_list_, + // and will not process tasks in retry_list_ or generate new flush tasks + is_fast_flush_meta_ = true; + fast_flush_meta_task_cnt_ += 1; + } + } else if (FlushState::TFFT_FILL_BLOCK_BUF > state) { + need_release_resource = true; + } else if (FlushState::TFFT_FILL_BLOCK_BUF == state && 0 == flush_task->get_data_length()) { + need_release_resource = true; + } else if (FlushState::TFFT_FILL_BLOCK_BUF == state && 0 < flush_task->get_data_length()) { + // ATTENTION! after the data has been copied, it should transition to the next state. + // staying in the TFFT_FILL_BLOCK_BUF is an abnormal behavior and the file should be discarded, + // do not free task here to avoid further corruption. + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("fail to switch state after data copied from tmp file", KR(ret), KPC(flush_task)); + } else if (FlushState::TFFT_FILL_BLOCK_BUF < state) { + push_retry_list_(flush_task); + ATOMIC_INC(&flushing_block_num_); + STORAGE_LOG(DEBUG, "push flush task to retry list", KPC(flush_task)); + } + + if (need_release_resource) { + if (ObTmpFileGlobal::INVALID_TMP_FILE_BLOCK_INDEX != flush_task->get_block_index()) { + flush_mgr_.free_tmp_file_block(*flush_task); + } + flush_mgr_.free_flush_task(flush_task); + } + } + } // end while + return ret; +} + +int ObTmpFileFlushTG::wash_(const int64_t expect_flush_size, const RUNNING_MODE mode) +{ + int ret = OB_SUCCESS; + int64_t flushing_task_cnt = 0; + ObSpLinkQueue flushing_list; + if (OB_FAIL(flush_mgr_.flush(flushing_list, flush_monitor_, expect_flush_size, is_fast_flush_meta_))) { + STORAGE_LOG(WARN, "flush mgr fail to do flush", KR(ret), KPC(this)); + } else if (OB_FAIL(handle_generated_flush_tasks_(flushing_list, flushing_task_cnt))) { + STORAGE_LOG(WARN, "fail to handle generated flush tasks", KR(ret), K(flushing_task_cnt), KPC(this)); + } + + bool idle_loop = flushing_task_cnt == 0; + if (idle_loop && wbp_.get_dirty_page_percentage() < ObTmpFileFlushManager::FLUSH_WATERMARK_F5) { + signal_io_finish(); + } + + if (RUNNING_MODE::NORMAL == mode) { + ++normal_loop_cnt_; + if (idle_loop) { + ++normal_idle_loop_cnt_; + } + } else if (RUNNING_MODE::FAST == mode) { + ++fast_loop_cnt_; + if (idle_loop) { + ++fast_idle_loop_cnt_; + } + } + return ret; +} + +int ObTmpFileFlushTG::retry_task_() +{ + int ret = OB_SUCCESS; + for (int64_t cnt = retry_list_size_; cnt > 0 && !retry_list_.is_empty(); --cnt) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_retry_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else { + STORAGE_LOG(DEBUG, "retry flush task", KPC(flush_task)); + if (OB_FAIL(flush_mgr_.retry(*flush_task))) { + STORAGE_LOG(WARN, "fail to retry flush task", KR(ret), KPC(flush_task)); + } + // push task into wait_list_/retry_list_ according to task state, ignore error code + FlushState state = flush_task->get_state(); + if (FlushState::TFFT_WAIT == state) { + push_wait_list_(flush_task); + } else if (FlushState::TFFT_FILL_BLOCK_BUF < state) { + push_retry_list_(flush_task); + if (FlushState::TFFT_INSERT_META_TREE == state && OB_ALLOCATE_TMP_FILE_PAGE_FAILED == ret) { + STORAGE_LOG(WARN, "fail to retry insert meta item in TFFT_INSERT_META_TREE", KPC(flush_task)); + is_fast_flush_meta_ = true; + if (OB_FAIL(special_flush_meta_tree_page_())) { + STORAGE_LOG(WARN, "fail to flush meta tree page", KR(ret)); + } + break; + } + } + } + } + return ret; +} + +int ObTmpFileFlushTG::special_flush_meta_tree_page_() +{ + int ret = OB_SUCCESS; + ObSpLinkQueue flushing_list; + int64_t flushing_task_cnt = 0; + int64_t expect_flush_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + if (OB_FAIL(flush_mgr_.flush(flushing_list, flush_monitor_, expect_flush_size, true/*is_flush_meta_tree*/))) { + STORAGE_LOG(ERROR, "flush mgr fail to do fast flush meta tree page", KR(ret), KPC(this)); + } else if (OB_FAIL(handle_generated_flush_tasks_(flushing_list, flushing_task_cnt))) { + STORAGE_LOG(WARN, "fail to handle fast flush meta tasks", KR(ret), K(flushing_task_cnt), KPC(this)); + } + return ret; +} + +int ObTmpFileFlushTG::check_flush_task_io_finished_() +{ + int ret = OB_SUCCESS; + for (int64_t cnt = ATOMIC_LOAD(&wait_list_size_); cnt > 0 && !wait_list_.is_empty(); --cnt) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_wait_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else if (OB_FAIL(flush_task->wait_macro_block_handle())) { + if (OB_EAGAIN == ret) { + push_wait_list_(flush_task); // IO is not completed, continue waiting + ret = OB_SUCCESS; + } else { + STORAGE_LOG(ERROR, "unexpected error in waiting flush task finished", KR(ret), KPC(this)); + } + } else if (!flush_task->atomic_get_io_finished()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(ERROR, "unexpected flush task state", KR(ret), KPC(flush_task)); + } else if (OB_FAIL(flush_mgr_.io_finished(*flush_task))) { + STORAGE_LOG(WARN, "fail to handle flush task finished", KR(ret), KPC(flush_task)); + } else { + if (FlushState::TFFT_FINISH == flush_task->get_state()) { + push_finished_list_(flush_task); + STORAGE_LOG(DEBUG, "flush task push to finished list", KPC(flush_task)); + } else if (FlushState::TFFT_ASYNC_WRITE == flush_task->get_state()) { + push_retry_list_(flush_task); + STORAGE_LOG(DEBUG, "flush task push to retry list", KPC(flush_task)); + } else { + STORAGE_LOG(ERROR, "unexpected flush task state", KR(ret), KPC(flush_task)); + } + } + } + + for (int64_t cnt = ATOMIC_LOAD(&finished_list_size_); cnt > 0 && !finished_list_.is_empty(); --cnt) { + ObTmpFileFlushTask *flush_task = nullptr; + pop_finished_list_(flush_task); + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else { + bool is_flush_tree_page = flush_task->get_is_fast_flush_tree(); + STORAGE_LOG(DEBUG, "flush task io complete", K(flushing_block_num_), KPC(flush_task)); + // if the update fails, it will be retried during the next wakeup + if (OB_FAIL(flush_mgr_.update_file_meta_after_flush(*flush_task))) { + STORAGE_LOG(WARN, "fail to drive flush state machine", KR(ret), KPC(flush_task)); + push_finished_list_(flush_task); + } else { + flush_mgr_.free_flush_task(flush_task); + ATOMIC_DEC(&flushing_block_num_); + fast_flush_meta_task_cnt_ -= is_flush_tree_page ? 1 : 0; + if (fast_flush_meta_task_cnt_ == 0) { + // reset is_fast_flush_meta_ flag to resume retry task and flush + is_fast_flush_meta_ = false; + } else if (OB_UNLIKELY(fast_flush_meta_task_cnt_ < 0)) { + STORAGE_LOG(ERROR, "fast_flush_meta_task_cnt_ is negative", KPC(this)); + } + signal_io_finish(); + } + } + } + return ret; +} + +int ObTmpFileFlushTG::push_wait_list_(ObTmpFileFlushTask *flush_task) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_task)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else if (OB_FAIL(wait_list_.push(flush_task))) { + STORAGE_LOG(WARN, "fail push flush task into wait_list_", KR(ret), KP(flush_task)); + } else { + ATOMIC_INC(&wait_list_size_); + } + return ret; +} + +int ObTmpFileFlushTG::pop_wait_list_(ObTmpFileFlushTask *&flush_task) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue::Link *link = nullptr; + if (OB_FAIL(wait_list_.pop(link))) { + STORAGE_LOG(DEBUG, "fail to pop flush task from wait_list_", KR(ret)); + } else if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task ptr in wait list is null", KR(ret)); + } else { + ATOMIC_DEC(&wait_list_size_); + flush_task = static_cast(link); + } + return ret; +} + +int ObTmpFileFlushTG::push_retry_list_(ObTmpFileFlushTask *flush_task) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_task)){ + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else if (OB_FAIL(retry_list_.push(flush_task))) { + STORAGE_LOG(WARN, "fail push flush task into retry_list_", KR(ret), KP(flush_task)); + } else { + ATOMIC_INC(&retry_list_size_); + } + return ret; +} + +int ObTmpFileFlushTG::pop_retry_list_(ObTmpFileFlushTask *&flush_task) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue::Link *link = nullptr; + if (OB_FAIL(retry_list_.pop(link))) { + STORAGE_LOG(DEBUG, "fail to pop flush task from retry_list_", KR(ret)); + } else if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task ptr is null", KR(ret)); + } else { + ATOMIC_DEC(&retry_list_size_); + flush_task = static_cast(link); + } + return ret; +} + +int ObTmpFileFlushTG::push_finished_list_(ObTmpFileFlushTask *flush_task) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(flush_task)){ + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task is nullptr", KR(ret)); + } else if (OB_FAIL(finished_list_.push(flush_task))) { + STORAGE_LOG(WARN, "fail push flush task into finished_list_", KR(ret), KP(flush_task)); + } else { + ATOMIC_INC(&finished_list_size_); + } + return ret; +} + +int ObTmpFileFlushTG::pop_finished_list_(ObTmpFileFlushTask *&flush_task) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue::Link *link = nullptr; + if (OB_FAIL(finished_list_.pop(link))) { + STORAGE_LOG(DEBUG, "fail to pop flush task from finished_list_", KR(ret)); + } else if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "flush task ptr is null", KR(ret)); + } else { + ATOMIC_DEC(&finished_list_size_); + flush_task = static_cast(link); + } + return ret; +} + +// fast mode flush size is max(2MB, min(5% * tmp_file_memory,30MB)) +int ObTmpFileFlushTG::get_fast_flush_size_() +{ + // TODO: move to page cache controller + const int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + int64_t wbp_mem_limit = wbp_.get_memory_limit(); + int64_t flush_size = max(BLOCK_SIZE, min(MAX_FLUSHING_BLOCK_NUM * BLOCK_SIZE, upper_align(0.05 * wbp_mem_limit, BLOCK_SIZE))); + return flush_size; +} + +// flushing threshold is MIN(20MB, (20% * tmp_file_memory)) +int ObTmpFileFlushTG::get_flushing_block_num_threshold_() +{ + const int64_t BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + int64_t wbp_mem_limit = wbp_.get_memory_limit(); + int64_t flush_threshold = max(BLOCK_SIZE, min(MAX_FLUSHING_BLOCK_NUM, static_cast(0.2 * wbp_mem_limit / BLOCK_SIZE))); + return flush_threshold; +} + +// --------------- swap ----------------// + +ObTmpFileSwapTG::ObTmpFileSwapTG(ObTmpWriteBufferPool &wbp, + ObTmpFileEvictionManager &elimination_mgr, + ObTmpFileFlushTG &flush_tg) + : is_inited_(false), + tg_id_(-1), + idle_cond_(), + last_swap_timestamp_(0), + swap_job_num_(0), + swap_job_list_(), + working_list_size_(0), + working_list_(), + swap_monitor_(), + flush_tg_ref_(flush_tg), + flush_io_finished_round_(0), + wbp_(wbp), + evict_mgr_(elimination_mgr), + file_mgr_(nullptr) +{ +} + +int ObTmpFileSwapTG::init(ObTenantTmpFileManager &file_mgr) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + STORAGE_LOG(WARN, "ObTmpFileSwapTG init twice"); + } else if (OB_FAIL(idle_cond_.init(ObWaitEventIds::NULL_EVENT))) { + STORAGE_LOG(WARN, "failed to init condition variable", KR(ret)); + } else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::TmpFileSwap, tg_id_))) { + STORAGE_LOG(WARN, "fail to create swap thread", KR(ret)); + } else if (OB_FAIL(TG_SET_RUNNABLE(tg_id_, *this))) { + STORAGE_LOG(WARN, "fail to set swap tg runnable", KR(ret)); + } else { + is_inited_ = true; + last_swap_timestamp_ = 0; + swap_job_num_ = 0; + working_list_size_ = 0; + flush_io_finished_round_ = 0; + file_mgr_ = &file_mgr; + } + return ret; +} + +int ObTmpFileSwapTG::start() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapTG thread is not inited", KR(ret)); + } else if (OB_FAIL(TG_START(tg_id_))) { + STORAGE_LOG(WARN, "fail to start tmp file ObTmpFileSwapTG thread", KR(ret)); + } + return ret; +} + +void ObTmpFileSwapTG::stop() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapTG thread is not inited", KR(ret)); + } else { + TG_STOP(tg_id_); + ObThreadCondGuard guard(idle_cond_); + idle_cond_.signal(); + } +} + +void ObTmpFileSwapTG::wait() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapTG thread is not inited", KR(ret)); + } else { + TG_WAIT(tg_id_); + } +} + +void ObTmpFileSwapTG::destroy() +{ + int ret = OB_SUCCESS; + if (-1 != tg_id_) { + TG_DESTROY(tg_id_); + tg_id_ = -1; + } + + clean_up_lists_(); + last_swap_timestamp_ = 0; + swap_job_num_ = 0; + working_list_size_ = 0; + flush_io_finished_round_ = 0; + idle_cond_.destroy(); + file_mgr_ = nullptr; + is_inited_ = false; +} + +int ObTmpFileSwapTG::swap_job_enqueue(ObTmpFileSwapJob *swap_job) +{ + int ret = OB_SUCCESS; + if (has_set_stop()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap thread has been stopped", KR(ret), K(tg_id_), KP(swap_job)); + } else if (OB_FAIL(swap_job_list_.push(swap_job))) { + STORAGE_LOG(WARN, "fail push swap job", KR(ret), K(tg_id_), KP(swap_job)); + } else { + ATOMIC_INC(&swap_job_num_); + } + return ret; +} + +int ObTmpFileSwapTG::swap_job_dequeue(ObTmpFileSwapJob *&swap_job) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue::Link *link = nullptr; + if (OB_FAIL(swap_job_list_.pop(link))) { + STORAGE_LOG(DEBUG, "fail to pop swap job", KR(ret), K(tg_id_)); + } else if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "fail to get swap job, ptr is nullptr", KR(ret), K(tg_id_)); + } else { + ATOMIC_DEC(&swap_job_num_); + swap_job = static_cast(link); + } + return ret; +} + +void ObTmpFileSwapTG::notify_doing_swap() +{ + ObThreadCondGuard guard(idle_cond_); + idle_cond_.signal(); +} + +int64_t ObTmpFileSwapTG::cal_idle_time() +{ + int64_t swap_idle_time = ObTmpFilePageCacheController::SWAP_INTERVAL; + if (ATOMIC_LOAD(&swap_job_num_) != 0 || ATOMIC_LOAD(&working_list_size_) != 0) { + if (flush_io_finished_round_ < flush_tg_ref_.get_flush_io_finished_round()) { + swap_idle_time = 0; + } else { + swap_idle_time = ObTmpFilePageCacheController::SWAP_FAST_INTERVAL; + } + } + return swap_idle_time; +} + +void ObTmpFileSwapTG::run1() +{ + int ret = OB_SUCCESS; + lib::set_thread_name("TFSwap"); + while (!has_set_stop()) { + if (OB_FAIL(try_work())) { + STORAGE_LOG(WARN, "fail to try swap work", KR(ret)); + } + + if (OB_FAIL(flush_tg_ref_.try_work())) { + // overwrite ret + STORAGE_LOG(WARN, "fail to try flush work", KR(ret)); + } + + ObThreadCondGuard guard(idle_cond_); + int64_t swap_idle_time = cal_idle_time(); + int64_t flush_idle_time = flush_tg_ref_.cal_idle_time(); + int64_t idle_time = min(swap_idle_time, flush_idle_time); + if (!has_set_stop() && idle_time != 0) { + idle_cond_.wait(idle_time); + } + + if (OB_NOT_NULL(file_mgr_)) { + file_mgr_->refresh_meta_memory_limit(); + } + } + + if (has_set_stop()) { + flush_tg_ref_.clean_up_lists(); + clean_up_lists_(); + } +} + +void ObTmpFileSwapTG::clean_up_lists_() +{ + int ret = OB_SUCCESS; + while (!swap_job_list_.is_empty()) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(swap_job_dequeue(swap_job))) { + STORAGE_LOG(WARN, "fail dequeue swap job or swap job is nullptr", KR(ret), KP(swap_job)); + } else if (OB_FAIL(swap_job->signal_swap_complete())){ + STORAGE_LOG(WARN, "fail to signal swap complete", KR(ret)); + } + } + + ret = OB_SUCCESS; + while (!working_list_.is_empty()) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(pop_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to pop working job or ptr is null", KR(ret), KP(swap_job)); + } else if (OB_FAIL(swap_job->signal_swap_complete())){ + STORAGE_LOG(WARN, "fail to signal swap complete", KR(ret)); + } + } +} + +int ObTmpFileSwapTG::try_work() +{ + int ret = OB_SUCCESS; + + int64_t cur_time = ObTimeUtility::current_monotonic_time(); + if (0 == last_swap_timestamp_ || cur_time - last_swap_timestamp_ >= cal_idle_time() * 1000) { + if (OB_FAIL(do_work_())) { + STORAGE_LOG(WARN, "fail do swap", KR(ret), KPC(this)); + } + last_swap_timestamp_ = ObTimeUtility::current_monotonic_time(); + } + + return ret; +} + +int ObTmpFileSwapTG::do_work_() +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + STORAGE_LOG(WARN, "ObTmpFileSwapTG not init", KR(ret)); + } + + if (OB_SUCC(ret)) { + if (ATOMIC_LOAD(&swap_job_num_) == 0 && ATOMIC_LOAD(&working_list_size_) == 0) { + if (OB_FAIL(swap_normal_())){ + STORAGE_LOG(WARN, "fail to do normal swap", KR(ret)); + } + } else { + if (OB_FAIL(swap_fast_())){ + STORAGE_LOG(WARN, "fail to do fast swap", KR(ret)); + } + if (OB_FAIL(swap_normal_())) { + STORAGE_LOG(WARN, "fail to do normal swap", KR(ret)); + } + } + if (ATOMIC_LOAD(&swap_job_num_) == 0 && ATOMIC_LOAD(&working_list_size_) == 0) { + flush_tg_ref_.set_running_mode(ObTmpFileFlushTG::RUNNING_MODE::NORMAL); + } + } + if (TC_REACH_TIME_INTERVAL(1 * 1000 * 1000)) { + swap_monitor_.print_statistics(); + } + return ret; +} + +// runs in normal mode when there is no swap job. +// since memory is not tight at this point, we try to evict pages +// but not guarantee eviction occurs if clean pages are not enough +int ObTmpFileSwapTG::swap_normal_() +{ + int ret = OB_SUCCESS; + + int64_t swap_size = wbp_.get_swap_size(); + int64_t actual_swap_page_cnt = 0; + if (swap_size > 0) { // do swap + int64_t swap_page_cnt = swap_size / PAGE_SIZE; + if (OB_FAIL(evict_mgr_.evict(swap_page_cnt, actual_swap_page_cnt))) { + STORAGE_LOG(WARN, "fail to swap out pages", KR(ret), K(swap_size)); + } + } + return ret; +} + +// attempt to evict pages and wake up caller threads as soon as possible, +// this may trigger the flush thread to flush a small number of pages (FAST mode). +// caller threads will be awakened if swap job timeout +int ObTmpFileSwapTG::swap_fast_() +{ + int ret = OB_SUCCESS; + while (OB_SUCC(ret) && !swap_job_list_.is_empty()) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(swap_job_dequeue(swap_job))) { + STORAGE_LOG(WARN, "fail to get swap job", KR(ret)); + } else if (OB_FAIL(push_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to push working job", KR(ret), KP(swap_job)); + } + } + + while (OB_SUCC(ret) && !working_list_.is_empty()) { + int64_t expect_swap_page_cnt = 0; + int64_t actual_swap_page_cnt = 0; + // calculate expect swap pages number for a batch of jobs + if (OB_FAIL(calculate_swap_page_num_(PROCCESS_JOB_NUM_PER_BATCH, expect_swap_page_cnt))) { + STORAGE_LOG(WARN, "fail to calculate swap page num", KR(ret)); + } else if (OB_UNLIKELY(expect_swap_page_cnt <= 0)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "cur expect swap page cnt is invalid", KR(ret), K(expect_swap_page_cnt)); + } else if (OB_FAIL(evict_mgr_.evict(expect_swap_page_cnt, actual_swap_page_cnt))) { + STORAGE_LOG(WARN, "fail to swap out pages", KR(ret), K(expect_swap_page_cnt), K(actual_swap_page_cnt)); + } + + int64_t wakeup_job_cnt = 0; + wakeup_satisfied_jobs_(wakeup_job_cnt); + wakeup_timeout_jobs_(); + + // do flush if could not evict enough pages + if (OB_SUCC(ret) && !working_list_.is_empty() && wakeup_job_cnt < PROCCESS_JOB_NUM_PER_BATCH) { + flush_io_finished_round_ = flush_tg_ref_.get_flush_io_finished_round(); + flush_tg_ref_.set_running_mode(ObTmpFileFlushTG::RUNNING_MODE::FAST); + flush_tg_ref_.notify_doing_flush(); + break; + } + } // end while + + return ret; +} + +int ObTmpFileSwapTG::calculate_swap_page_num_(const int64_t batch_size, int64_t &expect_swap_cnt) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue cur_working_list; + for (int64_t i = 0; OB_SUCC(ret) && i < batch_size && !working_list_.is_empty(); ++i) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(pop_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to pop working job or ptr is null", KR(ret), KP(swap_job)); + } else { + expect_swap_cnt += upper_align(swap_job->get_expect_swap_size(), PAGE_SIZE) / PAGE_SIZE; + cur_working_list.push_front(swap_job); + } + } + while (!cur_working_list.is_empty()) { + ObSpLinkQueue::Link *link = nullptr; + cur_working_list.pop(link); + ObTmpFileSwapJob *swap_job = static_cast(link); + if (OB_ISNULL(swap_job)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap job is invalid", KR(ret)); + } else if (OB_FAIL(push_working_job_front_(swap_job))) { + STORAGE_LOG(WARN, "fail to push swap job", KR(ret), KPC(swap_job)); + } + } + return ret; +} + +int ObTmpFileSwapTG::wakeup_satisfied_jobs_(int64_t& wakeup_job_cnt) +{ + int ret = OB_SUCCESS; + wakeup_job_cnt = 0; + int64_t wbp_free_page_cnt = wbp_.get_max_data_page_num() - wbp_.get_data_page_num(); + while (OB_SUCC(ret) && wbp_free_page_cnt > 0 && !working_list_.is_empty()) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(pop_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to pop working job or ptr is null", KR(ret), KP(swap_job)); + } else { + // wake up threads even if free page number < swap job's expect swap size + int64_t single_job_swap_page_cnt = upper_align(swap_job->get_expect_swap_size(), PAGE_SIZE) / PAGE_SIZE; + wbp_free_page_cnt -= min(wbp_free_page_cnt, single_job_swap_page_cnt); + int64_t response_time = ObTimeUtility::current_time() - swap_job->get_create_ts(); + swap_monitor_.record_swap_response_time(response_time); + if (OB_FAIL(swap_job->signal_swap_complete())) { + STORAGE_LOG(WARN, "fail to signal swap complete", KR(ret), KPC(swap_job)); + } else { + ++wakeup_job_cnt; + } + } + } + return ret; +} + +int ObTmpFileSwapTG::wakeup_timeout_jobs_() +{ + int ret = OB_SUCCESS; + for (int64_t i = working_list_size_; OB_SUCC(ret) && i > 0 && !working_list_.is_empty(); --i) { + ObTmpFileSwapJob *swap_job = nullptr; + if (OB_FAIL(pop_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to pop working job or ptr is null", KR(ret), KP(swap_job)); + } else if (swap_job->get_abs_timeout_ts() <= ObTimeUtility::current_time()) { + // timeout, wake it up + int64_t response_time = ObTimeUtility::current_time() - swap_job->get_create_ts(); + swap_monitor_.record_swap_response_time(response_time); + if (OB_FAIL(swap_job->signal_swap_complete())) { + STORAGE_LOG(WARN, "fail to signal swap complete", KR(ret), KP(swap_job)); + } + } else { + if (OB_FAIL(push_working_job_(swap_job))) { + STORAGE_LOG(WARN, "fail to push swap job", KR(ret), K(tg_id_), KPC(swap_job)); + } + } + } + return ret; +} + +int ObTmpFileSwapTG::push_working_job_(ObTmpFileSwapJob *swap_job) +{ + int ret = OB_SUCCESS; + if (has_set_stop()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap thread has been stopped", KR(ret), K(tg_id_), KP(swap_job)); + } else if (OB_FAIL(working_list_.push(swap_job))) { + STORAGE_LOG(WARN, "fail push swap job", KR(ret), K(tg_id_), KPC(swap_job)); + } else { + ATOMIC_INC(&working_list_size_); + } + return ret; +} + +int ObTmpFileSwapTG::pop_working_job_(ObTmpFileSwapJob *&swap_job) +{ + int ret = OB_SUCCESS; + ObSpLinkQueue::Link *link = nullptr; + if (OB_FAIL(working_list_.pop(link))) { + STORAGE_LOG(DEBUG, "fail to pop swap job", KR(ret), K(tg_id_)); + } else if (OB_ISNULL(link)) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "fail to get swap job, ptr is nullptr", KR(ret), K(tg_id_)); + } else { + ATOMIC_DEC(&working_list_size_); + swap_job = static_cast(link); + } + return ret; +} + +int ObTmpFileSwapTG::push_working_job_front_(ObTmpFileSwapJob *swap_job) +{ + int ret = OB_SUCCESS; + if (has_set_stop()) { + ret = OB_ERR_UNEXPECTED; + STORAGE_LOG(WARN, "swap thread has been stopped", KR(ret), K(tg_id_), KP(swap_job)); + } else if (OB_FAIL(working_list_.push_front(swap_job))) { + STORAGE_LOG(WARN, "fail push swap job", KR(ret), K(tg_id_), KPC(swap_job)); + } else { + ATOMIC_INC(&working_list_size_); + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_thread_wrapper.h b/src/storage/tmp_file/ob_tmp_file_thread_wrapper.h new file mode 100644 index 0000000000..79fe3fbae7 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_thread_wrapper.h @@ -0,0 +1,158 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_WRAPPER_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_WRAPPER_H_ + +#include "storage/tmp_file/ob_tmp_file_thread_job.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "storage/tmp_file/ob_tmp_file_eviction_manager.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "storage/tmp_file/ob_tmp_file_flush_manager.h" +#include "lib/thread/thread_mgr_interface.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +class ObTmpFileFlushManager; + +// When originally designed, ObTmpFileFlushTG was an independent thread. in order to reduce the +// number of threads, the thread of ObTmpFileFlushTG was drived ObTmpFileSwapTG. +class ObTmpFileFlushTG +{ +public: + typedef ObTmpFileFlushTask::ObTmpFileFlushTaskState FlushState; + static const int64_t MAX_FLUSHING_BLOCK_NUM = 50; + enum RUNNING_MODE { + INVALID = 0, + NORMAL = 1, + FAST = 2 + }; +public: + ObTmpFileFlushTG(ObTmpWriteBufferPool &wbp, + ObTmpFileFlushManager &flush_mgr, + ObIAllocator &allocator, + ObTmpFileBlockManager &tmp_file_block_mgr); + int init(); + void destroy(); + + int try_work(); + void set_running_mode(const RUNNING_MODE mode); + void notify_doing_flush(); + void signal_io_finish(); + int64_t get_flush_io_finished_round(); + int64_t cal_idle_time(); + void clean_up_lists(); + TO_STRING_KV(K(is_inited_), K(flushing_block_num_), K(is_fast_flush_meta_), K(fast_flush_meta_task_cnt_), K(mode_), + K(wait_list_size_), K(retry_list_size_), K(finished_list_size_), + K(normal_loop_cnt_), K(normal_idle_loop_cnt_), K(fast_loop_cnt_), K(fast_idle_loop_cnt_)); +private: + int do_work_(); + int handle_generated_flush_tasks_(ObSpLinkQueue &flushing_list, int64_t &task_num); + int wash_(const int64_t expect_flush_size, const RUNNING_MODE mode); + int check_flush_task_io_finished_(); + int retry_task_(); + int special_flush_meta_tree_page_(); + void flush_fast_(); + void flush_normal_(); + int get_fast_flush_size_(); + int get_flushing_block_num_threshold_(); + int push_wait_list_(ObTmpFileFlushTask *flush_task); + int pop_wait_list_(ObTmpFileFlushTask *&flush_task); + int push_retry_list_(ObTmpFileFlushTask *flush_task); + int pop_retry_list_(ObTmpFileFlushTask *&flush_task); + int push_finished_list_(ObTmpFileFlushTask *flush_task); + int pop_finished_list_(ObTmpFileFlushTask *&flush_task); +private: + bool is_inited_; + RUNNING_MODE mode_; + int64_t last_flush_timestamp_; + int64_t flush_io_finished_round_; + + int64_t flushing_block_num_; // maintain it when ObTmpFileFlushTask is created and freed + bool is_fast_flush_meta_; // indicate thread is fast flushing meta page, no new flush tasks will be added and no tasks will be retried + int64_t fast_flush_meta_task_cnt_; + int64_t wait_list_size_; + int64_t retry_list_size_; + int64_t finished_list_size_; + ObSpLinkQueue wait_list_; // list for tasks that are waiting for IO finished + ObSpLinkQueue retry_list_; // list for tasks that need to be retried + ObSpLinkQueue finished_list_; // list for tasks which have finished IO and need to update file's meta + ObTmpFileFlushMonitor flush_monitor_; + ObTmpFileFlushManager &flush_mgr_; + ObTmpWriteBufferPool &wbp_; + ObTmpFileBlockManager &tmp_file_block_mgr_; + + int64_t normal_loop_cnt_; + int64_t normal_idle_loop_cnt_; + int64_t fast_loop_cnt_; + int64_t fast_idle_loop_cnt_; +}; + +class ObTmpFileSwapTG : public lib::TGRunnable +{ +public: + ObTmpFileSwapTG(ObTmpWriteBufferPool &wbp, + ObTmpFileEvictionManager &elimination_mgr, + ObTmpFileFlushTG &flush_tg); + int init(ObTenantTmpFileManager &file_mgr); + int start(); + void stop(); + void wait(); + void destroy(); + void run1() override; + int64_t cal_idle_time(); + int try_work(); + int swap_job_enqueue(ObTmpFileSwapJob *swap_job); + int swap_job_dequeue(ObTmpFileSwapJob *&swap_job); + // wake up swap thread to do work + void notify_doing_swap(); + TO_STRING_KV(K(is_inited_), K(tg_id_), K(swap_job_num_), K(has_set_stop())); +private: + static const int64_t PAGE_SIZE = ObTmpFileGlobal::PAGE_SIZE; + static const int64_t PROCCESS_JOB_NUM_PER_BATCH = 128; + void clean_up_lists_(); + int do_work_(); + int swap_normal_(); + int swap_fast_(); + int push_working_job_(ObTmpFileSwapJob *swap_job); + int push_working_job_front_(ObTmpFileSwapJob *swap_job); + int pop_working_job_(ObTmpFileSwapJob *&swap_job); + int calculate_swap_page_num_(const int64_t batch_size, int64_t &expect_swap_cnt); + int wakeup_satisfied_jobs_(int64_t& wakeup_job_cnt); + int wakeup_timeout_jobs_(); +private: + bool is_inited_; + int tg_id_; + ObThreadCond idle_cond_; + int64_t last_swap_timestamp_; + + int64_t swap_job_num_; + ObSpLinkQueue swap_job_list_; // list for swap jobs to be processed + int64_t working_list_size_; + ObSpLinkQueue working_list_; // list for swap jobs that are being processed + + ObTmpFileSwapMonitor swap_monitor_; + + ObTmpFileFlushTG &flush_tg_ref_; + int64_t flush_io_finished_round_; + + ObTmpWriteBufferPool &wbp_; + ObTmpFileEvictionManager &evict_mgr_; + ObTenantTmpFileManager *file_mgr_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_THREAD_WRAPPER_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.cpp b/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.cpp new file mode 100644 index 0000000000..091f83b43c --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.cpp @@ -0,0 +1,783 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "storage/tmp_file/ob_tmp_file_global.h" +#include "share/ob_errno.h" +#include "share/rc/ob_tenant_base.h" + +namespace oceanbase +{ +namespace tmp_file +{ +ObTmpFileWBPIndexCache::ObTmpFileWBPIndexCache() : + ObTmpFileCircleArray(), bucket_array_allocator_(), bucket_allocator_(), + page_buckets_(nullptr) , + fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), wbp_(nullptr), + max_bucket_array_capacity_(MAX_BUCKET_ARRAY_CAPACITY) {} + +ObTmpFileWBPIndexCache::~ObTmpFileWBPIndexCache() +{ + destroy(); +} + +int ObTmpFileWBPIndexCache::init(const int64_t fd, ObTmpWriteBufferPool* wbp, + ObIAllocator *wbp_index_cache_allocator, + ObIAllocator *wbp_index_cache_bkt_allocator) +{ + int ret = OB_SUCCESS; + void *buf = nullptr; + uint64_t tenant_id = MTL_ID(); + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tmp file fd", KR(ret), K(fd)); + } else if (OB_ISNULL(wbp) || OB_ISNULL(wbp_index_cache_allocator) || + OB_ISNULL(wbp_index_cache_bkt_allocator)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(wbp), KP(wbp_index_cache_allocator), + KP(wbp_index_cache_bkt_allocator)); + } else if (OB_ISNULL(buf = wbp_index_cache_allocator->alloc(sizeof(ObArray), + lib::ObMemAttr(tenant_id, "TmpFileIdxCache")))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory for temporary file page index bucket", + KR(ret), K(tenant_id), K(sizeof(ObArray))); + } else if (FALSE_IT(page_buckets_ = new (buf) ObArray())) { + } else if (FALSE_IT(page_buckets_->set_attr(lib::ObMemAttr(tenant_id, "TmpFileIdxCache")))) { + } else if (OB_FAIL(page_buckets_->prepare_allocate(INIT_BUCKET_ARRAY_CAPACITY, nullptr))) { + page_buckets_->destroy(); + wbp_index_cache_allocator->free(buf); + page_buckets_ = nullptr; + LOG_WARN("fail to prepare allocate array", KR(ret)); + } else { + is_inited_ = true; + fd_ = fd; + wbp_ = wbp; + bucket_allocator_ = wbp_index_cache_bkt_allocator; + bucket_array_allocator_ = wbp_index_cache_allocator; + left_ = 0; + right_ = -1; + size_ = 0; + capacity_ = INIT_BUCKET_ARRAY_CAPACITY; + } + return ret; +} + +void ObTmpFileWBPIndexCache::reset() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + left_ = 0; + right_ = -1; + size_ = 0; + capacity_ = 0; + if (OB_ISNULL(page_buckets_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page buckets is null", KR(ret), K(fd_)); + } else if (OB_ISNULL(bucket_allocator_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket allocator is null", KR(ret), K(fd_)); + } else { + for (int64_t i = 0; i < page_buckets_->count(); ++i) { + if (OB_NOT_NULL(page_buckets_->at(i))) { + page_buckets_->at(i)->destroy(); + bucket_allocator_->free(page_buckets_->at(i)); + page_buckets_->at(i) = nullptr; + } + } + page_buckets_->reset(); + } + } +} + +void ObTmpFileWBPIndexCache::destroy() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + reset(); + is_inited_ = false; + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + wbp_ = nullptr; + if (OB_ISNULL(page_buckets_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("page buckets is null", KR(ret), K(fd_)); + } else if (OB_ISNULL(bucket_array_allocator_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("bucket array allocator is null", KR(ret), K(fd_)); + } else { + page_buckets_->destroy(); + bucket_array_allocator_->free(page_buckets_); + page_buckets_ = nullptr; + bucket_allocator_ = nullptr; + bucket_array_allocator_ = nullptr; + } + } +} + +int ObTmpFileWBPIndexCache::push(const uint32_t page_index) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (0 == capacity_ || is_full()) { + if (max_bucket_array_capacity_ <= capacity_) { + if (OB_FAIL(sparsify_())) { + LOG_WARN("fail to sparsify", KR(ret), KPC(this)); + } + } else if (OB_FAIL(expand_())) { + LOG_WARN("fail to expand array", KR(ret), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + if (is_empty() || (OB_NOT_NULL(page_buckets_->at(right_)) && page_buckets_->at(right_)->is_full())) { + // alloc a new bucket + inc_pos_(right_); + uint64_t tenant_id = MTL_ID(); + void *buf = nullptr; + ObTmpFilePageIndexBucket* bucket = nullptr; + if (OB_ISNULL(buf = bucket_allocator_->alloc(sizeof(ObTmpFilePageIndexBucket), + lib::ObMemAttr(tenant_id, "TmpFileIdxBkt")))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory for temporary file page index bucket", + KR(ret), K(fd_), K(tenant_id), K(sizeof(ObTmpFilePageIndexBucket)), KPC(this)); + } else if (FALSE_IT(bucket = new (buf) ObTmpFilePageIndexBucket())) { + } else if (OB_FAIL(bucket->init(fd_, wbp_))) { + LOG_WARN("fail to init temporary file page index bucket", KR(ret), K(fd_), KPC(this)); + } else if (OB_FAIL(bucket->push(page_index))) { + LOG_WARN("fail to push page_index", KR(ret), K(fd_), K(page_index), KPC(this)); + } else { + page_buckets_->at(right_) = bucket; + } + + if (OB_FAIL(ret)) { + if (OB_NOT_NULL(bucket)) { + bucket->destroy(); + bucket_allocator_->free(bucket); + page_buckets_->at(right_) = nullptr; + } + dec_pos_(right_); + } else { + size_ += 1; + } + } else if (OB_ISNULL(page_buckets_->at(right_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(left_), K(right_), K(size_), K(capacity_), KPC(this)); + } else if (OB_FAIL(page_buckets_->at(right_)->push(page_index))) { // bucket is not full + LOG_WARN("fail to push page index", KR(ret), K(fd_), K(page_index), KPC(this)); + } + } + + return ret; +} + +int ObTmpFileWBPIndexCache::truncate(const int64_t truncate_page_virtual_id) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (OB_UNLIKELY(is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("array is empty", KR(ret), K(fd_), K(size_)); + } else { + const int64_t logic_begin_pos = left_; + const int64_t logic_end_pos = get_logic_tail_(); + bool truncate_over = false; + for (int64_t i = logic_begin_pos; OB_SUCC(ret) && !truncate_over && i <= logic_end_pos; i++) { + int64_t bkt_min_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + int64_t bkt_min_page_index = ObTmpFileGlobal::INVALID_PAGE_ID; + ObTmpFilePageIndexBucket *&cur_bucket = page_buckets_->at(i % capacity_); + if (OB_ISNULL(cur_bucket)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(i), K(capacity_), KP(cur_bucket), KPC(this)); + } else if (OB_UNLIKELY((bkt_min_page_index = cur_bucket->get_min_page_index()) == + ObTmpFileGlobal::INVALID_PAGE_ID)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page index", KR(ret), K(fd_), KPC(cur_bucket), KPC(this)); + } else if (OB_FAIL(wbp_->get_page_virtual_id(fd_, bkt_min_page_index, bkt_min_page_virtual_id))) { + LOG_WARN("fail to get page virtual id in file", KR(ret), K(fd_), K(bkt_min_page_index), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == bkt_min_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(fd_), K(bkt_min_page_virtual_id), KPC(this)); + } else if (i != logic_begin_pos) { + // truncate previous bucket + if (truncate_page_virtual_id < bkt_min_page_virtual_id) { + truncate_over = true; + ObTmpFilePageIndexBucket *&previous_bucket = page_buckets_->at(get_previous_pos(i % capacity_)); + if (OB_ISNULL(previous_bucket)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(i), K(capacity_),KP(previous_bucket), + K(truncate_page_virtual_id), K(logic_begin_pos), K(logic_end_pos), KPC(this)); + } else if (OB_FAIL(previous_bucket->truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate bucket", KR(ret), K(fd_), K(truncate_page_virtual_id), KPC(previous_bucket), KPC(this)); + } else if (OB_UNLIKELY(previous_bucket->is_empty())) { + // when truncate_page_virtual_id is smaller than min_page_virtual_id of cur bucket, + // there must exist at least one page in previous bucket whose virtual page id is larger than + // or equal to truncate_page_virtual_id (which means this page index doesn't need to be truncated). + // however, truncate all page indexes of previous bucket will not cause some problems. + // thus, we just allow code continue to run + previous_bucket->destroy(); + bucket_allocator_->free(previous_bucket); + previous_bucket = nullptr; + inc_pos_(left_); + size_ -= 1; + } + } else { // truncate_page_virtual_id >= bkt_min_page_virtual_id + ObTmpFilePageIndexBucket *&previous_bucket = page_buckets_->at(get_previous_pos(i % capacity_)); + if (OB_ISNULL(previous_bucket)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(i), K(capacity_),KP(previous_bucket), + K(truncate_page_virtual_id), K(logic_begin_pos), K(logic_end_pos), KPC(this)); + } else { + previous_bucket->destroy(); + bucket_allocator_->free(previous_bucket); + previous_bucket = nullptr; + inc_pos_(left_); + size_ -= 1; + } + } + } + if (i == logic_end_pos && truncate_page_virtual_id > bkt_min_page_virtual_id) { + if (FAILEDx(cur_bucket->truncate(truncate_page_virtual_id))) { + LOG_WARN("fail to truncate bucket", KR(ret), K(fd_), K(truncate_page_virtual_id), KPC(cur_bucket), KPC(this)); + } else if (cur_bucket->is_empty()) { + cur_bucket->destroy(); + bucket_allocator_->free(cur_bucket); + cur_bucket = nullptr; + inc_pos_(left_); + size_ -= 1; + } + } + } // end for + } + + if (OB_FAIL(ret)) { + } else if (is_empty()) { + reset(); + } else { + shrink_(); + } + return ret; +} + +// 1. if 'target_page_virtual_id' is smaller than virtual page id of first page in cache, +// page_index will be INVALID_PAGE_ID. +// in this case, caller should directly search write buffer pool with the beginning page index of file +// 2. if 'target_page_virtual_id' is in the range of cached pages but the page index of it doesn't exist in cache, +// the cache will find a page whose virtual id is smaller than and closest to 'target_page_virtual_id' +// and then iterate the page index of wbp from the closest page index to find the according page_index +int ObTmpFileWBPIndexCache::binary_search(const int64_t target_page_virtual_id, uint32_t &page_index) +{ + int ret = OB_SUCCESS; + page_index = ObTmpFileGlobal::INVALID_PAGE_ID; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (OB_UNLIKELY(is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("array is empty", KR(ret), K(fd_), K(size_)); + } else { + int64_t left_pos = left_; + int64_t right_pos = get_logic_tail_(); + ObTmpFilePageIndexBucket *target_bucket = nullptr; + bool find = false; + while (left_pos <= right_pos && !find && OB_SUCC(ret)) { + // find the bucket whose min_page_virtual_id is small than and closest to target_page_virtual_id + const int64_t logic_mid = left_pos + (right_pos - left_pos) / 2; + const int64_t mid = logic_mid % capacity_; + ObTmpFilePageIndexBucket *mid_bucket = page_buckets_->at(mid); + int64_t min_page_virtual_id = -1; + if (OB_ISNULL(mid_bucket)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(logic_mid), K(mid), KP(mid_bucket), K(size_), K(capacity_), KPC(this)); + } else if (OB_FAIL(wbp_->get_page_virtual_id(fd_, mid_bucket->get_min_page_index(), min_page_virtual_id))) { + LOG_WARN("fail to get page virtual id in file", KR(ret), K(fd_), K(mid_bucket->get_min_page_index()), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == min_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(fd_), K(min_page_virtual_id), KPC(this)); + } else if (min_page_virtual_id <= target_page_virtual_id) { + target_bucket = mid_bucket; + if (min_page_virtual_id == target_page_virtual_id) { + find = true; + page_index = mid_bucket->get_min_page_index(); + } else { + left_pos = logic_mid + 1; + } + } else { + right_pos = logic_mid - 1; + } + } // end while + + if (OB_FAIL(ret)) { + } else if (find) { + // do nothing + } else if (OB_ISNULL(target_bucket)) { + // page_index = ObTmpFileGlobal::INVALID_PAGE_ID; + LOG_DEBUG("the target page_index might be removed from cache", K(fd_), K(target_page_virtual_id), KPC(this)); + } else if (OB_FAIL(target_bucket->binary_search(target_page_virtual_id, page_index))) { + LOG_WARN("fail to binary search page index", KR(ret), K(fd_), K(target_page_virtual_id), KPC(this)); + } + } + return ret; +} + +int ObTmpFileWBPIndexCache::expand_() +{ + int ret = OB_SUCCESS; + LOG_DEBUG("start to expand tmp file wbp index cache", KPC(this)); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(max_bucket_array_capacity_ == capacity_)) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("cannot expand array any more", KR(ret), K(fd_), K(capacity_)); + } else if (OB_UNLIKELY(capacity_ != 0 && !is_full())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("expand array when it is not full", KR(ret), K(fd_), K(size_), K(capacity_)); + } else { + int64_t new_capacity = 0 == capacity_ ? + INIT_BUCKET_ARRAY_CAPACITY: + MIN(capacity_ * 2, max_bucket_array_capacity_); + if (OB_FAIL(page_buckets_->prepare_allocate(new_capacity, nullptr))) { + LOG_WARN("fail to prepare allocate array", KR(ret), K(fd_), K(new_capacity)); + } else if (!is_empty() && right_ < left_) { + if (OB_UNLIKELY(right_ + capacity_ >= new_capacity)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid capacity", KR(ret), K(fd_), K(right_), K(capacity_), K(new_capacity)); + } else { + for (int64_t i = 0; i <= right_; ++i) { + ObTmpFilePageIndexBucket*& cur_bucket = page_buckets_->at(i); + ObTmpFilePageIndexBucket*& new_bucket = page_buckets_->at(capacity_ + i); + new_bucket = cur_bucket; + page_buckets_->at(i) = nullptr; + } + right_ += capacity_; + } + } + + if (OB_SUCC(ret)) { + capacity_ = new_capacity; + } + } + LOG_DEBUG("expand tmp file wbp index cache over", KR(ret), KPC(this)); + return ret; +} + +void ObTmpFileWBPIndexCache::shrink_() +{ + int ret = OB_SUCCESS; + LOG_DEBUG("start to shrink tmp file wbp index cache", KPC(this)); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(get_logic_tail_() - left_ + 1 != size_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("cache pos is wrong", KR(ret), K(fd_), K(left_), K(right_), K(size_), K(capacity_)); + } else if (size_ > capacity_ / SHRINK_THRESHOLD || + INIT_BUCKET_ARRAY_CAPACITY == capacity_) { + // no need to shrink + } else { + int64_t new_capacity = capacity_ / 2; + void *buf = nullptr; + uint64_t tenant_id = MTL_ID(); + ObArray *new_buckets = nullptr; + + if (OB_UNLIKELY(size_ > new_capacity)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid new_capacity", KR(ret), K(fd_), K(left_), K(right_), K(size_), K(capacity_), K(new_capacity)); + } else if (OB_ISNULL(buf = bucket_array_allocator_->alloc(sizeof(ObArray), + lib::ObMemAttr(tenant_id, "TmpFileIdxCache")))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory for temporary file page index bucket", + KR(ret), K(fd_), K(tenant_id), K(sizeof(ObArray))); + } else if (FALSE_IT(new_buckets = new (buf) ObArray())) { + } else if (FALSE_IT(new_buckets->set_attr(lib::ObMemAttr(tenant_id, "TmpFileIdxCache")))) { + } else if (OB_FAIL(new_buckets->prepare_allocate(new_capacity, nullptr))) { + new_buckets->destroy(); + bucket_array_allocator_->free(buf); + new_buckets = nullptr; + LOG_WARN("fail to prepare allocate array", KR(ret), K(fd_)); + } else { + for (int64_t i = left_; i <= get_logic_tail_() && OB_SUCC(ret); i++) { + ObTmpFilePageIndexBucket *bucket = page_buckets_->at(i % capacity_); + new_buckets->at(i - left_) = bucket; + } // end for + page_buckets_->destroy(); + bucket_array_allocator_->free(page_buckets_); + left_ = 0; + right_ = size_ - 1; + capacity_ = new_capacity; + page_buckets_ = new_buckets; + LOG_DEBUG("successfully shrink tmp file page index cache", K(fd_), K(size_), K(capacity_), KPC(this)); + } + } + LOG_DEBUG("shrink tmp file wbp index cache over", KR(ret), KPC(this)); +} + +int ObTmpFileWBPIndexCache::sparsify_() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(max_bucket_array_capacity_ != capacity_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected capacity", KR(ret), K(fd_), K(capacity_)); + } else if (OB_UNLIKELY(capacity_ % 2 != 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("capacity should be even", KR(ret), K(fd_), K(capacity_)); + } else if (OB_UNLIKELY(!is_full())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("cache is not full when try to eliminate half pages", KR(ret), K(fd_), K(size_), K(capacity_)); + } else { + int64_t cur_bucket_pos = left_; + for (int64_t i = left_; i <= get_logic_tail_() && OB_SUCC(ret); i++) { + if (OB_ISNULL(page_buckets_->at(i % capacity_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", KR(ret), K(fd_), K(cur_bucket_pos), K(i), KP(page_buckets_->at(i % capacity_))); + } else if (OB_FAIL(page_buckets_->at(i % capacity_)->shrink_half())) { + LOG_WARN("fail to shrink half", KR(ret), K(fd_), K(cur_bucket_pos), K(i), KP(page_buckets_->at(i % capacity_))); + } else if ((i - left_) % 2 == 0) { + page_buckets_->at(cur_bucket_pos % capacity_) = page_buckets_->at(i % capacity_); + } else { + if (OB_FAIL(page_buckets_->at(cur_bucket_pos % capacity_)->merge(*page_buckets_->at(i % capacity_)))) { + LOG_WARN("fail to merge two buckets", KR(ret), K(fd_), K(cur_bucket_pos), K(i), + KPC(page_buckets_->at(cur_bucket_pos % capacity_)), KP(page_buckets_->at(i % capacity_))); + } else { + page_buckets_->at(i % capacity_)->destroy(); + bucket_allocator_->free(page_buckets_->at(i % capacity_)); + cur_bucket_pos++; + } + } + + if (OB_SUCC(ret) && i > (get_logic_tail_() - left_) / 2 + left_) { + page_buckets_->at(i % capacity_) = nullptr; + } + } + + if (OB_SUCC(ret)) { + right_ = (cur_bucket_pos - 1) % capacity_; + size_ /= 2; + } + } + LOG_INFO("sparsify tmp file wbp index cache over", KR(ret), KPC(this)); + return ret; +} + +ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::ObTmpFilePageIndexBucket() : + page_indexes_(), fd_(ObTmpFileGlobal::INVALID_TMP_FILE_FD), + wbp_(nullptr), min_page_index_(ObTmpFileGlobal::INVALID_PAGE_ID) {} + +ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::~ObTmpFilePageIndexBucket() +{ + destroy(); +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::init(int64_t fd, ObTmpWriteBufferPool* wbp) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_UNLIKELY(fd == ObTmpFileGlobal::INVALID_TMP_FILE_FD) || OB_ISNULL(wbp)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), KP(wbp)); + } else if (FALSE_IT(page_indexes_.set_attr(ObMemAttr(MTL_ID(), "TmpFileIdxBkt")))) { + } else if (OB_FAIL(page_indexes_.prepare_allocate(BUCKET_CAPACITY, ObTmpFileGlobal::INVALID_PAGE_ID))) { + LOG_WARN("fail to prepare allocate array", KR(ret)); + } else { + is_inited_ = true; + fd_ = fd; + wbp_ = wbp; + min_page_index_ = ObTmpFileGlobal::INVALID_PAGE_ID; + left_ = 0; + right_ = -1; + size_ = 0; + capacity_ = BUCKET_CAPACITY; + } + return ret; +} + +void ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::destroy() +{ + is_inited_ = false; + fd_ = ObTmpFileGlobal::INVALID_TMP_FILE_FD; + wbp_ = nullptr; + left_ = 0; + right_ = -1; + size_ = 0; + capacity_ = 0; + min_page_index_ = ObTmpFileGlobal::INVALID_PAGE_ID; + page_indexes_.reset(); +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::push(const uint32_t page_index) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret), KPC(this)); + } else if (OB_UNLIKELY(is_full())) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("bucket is full", KR(ret), K(size_), K(capacity_), KPC(this)); + } else { + if (OB_UNLIKELY(is_empty())) { + min_page_index_ = page_index; + } + inc_pos_(right_); + page_indexes_[right_] = page_index; + size_ += 1; + } + + return ret; +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::pop_() +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret), K(fd_)); + } else if (OB_UNLIKELY(is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("pop a empty array", KR(ret), K(size_), K(fd_)); + } else { + page_indexes_[left_] = ObTmpFileGlobal::INVALID_PAGE_ID; + inc_pos_(left_); + size_ -= 1; + + if (is_empty()) { + destroy(); + } else { + min_page_index_ = page_indexes_[left_]; + } + } + return ret; +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::truncate(const int64_t truncate_page_virtual_id) +{ + int ret = OB_SUCCESS; + + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (OB_UNLIKELY(is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt binary search in an empty array", KR(ret), K(size_)); + } else { + const int64_t logic_begin_pos = left_; + const int64_t logic_end_pos = get_logic_tail_(); + bool truncate_over = false; + for (int64_t i = logic_begin_pos; OB_SUCC(ret) && !truncate_over && i <= logic_end_pos; i++) { + int64_t page_virtual_id = -1; + if (OB_UNLIKELY(page_indexes_[i % capacity_] == ObTmpFileGlobal::INVALID_PAGE_ID)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page index", KR(ret), K(page_indexes_[i % capacity_]), KPC(this)); + } else if (OB_FAIL(wbp_->get_page_virtual_id(fd_, page_indexes_[i % capacity_], page_virtual_id))) { + LOG_WARN("fail to get page virtual id in file", KR(ret), K(page_indexes_[i % capacity_]), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(page_virtual_id), KPC(this)); + } else if (page_virtual_id < truncate_page_virtual_id) { + if (OB_FAIL(pop_())) { + LOG_WARN("fail to pop", KR(ret), KPC(this)); + } + } else { + truncate_over = true; + } + } // end for + } + return ret; +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::binary_search( + const int64_t target_page_virtual_id, uint32_t &page_index) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (OB_UNLIKELY(is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt binary search in an empty array", KR(ret), K(size_)); + } else { + bool find = false; + int64_t left_pos = left_; + int64_t right_pos = get_logic_tail_(); + uint32_t target_page_index = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t cur_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + while (left_pos <= right_pos && !find && OB_SUCC(ret)) { + const int64_t logic_mid = left_pos + (right_pos - left_pos) / 2; + const int64_t mid = logic_mid % capacity_; + const uint32_t mid_page = page_indexes_[mid]; + if (OB_FAIL(wbp_->get_page_virtual_id(fd_, mid_page, cur_page_virtual_id))) { + LOG_WARN("fail to get page virtual id in file", KR(ret), K(logic_mid), K(mid), K(mid_page), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == cur_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(cur_page_virtual_id), KPC(this)); + } else if (cur_page_virtual_id <= target_page_virtual_id) { + target_page_index = mid_page; + if (cur_page_virtual_id == target_page_virtual_id) { + find = true; + } else { + left_pos = logic_mid + 1; + } + } else { + right_pos = logic_mid - 1; + } + } // end while + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == target_page_index)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("cannot find the target page index", KR(ret), K(page_indexes_[left_]), + K(target_page_index), + K(target_page_virtual_id), KPC(this)); + } else if (find) { + page_index = target_page_index; + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == target_page_index)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("target page index doesn't exist in bucket", KR(ret), K(left_), K(page_indexes_[left_]), + K(target_page_index), + K(target_page_virtual_id), KPC(this)); + } else { + int64_t cur_page_index = target_page_index; + while (OB_SUCC(ret) && !find) { + uint32_t next_page_index = ObTmpFileGlobal::INVALID_PAGE_ID; + int64_t page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + if (OB_FAIL(wbp_->get_page_virtual_id(fd_, cur_page_index, cur_page_virtual_id))) { + LOG_WARN("fail to get virtual page id", KR(ret), K(fd_), K(cur_page_index), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == cur_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(cur_page_virtual_id), KPC(this)); + } else if (OB_FAIL(wbp_->get_next_page_id(fd_, cur_page_index, ObTmpFilePageUniqKey(cur_page_virtual_id), + next_page_index))) { + LOG_WARN("fail to get next page id", KR(ret), K(fd_), K(cur_page_index), K(cur_page_virtual_id), KPC(this)); + } else if (ObTmpFileGlobal::INVALID_PAGE_ID == next_page_index) { + ret = OB_ENTRY_NOT_EXIST; + LOG_WARN("attempt to find a non-existent page in write buffer pool", KR(ret), + K(target_page_virtual_id), KPC(this)); + } else if (OB_FAIL(wbp_->get_page_virtual_id(fd_, next_page_index, page_virtual_id))) { + LOG_WARN("fail to get virtual page id", KR(ret), K(fd_), K(next_page_index), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(page_virtual_id), KPC(this)); + } else if (page_virtual_id == target_page_virtual_id) { + page_index = next_page_index; + find = true; + } else if (page_virtual_id < target_page_virtual_id) { + cur_page_index = next_page_index; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("lose target page in wbp", KR(ret), K(next_page_index), K(page_virtual_id), + K(target_page_virtual_id), KPC(this)); + } + } // end while + } + } + return ret; +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::shrink_half() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(capacity_ % 2 != 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("capacity_ should be even", KR(ret), K(capacity_), KPC(this)); + } else if (size_ <= capacity_ / 2) { + // no need to shrink, do nothing + } else { + const int64_t eliminate_num = size_ - capacity_ / 2; + const int64_t eliminate_modulus = size_ / eliminate_num; + int64_t remain_eliminate_num = eliminate_num; + int64_t logic_pos = left_; + for (int64_t i = left_; i <= get_logic_tail_(); i++) { + if ((i - left_) % eliminate_modulus != 0 || 0 == remain_eliminate_num) { + page_indexes_[logic_pos % capacity_] = page_indexes_[i % capacity_]; + logic_pos++; + } else { // eliminate current page index + remain_eliminate_num--; + } + } + + for (int64_t i = logic_pos; i <= get_logic_tail_(); i++) { + page_indexes_[i % capacity_] = ObTmpFileGlobal::INVALID_PAGE_ID; + } + + right_ = (logic_pos - 1) % capacity_; + size_ -= eliminate_num; + + if (!is_empty()) { + min_page_index_ = page_indexes_[left_]; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("tmp file page index bucket is unexpected empty", KR(ret), KPC(this)); + } + } + return ret; +} + +int ObTmpFileWBPIndexCache::ObTmpFilePageIndexBucket::merge(ObTmpFilePageIndexBucket& other) +{ + int ret = OB_SUCCESS; + + if (OB_UNLIKELY(other.is_empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to merge an empty array", KR(ret), K(other.size_), KPC(this)); + } else if (OB_UNLIKELY(other.size_ + size_ > capacity_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to merge array with too many elements", KR(ret), K(size_), K(other.size_), K(capacity_), KPC(this)); + } else if (OB_UNLIKELY(!is_empty())) { + int64_t bkt_min_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + int64_t other_bkt_min_page_virtual_id = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + if (OB_FAIL(wbp_->get_page_virtual_id(fd_, min_page_index_, bkt_min_page_virtual_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(bkt_min_page_virtual_id), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == bkt_min_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(bkt_min_page_virtual_id), KPC(this)); + } else if (OB_FAIL(wbp_->get_page_virtual_id(fd_, other.min_page_index_, other_bkt_min_page_virtual_id))) { + LOG_WARN("fail to get page virtual id", KR(ret), K(other_bkt_min_page_virtual_id), KPC(this)); + } else if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == other_bkt_min_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected invalid page virtual id", KR(ret), K(other_bkt_min_page_virtual_id), KPC(this)); + } else if (OB_UNLIKELY(bkt_min_page_virtual_id > other_bkt_min_page_virtual_id)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("attempt to merge a bucket whose min_page_virtual_id is less that itself", KR(ret), + K(min_page_index_), K(other_bkt_min_page_virtual_id), KPC(this)); + } + } + + if (OB_SUCC(ret)) { + for (int64_t i = other.left_; OB_SUCC(ret) && i <= other.get_logic_tail_(); i++) { + uint32_t page_index = other.page_indexes_[i % other.capacity_]; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == page_index)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid page index", KR(ret), K(page_index), K(i), KPC(this)); + } else if (OB_FAIL(push(page_index))) { + LOG_WARN("fail to push a page index", KR(ret), K(page_index), KPC(this)); + } + } + } + return ret; +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h b/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h new file mode 100644 index 0000000000..3650ebff50 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_write_buffer_index_cache.h @@ -0,0 +1,145 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_WRITE_BUFFER_INDEX_CACHE_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_WRITE_BUFFER_INDEX_CACHE_H_ + +#include "lib/container/ob_array.h" +#include "lib/allocator/ob_fifo_allocator.h" + +namespace oceanbase +{ +namespace tmp_file +{ +class ObTmpWriteBufferPool; + +class ObTmpFileCircleArray +{ +public: + ObTmpFileCircleArray() : is_inited_(false), left_(0), right_(-1), size_(0), capacity_(0) {} + virtual ~ObTmpFileCircleArray() {}; + virtual int push(const uint32_t page_index) = 0; + virtual int truncate(const int64_t truncate_page_virtual_id) = 0; + virtual int binary_search(const int64_t target_page_virtual_id, uint32_t &page_index) = 0; + +public: + OB_INLINE int64_t size() const { return size_; } + OB_INLINE bool is_empty() const { return size_ == 0; } + virtual OB_INLINE bool is_full() const { return size_ == capacity_; } + TO_STRING_KV(K(is_inited_), K(left_), K(right_), K(size_), K(capacity_)); + +protected: + OB_INLINE void inc_pos_(int64_t &pos) { pos = pos == capacity_ - 1 ? 0 : pos + 1; } + OB_INLINE void dec_pos_(int64_t &pos) { pos = pos == 0 ? capacity_ - 1 : pos - 1; } + OB_INLINE int64_t get_previous_pos(const int64_t pos) const { return pos == 0 ? capacity_ - 1 : pos - 1; } + OB_INLINE int64_t get_logic_tail_() const { return get_logic_pos_(right_); } + OB_INLINE int64_t get_logic_pos_(const int64_t pos) const + { + return pos >= left_ ? pos : capacity_ + pos; + } + +protected: + bool is_inited_; + int64_t left_; // close interval + int64_t right_; // close interval + int64_t size_; // record valid number of array + int64_t capacity_; // record the capacity of array +}; + +// Attention: +// ObTmpFileWBPIndexCache caches page indexes of a tmp file in write buffer pool. +// When ObTmpFileWBPIndexCache reaches the limitation of capacity, +// it will eliminates half page indexes of buckets for caching new page index. +// thus, the range of cached indexes might be not continuous. +class ObTmpFileWBPIndexCache : public ObTmpFileCircleArray +{ +public: + ObTmpFileWBPIndexCache(); + ~ObTmpFileWBPIndexCache(); + int init(const int64_t fd, ObTmpWriteBufferPool* wb, + ObIAllocator *wbp_index_cache_allocator, + ObIAllocator *wbp_index_cache_bkt_allocator); + void destroy(); + void reset(); + virtual int push(const uint32_t page_index) override; + // truncate page virtual id is open interval + virtual int truncate(const int64_t truncate_page_virtual_id) override; + // truncate page virtual id is close interval + virtual int binary_search(const int64_t target_page_virtual_id, uint32_t &page_index) override; + virtual OB_INLINE bool is_full() const override + { + return ObTmpFileCircleArray::is_full() && + right_ >= 0 && right_ < capacity_ && + OB_NOT_NULL(page_buckets_) && + OB_NOT_NULL(page_buckets_->at(right_)) && + page_buckets_->at(right_)->is_full(); + } + INHERIT_TO_STRING_KV("ObTmpFileCircleArray", ObTmpFileCircleArray, + K(fd_), KP(wbp_), KP(page_buckets_), + KP(bucket_array_allocator_), KP(bucket_allocator_)); + +private: + class ObTmpFilePageIndexBucket : public ObTmpFileCircleArray + { + public: + ObTmpFilePageIndexBucket(); + ~ObTmpFilePageIndexBucket(); + int init(int64_t fd, ObTmpWriteBufferPool* wbp); + void destroy(); + virtual int push(const uint32_t page_index) override; + // truncate offset is open interval + virtual int truncate(const int64_t truncate_page_virtual_id) override; + // target offset is close interval + virtual int binary_search(const int64_t target_page_virtual_id, uint32_t &page_index) override; + int shrink_half(); + int merge(ObTmpFilePageIndexBucket& other); + OB_INLINE int64_t get_min_page_index() const { return min_page_index_; } + INHERIT_TO_STRING_KV("ObTmpFileCircleArray", ObTmpFileCircleArray, + K(fd_), KP(wbp_), K(min_page_index_)); + private: + int pop_(); + private: + // due to each page index in bucket directs to a 8KB page in wbp, + // a bucket could indicate a 256KB data and hold on 128B memory + static const int64_t BUCKET_CAPACITY = 1 << 5; + + private: + common::ObArray page_indexes_; + int64_t fd_; + ObTmpWriteBufferPool* wbp_; + int64_t min_page_index_; + }; + +private: + // due to each bucket could indicate a 256KB data in wbp, + // the cache could indicate a 256MB data and hold on (128 + 8)*2^10 = 136KB memory at most + static const int64_t MAX_BUCKET_ARRAY_CAPACITY = 1 << 10; + static const int64_t INIT_BUCKET_ARRAY_CAPACITY = 1 << 3; // indicates 2MB data in wbp at most + static const int64_t SHRINK_THRESHOLD = 4; // attention: this value must be larger than 2 + +private: + int expand_(); + void shrink_(); + int sparsify_(); + +private: + ObIAllocator *bucket_array_allocator_; + ObIAllocator *bucket_allocator_; + common::ObArray *page_buckets_; + int64_t fd_; + ObTmpWriteBufferPool* wbp_; + int64_t max_bucket_array_capacity_; // only allowed to modify this var in unit test!!! +}; + +} // end namespace tmp_file +} // end namespace oceanbase +#endif // OCEANBASE_STORAGE_BLOCKSSTABLE_TMP_FILE_OB_TMP_FILE_WRITE_BUFFER_INDEX_CACHE_H_ diff --git a/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.cpp b/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.cpp new file mode 100644 index 0000000000..13190e11f5 --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.cpp @@ -0,0 +1,988 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX STORAGE + +#include "share/config/ob_server_config.h" +#include "storage/tmp_file/ob_tmp_file_write_buffer_pool.h" +#include "observer/omt/ob_tenant_config_mgr.h" +#include "storage/blocksstable/ob_block_manager.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +int ObPageEntry::switch_state(const int64_t op) +{ + int ret = OB_SUCCESS; + static const int64_t N = State::N; + static const int64_t INV = State::INVALID; + static const int64_t INITED = State::INITED; + static const int64_t LOADING = State::LOADING; + static const int64_t CACHED = State::CACHED; + static const int64_t DIRTY = State::DIRTY; + static const int64_t WRITE_BACK = State::WRITE_BACK; + static const int64_t MAX = State::MAX; + + static const int64_t STATE_MAP[State::MAX][Ops::MAX] = { + // ALLOC, LOAD, LOAD_FAIL, LOAD_SUCC, DELETE, WRITE, WRITE_BACK, WRITE_BACK_FAILED, WRITE_BACK_SUCC + {INITED, N, N, N, INV, N, N, N, N}, //INVALID + {N, LOADING, N, N, INV, DIRTY, N, N, N}, //INITED + {N, N, INITED, CACHED, N, N, N, N, N}, //LOADING + {N, N, N, N, INV, DIRTY, N, N, CACHED}, //CACHED + {N, N, N, N, INV, DIRTY, WRITE_BACK, N, N}, //DIRTY + {N, N, N, N, INV, DIRTY, WRITE_BACK, DIRTY, CACHED} //WRITE_BACK + }; + + if (OB_UNLIKELY(!Ops::is_valid(op))) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("invalid operation", KR(ret), K(op)); + } else if (OB_UNLIKELY(!State::is_valid(state_))) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid state", KR(ret), K(state_)); + } else { + const int64_t n_stat = STATE_MAP[state_][op]; + if (OB_UNLIKELY(!State::is_valid(n_stat))) { + ret = OB_STATE_NOT_MATCH; + LOG_ERROR("invalid state transition", KR(ret), K(state_), K(op), K(n_stat)); + } else { + state_ = n_stat; + } + } + return ret; +} + +double ObTmpWriteBufferPool::MAX_DATA_PAGE_USAGE_RATIO = 0.9; + +ObTmpWriteBufferPool::ObTmpWriteBufferPool() + : fat_(), + lock_(), + allocator_(), + is_inited_(false), + capacity_(0), + dirty_page_num_(0), + used_page_num_(0), + first_free_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + wbp_memory_limit_(-1), + default_wbp_memory_limit_(-1), + last_access_tenant_config_ts_(-1), + meta_page_cnt_(0), + data_page_cnt_(0), + dirty_meta_page_cnt_(0), + dirty_data_page_cnt_(0), + write_back_data_cnt_(0), + write_back_meta_cnt_(0) +{ +} + +ObTmpWriteBufferPool::~ObTmpWriteBufferPool() +{ + destroy(); +} + +int ObTmpWriteBufferPool::init() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("fail to init wbp, init twice", KR(ret), K(is_inited_)); + } else if (OB_FAIL(allocator_.init( + lib::ObMallocAllocator::get_instance(), OB_MALLOC_BIG_BLOCK_SIZE, + ObMemAttr(MTL_ID(), "TmpFileWBP", ObCtxIds::DEFAULT_CTX_ID)))) { + LOG_WARN("wbp fail to init fifo allocator", KR(ret)); + } else if (FALSE_IT(fat_.set_attr(ObMemAttr(MTL_ID(), "TmpFileWBP")))) { + } else if (OB_FAIL(expand_())) { + LOG_WARN("wbp fail to expand capacity", KR(ret)); + } else { + is_inited_ = true; + } + return ret; +} + +void ObTmpWriteBufferPool::destroy() +{ + reduce_(); + capacity_ = 0; + dirty_page_num_ = 0; + used_page_num_ = 0; + first_free_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + last_access_tenant_config_ts_ = -1; + data_page_cnt_ = 0; + meta_page_cnt_ = 0; + dirty_meta_page_cnt_ = 0; + dirty_data_page_cnt_ = 0; + write_back_data_cnt_ = 0; + write_back_meta_cnt_ = 0; + wbp_memory_limit_ = -1; + default_wbp_memory_limit_ = -1; + fat_.destroy(); + allocator_.reset(); + is_inited_ = false; +} + +// limit data pages to use a maximum of 90% of the total space in the write buffer pool; +// considering that the total amount of meta pages for a single file accounts for less than 1% of data pages, +// there is no limit set for meta pages when allocating pages. +int ObTmpWriteBufferPool::inner_alloc_page_(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&new_page_buf) +{ + int ret = OB_SUCCESS; + uint32_t curr_first_free_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + uint32_t next_first_free_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + common::TCRWLock::RLockGuard guard(lock_); + // limit page allocation depending on page type + if (has_free_page_(page_key.type_)) { + // fetch a page from the free list through CAS operation + bool cas_succeed = false; + do { + curr_first_free_page_id = ATOMIC_LOAD(&first_free_page_id_); + if (!is_valid_page_id_(curr_first_free_page_id)) { + ret = OB_SEARCH_NOT_FOUND; + break; + } + next_first_free_page_id = fat_[curr_first_free_page_id].next_page_id_; + cas_succeed = ATOMIC_BCAS(&first_free_page_id_, curr_first_free_page_id, next_first_free_page_id); + } while (OB_SUCC(ret) && !cas_succeed); + + if (OB_SUCC(ret) && is_valid_page_id_(curr_first_free_page_id)) { + fat_[curr_first_free_page_id].fd_ = fd; + fat_[curr_first_free_page_id].next_page_id_ = ObTmpFileGlobal::INVALID_PAGE_ID; + fat_[curr_first_free_page_id].page_key_ = page_key; + fat_[curr_first_free_page_id].switch_state(ObPageEntry::Ops::ALLOC); + new_page_id = curr_first_free_page_id; + new_page_buf = fat_[new_page_id].buf_; + ATOMIC_INC(&used_page_num_); + } + } + + if (ObTmpFileGlobal::INVALID_PAGE_ID == new_page_id) { + ret = OB_SEARCH_NOT_FOUND; + } + + return ret; +} + +int ObTmpWriteBufferPool::alloc_page_(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&new_page_buf) +{ + int ret = OB_SUCCESS; + + new_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + new_page_buf = nullptr; + + // validate input argument. + if (ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("wbp fail to alloc_page, invalid fd", KR(ret), K(fd)); + } + + int64_t memory_limit = 0; + int64_t current_capacity = -1; + // continuously trying to allocate page and expand pool size, until capacity reach maximum memory limit. + while (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID == new_page_id && current_capacity < memory_limit) { + if (OB_FAIL(inner_alloc_page_(fd, page_key, new_page_id, new_page_buf))) { + if (OB_SEARCH_NOT_FOUND != ret) { + LOG_WARN("wbp fail to inner alloc page", KR(ret), K(fd), K(page_key), K(new_page_id), KP(new_page_buf)); + } else { // no free pages, try to expand pool size + ret = OB_SUCCESS; + memory_limit = get_memory_limit(); + current_capacity = ATOMIC_LOAD(&capacity_); + if (current_capacity < memory_limit && OB_FAIL(expand_())) { + LOG_WARN("wbp fail to expand", KR(ret), K(fd), K(ATOMIC_LOAD(&capacity_))); + } + } + } + } + + if (OB_SUCC(ret) && ObTmpFileGlobal::INVALID_PAGE_ID == new_page_id) { + ret = OB_ALLOCATE_TMP_FILE_PAGE_FAILED; // reaches maximum memory limit, can not allocate page + } + + return ret; +} + +int ObTmpWriteBufferPool::alloc_page(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&new_page_buf) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || !page_key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_key)); + } else if (OB_FAIL(alloc_page_(fd, page_key, new_page_id, new_page_buf))) { + LOG_WARN("wbp fail to alloc page", KR(ret), K(fd), K(page_key)); + } else if (page_key.type_ == PageEntryType::META) { + ATOMIC_INC(&meta_page_cnt_); + LOG_INFO("alloc meta page", KR(ret), K(new_page_id), K(fd)); + } else { + ATOMIC_INC(&data_page_cnt_); + LOG_DEBUG("alloc data page", KR(ret), K(new_page_id), K(fd)); + } + return ret; +} + +int ObTmpWriteBufferPool::get_next_page_id( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + uint32_t &next_page_id) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + !is_valid_page_id_(page_id) || !page_key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key)); + } else if (OB_UNLIKELY(fd != fat_[page_id].fd_)) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("fd not match", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else if (OB_UNLIKELY(page_key != fat_[page_id].page_key_)) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("page key not match", KR(ret), K(page_key), K(page_id), K(fat_[page_id])); + } else { + next_page_id = ATOMIC_LOAD(&fat_[page_id].next_page_id_); + } + return ret; +} + +int ObTmpWriteBufferPool::read_page( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + char *&buf, + uint32_t &next_page_id) +{ + int ret = OB_SUCCESS; + buf = nullptr; + next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + common::TCRWLock::RLockGuard guard(lock_); + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + !is_valid_page_id_(page_id) || + !page_key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("wbp fail to read page, invalid page id", KR(ret), K(page_id), K(fd), K(page_key)); + } else if (OB_UNLIKELY(fd != fat_[page_id].fd_ || page_key != fat_[page_id].page_key_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp fail to fetch page, PageEntry fd or offset not equal", KR(ret), K(page_id), K(fd), + K(page_key), K(fat_[page_id])); + } else { + buf = fat_[page_id].buf_; + next_page_id = fat_[page_id].next_page_id_; + } + return ret; +} + +int ObTmpWriteBufferPool::get_page_id_by_virtual_id(const int64_t fd, + const int64_t virtual_page_id, + const uint32_t begin_page_id, + uint32_t &page_id) +{ + int ret = OB_SUCCESS; + page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID == virtual_page_id || + !is_valid_page_id_(begin_page_id) || + fd != fat_[begin_page_id].fd_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(virtual_page_id), K(begin_page_id)); + } else if (virtual_page_id < fat_[begin_page_id].page_key_.virtual_page_id_) { + ret = OB_SEARCH_NOT_FOUND; + LOG_WARN("virtual_page_id is smaller than that of page of begin_page_id", + KR(ret), K(virtual_page_id), K(begin_page_id), + K(fat_[begin_page_id].page_key_.virtual_page_id_)); + } else { + uint32_t cur_page_id = begin_page_id; + while (cur_page_id != ObTmpFileGlobal::INVALID_PAGE_ID) { // iter to the end of this file + if (fat_[cur_page_id].page_key_.virtual_page_id_ >= virtual_page_id) { + if (fat_[cur_page_id].page_key_.virtual_page_id_ >= virtual_page_id) { + page_id = cur_page_id; + } + break; + } else { + cur_page_id = fat_[cur_page_id].next_page_id_; + } + } + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_PAGE_ID == page_id)) { + ret = OB_ITER_END; + LOG_WARN("wbp fail to find page by given offset", KR(ret), K(virtual_page_id), K(begin_page_id), K(fat_[begin_page_id])); + } + } + return ret; +} + +int ObTmpWriteBufferPool::get_page_virtual_id(const int64_t fd, const uint32_t page_id, int64_t &virtual_page_id) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || !is_valid_page_id_(page_id))) { + ret = OB_SEARCH_NOT_FOUND; + LOG_WARN("wbp fail to get page offset in file, invalid page id", KR(ret), K(fd), K(page_id)); + } else if (fd != fat_[page_id].fd_) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("wbp fail to get page offset in file, fd not match", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else { + virtual_page_id = fat_[page_id].page_key_.virtual_page_id_; + } + return ret; +} + +int ObTmpWriteBufferPool::truncate_page(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + const int64_t truncate_size) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + !is_valid_page_id_(page_id) || !page_key.is_valid() || + truncate_size > ObTmpFileGlobal::PAGE_SIZE || truncate_size <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(truncate_size)); + } else if (fd != fat_[page_id].fd_) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("wbp fail to truncate page, fd not match", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else if (page_key != fat_[page_id].page_key_) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("wbp fail to truncate page, page_key not match", KR(ret), K(page_key), K(page_id), K(fat_[page_id])); + } else { + MEMSET(fat_[page_id].buf_, 0, truncate_size); + } + return ret; +} + +int ObTmpWriteBufferPool::link_page( + const int64_t fd, + const uint32_t page_id, + const uint32_t prev_page_id, + const ObTmpFilePageUniqKey prev_page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + !is_valid_page_id_(page_id) || !is_valid_page_id_(prev_page_id) || + !prev_page_key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(page_id), K(prev_page_id), K(prev_page_key)); + } else if (OB_UNLIKELY(fat_[page_id].fd_ != fd || fat_[prev_page_id].fd_ != fd || + fat_[prev_page_id].next_page_id_ != ObTmpFileGlobal::INVALID_PAGE_ID || + prev_page_key != fat_[prev_page_id].page_key_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("fail to link page, unexpected page id or offset", KR(ret), K(fd), + K(page_id), K(fat_[page_id]), K(prev_page_key), K(prev_page_id), + K(fat_[prev_page_id])); + } else if (prev_page_key.type_ == PageEntryType::META) { + //just for meta page check + ObTmpFilePageUniqKey page_key(prev_page_key.tree_level_, prev_page_key.level_page_index_ + 1); + if (OB_UNLIKELY(prev_page_key != fat_[prev_page_id].page_key_ + || page_key != fat_[page_id].page_key_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("fail to link meta page, unexpected page offset", KR(ret), K(fd), K(prev_page_key), K(page_key), + K(fat_[prev_page_id]), K(fat_[page_id])); + } + } + + if (OB_SUCC(ret)) { + fat_[prev_page_id].next_page_id_ = page_id; + if (prev_page_key.type_ == PageEntryType::META) { + LOG_INFO("link meta page", KR(ret), K(fd), K(page_id), K(prev_page_id)); + } else { + LOG_DEBUG("link data page", KR(ret), K(fd), K(page_id), K(prev_page_id)); + } + } + return ret; +} + +int ObTmpWriteBufferPool::free_page( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + uint32_t &next_page_id) +{ + int ret = OB_SUCCESS; + next_page_id = ObTmpFileGlobal::INVALID_PAGE_ID; + common::TCRWLock::RLockGuard guard(lock_); + + if (OB_UNLIKELY(ObTmpFileGlobal::INVALID_TMP_FILE_FD == fd || + !is_valid_page_id_(page_id) || + !page_key.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key)); + } else if (OB_UNLIKELY(fd != fat_[page_id].fd_ + || page_key != fat_[page_id].page_key_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp fail to free page, fd or offset not equal", KR(ret), K(page_id), K(fd), + K(page_key), K(fat_[page_id])); + } else { + // reset PageEntry, add it to free list head and update buffer pool statistics + next_page_id = ATOMIC_LOAD(&(fat_[page_id].next_page_id_)); + PageEntryType page_type = fat_[page_id].page_key_.type_; + if (PageEntryType::DATA == page_type) { + LOG_DEBUG("free data page", KR(ret), K(page_id), K(fd), K(fat_[page_id])); + } else { + LOG_INFO("free meta page", KR(ret), K(page_id), K(fd), K(fat_[page_id])); + } + ATOMIC_SET(&(fat_[page_id].fd_), -1); + ATOMIC_SET(&(fat_[page_id].next_page_id_), ObTmpFileGlobal::INVALID_PAGE_ID); + if (ObPageEntry::State::DIRTY == ATOMIC_LOAD(&fat_[page_id].state_)) { + ATOMIC_DEC(&dirty_page_num_); + if (PageEntryType::DATA == page_type) { + ATOMIC_DEC(&dirty_data_page_cnt_); + } else if (PageEntryType::META == page_type) { + ATOMIC_DEC(&dirty_meta_page_cnt_); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid tmp file page type", KR(ret), K(page_id), K(fd), K(page_type)); + } + } + if (ObPageEntry::State::WRITE_BACK == ATOMIC_LOAD(&fat_[page_id].state_)) { + if (PageEntryType::DATA == page_type) { + ATOMIC_DEC(&write_back_data_cnt_); + } else if (PageEntryType::META == page_type) { + ATOMIC_DEC(&write_back_meta_cnt_); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid tmp file page type", KR(ret), K(page_id), K(fd), K(page_type)); + } + } + fat_[page_id].switch_state(ObPageEntry::Ops::DELETE); + fat_[page_id].page_key_.reset(); + + bool cas_succeed = false; + do { + uint32_t first_free_page_id_before = ATOMIC_LOAD(&first_free_page_id_); + ATOMIC_SET(&(fat_[page_id].next_page_id_), first_free_page_id_before); + cas_succeed = ATOMIC_BCAS(&first_free_page_id_, first_free_page_id_before, page_id); + } while (false == cas_succeed); + + ATOMIC_DEC(&used_page_num_); + + if (PageEntryType::DATA == page_type) { + ATOMIC_DEC(&data_page_cnt_); + } else if (PageEntryType::META == page_type) { + ATOMIC_DEC(&meta_page_cnt_); + } else { + LOG_ERROR("invalid tmp file page type", KR(ret), K(page_id), K(fd), K(page_type)); + } + } + return ret; +} + +// allocate a block size of WBP_BLOCK_SIZE each iteration +// therefore max capacity may slightly exceed memory_limit +int ObTmpWriteBufferPool::expand_() +{ + int ret = OB_SUCCESS; + + // expand the buffer pool to twice the current size, with a minimum of WBP_BLOCK_SIZE + const int64_t memory_limit = get_memory_limit(); + int64_t current_capacity = ATOMIC_LOAD(&capacity_); + const int64_t expect_capacity = std::min( + memory_limit, std::max(current_capacity * 2, int64_t(WBP_BLOCK_SIZE))); + + // continuously allocate 2MB blocks and add them into the buffer pool. + while (OB_SUCC(ret) && current_capacity < expect_capacity) { + common::TCRWLock::WLockGuard guard(lock_); + current_capacity = ATOMIC_LOAD(&capacity_); + if (current_capacity < expect_capacity) { + char * new_expand_buf = nullptr; + // allocate a chunk of WBP_BLOCK_SIZE each time + if (OB_ISNULL(new_expand_buf = static_cast(allocator_.alloc(WBP_BLOCK_SIZE)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("wbp fail to allocate new expand buffer", KR(ret)); + } else { + uint32_t new_page_id = fat_.count(); + for (uint32_t count = 0; OB_SUCC(ret) && count < BLOCK_PAGE_NUMS; ++new_page_id, ++count) { + if (OB_FAIL(fat_.push_back(ObPageEntry(-1, + ObTmpFileGlobal::INVALID_PAGE_ID, + new_expand_buf + count * ObTmpFileGlobal::PAGE_SIZE)))) { + LOG_WARN("wbp fail to push back page into fat", KR(ret), K(count), K(new_page_id)); + } else { + fat_[new_page_id].next_page_id_ = ATOMIC_LOAD(&first_free_page_id_); + ATOMIC_SET(&first_free_page_id_, new_page_id); + ATOMIC_FAA(&capacity_, ObTmpFileGlobal::PAGE_SIZE); + } + } + current_capacity += WBP_BLOCK_SIZE; + } + } else { + // maybe another thread has finish allocation, do nothing. + } + } + + LOG_INFO("wbp expand", K(expect_capacity), K(memory_limit), K(ATOMIC_LOAD(&capacity_))); + + return ret; +} + +int ObTmpWriteBufferPool::reduce_() +{ + int ret = OB_SUCCESS; + // TODO(wendongbo): write buffer pool shrinking is currently not supported, use it as destroy() now + common::TCRWLock::WLockGuard guard(lock_); + for (int64_t i = 0; i < fat_.count(); i += BLOCK_PAGE_NUMS) { + char * buf = fat_.at(i).buf_; + if (OB_ISNULL(buf)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page buffer", KR(ret), K(i), KP(buf)); + } else { + allocator_.free(buf); + } + } + return ret; +} + +int64_t ObTmpWriteBufferPool::get_memory_limit() +{ + int64_t memory_limit = 0; + int64_t last_access_ts = ATOMIC_LOAD(&last_access_tenant_config_ts_); + if (default_wbp_memory_limit_ > 0) { + memory_limit = default_wbp_memory_limit_; + } else if (last_access_ts > 0 && common::ObClockGenerator::getClock() - last_access_ts < 10000000) { // 10s + memory_limit = ATOMIC_LOAD(&wbp_memory_limit_); + } else { + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); + if (!tenant_config.is_valid()) { + static const int64_t DEFAULT_MEMORY_LIMIT = 64 * 2 * 1024 * 1024; // 128MB + memory_limit = wbp_memory_limit_ == 0 ? DEFAULT_MEMORY_LIMIT : wbp_memory_limit_; + LOG_INFO("failed to get tenant config", K(MTL_ID()), K(memory_limit), K(wbp_memory_limit_)); + } else if (0 == tenant_config->_temporary_file_io_area_size) { + memory_limit = 0; + } else { + memory_limit = common::upper_align( + lib::get_tenant_memory_limit(MTL_ID()) * tenant_config->_temporary_file_io_area_size / 100, + WBP_BLOCK_SIZE); + } + ATOMIC_STORE(&wbp_memory_limit_, memory_limit); + ATOMIC_STORE(&last_access_tenant_config_ts_, common::ObClockGenerator::getClock()); + } + return memory_limit; +} + +// expect swap min(10% * page cache memory, 20MB) each time +// if clean data size smaller than this min_swap_size return 0 +int64_t ObTmpWriteBufferPool::get_swap_size() +{ + const int64_t HIGH_WATERMARK_PECENTAGE = 55; + const int64_t LOW_WATERMARK_PECENTAGE = 30; + + int64_t memory_limit = get_memory_limit(); + int64_t used_page_num = ATOMIC_LOAD(&used_page_num_); + + int64_t high_watermark_bytes = (double)HIGH_WATERMARK_PECENTAGE / 100 * memory_limit; + int64_t low_watermark_bytes = (double)LOW_WATERMARK_PECENTAGE / 100 * memory_limit; + int64_t used_bytes = used_page_num * ObTmpFileGlobal::PAGE_SIZE; + + const int64_t MACRO_BLOCK_SIZE = OB_SERVER_BLOCK_MGR.get_macro_block_size(); + + int64_t swap_size = 0; + if (used_bytes > high_watermark_bytes) { + int64_t expected_swap_size = used_bytes - low_watermark_bytes; + int64_t dirty_data_bytes = ATOMIC_LOAD(&dirty_page_num_) * ObTmpFileGlobal::PAGE_SIZE; + swap_size = min(used_bytes - dirty_data_bytes, expected_swap_size); + } + + return swap_size; +} + +bool ObTmpWriteBufferPool::is_exist(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool exist = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + exist = ObPageEntry::State::INVALID < fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; // TODO: too many warn logs + LOG_WARN("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return exist; +} + +bool ObTmpWriteBufferPool::is_inited(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool inited = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + inited = ObPageEntry::State::INITED == fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return inited; +} + +bool ObTmpWriteBufferPool::is_loading(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool loading = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + loading = ObPageEntry::State::LOADING == fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return loading; +} + +bool ObTmpWriteBufferPool::is_cached( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool cached = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + cached = ObPageEntry::State::CACHED == fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return cached; +} + +bool ObTmpWriteBufferPool::is_write_back(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool write_back = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + write_back = ObPageEntry::State::WRITE_BACK == fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return write_back; +} + +bool ObTmpWriteBufferPool::is_dirty( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + common::TCRWLock::RLockGuard guard(lock_); + bool dirty = false; + if (OB_LIKELY(is_valid_page_id_(page_id) + && fd == fat_[page_id].fd_ + && page_key.is_valid() + && page_key == fat_[page_id].page_key_)) { + dirty = ObPageEntry::State::DIRTY == fat_[page_id].state_; + } else { + int ret = OB_ERR_UNEXPECTED; + LOG_ERROR("wbp get unexpected page entry", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } + return dirty; +} + +// 允许 INITED/CACHED/DIRTY/WRITE_BACK 状态页面切换为 DIRTY 状态 +int ObTmpWriteBufferPool::notify_dirty( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + bool is_already_dirty = false; + bool is_write_back = false; + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (FALSE_IT(is_already_dirty = (ObPageEntry::State::DIRTY == fat_[page_id].state_))) { + } else if (FALSE_IT(is_write_back = (ObPageEntry::State::WRITE_BACK == fat_[page_id].state_))) { + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::WRITE))) { + LOG_WARN("fail to switch state to DIRTY", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else { + if (!is_already_dirty) { + ATOMIC_INC(&dirty_page_num_); + if (PageEntryType::DATA == page_key.type_) { + ATOMIC_INC(&dirty_data_page_cnt_); + } else if (PageEntryType::META == page_key.type_) { + ATOMIC_INC(&dirty_meta_page_cnt_); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid page type", K(page_key)); + } + } + if (is_write_back) { + if (PageEntryType::DATA == page_key.type_) { + ATOMIC_DEC(&write_back_data_cnt_); + } else if (PageEntryType::META == page_key.type_) { + ATOMIC_DEC(&write_back_meta_cnt_); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid page type", K(page_key)); + } + } + } + return ret; +} + +int ObTmpWriteBufferPool::notify_load( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::LOAD))) { + LOG_WARN("fail to switch state from INITED to LOADING", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } + return ret; +} + +int ObTmpWriteBufferPool::notify_load_succ(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::LOAD_SUCC))) { + LOG_WARN("fail to switch state from LOADING to CACHED", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } + return ret; +} + +int ObTmpWriteBufferPool::notify_load_fail( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::LOAD_FAIL))) { + LOG_WARN("fail to switch state from LOADING to INITED", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } + return ret; +} + +int ObTmpWriteBufferPool::notify_write_back( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + bool is_dirty = false; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (FALSE_IT(is_dirty = (ObPageEntry::State::DIRTY == fat_[page_id].state_))) { + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::WRITE_BACK))) { + LOG_WARN("fail to switch state from DIRTY to WRITE_BACK", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else if (is_dirty) { + ATOMIC_DEC(&dirty_page_num_); + if (PageEntryType::DATA == fat_[page_id].page_key_.type_) { + ATOMIC_DEC(&dirty_data_page_cnt_); + ATOMIC_INC(&write_back_data_cnt_); + LOG_DEBUG("notify data write back", K(fd), K(page_id), K(fat_[page_id])); + } else if (PageEntryType::META == fat_[page_id].page_key_.type_) { + ATOMIC_DEC(&dirty_meta_page_cnt_); + ATOMIC_INC(&write_back_meta_cnt_); + LOG_INFO("notify meta write back", K(fd), K(page_id), K(fat_[page_id])); + } + } + return ret; +} + +int ObTmpWriteBufferPool::notify_write_back_succ( + const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::WRITE_BACK_SUCC))) { + LOG_WARN("fail to switch state from WRITE_BACK to CACHED", KR(ret), K(fd), K(page_id), K(fat_[page_id])); + } else { + if (PageEntryType::DATA == fat_[page_id].page_key_.type_) { + ATOMIC_DEC(&write_back_data_cnt_); + LOG_DEBUG("notify data write back succ", K(fd), K(page_id), K(fat_[page_id])); + } else if (PageEntryType::META == fat_[page_id].page_key_.type_) { + ATOMIC_DEC(&write_back_meta_cnt_); + LOG_INFO("notify meta write back succ", K(fd), K(page_id), K(fat_[page_id])); + } + } + return ret; +} + +int ObTmpWriteBufferPool::notify_write_back_fail(int64_t fd, uint32_t page_id, + const ObTmpFilePageUniqKey page_key) +{ + int ret = OB_SUCCESS; + common::TCRWLock::RLockGuard guard(lock_); + if (OB_UNLIKELY(!is_valid_page_id_(page_id) + || INVALID_FD == fd + || fd != fat_[page_id].fd_ + || !page_key.is_valid() + || page_key != fat_[page_id].page_key_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(fd), K(page_id), K(page_key), K(fat_[page_id])); + } else if (OB_FAIL(fat_[page_id].switch_state(ObPageEntry::Ops::WRITE_BACK_FAILED))) { + LOG_WARN("fail to switch state from WRITE_BACK to DIRTY", K(fd), K(page_id), K(fat_[page_id])); + } else { + ATOMIC_INC(&dirty_page_num_); + if (PageEntryType::DATA == fat_[page_id].page_key_.type_) { + ATOMIC_INC(&dirty_data_page_cnt_); + ATOMIC_DEC(&write_back_data_cnt_); + LOG_INFO("notify data write back fail", K(fd), K(page_id), K(fat_[page_id])); + } else if (PageEntryType::META == fat_[page_id].page_key_.type_) { + ATOMIC_INC(&dirty_meta_page_cnt_); + ATOMIC_DEC(&write_back_meta_cnt_); + LOG_INFO("notify meta write back fail", K(fd), K(page_id), K(fat_[page_id])); + } + } + return ret; +} + +// return write buffer pool maximum page number, which is determined by tenant memory and config +int64_t ObTmpWriteBufferPool::get_max_page_num() +{ + int64_t mem_limit = get_memory_limit(); + return mem_limit / ObTmpFileGlobal::PAGE_SIZE; +} + +// return dirty page percentage +int64_t ObTmpWriteBufferPool::get_dirty_page_percentage() +{ + int64_t max_page_num = get_max_page_num(); + int64_t dirty_page_num = ATOMIC_LOAD(&dirty_page_num_); + return max_page_num == 0 ? 0 : dirty_page_num * 100 / max_page_num; +} + +int64_t ObTmpWriteBufferPool::get_dirty_page_num() +{ + return ATOMIC_LOAD(&dirty_page_num_); +} + +int64_t ObTmpWriteBufferPool::get_dirty_meta_page_num() +{ + return ATOMIC_LOAD(&dirty_meta_page_cnt_); +} + +int64_t ObTmpWriteBufferPool::get_dirty_data_page_num() +{ + return ATOMIC_LOAD(&dirty_data_page_cnt_); +} + +int64_t ObTmpWriteBufferPool::get_data_page_num() +{ + return ATOMIC_LOAD(&data_page_cnt_); +} + +int64_t ObTmpWriteBufferPool::get_max_data_page_num() +{ + return get_max_page_num() * MAX_DATA_PAGE_USAGE_RATIO; +} + +int64_t ObTmpWriteBufferPool::get_meta_page_num() +{ + return ATOMIC_LOAD(&meta_page_cnt_); +} + +bool ObTmpWriteBufferPool::has_free_page_(PageEntryType type) +{ + int ret = OB_SUCCESS; + bool b_ret = true; + if (PageEntryType::DATA == type) { + b_ret = get_data_page_num() < get_max_data_page_num(); + } else if (PageEntryType::META == type) { + b_ret = true; // no limit for meta page + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected page type", KR(ret), K(type)); + } + return b_ret; +} + +void ObTmpWriteBufferPool::print_statistics() +{ + int64_t dirty_page_percentage = get_dirty_page_percentage(); + int64_t max_page_num = get_max_page_num(); + int64_t meta_page_num = get_meta_page_num(); + int64_t data_page_num = get_data_page_num(); + int64_t dirty_page_num = get_dirty_page_num(); + int64_t dirty_meta_page_num = get_dirty_meta_page_num(); + int64_t dirty_data_page_num = get_dirty_data_page_num(); + int64_t write_back_data_num = ATOMIC_LOAD(&write_back_data_cnt_); + int64_t write_back_meta_num = ATOMIC_LOAD(&write_back_meta_cnt_); + int64_t data_page_watermark = data_page_num * 100 / max(max_page_num, 1); + int64_t meta_page_watermark = meta_page_num * 100 / max(max_page_num, 1); + int64_t total_write_back_num = write_back_data_num + write_back_meta_num; + LOG_INFO("tmp file write buffer pool statistics", + K(dirty_page_percentage), K(max_page_num), K(dirty_page_num), K(total_write_back_num), + K(meta_page_num), K(dirty_meta_page_num), K(write_back_meta_num), + K(data_page_num), K(dirty_data_page_num), K(write_back_data_num), + K(data_page_watermark), K(meta_page_watermark)); +} + +} // end namespace tmp_file +} // end namespace oceanbase diff --git a/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.h b/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.h new file mode 100644 index 0000000000..58a741c10e --- /dev/null +++ b/src/storage/tmp_file/ob_tmp_file_write_buffer_pool.h @@ -0,0 +1,305 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_WRITE_BUFFER_POOL_H_ +#define OCEANBASE_STORAGE_BLOCKSSTABLE_OB_TMP_WRITE_BUFFER_POOL_H_ + +#include "lib/lock/ob_spin_rwlock.h" +#include "lib/container/ob_array.h" +#include "lib/queue/ob_link.h" +#include "lib/queue/ob_link_queue.h" +#include "lib/allocator/ob_fifo_allocator.h" +#include "share/io/ob_io_define.h" +#include "storage/blocksstable/ob_macro_block_id.h" +#include "storage/tmp_file/ob_tmp_file_global.h" + +namespace oceanbase +{ +namespace tmp_file +{ + +class ObTmpWriteBufferPool; +enum class PageEntryType +{ + INVALID = -1, + DATA = 0, + META = 1 +}; + +struct ObTmpFilePageUniqKey +{ +public: + ObTmpFilePageUniqKey() : type_(PageEntryType::INVALID), virtual_page_id_(ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID) {} + explicit ObTmpFilePageUniqKey(const int64_t virtual_page_id) : + type_(PageEntryType::DATA), virtual_page_id_(virtual_page_id) {} + explicit ObTmpFilePageUniqKey(const int64_t tree_level, const int64_t level_page_index) : + type_(PageEntryType::META), tree_level_(tree_level), + level_page_index_ (level_page_index) {} + + void reset() + { + virtual_page_id_ = ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID; + type_ = PageEntryType::INVALID; + } + + OB_INLINE bool is_valid() const + { + return type_ != PageEntryType::INVALID && + type_ == PageEntryType::META ? + 0 <= tree_level_ && 0 <= level_page_index_ : + ObTmpFileGlobal::INVALID_VIRTUAL_PAGE_ID != virtual_page_id_; + } + bool operator==(const ObTmpFilePageUniqKey other) const + { + return type_ == other.type_ && virtual_page_id_ == other.virtual_page_id_; + } + bool operator!=(const ObTmpFilePageUniqKey other) const + { + return type_ != other.type_ || virtual_page_id_ != other.virtual_page_id_; + } + +public: + PageEntryType type_; + union { + int64_t virtual_page_id_; // page_offset / page_size + struct { //The specific value for the tree pages + int64_t tree_level_:16; + int64_t level_page_index_:48; + }; + }; + TO_STRING_KV(K(type_), K(virtual_page_id_), K(tree_level_), K(level_page_index_)); +}; + +struct ObPageEntry final +{ + friend class ObTmpWriteBufferPool; +public: + ObPageEntry() + : buf_(nullptr), + fd_(-1), + state_(State::INVALID), + next_page_id_(ObTmpFileGlobal::INVALID_PAGE_ID), + page_key_() {} + ObPageEntry(const int64_t fd, const uint32_t next_page_id, char *buf) + : buf_(buf), + fd_(fd), + state_(State::INVALID), + next_page_id_(next_page_id), + page_key_() {} + + int switch_state(const int64_t op); + + TO_STRING_KV(K(fd_), K(page_key_), K(next_page_id_), K(state_), KP(buf_)); +public: + struct State + { + public: + static const int32_t N = -1; // illegal state + static const int32_t INVALID = 0; // page entry is INVALID after page is freed or before allocating + static const int32_t INITED = 1; // page entry is INITED after allocating + static const int32_t LOADING = 2; // page entry is LOADING after sending async io to read page from disk + static const int32_t CACHED = 3; // page entry is CACHED when page is clean + static const int32_t DIRTY = 4; // page entry is DIRTY after page is written + static const int32_t WRITE_BACK = 5; // page entry is WRITE_BACK when sending async io to write page to disk + static const int32_t MAX = 6; + public: + static bool is_valid(const int32_t state){ + return state > N && state < MAX; + } + }; + + struct Ops + { + public: + static const int64_t INVALID = -1; + static const int64_t ALLOC = 0; + static const int64_t LOAD = 1; + static const int64_t LOAD_FAIL = 2; + static const int64_t LOAD_SUCC = 3; + static const int64_t DELETE = 4; + static const int64_t WRITE = 5; + static const int64_t WRITE_BACK = 6; + static const int64_t WRITE_BACK_FAILED = 7; + static const int64_t WRITE_BACK_SUCC = 8; + static const int64_t MAX = 9; + public: + static bool is_valid(const int64_t op){ + return op > INVALID && op < MAX; + } + }; +public: + char *buf_; + int64_t fd_; + int32_t state_; + uint32_t next_page_id_; + ObTmpFilePageUniqKey page_key_; +}; + +// preallocate a set of pages for the tmp file to write data. the pages are divided into data and meta types. +// data type pages can use up to 90% of the entire buffer pool space, while meta type pages have no upper limit. +// we build ObTmpWriteBufferPool upon this assumption: caller ensure only 1 writer operating a page entry at a time, +// and write operation must be exclusive with other r/w operations, therefore we have no need to limit the concurrency +// for every single page entry here. +class ObTmpWriteBufferPool final +{ +public: + // block size: 2MB - 24KB (header), use block size smaller than 2MB to avoid redundant AObject header + static const int64_t WBP_BLOCK_SIZE = 2 * 1024 * 1024 - 24 * 1024; + static const int64_t BLOCK_PAGE_NUMS = WBP_BLOCK_SIZE / ObTmpFileGlobal::PAGE_SIZE; // 253 pages per block (24KB for header) + static const int64_t INITIAL_POOL_SIZE = WBP_BLOCK_SIZE; + static const int64_t INITIAL_PAGE_NUMS = INITIAL_POOL_SIZE / ObTmpFileGlobal::PAGE_SIZE; + +public: + ObTmpWriteBufferPool(); + ~ObTmpWriteBufferPool(); + int init(); + void destroy(); + +public: + // 1. according to the type_ of page_key, allocate a meta page or data page and set its state to INITED + // 2. return OB_ALLOCATE_TMP_FILE_PAGE_FAILED if data page number exceeds limits + // 3. always allow to alloc a meta page + int alloc_page(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&buf); + + // read the content of a page and keep its original page state + int read_page(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key, + char *&buf, uint32_t &next_page_id); + + // set prev_page_id.next_page_id to point to 'page_id' without changing the page status + int link_page(const int64_t fd, const uint32_t page_id, const uint32_t prev_page_id, + const ObTmpFilePageUniqKey prev_page_key); + + // free given pages with INITED/CACHED/DIRTY state, + // return OB_STATE_NOT_MATCH if try to delete pages with other states + int free_page(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, uint32_t &next_page_id); + + /** + * truncate a page from beginning to 'truncate_size' [0, truncate_size - 1], set data to 0; + * truncate_page will not change page state. if page is already flushed to disk, truncate_page + * only truncate data in buffer pool and will not mark page as dirty + */ + int truncate_page(const int64_t fd, const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + const int64_t truncate_size); +public: + /** + * given page_id, output next_page_id if existed; + * return OB_ITER_END for no more pages, others for error + */ + int get_next_page_id(const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + uint32_t &next_page_id); + + /** + * iter from the given page_id to end of the page list of a tmp file, + * to find a page which has a same virtual_page_id + * @param[in] virtual_page_id: the page we want to read(it equal to page_offset / page_size) + * @param[in] begin_page_id: the first cached page id of the tmp file + * @param[out] page_id: the target page + */ + int get_page_id_by_virtual_id(const int64_t fd, + const int64_t virtual_page_id, + const uint32_t begin_page_id, + uint32_t &page_id); + + /** + * get page_virtual_id of the given page_id, + * return OB_SEARCH_NOT_FOUND for page is INVALID, + */ + int get_page_virtual_id(const int64_t fd, const uint32_t page_id, int64_t &virtual_page_id); + +public: + int64_t get_swap_size(); + int64_t get_memory_limit(); + bool is_exist(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + bool is_inited(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + bool is_loading(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + bool is_cached(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + bool is_dirty(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + bool is_write_back(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_dirty(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_load(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_load_succ(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_load_fail(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_write_back(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_write_back_succ(const int64_t fd, const uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int notify_write_back_fail(int64_t fd, uint32_t page_id, const ObTmpFilePageUniqKey page_key); + int64_t get_max_page_num(); + int64_t get_data_page_num(); + int64_t get_dirty_page_num(); + int64_t get_dirty_meta_page_num(); + int64_t get_dirty_data_page_num(); + int64_t get_dirty_page_percentage(); + int64_t get_max_data_page_num(); + int64_t get_meta_page_num(); + void print_statistics(); +private: + static double MAX_DATA_PAGE_USAGE_RATIO; // control data pages ratio, can be preempted by meta pages + // only for unittest + OB_INLINE void set_max_data_page_usage_ratio_(const double ratio) + { + MAX_DATA_PAGE_USAGE_RATIO = ratio; + } + int read_page_(const int64_t fd, + const uint32_t page_id, + const ObTmpFilePageUniqKey page_key, + char *&buf, + uint32_t &next_page_id); + int alloc_page_(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&buf); + int inner_alloc_page_(const int64_t fd, + const ObTmpFilePageUniqKey page_key, + uint32_t &new_page_id, + char *&buf); + // check if the specified PageEntryType has available space + bool has_free_page_(PageEntryType type); + // ATTENTION! access fat_, needs to be protected by r-lock + OB_INLINE bool is_valid_page_id_(const uint32_t page_id) const + { + return page_id != ObTmpFileGlobal::INVALID_PAGE_ID && page_id >= 0 && + page_id < fat_.count() && OB_NOT_NULL(fat_[page_id].buf_); + } + int expand_(); + int reduce_(); + DISALLOW_COPY_AND_ASSIGN(ObTmpWriteBufferPool); +private: + common::ObArray fat_; // file allocation table + common::TCRWLock lock_; // holds w-lock when expanding and shrinking fat_, holds r-lock when reading fat_ + common::ObFIFOAllocator allocator_; + bool is_inited_; + int64_t capacity_; // in bytes + int64_t dirty_page_num_; + int64_t used_page_num_; + uint32_t first_free_page_id_; + int64_t wbp_memory_limit_; // in bytes + int64_t default_wbp_memory_limit_; // if this var is valid, the wbp memory limit will always be it. + // currently, this var is only modified in ut. + int64_t last_access_tenant_config_ts_; + int64_t meta_page_cnt_; + int64_t data_page_cnt_; + int64_t dirty_meta_page_cnt_; + int64_t dirty_data_page_cnt_; + int64_t write_back_data_cnt_; + int64_t write_back_meta_cnt_; +}; + +} // end namespace tmp_file +} // end namespace oceanbase + +#endif diff --git a/unittest/sql/engine/basic/test_chunk_datum_store.cpp b/unittest/sql/engine/basic/test_chunk_datum_store.cpp index 4a686fcf5e..b4e06b7607 100644 --- a/unittest/sql/engine/basic/test_chunk_datum_store.cpp +++ b/unittest/sql/engine/basic/test_chunk_datum_store.cpp @@ -13,10 +13,10 @@ #define USING_LOG_PREFIX SQL #include +#include "mtlenv/mock_tenant_module_env.h" #include "lib/alloc/ob_malloc_allocator.h" #include "lib/allocator/ob_malloc.h" #include "storage/blocksstable/ob_data_file_prepare.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "sql/engine/basic/ob_chunk_datum_store.h" #include "sql/engine/basic/ob_ra_row_store.h" #include "common/row/ob_row_store.h" @@ -163,8 +163,8 @@ public: int ret = OB_SUCCESS; ASSERT_EQ(OB_SUCCESS, init_tenant_mgr()); blocksstable::TestDataFilePrepare::SetUp(); - ret = blocksstable::ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); if (!is_server_tenant(tenant_id_)) { static ObTenantBase tenant_ctx(tenant_id_); ObTenantEnv::set_tenant(&tenant_ctx); @@ -173,15 +173,23 @@ public: EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } cell_cnt_ = COLS; init_exprs(); - plan_.set_batch_size(batch_size_); - plan_ctx_.set_phy_plan(&plan_); - eval_ctx_.set_max_batch_size(batch_size_); + plan_.set_batch_size(batch_size_); + plan_ctx_.set_phy_plan(&plan_); + eval_ctx_.set_max_batch_size(batch_size_); exec_ctx_.set_physical_plan_ctx(&plan_ctx_); skip_ = (ObBitVector *)alloc_.alloc(ObBitVector::memory_size(batch_size_)); @@ -207,7 +215,8 @@ public: rs_.reset(); rs_.~ObChunkDatumStore(); - blocksstable::ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); blocksstable::TestDataFilePrepare::TearDown(); LOG_INFO("TearDown finished", K_(rs)); } diff --git a/unittest/sql/engine/basic/test_chunk_row_store.cpp b/unittest/sql/engine/basic/test_chunk_row_store.cpp index f3f82aabfa..f63dce500c 100644 --- a/unittest/sql/engine/basic/test_chunk_row_store.cpp +++ b/unittest/sql/engine/basic/test_chunk_row_store.cpp @@ -13,10 +13,10 @@ #define USING_LOG_PREFIX SQL #include +#include "mtlenv/mock_tenant_module_env.h" #include "lib/alloc/ob_malloc_allocator.h" #include "lib/allocator/ob_malloc.h" #include "storage/blocksstable/ob_data_file_prepare.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "sql/engine/basic/ob_chunk_row_store.h" #include "sql/engine/basic/ob_ra_row_store.h" #include "common/row/ob_row_store.h" @@ -24,7 +24,6 @@ #include "sql/ob_sql_init.h" #include "share/ob_simple_mem_limit_getter.h" - namespace oceanbase { namespace sql @@ -69,16 +68,25 @@ public: int ret = OB_SUCCESS; ASSERT_EQ(OB_SUCCESS, init_tenant_mgr()); blocksstable::TestDataFilePrepare::SetUp(); - ret = blocksstable::ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); if (!is_server_tenant(tenant_id_)) { static ObTenantBase tenant_ctx(tenant_id_); ObTenantEnv::set_tenant(&tenant_ctx); + ObTenantIOManager *io_service = nullptr; EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_new(io_service)); EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } @@ -107,7 +115,8 @@ public: rs_.reset(); rs_.~ObChunkRowStore(); - blocksstable::ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); blocksstable::TestDataFilePrepare::TearDown(); LOG_INFO("TearDown finished", K_(rs)); } diff --git a/unittest/sql/engine/test_op_engine.cpp b/unittest/sql/engine/test_op_engine.cpp index 5edb34ef64..692ac6ce95 100644 --- a/unittest/sql/engine/test_op_engine.cpp +++ b/unittest/sql/engine/test_op_engine.cpp @@ -79,7 +79,6 @@ void TestOpEngine::destory() ObIOManager::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); ObClusterVersion::get_instance().destroy(); - oceanbase::blocksstable::ObTmpFileManager::get_instance().destroy(); // THE_IO_DEVICE->destroy(); } @@ -179,7 +178,6 @@ int TestOpEngine::prepare_io(const string & test_data_name_suffix) LOG_WARN("fail to init OB_STORE_CACHE, ", K(ret)); } else { } - FILE_MANAGER_INSTANCE_V2.init(); return ret; } diff --git a/unittest/storage/backup/test_backup_index_merger.cpp b/unittest/storage/backup/test_backup_index_merger.cpp index 9a9a495afd..c35292c1bb 100644 --- a/unittest/storage/backup/test_backup_index_merger.cpp +++ b/unittest/storage/backup/test_backup_index_merger.cpp @@ -28,6 +28,7 @@ #include "test_backup.h" #include "test_backup_include.h" #include "storage/blocksstable/ob_logic_macro_id.h" +#include "mtlenv/mock_tenant_module_env.h" using namespace testing; using namespace oceanbase; @@ -317,12 +318,11 @@ void TestBackupIndexMerger::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - if (OB_INIT_TWICE == ret) { - ret = OB_SUCCESS; - } else { - ASSERT_EQ(OB_SUCCESS, ret); - } + + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -330,14 +330,24 @@ void TestBackupIndexMerger::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); inner_init_(); } void TestBackupIndexMerger::TearDown() { - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } void TestBackupIndexMerger::inner_init_() diff --git a/unittest/storage/backup/test_backup_iterator.cpp b/unittest/storage/backup/test_backup_iterator.cpp index 2138134c6a..e46bb75fbf 100644 --- a/unittest/storage/backup/test_backup_iterator.cpp +++ b/unittest/storage/backup/test_backup_iterator.cpp @@ -24,6 +24,7 @@ #include "test_backup_include.h" #include "share/backup/ob_backup_io_adapter.h" #include "lib/string/ob_string.h" +#include "mtlenv/mock_tenant_module_env.h" using namespace oceanbase; using namespace oceanbase::common; @@ -111,7 +112,11 @@ void TestBackupIndexIterator::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); + + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + if (OB_INIT_TWICE == ret) { ret = OB_SUCCESS; } else { @@ -124,6 +129,14 @@ void TestBackupIndexIterator::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); inner_init_(); ASSERT_EQ(OB_SUCCESS, bandwidth_throttle_.init(1024 * 1024 * 60)); @@ -131,8 +144,10 @@ void TestBackupIndexIterator::SetUp() void TestBackupIndexIterator::TearDown() { - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } void TestBackupIndexIterator::inner_init_() diff --git a/unittest/storage/backup/test_backup_macro_block_index_merger.cpp b/unittest/storage/backup/test_backup_macro_block_index_merger.cpp index 15eee13066..dd338f9fe1 100644 --- a/unittest/storage/backup/test_backup_macro_block_index_merger.cpp +++ b/unittest/storage/backup/test_backup_macro_block_index_merger.cpp @@ -15,6 +15,7 @@ #define private public #define protected public +#include "observer/omt/ob_tenant_mtl_helper.h" #include "storage/backup/ob_backup_data_struct.h" #include "storage/backup/ob_backup_index_merger.h" #include "storage/backup/ob_backup_index_store.h" @@ -190,12 +191,8 @@ void TestBackupMacroIndexMerger::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - if (OB_INIT_TWICE == ret) { - ret = OB_SUCCESS; - } else { - ASSERT_EQ(OB_SUCCESS, ret); - } + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -203,14 +200,23 @@ void TestBackupMacroIndexMerger::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); inner_init_(); } void TestBackupMacroIndexMerger::TearDown() { - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } void TestBackupMacroIndexMerger::clean_env_() diff --git a/unittest/storage/backup/test_backup_tmp_file.cpp b/unittest/storage/backup/test_backup_tmp_file.cpp index ed0a83c26a..60be7510a8 100644 --- a/unittest/storage/backup/test_backup_tmp_file.cpp +++ b/unittest/storage/backup/test_backup_tmp_file.cpp @@ -20,6 +20,7 @@ #include "storage/backup/ob_backup_data_struct.h" #include "storage/blocksstable/ob_data_file_prepare.h" #include "test_backup.h" +#include "mtlenv/mock_tenant_module_env.h" using namespace oceanbase; using namespace oceanbase::common; @@ -66,8 +67,10 @@ void TestBackupTmpFile::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -75,14 +78,24 @@ void TestBackupTmpFile::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } void TestBackupTmpFile::TearDown() { - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); TestDataFilePrepare::TearDown(); + common::ObClockGenerator::destroy(); } static void make_meta_index(backup::ObBackupMetaIndex &meta_index) diff --git a/unittest/storage/backup/test_backup_utils.cpp b/unittest/storage/backup/test_backup_utils.cpp index 55c54fe51b..a3fc638a0e 100644 --- a/unittest/storage/backup/test_backup_utils.cpp +++ b/unittest/storage/backup/test_backup_utils.cpp @@ -23,6 +23,7 @@ #include "storage/blocksstable/ob_logic_macro_id.h" #include "storage/ob_parallel_external_sort.h" #include "storage/blocksstable/ob_data_file_prepare.h" +#include "mtlenv/mock_tenant_module_env.h" #include "storage/backup/ob_backup_ctx.h" using namespace oceanbase; @@ -261,7 +262,10 @@ void TestBackupExternalSort::SetUp() { TestDataFilePrepare::SetUp(); EXPECT_EQ(OB_SUCCESS, init_tenant_mgr()); - EXPECT_EQ(OB_SUCCESS, ObTmpFileManager::get_instance().init()); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -269,14 +273,26 @@ void TestBackupExternalSort::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } void TestBackupExternalSort::TearDown() { allocator_.reuse(); - ObTmpFileManager::get_instance().destroy(); + + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); TestDataFilePrepare::TearDown(); + common::ObClockGenerator::destroy(); + destroy_tenant_mgr(); } diff --git a/unittest/storage/blocksstable/CMakeLists.txt b/unittest/storage/blocksstable/CMakeLists.txt index 1f900df20b..1496d57cfa 100644 --- a/unittest/storage/blocksstable/CMakeLists.txt +++ b/unittest/storage/blocksstable/CMakeLists.txt @@ -4,7 +4,6 @@ storage_unittest(test_block_sstable_struct) storage_unittest(test_data_buffer) storage_unittest(test_index_block_aggregator) #storage_unittest(test_storage_cache_suite) -storage_unittest(test_tmp_file) #storage_unittest(test_sstable_sec_meta_iterator) storage_unittest(test_sstable_meta) #storage_unittest(test_inspect_bad_block) diff --git a/unittest/storage/blocksstable/test_block_manager.cpp b/unittest/storage/blocksstable/test_block_manager.cpp index 6654a136d0..1b4745fe47 100644 --- a/unittest/storage/blocksstable/test_block_manager.cpp +++ b/unittest/storage/blocksstable/test_block_manager.cpp @@ -21,10 +21,10 @@ #define private public #include "storage/blocksstable/ob_data_file_prepare.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "share/ob_simple_mem_limit_getter.h" #include "observer/omt/ob_worker_processor.h" #include "observer/ob_srv_network_frame.h" +#include "mtlenv/mock_tenant_module_env.h" namespace oceanbase { @@ -138,9 +138,18 @@ TEST_F(TestBlockManager, test_mark_and_sweep) } else { ASSERT_EQ(common::OB_SUCCESS, ret); } - ret = FILE_MANAGER_INSTANCE_V2.init(); - ASSERT_EQ(common::OB_SUCCESS, ret); - ASSERT_EQ(common::OB_SUCCESS, ret); + + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); + ObTenantEnv::set_tenant(&tenant_ctx); + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); ASSERT_EQ(0, OB_SERVER_BLOCK_MGR.block_map_.count()); @@ -188,8 +197,10 @@ TEST_F(TestBlockManager, test_mark_and_sweep) macro_handle.reset(); - FILE_MANAGER_INSTANCE_V2.destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } TEST_F(TestBlockManager, test_ref_cnt_wash_and_load) diff --git a/unittest/storage/blocksstable/test_tmp_file.cpp b/unittest/storage/blocksstable/test_tmp_file.cpp deleted file mode 100644 index 890cd82790..0000000000 --- a/unittest/storage/blocksstable/test_tmp_file.cpp +++ /dev/null @@ -1,2378 +0,0 @@ -/** - * Copyright (c) 2021 OceanBase - * OceanBase CE is licensed under Mulan PubL v2. - * You can use this software according to the terms and conditions of the Mulan PubL v2. - * You may obtain a copy of Mulan PubL v2 at: - * http://license.coscl.org.cn/MulanPubL-2.0 - * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, - * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, - * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. - * See the Mulan PubL v2 for more details. - */ - -#include -#include -#include -#define protected public -#define private public -#include "lib/allocator/ob_fifo_allocator.h" -#include "storage/blocksstable/ob_tmp_file.h" -#include "storage/blocksstable/ob_tmp_file_store.h" -#include "storage/blocksstable/ob_tmp_file_cache.h" -#include "ob_row_generate.h" -#include "ob_data_file_prepare.h" -#include "share/ob_simple_mem_limit_getter.h" - -namespace oceanbase -{ -using namespace common; -using namespace blocksstable; -using namespace storage; -using namespace share::schema; -static ObSimpleMemLimitGetter getter; - -namespace unittest -{ - -static const int64_t TEST_COLUMN_CNT = ObExtendType - 1; -static const int64_t TEST_ROWKEY_COLUMN_CNT = 2; - -struct BufHeader -{ -public: - BufHeader() - : data_size_(0), start_row_(0) - {} - virtual ~BufHeader() {} - int serialize(char *buf, const int64_t buf_len, int64_t &pos); - int deserialize(const char *buf, const int64_t data_len, int64_t &pos); - int64_t get_serialize_size() const; - int64_t data_size_; - int64_t start_row_; -}; - -int BufHeader::serialize(char *buf, const int64_t buf_len, int64_t &pos) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(serialization::encode_i64(buf, buf_len, pos, data_size_))) { - STORAGE_LOG(WARN, "fail to serialize data size", K(ret)); - } else if (OB_FAIL(serialization::encode_i64(buf, buf_len, pos, start_row_))) { - STORAGE_LOG(WARN, "fail to serialize start row", K(ret)); - } - return ret; -} - -int BufHeader::deserialize(const char *buf, const int64_t data_len, int64_t &pos) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(serialization::decode_i64(buf, data_len, pos, &data_size_))) { - STORAGE_LOG(WARN, "fail to decode data size", K(ret)); - } else if (OB_FAIL(serialization::decode_i64(buf, data_len, pos, &start_row_))) { - STORAGE_LOG(WARN, "fail to decode start row", K(ret)); - } - return ret; -} - -int64_t BufHeader::get_serialize_size() const -{ - int64_t size = 0; - size += serialization::encoded_length_i64(1L); - size += serialization::encoded_length_i64(1L); - return size; -} - -class TestTmpFileStress : public share::ObThreadPool -{ -public: - TestTmpFileStress(); - TestTmpFileStress(ObTenantBase *tenant_ctx); - virtual ~TestTmpFileStress(); - int init(const int fd, const bool is_write, const int64_t thread_cnt, ObTableSchema *table_schema, - const bool is_plain_data, const bool is_big_file, bool is_truncate = false); - virtual void run1(); -private: - void prepare_data(char *buf, const int64_t macro_block_size); - void prepare_plain_data(const int64_t buf_size, char *buf, ObIArray &size_array); - void prepare_one_buffer(const int64_t macro_block_size, const int64_t start_index, char *buf, int64_t &end_index); - void check_data(const char *buf, const int64_t buf_len); - void check_plain_data(const char *read_buf, const char *right_buf, const int64_t buf_len); - void write_data(const int64_t macro_block_size); - void write_plain_data(char *&buf, const int64_t macro_block_size); - void read_data(const int64_t macro_block_size); - void read_and_truncate(const int64_t macro_block_size); - void read_plain_data(const char *buf, const int64_t macro_block_size); -private: - static const int64_t BUF_COUNT = 16; - int64_t thread_cnt_; - int64_t size_; - int fd_; - bool is_write_; - bool is_big_file_; - ObTableSchema *table_schema_; - bool is_plain_; - bool is_truncate_; - ObTenantBase *tenant_ctx_; -}; - -TestTmpFileStress::TestTmpFileStress() - : thread_cnt_(0), size_(OB_SERVER_BLOCK_MGR.get_macro_block_size()), fd_(0), - is_write_(false), is_big_file_(false), table_schema_(NULL), is_plain_(false), - is_truncate_(false) -{ -} - -TestTmpFileStress::TestTmpFileStress(ObTenantBase *tenant_ctx) - : thread_cnt_(0), size_(OB_SERVER_BLOCK_MGR.get_macro_block_size()), fd_(0), - is_write_(false), is_big_file_(false), table_schema_(NULL), is_plain_(false), - is_truncate_(false), tenant_ctx_(tenant_ctx) -{ -} - -TestTmpFileStress::~TestTmpFileStress() -{ -} - -int TestTmpFileStress::init(const int fd, const bool is_write, - const int64_t thread_cnt, ObTableSchema *table_schema, - const bool is_plain, const bool is_big_file, const bool is_truncate) -{ - int ret = OB_SUCCESS; - if (thread_cnt < 0) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(thread_cnt)); - } else { - thread_cnt_ = thread_cnt; - fd_ = fd; - is_write_ = is_write; - table_schema_ = table_schema; - is_plain_ = is_plain; - is_big_file_ = is_big_file; - if (!is_big_file_) { - size_ = 16L * 1024L; - } - is_truncate_ = is_truncate; - set_thread_count(static_cast(thread_cnt)); - } - return ret; -} - -void TestTmpFileStress::prepare_one_buffer(const int64_t macro_block_size, const int64_t start_index, char *buf, int64_t &end_index) -{ - int ret = OB_SUCCESS; - ObStoreRow row; - BufHeader header; - ObArenaAllocator allocator; - ObRowGenerate row_generate; - int64_t buf_pos = header.get_serialize_size(); - int64_t header_pos = 0; - ObObj cells[TEST_COLUMN_CNT]; - row.row_val_.cells_ = cells; - row.row_val_.count_ = TEST_COLUMN_CNT; - header.start_row_ = start_index; - const int64_t buf_capacity = macro_block_size; - ASSERT_EQ(OB_SUCCESS, row_generate.init(*table_schema_, &allocator)); - for (int64_t i = start_index; OB_SUCC(ret) && buf_pos < buf_capacity; ++i) { - ret = row_generate.get_next_row(i, row); - ASSERT_EQ(OB_SUCCESS, ret); - if (buf_pos + row.get_serialize_size() <= buf_capacity) { - ASSERT_EQ(OB_SUCCESS, row.serialize(buf, buf_capacity, buf_pos)); - } else { - end_index = i; - break; - } - } - header.data_size_ = buf_pos; - ASSERT_EQ(OB_SUCCESS, header.serialize(buf, buf_capacity, header_pos)); -} - -void TestTmpFileStress::prepare_data(char *buf, const int64_t macro_block_size) -{ - const int64_t macro_block_buffer_count = BUF_COUNT; - int64_t buf_pos = 0; - int64_t start_index = 0; - for (int64_t i = 0; i < macro_block_buffer_count; ++i) { - int64_t end_index = 0; - prepare_one_buffer(macro_block_size, start_index, buf + buf_pos, end_index); - buf_pos += macro_block_size; - start_index = end_index; - } -} - -void TestTmpFileStress::prepare_plain_data(const int64_t buf_capacity, char *buf, - ObIArray &size_array) -{ - ObRandom random; - int64_t left_size = buf_capacity; - int8_t data = 0; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - while (left_size > 0) { - if (left_size < macro_block_size) { - memset(buf, data, left_size); - ASSERT_EQ(OB_SUCCESS, size_array.push_back(left_size)); - left_size = 0; - } else { - const int64_t rand_data = random.get(0, macro_block_size); - memset(buf, data, rand_data); - left_size -= rand_data; - buf += rand_data; - ASSERT_EQ(OB_SUCCESS, size_array.push_back(rand_data)); - } - ++data; - } -} - -void TestTmpFileStress::check_plain_data(const char *read_buf, const char *right_buf, const int64_t buf_len) -{ - int cmp = memcmp(read_buf, right_buf, buf_len); - ASSERT_EQ(0, cmp); -} - -void TestTmpFileStress::check_data(const char *buf, const int64_t buf_len) -{ - int ret = OB_SUCCESS; - int64_t header_pos = 0; - int64_t data_pos = 0; - const char *data = NULL; - BufHeader header; - ObArenaAllocator allocator; - ObRowGenerate row_generate; - ASSERT_EQ(OB_SUCCESS, row_generate.init(*table_schema_, &allocator)); - const int64_t serialize_size = header.get_serialize_size(); - ObStoreRow lhs_row; - ObStoreRow rhs_row; - ObObj lhs_cells[TEST_COLUMN_CNT]; - ObObj rhs_cells[TEST_COLUMN_CNT]; - lhs_row.row_val_.cells_ = lhs_cells; - lhs_row.row_val_.count_ = TEST_COLUMN_CNT; - rhs_row.row_val_.cells_ = rhs_cells; - rhs_row.row_val_.count_ = TEST_COLUMN_CNT; - ret = header.deserialize(buf, buf_len, header_pos); - const int64_t data_len = header.data_size_; - int64_t i = header.start_row_; - ASSERT_EQ(OB_SUCCESS, ret); - data = buf + header_pos; - while (data_pos < data_len - serialize_size) { - ret = lhs_row.deserialize(data, data_len, data_pos); - ASSERT_EQ(OB_SUCCESS, ret); - ret = row_generate.get_next_row(i, rhs_row); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_TRUE(lhs_row.row_val_ == rhs_row.row_val_); - ++i; - } -} - -void TestTmpFileStress::write_data(const int64_t macro_block_size) -{ - int ret = OB_SUCCESS; - ObArenaAllocator allocator; - ObRowGenerate row_generate; - ObTmpFileIOInfo io_info; - row_generate.reset(); - ret = row_generate.init(*table_schema_, &allocator); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd_; - io_info.size_ = macro_block_size; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - char *buf = new char[BUF_COUNT * macro_block_size]; - const int64_t timeout_ms = 5000; - prepare_data(buf, macro_block_size); - for (int64_t i = 0; i < BUF_COUNT; ++i) { - io_info.buf_ = buf + i * macro_block_size; - check_data(io_info.buf_, macro_block_size); - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - } -} - -void TestTmpFileStress::write_plain_data(char *&buf, const int64_t macro_block_size) -{ - int ret = OB_SUCCESS; - ObArray size_array; - ObTmpFileIOInfo io_info; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd_; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - buf = new char[BUF_COUNT * macro_block_size]; - const int64_t timeout_ms = 5000; - int64_t sum_size = 0; - prepare_plain_data(BUF_COUNT * macro_block_size, buf, size_array); - for (int64_t i = 0; i < size_array.count(); ++i) { - io_info.buf_ = buf + sum_size; - io_info.size_ = size_array.at(i); - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - sum_size += size_array.at(i); - } - ret = ObTmpFileManager::get_instance().sync(fd_, timeout_ms); - ASSERT_EQ(OB_SUCCESS, ret); -} - -void TestTmpFileStress::read_data(const int64_t macro_block_size) -{ - int ret = OB_SUCCESS; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - io_info.fd_ = fd_; - io_info.size_ = macro_block_size; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - char *buf = new char[macro_block_size]; - for (int64_t i = 0; i < BUF_COUNT; ++i) { - io_info.buf_ = buf; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - check_data(handle.get_buffer(), handle.get_data_size()); - ASSERT_EQ(OB_SUCCESS, ret); - } - handle.reset(); -} - -void TestTmpFileStress::read_and_truncate(const int64_t macro_block_size) -{ - int ret = OB_SUCCESS; - const int64_t timeout_ms = 100000; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - io_info.fd_ = fd_; - io_info.size_ = macro_block_size; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - int64_t trunc_offset = 0; - char *buf = new char[macro_block_size]; - char *zero_buf = new char[macro_block_size]; - memset(zero_buf, 0, macro_block_size); - for (int64_t i = 0; i < BUF_COUNT; ++i) { - io_info.buf_ = buf; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - check_data(handle.get_buffer(), handle.get_data_size()); - ASSERT_EQ(OB_SUCCESS, ret); - // truncate data - // truncate trunc_offset + macro_block_size; - ret = ObTmpFileManager::get_instance().truncate(io_info.fd_, trunc_offset + macro_block_size); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().pread(io_info, trunc_offset, handle); - ASSERT_EQ(OB_SUCCESS, ret); - check_plain_data(zero_buf, buf, macro_block_size); - trunc_offset += macro_block_size; - } - // check tuncated(0) won't reset the read_guard; - if (ret == OB_SUCCESS) { - io_info.buf_ = buf; - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - ASSERT_EQ(OB_SUCCESS, ret); - check_plain_data(zero_buf, buf, macro_block_size); - ret = ObTmpFileManager::get_instance().truncate(io_info.fd_, 0); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - ASSERT_EQ(OB_SUCCESS, ret); - check_plain_data(zero_buf, buf, macro_block_size); - } - handle.reset(); -} - -void TestTmpFileStress::read_plain_data(const char *read_buf, const int64_t macro_block_size) -{ - int ret = OB_SUCCESS; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - io_info.fd_ = fd_; - io_info.size_ = macro_block_size; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - char *buf = new char[BUF_COUNT * macro_block_size]; - int64_t offset = 0; - for (int64_t i = 0; i < BUF_COUNT; ++i) { - io_info.buf_ = buf + i * macro_block_size; - offset = i * macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, offset, handle); - ASSERT_EQ(OB_SUCCESS, ret); - } - offset += macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, offset, handle); - ASSERT_EQ(OB_ITER_END, ret); - check_plain_data(read_buf, buf, BUF_COUNT * macro_block_size); - handle.reset(); -} - -void TestTmpFileStress::run1() -{ - ObTenantEnv::set_tenant(tenant_ctx_); - if (is_plain_) { - char *buf = NULL; - write_plain_data(buf, size_); - read_plain_data(buf, size_); - } else { - if (is_write_) { - write_data(size_); - } else { - read_data(size_); - } - } -} - -class TestMultiTmpFileStress : public share::ObThreadPool -{ -public: - TestMultiTmpFileStress(); - TestMultiTmpFileStress(ObTenantBase *tenant_ctx); - virtual ~TestMultiTmpFileStress(); - int init(const int64_t file_cnt, const int64_t dir_id, const int64_t thread_cnt, - ObTableSchema *table_schema, const bool is_plain_data, const bool is_big_file, - const bool is_truncate = false); - virtual void run1(); -private: - void run_plain_case(); - void run_normal_case(); -private: - int64_t file_cnt_; - int64_t dir_id_; - int64_t thread_cnt_perf_file_; - ObTableSchema *table_schema_; - bool is_big_file_; - bool is_plain_data_; - bool is_truncate_; - ObTenantBase *tenant_ctx_; -}; - -TestMultiTmpFileStress::TestMultiTmpFileStress() - : file_cnt_(0), - dir_id_(-1), - thread_cnt_perf_file_(0), - table_schema_(NULL), - is_big_file_(false), - is_plain_data_(false), - is_truncate_(false) -{ -} -TestMultiTmpFileStress::TestMultiTmpFileStress(ObTenantBase *tenant_ctx) - : file_cnt_(0), - dir_id_(-1), - thread_cnt_perf_file_(0), - table_schema_(NULL), - is_big_file_(false), - is_plain_data_(false), - is_truncate_(false), - tenant_ctx_(tenant_ctx) -{ -} - -TestMultiTmpFileStress::~TestMultiTmpFileStress() -{ -} - -int TestMultiTmpFileStress::init(const int64_t file_cnt, - const int64_t dir_id, - const int64_t thread_cnt, - ObTableSchema *table_schema, - const bool is_plain_data, - const bool is_big_file, - const bool is_truncate) -{ - int ret = OB_SUCCESS; - if (file_cnt < 0 || thread_cnt < 0 || NULL == table_schema) { - ret = OB_INVALID_ARGUMENT; - STORAGE_LOG(WARN, "invalid argument", K(ret), K(file_cnt), K(thread_cnt), - KP(table_schema)); - } else { - file_cnt_ = file_cnt; - dir_id_ = dir_id; - thread_cnt_perf_file_ = thread_cnt; - table_schema_ = table_schema; - is_big_file_ = is_big_file; - is_plain_data_ = is_plain_data; - is_truncate_ = is_truncate; - set_thread_count(static_cast(file_cnt)); - } - return ret; -} - -void TestMultiTmpFileStress::run_plain_case() -{ - int ret = OB_SUCCESS; - int64_t fd = 0; - TestTmpFileStress test(tenant_ctx_); - ret = ObTmpFileManager::get_instance().open(fd, dir_id_); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(fd, true, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_); - ASSERT_EQ(OB_SUCCESS, ret); - test.start(); - test.wait(); - ret = ObTmpFileManager::get_instance().remove(fd); - ASSERT_EQ(OB_SUCCESS, ret); -} - -void TestMultiTmpFileStress::run_normal_case() -{ - int ret = OB_SUCCESS; - int64_t fd = 0; - const int64_t timeout_ms = 5000; - TestTmpFileStress test_write(tenant_ctx_); - TestTmpFileStress test_read(tenant_ctx_); - ret = ObTmpFileManager::get_instance().open(fd, dir_id_); - ASSERT_EQ(OB_SUCCESS, ret); - STORAGE_LOG(INFO, "open file success", K(fd)); - ret = test_write.init(fd, true, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_, is_truncate_); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test_read.init(fd, false, thread_cnt_perf_file_, table_schema_, is_plain_data_, is_big_file_, is_truncate_); - ASSERT_EQ(OB_SUCCESS, ret); - test_write.start(); - test_write.wait(); - ret = ObTmpFileManager::get_instance().sync(fd, timeout_ms); - ASSERT_EQ(OB_SUCCESS, ret); - test_read.start(); - test_read.wait(); - ret = ObTmpFileManager::get_instance().remove(fd); - ASSERT_EQ(OB_SUCCESS, ret); -} - -void TestMultiTmpFileStress::run1() -{ - ObTenantEnv::set_tenant(tenant_ctx_); - if (is_plain_data_) { - run_plain_case(); - } else { - run_normal_case(); - } -} - -class TestTmpFile : public TestDataFilePrepare -{ -public: - TestTmpFile(); - virtual ~TestTmpFile(); - virtual void SetUp(); - virtual void TearDown(); -protected: - ObTableSchema table_schema_; -private: - void prepare_schema(); -}; - -TestTmpFile::TestTmpFile() - : TestDataFilePrepare(&getter, "TestTmpFile", 2 * 1024 * 1024, 2048) -{ -} - -TestTmpFile::~TestTmpFile() -{ -} - -void TestTmpFile::prepare_schema() -{ - ObColumnSchemaV2 column; - int64_t table_id = 3001; - int64_t micro_block_size = 16 * 1024; - //init table schema - table_schema_.reset(); - ASSERT_EQ(OB_SUCCESS, table_schema_.set_table_name("test_macro_file")); - table_schema_.set_tenant_id(1); - table_schema_.set_tablegroup_id(1); - table_schema_.set_database_id(1); - table_schema_.set_table_id(table_id); - table_schema_.set_rowkey_column_num(TEST_ROWKEY_COLUMN_CNT); - table_schema_.set_max_used_column_id(TEST_COLUMN_CNT); - table_schema_.set_block_size(micro_block_size); - table_schema_.set_compress_func_name("none"); - //init column - char name[OB_MAX_FILE_NAME_LENGTH]; - memset(name, 0, sizeof(name)); - for(int64_t i = 0; i < TEST_COLUMN_CNT; ++i) { - ObObjType obj_type = static_cast(i + 1); - column.reset(); - column.set_table_id(table_id); - column.set_column_id(i + OB_APP_MIN_COLUMN_ID); - sprintf(name, "test%020ld", i); - ASSERT_EQ(OB_SUCCESS, column.set_column_name(name)); - column.set_data_type(obj_type); - column.set_collation_type(CS_TYPE_UTF8MB4_GENERAL_CI); - column.set_data_length(1); - if(obj_type == common::ObIntType){ - column.set_rowkey_position(1); - } else if(obj_type == common::ObVarcharType) { - column.set_rowkey_position(2); - } else { - column.set_rowkey_position(0); - } - ASSERT_EQ(OB_SUCCESS, table_schema_.add_column(column)); - } - ObTmpFileManager::get_instance().destroy(); -} - -void TestTmpFile::SetUp() -{ - int ret = OB_SUCCESS; - const int64_t bucket_num = 1024; - const int64_t max_cache_size = 1024 * 1024 * 1024; - const int64_t block_size = common::OB_MALLOC_BIG_BLOCK_SIZE; - TestDataFilePrepare::SetUp(); - prepare_schema(); - - ret = getter.add_tenant(1, - 8L * 1024L * 1024L, 2L * 1024L * 1024L * 1024L); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObKVGlobalCache::get_instance().init(&getter, bucket_num, max_cache_size, block_size); - if (OB_INIT_TWICE == ret) { - ret = OB_SUCCESS; - } else { - ASSERT_EQ(OB_SUCCESS, ret); - } - // set observer memory limit - CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); - static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); - ObTenantEnv::set_tenant(&tenant_ctx); - ObTenantIOManager *io_service = nullptr; - EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_new(io_service)); - EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); - EXPECT_EQ(OB_SUCCESS, io_service->start()); - tenant_ctx.set(io_service); - ObTenantEnv::set_tenant(&tenant_ctx); - - ObMallocAllocator::get_instance()->set_tenant_limit(1, 8L * 1024L * 1024L * 1024L /* 8 GB */); -} - -void TestTmpFile::TearDown() -{ - table_schema_.reset(); - ObTmpFileManager::get_instance().destroy(); - ObKVGlobalCache::get_instance().destroy(); - ObTmpFileStore::get_instance().destroy(); - TestDataFilePrepare::TearDown(); -} - -TEST_F(TestTmpFile, test_big_file) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t write_size = macro_block_size * 512; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = (char *)malloc(write_size); - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - write_time = ObTimeUtility::current_time() - write_time; - io_info.buf_ = read_buf; - - // Flush all held block caches to ensure that subsequent read processes will go through I/O. - ObKVGlobalCache::get_instance().erase_cache(1, "tmp_block_cache"); - - io_info.size_ = write_size; - ret = ObTmpFileManager::get_instance().aio_read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_TRUE(handle.size_ < handle.expect_read_size_); - ASSERT_EQ(OB_SUCCESS, handle.wait()); - ASSERT_EQ(write_size, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, 100, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 100, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - io_info.size_ = write_size; - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(write_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, write_size); - ASSERT_EQ(0, cmp); - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().pread(io_info, 200, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 200, 200); - ASSERT_EQ(0, cmp); - - free(write_buf); - free(read_buf); - - STORAGE_LOG(INFO, "test_big_file"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_big_file_disable_page_cache) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t write_size = macro_block_size * 512; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = (char *)malloc(write_size); - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - io_info.disable_page_cache_ = true; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - write_time = ObTimeUtility::current_time() - write_time; - io_info.buf_ = read_buf; - - // Flush all held block caches to ensure that subsequent read processes will go through I/O. - ObKVGlobalCache::get_instance().erase_cache(1, "tmp_block_cache"); - - io_info.size_ = write_size; - ret = ObTmpFileManager::get_instance().aio_read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_TRUE(handle.size_ < handle.expect_read_size_); - ASSERT_EQ(OB_SUCCESS, handle.wait()); - ASSERT_EQ(write_size, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, 100, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 100, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - io_info.size_ = write_size; - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(write_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, write_size); - ASSERT_EQ(0, cmp); - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().pread(io_info, 200, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 200, 200); - ASSERT_EQ(0, cmp); - - free(write_buf); - free(read_buf); - - STORAGE_LOG(INFO, "test_big_file_disable_page_cache"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_multi_small_file_single_thread_read_write) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 1; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = false; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - STORAGE_LOG(INFO, "test_multi_small_file_single_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_multi_small_file_multi_thread_read_write ) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 4; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = false; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - STORAGE_LOG(INFO, "test_multi_small_file_multi_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_inner_read_offset_and_seek) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = macro_block_size + 256; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.buf_ = read_buf; - - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().read(io_info, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_ITER_END, ret); - - - ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - - io_info.size_ = 201; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(201, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, 201); - ASSERT_EQ(0, cmp); - - - ret = ObTmpFileManager::get_instance().seek(fd, 199, ObTmpFile::CUR_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - - io_info.size_ = 199; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(199, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 400, 199); - ASSERT_EQ(0, cmp); - - - STORAGE_LOG(INFO, "test_inner_read_offset_and_seek"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_single_file_single_thread_read_write) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 1; - const int64_t file_cnt = 1; - const bool is_plain_data = false; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_single_file_single_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_aio_read_and_write) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - - - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = macro_block_size + 256; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().aio_write(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ret = handle.wait(); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - handle.reset(); - - io_info.buf_ = read_buf; - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().read(io_info, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - - ret = ObTmpFileManager::get_instance().seek(fd, 100, ObTmpFile::SET_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().aio_read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ret = handle.wait(); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 100, macro_block_size); - ASSERT_EQ(0, cmp); - handle.reset(); - - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().aio_pread(io_info, 0, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ret = handle.wait(); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size); - ASSERT_EQ(0, cmp); - handle.reset(); - - - STORAGE_LOG(INFO, "test_aio_read_and_write"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_100_small_files) -{ - int ret = OB_SUCCESS; - int64_t dir = 0; - int64_t fd = 0; - int count = 100; - const int64_t timeout_ms = 5000; - TestTmpFileStress test_write(MTL_CTX()); - TestTmpFileStress test_read(MTL_CTX()); - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - while (count--) { - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - STORAGE_LOG(INFO, "open file success", K(fd)); - ret = test_write.init(fd, true, 1, &table_schema_, false, false); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test_read.init(fd, false, 1, &table_schema_, false, false); - ASSERT_EQ(OB_SUCCESS, ret); - test_write.start(); - test_write.wait(); - ret = ObTmpFileManager::get_instance().sync(fd, timeout_ms); - ASSERT_EQ(OB_SUCCESS, ret); - test_read.start(); - test_read.wait(); - } - - STORAGE_LOG(INFO, "test_1000_small_files"); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - count = 100; - while (count--) { - ret = ObTmpFileManager::get_instance().remove(count); - ASSERT_EQ(OB_SUCCESS, ret); - } -} - -TEST_F(TestTmpFile, test_single_file_multi_thread_read_write) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 4; - const int64_t file_cnt = 1; - const bool is_plain_data = false; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_single_file_multi_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_multi_file_single_thread_read_write) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 1; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_multi_file_single_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_multi_file_multi_thread_read_write) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 4; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_multi_file_multi_thread_read_write"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_write_not_macro_size) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 1; - const int64_t file_cnt = 1; - const bool is_plain_data = true; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_write_not_macro_size"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_write_less_than_macro_block_size) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [256]; - for (int i = 0; i < 256; ++i) { - write_buf[i] = static_cast(i); - } - char *read_buf = new char [256]; - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = 256; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.buf_ = read_buf; - - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, 256); - ASSERT_EQ(0, cmp); - - - io_info.size_ = 255; - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(255, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, 255); - ASSERT_EQ(0, cmp); - - - ret = ObTmpFileManager::get_instance().pread(io_info, 20, handle); - ASSERT_EQ(OB_ITER_END, ret); - ASSERT_EQ(256 - 20, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 20, 256 - 20); - ASSERT_EQ(0, cmp); - - - io_info.size_ = 20; - ret = ObTmpFileManager::get_instance().pread(io_info, 40, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(20, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 40, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - - io_info.size_ = 100; - ret = ObTmpFileManager::get_instance().pread(io_info, 156, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(100, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 156, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - - ret = ObTmpFileManager::get_instance().pread(io_info, 256, handle); - ASSERT_EQ(OB_ITER_END, ret); - - - STORAGE_LOG(INFO, "test_write_less_than_macro_block_size"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_write_more_than_one_macro_block) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = macro_block_size + 256; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.buf_ = read_buf; - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().pread(io_info, 200, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 200, 200); - ASSERT_EQ(0, cmp); - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, 200, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 200, macro_block_size); - ASSERT_EQ(0, cmp); - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, 400, handle); - ASSERT_EQ(OB_ITER_END, ret); - ASSERT_EQ(macro_block_size + 256 - 400, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 400, macro_block_size + 256 - 400); - ASSERT_EQ(0, cmp); - - io_info.size_ = 100; - ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(100, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + macro_block_size, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - io_info.size_ = 100; - ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size + 10, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(100, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + macro_block_size + 10, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size + 100, handle); - ASSERT_EQ(OB_ITER_END, ret); - ASSERT_EQ(156, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + macro_block_size + 100, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - ret = ObTmpFileManager::get_instance().pread(io_info, macro_block_size + 256, handle); - ASSERT_EQ(OB_ITER_END, ret); - - - STORAGE_LOG(INFO, "test_write_more_than_one_macro_block"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_single_dir_two_file) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd_1 = -1; - int64_t fd_2 = -1; - const int64_t macro_block_size = 64 * 1024; - ObTmpFileIOInfo io_info1; - ObTmpFileIOInfo io_info2; - ObTmpFileIOHandle handle1; - ObTmpFileIOHandle handle2; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - - ret = ObTmpFileManager::get_instance().open(fd_1, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info1.fd_ = fd_1; - io_info1.tenant_id_ = 1; - io_info1.io_desc_.set_wait_event(2); - io_info1.buf_ = write_buf; - io_info1.size_ = macro_block_size + 256; - io_info1.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - - - ret = ObTmpFileManager::get_instance().open(fd_2, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info2.fd_ = fd_2; - io_info2.tenant_id_ = 1; - io_info2.io_desc_.set_wait_event(2); - io_info2.buf_ = write_buf; - io_info2.size_ = macro_block_size + 256; - io_info2.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - - const int64_t timeout_ms = 5000; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info1); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - - write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info2); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - - - io_info1.buf_ = read_buf; - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info1, 0, handle1); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle1.get_data_size()); - int cmp = memcmp(handle1.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - - io_info2.buf_ = read_buf; - read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info2, 0, handle2); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle2.get_data_size()); - cmp = memcmp(handle2.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObTmpFileManager::get_instance().remove(fd_1); - ObTmpFileManager::get_instance().remove(fd_2); - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); -} - -/*TEST_F(TestTmpFile, test_iter_end) -{ - int old_ret = OB_SUCCESS; - int new_ret = OB_SUCCESS; - int64_t new_dir = -1; - int64_t new_fd = -1; - int old_fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo new_io_info; - ObTmpFileIOHandle new_handle; - ObMacroFileIOInfo old_io_info; - ObMacroFileIOHandle old_handle; - new_ret = ObTmpFileManager::get_instance().alloc_dir(new_dir); - ASSERT_EQ(OB_SUCCESS, new_ret); - new_ret = ObTmpFileManager::get_instance().open(new_fd, new_dir); - ASSERT_EQ(OB_SUCCESS, new_ret); - old_ret = ObMacroFileManager::get_instance().open(old_fd); - ASSERT_EQ(OB_SUCCESS, old_ret); - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - - new_io_info.fd_ = new_fd; - new_io_info.tenant_id_ = 1; - new_io_info.io_desc_.set_wait_event(2); - new_io_info.buf_ = write_buf; - new_io_info.size_ = macro_block_size + 256; - - old_io_info.fd_ = old_fd; - old_io_info.tenant_id_ = 1; - old_io_info.io_desc_.set_wait_event(2); - old_io_info.buf_ = write_buf; - old_io_info.size_ = macro_block_size + 256; - - const int64_t timeout_ms = 5000; - int64_t write_time = ObTimeUtility::current_time(); - new_ret = ObTmpFileManager::get_instance().write(new_io_info, timeout_ms); - write_time = ObTimeUtility::current_time() - write_time; - old_ret = ObMacroFileManager::get_instance().write(old_io_info, timeout_ms); - ASSERT_EQ(OB_SUCCESS, new_ret); - ASSERT_EQ(old_ret, new_ret); - - new_io_info.buf_ = read_buf; - old_io_info.buf_ = read_buf; - - - int64_t read_time = ObTimeUtility::current_time(); - new_ret = ObTmpFileManager::get_instance().pread(new_io_info, 0, timeout_ms, new_handle); - read_time = ObTimeUtility::current_time() - read_time; - old_ret = ObMacroFileManager::get_instance().pread(old_io_info, 0,timeout_ms, old_handle); - ASSERT_EQ(OB_SUCCESS, new_ret); - ASSERT_EQ(old_ret, new_ret); - - ASSERT_EQ(macro_block_size + 256, new_handle.get_data_size()); - int cmp = memcmp(new_handle.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - new_ret = ObTmpFileManager::get_instance().pread(new_io_info, macro_block_size + 256, timeout_ms, - new_handle); - old_ret = ObMacroFileManager::get_instance().pread(old_io_info, macro_block_size + 256, - timeout_ms, old_handle); - ASSERT_EQ(OB_ITER_END, new_ret); - ASSERT_EQ(OB_ITER_END, old_ret); - ASSERT_EQ(old_ret, new_ret); -}*/ - -TEST_F(TestTmpFile, test_single_dir_multi_file) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 1; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = false; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_single_dir_multi_file"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_drop_tenant_file) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 4; - const int64_t file_cnt = 4; - const bool is_plain_data = false; - const bool is_big_file = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file); - ASSERT_EQ(OB_SUCCESS, ret); - test.start(); - test.wait(); - ASSERT_EQ(0, ObTmpFileManager::get_instance().files_.map_.size()); - ASSERT_EQ(1, ObTmpFileStore::get_instance().tenant_file_stores_.size()); - - - ret = ObTmpFileManager::get_instance().remove_tenant_file(1); - ASSERT_EQ(OB_SUCCESS, ret); - - ASSERT_EQ(0, ObTmpFileManager::get_instance().files_.map_.size()); - ASSERT_EQ(0, ObTmpFileStore::get_instance().tenant_file_stores_.size()); - - int64_t fd = 0; - int count = 100; - const int64_t timeout_ms = 5000; - TestTmpFileStress test_write(MTL_CTX()); - TestTmpFileStress test_read(MTL_CTX()); - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - while (count--) { - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - STORAGE_LOG(INFO, "open file success", K(fd)); - ret = test_write.init(fd, true, 1, &table_schema_, false, false); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test_read.init(fd, false, 1, &table_schema_, false, false); - ASSERT_EQ(OB_SUCCESS, ret); - test_write.start(); - test_write.wait(); - ret = ObTmpFileManager::get_instance().sync(fd, timeout_ms); - ASSERT_EQ(OB_SUCCESS, ret); - test_read.start(); - test_read.wait(); - } - - ASSERT_EQ(100, ObTmpFileManager::get_instance().files_.map_.size()); - ASSERT_EQ(1, ObTmpFileStore::get_instance().tenant_file_stores_.size()); - - - ret = ObTmpFileManager::get_instance().remove_tenant_file(1); - ASSERT_EQ(OB_SUCCESS, ret); - - ASSERT_EQ(0, ObTmpFileManager::get_instance().files_.map_.size()); - ASSERT_EQ(0, ObTmpFileStore::get_instance().tenant_file_stores_.size()); -} - -TEST_F(TestTmpFile, test_handle_double_wait) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [256]; - for (int i = 0; i < 256; ++i) { - write_buf[i] = static_cast(i); - } - char *read_buf = new char [256]; - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = 256; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.buf_ = read_buf; - - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().pread(io_info, 0, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, 256); - ASSERT_EQ(0, cmp); - - ASSERT_EQ(OB_SUCCESS, handle.wait()); - - STORAGE_LOG(INFO, "test_handle_double_wait"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_sql_workload) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - const int64_t blk_cnt = 16; - int64_t write_size = macro_block_size * blk_cnt; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = (char *)malloc(write_size); - - - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - - const int cnt = 1; - const int64_t sql_read_size = 64 * 1024; - const int64_t sql_cnt = write_size / sql_read_size; - - for (int i = 0; i < cnt; i++) { - for (int64_t j = 0; j < sql_cnt; j++) { - io_info.size_ = sql_read_size; - io_info.buf_ = write_buf + j * sql_read_size; - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - } - } - write_time = ObTimeUtility::current_time() - write_time; - - - io_info.buf_ = read_buf; - - io_info.size_ = macro_block_size; - ret = ObTmpFileManager::get_instance().pread(io_info, 100, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf + 100, handle.get_data_size()); - ASSERT_EQ(0, cmp); - - - io_info.size_ = write_size; - int64_t read_time = ObTimeUtility::current_time(); - - ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - - for (int i = 0; i < cnt; i++) { - for (int64_t j = 0; j < sql_cnt; j++) { - io_info.size_ = sql_read_size; - io_info.buf_ = read_buf + j * sql_read_size; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(sql_read_size, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + j * sql_read_size, sql_read_size); - ASSERT_EQ(0, cmp); - } - } - read_time = ObTimeUtility::current_time() - read_time; - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().pread(io_info, 200, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf + 200, 200); - ASSERT_EQ(0, cmp); - - free(write_buf); - free(read_buf); - - - STORAGE_LOG(INFO, "test_sql_workload"); - STORAGE_LOG(INFO, "io time", K((write_size * cnt) / (1024*1024*1024)), K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_page_buddy) -{ - int ret = OB_SUCCESS; - ObArenaAllocator allocator; - ObTmpFilePageBuddy page_buddy_1; - - ret = page_buddy_1.init(allocator); - ASSERT_EQ(OB_SUCCESS, ret); - - uint8_t page_nums = 64; - uint8_t alloced_page_nums = 64; - uint8_t start_page_id = 255; - ASSERT_EQ(true, page_buddy_1.is_empty()); - ret = page_buddy_1.alloc(page_nums, start_page_id, alloced_page_nums); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(false, page_buddy_1.is_empty()); - - uint8_t start_page_id_2 = 255; - ret = page_buddy_1.alloc(page_nums, start_page_id_2, alloced_page_nums); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(false, page_buddy_1.is_empty()); - - page_buddy_1.free(start_page_id + 63, page_nums -63); - page_buddy_1.free(start_page_id_2 + 1, page_nums - 1); - page_nums = 63; - page_buddy_1.free(start_page_id, page_nums); - page_nums = 1; - page_buddy_1.free(start_page_id_2, page_nums); - STORAGE_LOG(INFO, "page buddy", K(page_buddy_1)); - ASSERT_EQ(true, page_buddy_1.is_empty()); - - ObTmpFilePageBuddy page_buddy_2; - ret = page_buddy_2.init(allocator); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(true, page_buddy_2.is_empty()); - start_page_id = 0; - ret = page_buddy_2.alloc_all_pages(); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(false, page_buddy_2.is_empty()); - - int32_t free_nums = 252 - 129; - page_buddy_2.free(start_page_id + 129, free_nums); - free_nums = 127; - page_buddy_2.free(start_page_id + 2, free_nums); - free_nums = 2; - page_buddy_2.free(start_page_id, free_nums); - STORAGE_LOG(INFO, "page buddy", K(page_buddy_2)); - ASSERT_EQ(true, page_buddy_2.is_empty()); - - for (int32_t i = 1; i < 129; i++) { - ObTmpFilePageBuddy page_buddy_3; - int32_t page_num_2 = i; - ret = page_buddy_3.init(allocator); - ASSERT_EQ(OB_SUCCESS, ret); - ret = page_buddy_3.alloc(page_num_2, start_page_id, alloced_page_nums); - ASSERT_EQ(OB_SUCCESS, ret); - page_buddy_3.free(start_page_id, alloced_page_nums); - STORAGE_LOG(INFO, "page buddy", K(page_buddy_3)); - ASSERT_EQ(true, page_buddy_3.is_empty()); - STORAGE_LOG(INFO, "page buddy", K(page_buddy_3)); - } - - ObTmpFilePageBuddy page_buddy_4; - ret = page_buddy_4.init(allocator); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(true, page_buddy_4.is_empty()); - - page_nums = 2; - alloced_page_nums = -1; - start_page_id = -1; - ASSERT_EQ(true, page_buddy_4.is_empty()); - ret = page_buddy_4.alloc(page_nums, start_page_id, alloced_page_nums); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(alloced_page_nums, page_nums); - ASSERT_EQ(false, page_buddy_4.is_empty()); -} - -TEST_F(TestTmpFile, test_page_io_info_unrelease) -{ - int ret = OB_SUCCESS; - - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - common::ObFIFOAllocator *fifo_allocator = &(store_handle.get_tenant_store()->io_allocator_); - - // case 1: construct callback - { - int64_t begin_used = fifo_allocator->used(); - { - ObTmpPageCache::ObTmpMultiPageIOCallback callback; - callback.allocator_ = fifo_allocator; - callback.cache_ = &(ObTmpPageCache::get_instance()); - callback.page_io_infos_.assign(common::ObSEArray()); - - int64_t final_used = fifo_allocator->used(); - ASSERT_EQ(final_used, begin_used); - } - ASSERT_EQ(begin_used, fifo_allocator->used()); - } - - // case 2: never call alloc_data_buf - { - int64_t begin_used = fifo_allocator->used(); - { - ObTmpPageCache::ObTmpMultiPageIOCallback callback; - callback.allocator_ = fifo_allocator; - callback.cache_ = &(ObTmpPageCache::get_instance()); - callback.page_io_infos_.assign(common::ObSEArray()); - - int64_t tmp_buf_size = 4096; - char *tmp_buf = static_cast(fifo_allocator->alloc(tmp_buf_size)); - ASSERT_EQ(callback.alloc_data_buf(tmp_buf, tmp_buf_size), OB_SUCCESS); - fifo_allocator->free(tmp_buf); - int64_t after_alloc_io_buf_used = fifo_allocator->used(); - ASSERT_EQ(after_alloc_io_buf_used, begin_used + tmp_buf_size); - - int64_t after_process_pos = fifo_allocator->used(); - ASSERT_EQ(after_process_pos, begin_used + tmp_buf_size); - ASSERT_EQ(after_alloc_io_buf_used, after_process_pos); - } - ASSERT_EQ(begin_used, fifo_allocator->used()); - } - - // case 3: call inner_process - { - int64_t begin_used = fifo_allocator->used(); - { - ObTmpPageCache::ObTmpMultiPageIOCallback callback; - callback.allocator_ = fifo_allocator; - callback.cache_ = &(ObTmpPageCache::get_instance()); - callback.page_io_infos_.assign(common::ObSEArray()); - - int64_t tmp_buf_size = 4096; - char *tmp_buf = static_cast(fifo_allocator->alloc(tmp_buf_size)); - ASSERT_EQ(callback.inner_process(tmp_buf, tmp_buf_size), OB_SUCCESS); - int64_t after_process_used = fifo_allocator->used(); - ASSERT_EQ(after_process_used, begin_used + tmp_buf_size * 2); - fifo_allocator->free(tmp_buf); - int64_t final_used = fifo_allocator->used(); - ASSERT_EQ(final_used, begin_used + tmp_buf_size); - } - ASSERT_EQ(begin_used, fifo_allocator->used()); - } -} - -TEST_F(TestTmpFile, test_tmp_file_sync) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - ObTmpFileIOInfo io_info; - ObTmpFileIOHandle handle; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t write_size = 16*1024; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - free(write_buf); - - STORAGE_LOG(INFO, "test_tmp_file_sync"); - STORAGE_LOG(INFO, "io time", K(write_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - ASSERT_EQ(1, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - ObTmpFileManager::get_instance().sync(fd, 5000); - ASSERT_EQ(0, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_tmp_file_sync_same_block) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd1, fd2 = -1; - const int64_t timeout_ms = 5000; - ObTmpFileIOHandle handle; - ObTmpFileIOInfo io_info; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - io_info.io_desc_.set_wait_event(2); - int64_t write_size = 16 *1024; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - - ret = ObTmpFileManager::get_instance().open(fd1, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd1; - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - - ret = ObTmpFileManager::get_instance().open(fd2, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd2; - write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - - free(write_buf); - - STORAGE_LOG(INFO, "test_tmp_file_sync_same_block"); - STORAGE_LOG(INFO, "io time", K(write_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - ASSERT_EQ(1, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - ObTmpFileManager::get_instance().sync(fd1, 5000); - ASSERT_EQ(1, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - ObTmpFileManager::get_instance().sync(fd2, 5000); - ASSERT_EQ(0, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd1); - ObTmpFileManager::get_instance().remove(fd2); -} - -TEST_F(TestTmpFile, test_tmp_file_wash) -{ - int ret = OB_SUCCESS; - const int64_t timeout_ms = 5000; - int count = 64 * 0.8; - int64_t dir = -1; - int64_t fd = -1; - ObTmpFileIOHandle handle; - ObTmpFileIOInfo io_info, io_info_2; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - io_info.io_desc_.set_wait_event(2); - int64_t write_size = 1024 *1024; - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - io_info.buf_ = write_buf; - io_info.size_ = write_size; - io_info.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - - io_info_2 = io_info; - - int64_t write_size_2 = 2016 *1024; - char *write_buf_2 = (char *)malloc(write_size_2); - for (int64_t i = 0; i < write_size_2; ++i) { - write_buf_2[i] = static_cast(i % 256); - } - io_info_2.buf_ = write_buf_2; - io_info_2.size_ = write_size_2; - io_info_2.io_timeout_ms_ = DEFAULT_IO_WAIT_TIME_MS; - - STORAGE_LOG(INFO, "test_tmp_file_wash"); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - - for (int64_t i=0; itmp_mem_block_manager_.t_mblk_map_.begin(); - iter != store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.end(); ++iter) { - int64_t alloc_time = iter->second->get_alloc_time(); - if (alloc_time < oldest_time) { - oldest_id = iter->first; - oldest_time = alloc_time; - } - if (alloc_time > newest_time) { - newest_id = iter->first; - newest_time = alloc_time; - } - } - ObTmpMacroBlock* wash_block; - ret = store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.get_refactored(newest_id, wash_block); - ASSERT_EQ(OB_SUCCESS, ret); - wash_block->alloc_time_ = wash_block->alloc_time_ - 60 * 1000000L; - - ObArray free_blocks; - // 1 macro block has been disked immediately because its memory has been exhausted. - ASSERT_EQ(count-1, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - - for (int64_t i=0; i< 3; i++) { - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd; - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - } - - store_handle.get_tenant_store()->tmp_mem_block_manager_.cleanup(); - - std::chrono::milliseconds(50); - ret = store_handle.get_tenant_store()->tmp_mem_block_manager_.wait_write_finish(oldest_id, ObTmpTenantMemBlockManager::get_default_timeout_ms()); - ASSERT_EQ(OB_SUCCESS, ret); - ret = store_handle.get_tenant_store()->tmp_mem_block_manager_.wait_write_finish(newest_id, ObTmpTenantMemBlockManager::get_default_timeout_ms()); - ASSERT_EQ(OB_SUCCESS, ret); - - ret = store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.get_refactored(oldest_id, wash_block); - ASSERT_NE(OB_SUCCESS, ret); - ret = store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.get_refactored(newest_id, wash_block); - ASSERT_NE(OB_SUCCESS, ret); - ASSERT_EQ(count, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - - free(write_buf); - free(write_buf_2); - - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - count = 64 * 0.8 + 3; - while (count--) { - ret = ObTmpFileManager::get_instance().remove(count); - ASSERT_EQ(OB_SUCCESS, ret); - } -} - -// test truncate, simple thread and multi thread -TEST_F(TestTmpFile, test_tmp_file_truncate) -{ - int ret = OB_SUCCESS; - int64_t dir = -1; - int64_t fd = -1; - const int64_t timeout_ms = 5000; - ObTmpFileIOHandle handle; - ObTmpFileIOInfo io_info; - const int64_t macro_block_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - char *write_buf = new char [macro_block_size + 256]; - for (int i = 0; i < macro_block_size + 256; ++i) { - write_buf[i] = static_cast(i % 256); - } - char *read_buf = new char [macro_block_size + 256]; - io_info.fd_ = fd; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_wait_event(2); - io_info.buf_ = write_buf; - io_info.size_ = macro_block_size + 256; - io_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - - int64_t write_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().write(io_info); - write_time = ObTimeUtility::current_time() - write_time; - ASSERT_EQ(OB_SUCCESS, ret); - io_info.buf_ = read_buf; - - int64_t read_time = ObTimeUtility::current_time(); - ret = ObTmpFileManager::get_instance().read(io_info, handle); - read_time = ObTimeUtility::current_time() - read_time; - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(macro_block_size + 256, handle.get_data_size()); - int cmp = memcmp(handle.get_buffer(), write_buf, macro_block_size + 256); - ASSERT_EQ(0, cmp); - - - ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - cmp = memcmp(handle.get_buffer(), write_buf, 200); - ASSERT_EQ(0, cmp); - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().seek(fd, 0, ObTmpFile::SET_SEEK); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().truncate(fd, 100); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - MEMSET(write_buf, 0, 100); - cmp = memcmp(handle.get_buffer(), write_buf, 200); - ASSERT_EQ(0, cmp); - - io_info.size_ = 200; - ret = ObTmpFileManager::get_instance().truncate(fd, 300); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().read(io_info, handle); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(200, handle.get_data_size()); - MEMSET(write_buf + 100, 0, 200); - cmp = memcmp(handle.get_buffer(), write_buf + 200, 200); - ASSERT_EQ(0, cmp); - - free(write_buf); - free(read_buf); - - STORAGE_LOG(INFO, "test_tmp_file_truncate"); - STORAGE_LOG(INFO, "io time", K(write_time), K(read_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - ObTmpFileManager::get_instance().remove(fd); -} - -TEST_F(TestTmpFile, test_multi_thread_truncate) -{ - int ret = OB_SUCCESS; - const int64_t thread_cnt = 4; - const int64_t file_cnt = 1; - const bool is_plain_data = false; - const bool is_big_file = true; - const bool is_truncate = true; - TestMultiTmpFileStress test(MTL_CTX()); - int64_t dir = -1; - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = test.init(file_cnt, dir, thread_cnt, &table_schema_, is_plain_data, is_big_file, is_truncate); - ASSERT_EQ(OB_SUCCESS, ret); - int64_t io_time = ObTimeUtility::current_time(); - test.start(); - test.wait(); - io_time = ObTimeUtility::current_time() - io_time; - - - STORAGE_LOG(INFO, "test_multi_thread_truncate"); - STORAGE_LOG(INFO, "io time", K(io_time)); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); -} - -TEST_F(TestTmpFile, test_truncate_free_block) { - int ret = OB_SUCCESS; - int count = 32; - int64_t dir = -1; - int64_t fd = -1; - ObTmpFileIOHandle handle; - ObTmpFileIOInfo io_info; - io_info.tenant_id_ = 1; - io_info.io_desc_.set_resource_group_id(THIS_WORKER.get_group_id()); - io_info.io_desc_.set_wait_event(2); - //int64_t write_size = OB_SERVER_BLOCK_MGR.get_macro_block_size(); - int64_t write_size = 1024 * 1024; - - char *write_buf = (char *)malloc(write_size); - for (int64_t i = 0; i < write_size; ++i) { - write_buf[i] = static_cast(i % 256); - } - ret = ObTmpFileManager::get_instance().alloc_dir(dir); - ASSERT_EQ(OB_SUCCESS, ret); - ret = ObTmpFileManager::get_instance().open(fd, dir); - ASSERT_EQ(OB_SUCCESS, ret); - io_info.fd_ = fd; - io_info.buf_ = write_buf; - io_info.size_ = write_size; - - for (int64_t i = 0; i < count; i++) { - ret = ObTmpFileManager::get_instance().write(io_info); - ASSERT_EQ(OB_SUCCESS, ret); - } - - STORAGE_LOG(INFO, "test_truncate_free_block"); - ObTmpTenantFileStoreHandle store_handle; - OB_TMP_FILE_STORE.get_store(1, store_handle); - ASSERT_EQ(count, store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - - for (int64_t i = 0; i < count; i++) { - ret = ObTmpFileManager::get_instance().truncate(fd, (i + 1) * write_size); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(count - i - 1 , store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - } - - ret = ObTmpFileManager::get_instance().truncate(fd, 0); - ASSERT_EQ(OB_SUCCESS, ret); - ASSERT_EQ(0 , store_handle.get_tenant_store()->tmp_mem_block_manager_.t_mblk_map_.size()); - int64_t read_size = write_size; - char *read_buf = (char *)malloc(read_size); - memset(write_buf, 0, write_size); - io_info.buf_ = read_buf; - ret = ObTmpFileManager::get_instance().read(io_info, handle); - int cmp = memcmp(read_buf, write_buf, read_size); - ASSERT_EQ(0, cmp); - free(write_buf); - free(read_buf); - - store_handle.get_tenant_store()->print_block_usage(); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(1); - ObMallocAllocator::get_instance()->print_tenant_memory_usage(500); - ObMallocAllocator::get_instance()->print_tenant_ctx_memory_usage(500); - - ObTmpFileManager::get_instance().remove(fd); - -} - - -} // end namespace unittest -} // end namespace oceanbase - -void sig_49_handler(int signo) -{ - // do nothing. -} - -int main(int argc, char **argv) -{ - struct sigaction sa; - sa.sa_handler = sig_49_handler; - sa.sa_flags = 0; - sigemptyset(&sa.sa_mask); - - // catch 49 signal and do nothing. - if (sigaction(49, &sa, NULL) == -1) { - perror("sigaction"); - return 1; - } - - system("rm -f test_tmp_file.log*"); - oceanbase::common::ObLogger::get_logger().set_log_level("INFO"); - OB_LOGGER.set_file_name("test_tmp_file.log", true, true); - testing::InitGoogleTest(&argc, argv); - return RUN_ALL_TESTS(); -} diff --git a/unittest/storage/ddl/test_chunk_compact_store.cpp b/unittest/storage/ddl/test_chunk_compact_store.cpp index c809c92893..164f4bf220 100644 --- a/unittest/storage/ddl/test_chunk_compact_store.cpp +++ b/unittest/storage/ddl/test_chunk_compact_store.cpp @@ -16,6 +16,7 @@ #include #define private public +#include "share/ob_tenant_mgr.h" #include "storage/meta_mem/ob_tenant_meta_mem_mgr.h" #include "storage/blocksstable/ob_row_generate.h" #include "storage/blocksstable/ob_data_file_prepare.h" @@ -23,7 +24,7 @@ #include "unittest/storage/blocksstable/ob_data_file_prepare.h" #include "src/sql/engine/basic/chunk_store/ob_compact_store.h" #include "src/sql/engine/basic/ob_temp_block_store.h" - +#include "mtlenv/mock_tenant_module_env.h" #undef private namespace oceanbase @@ -162,6 +163,25 @@ public: void SetUp(); void TearDown(); + int init_tenant_mgr() + { + int ret = OB_SUCCESS; + ObAddr self; + obrpc::ObSrvRpcProxy rpc_proxy; + obrpc::ObCommonRpcProxy rs_rpc_proxy; + share::ObRsMgr rs_mgr; + self.set_ip_addr("127.0.0.1", 8086); + rpc::frame::ObReqTransport req_transport(NULL, NULL); + const int64_t ulmt = 128LL << 30; + const int64_t llmt = 128LL << 30; + ret = getter.add_tenant(OB_SYS_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + ret = getter.add_tenant(OB_SERVER_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + lib::set_memory_limit(128LL << 32); + return ret; + } + protected: ObStoredRowGenerate row_generate_; ObArenaAllocator allocator_; @@ -184,8 +204,12 @@ void TestCompactChunk::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + + EXPECT_EQ(OB_SUCCESS, init_tenant_mgr()); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -193,17 +217,27 @@ void TestCompactChunk::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } void TestCompactChunk::TearDown() { - ObTmpFileManager::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); - ObTmpFileStore::get_instance().destroy(); allocator_.reuse(); row_generate_.allocator_.reuse(); TestDataFilePrepare::TearDown(); + + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } TEST_F(TestCompactChunk, test_read_writer_compact) diff --git a/unittest/storage/direct_load/test_direct_load_data_block_writer.cpp b/unittest/storage/direct_load/test_direct_load_data_block_writer.cpp index 4780a75953..a8b62475f9 100644 --- a/unittest/storage/direct_load/test_direct_load_data_block_writer.cpp +++ b/unittest/storage/direct_load/test_direct_load_data_block_writer.cpp @@ -15,15 +15,16 @@ #define protected public #include #include +#include "share/ob_tenant_mgr.h" #include "../unittest/storage/blocksstable/ob_data_file_prepare.h" #include "../unittest/storage/blocksstable/ob_row_generate.h" #include "observer/table_load/ob_table_load_partition_location.h" #include "share/ob_simple_mem_limit_getter.h" #include "share/table/ob_table_load_define.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "storage/direct_load/ob_direct_load_sstable_scanner.h" #include "storage/direct_load/ob_direct_load_sstable_compactor.h" #include "storage/ob_i_store.h" +#include "mtlenv/mock_tenant_module_env.h" namespace oceanbase { @@ -50,11 +51,29 @@ public: static const int64_t SNAPSHOT_VERSION = 2; public: - TestDataBlockWriter() : TestDataFilePrepare(&getter, "TestDataBlockWriter", 8 * 1024 * 1024, 2048){}; + TestDataBlockWriter() : TestDataFilePrepare(&getter, "TestDataBlockWriter", 2 * 1024 * 1024, 2048){}; virtual void SetUp(); virtual void TearDown(); void check_row(const ObDatumRow *next_row, const ObDatumRow *curr_row); void test_alloc(char *&ptr, const int64_t size); + int init_tenant_mgr() + { + int ret = OB_SUCCESS; + ObAddr self; + obrpc::ObSrvRpcProxy rpc_proxy; + obrpc::ObCommonRpcProxy rs_rpc_proxy; + share::ObRsMgr rs_mgr; + self.set_ip_addr("127.0.0.1", 8086); + rpc::frame::ObReqTransport req_transport(NULL, NULL); + const int64_t ulmt = 128LL << 30; + const int64_t llmt = 128LL << 30; + ret = getter.add_tenant(OB_SYS_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + ret = getter.add_tenant(OB_SERVER_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + lib::set_memory_limit(128LL << 32); + return ret; + } private: void prepare_schema(); @@ -124,7 +143,6 @@ void TestDataBlockWriter::prepare_schema() column.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_GENERAL_CI); ASSERT_EQ(OB_SUCCESS, table_schema_.add_column(column)); } - ObTmpFileManager::get_instance().destroy(); } void TestDataBlockWriter::SetUp() @@ -165,8 +183,11 @@ void TestDataBlockWriter::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + EXPECT_EQ(OB_SUCCESS, init_tenant_mgr()); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); @@ -175,13 +196,23 @@ void TestDataBlockWriter::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } void TestDataBlockWriter::TearDown() { file_mgr_->~ObDirectLoadTmpFileManager(); - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); ObKVGlobalCache::get_instance().destroy(); TestDataFilePrepare::TearDown(); } diff --git a/unittest/storage/direct_load/test_direct_load_index_block_writer.cpp b/unittest/storage/direct_load/test_direct_load_index_block_writer.cpp index 5291cee34d..d255ee5136 100644 --- a/unittest/storage/direct_load/test_direct_load_index_block_writer.cpp +++ b/unittest/storage/direct_load/test_direct_load_index_block_writer.cpp @@ -15,14 +15,15 @@ #define protected public #include #include +#include "share/ob_tenant_mgr.h" #include "../unittest/storage/blocksstable/ob_data_file_prepare.h" #include "../unittest/storage/blocksstable/ob_row_generate.h" #include "observer/table_load/ob_table_load_partition_location.h" #include "share/ob_simple_mem_limit_getter.h" -#include "storage/blocksstable/ob_tmp_file.h" #include "storage/direct_load/ob_direct_load_tmp_file.h" #include "storage/direct_load/ob_direct_load_sstable_scanner.h" #include "storage/ob_i_store.h" +#include "mtlenv/mock_tenant_module_env.h" namespace oceanbase { using namespace common; @@ -53,6 +54,25 @@ public: static void TearDownTestCase() {} void test_alloc(char *&ptr, const int64_t size); + int init_tenant_mgr() + { + int ret = OB_SUCCESS; + ObAddr self; + obrpc::ObSrvRpcProxy rpc_proxy; + obrpc::ObCommonRpcProxy rs_rpc_proxy; + share::ObRsMgr rs_mgr; + self.set_ip_addr("127.0.0.1", 8086); + rpc::frame::ObReqTransport req_transport(NULL, NULL); + const int64_t ulmt = 128LL << 30; + const int64_t llmt = 128LL << 30; + ret = getter.add_tenant(OB_SYS_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + ret = getter.add_tenant(OB_SERVER_TENANT_ID, ulmt, llmt); + EXPECT_EQ(OB_SUCCESS, ret); + lib::set_memory_limit(128LL << 32); + return ret; + } + private: void prepare_schema(); @@ -106,7 +126,6 @@ void TestIndexBlockWriter::prepare_schema() column.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_GENERAL_CI); ASSERT_EQ(OB_SUCCESS, table_schema_.add_column(column)); } - ObTmpFileManager::get_instance().destroy(); } void TestIndexBlockWriter::SetUp() @@ -134,8 +153,11 @@ void TestIndexBlockWriter::SetUp() } // set observer memory limit CHUNK_MGR.set_limit(8L * 1024L * 1024L * 1024L); - ret = ObTmpFileManager::get_instance().init(); - ASSERT_EQ(OB_SUCCESS, ret); + EXPECT_EQ(OB_SUCCESS, init_tenant_mgr()); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); + static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -143,6 +165,14 @@ void TestIndexBlockWriter::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } @@ -150,9 +180,11 @@ void TestIndexBlockWriter::TearDown() { file_mgr_->~ObDirectLoadTmpFileManager(); table_schema_.reset(); - ObTmpFileManager::get_instance().destroy(); ObKVGlobalCache::get_instance().destroy(); TestDataFilePrepare::TearDown(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); + common::ObClockGenerator::destroy(); } TEST_F(TestIndexBlockWriter, test_write_and_read) diff --git a/unittest/storage/test_io_manager.cpp b/unittest/storage/test_io_manager.cpp index cf70dac6d2..a54c7a72ab 100644 --- a/unittest/storage/test_io_manager.cpp +++ b/unittest/storage/test_io_manager.cpp @@ -27,8 +27,7 @@ #include "lib/file/file_directory_utils.h" #include "common/ob_clock_generator.h" #include "storage/blocksstable/ob_micro_block_cache.h" -#include "storage/blocksstable/ob_tmp_file_cache.h" -#include "storage/blocksstable/ob_tmp_file_store.h" +#include "storage/tmp_file/ob_tmp_file_cache.h" #include "storage/meta_mem/ob_storage_meta_cache.h" #define ASSERT_SUCC(ret) ASSERT_EQ((ret), ::oceanbase::common::OB_SUCCESS) @@ -38,6 +37,7 @@ using namespace oceanbase::lib; using namespace oceanbase::common; using namespace oceanbase::share; using namespace oceanbase::blocksstable; +using namespace oceanbase::tmp_file; #define TEST_ROOT_DIR "./" #define TEST_DATA_DIR TEST_ROOT_DIR "/data_dir" @@ -615,8 +615,8 @@ TEST_F(TestIOStruct, Test_Size) int64_t size1 = sizeof(ObAsyncSingleMicroBlockIOCallback); int64_t size2 = sizeof(ObMultiDataBlockIOCallback); int64_t size3 = sizeof(ObSyncSingleMicroBLockIOCallback); - int64_t size4 = sizeof(ObTmpPageCache::ObTmpPageIOCallback); - int64_t size5 = sizeof(ObTmpPageCache::ObTmpMultiPageIOCallback); + int64_t size4 = sizeof(ObTmpPageCache::ObTmpCachedReadPageIOCallback); + int64_t size5 = sizeof(ObTmpPageCache::ObTmpDirectReadPageIOCallback); int64_t size6 = sizeof(oceanbase::ObStorageMetaCache::ObStorageMetaIOCallback); int64_t max_callback_size = std::max({size1, size2, size3, size4, size5, size6}); int64_t size_request = sizeof(ObIORequest); diff --git a/unittest/storage/test_parallel_external_sort.cpp b/unittest/storage/test_parallel_external_sort.cpp index 160d04172e..e5e06cdd75 100644 --- a/unittest/storage/test_parallel_external_sort.cpp +++ b/unittest/storage/test_parallel_external_sort.cpp @@ -24,6 +24,7 @@ #include "share/ob_srv_rpc_proxy.h" #include "./blocksstable/ob_data_file_prepare.h" #include "share/ob_simple_mem_limit_getter.h" +#include "mtlenv/mock_tenant_module_env.h" namespace oceanbase { @@ -186,7 +187,9 @@ void TestParallelExternalSort::SetUp() { TestDataFilePrepare::SetUp(); ASSERT_EQ(OB_SUCCESS, init_tenant_mgr()); - ASSERT_EQ(OB_SUCCESS, ObTmpFileManager::get_instance().init()); + ASSERT_EQ(OB_SUCCESS, common::ObClockGenerator::init()); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpBlockCache::get_instance().init("tmp_block_cache", 1)); + ASSERT_EQ(OB_SUCCESS, tmp_file::ObTmpPageCache::get_instance().init("tmp_page_cache", 1)); static ObTenantBase tenant_ctx(OB_SYS_TENANT_ID); ObTenantEnv::set_tenant(&tenant_ctx); ObTenantIOManager *io_service = nullptr; @@ -194,14 +197,24 @@ void TestParallelExternalSort::SetUp() EXPECT_EQ(OB_SUCCESS, ObTenantIOManager::mtl_init(io_service)); EXPECT_EQ(OB_SUCCESS, io_service->start()); tenant_ctx.set(io_service); + + tmp_file::ObTenantTmpFileManager *tf_mgr = nullptr; + EXPECT_EQ(OB_SUCCESS, mtl_new_default(tf_mgr)); + EXPECT_EQ(OB_SUCCESS, tmp_file::ObTenantTmpFileManager::mtl_init(tf_mgr)); + tf_mgr->page_cache_controller_.write_buffer_pool_.default_wbp_memory_limit_ = 40*1024*1024; + EXPECT_EQ(OB_SUCCESS, tf_mgr->start()); + tenant_ctx.set(tf_mgr); + ObTenantEnv::set_tenant(&tenant_ctx); } void TestParallelExternalSort::TearDown() { allocator_.reuse(); - ObTmpFileManager::get_instance().destroy(); + tmp_file::ObTmpBlockCache::get_instance().destroy(); + tmp_file::ObTmpPageCache::get_instance().destroy(); TestDataFilePrepare::TearDown(); + common::ObClockGenerator::destroy(); destroy_tenant_mgr(); } @@ -978,6 +991,34 @@ TEST_F(TestParallelExternalSort, test_get_before_sort) ASSERT_EQ(OB_ITER_END, ret); } +TEST_F(TestParallelExternalSort, test_sort_then_get) +{ + int ret = OB_SUCCESS; + const int64_t file_buf_size = MACRO_BLOCK_SIZE; + const int64_t buf_mem_limit = 8 * 1024 * 1024L; + typedef ObExternalSort ExternalSort; + ExternalSort external_sort; + ObVectortotal_items; + TestItemCompare compare(ret); + const int64_t expire_timestamp = 0; + ret = external_sort.init(buf_mem_limit, file_buf_size, expire_timestamp, OB_SYS_TENANT_ID, &compare); + ASSERT_EQ(OB_SUCCESS, ret); + ret = generate_items(81920, false, total_items); + ASSERT_EQ(OB_SUCCESS, ret); + for (int64_t i = 0; OB_SUCC(ret) && i < total_items.size(); ++i) { + ret = external_sort.add_item(*total_items.at(i)); + ASSERT_EQ(OB_SUCCESS, ret); + } + ASSERT_EQ(OB_SUCCESS, external_sort.do_sort(true)); + const TestItem *item = NULL; + for (int64_t i = 0; OB_SUCC(ret) && i < total_items.size(); ++i) { + ret = external_sort.get_next_item(item); + ASSERT_EQ(OB_SUCCESS, ret); + } + ret = external_sort.get_next_item(item); + ASSERT_EQ(OB_ITER_END, ret); +} + } // end namespace common } // end namespace oceanbase