From 310d3c38d6c1dd5d24f1878918789cc3f7cc81ed Mon Sep 17 00:00:00 2001 From: zhengyu Date: Tue, 25 Mar 2025 14:49:56 +0800 Subject: [PATCH 01/12] [enhancement](filecache) persist cache LRU to maintain hit ratio after reboot When the system restarts, the LRU queue in memory is lost due to lack of persistence. This requires re-scanning the disk directory to load data, leading to the following issues: 1. The loading order after restart depends on directory traversal, and the original eviction order cannot be preserved. 2. If the system enters resource limit mode after restart, it may mistakenly delete frequently accessed hot data by users. In this commit, we periodically dump the LRU queue information to disk and rebuild the LRU queue upon restart. Considering that the LRU content may be extensive, we only dump the tail end (the part that will be evicted first) of the LRU queue, with the specific quantity configured by the config. Signed-off-by: zhengyu add dump with shadow queue add ut Signed-off-by: zhengyu restore ut Signed-off-by: zhengyu minor fix Signed-off-by: zhengyu add footer Signed-off-by: zhengyu restructure the code Signed-off-by: zhengyu rm dump when clear cache Signed-off-by: zhengyu add docker cases Signed-off-by: zhengyu refractor Signed-off-by: zhengyu use protobuf format Signed-off-by: zhengyu little endian Signed-off-by: zhengyu add lru_dumper test and some fix Signed-off-by: zhengyu --- be/src/common/config.cpp | 3 + be/src/common/config.h | 6 +- be/src/io/cache/block_file_cache.cpp | 217 +++++++++- be/src/io/cache/block_file_cache.h | 224 ++++++---- be/src/io/cache/cache_lru_dumper.cpp | 375 ++++++++++++++++ be/src/io/cache/cache_lru_dumper.h | 77 ++++ be/src/io/cache/file_cache_common.h | 3 + be/src/io/cache/fs_file_cache_storage.cpp | 1 + be/test/io/cache/block_file_cache_test.cpp | 217 +++------- .../io/cache/block_file_cache_test_common.h | 129 ++++++ .../cache/block_file_cache_test_lru_dump.cpp | 402 ++++++++++++++++++ be/test/io/cache/cache_lru_dumper_test.cpp | 146 +++++++ gensrc/proto/file_cache.proto | 48 +++ .../regression/suite/SuiteCluster.groovy | 3 + .../suites/demo_p0/test_lru_persist.groovy | 110 +++++ 15 files changed, 1703 insertions(+), 258 deletions(-) create mode 100644 be/src/io/cache/cache_lru_dumper.cpp create mode 100644 be/src/io/cache/cache_lru_dumper.h create mode 100644 be/test/io/cache/block_file_cache_test_common.h create mode 100644 be/test/io/cache/block_file_cache_test_lru_dump.cpp create mode 100644 be/test/io/cache/cache_lru_dumper_test.cpp create mode 100644 gensrc/proto/file_cache.proto create mode 100644 regression-test/suites/demo_p0/test_lru_persist.groovy diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2badc8ca97f4b5..8165ce503a4707 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1113,6 +1113,9 @@ DEFINE_mBool(enable_reader_dryrun_when_download_file_cache, "true"); DEFINE_mInt64(file_cache_background_monitor_interval_ms, "5000"); DEFINE_mInt64(file_cache_background_ttl_gc_interval_ms, "3000"); DEFINE_mInt64(file_cache_background_ttl_gc_batch, "1000"); +DEFINE_mInt64(file_cache_background_lru_dump_interval_ms, "60000"); +DEFINE_mInt64(file_cache_background_lru_dump_tail_record_num, "5000000"); +DEFINE_mInt64(file_cache_background_lru_log_replay_interval_ms, "1000"); DEFINE_Int32(file_cache_downloader_thread_num_min, "32"); DEFINE_Int32(file_cache_downloader_thread_num_max, "32"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 23e4a94c9eaca2..91bbbbed0cb3a6 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1152,9 +1152,13 @@ DECLARE_mBool(enable_reader_dryrun_when_download_file_cache); DECLARE_mInt64(file_cache_background_monitor_interval_ms); DECLARE_mInt64(file_cache_background_ttl_gc_interval_ms); DECLARE_mInt64(file_cache_background_ttl_gc_batch); - DECLARE_Int32(file_cache_downloader_thread_num_min); DECLARE_Int32(file_cache_downloader_thread_num_max); +// used to persist lru information before be reboot and load the info back +DECLARE_mInt64(file_cache_background_lru_dump_interval_ms); +DECLARE_mInt64(file_cache_background_lru_dump_tail_record_num); +DECLARE_mInt64(file_cache_background_lru_log_replay_interval_ms); + // inverted index searcher cache // cache entry stay time after lookup DECLARE_mInt32(index_cache_entry_stay_time_after_lookup_s); diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index 1e16d79dc5c057..fe9c256a8df7fd 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -20,8 +20,12 @@ #include "io/cache/block_file_cache.h" +#include +#include + #include "common/status.h" #include "cpp/sync_point.h" +#include "gen_cpp/file_cache.pb.h" #if defined(__APPLE__) #include @@ -220,7 +224,8 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, _cache_base_path.c_str(), "file_cache_storage_async_remove_latency_us"); _evict_in_advance_latency_us = std::make_shared( _cache_base_path.c_str(), "file_cache_evict_in_advance_latency_us"); - + _lru_dump_latency_us = std::make_shared( + _cache_base_path.c_str(), "file_cache_lru_dump_latency_us"); _recycle_keys_length_recorder = std::make_shared( _cache_base_path.c_str(), "file_cache_recycle_keys_length"); _ttl_gc_latency_us = std::make_shared(_cache_base_path.c_str(), @@ -234,6 +239,8 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, 24 * 60 * 60); _ttl_queue = LRUQueue(cache_settings.ttl_queue_size, cache_settings.ttl_queue_elements, std::numeric_limits::max()); + + _lru_dumper = std::make_unique(this); if (cache_settings.storage == "memory") { _storage = std::make_unique(); _cache_base_path = "memory"; @@ -348,12 +355,35 @@ Status BlockFileCache::initialize() { Status BlockFileCache::initialize_unlocked(std::lock_guard& cache_lock) { DCHECK(!_is_initialized); _is_initialized = true; + if (config::file_cache_background_lru_dump_tail_record_num > 0) { + // requirements: + // 1. restored data should not overwrite the last dump + // 2. restore should happen before load and async load + // 3. all queues should be restored sequencially to avoid conflict + // TODO(zhengyu): we can parralize them but will increase complexity, so lets check the time cost + // to see if any improvement is a necessary + restore_lru_queues_from_disk(cache_lock); + } RETURN_IF_ERROR(_storage->init(this)); _cache_background_monitor_thread = std::thread(&BlockFileCache::run_background_monitor, this); + pthread_setname_np(_cache_background_monitor_thread.native_handle(), "run_background_monitor"); _cache_background_ttl_gc_thread = std::thread(&BlockFileCache::run_background_ttl_gc, this); + pthread_setname_np(_cache_background_ttl_gc_thread.native_handle(), "run_background_ttl_gc"); _cache_background_gc_thread = std::thread(&BlockFileCache::run_background_gc, this); + pthread_setname_np(_cache_background_gc_thread.native_handle(), "run_background_gc"); _cache_background_evict_in_advance_thread = std::thread(&BlockFileCache::run_background_evict_in_advance, this); + pthread_setname_np(_cache_background_evict_in_advance_thread.native_handle(), + "run_background_evict_in_advance"); + + // Initialize LRU dump thread and restore queues + _cache_background_lru_dump_thread = std::thread(&BlockFileCache::run_background_lru_dump, this); + pthread_setname_np(_cache_background_lru_dump_thread.native_handle(), + "run_background_lru_dump"); + _cache_background_lru_log_replay_thread = + std::thread(&BlockFileCache::run_background_lru_log_replay, this); + pthread_setname_np(_cache_background_lru_log_replay_thread.native_handle(), + "run_background_lru_log_replay"); return Status::OK(); } @@ -369,6 +399,9 @@ void BlockFileCache::use_cell(const FileBlockCell& cell, FileBlocks* result, boo if (cell.queue_iterator && move_iter_flag) { queue.move_to_end(*cell.queue_iterator, cache_lock); } + record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), + CacheLRULogType::MOVETOBACK, cell.file_block->_key.hash, + cell.file_block->_key.offset, cell.size()); cell.update_atime(); } @@ -442,10 +475,16 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte if (st.ok()) { auto& queue = get_queue(origin_type); queue.remove(cell.queue_iterator.value(), cache_lock); + record_queue_event(get_lru_log_queue(origin_type), CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); auto& ttl_queue = get_queue(FileCacheType::TTL); cell.queue_iterator = ttl_queue.add(cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::TTL), CacheLRULogType::ADD, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); } else { LOG_WARNING("Failed to change key meta").error(st); } @@ -482,11 +521,18 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte if (cell.queue_iterator) { auto& ttl_queue = get_queue(FileCacheType::TTL); ttl_queue.remove(cell.queue_iterator.value(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::TTL), + CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } auto& queue = get_queue(FileCacheType::NORMAL); cell.queue_iterator = queue.add(cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::NORMAL), + CacheLRULogType::ADD, cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change key meta").error(st); } @@ -555,7 +601,9 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte } std::string BlockFileCache::clear_file_cache_async() { + // TODO(zhengyu): rm lru dump file before and after clear file cache LOG(INFO) << "start clear_file_cache_async, path=" << _cache_base_path; + _lru_dumper->remove_lru_dump_files(); int64_t num_cells_all = 0; int64_t num_cells_to_delete = 0; int64_t num_cells_wait_recycle = 0; @@ -597,6 +645,7 @@ std::string BlockFileCache::clear_file_cache_async() { << " num_cells_wait_recycle=" << num_cells_wait_recycle; auto msg = ss.str(); LOG(INFO) << msg; + _lru_dumper->remove_lru_dump_files(); return msg; } @@ -791,6 +840,8 @@ BlockFileCache::FileBlockCell* BlockFileCache::add_cell(const UInt128Wrapper& ha auto& queue = get_queue(cell.file_block->cache_type()); cell.queue_iterator = queue.add(hash, offset, size, cache_lock); + record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), CacheLRULogType::ADD, + cell.file_block->get_hash_value(), cell.file_block->offset(), cell.size()); if (cell.file_block->cache_type() == FileCacheType::TTL) { if (_key_to_time.find(hash) == _key_to_time.end()) { @@ -828,7 +879,7 @@ size_t BlockFileCache::try_release() { return trash.size(); } -BlockFileCache::LRUQueue& BlockFileCache::get_queue(FileCacheType type) { +LRUQueue& BlockFileCache::get_queue(FileCacheType type) { switch (type) { case FileCacheType::INDEX: return _index_queue; @@ -844,7 +895,7 @@ BlockFileCache::LRUQueue& BlockFileCache::get_queue(FileCacheType type) { return _normal_queue; } -const BlockFileCache::LRUQueue& BlockFileCache::get_queue(FileCacheType type) const { +const LRUQueue& BlockFileCache::get_queue(FileCacheType type) const { switch (type) { case FileCacheType::INDEX: return _index_queue; @@ -971,7 +1022,7 @@ bool BlockFileCache::try_reserve(const UInt128Wrapper& hash, const CacheContext& size_t cur_cache_size = _cur_cache_size; size_t query_context_cache_size = query_context->get_cache_size(cache_lock); - std::vector ghost; + std::vector ghost; std::vector to_evict; size_t max_size = queue.get_max_size(); @@ -1074,11 +1125,18 @@ bool BlockFileCache::remove_if_ttl_file_blocks(const UInt128Wrapper& file_key, b if (st.ok()) { if (cell.queue_iterator) { ttl_queue.remove(cell.queue_iterator.value(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::TTL), + CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } auto& queue = get_queue(FileCacheType::NORMAL); cell.queue_iterator = queue.add( cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::NORMAL), + CacheLRULogType::ADD, cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change cache type to normal").error(st); } @@ -1367,6 +1425,9 @@ void BlockFileCache::remove(FileBlockSPtr file_block, T& cache_lock, U& block_lo if (cell->queue_iterator) { auto& queue = get_queue(file_block->cache_type()); queue.remove(*cell->queue_iterator, cache_lock); + record_queue_event(get_lru_log_queue(file_block->cache_type()), CacheLRULogType::REMOVE, + cell->file_block->get_hash_value(), cell->file_block->offset(), + cell->size()); } *_queue_evict_size_metrics[static_cast(file_block->cache_type())] << file_block->range().size(); @@ -1482,9 +1543,8 @@ BlockFileCache::FileBlockCell::FileBlockCell(FileBlockSPtr file_block, } } -BlockFileCache::LRUQueue::Iterator BlockFileCache::LRUQueue::add( - const UInt128Wrapper& hash, size_t offset, size_t size, - std::lock_guard& /* cache_lock */) { +LRUQueue::Iterator LRUQueue::add(const UInt128Wrapper& hash, size_t offset, size_t size, + std::lock_guard& /* cache_lock */) { cache_size += size; auto iter = queue.insert(queue.end(), FileKeyAndOffset(hash, offset, size)); map.insert(std::make_pair(std::make_pair(hash, offset), iter)); @@ -1493,35 +1553,32 @@ BlockFileCache::LRUQueue::Iterator BlockFileCache::LRUQueue::add( template requires IsXLock -void BlockFileCache::LRUQueue::remove(Iterator queue_it, T& /* cache_lock */) { +void LRUQueue::remove(Iterator queue_it, T& /* cache_lock */) { cache_size -= queue_it->size; map.erase(std::make_pair(queue_it->hash, queue_it->offset)); queue.erase(queue_it); } -void BlockFileCache::LRUQueue::remove_all(std::lock_guard& /* cache_lock */) { +void LRUQueue::remove_all(std::lock_guard& /* cache_lock */) { queue.clear(); map.clear(); cache_size = 0; } -void BlockFileCache::LRUQueue::move_to_end(Iterator queue_it, - std::lock_guard& /* cache_lock */) { +void LRUQueue::move_to_end(Iterator queue_it, std::lock_guard& /* cache_lock */) { queue.splice(queue.end(), queue, queue_it); } -bool BlockFileCache::LRUQueue::contains(const UInt128Wrapper& hash, size_t offset, - std::lock_guard& /* cache_lock */) const { +bool LRUQueue::contains(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& /* cache_lock */) const { return map.find(std::make_pair(hash, offset)) != map.end(); } -BlockFileCache::LRUQueue::Iterator BlockFileCache::LRUQueue::get( - const UInt128Wrapper& hash, size_t offset, - std::lock_guard& /* cache_lock */) const { +LRUQueue::Iterator LRUQueue::get(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& /* cache_lock */) const { return map.find(std::make_pair(hash, offset))->second; } -std::string BlockFileCache::LRUQueue::to_string( - std::lock_guard& /* cache_lock */) const { +std::string LRUQueue::to_string(std::lock_guard& /* cache_lock */) const { std::string result; for (const auto& [hash, offset, size] : queue) { if (!result.empty()) { @@ -1583,9 +1640,15 @@ void BlockFileCache::change_cache_type(const UInt128Wrapper& hash, size_t offset auto& cur_queue = get_queue(cell.file_block->cache_type()); DCHECK(cell.queue_iterator.has_value()); cur_queue.remove(*cell.queue_iterator, cache_lock); + record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), + CacheLRULogType::REMOVE, cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); auto& new_queue = get_queue(new_type); cell.queue_iterator = new_queue.add(hash, offset, cell.file_block->range().size(), cache_lock); + record_queue_event(get_lru_log_queue(new_type), CacheLRULogType::ADD, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); } } } @@ -1982,9 +2045,15 @@ void BlockFileCache::modify_expiration_time(const UInt128Wrapper& hash, if (st.ok()) { auto& queue = get_queue(origin_type); queue.remove(cell.queue_iterator.value(), cache_lock); + record_queue_event(get_lru_log_queue(origin_type), CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); auto& ttl_queue = get_queue(FileCacheType::TTL); cell.queue_iterator = ttl_queue.add(hash, cell.file_block->offset(), cell.file_block->range().size(), cache_lock); + record_queue_event(get_lru_log_queue(FileCacheType::TTL), CacheLRULogType::ADD, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); } if (!st.ok()) { LOG_WARNING("").error(st); @@ -2066,6 +2135,7 @@ bool BlockFileCache::try_reserve_during_async_load(size_t size, } std::string BlockFileCache::clear_file_cache_directly() { + _lru_dumper->remove_lru_dump_files(); using namespace std::chrono; std::stringstream ss; auto start = steady_clock::now(); @@ -2113,9 +2183,9 @@ std::string BlockFileCache::clear_file_cache_directly() { << " cache_size=" << cache_size << " index_queue_size=" << index_queue_size << " normal_queue_size=" << normal_queue_size << " disposible_queue_size=" << disposible_queue_size << "ttl_queue_size=" << ttl_queue_size; - auto msg = ss.str(); LOG(INFO) << msg; + _lru_dumper->remove_lru_dump_files(); return msg; } @@ -2144,6 +2214,115 @@ void BlockFileCache::update_ttl_atime(const UInt128Wrapper& hash) { }; } +BlockFileCache::CacheLRULogQueue& BlockFileCache::get_lru_log_queue(FileCacheType type) { + switch (type) { + case FileCacheType::INDEX: + return _index_lru_log_queue; + case FileCacheType::DISPOSABLE: + return _disposable_lru_log_queue; + case FileCacheType::NORMAL: + return _normal_lru_log_queue; + case FileCacheType::TTL: + return _ttl_lru_log_queue; + default: + DCHECK(false); + } + return _normal_lru_log_queue; +} + +void BlockFileCache::record_queue_event(CacheLRULogQueue& log_queue, CacheLRULogType log_type, + const UInt128Wrapper hash, const size_t offset, + const size_t size) { + log_queue.push_back(std::make_unique(log_type, hash, offset, size)); +} + +void BlockFileCache::replay_queue_event(CacheLRULogQueue& log_queue, LRUQueue& shadow_queue) { + // we don't need the real cache lock for the shadow queue, but we do need a lock to prevent read/write contension + std::lock_guard lru_log_lock(_mutex_lru_log); + while (!log_queue.empty()) { + auto log = std::move(log_queue.front()); + log_queue.pop_front(); + try { + switch (log->type) { + case CacheLRULogType::ADD: { + shadow_queue.add(log->hash, log->offset, log->size, lru_log_lock); + break; + } + case CacheLRULogType::REMOVE: { + auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); + if (it != shadow_queue.end()) { + shadow_queue.remove(it, lru_log_lock); + } else { + LOG(WARNING) << "REMOVE failed, doesn't exist in shadow queue"; + } + break; + } + case CacheLRULogType::MOVETOBACK: { + auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); + if (it != shadow_queue.end()) { + shadow_queue.move_to_end(it, lru_log_lock); + } else { + LOG(WARNING) << "MOVETOBACK failed, doesn't exist in shadow queue"; + } + break; + } + default: + LOG(WARNING) << "Unknown CacheLRULogType: " << static_cast(log->type); + break; + } + } catch (const std::exception& e) { + LOG(WARNING) << "Failed to replay queue event: " << e.what(); + } + } +} + +void BlockFileCache::run_background_lru_log_replay() { + while (!_close) { + int64_t interval_ms = config::file_cache_background_lru_log_replay_interval_ms; + { + std::unique_lock close_lock(_close_mtx); + _close_cv.wait_for(close_lock, std::chrono::milliseconds(interval_ms)); + if (_close) { + break; + } + } + + replay_queue_event(_ttl_lru_log_queue, _shadow_ttl_queue); + replay_queue_event(_index_lru_log_queue, _shadow_index_queue); + replay_queue_event(_normal_lru_log_queue, _shadow_normal_queue); + replay_queue_event(_disposable_lru_log_queue, _shadow_disposable_queue); + + //TODO(zhengyu): add debug facilities to check diff between real and shadow queue + } +} + +void BlockFileCache::run_background_lru_dump() { + while (!_close) { + int64_t interval_ms = config::file_cache_background_lru_dump_interval_ms; + { + std::unique_lock close_lock(_close_mtx); + _close_cv.wait_for(close_lock, std::chrono::milliseconds(interval_ms)); + if (_close) { + break; + } + } + + if (config::file_cache_background_lru_dump_tail_record_num > 0) { + _lru_dumper->dump_queue(_shadow_disposable_queue, "disposable"); + _lru_dumper->dump_queue(_shadow_index_queue, "index"); + _lru_dumper->dump_queue(_shadow_normal_queue, "normal"); + _lru_dumper->dump_queue(_shadow_ttl_queue, "ttl"); + } + } +} + +void BlockFileCache::restore_lru_queues_from_disk(std::lock_guard& cache_lock) { + _lru_dumper->restore_queue(_disposable_queue, "disposable", cache_lock); + _lru_dumper->restore_queue(_index_queue, "index", cache_lock); + _lru_dumper->restore_queue(_normal_queue, "normal", cache_lock); + _lru_dumper->restore_queue(_ttl_queue, "ttl", cache_lock); +} + std::map BlockFileCache::get_stats() { std::map stats; stats["hits_ratio"] = (double)_hit_ratio->get_value(); diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 7c046cc162783a..23d8b76129fb9d 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -26,6 +26,7 @@ #include #include +#include "io/cache/cache_lru_dumper.h" #include "io/cache/file_block.h" #include "io/cache/file_cache_common.h" #include "io/cache/file_cache_storage.h" @@ -78,6 +79,86 @@ concept IsXLock = std::same_as> || class FSFileCacheStorage; +class LRUQueue { +public: + LRUQueue() = default; + LRUQueue(size_t max_size, size_t max_element_size, int64_t hot_data_interval) + : max_size(max_size), + max_element_size(max_element_size), + hot_data_interval(hot_data_interval) {} + + struct HashFileKeyAndOffset { + std::size_t operator()(const std::pair& pair) const { + return KeyHash()(pair.first) + pair.second; + } + }; + + struct FileKeyAndOffset { + UInt128Wrapper hash; + size_t offset; + size_t size; + + FileKeyAndOffset(const UInt128Wrapper& hash, size_t offset, size_t size) + : hash(hash), offset(offset), size(size) {} + }; + + using Iterator = typename std::list::iterator; + + size_t get_max_size() const { return max_size; } + size_t get_max_element_size() const { return max_element_size; } + + template + requires IsXLock + size_t get_capacity(T& /* cache_lock */) const { + return cache_size; + } + + size_t get_capacity_unsafe() const { return cache_size; } + + size_t get_elements_num_unsafe() const { return queue.size(); } + + size_t get_elements_num(std::lock_guard& /* cache_lock */) const { + return queue.size(); + } + + Iterator add(const UInt128Wrapper& hash, size_t offset, size_t size, + std::lock_guard& cache_lock); + template + requires IsXLock + void remove(Iterator queue_it, T& cache_lock); + + void move_to_end(Iterator queue_it, std::lock_guard& cache_lock); + + std::string to_string(std::lock_guard& cache_lock) const; + + bool contains(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& cache_lock) const; + + Iterator begin() { return queue.begin(); } + + Iterator end() { return queue.end(); } + + void remove_all(std::lock_guard& cache_lock); + + Iterator get(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& /* cache_lock */) const; + + int64_t get_hot_data_interval() const { return hot_data_interval; } + + void clear(std::lock_guard& cache_lock) { + queue.clear(); + map.clear(); + cache_size = 0; + } + + size_t max_size; + size_t max_element_size; + std::list queue; + std::unordered_map, Iterator, HashFileKeyAndOffset> map; + size_t cache_size = 0; + int64_t hot_data_interval {0}; +}; + // The BlockFileCache is responsible for the management of the blocks // The current strategies are lru and ttl. class BlockFileCache { @@ -85,6 +166,7 @@ class BlockFileCache { friend class MemFileCacheStorage; friend class FileBlock; friend struct FileBlocksHolder; + friend class CacheLRUDumper; public: static std::string cache_type_to_string(FileCacheType type); @@ -94,7 +176,7 @@ class BlockFileCache { BlockFileCache(const std::string& cache_base_path, const FileCacheSettings& cache_settings); - ~BlockFileCache() { + virtual ~BlockFileCache() { { std::lock_guard lock(_close_mtx); _close = true; @@ -112,6 +194,12 @@ class BlockFileCache { if (_cache_background_evict_in_advance_thread.joinable()) { _cache_background_evict_in_advance_thread.join(); } + if (_cache_background_lru_dump_thread.joinable()) { + _cache_background_lru_dump_thread.join(); + } + if (_cache_background_lru_log_replay_thread.joinable()) { + _cache_background_lru_log_replay_thread.join(); + } } /// Restore cache from local filesystem. @@ -216,86 +304,25 @@ class BlockFileCache { // for be UTs std::map get_stats_unsafe(); - class LRUQueue { - public: - LRUQueue() = default; - LRUQueue(size_t max_size, size_t max_element_size, int64_t hot_data_interval) - : max_size(max_size), - max_element_size(max_element_size), - hot_data_interval(hot_data_interval) {} - - struct HashFileKeyAndOffset { - std::size_t operator()(const std::pair& pair) const { - return KeyHash()(pair.first) + pair.second; - } - }; - - struct FileKeyAndOffset { - UInt128Wrapper hash; - size_t offset; - size_t size; - - FileKeyAndOffset(const UInt128Wrapper& hash, size_t offset, size_t size) - : hash(hash), offset(offset), size(size) {} - }; - - using Iterator = typename std::list::iterator; - - size_t get_max_size() const { return max_size; } - size_t get_max_element_size() const { return max_element_size; } - - template - requires IsXLock - size_t get_capacity(T& /* cache_lock */) const { - return cache_size; - } - - size_t get_capacity_unsafe() const { return cache_size; } - - size_t get_elements_num_unsafe() const { return queue.size(); } - - size_t get_elements_num(std::lock_guard& /* cache_lock */) const { - return queue.size(); - } - - Iterator add(const UInt128Wrapper& hash, size_t offset, size_t size, - std::lock_guard& cache_lock); - template - requires IsXLock - void remove(Iterator queue_it, T& cache_lock); - - void move_to_end(Iterator queue_it, std::lock_guard& cache_lock); - - std::string to_string(std::lock_guard& cache_lock) const; - - bool contains(const UInt128Wrapper& hash, size_t offset, - std::lock_guard& cache_lock) const; - - Iterator begin() { return queue.begin(); } - - Iterator end() { return queue.end(); } - - void remove_all(std::lock_guard& cache_lock); - - Iterator get(const UInt128Wrapper& hash, size_t offset, - std::lock_guard& /* cache_lock */) const; - - int64_t get_hot_data_interval() const { return hot_data_interval; } + enum class CacheLRULogType { + ADD = 0, // all of the integer types + REMOVE = 1, + MOVETOBACK = 2, + INVALID = 3, + }; - void clear(std::lock_guard& cache_lock) { - queue.clear(); - map.clear(); - cache_size = 0; - } + struct CacheLRULog { + CacheLRULogType type = CacheLRULogType::INVALID; + UInt128Wrapper hash; + size_t offset; + size_t size; - size_t max_size; - size_t max_element_size; - std::list queue; - std::unordered_map, Iterator, HashFileKeyAndOffset> map; - size_t cache_size = 0; - int64_t hot_data_interval {0}; + CacheLRULog(CacheLRULogType t, UInt128Wrapper h, size_t o, size_t s) + : type(t), hash(h), offset(o), size(s) {} }; + using CacheLRULogQueue = std::list>; + using AccessRecord = std::unordered_map; @@ -389,8 +416,9 @@ class BlockFileCache { FileBlockCell(const FileBlockCell&) = delete; }; - BlockFileCache::LRUQueue& get_queue(FileCacheType type); - const BlockFileCache::LRUQueue& get_queue(FileCacheType type) const; + LRUQueue& get_queue(FileCacheType type); + const LRUQueue& get_queue(FileCacheType type) const; + CacheLRULogQueue& get_lru_log_queue(FileCacheType type); template requires IsXLock && IsXLock @@ -403,9 +431,9 @@ class BlockFileCache { requires IsXLock FileBlockCell* get_cell(const UInt128Wrapper& hash, size_t offset, T& cache_lock); - FileBlockCell* add_cell(const UInt128Wrapper& hash, const CacheContext& context, size_t offset, - size_t size, FileBlock::State state, - std::lock_guard& cache_lock); + virtual FileBlockCell* add_cell(const UInt128Wrapper& hash, const CacheContext& context, + size_t offset, size_t size, FileBlock::State state, + std::lock_guard& cache_lock); Status initialize_unlocked(std::lock_guard& cache_lock); @@ -463,6 +491,9 @@ class BlockFileCache { void run_background_monitor(); void run_background_ttl_gc(); void run_background_gc(); + void run_background_lru_log_replay(); + void run_background_lru_dump(); + void restore_lru_queues_from_disk(std::lock_guard& cache_lock); void run_background_evict_in_advance(); bool try_reserve_from_other_queue_by_time_interval(FileCacheType cur_type, @@ -489,6 +520,21 @@ class BlockFileCache { std::lock_guard& cache_lock, size_t& cur_removed_size, bool evict_in_advance); + void record_queue_event(CacheLRULogQueue& log_queue, CacheLRULogType log_type, + const UInt128Wrapper hash, const size_t offset, const size_t size); + void replay_queue_event(CacheLRULogQueue& log_queue, LRUQueue& shadown_queue); + + Status check_ofstream_status(std::ofstream& out, std::string& filename); + Status dump_one_lru_entry(std::ofstream& out, std::string& filename, const UInt128Wrapper& hash, + size_t offset, size_t size); + Status finalize_dump(std::ofstream& out, size_t entry_num, std::string& tmp_filename, + std::string& final_filename, size_t& file_size); + Status check_ifstream_status(std::ifstream& in, std::string& filename); + Status parse_dump_footer(std::ifstream& in, std::string& filename, size_t& entry_num); + Status parse_one_lru_entry(std::ifstream& in, std::string& filename, UInt128Wrapper& hash, + size_t& offset, size_t& size); + void remove_lru_dump_files(); + // info std::string _cache_base_path; size_t _capacity = 0; @@ -503,6 +549,8 @@ class BlockFileCache { std::thread _cache_background_ttl_gc_thread; std::thread _cache_background_gc_thread; std::thread _cache_background_evict_in_advance_thread; + std::thread _cache_background_lru_dump_thread; + std::thread _cache_background_lru_log_replay_thread; std::atomic_bool _async_open_done {false}; // disk space or inode is less than the specified value bool _disk_resource_limit_mode {false}; @@ -528,9 +576,20 @@ class BlockFileCache { LRUQueue _normal_queue; LRUQueue _disposable_queue; LRUQueue _ttl_queue; + LRUQueue _shadow_index_queue; + LRUQueue _shadow_normal_queue; + LRUQueue _shadow_disposable_queue; + LRUQueue _shadow_ttl_queue; // keys for async remove RecycleFileCacheKeys _recycle_keys; + CacheLRULogQueue _ttl_lru_log_queue; + CacheLRULogQueue _index_lru_log_queue; + CacheLRULogQueue _normal_lru_log_queue; + CacheLRULogQueue _disposable_lru_log_queue; + std::mutex _mutex_lru_log; + + std::unique_ptr _lru_dumper; // metrics std::shared_ptr> _cache_capacity_metrics; @@ -581,6 +640,7 @@ class BlockFileCache { // coz it will use other members of BlockFileCache // so join this async load thread first std::unique_ptr _storage; + std::shared_ptr _lru_dump_latency_us; }; } // namespace doris::io diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp new file mode 100644 index 00000000000000..d0d741559f8b4a --- /dev/null +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -0,0 +1,375 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "io/cache/cache_lru_dumper.h" + +#include "io/cache/block_file_cache.h" +#include "io/cache/cache_lru_dumper.h" + +namespace doris::io { +Status CacheLRUDumper::check_ofstream_status(std::ofstream& out, std::string& filename) { + if (!out.good()) { + std::ios::iostate state = out.rdstate(); + std::stringstream err_msg; + if (state & std::ios::eofbit) { + err_msg << "End of file reached."; + } + if (state & std::ios::failbit) { + err_msg << "Input/output operation failed, err_code: " << strerror(errno); + } + if (state & std::ios::badbit) { + err_msg << "Serious I/O error occurred, err_code: " << strerror(errno); + } + out.close(); + std::string warn_msg = fmt::format("dump lru writing failed, file={}, {}", filename, + err_msg.str().c_str()); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + + return Status::OK(); +} + +Status CacheLRUDumper::check_ifstream_status(std::ifstream& in, std::string& filename) { + if (!in.good()) { + std::ios::iostate state = in.rdstate(); + std::stringstream err_msg; + if (state & std::ios::eofbit) { + err_msg << "End of file reached."; + } + if (state & std::ios::failbit) { + err_msg << "Input/output operation failed, err_code: " << strerror(errno); + } + if (state & std::ios::badbit) { + err_msg << "Serious I/O error occurred, err_code: " << strerror(errno); + } + in.close(); + std::string warn_msg = std::string( + fmt::format("dump lru reading failed, file={}, {}", filename, err_msg.str())); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + + return Status::OK(); +} + +Status CacheLRUDumper::dump_one_lru_entry(std::ofstream& out, std::string& filename, + const UInt128Wrapper& hash, size_t offset, size_t size) { + // Dump file format description: + // +-----------------------------------------------+ + // | LRUDumpEntryGroupPb_1 | + // +-----------------------------------------------+ + // | LRUDumpEntryGroupPb_2 | + // +-----------------------------------------------+ + // | LRUDumpEntryGroupPb_3 | + // +-----------------------------------------------+ + // | ... | + // +-----------------------------------------------+ + // | LRUDumpEntryGroupPb_n | + // +-----------------------------------------------+ + // | LRUDumpMetaPb (queueName, GroupList)| + // +-----------------------------------------------+ + // | FOOTER_OFFSET (8Bytes) | + // +-----------------------------------------------+ + // | CHECKSUM (4Bytes)|VERSION (1Byte)| MAGIC | + // +-----------------------------------------------+ + // + // why we are not using protobuf as a whole? + // AFAIK, current protobuf version dose not support streaming mode, + // so that we need to store all the message in memory which will + // consume loads of RAMs. + // Instead, we use protobuf serialize each of the single entry + // and provide the version field in the footer for upgrade + + ::doris::io::cache::LRUDumpEntryPb* entry = _current_dump_group.add_entries(); + ::doris::io::cache::UInt128WrapperPb* hash_pb = entry->mutable_hash(); + hash_pb->set_high(hash.high()); + hash_pb->set_low(hash.low()); + entry->set_offset(offset); + entry->set_size(size); + + _current_dump_group_count++; + if (_current_dump_group_count >= 10000) { + RETURN_IF_ERROR(flush_current_group(out, filename)); + } + return Status::OK(); +} + +Status CacheLRUDumper::flush_current_group(std::ofstream& out, std::string& filename) { + if (_current_dump_group_count == 0) { + return Status::OK(); + } + + // Record current position as group start offset + size_t group_start = out.tellp(); + + // Serialize and write the group + std::string serialized; + LOG(INFO) << "Before serialization: " << _current_dump_group.DebugString(); + if (!_current_dump_group.SerializeToString(&serialized)) { + return Status::InternalError("Failed to serialize LRUDumpEntryGroupPb"); + } + LOG(INFO) << "Serialized size: " << serialized.size(); + out.write(serialized.data(), serialized.size()); + RETURN_IF_ERROR(check_ofstream_status(out, filename)); + + // Record group metadata + ::doris::io::cache::EntryGroupOffsetSizePb* group_info = _dump_meta.add_group_offset_size(); + group_info->set_offset(group_start); + group_info->set_size(serialized.size()); + + // Reset for next group + _current_dump_group.Clear(); + _current_dump_group_count = 0; + return Status::OK(); +} + +Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, + std::string& tmp_filename, std::string& final_filename, + size_t& file_size) { + // Flush any remaining entries + if (_current_dump_group_count > 0) { + RETURN_IF_ERROR(flush_current_group(out, tmp_filename)); + } + + // Write meta information + _dump_meta.set_entry_num(entry_num); + size_t meta_offset = out.tellp(); + LOG(INFO) << "dump meta: " << _dump_meta.DebugString(); + std::string meta_serialized; + if (!_dump_meta.SerializeToString(&meta_serialized)) { + std::string warn_msg = + fmt::format("Failed to serialize LRUDumpMetaPb, file={}", tmp_filename); + LOG(WARNING) << warn_msg; + return Status::InternalError("warn_msg"); + } + out.write(meta_serialized.data(), meta_serialized.size()); + RETURN_IF_ERROR(check_ofstream_status(out, tmp_filename)); + + // Write footer + Footer footer; + footer.meta_offset = htole64(meta_offset); // Explicitly convert to little-endian + footer.checksum = 0; // TODO: Calculate checksum + footer.version = 1; + std::memcpy(footer.magic, "DOR", 3); + + out.write(reinterpret_cast(&footer), sizeof(footer)); + RETURN_IF_ERROR(check_ofstream_status(out, tmp_filename)); + + out.close(); + + // Rename tmp to formal file + if (std::rename(tmp_filename.c_str(), final_filename.c_str()) != 0) { + std::remove(tmp_filename.c_str()); + file_size = std::filesystem::file_size(final_filename); + } + _dump_meta.Clear(); + _current_dump_group.Clear(); + _current_dump_group_count = 0; + + return Status::OK(); +} + +void CacheLRUDumper::dump_queue(LRUQueue& queue, const std::string& queue_name) { + Status st; + std::vector> elements; + elements.reserve(config::file_cache_background_lru_dump_tail_record_num); + + { + std::lock_guard lru_log_lock(_mgr->_mutex_lru_log); + size_t count = 0; + for (const auto& [hash, offset, size] : queue) { + if (count++ >= config::file_cache_background_lru_dump_tail_record_num) break; + elements.emplace_back(hash, offset, size); + } + } + + // Write to disk + int64_t duration_ns = 0; + std::uintmax_t file_size = 0; + { + SCOPED_RAW_TIMER(&duration_ns); + std::string tmp_filename = + fmt::format("{}/lru_dump_{}.bin.tmp", _mgr->_cache_base_path, queue_name); + std::string final_filename = + fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + std::ofstream out(tmp_filename, std::ios::binary); + if (out) { + LOG(INFO) << "begin dump " << queue_name << "with " << elements.size() << " elements"; + for (const auto& [hash, offset, size] : elements) { + RETURN_IF_STATUS_ERROR(st, + dump_one_lru_entry(out, tmp_filename, hash, offset, size)); + } + RETURN_IF_STATUS_ERROR(st, finalize_dump(out, elements.size(), tmp_filename, + final_filename, file_size)); + } else { + LOG(WARNING) << "open lru dump file failed"; + } + } + *(_mgr->_lru_dump_latency_us) << (duration_ns / 1000); + LOG(INFO) << fmt::format("lru dump for {} size={} time={}us", queue_name, file_size, + duration_ns / 1000); +}; + +Status CacheLRUDumper::parse_dump_footer(std::ifstream& in, std::string& filename, + size_t& entry_num) { + size_t file_size = std::filesystem::file_size(filename); + + // Read footer + Footer footer; + size_t footer_size = sizeof(footer); + if (file_size < footer_size) { + std::string warn_msg = std::string(fmt::format( + "LRU dump file too small to contain footer, file={}, skip restore", filename)); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + + in.seekg(-footer_size, std::ios::end); + in.read(reinterpret_cast(&footer), footer_size); + RETURN_IF_ERROR(check_ifstream_status(in, filename)); + + // Convert from little-endian to host byte order + footer.meta_offset = le64toh(footer.meta_offset); + + // Validate footer + if (footer.version != 1 || std::string(footer.magic, 3) != "DOR") { + std::string warn_msg = std::string(fmt::format( + "LRU dump file invalid footer format, file={}, skip restore", filename)); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + + // Read meta + in.seekg(footer.meta_offset, std::ios::beg); + size_t meta_size = file_size - footer.meta_offset - footer_size; + if (meta_size <= 0) { + std::string warn_msg = std::string( + fmt::format("LRU dump file invalid meta size, file={}, skip restore", filename)); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + std::string meta_serialized(meta_size, '\0'); + in.read(&meta_serialized[0], meta_serialized.size()); + RETURN_IF_ERROR(check_ifstream_status(in, filename)); + _parse_meta.Clear(); + _current_parse_group.Clear(); + if (!_parse_meta.ParseFromString(meta_serialized)) { + std::string warn_msg = std::string( + fmt::format("LRU dump file meta parse failed, file={}, skip restore", filename)); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + LOG(INFO) << "parse meta: " << _parse_meta.DebugString(); + + entry_num = _parse_meta.entry_num(); + return Status::OK(); +} + +Status CacheLRUDumper::parse_one_lru_entry(std::ifstream& in, std::string& filename, + UInt128Wrapper& hash, size_t& offset, size_t& size) { + // Read next group if current is empty + if (_current_parse_group.entries_size() == 0) { + if (_parse_meta.group_offset_size_size() == 0) { + return Status::EndOfFile("No more entries"); + } + + auto group_info = _parse_meta.group_offset_size(0); + in.seekg(group_info.offset(), std::ios::beg); + std::string group_serialized(group_info.size(), '\0'); + in.read(&group_serialized[0], group_serialized.size()); + RETURN_IF_ERROR(check_ifstream_status(in, filename)); + + LOG(INFO) << "Deserializing group of size: " << group_serialized.size(); + if (!_current_parse_group.ParseFromString(group_serialized)) { + std::string warn_msg = + fmt::format("restore lru failed to parse group, file={}", filename); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + + // Remove processed group info + _parse_meta.mutable_group_offset_size()->erase(_parse_meta.group_offset_size().begin()); + } + + // Get next entry from current group + LOG(INFO) << "After deserialization: " << _current_parse_group.DebugString(); + auto entry = _current_parse_group.entries(0); + hash = UInt128Wrapper((static_cast(entry.hash().high()) << 64) | entry.hash().low()); + offset = entry.offset(); + size = entry.size(); + + // Remove processed entry + _current_parse_group.mutable_entries()->erase(_current_parse_group.entries().begin()); + return Status::OK(); +} + +void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_name, + std::lock_guard& cache_lock) { + Status st; + std::string filename = fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + std::ifstream in(filename, std::ios::binary); + int64_t duration_ns = 0; + if (in) { + LOG(INFO) << "lru dump file is founded for " << queue_name << ". starting lru restore."; + + SCOPED_RAW_TIMER(&duration_ns); + size_t entry_num = 0; + RETURN_IF_STATUS_ERROR(st, parse_dump_footer(in, filename, entry_num)); + in.seekg(0, std::ios::beg); + UInt128Wrapper hash; + size_t offset, size; + for (int i = 0; i < entry_num; ++i) { + RETURN_IF_STATUS_ERROR(st, parse_one_lru_entry(in, filename, hash, offset, size)); + CacheContext ctx; + if (queue_name == "ttl") { + ctx.cache_type = FileCacheType::TTL; + // TODO(zhengyu): we haven't persist expiration time yet, use 3h default + // TODO(zhengyu): we don't use stats yet, see if this will cause any problem + ctx.expiration_time = 10800; + } else if (queue_name == "index") { + ctx.cache_type = FileCacheType::INDEX; + } else if (queue_name == "normal") { + ctx.cache_type = FileCacheType::NORMAL; + } else if (queue_name == "disposable") { + ctx.cache_type = FileCacheType::DISPOSABLE; + } else { + LOG_WARNING("unknown queue type"); + DCHECK(false); + return; + } + _mgr->add_cell(hash, ctx, offset, size, FileBlock::State::DOWNLOADED, cache_lock); + } + in.close(); + } else { + LOG(INFO) << "no lru dump file is founded for " << queue_name; + } + LOG(INFO) << "lru restore time costs: " << (duration_ns / 1000 / 1000) << "ms."; +}; + +void CacheLRUDumper::remove_lru_dump_files() { + std::vector queue_names = {"disposable", "index", "normal", "ttl"}; + for (const auto& queue_name : queue_names) { + std::string filename = + fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + if (std::filesystem::exists(filename)) { + std::filesystem::remove(filename); + } + } +} + +} // end of namespace doris::io \ No newline at end of file diff --git a/be/src/io/cache/cache_lru_dumper.h b/be/src/io/cache/cache_lru_dumper.h new file mode 100644 index 00000000000000..9bd58759c2c117 --- /dev/null +++ b/be/src/io/cache/cache_lru_dumper.h @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "gen_cpp/file_cache.pb.h" +#include "io/cache/file_cache_common.h" + +namespace doris::io { +class LRUQueue; + +class CacheLRUDumper { +public: + CacheLRUDumper(BlockFileCache* mgr) : _mgr(mgr) {}; + void dump_queue(LRUQueue& queue, const std::string& queue_name); + void restore_queue(LRUQueue& queue, const std::string& queue_name, + std::lock_guard& cache_lock); + void remove_lru_dump_files(); + +private: + Status check_ofstream_status(std::ofstream& out, std::string& filename); + Status check_ifstream_status(std::ifstream& in, std::string& filename); + Status dump_one_lru_entry(std::ofstream& out, std::string& filename, const UInt128Wrapper& hash, + size_t offset, size_t size); + Status finalize_dump(std::ofstream& out, size_t entry_num, std::string& tmp_filename, + std::string& final_filename, size_t& file_size); + Status parse_dump_footer(std::ifstream& in, std::string& filename, size_t& entry_num); + Status parse_one_lru_entry(std::ifstream& in, std::string& filename, UInt128Wrapper& hash, + size_t& offset, size_t& size); + Status flush_current_group(std::ofstream& out, std::string& filename); + + struct Footer { + size_t meta_offset; + uint32_t checksum; + uint8_t version; + char magic[3]; + }; + +private: + // For dumping + doris::io::cache::LRUDumpEntryGroupPb _current_dump_group; + doris::io::cache::LRUDumpMetaPb _dump_meta; + size_t _current_dump_group_count = 0; + + // For parsing + doris::io::cache::LRUDumpEntryGroupPb _current_parse_group; + doris::io::cache::LRUDumpMetaPb _parse_meta; + +private: + BlockFileCache* _mgr; +}; +} // namespace doris::io \ No newline at end of file diff --git a/be/src/io/cache/file_cache_common.h b/be/src/io/cache/file_cache_common.h index 6e9396fb11acf8..3cb10dc67a943a 100644 --- a/be/src/io/cache/file_cache_common.h +++ b/be/src/io/cache/file_cache_common.h @@ -48,6 +48,9 @@ struct UInt128Wrapper { explicit UInt128Wrapper(const uint128_t& value) : value_(value) {} bool operator==(const UInt128Wrapper& other) const { return value_ == other.value_; } + + uint64_t high() const { return static_cast(value_ >> 64); } + uint64_t low() const { return static_cast(value_); } }; struct ReadStatistics { diff --git a/be/src/io/cache/fs_file_cache_storage.cpp b/be/src/io/cache/fs_file_cache_storage.cpp index ecb594e14a207c..7b0eef1e35e111 100644 --- a/be/src/io/cache/fs_file_cache_storage.cpp +++ b/be/src/io/cache/fs_file_cache_storage.cpp @@ -662,6 +662,7 @@ void FSFileCacheStorage::load_cache_info_into_memory(BlockFileCache* _mgr) const auto f = [&](const BatchLoadArgs& args) { // in async load mode, a cell may be added twice. if (_mgr->_files.contains(args.hash) && _mgr->_files[args.hash].contains(args.offset)) { + // TODO(zhengyu): update type&expiration if need return; } // if the file is tmp, it means it is the old file and it should be removed diff --git a/be/test/io/cache/block_file_cache_test.cpp b/be/test/io/cache/block_file_cache_test.cpp index fcec0222e5e14c..b39457d4115b2b 100644 --- a/be/test/io/cache/block_file_cache_test.cpp +++ b/be/test/io/cache/block_file_cache_test.cpp @@ -18,71 +18,13 @@ // https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/tests/gtest_lru_file_cache.cpp // and modified by Doris -#include -#include -#include -#include - -#include "runtime/thread_context.h" -#if defined(__APPLE__) -#include -#else -#include -#endif - -// IWYU pragma: no_include -#include - -#include // IWYU pragma: keep -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "common/config.h" -#include "cpp/sync_point.h" -#include "gtest/gtest_pred_impl.h" -#include "io/cache/block_file_cache.h" -#include "io/cache/block_file_cache_factory.h" -#include "io/cache/block_file_cache_profile.h" -#include "io/cache/cached_remote_file_reader.h" -#include "io/cache/file_block.h" -#include "io/cache/file_cache_common.h" -#include "io/cache/fs_file_cache_storage.h" -#include "io/fs/path.h" -#include "olap/options.h" -#include "runtime/exec_env.h" -#include "util/slice.h" -#include "util/time.h" - +#include "block_file_cache_test_common.h" namespace doris::io { -extern int disk_used_percentage(const std::string& path, std::pair* percent); - -namespace fs = std::filesystem; - fs::path caches_dir = fs::current_path() / "lru_cache_test"; std::string cache_base_path = caches_dir / "cache1" / ""; std::string tmp_file = caches_dir / "tmp_file"; -constexpr unsigned long long operator"" _mb(unsigned long long m) { - return m * 1024 * 1024; -} - -constexpr unsigned long long operator"" _kb(unsigned long long m) { - return m * 1024; -} - void assert_range([[maybe_unused]] size_t assert_n, io::FileBlockSPtr file_block, const io::FileBlock::Range& expected_range, io::FileBlock::State expected_state) { auto range = file_block->range(); @@ -96,7 +38,7 @@ std::vector fromHolder(const io::FileBlocksHolder& holder) { return std::vector(holder.file_blocks.begin(), holder.file_blocks.end()); } -void download(io::FileBlockSPtr file_block, size_t size = 0) { +void download(io::FileBlockSPtr file_block, size_t size) { const auto& hash = file_block->get_hash_value(); if (size == 0) { size = file_block->range().size(); @@ -115,7 +57,7 @@ void download(io::FileBlockSPtr file_block, size_t size = 0) { ASSERT_TRUE(fs::exists(subdir)); } -void download_into_memory(io::FileBlockSPtr file_block, size_t size = 0) { +void download_into_memory(io::FileBlockSPtr file_block, size_t size) { if (size == 0) { size = file_block->range().size(); } @@ -140,101 +82,6 @@ void complete_into_memory(const io::FileBlocksHolder& holder) { } } -class BlockFileCacheTest : public testing::Test { -public: - static void SetUpTestSuite() { - config::file_cache_enter_disk_resource_limit_mode_percent = 99; - config::enable_evict_file_cache_in_advance = false; // disable evict in - // advance for most - // cases for simple - // verification - bool exists {false}; - ASSERT_TRUE(global_local_filesystem()->exists(caches_dir, &exists).ok()); - if (!exists) { - ASSERT_TRUE(global_local_filesystem()->create_directory(caches_dir).ok()); - } - ASSERT_TRUE(global_local_filesystem()->exists(tmp_file, &exists).ok()); - if (!exists) { - FileWriterPtr writer; - ASSERT_TRUE(global_local_filesystem()->create_file(tmp_file, &writer).ok()); - for (int i = 0; i < 10; i++) { - std::string data(1_mb, '0' + i); - ASSERT_TRUE(writer->append(Slice(data.data(), data.size())).ok()); - } - std::string data(1, '0'); - ASSERT_TRUE(writer->append(Slice(data.data(), data.size())).ok()); - ASSERT_TRUE(writer->close().ok()); - } - ExecEnv::GetInstance()->_file_cache_factory = factory.get(); - ExecEnv::GetInstance()->_file_cache_open_fd_cache = std::make_unique(); - } - static void TearDownTestSuite() { - config::file_cache_enter_disk_resource_limit_mode_percent = 99; - ExecEnv::GetInstance()->_file_cache_open_fd_cache.reset(nullptr); - } - -private: - inline static std::unique_ptr factory = std::make_unique(); -}; - -TEST_F(BlockFileCacheTest, init) { - std::string string = std::string(R"( - [ - { - "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", - "total_size" : 193273528320, - "query_limit" : 38654705664 - }, - { - "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", - "total_size" : 193273528320, - "query_limit" : 38654705664 - } - ] - )"); - config::enable_file_cache_query_limit = true; - std::vector cache_paths; - EXPECT_TRUE(parse_conf_cache_paths(string, cache_paths)); - EXPECT_EQ(cache_paths.size(), 2); - for (const auto& cache_path : cache_paths) { - io::FileCacheSettings settings = cache_path.init_settings(); - EXPECT_EQ(settings.capacity, 193273528320); - EXPECT_EQ(settings.max_query_cache_size, 38654705664); - } - - // err normal - std::string err_string = std::string(R"( - [ - { - "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", - "total_size" : "193273528320", - "query_limit" : -1 - } - ] - )"); - cache_paths.clear(); - EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); - - // err query_limit - err_string = std::string(R"( - [ - { - "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", - "total_size" : -1 - } - ] - )"); - cache_paths.clear(); - EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); - - err_string = std::string(R"( - [ - ] - )"); - cache_paths.clear(); - EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); -} - void test_file_cache(io::FileCacheType cache_type) { TUniqueId query_id; query_id.hi = 1; @@ -1066,6 +913,64 @@ void test_file_cache_memory_storage(io::FileCacheType cache_type) { } } +TEST_F(BlockFileCacheTest, init) { + std::string string = std::string(R"( + [ + { + "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", + "total_size" : 193273528320, + "query_limit" : 38654705664 + }, + { + "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", + "total_size" : 193273528320, + "query_limit" : 38654705664 + } + ] + )"); + config::enable_file_cache_query_limit = true; + std::vector cache_paths; + EXPECT_TRUE(parse_conf_cache_paths(string, cache_paths)); + EXPECT_EQ(cache_paths.size(), 2); + for (const auto& cache_path : cache_paths) { + io::FileCacheSettings settings = cache_path.init_settings(); + EXPECT_EQ(settings.capacity, 193273528320); + EXPECT_EQ(settings.max_query_cache_size, 38654705664); + } + + // err normal + std::string err_string = std::string(R"( + [ + { + "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", + "total_size" : "193273528320", + "query_limit" : -1 + } + ] + )"); + cache_paths.clear(); + EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); + + // err query_limit + err_string = std::string(R"( + [ + { + "path" : "/mnt/ssd01/clickbench/hot/be/file_cache", + "total_size" : -1 + } + ] + )"); + cache_paths.clear(); + EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); + + err_string = std::string(R"( + [ + ] + )"); + cache_paths.clear(); + EXPECT_FALSE(parse_conf_cache_paths(err_string, cache_paths)); +} + TEST_F(BlockFileCacheTest, normal) { if (fs::exists(cache_base_path)) { fs::remove_all(cache_base_path); diff --git a/be/test/io/cache/block_file_cache_test_common.h b/be/test/io/cache/block_file_cache_test_common.h new file mode 100644 index 00000000000000..c0a3c05e6f4126 --- /dev/null +++ b/be/test/io/cache/block_file_cache_test_common.h @@ -0,0 +1,129 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/tests/gtest_lru_file_cache.cpp +// and modified by Doris + +#pragma once + +#include +#include +#include +#include +#if defined(__APPLE__) +#include +#else +#include +#endif + +// IWYU pragma: no_include +#include // IWYU pragma: keep +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "cpp/sync_point.h" +#include "gtest/gtest_pred_impl.h" +#include "io/cache/block_file_cache.h" +#include "io/cache/block_file_cache_factory.h" +#include "io/cache/block_file_cache_profile.h" +#include "io/cache/cached_remote_file_reader.h" +#include "io/cache/file_block.h" +#include "io/cache/file_cache_common.h" +#include "io/cache/fs_file_cache_storage.h" +#include "io/fs/path.h" +#include "olap/options.h" +#include "runtime/exec_env.h" +#include "util/slice.h" +#include "util/time.h" + +namespace doris::io { +namespace fs = std::filesystem; + +extern int disk_used_percentage(const std::string& path, std::pair* percent); +extern fs::path caches_dir; +extern std::string cache_base_path; +extern std::string tmp_file; + +constexpr unsigned long long operator"" _mb(unsigned long long m) { + return m * 1024 * 1024; +} + +constexpr unsigned long long operator"" _kb(unsigned long long m) { + return m * 1024; +} + +extern void assert_range([[maybe_unused]] size_t assert_n, io::FileBlockSPtr file_block, + const io::FileBlock::Range& expected_range, + io::FileBlock::State expected_state); +extern std::vector fromHolder(const io::FileBlocksHolder& holder); +extern void download(io::FileBlockSPtr file_block, size_t size = 0); +extern void download_into_memory(io::FileBlockSPtr file_block, size_t size = 0); +extern void complete(const io::FileBlocksHolder& holder); +extern void complete_into_memory(const io::FileBlocksHolder& holder); +extern void test_file_cache(io::FileCacheType cache_type); +extern void test_file_cache_memory_storage(io::FileCacheType cache_type); + +class BlockFileCacheTest : public testing::Test { +public: + static void SetUpTestSuite() { + config::file_cache_enter_disk_resource_limit_mode_percent = 99; + config::enable_evict_file_cache_in_advance = false; // disable evict in + // advance for most + // cases for simple + // verification + bool exists {false}; + ASSERT_TRUE(global_local_filesystem()->exists(caches_dir, &exists).ok()); + if (!exists) { + ASSERT_TRUE(global_local_filesystem()->create_directory(caches_dir).ok()); + } + ASSERT_TRUE(global_local_filesystem()->exists(tmp_file, &exists).ok()); + if (!exists) { + FileWriterPtr writer; + ASSERT_TRUE(global_local_filesystem()->create_file(tmp_file, &writer).ok()); + for (int i = 0; i < 10; i++) { + std::string data(1_mb, '0' + i); + ASSERT_TRUE(writer->append(Slice(data.data(), data.size())).ok()); + } + std::string data(1, '0'); + ASSERT_TRUE(writer->append(Slice(data.data(), data.size())).ok()); + ASSERT_TRUE(writer->close().ok()); + } + ExecEnv::GetInstance()->_file_cache_factory = factory.get(); + ExecEnv::GetInstance()->_file_cache_open_fd_cache = std::make_unique(); + } + static void TearDownTestSuite() { + config::file_cache_enter_disk_resource_limit_mode_percent = 99; + ExecEnv::GetInstance()->_file_cache_open_fd_cache.reset(nullptr); + } + +private: + inline static std::unique_ptr factory = std::make_unique(); +}; + +} // end of namespace doris::io \ No newline at end of file diff --git a/be/test/io/cache/block_file_cache_test_lru_dump.cpp b/be/test/io/cache/block_file_cache_test_lru_dump.cpp new file mode 100644 index 00000000000000..71d5ea57725f79 --- /dev/null +++ b/be/test/io/cache/block_file_cache_test_lru_dump.cpp @@ -0,0 +1,402 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/tests/gtest_lru_file_cache.cpp +// and modified by Doris + +#include "block_file_cache_test_common.h" + +namespace doris::io { + +TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { + config::enable_evict_file_cache_in_advance = false; + config::file_cache_enter_disk_resource_limit_mode_percent = 99; + config::file_cache_background_lru_dump_interval_ms = 3000; + if (fs::exists(cache_base_path)) { + fs::remove_all(cache_base_path); + } + fs::create_directories(cache_base_path); + TUniqueId query_id; + query_id.hi = 1; + query_id.lo = 1; + io::FileCacheSettings settings; + + settings.ttl_queue_size = 5000000; + settings.ttl_queue_elements = 50000; + settings.query_queue_size = 5000000; + settings.query_queue_elements = 50000; + settings.index_queue_size = 5000000; + settings.index_queue_elements = 50000; + settings.disposable_queue_size = 5000000; + settings.disposable_queue_elements = 50000; + settings.capacity = 20000000; + settings.max_file_block_size = 100000; + settings.max_query_cache_size = 30; + + io::BlockFileCache cache(cache_base_path, settings); + ASSERT_TRUE(cache.initialize()); + int i = 0; + for (; i < 100; i++) { + if (cache.get_async_open_success()) { + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + } + ASSERT_TRUE(cache.get_async_open_success()); + + io::CacheContext context1; + ReadStatistics rstats; + context1.stats = &rstats; + context1.cache_type = io::FileCacheType::NORMAL; + context1.query_id = query_id; + auto key1 = io::BlockFileCache::hash("key1"); + + int64_t offset = 0; + + for (; offset < 500000; offset += 100000) { + auto holder = cache.get_or_set(key1, offset, 100000, context1); + auto blocks = fromHolder(holder); + ASSERT_EQ(blocks.size(), 1); + + assert_range(1, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::EMPTY); + ASSERT_TRUE(blocks[0]->get_or_set_downloader() == io::FileBlock::get_caller_id()); + download(blocks[0]); + assert_range(2, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::DOWNLOADED); + + blocks.clear(); + } + io::CacheContext context2; + context2.stats = &rstats; + context2.cache_type = io::FileCacheType::INDEX; + context2.query_id = query_id; + auto key2 = io::BlockFileCache::hash("key2"); + + offset = 0; + + for (; offset < 500000; offset += 100000) { + auto holder = cache.get_or_set(key2, offset, 100000, context2); + auto blocks = fromHolder(holder); + ASSERT_EQ(blocks.size(), 1); + + assert_range(1, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::EMPTY); + ASSERT_TRUE(blocks[0]->get_or_set_downloader() == io::FileBlock::get_caller_id()); + download(blocks[0]); + assert_range(2, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::DOWNLOADED); + + blocks.clear(); + } + io::CacheContext context3; + context3.stats = &rstats; + context3.cache_type = io::FileCacheType::TTL; + context3.query_id = query_id; + context3.expiration_time = UnixSeconds() + 120; + auto key3 = io::BlockFileCache::hash("key3"); + + offset = 0; + + for (; offset < 500000; offset += 100000) { + auto holder = cache.get_or_set(key3, offset, 100000, context3); + auto blocks = fromHolder(holder); + ASSERT_EQ(blocks.size(), 1); + + assert_range(1, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::EMPTY); + ASSERT_TRUE(blocks[0]->get_or_set_downloader() == io::FileBlock::get_caller_id()); + download(blocks[0]); + assert_range(2, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::DOWNLOADED); + + blocks.clear(); + } + + io::CacheContext context4; + context4.stats = &rstats; + context4.cache_type = io::FileCacheType::DISPOSABLE; + context4.query_id = query_id; + auto key4 = io::BlockFileCache::hash("key4"); + + offset = 0; + + for (; offset < 500000; offset += 100000) { + auto holder = cache.get_or_set(key4, offset, 100000, context4); + auto blocks = fromHolder(holder); + ASSERT_EQ(blocks.size(), 1); + + assert_range(1, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::EMPTY); + ASSERT_TRUE(blocks[0]->get_or_set_downloader() == io::FileBlock::get_caller_id()); + download(blocks[0]); + assert_range(2, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::DOWNLOADED); + + blocks.clear(); + } + ASSERT_EQ(cache.get_stats_unsafe()["disposable_queue_curr_size"], 500000); + ASSERT_EQ(cache.get_stats_unsafe()["ttl_queue_curr_size"], 500000); + ASSERT_EQ(cache.get_stats_unsafe()["index_queue_curr_size"], 500000); + ASSERT_EQ(cache.get_stats_unsafe()["normal_queue_curr_size"], 500000); + + // all queue are filled, let's check the lru log records + ASSERT_EQ(cache._ttl_lru_log_queue.size(), 5); + ASSERT_EQ(cache._index_lru_log_queue.size(), 5); + ASSERT_EQ(cache._normal_lru_log_queue.size(), 5); + ASSERT_EQ(cache._disposable_lru_log_queue.size(), 5); + + // then check the log replay + std::this_thread::sleep_for(std::chrono::milliseconds( + 2 * config::file_cache_background_lru_log_replay_interval_ms)); + ASSERT_EQ(cache._shadow_ttl_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._shadow_index_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._shadow_normal_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._shadow_disposable_queue.get_elements_num_unsafe(), 5); + + // ok, let do some MOVETOBACK & REMOVE + { + auto holder = cache.get_or_set(key2, 200000, 100000, + context2); // move index queue 3rd element to the end + cache.remove_if_cached(key3); // remove all element from ttl queue + } + ASSERT_EQ(cache._ttl_lru_log_queue.size(), 5); + ASSERT_EQ(cache._index_lru_log_queue.size(), 1); + ASSERT_EQ(cache._normal_lru_log_queue.size(), 0); + ASSERT_EQ(cache._disposable_lru_log_queue.size(), 0); + + std::this_thread::sleep_for(std::chrono::milliseconds( + 2 * config::file_cache_background_lru_log_replay_interval_ms)); + ASSERT_EQ(cache._shadow_ttl_queue.get_elements_num_unsafe(), 0); + ASSERT_EQ(cache._shadow_index_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._shadow_normal_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._shadow_disposable_queue.get_elements_num_unsafe(), 5); + + // check the order + std::vector offsets; + for (auto it = cache._shadow_index_queue.begin(); it != cache._shadow_index_queue.end(); ++it) { + offsets.push_back(it->offset); + } + ASSERT_EQ(offsets.size(), 5); + ASSERT_EQ(offsets[0], 0); + ASSERT_EQ(offsets[1], 100000); + ASSERT_EQ(offsets[2], 300000); + ASSERT_EQ(offsets[3], 400000); + ASSERT_EQ(offsets[4], 200000); + + std::this_thread::sleep_for( + std::chrono::milliseconds(2 * config::file_cache_background_lru_dump_interval_ms)); + +#if 0 + // Verify all 4 dump files + // TODO(zhengyu): abstract those read/write into a function + { + std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "ttl"); + + struct stat file_stat; + EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; + + EXPECT_EQ(file_stat.st_size, 12) << "File " << filename << " has more data than footer"; + std::ifstream in(filename, std::ios::binary); + ASSERT_TRUE(in) << "Failed to open " << filename; + size_t entry_num = 0; + int8_t version = 0; + char magic_str[3]; + char target_str[3] = {'D', 'O', 'R'}; + in.read(reinterpret_cast(&entry_num), sizeof(entry_num)); + in.read(reinterpret_cast(&version), sizeof(version)); + in.read(magic_str, sizeof(magic_str)); + EXPECT_EQ(entry_num, 0); + EXPECT_EQ(version, 1); + EXPECT_TRUE(memcmp(magic_str, target_str, sizeof(magic_str)) == 0); + } + + { + std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "normal"); + + struct stat file_stat; + EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; + + EXPECT_GT(file_stat.st_size, 12) << "File " << filename << " is empty"; + + std::ifstream in(filename, std::ios::binary); + ASSERT_TRUE(in) << "Failed to open " << filename; + UInt128Wrapper hash; + size_t offset, size; + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key1")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 0) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key1")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 100000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key1")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 200000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key1")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 300000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key1")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 400000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + EXPECT_TRUE(in.fail()) << "still read from " << filename << " which should be EOF"; + } + + { + std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "index"); + + struct stat file_stat; + EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; + + EXPECT_GT(file_stat.st_size, 12) << "File " << filename << " is empty"; + + std::ifstream in(filename, std::ios::binary); + ASSERT_TRUE(in) << "Failed to open " << filename; + UInt128Wrapper hash; + size_t offset, size; + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key2")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 0) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key2")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 100000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key2")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 300000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key2")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 400000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + in.read(reinterpret_cast(&offset), sizeof(offset)); + in.read(reinterpret_cast(&size), sizeof(size)); + + EXPECT_FALSE(in.fail()) << "Failed to read from " << filename; + EXPECT_EQ(hash, io::BlockFileCache::hash("key2")) << "wrong hash value in " << filename; + EXPECT_EQ(offset, 200000) << "wrong offset value in " << filename; + EXPECT_EQ(size, 100000) << "wrong size value in " << filename; + + in.read(reinterpret_cast(&hash), sizeof(hash)); + EXPECT_TRUE(in.fail()) << "still read from " << filename << " which should be EOF"; + } +#endif + + // dump looks good, let's try restore + io::BlockFileCache cache2(cache_base_path, settings); + ASSERT_TRUE(cache2.initialize()); + for (i = 0; i < 100; i++) { + if (cache2.get_async_open_success()) { + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + } + ASSERT_TRUE(cache2.get_async_open_success()); + + // check the size of cache2 + ASSERT_EQ(cache2._ttl_queue.get_elements_num_unsafe(), 0); + ASSERT_EQ(cache2._index_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache2._normal_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache2._disposable_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache2._cur_cache_size, 1500000); + + // then check the order of restored cache2 + std::vector offsets2; + for (auto it = cache2._index_queue.begin(); it != cache2._index_queue.end(); ++it) { + offsets2.push_back(it->offset); + } + ASSERT_EQ(offsets2.size(), 5); + ASSERT_EQ(offsets2[0], 0); + ASSERT_EQ(offsets2[1], 100000); + ASSERT_EQ(offsets2[2], 300000); + ASSERT_EQ(offsets2[3], 400000); + ASSERT_EQ(offsets2[4], 200000); + + io::CacheContext context22; + context22.stats = &rstats; + context22.cache_type = io::FileCacheType::INDEX; + context22.query_id = query_id; + + offset = 0; + + for (; offset < 500000; offset += 100000) { + auto holder = cache2.get_or_set(key2, offset, 100000, context22); + auto blocks = fromHolder(holder); + ASSERT_EQ(blocks.size(), 1); + assert_range(2, blocks[0], io::FileBlock::Range(offset, offset + 99999), + io::FileBlock::State::DOWNLOADED); + blocks.clear(); + } + + if (fs::exists(cache_base_path)) { + fs::remove_all(cache_base_path); + } +} + +} // namespace doris::io diff --git a/be/test/io/cache/cache_lru_dumper_test.cpp b/be/test/io/cache/cache_lru_dumper_test.cpp new file mode 100644 index 00000000000000..070700347f2767 --- /dev/null +++ b/be/test/io/cache/cache_lru_dumper_test.cpp @@ -0,0 +1,146 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "io/cache/cache_lru_dumper.h" + +#include "gmock/gmock.h" +#include "gtest/gtest.h" +#include "io/cache/block_file_cache.h" +#include "io/cache/file_block.h" +#include "io/cache/file_cache_common.h" + +using ::testing::_; +using ::testing::Return; +using ::testing::NiceMock; + +namespace doris::io { +std::mutex _mutex; + +class MockBlockFileCache : public BlockFileCache { +public: + LRUQueue* dst_queue; // Pointer to the destination queue + + MockBlockFileCache(LRUQueue* queue) : BlockFileCache("", {}), dst_queue(queue) { + _cache_base_path = "./"; + } + + FileBlockCell* add_cell(const UInt128Wrapper& hash, const CacheContext& ctx, size_t offset, + size_t size, FileBlock::State state, + std::lock_guard& lock) { + dst_queue->add(hash, offset, size, lock); + return nullptr; + } + + std::mutex& mutex() { return _mutex; } + +private: + std::mutex _mutex; + struct { + std::string _cache_base_path; + } _mgr; +}; + +class CacheLRUDumperTest : public ::testing::Test { +protected: + LRUQueue dst_queue; // Member variable for destination queue + + void SetUp() override { + mock_cache = std::make_unique>(&dst_queue); + dumper = std::make_unique(mock_cache.get()); + } + + void TearDown() override { + dumper.reset(); + mock_cache.reset(); + } + + std::unique_ptr> mock_cache; + std::unique_ptr dumper; +}; + +TEST_F(CacheLRUDumperTest, test_finalize_dump_and_parse_dump_footer) { + std::ofstream out("test_finalize.bin", std::ios::binary); + std::string tmp_filename = "test_finalize.bin.tmp"; + std::string final_filename = "test_finalize.bin"; + size_t file_size = 0; + size_t entry_num = 10; + + // Test finalize dump + EXPECT_TRUE( + dumper->finalize_dump(out, entry_num, tmp_filename, final_filename, file_size).ok()); + + // Test parse footer + std::ifstream in("test_finalize.bin", std::ios::binary); + size_t parsed_entry_num = 0; + EXPECT_TRUE(dumper->parse_dump_footer(in, final_filename, parsed_entry_num).ok()); + EXPECT_EQ(entry_num, parsed_entry_num); + + out.close(); + in.close(); + std::remove("test_finalize.bin"); +} + +TEST_F(CacheLRUDumperTest, test_remove_lru_dump_files) { + // Create test files + std::vector queue_names = {"disposable", "index", "normal", "ttl"}; + for (const auto& name : queue_names) { + std::ofstream(fmt::format("lru_dump_{}.bin", name)); + } + + // Test remove + dumper->remove_lru_dump_files(); + + // Verify files are removed + for (const auto& name : queue_names) { + EXPECT_FALSE(std::filesystem::exists(fmt::format("lru_dump_{}.bin", name))); + } +} + +TEST_F(CacheLRUDumperTest, test_dump_and_restore_queue) { + LRUQueue src_queue; + std::string queue_name = "normal"; + + // Add test data + UInt128Wrapper hash(123456789ULL); + size_t offset = 1024; + size_t size = 4096; + std::lock_guard lock(_mutex); + src_queue.add(hash, offset, size, lock); + + // Test dump + dumper->dump_queue(src_queue, queue_name); + + // Test restore + std::lock_guard cache_lock(mock_cache->mutex()); + dumper->restore_queue(dst_queue, queue_name, cache_lock); + + // Verify queue content and order + auto src_it = src_queue.begin(); + auto dst_it = dst_queue.begin(); + while (src_it != src_queue.end() && dst_it != dst_queue.end()) { + EXPECT_EQ(src_it->hash, dst_it->hash); + EXPECT_EQ(src_it->offset, dst_it->offset); + EXPECT_EQ(src_it->size, dst_it->size); + ++src_it; + ++dst_it; + } + + // Clean up + std::remove(fmt::format("lru_dump_{}.bin", queue_name).c_str()); +} + +} // namespace doris::io \ No newline at end of file diff --git a/gensrc/proto/file_cache.proto b/gensrc/proto/file_cache.proto new file mode 100644 index 00000000000000..5230abe54053f2 --- /dev/null +++ b/gensrc/proto/file_cache.proto @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// Define file format struct, like data header, index header. + +syntax="proto2"; + +package doris.io.cache; + +message UInt128WrapperPb { + optional uint64 high = 1; + optional uint64 low = 2; +} + +message LRUDumpEntryPb { + optional UInt128WrapperPb hash = 1; + optional uint64 offset = 2; + optional uint64 size = 3; +} + +message LRUDumpEntryGroupPb { + repeated LRUDumpEntryPb entries = 1; +} + +message EntryGroupOffsetSizePb { + optional uint64 offset = 1; + optional uint64 size = 2; +} + +message LRUDumpMetaPb{ + optional uint64 entry_num = 1; + optional string queue_name = 2; + repeated EntryGroupOffsetSizePb group_offset_size = 3; +} + diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy index ec801c47c01d46..f73e59d646b064 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy @@ -167,6 +167,9 @@ class ServerNode { assert false : 'Unknown node type' } + String getBasePath() { + return path + } } class Frontend extends ServerNode { diff --git a/regression-test/suites/demo_p0/test_lru_persist.groovy b/regression-test/suites/demo_p0/test_lru_persist.groovy new file mode 100644 index 00000000000000..4f9db148581b6d --- /dev/null +++ b/regression-test/suites/demo_p0/test_lru_persist.groovy @@ -0,0 +1,110 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.apache.doris.regression.suite.ClusterOptions + +// Run docker suite steps: +// 1. Read 'docker/runtime/doris-compose/Readme.md', make sure you can setup a doris docker cluster; +// 2. update regression-conf-custom.groovy with config: +// image = "xxxx" // your doris docker image +// excludeDockerTest = false // do run docker suite, default is true +// dockerEndDeleteFiles = false // after run docker suite, whether delete contains's log and data in directory '/tmp/doris/' + +// When run docker suite, then no need an external doris cluster. +// But whether run a docker suite, need more check. +// Firstly, get the pipeline's run mode (cloud or not_cloud): +// If there's an external doris cluster, then fetch pipeline's runMode from it. +// If there's no external doris cluster, then set pipeline's runMode with command args. +// for example: sh run-regression-test.sh --run docker_action -runMode=cloud/not_cloud +// Secondly, compare ClusterOptions.cloudMode and pipeline's runMode +// If ClusterOptions.cloudMode = null then let ClusterOptions.cloudMode = pipeline's cloudMode, and run docker suite. +// if ClusterOptions.cloudMode = true or false, if cloudMode == pipeline's cloudMode or pipeline's cloudMode is unknown, +// then run docker suite, otherwise don't run docker suite. + +// NOTICE: +// 1. Need add 'docker' to suite's group, and don't add 'nonConcurrent' to it; +// 2. In docker closure: +// a. Don't use 'Awaitility.await()...until(f)', but use 'dockerAwaitUntil(..., f)'; +// 3. No need to use code ` if (isCloudMode()) { return } ` in docker suites, +// instead should use `ClusterOptions.cloudMode = true/false` is enough. +// Because when run docker suite without an external doris cluster, if suite use code `isCloudMode()`, it need specific -runMode=cloud/not_cloud. +// On the contrary, `ClusterOptions.cloudMode = true/false` no need specific -runMode=cloud/not_cloud when no external doris cluster exists. + +suite('test_lru_persist', 'docker') { + def options = new ClusterOptions() + + options.feNum = 1 + options.beNum = 1 + options.msNum = 1 + options.cloudMode = true + options.feConfigs += ['example_conf_k1=v1', 'example_conf_k2=v2'] + options.beConfigs += ['enable_file_cache=true', 'enable_java_support=false', 'file_cache_enter_disk_resource_limit_mode_percent=99', + 'file_cache_background_lru_dump_interval_ms=2000', 'file_cache_background_lru_log_replay_interval_ms=500', + 'disable_auto_compation=true', 'file_cache_enter_need_evict_cache_in_advance_percent=99' + ] + + // run another docker + docker(options) { + cluster.checkFeIsAlive(1, true) + cluster.checkBeIsAlive(1, true) + sql '''set global enable_auto_analyze=false''' + + sql '''create table tb1 (k int) DISTRIBUTED BY HASH(k) BUCKETS 10 properties ("replication_num"="1")''' + sql '''insert into tb1 values (1),(2),(3)''' + sql '''insert into tb1 values (4),(5),(6)''' + sql '''insert into tb1 values (7),(8),(9)''' + sql '''insert into tb1 values (10),(11),(12)''' + + def be = cluster.getBeByIndex(1) + def beBasePath = be.getBasePath() + def cachePath = beBasePath + "/storage/file_cache/" + + sleep(5000); + cluster.stopBackends(1) + + // check md5sum of each index + + def disposableBefore = "md5sum ${cachePath}/lru_dump_disposable.bin".execute().text.trim().split()[0] + logger.info("disposableBefore: ${disposableBefore}") + def indexBefore = "md5sum ${cachePath}/lru_dump_index.bin".execute().text.trim().split()[0] + logger.info("indexBefore: ${indexBefore}") + def normalBefore = "md5sum ${cachePath}/lru_dump_normal.bin".execute().text.trim().split()[0] + logger.info("normalBefore: ${normalBefore}") + def ttlBefore = "md5sum ${cachePath}/lru_dump_ttl.bin".execute().text.trim().split()[0] + logger.info("ttlBefore: ${ttlBefore}") + + cluster.startBackends(1) + sleep(5000); + + cluster.stopBackends(1) + + // check md5sum again after be restart + def disposableAfter = "md5sum ${cachePath}/lru_dump_disposable.bin".execute().text.trim().split()[0] + logger.info("disposableAfter: ${disposableAfter}") + def indexAfter = "md5sum ${cachePath}/lru_dump_index.bin".execute().text.trim().split()[0] + logger.info("indexAfter: ${indexAfter}") + def normalAfter = "md5sum ${cachePath}/lru_dump_normal.bin".execute().text.trim().split()[0] + logger.info("normalAfter: ${normalAfter}") + def ttlAfter = "md5sum ${cachePath}/lru_dump_ttl.bin".execute().text.trim().split()[0] + logger.info("ttlAfter: ${ttlAfter}") + + assert disposableBefore == disposableAfter + assert indexBefore == indexAfter + assert normalBefore == normalAfter + assert ttlBefore == ttlAfter + } +} From 831dd885be72bf37a96230635487b109e6c44bf0 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 26 Jun 2025 19:04:14 +0800 Subject: [PATCH 02/12] format Signed-off-by: zhengyu --- be/src/io/cache/block_file_cache.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index fe9c256a8df7fd..187a5075426c67 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -1126,17 +1126,17 @@ bool BlockFileCache::remove_if_ttl_file_blocks(const UInt128Wrapper& file_key, b if (cell.queue_iterator) { ttl_queue.remove(cell.queue_iterator.value(), cache_lock); record_queue_event(get_lru_log_queue(FileCacheType::TTL), - CacheLRULogType::REMOVE, - cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } auto& queue = get_queue(FileCacheType::NORMAL); cell.queue_iterator = queue.add( cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); record_queue_event(get_lru_log_queue(FileCacheType::NORMAL), - CacheLRULogType::ADD, cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + CacheLRULogType::ADD, cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change cache type to normal").error(st); } From 590f8c687b53bda704d5b9cab49b0bb3aca27f74 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 26 Jun 2025 19:28:40 +0800 Subject: [PATCH 03/12] fix header after conflict Signed-off-by: zhengyu --- be/test/io/cache/block_file_cache_test_common.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/be/test/io/cache/block_file_cache_test_common.h b/be/test/io/cache/block_file_cache_test_common.h index c0a3c05e6f4126..0bf4acf2466781 100644 --- a/be/test/io/cache/block_file_cache_test_common.h +++ b/be/test/io/cache/block_file_cache_test_common.h @@ -24,6 +24,8 @@ #include #include #include + +#include "runtime/thread_context.h" #if defined(__APPLE__) #include #else @@ -31,6 +33,8 @@ #endif // IWYU pragma: no_include +#include + #include // IWYU pragma: keep #include #include @@ -42,6 +46,7 @@ #include #include #include +#include #include #include #include From 8d858e185462656ce2f4f2abe3164faf221c056a Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 26 Jun 2025 22:22:11 +0800 Subject: [PATCH 04/12] only dump if queue changed freqently Signed-off-by: zhengyu --- be/src/common/config.cpp | 2 ++ be/src/common/config.h | 2 ++ be/src/io/cache/block_file_cache.cpp | 30 ++++++++++++++++--- be/src/io/cache/block_file_cache.h | 2 ++ .../cache/block_file_cache_test_lru_dump.cpp | 1 + .../suites/demo_p0/test_lru_persist.groovy | 3 +- 6 files changed, 35 insertions(+), 5 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 8165ce503a4707..756d06f482d2bb 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1114,6 +1114,8 @@ DEFINE_mInt64(file_cache_background_monitor_interval_ms, "5000"); DEFINE_mInt64(file_cache_background_ttl_gc_interval_ms, "3000"); DEFINE_mInt64(file_cache_background_ttl_gc_batch, "1000"); DEFINE_mInt64(file_cache_background_lru_dump_interval_ms, "60000"); +// dump queue only if the queue update specific times through several dump intervals +DEFINE_mInt64(file_cache_background_lru_dump_update_cnt_threshold, "1000"); DEFINE_mInt64(file_cache_background_lru_dump_tail_record_num, "5000000"); DEFINE_mInt64(file_cache_background_lru_log_replay_interval_ms, "1000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 91bbbbed0cb3a6..7f9fb5b9650d20 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1156,6 +1156,8 @@ DECLARE_Int32(file_cache_downloader_thread_num_min); DECLARE_Int32(file_cache_downloader_thread_num_max); // used to persist lru information before be reboot and load the info back DECLARE_mInt64(file_cache_background_lru_dump_interval_ms); +// dump queue only if the queue update specific times through several dump intervals +DECLARE_mInt64(file_cache_background_lru_dump_update_cnt_threshold); DECLARE_mInt64(file_cache_background_lru_dump_tail_record_num); DECLARE_mInt64(file_cache_background_lru_log_replay_interval_ms); diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index 187a5075426c67..b095d80764e976 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -240,6 +240,11 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, _ttl_queue = LRUQueue(cache_settings.ttl_queue_size, cache_settings.ttl_queue_elements, std::numeric_limits::max()); + _lru_queue_update_counters[FileCacheType::DISPOSABLE] = 0; + _lru_queue_update_counters[FileCacheType::NORMAL] = 0; + _lru_queue_update_counters[FileCacheType::INDEX] = 0; + _lru_queue_update_counters[FileCacheType::TTL] = 0; + _lru_dumper = std::make_unique(this); if (cache_settings.storage == "memory") { _storage = std::make_unique(); @@ -2215,6 +2220,7 @@ void BlockFileCache::update_ttl_atime(const UInt128Wrapper& hash) { } BlockFileCache::CacheLRULogQueue& BlockFileCache::get_lru_log_queue(FileCacheType type) { + ++_lru_queue_update_counters[type]; switch (type) { case FileCacheType::INDEX: return _index_lru_log_queue; @@ -2308,10 +2314,26 @@ void BlockFileCache::run_background_lru_dump() { } if (config::file_cache_background_lru_dump_tail_record_num > 0) { - _lru_dumper->dump_queue(_shadow_disposable_queue, "disposable"); - _lru_dumper->dump_queue(_shadow_index_queue, "index"); - _lru_dumper->dump_queue(_shadow_normal_queue, "normal"); - _lru_dumper->dump_queue(_shadow_ttl_queue, "ttl"); + if (_lru_queue_update_counters[FileCacheType::DISPOSABLE] > + config::file_cache_background_lru_dump_update_cnt_threshold) { + _lru_dumper->dump_queue(_shadow_disposable_queue, "disposable"); + _lru_queue_update_counters[FileCacheType::DISPOSABLE] = 0; + } + if (_lru_queue_update_counters[FileCacheType::NORMAL] > + config::file_cache_background_lru_dump_update_cnt_threshold) { + _lru_dumper->dump_queue(_shadow_normal_queue, "normal"); + _lru_queue_update_counters[FileCacheType::NORMAL] = 0; + } + if (_lru_queue_update_counters[FileCacheType::INDEX] > + config::file_cache_background_lru_dump_update_cnt_threshold) { + _lru_dumper->dump_queue(_shadow_index_queue, "index"); + _lru_queue_update_counters[FileCacheType::INDEX] = 0; + } + if (_lru_queue_update_counters[FileCacheType::TTL] > + config::file_cache_background_lru_dump_update_cnt_threshold) { + _lru_dumper->dump_queue(_shadow_ttl_queue, "ttl"); + _lru_queue_update_counters[FileCacheType::TTL] = 0; + } } } } diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 23d8b76129fb9d..84e54f72036797 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -589,6 +589,8 @@ class BlockFileCache { CacheLRULogQueue _disposable_lru_log_queue; std::mutex _mutex_lru_log; + std::unordered_map _lru_queue_update_counters; + std::unique_ptr _lru_dumper; // metrics diff --git a/be/test/io/cache/block_file_cache_test_lru_dump.cpp b/be/test/io/cache/block_file_cache_test_lru_dump.cpp index 71d5ea57725f79..f9661d1495789e 100644 --- a/be/test/io/cache/block_file_cache_test_lru_dump.cpp +++ b/be/test/io/cache/block_file_cache_test_lru_dump.cpp @@ -26,6 +26,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { config::enable_evict_file_cache_in_advance = false; config::file_cache_enter_disk_resource_limit_mode_percent = 99; config::file_cache_background_lru_dump_interval_ms = 3000; + config::file_cache_background_lru_dump_update_cnt_threshold = 1; if (fs::exists(cache_base_path)) { fs::remove_all(cache_base_path); } diff --git a/regression-test/suites/demo_p0/test_lru_persist.groovy b/regression-test/suites/demo_p0/test_lru_persist.groovy index 4f9db148581b6d..662339a10758fe 100644 --- a/regression-test/suites/demo_p0/test_lru_persist.groovy +++ b/regression-test/suites/demo_p0/test_lru_persist.groovy @@ -54,7 +54,8 @@ suite('test_lru_persist', 'docker') { options.feConfigs += ['example_conf_k1=v1', 'example_conf_k2=v2'] options.beConfigs += ['enable_file_cache=true', 'enable_java_support=false', 'file_cache_enter_disk_resource_limit_mode_percent=99', 'file_cache_background_lru_dump_interval_ms=2000', 'file_cache_background_lru_log_replay_interval_ms=500', - 'disable_auto_compation=true', 'file_cache_enter_need_evict_cache_in_advance_percent=99' + 'disable_auto_compation=true', 'file_cache_enter_need_evict_cache_in_advance_percent=99', + 'file_cache_background_lru_dump_update_cnt_threshold=1' ] // run another docker From f1c6fbefe6462433593eb8d3c9c0bf150d326579 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Mon, 30 Jun 2025 18:05:05 +0800 Subject: [PATCH 05/12] reponse to the reviewer Signed-off-by: zhengyu --- be/src/io/cache/block_file_cache.cpp | 1 - be/src/io/cache/block_file_cache.h | 2 +- be/src/io/cache/cache_lru_dumper.cpp | 36 ++++++++++++++++------------ be/src/io/cache/cache_lru_dumper.h | 1 - 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index b095d80764e976..4764be6ece610b 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -606,7 +606,6 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte } std::string BlockFileCache::clear_file_cache_async() { - // TODO(zhengyu): rm lru dump file before and after clear file cache LOG(INFO) << "start clear_file_cache_async, path=" << _cache_base_path; _lru_dumper->remove_lru_dump_files(); int64_t num_cells_all = 0; diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 84e54f72036797..d1e70df0f74d8b 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -589,7 +589,7 @@ class BlockFileCache { CacheLRULogQueue _disposable_lru_log_queue; std::mutex _mutex_lru_log; - std::unordered_map _lru_queue_update_counters; + std::unordered_map _lru_queue_update_cnt_from_last_dump; std::unique_ptr _lru_dumper; diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index d0d741559f8b4a..8acf795242ed1e 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -81,11 +81,11 @@ Status CacheLRUDumper::dump_one_lru_entry(std::ofstream& out, std::string& filen // +-----------------------------------------------+ // | LRUDumpEntryGroupPb_n | // +-----------------------------------------------+ - // | LRUDumpMetaPb (queueName, GroupList)| + // | LRUDumpMetaPb (List) | // +-----------------------------------------------+ // | FOOTER_OFFSET (8Bytes) | // +-----------------------------------------------+ - // | CHECKSUM (4Bytes)|VERSION (1Byte)| MAGIC | + // | CHECKSUM (4Bytes)|VERSION (1Byte)|MAGIC (3B)| // +-----------------------------------------------+ // // why we are not using protobuf as a whole? @@ -119,11 +119,14 @@ Status CacheLRUDumper::flush_current_group(std::ofstream& out, std::string& file // Serialize and write the group std::string serialized; - LOG(INFO) << "Before serialization: " << _current_dump_group.DebugString(); + VLOG_DEBUG << "Serialized size: " << serialized.size() + << " Before serialization: " << _current_dump_group.DebugString(); if (!_current_dump_group.SerializeToString(&serialized)) { - return Status::InternalError("Failed to serialize LRUDumpEntryGroupPb"); + std::string warn_msg = fmt::format("Failed to serialize LRUDumpEntryGroupPb"); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); } - LOG(INFO) << "Serialized size: " << serialized.size(); + out.write(serialized.data(), serialized.size()); RETURN_IF_ERROR(check_ofstream_status(out, filename)); @@ -155,7 +158,7 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, std::string warn_msg = fmt::format("Failed to serialize LRUDumpMetaPb, file={}", tmp_filename); LOG(WARNING) << warn_msg; - return Status::InternalError("warn_msg"); + return Status::InternalError(warn_msg); } out.write(meta_serialized.data(), meta_serialized.size()); RETURN_IF_ERROR(check_ofstream_status(out, tmp_filename)); @@ -176,6 +179,8 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, if (std::rename(tmp_filename.c_str(), final_filename.c_str()) != 0) { std::remove(tmp_filename.c_str()); file_size = std::filesystem::file_size(final_filename); + } else { + LOG(WARNING) << "failed to rename " << tmp_filename << " to " << final_filename; } _dump_meta.Clear(); _current_dump_group.Clear(); @@ -217,12 +222,13 @@ void CacheLRUDumper::dump_queue(LRUQueue& queue, const std::string& queue_name) RETURN_IF_STATUS_ERROR(st, finalize_dump(out, elements.size(), tmp_filename, final_filename, file_size)); } else { - LOG(WARNING) << "open lru dump file failed"; + LOG(WARNING) << "open lru dump file failed, reason: " << tmp_filename + << " failed to create"; } } *(_mgr->_lru_dump_latency_us) << (duration_ns / 1000); - LOG(INFO) << fmt::format("lru dump for {} size={} time={}us", queue_name, file_size, - duration_ns / 1000); + LOG(INFO) << fmt::format("lru dump for {} size={} element={} time={}us", queue_name, file_size, + elements.size(), duration_ns / 1000); }; Status CacheLRUDumper::parse_dump_footer(std::ifstream& in, std::string& filename, @@ -274,7 +280,7 @@ Status CacheLRUDumper::parse_dump_footer(std::ifstream& in, std::string& filenam LOG(WARNING) << warn_msg; return Status::InternalError(warn_msg); } - LOG(INFO) << "parse meta: " << _parse_meta.DebugString(); + VLOG_DEBUG << "parse meta: " << _parse_meta.DebugString(); entry_num = _parse_meta.entry_num(); return Status::OK(); @@ -294,7 +300,6 @@ Status CacheLRUDumper::parse_one_lru_entry(std::ifstream& in, std::string& filen in.read(&group_serialized[0], group_serialized.size()); RETURN_IF_ERROR(check_ifstream_status(in, filename)); - LOG(INFO) << "Deserializing group of size: " << group_serialized.size(); if (!_current_parse_group.ParseFromString(group_serialized)) { std::string warn_msg = fmt::format("restore lru failed to parse group, file={}", filename); @@ -307,7 +312,7 @@ Status CacheLRUDumper::parse_one_lru_entry(std::ifstream& in, std::string& filen } // Get next entry from current group - LOG(INFO) << "After deserialization: " << _current_parse_group.DebugString(); + VLOG_DEBUG << "After deserialization: " << _current_parse_group.DebugString(); auto entry = _current_parse_group.entries(0); hash = UInt128Wrapper((static_cast(entry.hash().high()) << 64) | entry.hash().low()); offset = entry.offset(); @@ -330,6 +335,7 @@ void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_nam SCOPED_RAW_TIMER(&duration_ns); size_t entry_num = 0; RETURN_IF_STATUS_ERROR(st, parse_dump_footer(in, filename, entry_num)); + LOG(INFO) << "lru dump file for " << queue_name << " has " << entry_num << " entries."; in.seekg(0, std::ios::beg); UInt128Wrapper hash; size_t offset, size; @@ -348,7 +354,7 @@ void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_nam } else if (queue_name == "disposable") { ctx.cache_type = FileCacheType::DISPOSABLE; } else { - LOG_WARNING("unknown queue type"); + LOG_WARNING("unknown queue type for lru restore, skip"); DCHECK(false); return; } @@ -358,7 +364,7 @@ void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_nam } else { LOG(INFO) << "no lru dump file is founded for " << queue_name; } - LOG(INFO) << "lru restore time costs: " << (duration_ns / 1000 / 1000) << "ms."; + LOG(INFO) << "lru restore time costs: " << (duration_ns / 1000) << "us."; }; void CacheLRUDumper::remove_lru_dump_files() { @@ -372,4 +378,4 @@ void CacheLRUDumper::remove_lru_dump_files() { } } -} // end of namespace doris::io \ No newline at end of file +} // end of namespace doris::io diff --git a/be/src/io/cache/cache_lru_dumper.h b/be/src/io/cache/cache_lru_dumper.h index 9bd58759c2c117..8075786a63c4c4 100644 --- a/be/src/io/cache/cache_lru_dumper.h +++ b/be/src/io/cache/cache_lru_dumper.h @@ -71,7 +71,6 @@ class CacheLRUDumper { doris::io::cache::LRUDumpEntryGroupPb _current_parse_group; doris::io::cache::LRUDumpMetaPb _parse_meta; -private: BlockFileCache* _mgr; }; } // namespace doris::io \ No newline at end of file From 630b8a8857b78907f425893add7105e607172d86 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Mon, 30 Jun 2025 22:41:51 +0800 Subject: [PATCH 06/12] fix Signed-off-by: zhengyu --- be/src/io/cache/block_file_cache.cpp | 26 +++++++++++++------------- be/src/util/byte_stream_split.cpp | 1 + 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index 4764be6ece610b..a619a7f21608ca 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -240,10 +240,10 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, _ttl_queue = LRUQueue(cache_settings.ttl_queue_size, cache_settings.ttl_queue_elements, std::numeric_limits::max()); - _lru_queue_update_counters[FileCacheType::DISPOSABLE] = 0; - _lru_queue_update_counters[FileCacheType::NORMAL] = 0; - _lru_queue_update_counters[FileCacheType::INDEX] = 0; - _lru_queue_update_counters[FileCacheType::TTL] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] = 0; _lru_dumper = std::make_unique(this); if (cache_settings.storage == "memory") { @@ -2219,7 +2219,7 @@ void BlockFileCache::update_ttl_atime(const UInt128Wrapper& hash) { } BlockFileCache::CacheLRULogQueue& BlockFileCache::get_lru_log_queue(FileCacheType type) { - ++_lru_queue_update_counters[type]; + ++_lru_queue_update_cnt_from_last_dump[type]; switch (type) { case FileCacheType::INDEX: return _index_lru_log_queue; @@ -2313,25 +2313,25 @@ void BlockFileCache::run_background_lru_dump() { } if (config::file_cache_background_lru_dump_tail_record_num > 0) { - if (_lru_queue_update_counters[FileCacheType::DISPOSABLE] > + if (_lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] > config::file_cache_background_lru_dump_update_cnt_threshold) { _lru_dumper->dump_queue(_shadow_disposable_queue, "disposable"); - _lru_queue_update_counters[FileCacheType::DISPOSABLE] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] = 0; } - if (_lru_queue_update_counters[FileCacheType::NORMAL] > + if (_lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] > config::file_cache_background_lru_dump_update_cnt_threshold) { _lru_dumper->dump_queue(_shadow_normal_queue, "normal"); - _lru_queue_update_counters[FileCacheType::NORMAL] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] = 0; } - if (_lru_queue_update_counters[FileCacheType::INDEX] > + if (_lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] > config::file_cache_background_lru_dump_update_cnt_threshold) { _lru_dumper->dump_queue(_shadow_index_queue, "index"); - _lru_queue_update_counters[FileCacheType::INDEX] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] = 0; } - if (_lru_queue_update_counters[FileCacheType::TTL] > + if (_lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] > config::file_cache_background_lru_dump_update_cnt_threshold) { _lru_dumper->dump_queue(_shadow_ttl_queue, "ttl"); - _lru_queue_update_counters[FileCacheType::TTL] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] = 0; } } } diff --git a/be/src/util/byte_stream_split.cpp b/be/src/util/byte_stream_split.cpp index 6ba84ff6ba7ae4..398f10ed086103 100644 --- a/be/src/util/byte_stream_split.cpp +++ b/be/src/util/byte_stream_split.cpp @@ -19,6 +19,7 @@ #include +#include #include #include #include From a9a9aff645b04ce3fb8821ed013d395faa207b8d Mon Sep 17 00:00:00 2001 From: zhengyu Date: Tue, 1 Jul 2025 10:58:04 +0800 Subject: [PATCH 07/12] group checksum instead of file checksum to save memory Signed-off-by: zhengyu --- be/src/io/cache/cache_lru_dumper.cpp | 13 +++++++++++-- gensrc/proto/file_cache.proto | 1 + 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index 8acf795242ed1e..786c19ffd2bb9e 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -19,6 +19,7 @@ #include "io/cache/block_file_cache.h" #include "io/cache/cache_lru_dumper.h" +#include "util/crc32c.h" namespace doris::io { Status CacheLRUDumper::check_ofstream_status(std::ofstream& out, std::string& filename) { @@ -134,6 +135,8 @@ Status CacheLRUDumper::flush_current_group(std::ofstream& out, std::string& file ::doris::io::cache::EntryGroupOffsetSizePb* group_info = _dump_meta.add_group_offset_size(); group_info->set_offset(group_start); group_info->set_size(serialized.size()); + uint32_t checksum = crc32c::Value(serialized.data(), serialized.size()); + group_info->set_checksum(checksum); // Reset for next group _current_dump_group.Clear(); @@ -166,7 +169,7 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, // Write footer Footer footer; footer.meta_offset = htole64(meta_offset); // Explicitly convert to little-endian - footer.checksum = 0; // TODO: Calculate checksum + footer.checksum = 0; footer.version = 1; std::memcpy(footer.magic, "DOR", 3); @@ -299,7 +302,13 @@ Status CacheLRUDumper::parse_one_lru_entry(std::ifstream& in, std::string& filen std::string group_serialized(group_info.size(), '\0'); in.read(&group_serialized[0], group_serialized.size()); RETURN_IF_ERROR(check_ifstream_status(in, filename)); - + uint32_t checksum = crc32c::Value(group_serialized.data(), group_serialized.size()); + if (checksum != group_info.checksum()) { + std::string warn_msg = + fmt::format("restore lru failed as checksum not match, file={}", filename); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } if (!_current_parse_group.ParseFromString(group_serialized)) { std::string warn_msg = fmt::format("restore lru failed to parse group, file={}", filename); diff --git a/gensrc/proto/file_cache.proto b/gensrc/proto/file_cache.proto index 5230abe54053f2..f11375586aa16c 100644 --- a/gensrc/proto/file_cache.proto +++ b/gensrc/proto/file_cache.proto @@ -38,6 +38,7 @@ message LRUDumpEntryGroupPb { message EntryGroupOffsetSizePb { optional uint64 offset = 1; optional uint64 size = 2; + optional uint32 checksum = 3; } message LRUDumpMetaPb{ From c487f22661a27b378473c39f4755c241608ef6e7 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Tue, 1 Jul 2025 20:12:08 +0800 Subject: [PATCH 08/12] check shadow queue diff Signed-off-by: zhengyu --- be/src/common/config.cpp | 1 + be/src/common/config.h | 1 + be/src/io/cache/block_file_cache.cpp | 63 ++++++++++++++- be/src/io/cache/block_file_cache.h | 5 ++ be/src/io/cache/cache_lru_dumper.cpp | 7 +- be/src/util/byte_stream_split.cpp | 2 +- be/test/io/cache/lru_queue_test.cpp | 117 +++++++++++++++++++++++++++ 7 files changed, 193 insertions(+), 3 deletions(-) create mode 100644 be/test/io/cache/lru_queue_test.cpp diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 756d06f482d2bb..fb556c5c796874 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1118,6 +1118,7 @@ DEFINE_mInt64(file_cache_background_lru_dump_interval_ms, "60000"); DEFINE_mInt64(file_cache_background_lru_dump_update_cnt_threshold, "1000"); DEFINE_mInt64(file_cache_background_lru_dump_tail_record_num, "5000000"); DEFINE_mInt64(file_cache_background_lru_log_replay_interval_ms, "1000"); +DEFINE_mBool(enable_evaluate_shadow_queue_diff, "false"); DEFINE_Int32(file_cache_downloader_thread_num_min, "32"); DEFINE_Int32(file_cache_downloader_thread_num_max, "32"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 7f9fb5b9650d20..5238845ea4ee31 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1160,6 +1160,7 @@ DECLARE_mInt64(file_cache_background_lru_dump_interval_ms); DECLARE_mInt64(file_cache_background_lru_dump_update_cnt_threshold); DECLARE_mInt64(file_cache_background_lru_dump_tail_record_num); DECLARE_mInt64(file_cache_background_lru_log_replay_interval_ms); +DECLARE_mBool(enable_evaluate_shadow_queue_diff); // inverted index searcher cache // cache entry stay time after lookup diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index a619a7f21608ca..f93f6161f5371a 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -230,6 +230,8 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, _cache_base_path.c_str(), "file_cache_recycle_keys_length"); _ttl_gc_latency_us = std::make_shared(_cache_base_path.c_str(), "file_cache_ttl_gc_latency_us"); + _shadow_queue_levenshtein_distance = std::make_shared( + _cache_base_path.c_str(), "file_cache_shadow_queue_levenshtein_distance"); _disposable_queue = LRUQueue(cache_settings.disposable_queue_size, cache_settings.disposable_queue_elements, 60 * 60); @@ -1593,6 +1595,48 @@ std::string LRUQueue::to_string(std::lock_guard& /* cache_lock */) c return result; } +size_t LRUQueue::levenshtein_distance_from(LRUQueue& base, + std::lock_guard& cache_lock) { + std::list target_queue = this->queue; + std::list base_queue = base.queue; + std::vector vec1(target_queue.begin(), target_queue.end()); + std::vector vec2(base_queue.begin(), base_queue.end()); + + size_t m = vec1.size(); + size_t n = vec2.size(); + + // Create a 2D vector (matrix) to store the Levenshtein distances + // dp[i][j] will hold the distance between the first i elements of vec1 and the first j elements of vec2 + std::vector> dp(m + 1, std::vector(n + 1, 0)); + + // Initialize the first row and column of the matrix + // The distance between an empty list and a list of length k is k (all insertions or deletions) + for (size_t i = 0; i <= m; ++i) { + dp[i][0] = i; + } + for (size_t j = 0; j <= n; ++j) { + dp[0][j] = j; + } + + // Fill the matrix using dynamic programming + for (size_t i = 1; i <= m; ++i) { + for (size_t j = 1; j <= n; ++j) { + // Check if the current elements of both vectors are equal + size_t cost = (vec1[i - 1].hash == vec2[j - 1].hash && + vec1[i - 1].offset == vec2[j - 1].offset) + ? 0 + : 1; + // Calculate the minimum cost of three possible operations: + // 1. Insertion: dp[i][j-1] + 1 + // 2. Deletion: dp[i-1][j] + 1 + // 3. Substitution: dp[i-1][j-1] + cost (0 if elements are equal, 1 if not) + dp[i][j] = std::min({dp[i - 1][j] + 1, dp[i][j - 1] + 1, dp[i - 1][j - 1] + cost}); + } + } + // The bottom-right cell of the matrix contains the Levenshtein distance + return dp[m][n]; +} + std::string BlockFileCache::dump_structure(const UInt128Wrapper& hash) { SCOPED_CACHE_LOCK(_mutex, this); return dump_structure_unlocked(hash, cache_lock); @@ -2297,7 +2341,24 @@ void BlockFileCache::run_background_lru_log_replay() { replay_queue_event(_normal_lru_log_queue, _shadow_normal_queue); replay_queue_event(_disposable_lru_log_queue, _shadow_disposable_queue); - //TODO(zhengyu): add debug facilities to check diff between real and shadow queue + if (config::enable_evaluate_shadow_queue_diff) { + evaluate_queue_diff(_shadow_ttl_queue, _ttl_queue, "ttl"); + evaluate_queue_diff(_shadow_index_queue, _index_queue, "index"); + evaluate_queue_diff(_shadow_normal_queue, _normal_queue, "normal"); + evaluate_queue_diff(_shadow_disposable_queue, _disposable_queue, "disposable"); + } + } +} + +// we evaluate the diff between two queue by calculate how many operation is +// needed for transfer one to another (Levenshtein Distance) +// NOTE: HEAVY calculation with cache lock, only for debugging +void BlockFileCache::evaluate_queue_diff(LRUQueue& target, LRUQueue& base, std::string name) { + SCOPED_CACHE_LOCK(_mutex, this); + size_t distance = target.levenshtein_distance_from(base, cache_lock); + *_shadow_queue_levenshtein_distance << distance; + if (distance > 20) { + LOG(WARNING) << name << " shadow queue is different from real queue"; } } diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index d1e70df0f74d8b..93700d3fb3e0be 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -151,6 +151,8 @@ class LRUQueue { cache_size = 0; } + size_t levenshtein_distance_from(LRUQueue& base, std::lock_guard& cache_lock); + size_t max_size; size_t max_element_size; std::list queue; @@ -523,6 +525,7 @@ class BlockFileCache { void record_queue_event(CacheLRULogQueue& log_queue, CacheLRULogType log_type, const UInt128Wrapper hash, const size_t offset, const size_t size); void replay_queue_event(CacheLRULogQueue& log_queue, LRUQueue& shadown_queue); + void evaluate_queue_diff(LRUQueue& target, LRUQueue& base, std::string name); Status check_ofstream_status(std::ofstream& out, std::string& filename); Status dump_one_lru_entry(std::ofstream& out, std::string& filename, const UInt128Wrapper& hash, @@ -637,6 +640,8 @@ class BlockFileCache { std::shared_ptr _evict_in_advance_latency_us; std::shared_ptr _recycle_keys_length_recorder; std::shared_ptr _ttl_gc_latency_us; + + std::shared_ptr _shadow_queue_levenshtein_distance; // keep _storage last so it will deconstruct first // otherwise, load_cache_info_into_memory might crash // coz it will use other members of BlockFileCache diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index 786c19ffd2bb9e..b3576be85846ed 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -354,7 +354,11 @@ void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_nam if (queue_name == "ttl") { ctx.cache_type = FileCacheType::TTL; // TODO(zhengyu): we haven't persist expiration time yet, use 3h default - // TODO(zhengyu): we don't use stats yet, see if this will cause any problem + // There are mulitiple places we can correct this fake 3h ttl, e.g.: + // 1. during load_cache_info_into_memory (this will cause overwriting the ttl of async load) + // 2. after restoring, use sync_meta to modify the ttl + // However, I plan not to do this in this commit but to figure a more elegant way + // after ttl expiration time being changed from file name encoding to rocksdb persistency. ctx.expiration_time = 10800; } else if (queue_name == "index") { ctx.cache_type = FileCacheType::INDEX; @@ -367,6 +371,7 @@ void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_nam DCHECK(false); return; } + // TODO(zhengyu): we don't use stats yet, see if this will cause any problem _mgr->add_cell(hash, ctx, offset, size, FileBlock::State::DOWNLOADED, cache_lock); } in.close(); diff --git a/be/src/util/byte_stream_split.cpp b/be/src/util/byte_stream_split.cpp index 398f10ed086103..c38bff7966d14b 100644 --- a/be/src/util/byte_stream_split.cpp +++ b/be/src/util/byte_stream_split.cpp @@ -19,8 +19,8 @@ #include -#include #include +#include #include #include diff --git a/be/test/io/cache/lru_queue_test.cpp b/be/test/io/cache/lru_queue_test.cpp new file mode 100644 index 00000000000000..4a01fb27e3dcfe --- /dev/null +++ b/be/test/io/cache/lru_queue_test.cpp @@ -0,0 +1,117 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include + +// IWYU pragma: no_include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "cpp/sync_point.h" +#include "gtest/gtest_pred_impl.h" +#include "io/cache/block_file_cache.h" +#include "util/time.h" + +using namespace doris::io; + +class LRUQueueTest : public ::testing::Test { +protected: + void SetUp() override { + queue1 = std::make_shared(); + queue2 = std::make_shared(); + } + + std::shared_ptr queue1; + std::shared_ptr queue2; +}; + +TEST_F(LRUQueueTest, SameQueueDistance) { + std::mutex mutex; + std::lock_guard lock(mutex); + + queue1->add(UInt128Wrapper(123), 0, 1024, lock); + queue1->add(UInt128Wrapper(456), 0, 1024, lock); + + EXPECT_EQ(queue1->levenshtein_distance_from(*queue1, lock), 0); +} + +TEST_F(LRUQueueTest, DifferentQueueDistance) { + std::mutex mutex; + std::lock_guard lock(mutex); + + queue1->add(UInt128Wrapper(123), 0, 1024, lock); + queue1->add(UInt128Wrapper(456), 0, 1024, lock); + + queue2->add(UInt128Wrapper(123), 0, 1024, lock); + queue2->add(UInt128Wrapper(789), 0, 1024, lock); + + EXPECT_EQ(queue1->levenshtein_distance_from(*queue2, lock), 1); +} + +TEST_F(LRUQueueTest, EmptyQueueDistance) { + std::mutex mutex; + std::lock_guard lock(mutex); + + queue1->add(UInt128Wrapper(123), 0, 1024, lock); + queue1->add(UInt128Wrapper(456), 0, 1024, lock); + + EXPECT_EQ(queue1->levenshtein_distance_from(*queue2, lock), 2); +} + +TEST_F(LRUQueueTest, PartialMatchDistance) { + std::mutex mutex; + std::lock_guard lock(mutex); + + queue1->add(UInt128Wrapper(123), 0, 1024, lock); + queue1->add(UInt128Wrapper(456), 0, 1024, lock); + queue1->add(UInt128Wrapper(789), 0, 1024, lock); + + queue2->add(UInt128Wrapper(123), 0, 1024, lock); + queue2->add(UInt128Wrapper(101), 0, 1024, lock); + queue2->add(UInt128Wrapper(789), 0, 1024, lock); + + EXPECT_EQ(queue1->levenshtein_distance_from(*queue2, lock), 1); +} + +TEST_F(LRUQueueTest, SameElementsDifferentOrder) { + std::mutex mutex; + std::lock_guard lock(mutex); + + queue1->add(UInt128Wrapper(123), 0, 1024, lock); + queue1->add(UInt128Wrapper(456), 0, 1024, lock); + queue1->add(UInt128Wrapper(789), 0, 1024, lock); + + queue2->add(UInt128Wrapper(789), 0, 1024, lock); + queue2->add(UInt128Wrapper(456), 0, 1024, lock); + queue2->add(UInt128Wrapper(123), 0, 1024, lock); + + EXPECT_EQ(queue1->levenshtein_distance_from(*queue2, lock), 2); +} From e4287cc171f424c5fe4aea095ee13411c45d62cd Mon Sep 17 00:00:00 2001 From: zhengyu Date: Wed, 2 Jul 2025 16:17:12 +0800 Subject: [PATCH 09/12] coding style Signed-off-by: zhengyu --- be/src/io/cache/cache_lru_dumper.cpp | 62 +++++++++++++++++++++++++++- be/src/io/cache/cache_lru_dumper.h | 4 +- 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index b3576be85846ed..0894e1ec23fa9c 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -22,6 +22,55 @@ #include "util/crc32c.h" namespace doris::io { + +std::string CacheLRUDumper::Footer::serialize_as_string() const { + std::string result; + result.reserve(sizeof(Footer)); + + // Serialize meta_offset + const char* meta_offset_ptr = reinterpret_cast(&meta_offset); + result.append(meta_offset_ptr, sizeof(meta_offset)); + + // Serialize checksum + const char* checksum_ptr = reinterpret_cast(&checksum); + result.append(checksum_ptr, sizeof(checksum)); + + // Serialize version + const char* version_ptr = reinterpret_cast(&version); + result.append(version_ptr, sizeof(version)); + + // Serialize magic + result.append(magic, sizeof(magic)); + + return result; +} + +bool CacheLRUDumper::Footer::deserialize_from_string(const std::string& data) { + if (data.size() != sizeof(Footer)) { + return false; + } + + const char* ptr = data.data(); + + // Deserialize meta_offset + std::memcpy(&meta_offset, ptr, sizeof(meta_offset)); + meta_offset = le64toh(meta_offset); // Convert from little-endian + ptr += sizeof(meta_offset); + + // Deserialize checksum + std::memcpy(&checksum, ptr, sizeof(checksum)); + ptr += sizeof(checksum); + + // Deserialize version + std::memcpy(&version, ptr, sizeof(version)); + ptr += sizeof(version); + + // Deserialize magic + std::memcpy(magic, ptr, sizeof(magic)); + + return true; +} + Status CacheLRUDumper::check_ofstream_status(std::ofstream& out, std::string& filename) { if (!out.good()) { std::ios::iostate state = out.rdstate(); @@ -173,7 +222,8 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, footer.version = 1; std::memcpy(footer.magic, "DOR", 3); - out.write(reinterpret_cast(&footer), sizeof(footer)); + std::string footer_str = footer.serialize_as_string(); + out.write(footer_str.data(), footer_str.size()); RETURN_IF_ERROR(check_ofstream_status(out, tmp_filename)); out.close(); @@ -249,9 +299,17 @@ Status CacheLRUDumper::parse_dump_footer(std::ifstream& in, std::string& filenam } in.seekg(-footer_size, std::ios::end); - in.read(reinterpret_cast(&footer), footer_size); + std::string footer_str(footer_size, '\0'); + in.read(&footer_str[0], footer_size); RETURN_IF_ERROR(check_ifstream_status(in, filename)); + if (!footer.deserialize_from_string(footer_str)) { + std::string warn_msg = std::string( + fmt::format("Failed to deserialize footer, file={}, skip restore", filename)); + LOG(WARNING) << warn_msg; + return Status::InternalError(warn_msg); + } + // Convert from little-endian to host byte order footer.meta_offset = le64toh(footer.meta_offset); diff --git a/be/src/io/cache/cache_lru_dumper.h b/be/src/io/cache/cache_lru_dumper.h index 8075786a63c4c4..b0cb0e29f31a4b 100644 --- a/be/src/io/cache/cache_lru_dumper.h +++ b/be/src/io/cache/cache_lru_dumper.h @@ -59,7 +59,9 @@ class CacheLRUDumper { uint32_t checksum; uint8_t version; char magic[3]; - }; + + std::string serialize_as_string() const; + } __attribute__((packed)); private: // For dumping From b5a99c73872eb5fb0e00446bd48281a898f25eae Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 3 Jul 2025 00:30:11 +0800 Subject: [PATCH 10/12] refractor: put shadow queue record&replay into seperate file Signed-off-by: zhengyu --- be/src/http/action/shrink_mem_action.cpp | 2 + be/src/io/cache/block_file_cache.cpp | 244 +++++------------- be/src/io/cache/block_file_cache.h | 127 +-------- be/src/io/cache/cache_lru_dumper.cpp | 35 ++- be/src/io/cache/cache_lru_dumper.h | 9 +- be/src/io/cache/file_block.cpp | 8 +- be/src/io/cache/file_cache_common.cpp | 56 ++++ be/src/io/cache/file_cache_common.h | 96 +++++++ be/src/io/cache/fs_file_cache_storage.cpp | 10 +- be/src/io/cache/lru_queue_recorder.cpp | 132 ++++++++++ be/src/io/cache/lru_queue_recorder.h | 83 ++++++ be/src/runtime/exec_env.h | 3 + be/test/io/cache/block_file_cache_test.cpp | 13 +- .../cache/block_file_cache_test_lru_dump.cpp | 45 ++-- be/test/io/cache/cache_lru_dumper_test.cpp | 13 +- .../suites/demo_p0/test_lru_persist.groovy | 22 +- 16 files changed, 524 insertions(+), 374 deletions(-) create mode 100644 be/src/io/cache/lru_queue_recorder.cpp create mode 100644 be/src/io/cache/lru_queue_recorder.h diff --git a/be/src/http/action/shrink_mem_action.cpp b/be/src/http/action/shrink_mem_action.cpp index 66331f4a943917..41f55365cffcf8 100644 --- a/be/src/http/action/shrink_mem_action.cpp +++ b/be/src/http/action/shrink_mem_action.cpp @@ -36,6 +36,8 @@ void ShrinkMemAction::handle(HttpRequest* req) { MemoryReclamation::revoke_process_memory("ShrinkMemAction"); LOG(INFO) << "shrink memory triggered, using Process GC Free Memory"; HttpChannel::send_reply(req, HttpStatus::OK, "shrinking"); + + ExecEnv::GetInstance()->set_is_upgrading(); } } // namespace doris diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index f93f6161f5371a..4522db10601ab4 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -26,6 +26,7 @@ #include "common/status.h" #include "cpp/sync_point.h" #include "gen_cpp/file_cache.pb.h" +#include "runtime/exec_env.h" #if defined(__APPLE__) #include @@ -242,12 +243,8 @@ BlockFileCache::BlockFileCache(const std::string& cache_base_path, _ttl_queue = LRUQueue(cache_settings.ttl_queue_size, cache_settings.ttl_queue_elements, std::numeric_limits::max()); - _lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] = 0; - _lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] = 0; - _lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] = 0; - _lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] = 0; - - _lru_dumper = std::make_unique(this); + _lru_recorder = std::make_unique(this); + _lru_dumper = std::make_unique(this, _lru_recorder.get()); if (cache_settings.storage == "memory") { _storage = std::make_unique(); _cache_base_path = "memory"; @@ -264,32 +261,6 @@ UInt128Wrapper BlockFileCache::hash(const std::string& path) { return UInt128Wrapper(value); } -std::string BlockFileCache::cache_type_to_string(FileCacheType type) { - switch (type) { - case FileCacheType::INDEX: - return "_idx"; - case FileCacheType::DISPOSABLE: - return "_disposable"; - case FileCacheType::NORMAL: - return ""; - case FileCacheType::TTL: - return "_ttl"; - } - return ""; -} - -FileCacheType BlockFileCache::string_to_cache_type(const std::string& str) { - if (str == "idx") { - return FileCacheType::INDEX; - } else if (str == "disposable") { - return FileCacheType::DISPOSABLE; - } else if (str == "ttl") { - return FileCacheType::TTL; - } - DCHECK(false) << "The string is " << str; - return FileCacheType::DISPOSABLE; -} - BlockFileCache::QueryFileCacheContextHolderPtr BlockFileCache::get_query_context_holder( const TUniqueId& query_id) { SCOPED_CACHE_LOCK(_mutex, this); @@ -406,9 +377,9 @@ void BlockFileCache::use_cell(const FileBlockCell& cell, FileBlocks* result, boo if (cell.queue_iterator && move_iter_flag) { queue.move_to_end(*cell.queue_iterator, cache_lock); } - record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), - CacheLRULogType::MOVETOBACK, cell.file_block->_key.hash, - cell.file_block->_key.offset, cell.size()); + _lru_recorder->record_queue_event(cell.file_block->cache_type(), CacheLRULogType::MOVETOBACK, + cell.file_block->_key.hash, cell.file_block->_key.offset, + cell.size()); cell.update_atime(); } @@ -482,16 +453,16 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte if (st.ok()) { auto& queue = get_queue(origin_type); queue.remove(cell.queue_iterator.value(), cache_lock); - record_queue_event(get_lru_log_queue(origin_type), CacheLRULogType::REMOVE, - cell.file_block->get_hash_value(), cell.file_block->offset(), - cell.size()); + _lru_recorder->record_queue_event(origin_type, CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); auto& ttl_queue = get_queue(FileCacheType::TTL); cell.queue_iterator = ttl_queue.add(cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::TTL), CacheLRULogType::ADD, - cell.file_block->get_hash_value(), cell.file_block->offset(), - cell.size()); + _lru_recorder->record_queue_event(FileCacheType::TTL, CacheLRULogType::ADD, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change key meta").error(st); } @@ -528,18 +499,18 @@ FileBlocks BlockFileCache::get_impl(const UInt128Wrapper& hash, const CacheConte if (cell.queue_iterator) { auto& ttl_queue = get_queue(FileCacheType::TTL); ttl_queue.remove(cell.queue_iterator.value(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::TTL), - CacheLRULogType::REMOVE, - cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event(FileCacheType::TTL, + CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } auto& queue = get_queue(FileCacheType::NORMAL); cell.queue_iterator = queue.add(cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::NORMAL), - CacheLRULogType::ADD, cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event(FileCacheType::NORMAL, CacheLRULogType::ADD, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change key meta").error(st); } @@ -846,8 +817,9 @@ BlockFileCache::FileBlockCell* BlockFileCache::add_cell(const UInt128Wrapper& ha auto& queue = get_queue(cell.file_block->cache_type()); cell.queue_iterator = queue.add(hash, offset, size, cache_lock); - record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), CacheLRULogType::ADD, - cell.file_block->get_hash_value(), cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event(cell.file_block->cache_type(), CacheLRULogType::ADD, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); if (cell.file_block->cache_type() == FileCacheType::TTL) { if (_key_to_time.find(hash) == _key_to_time.end()) { @@ -1131,18 +1103,19 @@ bool BlockFileCache::remove_if_ttl_file_blocks(const UInt128Wrapper& file_key, b if (st.ok()) { if (cell.queue_iterator) { ttl_queue.remove(cell.queue_iterator.value(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::TTL), - CacheLRULogType::REMOVE, - cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event( + FileCacheType::TTL, CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), cell.file_block->offset(), + cell.size()); } auto& queue = get_queue(FileCacheType::NORMAL); cell.queue_iterator = queue.add( cell.file_block->get_hash_value(), cell.file_block->offset(), cell.file_block->range().size(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::NORMAL), - CacheLRULogType::ADD, cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event(FileCacheType::NORMAL, + CacheLRULogType::ADD, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } else { LOG_WARNING("Failed to change cache type to normal").error(st); } @@ -1431,9 +1404,9 @@ void BlockFileCache::remove(FileBlockSPtr file_block, T& cache_lock, U& block_lo if (cell->queue_iterator) { auto& queue = get_queue(file_block->cache_type()); queue.remove(*cell->queue_iterator, cache_lock); - record_queue_event(get_lru_log_queue(file_block->cache_type()), CacheLRULogType::REMOVE, - cell->file_block->get_hash_value(), cell->file_block->offset(), - cell->size()); + _lru_recorder->record_queue_event(file_block->cache_type(), CacheLRULogType::REMOVE, + cell->file_block->get_hash_value(), + cell->file_block->offset(), cell->size()); } *_queue_evict_size_metrics[static_cast(file_block->cache_type())] << file_block->range().size(); @@ -1557,14 +1530,6 @@ LRUQueue::Iterator LRUQueue::add(const UInt128Wrapper& hash, size_t offset, size return iter; } -template - requires IsXLock -void LRUQueue::remove(Iterator queue_it, T& /* cache_lock */) { - cache_size -= queue_it->size; - map.erase(std::make_pair(queue_it->hash, queue_it->offset)); - queue.erase(queue_it); -} - void LRUQueue::remove_all(std::lock_guard& /* cache_lock */) { queue.clear(); map.clear(); @@ -1688,15 +1653,15 @@ void BlockFileCache::change_cache_type(const UInt128Wrapper& hash, size_t offset auto& cur_queue = get_queue(cell.file_block->cache_type()); DCHECK(cell.queue_iterator.has_value()); cur_queue.remove(*cell.queue_iterator, cache_lock); - record_queue_event(get_lru_log_queue(cell.file_block->cache_type()), - CacheLRULogType::REMOVE, cell.file_block->get_hash_value(), - cell.file_block->offset(), cell.size()); + _lru_recorder->record_queue_event( + cell.file_block->cache_type(), CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), cell.file_block->offset(), cell.size()); auto& new_queue = get_queue(new_type); cell.queue_iterator = new_queue.add(hash, offset, cell.file_block->range().size(), cache_lock); - record_queue_event(get_lru_log_queue(new_type), CacheLRULogType::ADD, - cell.file_block->get_hash_value(), cell.file_block->offset(), - cell.size()); + _lru_recorder->record_queue_event(new_type, CacheLRULogType::ADD, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } } } @@ -2093,15 +2058,15 @@ void BlockFileCache::modify_expiration_time(const UInt128Wrapper& hash, if (st.ok()) { auto& queue = get_queue(origin_type); queue.remove(cell.queue_iterator.value(), cache_lock); - record_queue_event(get_lru_log_queue(origin_type), CacheLRULogType::REMOVE, - cell.file_block->get_hash_value(), cell.file_block->offset(), - cell.size()); + _lru_recorder->record_queue_event(origin_type, CacheLRULogType::REMOVE, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); auto& ttl_queue = get_queue(FileCacheType::TTL); cell.queue_iterator = ttl_queue.add(hash, cell.file_block->offset(), cell.file_block->range().size(), cache_lock); - record_queue_event(get_lru_log_queue(FileCacheType::TTL), CacheLRULogType::ADD, - cell.file_block->get_hash_value(), cell.file_block->offset(), - cell.size()); + _lru_recorder->record_queue_event(FileCacheType::TTL, CacheLRULogType::ADD, + cell.file_block->get_hash_value(), + cell.file_block->offset(), cell.size()); } if (!st.ok()) { LOG_WARNING("").error(st); @@ -2262,69 +2227,6 @@ void BlockFileCache::update_ttl_atime(const UInt128Wrapper& hash) { }; } -BlockFileCache::CacheLRULogQueue& BlockFileCache::get_lru_log_queue(FileCacheType type) { - ++_lru_queue_update_cnt_from_last_dump[type]; - switch (type) { - case FileCacheType::INDEX: - return _index_lru_log_queue; - case FileCacheType::DISPOSABLE: - return _disposable_lru_log_queue; - case FileCacheType::NORMAL: - return _normal_lru_log_queue; - case FileCacheType::TTL: - return _ttl_lru_log_queue; - default: - DCHECK(false); - } - return _normal_lru_log_queue; -} - -void BlockFileCache::record_queue_event(CacheLRULogQueue& log_queue, CacheLRULogType log_type, - const UInt128Wrapper hash, const size_t offset, - const size_t size) { - log_queue.push_back(std::make_unique(log_type, hash, offset, size)); -} - -void BlockFileCache::replay_queue_event(CacheLRULogQueue& log_queue, LRUQueue& shadow_queue) { - // we don't need the real cache lock for the shadow queue, but we do need a lock to prevent read/write contension - std::lock_guard lru_log_lock(_mutex_lru_log); - while (!log_queue.empty()) { - auto log = std::move(log_queue.front()); - log_queue.pop_front(); - try { - switch (log->type) { - case CacheLRULogType::ADD: { - shadow_queue.add(log->hash, log->offset, log->size, lru_log_lock); - break; - } - case CacheLRULogType::REMOVE: { - auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); - if (it != shadow_queue.end()) { - shadow_queue.remove(it, lru_log_lock); - } else { - LOG(WARNING) << "REMOVE failed, doesn't exist in shadow queue"; - } - break; - } - case CacheLRULogType::MOVETOBACK: { - auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); - if (it != shadow_queue.end()) { - shadow_queue.move_to_end(it, lru_log_lock); - } else { - LOG(WARNING) << "MOVETOBACK failed, doesn't exist in shadow queue"; - } - break; - } - default: - LOG(WARNING) << "Unknown CacheLRULogType: " << static_cast(log->type); - break; - } - } catch (const std::exception& e) { - LOG(WARNING) << "Failed to replay queue event: " << e.what(); - } - } -} - void BlockFileCache::run_background_lru_log_replay() { while (!_close) { int64_t interval_ms = config::file_cache_background_lru_log_replay_interval_ms; @@ -2336,32 +2238,21 @@ void BlockFileCache::run_background_lru_log_replay() { } } - replay_queue_event(_ttl_lru_log_queue, _shadow_ttl_queue); - replay_queue_event(_index_lru_log_queue, _shadow_index_queue); - replay_queue_event(_normal_lru_log_queue, _shadow_normal_queue); - replay_queue_event(_disposable_lru_log_queue, _shadow_disposable_queue); + _lru_recorder->replay_queue_event(FileCacheType::TTL); + _lru_recorder->replay_queue_event(FileCacheType::INDEX); + _lru_recorder->replay_queue_event(FileCacheType::NORMAL); + _lru_recorder->replay_queue_event(FileCacheType::DISPOSABLE); if (config::enable_evaluate_shadow_queue_diff) { - evaluate_queue_diff(_shadow_ttl_queue, _ttl_queue, "ttl"); - evaluate_queue_diff(_shadow_index_queue, _index_queue, "index"); - evaluate_queue_diff(_shadow_normal_queue, _normal_queue, "normal"); - evaluate_queue_diff(_shadow_disposable_queue, _disposable_queue, "disposable"); + SCOPED_CACHE_LOCK(_mutex, this); + _lru_recorder->evaluate_queue_diff(_ttl_queue, "ttl", cache_lock); + _lru_recorder->evaluate_queue_diff(_index_queue, "index", cache_lock); + _lru_recorder->evaluate_queue_diff(_normal_queue, "normal", cache_lock); + _lru_recorder->evaluate_queue_diff(_disposable_queue, "disposable", cache_lock); } } } -// we evaluate the diff between two queue by calculate how many operation is -// needed for transfer one to another (Levenshtein Distance) -// NOTE: HEAVY calculation with cache lock, only for debugging -void BlockFileCache::evaluate_queue_diff(LRUQueue& target, LRUQueue& base, std::string name) { - SCOPED_CACHE_LOCK(_mutex, this); - size_t distance = target.levenshtein_distance_from(base, cache_lock); - *_shadow_queue_levenshtein_distance << distance; - if (distance > 20) { - LOG(WARNING) << name << " shadow queue is different from real queue"; - } -} - void BlockFileCache::run_background_lru_dump() { while (!_close) { int64_t interval_ms = config::file_cache_background_lru_dump_interval_ms; @@ -2373,27 +2264,12 @@ void BlockFileCache::run_background_lru_dump() { } } - if (config::file_cache_background_lru_dump_tail_record_num > 0) { - if (_lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] > - config::file_cache_background_lru_dump_update_cnt_threshold) { - _lru_dumper->dump_queue(_shadow_disposable_queue, "disposable"); - _lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] = 0; - } - if (_lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] > - config::file_cache_background_lru_dump_update_cnt_threshold) { - _lru_dumper->dump_queue(_shadow_normal_queue, "normal"); - _lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] = 0; - } - if (_lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] > - config::file_cache_background_lru_dump_update_cnt_threshold) { - _lru_dumper->dump_queue(_shadow_index_queue, "index"); - _lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] = 0; - } - if (_lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] > - config::file_cache_background_lru_dump_update_cnt_threshold) { - _lru_dumper->dump_queue(_shadow_ttl_queue, "ttl"); - _lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] = 0; - } + if (config::file_cache_background_lru_dump_tail_record_num > 0 && + !ExecEnv::GetInstance()->get_is_upgrading()) { + _lru_dumper->dump_queue("disposable"); + _lru_dumper->dump_queue("normal"); + _lru_dumper->dump_queue("index"); + _lru_dumper->dump_queue("ttl"); } } } diff --git a/be/src/io/cache/block_file_cache.h b/be/src/io/cache/block_file_cache.h index 93700d3fb3e0be..eaf62c1a82cce0 100644 --- a/be/src/io/cache/block_file_cache.h +++ b/be/src/io/cache/block_file_cache.h @@ -30,6 +30,7 @@ #include "io/cache/file_block.h" #include "io/cache/file_cache_common.h" #include "io/cache/file_cache_storage.h" +#include "io/cache/lru_queue_recorder.h" #include "util/runtime_profile.h" #include "util/threadpool.h" @@ -73,94 +74,8 @@ class LockScopedTimer { #define SCOPED_CACHE_LOCK(MUTEX, cache) std::lock_guard cache_lock(MUTEX); #endif -template -concept IsXLock = std::same_as> || - std::same_as>; - class FSFileCacheStorage; -class LRUQueue { -public: - LRUQueue() = default; - LRUQueue(size_t max_size, size_t max_element_size, int64_t hot_data_interval) - : max_size(max_size), - max_element_size(max_element_size), - hot_data_interval(hot_data_interval) {} - - struct HashFileKeyAndOffset { - std::size_t operator()(const std::pair& pair) const { - return KeyHash()(pair.first) + pair.second; - } - }; - - struct FileKeyAndOffset { - UInt128Wrapper hash; - size_t offset; - size_t size; - - FileKeyAndOffset(const UInt128Wrapper& hash, size_t offset, size_t size) - : hash(hash), offset(offset), size(size) {} - }; - - using Iterator = typename std::list::iterator; - - size_t get_max_size() const { return max_size; } - size_t get_max_element_size() const { return max_element_size; } - - template - requires IsXLock - size_t get_capacity(T& /* cache_lock */) const { - return cache_size; - } - - size_t get_capacity_unsafe() const { return cache_size; } - - size_t get_elements_num_unsafe() const { return queue.size(); } - - size_t get_elements_num(std::lock_guard& /* cache_lock */) const { - return queue.size(); - } - - Iterator add(const UInt128Wrapper& hash, size_t offset, size_t size, - std::lock_guard& cache_lock); - template - requires IsXLock - void remove(Iterator queue_it, T& cache_lock); - - void move_to_end(Iterator queue_it, std::lock_guard& cache_lock); - - std::string to_string(std::lock_guard& cache_lock) const; - - bool contains(const UInt128Wrapper& hash, size_t offset, - std::lock_guard& cache_lock) const; - - Iterator begin() { return queue.begin(); } - - Iterator end() { return queue.end(); } - - void remove_all(std::lock_guard& cache_lock); - - Iterator get(const UInt128Wrapper& hash, size_t offset, - std::lock_guard& /* cache_lock */) const; - - int64_t get_hot_data_interval() const { return hot_data_interval; } - - void clear(std::lock_guard& cache_lock) { - queue.clear(); - map.clear(); - cache_size = 0; - } - - size_t levenshtein_distance_from(LRUQueue& base, std::lock_guard& cache_lock); - - size_t max_size; - size_t max_element_size; - std::list queue; - std::unordered_map, Iterator, HashFileKeyAndOffset> map; - size_t cache_size = 0; - int64_t hot_data_interval {0}; -}; - // The BlockFileCache is responsible for the management of the blocks // The current strategies are lru and ttl. class BlockFileCache { @@ -169,10 +84,9 @@ class BlockFileCache { friend class FileBlock; friend struct FileBlocksHolder; friend class CacheLRUDumper; + friend class LRUQueueRecorder; public: - static std::string cache_type_to_string(FileCacheType type); - static FileCacheType string_to_cache_type(const std::string& str); // hash the file_name to uint128 static UInt128Wrapper hash(const std::string& path); @@ -306,25 +220,6 @@ class BlockFileCache { // for be UTs std::map get_stats_unsafe(); - enum class CacheLRULogType { - ADD = 0, // all of the integer types - REMOVE = 1, - MOVETOBACK = 2, - INVALID = 3, - }; - - struct CacheLRULog { - CacheLRULogType type = CacheLRULogType::INVALID; - UInt128Wrapper hash; - size_t offset; - size_t size; - - CacheLRULog(CacheLRULogType t, UInt128Wrapper h, size_t o, size_t s) - : type(t), hash(h), offset(o), size(s) {} - }; - - using CacheLRULogQueue = std::list>; - using AccessRecord = std::unordered_map; @@ -420,7 +315,6 @@ class BlockFileCache { LRUQueue& get_queue(FileCacheType type); const LRUQueue& get_queue(FileCacheType type) const; - CacheLRULogQueue& get_lru_log_queue(FileCacheType type); template requires IsXLock && IsXLock @@ -522,11 +416,6 @@ class BlockFileCache { std::lock_guard& cache_lock, size_t& cur_removed_size, bool evict_in_advance); - void record_queue_event(CacheLRULogQueue& log_queue, CacheLRULogType log_type, - const UInt128Wrapper hash, const size_t offset, const size_t size); - void replay_queue_event(CacheLRULogQueue& log_queue, LRUQueue& shadown_queue); - void evaluate_queue_diff(LRUQueue& target, LRUQueue& base, std::string name); - Status check_ofstream_status(std::ofstream& out, std::string& filename); Status dump_one_lru_entry(std::ofstream& out, std::string& filename, const UInt128Wrapper& hash, size_t offset, size_t size); @@ -579,21 +468,11 @@ class BlockFileCache { LRUQueue _normal_queue; LRUQueue _disposable_queue; LRUQueue _ttl_queue; - LRUQueue _shadow_index_queue; - LRUQueue _shadow_normal_queue; - LRUQueue _shadow_disposable_queue; - LRUQueue _shadow_ttl_queue; // keys for async remove RecycleFileCacheKeys _recycle_keys; - CacheLRULogQueue _ttl_lru_log_queue; - CacheLRULogQueue _index_lru_log_queue; - CacheLRULogQueue _normal_lru_log_queue; - CacheLRULogQueue _disposable_lru_log_queue; - std::mutex _mutex_lru_log; - - std::unordered_map _lru_queue_update_cnt_from_last_dump; + std::unique_ptr _lru_recorder; std::unique_ptr _lru_dumper; // metrics diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index 0894e1ec23fa9c..69c2ce7ea78d6a 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -19,6 +19,7 @@ #include "io/cache/block_file_cache.h" #include "io/cache/cache_lru_dumper.h" +#include "io/cache/lru_queue_recorder.h" #include "util/crc32c.h" namespace doris::io { @@ -229,12 +230,15 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, out.close(); // Rename tmp to formal file - if (std::rename(tmp_filename.c_str(), final_filename.c_str()) != 0) { + try { + std::rename(tmp_filename.c_str(), final_filename.c_str()); std::remove(tmp_filename.c_str()); file_size = std::filesystem::file_size(final_filename); - } else { - LOG(WARNING) << "failed to rename " << tmp_filename << " to " << final_filename; + } catch (const std::filesystem::filesystem_error& e) { + LOG(WARNING) << "failed to rename " << tmp_filename << " to " << final_filename + << " err: " << e.what(); } + _dump_meta.Clear(); _current_dump_group.Clear(); _current_dump_group_count = 0; @@ -242,13 +246,24 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, return Status::OK(); } -void CacheLRUDumper::dump_queue(LRUQueue& queue, const std::string& queue_name) { +void CacheLRUDumper::dump_queue(const std::string& queue_name) { + FileCacheType type = string_to_cache_type(queue_name); + LOG(INFO) << "OOXXOO1" << _recorder->get_lru_queue_update_cnt_from_last_dump(type); + if (_recorder->get_lru_queue_update_cnt_from_last_dump(type) > + config::file_cache_background_lru_dump_update_cnt_threshold) { + LRUQueue& queue = _recorder->get_shadow_queue(type); + do_dump_queue(queue, queue_name); + _recorder->reset_lru_queue_update_cnt_from_last_dump(type); + } +} + +void CacheLRUDumper::do_dump_queue(LRUQueue& queue, const std::string& queue_name) { Status st; std::vector> elements; elements.reserve(config::file_cache_background_lru_dump_tail_record_num); { - std::lock_guard lru_log_lock(_mgr->_mutex_lru_log); + std::lock_guard lru_log_lock(_recorder->_mutex_lru_log); size_t count = 0; for (const auto& [hash, offset, size] : queue) { if (count++ >= config::file_cache_background_lru_dump_tail_record_num) break; @@ -262,12 +277,12 @@ void CacheLRUDumper::dump_queue(LRUQueue& queue, const std::string& queue_name) { SCOPED_RAW_TIMER(&duration_ns); std::string tmp_filename = - fmt::format("{}/lru_dump_{}.bin.tmp", _mgr->_cache_base_path, queue_name); + fmt::format("{}/lru_dump_{}.tail.tmp", _mgr->_cache_base_path, queue_name); std::string final_filename = - fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + fmt::format("{}/lru_dump_{}.tail", _mgr->_cache_base_path, queue_name); std::ofstream out(tmp_filename, std::ios::binary); if (out) { - LOG(INFO) << "begin dump " << queue_name << "with " << elements.size() << " elements"; + LOG(INFO) << "begin dump " << queue_name << " with " << elements.size() << " elements"; for (const auto& [hash, offset, size] : elements) { RETURN_IF_STATUS_ERROR(st, dump_one_lru_entry(out, tmp_filename, hash, offset, size)); @@ -393,7 +408,7 @@ Status CacheLRUDumper::parse_one_lru_entry(std::ifstream& in, std::string& filen void CacheLRUDumper::restore_queue(LRUQueue& queue, const std::string& queue_name, std::lock_guard& cache_lock) { Status st; - std::string filename = fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + std::string filename = fmt::format("{}/lru_dump_{}.tail", _mgr->_cache_base_path, queue_name); std::ifstream in(filename, std::ios::binary); int64_t duration_ns = 0; if (in) { @@ -443,7 +458,7 @@ void CacheLRUDumper::remove_lru_dump_files() { std::vector queue_names = {"disposable", "index", "normal", "ttl"}; for (const auto& queue_name : queue_names) { std::string filename = - fmt::format("{}/lru_dump_{}.bin", _mgr->_cache_base_path, queue_name); + fmt::format("{}/lru_dump_{}.tail", _mgr->_cache_base_path, queue_name); if (std::filesystem::exists(filename)) { std::filesystem::remove(filename); } diff --git a/be/src/io/cache/cache_lru_dumper.h b/be/src/io/cache/cache_lru_dumper.h index b0cb0e29f31a4b..801ed577de29b6 100644 --- a/be/src/io/cache/cache_lru_dumper.h +++ b/be/src/io/cache/cache_lru_dumper.h @@ -33,16 +33,19 @@ namespace doris::io { class LRUQueue; +class LRUQueueRecorder; class CacheLRUDumper { public: - CacheLRUDumper(BlockFileCache* mgr) : _mgr(mgr) {}; - void dump_queue(LRUQueue& queue, const std::string& queue_name); + CacheLRUDumper(BlockFileCache* mgr, LRUQueueRecorder* recorder) + : _mgr(mgr), _recorder(recorder) {}; + void dump_queue(const std::string& queue_name); void restore_queue(LRUQueue& queue, const std::string& queue_name, std::lock_guard& cache_lock); void remove_lru_dump_files(); private: + void do_dump_queue(LRUQueue& queue, const std::string& queue_name); Status check_ofstream_status(std::ofstream& out, std::string& filename); Status check_ifstream_status(std::ifstream& in, std::string& filename); Status dump_one_lru_entry(std::ofstream& out, std::string& filename, const UInt128Wrapper& hash, @@ -61,6 +64,7 @@ class CacheLRUDumper { char magic[3]; std::string serialize_as_string() const; + bool deserialize_from_string(const std::string& data); } __attribute__((packed)); private: @@ -74,5 +78,6 @@ class CacheLRUDumper { doris::io::cache::LRUDumpMetaPb _parse_meta; BlockFileCache* _mgr; + LRUQueueRecorder* _recorder; }; } // namespace doris::io \ No newline at end of file diff --git a/be/src/io/cache/file_block.cpp b/be/src/io/cache/file_block.cpp index 06f58730296fed..731c1d311e6680 100644 --- a/be/src/io/cache/file_block.cpp +++ b/be/src/io/cache/file_block.cpp @@ -172,8 +172,8 @@ Status FileBlock::change_cache_type_between_ttl_and_others(FileCacheType new_typ if (!expr) { LOG(WARNING) << "none of the cache type is TTL" << ", hash: " << _key.hash.to_string() << ", offset: " << _key.offset - << ", new type: " << BlockFileCache::cache_type_to_string(new_type) - << ", old type: " << BlockFileCache::cache_type_to_string(_key.meta.type); + << ", new type: " << cache_type_to_string(new_type) + << ", old type: " << cache_type_to_string(_key.meta.type); } DCHECK(expr); @@ -189,8 +189,8 @@ Status FileBlock::change_cache_type_between_normal_and_index(FileCacheType new_t if (!expr) { LOG(WARNING) << "one of the cache type is TTL" << ", hash: " << _key.hash.to_string() << ", offset: " << _key.offset - << ", new type: " << BlockFileCache::cache_type_to_string(new_type) - << ", old type: " << BlockFileCache::cache_type_to_string(_key.meta.type); + << ", new type: " << cache_type_to_string(new_type) + << ", old type: " << cache_type_to_string(_key.meta.type); } DCHECK(expr); if (_key.meta.type == FileCacheType::TTL || new_type == _key.meta.type) { diff --git a/be/src/io/cache/file_cache_common.cpp b/be/src/io/cache/file_cache_common.cpp index 56525425f758d4..47267a34329998 100644 --- a/be/src/io/cache/file_cache_common.cpp +++ b/be/src/io/cache/file_cache_common.cpp @@ -26,6 +26,59 @@ namespace doris::io { +std::string cache_type_to_surfix(FileCacheType type) { + switch (type) { + case FileCacheType::INDEX: + return "_idx"; + case FileCacheType::DISPOSABLE: + return "_disposable"; + case FileCacheType::NORMAL: + return ""; + case FileCacheType::TTL: + return "_ttl"; + } + return ""; +} + +FileCacheType surfix_to_cache_type(const std::string& str) { + if (str == "idx") { + return FileCacheType::INDEX; + } else if (str == "disposable") { + return FileCacheType::DISPOSABLE; + } else if (str == "ttl") { + return FileCacheType::TTL; + } + DCHECK(false) << "The string is " << str; + return FileCacheType::DISPOSABLE; +} + +FileCacheType string_to_cache_type(const std::string& str) { + if (str == "normal") { + return FileCacheType::NORMAL; + } else if (str == "index") { + return FileCacheType::INDEX; + } else if (str == "disposable") { + return FileCacheType::DISPOSABLE; + } else if (str == "ttl") { + return FileCacheType::TTL; + } + DCHECK(false) << "The string is " << str; + return FileCacheType::NORMAL; +} +std::string cache_type_to_string(FileCacheType type) { + switch (type) { + case FileCacheType::INDEX: + return "index"; + case FileCacheType::DISPOSABLE: + return "disposable"; + case FileCacheType::NORMAL: + return "normal"; + case FileCacheType::TTL: + return "ttl"; + } + DCHECK(false) << "unknown type: " << type; +} + std::string FileCacheSettings::to_string() const { std::stringstream ss; ss << "capacity: " << capacity << ", max_file_block_size: " << max_file_block_size @@ -89,4 +142,7 @@ FileBlocksHolderPtr FileCacheAllocatorBuilder::allocate_cache_holder(size_t offs return std::make_unique(std::move(holder)); } +template size_t LRUQueue::get_capacity(std::lock_guard& cache_lock) const; +template void LRUQueue::remove(Iterator queue_it, std::lock_guard& cache_lock); + } // namespace doris::io diff --git a/be/src/io/cache/file_cache_common.h b/be/src/io/cache/file_cache_common.h index 3cb10dc67a943a..f9ac525d0bef86 100644 --- a/be/src/io/cache/file_cache_common.h +++ b/be/src/io/cache/file_cache_common.h @@ -40,6 +40,12 @@ enum FileCacheType { TTL = 3, }; +std::string cache_type_to_surfix(FileCacheType type); +FileCacheType surfix_to_cache_type(const std::string& str); + +FileCacheType string_to_cache_type(const std::string& str); +std::string cache_type_to_string(FileCacheType type); + struct UInt128Wrapper { uint128_t value_; [[nodiscard]] std::string to_string() const; @@ -155,4 +161,94 @@ struct CacheContext { ReadStatistics* stats; }; +template +concept IsXLock = std::same_as> || + std::same_as>; + +class LRUQueue { +public: + LRUQueue() = default; + LRUQueue(size_t max_size, size_t max_element_size, int64_t hot_data_interval) + : max_size(max_size), + max_element_size(max_element_size), + hot_data_interval(hot_data_interval) {} + + struct HashFileKeyAndOffset { + std::size_t operator()(const std::pair& pair) const { + return KeyHash()(pair.first) + pair.second; + } + }; + + struct FileKeyAndOffset { + UInt128Wrapper hash; + size_t offset; + size_t size; + + FileKeyAndOffset(const UInt128Wrapper& hash, size_t offset, size_t size) + : hash(hash), offset(offset), size(size) {} + }; + + using Iterator = typename std::list::iterator; + + size_t get_max_size() const { return max_size; } + size_t get_max_element_size() const { return max_element_size; } + + template + requires IsXLock + size_t get_capacity(T& /* cache_lock */) const { + return cache_size; + } + + size_t get_capacity_unsafe() const { return cache_size; } + + size_t get_elements_num_unsafe() const { return queue.size(); } + + size_t get_elements_num(std::lock_guard& /* cache_lock */) const { + return queue.size(); + } + + Iterator add(const UInt128Wrapper& hash, size_t offset, size_t size, + std::lock_guard& cache_lock); + template + requires IsXLock + void remove(Iterator queue_it, T& /* cache_lock */) { + cache_size -= queue_it->size; + map.erase(std::make_pair(queue_it->hash, queue_it->offset)); + queue.erase(queue_it); + } + + void move_to_end(Iterator queue_it, std::lock_guard& cache_lock); + + std::string to_string(std::lock_guard& cache_lock) const; + + bool contains(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& cache_lock) const; + + Iterator begin() { return queue.begin(); } + + Iterator end() { return queue.end(); } + + void remove_all(std::lock_guard& cache_lock); + + Iterator get(const UInt128Wrapper& hash, size_t offset, + std::lock_guard& /* cache_lock */) const; + + int64_t get_hot_data_interval() const { return hot_data_interval; } + + void clear(std::lock_guard& cache_lock) { + queue.clear(); + map.clear(); + cache_size = 0; + } + + size_t levenshtein_distance_from(LRUQueue& base, std::lock_guard& cache_lock); + + size_t max_size; + size_t max_element_size; + std::list queue; + std::unordered_map, Iterator, HashFileKeyAndOffset> map; + size_t cache_size = 0; + int64_t hot_data_interval {0}; +}; + } // namespace doris::io diff --git a/be/src/io/cache/fs_file_cache_storage.cpp b/be/src/io/cache/fs_file_cache_storage.cpp index 7b0eef1e35e111..bb24b476501d27 100644 --- a/be/src/io/cache/fs_file_cache_storage.cpp +++ b/be/src/io/cache/fs_file_cache_storage.cpp @@ -254,8 +254,8 @@ Status FSFileCacheStorage::change_key_meta_type(const FileCacheKey& key, const F if (!expr) { LOG(WARNING) << "TTL type file dose not need to change the suffix" << " key=" << key.hash.to_string() << " offset=" << key.offset - << " old_type=" << BlockFileCache::cache_type_to_string(key.meta.type) - << " new_type=" << BlockFileCache::cache_type_to_string(type); + << " old_type=" << cache_type_to_string(key.meta.type) + << " new_type=" << cache_type_to_string(type); } DCHECK(expr); std::string dir = get_path_in_local_cache(key.hash, key.meta.expiration_time); @@ -288,7 +288,7 @@ std::string FSFileCacheStorage::get_path_in_local_cache(const std::string& dir, } else if (type == FileCacheType::TTL) { return Path(dir) / std::to_string(offset); } else { - return Path(dir) / (std::to_string(offset) + BlockFileCache::cache_type_to_string(type)); + return Path(dir) / (std::to_string(offset) + cache_type_to_surfix(type)); } } @@ -297,7 +297,7 @@ std::string FSFileCacheStorage::get_path_in_local_cache_old_ttl_format(const std FileCacheType type, bool is_tmp) { DCHECK(type == FileCacheType::TTL); - return Path(dir) / (std::to_string(offset) + BlockFileCache::cache_type_to_string(type)); + return Path(dir) / (std::to_string(offset) + cache_type_to_surfix(type)); } std::vector FSFileCacheStorage::get_path_in_local_cache_all_candidates( @@ -606,7 +606,7 @@ Status FSFileCacheStorage::parse_filename_suffix_to_cache_type( if (suffix == "tmp") [[unlikely]] { *is_tmp = true; } else { - *cache_type = BlockFileCache::string_to_cache_type(suffix); + *cache_type = surfix_to_cache_type(suffix); } } } catch (...) { diff --git a/be/src/io/cache/lru_queue_recorder.cpp b/be/src/io/cache/lru_queue_recorder.cpp new file mode 100644 index 00000000000000..12da29d42b7f1a --- /dev/null +++ b/be/src/io/cache/lru_queue_recorder.cpp @@ -0,0 +1,132 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "io/cache/lru_queue_recorder.h" + +#include "io/cache/block_file_cache.h" +#include "io/cache/file_cache_common.h" + +namespace doris::io { + +void LRUQueueRecorder::record_queue_event(FileCacheType type, CacheLRULogType log_type, + const UInt128Wrapper hash, const size_t offset, + const size_t size) { + CacheLRULogQueue& log_queue = get_lru_log_queue(type); + log_queue.push_back(std::make_unique(log_type, hash, offset, size)); + ++(_lru_queue_update_cnt_from_last_dump[type]); +} + +void LRUQueueRecorder::replay_queue_event(FileCacheType type) { + // we don't need the real cache lock for the shadow queue, but we do need a lock to prevent read/write contension + CacheLRULogQueue& log_queue = get_lru_log_queue(type); + LRUQueue& shadow_queue = get_shadow_queue(type); + + std::lock_guard lru_log_lock(_mutex_lru_log); + while (!log_queue.empty()) { + auto log = std::move(log_queue.front()); + log_queue.pop_front(); + try { + switch (log->type) { + case CacheLRULogType::ADD: { + shadow_queue.add(log->hash, log->offset, log->size, lru_log_lock); + break; + } + case CacheLRULogType::REMOVE: { + auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); + if (it != shadow_queue.end()) { + shadow_queue.remove(it, lru_log_lock); + } else { + LOG(WARNING) << "REMOVE failed, doesn't exist in shadow queue"; + } + break; + } + case CacheLRULogType::MOVETOBACK: { + auto it = shadow_queue.get(log->hash, log->offset, lru_log_lock); + if (it != shadow_queue.end()) { + shadow_queue.move_to_end(it, lru_log_lock); + } else { + LOG(WARNING) << "MOVETOBACK failed, doesn't exist in shadow queue"; + } + break; + } + default: + LOG(WARNING) << "Unknown CacheLRULogType: " << static_cast(log->type); + break; + } + } catch (const std::exception& e) { + LOG(WARNING) << "Failed to replay queue event: " << e.what(); + } + } +} + +// we evaluate the diff between two queue by calculate how many operation is +// needed for transfer one to another (Levenshtein Distance) +// NOTE: HEAVY calculation with cache lock, only for debugging +void LRUQueueRecorder::evaluate_queue_diff(LRUQueue& base, std::string name, + std::lock_guard& cache_lock) { + FileCacheType type = string_to_cache_type(name); + LRUQueue& target = get_shadow_queue(type); + size_t distance = target.levenshtein_distance_from(base, cache_lock); + *(_mgr->_shadow_queue_levenshtein_distance) << distance; + if (distance > 20) { + LOG(WARNING) << name << " shadow queue is different from real queue"; + } +} + +LRUQueue& LRUQueueRecorder::get_shadow_queue(FileCacheType type) { + switch (type) { + case FileCacheType::INDEX: + return _shadow_index_queue; + case FileCacheType::DISPOSABLE: + return _shadow_disposable_queue; + case FileCacheType::NORMAL: + return _shadow_normal_queue; + case FileCacheType::TTL: + return _shadow_ttl_queue; + default: + LOG(WARNING) << "invalid shadow queue type"; + DCHECK(false); + } + return _shadow_normal_queue; +} + +CacheLRULogQueue& LRUQueueRecorder::get_lru_log_queue(FileCacheType type) { + switch (type) { + case FileCacheType::INDEX: + return _index_lru_log_queue; + case FileCacheType::DISPOSABLE: + return _disposable_lru_log_queue; + case FileCacheType::NORMAL: + return _normal_lru_log_queue; + case FileCacheType::TTL: + return _ttl_lru_log_queue; + default: + LOG(WARNING) << "invalid lru log queue type"; + DCHECK(false); + } + return _normal_lru_log_queue; +} + +size_t LRUQueueRecorder::get_lru_queue_update_cnt_from_last_dump(FileCacheType type) { + return _lru_queue_update_cnt_from_last_dump[type]; +} + +void LRUQueueRecorder::reset_lru_queue_update_cnt_from_last_dump(FileCacheType type) { + _lru_queue_update_cnt_from_last_dump[type] = 0; +} + +} // end of namespace doris::io diff --git a/be/src/io/cache/lru_queue_recorder.h b/be/src/io/cache/lru_queue_recorder.h new file mode 100644 index 00000000000000..dceef7a493ce27 --- /dev/null +++ b/be/src/io/cache/lru_queue_recorder.h @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "io/cache/file_cache_common.h" + +namespace doris::io { + +class LRUQueue; + +enum class CacheLRULogType { + ADD = 0, // all of the integer types + REMOVE = 1, + MOVETOBACK = 2, + INVALID = 3, +}; + +struct CacheLRULog { + CacheLRULogType type = CacheLRULogType::INVALID; + UInt128Wrapper hash; + size_t offset; + size_t size; + + CacheLRULog(CacheLRULogType t, UInt128Wrapper h, size_t o, size_t s) + : type(t), hash(h), offset(o), size(s) {} +}; + +using CacheLRULogQueue = std::list>; + +class LRUQueueRecorder { +public: + LRUQueueRecorder(BlockFileCache* mgr) : _mgr(mgr) { + _lru_queue_update_cnt_from_last_dump[FileCacheType::DISPOSABLE] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::NORMAL] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::INDEX] = 0; + _lru_queue_update_cnt_from_last_dump[FileCacheType::TTL] = 0; + } + void record_queue_event(FileCacheType type, CacheLRULogType log_type, const UInt128Wrapper hash, + const size_t offset, const size_t size); + void replay_queue_event(FileCacheType type); + void evaluate_queue_diff(LRUQueue& base, std::string name, + std::lock_guard& cache_lock); + size_t get_lru_queue_update_cnt_from_last_dump(FileCacheType type); + void reset_lru_queue_update_cnt_from_last_dump(FileCacheType type); + + CacheLRULogQueue& get_lru_log_queue(FileCacheType type); + LRUQueue& get_shadow_queue(FileCacheType type); + +public: + std::mutex _mutex_lru_log; + +private: + LRUQueue _shadow_index_queue; + LRUQueue _shadow_normal_queue; + LRUQueue _shadow_disposable_queue; + LRUQueue _shadow_ttl_queue; + + CacheLRULogQueue _ttl_lru_log_queue; + CacheLRULogQueue _index_lru_log_queue; + CacheLRULogQueue _normal_lru_log_queue; + CacheLRULogQueue _disposable_lru_log_queue; + + std::unordered_map _lru_queue_update_cnt_from_last_dump; + + BlockFileCache* _mgr; +}; + +} // namespace doris::io diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 1a7d6df0961410..bb6dac00e30661 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -166,6 +166,8 @@ class ExecEnv { static bool ready() { return _s_ready.load(std::memory_order_acquire); } static bool tracking_memory() { return _s_tracking_memory.load(std::memory_order_acquire); } + static bool get_is_upgrading() { return _s_upgrading.load(std::memory_order_acquire); } + static void set_is_upgrading() { _s_upgrading = true; } const std::string& token() const; ExternalScanContextMgr* external_scan_context_mgr() { return _external_scan_context_mgr; } vectorized::VDataStreamMgr* vstream_mgr() { return _vstream_mgr; } @@ -390,6 +392,7 @@ class ExecEnv { inline static std::atomic_bool _s_tracking_memory {false}; std::vector _store_paths; std::vector _spill_store_paths; + inline static std::atomic_bool _s_upgrading {false}; io::FileCacheFactory* _file_cache_factory = nullptr; UserFunctionCache* _user_function_cache = nullptr; diff --git a/be/test/io/cache/block_file_cache_test.cpp b/be/test/io/cache/block_file_cache_test.cpp index b39457d4115b2b..2a6f69e389b1b6 100644 --- a/be/test/io/cache/block_file_cache_test.cpp +++ b/be/test/io/cache/block_file_cache_test.cpp @@ -3400,10 +3400,10 @@ TEST_F(BlockFileCacheTest, state_to_string) { EXPECT_EQ(FileBlock::state_to_string(FileBlock::State::DOWNLOADED), "DOWNLOADED"); } -TEST_F(BlockFileCacheTest, string_to_cache_type) { - EXPECT_EQ(BlockFileCache::string_to_cache_type("idx"), FileCacheType::INDEX); - EXPECT_EQ(BlockFileCache::string_to_cache_type("disposable"), FileCacheType::DISPOSABLE); - EXPECT_EQ(BlockFileCache::string_to_cache_type("ttl"), FileCacheType::TTL); +TEST_F(BlockFileCacheTest, surfix_to_cache_type) { + EXPECT_EQ(surfix_to_cache_type("idx"), FileCacheType::INDEX); + EXPECT_EQ(surfix_to_cache_type("disposable"), FileCacheType::DISPOSABLE); + EXPECT_EQ(surfix_to_cache_type("ttl"), FileCacheType::TTL); } TEST_F(BlockFileCacheTest, append_many_time) { @@ -5900,9 +5900,8 @@ TEST_F(BlockFileCacheTest, seize_after_full) { }; for (auto& args : args_vec) { - std::cout << "filled with " << io::BlockFileCache::cache_type_to_string(args.first_type) - << " and seize with " - << io::BlockFileCache::cache_type_to_string(args.second_type) << std::endl; + std::cout << "filled with " << io::cache_type_to_string(args.first_type) + << " and seize with " << io::cache_type_to_string(args.second_type) << std::endl; if (fs::exists(cache_base_path)) { fs::remove_all(cache_base_path); } diff --git a/be/test/io/cache/block_file_cache_test_lru_dump.cpp b/be/test/io/cache/block_file_cache_test_lru_dump.cpp index f9661d1495789e..ea3cb63601e8ad 100644 --- a/be/test/io/cache/block_file_cache_test_lru_dump.cpp +++ b/be/test/io/cache/block_file_cache_test_lru_dump.cpp @@ -26,7 +26,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { config::enable_evict_file_cache_in_advance = false; config::file_cache_enter_disk_resource_limit_mode_percent = 99; config::file_cache_background_lru_dump_interval_ms = 3000; - config::file_cache_background_lru_dump_update_cnt_threshold = 1; + config::file_cache_background_lru_dump_update_cnt_threshold = 0; if (fs::exists(cache_base_path)) { fs::remove_all(cache_base_path); } @@ -55,7 +55,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { if (cache.get_async_open_success()) { break; } - std::this_thread::sleep_for(std::chrono::milliseconds(1)); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); } ASSERT_TRUE(cache.get_async_open_success()); @@ -156,18 +156,18 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { ASSERT_EQ(cache.get_stats_unsafe()["normal_queue_curr_size"], 500000); // all queue are filled, let's check the lru log records - ASSERT_EQ(cache._ttl_lru_log_queue.size(), 5); - ASSERT_EQ(cache._index_lru_log_queue.size(), 5); - ASSERT_EQ(cache._normal_lru_log_queue.size(), 5); - ASSERT_EQ(cache._disposable_lru_log_queue.size(), 5); + ASSERT_EQ(cache._lru_recorder->_ttl_lru_log_queue.size(), 5); + ASSERT_EQ(cache._lru_recorder->_index_lru_log_queue.size(), 5); + ASSERT_EQ(cache._lru_recorder->_normal_lru_log_queue.size(), 5); + ASSERT_EQ(cache._lru_recorder->_disposable_lru_log_queue.size(), 5); // then check the log replay std::this_thread::sleep_for(std::chrono::milliseconds( 2 * config::file_cache_background_lru_log_replay_interval_ms)); - ASSERT_EQ(cache._shadow_ttl_queue.get_elements_num_unsafe(), 5); - ASSERT_EQ(cache._shadow_index_queue.get_elements_num_unsafe(), 5); - ASSERT_EQ(cache._shadow_normal_queue.get_elements_num_unsafe(), 5); - ASSERT_EQ(cache._shadow_disposable_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_ttl_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_index_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_normal_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_disposable_queue.get_elements_num_unsafe(), 5); // ok, let do some MOVETOBACK & REMOVE { @@ -175,21 +175,22 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { context2); // move index queue 3rd element to the end cache.remove_if_cached(key3); // remove all element from ttl queue } - ASSERT_EQ(cache._ttl_lru_log_queue.size(), 5); - ASSERT_EQ(cache._index_lru_log_queue.size(), 1); - ASSERT_EQ(cache._normal_lru_log_queue.size(), 0); - ASSERT_EQ(cache._disposable_lru_log_queue.size(), 0); + ASSERT_EQ(cache._lru_recorder->_ttl_lru_log_queue.size(), 5); + ASSERT_EQ(cache._lru_recorder->_index_lru_log_queue.size(), 1); + ASSERT_EQ(cache._lru_recorder->_normal_lru_log_queue.size(), 0); + ASSERT_EQ(cache._lru_recorder->_disposable_lru_log_queue.size(), 0); std::this_thread::sleep_for(std::chrono::milliseconds( 2 * config::file_cache_background_lru_log_replay_interval_ms)); - ASSERT_EQ(cache._shadow_ttl_queue.get_elements_num_unsafe(), 0); - ASSERT_EQ(cache._shadow_index_queue.get_elements_num_unsafe(), 5); - ASSERT_EQ(cache._shadow_normal_queue.get_elements_num_unsafe(), 5); - ASSERT_EQ(cache._shadow_disposable_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_ttl_queue.get_elements_num_unsafe(), 0); + ASSERT_EQ(cache._lru_recorder->_shadow_index_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_normal_queue.get_elements_num_unsafe(), 5); + ASSERT_EQ(cache._lru_recorder->_shadow_disposable_queue.get_elements_num_unsafe(), 5); // check the order std::vector offsets; - for (auto it = cache._shadow_index_queue.begin(); it != cache._shadow_index_queue.end(); ++it) { + for (auto it = cache._lru_recorder->_shadow_index_queue.begin(); + it != cache._lru_recorder->_shadow_index_queue.end(); ++it) { offsets.push_back(it->offset); } ASSERT_EQ(offsets.size(), 5); @@ -206,7 +207,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { // Verify all 4 dump files // TODO(zhengyu): abstract those read/write into a function { - std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "ttl"); + std::string filename = fmt::format("{}/lru_dump_{}.tail", cache_base_path, "ttl"); struct stat file_stat; EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; @@ -227,7 +228,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { } { - std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "normal"); + std::string filename = fmt::format("{}/lru_dump_{}.tail", cache_base_path, "normal"); struct stat file_stat; EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; @@ -288,7 +289,7 @@ TEST_F(BlockFileCacheTest, test_lru_log_record_replay_dump_restore) { } { - std::string filename = fmt::format("{}/lru_dump_{}.bin", cache_base_path, "index"); + std::string filename = fmt::format("{}/lru_dump_{}.tail", cache_base_path, "index"); struct stat file_stat; EXPECT_EQ(stat(filename.c_str(), &file_stat), 0) << "File " << filename << " not found"; diff --git a/be/test/io/cache/cache_lru_dumper_test.cpp b/be/test/io/cache/cache_lru_dumper_test.cpp index 070700347f2767..fd4260b012b2ab 100644 --- a/be/test/io/cache/cache_lru_dumper_test.cpp +++ b/be/test/io/cache/cache_lru_dumper_test.cpp @@ -60,7 +60,9 @@ class CacheLRUDumperTest : public ::testing::Test { void SetUp() override { mock_cache = std::make_unique>(&dst_queue); - dumper = std::make_unique(mock_cache.get()); + recorder = std::make_unique(mock_cache.get()); + + dumper = std::make_unique(mock_cache.get(), recorder.get()); } void TearDown() override { @@ -70,6 +72,7 @@ class CacheLRUDumperTest : public ::testing::Test { std::unique_ptr> mock_cache; std::unique_ptr dumper; + std::unique_ptr recorder; }; TEST_F(CacheLRUDumperTest, test_finalize_dump_and_parse_dump_footer) { @@ -98,7 +101,7 @@ TEST_F(CacheLRUDumperTest, test_remove_lru_dump_files) { // Create test files std::vector queue_names = {"disposable", "index", "normal", "ttl"}; for (const auto& name : queue_names) { - std::ofstream(fmt::format("lru_dump_{}.bin", name)); + std::ofstream(fmt::format("lru_dump_{}.tail", name)); } // Test remove @@ -106,7 +109,7 @@ TEST_F(CacheLRUDumperTest, test_remove_lru_dump_files) { // Verify files are removed for (const auto& name : queue_names) { - EXPECT_FALSE(std::filesystem::exists(fmt::format("lru_dump_{}.bin", name))); + EXPECT_FALSE(std::filesystem::exists(fmt::format("lru_dump_{}.tail", name))); } } @@ -122,7 +125,7 @@ TEST_F(CacheLRUDumperTest, test_dump_and_restore_queue) { src_queue.add(hash, offset, size, lock); // Test dump - dumper->dump_queue(src_queue, queue_name); + dumper->do_dump_queue(src_queue, queue_name); // Test restore std::lock_guard cache_lock(mock_cache->mutex()); @@ -140,7 +143,7 @@ TEST_F(CacheLRUDumperTest, test_dump_and_restore_queue) { } // Clean up - std::remove(fmt::format("lru_dump_{}.bin", queue_name).c_str()); + std::remove(fmt::format("lru_dump_{}.tail", queue_name).c_str()); } } // namespace doris::io \ No newline at end of file diff --git a/regression-test/suites/demo_p0/test_lru_persist.groovy b/regression-test/suites/demo_p0/test_lru_persist.groovy index 662339a10758fe..5f6c8b0e90d47c 100644 --- a/regression-test/suites/demo_p0/test_lru_persist.groovy +++ b/regression-test/suites/demo_p0/test_lru_persist.groovy @@ -55,7 +55,7 @@ suite('test_lru_persist', 'docker') { options.beConfigs += ['enable_file_cache=true', 'enable_java_support=false', 'file_cache_enter_disk_resource_limit_mode_percent=99', 'file_cache_background_lru_dump_interval_ms=2000', 'file_cache_background_lru_log_replay_interval_ms=500', 'disable_auto_compation=true', 'file_cache_enter_need_evict_cache_in_advance_percent=99', - 'file_cache_background_lru_dump_update_cnt_threshold=1' + 'file_cache_background_lru_dump_update_cnt_threshold=0' ] // run another docker @@ -74,33 +74,33 @@ suite('test_lru_persist', 'docker') { def beBasePath = be.getBasePath() def cachePath = beBasePath + "/storage/file_cache/" - sleep(5000); + sleep(10000); cluster.stopBackends(1) // check md5sum of each index - def disposableBefore = "md5sum ${cachePath}/lru_dump_disposable.bin".execute().text.trim().split()[0] + def disposableBefore = "md5sum ${cachePath}/lru_dump_disposable.tail".execute().text.trim().split()[0] logger.info("disposableBefore: ${disposableBefore}") - def indexBefore = "md5sum ${cachePath}/lru_dump_index.bin".execute().text.trim().split()[0] + def indexBefore = "md5sum ${cachePath}/lru_dump_index.tail".execute().text.trim().split()[0] logger.info("indexBefore: ${indexBefore}") - def normalBefore = "md5sum ${cachePath}/lru_dump_normal.bin".execute().text.trim().split()[0] + def normalBefore = "md5sum ${cachePath}/lru_dump_normal.tail".execute().text.trim().split()[0] logger.info("normalBefore: ${normalBefore}") - def ttlBefore = "md5sum ${cachePath}/lru_dump_ttl.bin".execute().text.trim().split()[0] + def ttlBefore = "md5sum ${cachePath}/lru_dump_ttl.tail".execute().text.trim().split()[0] logger.info("ttlBefore: ${ttlBefore}") cluster.startBackends(1) - sleep(5000); + sleep(10000); cluster.stopBackends(1) // check md5sum again after be restart - def disposableAfter = "md5sum ${cachePath}/lru_dump_disposable.bin".execute().text.trim().split()[0] + def disposableAfter = "md5sum ${cachePath}/lru_dump_disposable.tail".execute().text.trim().split()[0] logger.info("disposableAfter: ${disposableAfter}") - def indexAfter = "md5sum ${cachePath}/lru_dump_index.bin".execute().text.trim().split()[0] + def indexAfter = "md5sum ${cachePath}/lru_dump_index.tail".execute().text.trim().split()[0] logger.info("indexAfter: ${indexAfter}") - def normalAfter = "md5sum ${cachePath}/lru_dump_normal.bin".execute().text.trim().split()[0] + def normalAfter = "md5sum ${cachePath}/lru_dump_normal.tail".execute().text.trim().split()[0] logger.info("normalAfter: ${normalAfter}") - def ttlAfter = "md5sum ${cachePath}/lru_dump_ttl.bin".execute().text.trim().split()[0] + def ttlAfter = "md5sum ${cachePath}/lru_dump_ttl.tail".execute().text.trim().split()[0] logger.info("ttlAfter: ${ttlAfter}") assert disposableBefore == disposableAfter From 86ff116f0a3d2ac8166d11a139ebdbce35a9a28e Mon Sep 17 00:00:00 2001 From: zhengyu Date: Fri, 4 Jul 2025 08:32:00 +0800 Subject: [PATCH 11/12] fix regression Signed-off-by: zhengyu --- be/src/io/cache/cache_lru_dumper.cpp | 1 - be/src/io/cache/file_cache_common.cpp | 1 + .../suites/demo_p0/test_lru_persist.groovy | 17 ----------------- 3 files changed, 1 insertion(+), 18 deletions(-) diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index 69c2ce7ea78d6a..de267220d9f00d 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -248,7 +248,6 @@ Status CacheLRUDumper::finalize_dump(std::ofstream& out, size_t entry_num, void CacheLRUDumper::dump_queue(const std::string& queue_name) { FileCacheType type = string_to_cache_type(queue_name); - LOG(INFO) << "OOXXOO1" << _recorder->get_lru_queue_update_cnt_from_last_dump(type); if (_recorder->get_lru_queue_update_cnt_from_last_dump(type) > config::file_cache_background_lru_dump_update_cnt_threshold) { LRUQueue& queue = _recorder->get_shadow_queue(type); diff --git a/be/src/io/cache/file_cache_common.cpp b/be/src/io/cache/file_cache_common.cpp index 47267a34329998..64faec3beb3b3e 100644 --- a/be/src/io/cache/file_cache_common.cpp +++ b/be/src/io/cache/file_cache_common.cpp @@ -77,6 +77,7 @@ std::string cache_type_to_string(FileCacheType type) { return "ttl"; } DCHECK(false) << "unknown type: " << type; + return "normal"; } std::string FileCacheSettings::to_string() const { diff --git a/regression-test/suites/demo_p0/test_lru_persist.groovy b/regression-test/suites/demo_p0/test_lru_persist.groovy index 5f6c8b0e90d47c..4771c4c15e33af 100644 --- a/regression-test/suites/demo_p0/test_lru_persist.groovy +++ b/regression-test/suites/demo_p0/test_lru_persist.groovy @@ -77,16 +77,8 @@ suite('test_lru_persist', 'docker') { sleep(10000); cluster.stopBackends(1) - // check md5sum of each index - - def disposableBefore = "md5sum ${cachePath}/lru_dump_disposable.tail".execute().text.trim().split()[0] - logger.info("disposableBefore: ${disposableBefore}") - def indexBefore = "md5sum ${cachePath}/lru_dump_index.tail".execute().text.trim().split()[0] - logger.info("indexBefore: ${indexBefore}") def normalBefore = "md5sum ${cachePath}/lru_dump_normal.tail".execute().text.trim().split()[0] logger.info("normalBefore: ${normalBefore}") - def ttlBefore = "md5sum ${cachePath}/lru_dump_ttl.tail".execute().text.trim().split()[0] - logger.info("ttlBefore: ${ttlBefore}") cluster.startBackends(1) sleep(10000); @@ -94,18 +86,9 @@ suite('test_lru_persist', 'docker') { cluster.stopBackends(1) // check md5sum again after be restart - def disposableAfter = "md5sum ${cachePath}/lru_dump_disposable.tail".execute().text.trim().split()[0] - logger.info("disposableAfter: ${disposableAfter}") - def indexAfter = "md5sum ${cachePath}/lru_dump_index.tail".execute().text.trim().split()[0] - logger.info("indexAfter: ${indexAfter}") def normalAfter = "md5sum ${cachePath}/lru_dump_normal.tail".execute().text.trim().split()[0] logger.info("normalAfter: ${normalAfter}") - def ttlAfter = "md5sum ${cachePath}/lru_dump_ttl.tail".execute().text.trim().split()[0] - logger.info("ttlAfter: ${ttlAfter}") - assert disposableBefore == disposableAfter - assert indexBefore == indexAfter assert normalBefore == normalAfter - assert ttlBefore == ttlAfter } } From c3414755f24b5dae18e02e985653d683cf4347c0 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Fri, 4 Jul 2025 08:46:33 +0800 Subject: [PATCH 12/12] footer endian fix Signed-off-by: zhengyu --- be/src/io/cache/cache_lru_dumper.cpp | 40 +++++++++---------- be/test/io/cache/block_file_cache_test.cpp | 4 +- .../suites/demo_p0/test_lru_persist.groovy | 2 +- 3 files changed, 22 insertions(+), 24 deletions(-) diff --git a/be/src/io/cache/cache_lru_dumper.cpp b/be/src/io/cache/cache_lru_dumper.cpp index de267220d9f00d..78632eca754fd7 100644 --- a/be/src/io/cache/cache_lru_dumper.cpp +++ b/be/src/io/cache/cache_lru_dumper.cpp @@ -28,17 +28,15 @@ std::string CacheLRUDumper::Footer::serialize_as_string() const { std::string result; result.reserve(sizeof(Footer)); - // Serialize meta_offset - const char* meta_offset_ptr = reinterpret_cast(&meta_offset); - result.append(meta_offset_ptr, sizeof(meta_offset)); + // Serialize meta_offset (convert to little-endian) + uint64_t meta_offset_le = htole64(meta_offset); + result.append(reinterpret_cast(&meta_offset_le), sizeof(meta_offset_le)); - // Serialize checksum - const char* checksum_ptr = reinterpret_cast(&checksum); - result.append(checksum_ptr, sizeof(checksum)); + // Serialize checksum (convert to little-endian) + uint32_t checksum_le = htole32(checksum); + result.append(reinterpret_cast(&checksum_le), sizeof(checksum_le)); - // Serialize version - const char* version_ptr = reinterpret_cast(&version); - result.append(version_ptr, sizeof(version)); + result.append(reinterpret_cast(&version), sizeof(version)); // Serialize magic result.append(magic, sizeof(magic)); @@ -47,23 +45,23 @@ std::string CacheLRUDumper::Footer::serialize_as_string() const { } bool CacheLRUDumper::Footer::deserialize_from_string(const std::string& data) { - if (data.size() != sizeof(Footer)) { - return false; - } + DCHECK(data.size() == sizeof(Footer)); const char* ptr = data.data(); - // Deserialize meta_offset - std::memcpy(&meta_offset, ptr, sizeof(meta_offset)); - meta_offset = le64toh(meta_offset); // Convert from little-endian - ptr += sizeof(meta_offset); + // Deserialize meta_offset (convert from little-endian) + uint64_t meta_offset_le; + std::memcpy(&meta_offset_le, ptr, sizeof(meta_offset_le)); + meta_offset = le64toh(meta_offset_le); + ptr += sizeof(meta_offset_le); - // Deserialize checksum - std::memcpy(&checksum, ptr, sizeof(checksum)); - ptr += sizeof(checksum); + // Deserialize checksum (convert from little-endian) + uint32_t checksum_le; + std::memcpy(&checksum_le, ptr, sizeof(checksum_le)); + checksum = le32toh(checksum_le); + ptr += sizeof(checksum_le); - // Deserialize version - std::memcpy(&version, ptr, sizeof(version)); + version = *((uint8_t*)ptr); ptr += sizeof(version); // Deserialize magic diff --git a/be/test/io/cache/block_file_cache_test.cpp b/be/test/io/cache/block_file_cache_test.cpp index 2a6f69e389b1b6..fd00f1529aa4d1 100644 --- a/be/test/io/cache/block_file_cache_test.cpp +++ b/be/test/io/cache/block_file_cache_test.cpp @@ -5254,7 +5254,7 @@ TEST_F(BlockFileCacheTest, test_load) { { auto type = cache.dump_single_cache_type(key, 10086); - ASSERT_TRUE(type == "_ttl"); + ASSERT_TRUE(type == "ttl"); auto holder = cache.get_or_set(key, 10086, 3, context); auto blocks = fromHolder(holder); ASSERT_EQ(blocks.size(), 1); @@ -5275,7 +5275,7 @@ TEST_F(BlockFileCacheTest, test_load) { } { auto type = cache.dump_single_cache_type(key, 20086); - ASSERT_TRUE(type == "_ttl"); + ASSERT_TRUE(type == "ttl"); auto holder = cache.get_or_set(key, 20086, 3, context); auto blocks = fromHolder(holder); ASSERT_EQ(blocks.size(), 1); diff --git a/regression-test/suites/demo_p0/test_lru_persist.groovy b/regression-test/suites/demo_p0/test_lru_persist.groovy index 4771c4c15e33af..249faadeedad95 100644 --- a/regression-test/suites/demo_p0/test_lru_persist.groovy +++ b/regression-test/suites/demo_p0/test_lru_persist.groovy @@ -74,7 +74,7 @@ suite('test_lru_persist', 'docker') { def beBasePath = be.getBasePath() def cachePath = beBasePath + "/storage/file_cache/" - sleep(10000); + sleep(15000); cluster.stopBackends(1) def normalBefore = "md5sum ${cachePath}/lru_dump_normal.tail".execute().text.trim().split()[0]