From 25f8e8467a350bb01d7ecd88dd339fcee78788c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 4 Jan 2023 18:56:45 +0100 Subject: [PATCH] Rewrite locking in cache - intermediate state --- src/Common/ProfileEvents.cpp | 48 + .../IO/CachedOnDiskReadBufferFromFile.cpp | 727 ++++---- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 25 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 34 +- .../IO/CachedOnDiskWriteBufferFromFile.h | 4 +- .../IO/ElapsedTimeProfileEventIncrement.h | 41 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 5 +- .../Cached/CachedObjectStorage.cpp | 11 +- .../Cached/CachedObjectStorage.h | 2 + ...ynchronousReadBufferFromFileDescriptor.cpp | 2 + src/IO/ReadBufferFromFile.cpp | 22 + src/IO/ReadBufferFromFile.h | 10 +- src/Interpreters/Cache/FileCache.cpp | 1462 +++++++++-------- src/Interpreters/Cache/FileCache.h | 280 ++-- src/Interpreters/Cache/FileCacheKey.cpp | 27 + src/Interpreters/Cache/FileCacheKey.h | 19 +- src/Interpreters/Cache/FileSegment.cpp | 496 +++--- src/Interpreters/Cache/FileSegment.h | 115 +- src/Interpreters/Cache/Guards.h | 79 + src/Interpreters/Cache/IFileCachePriority.h | 67 +- .../Cache/LRUFileCachePriority.cpp | 48 +- src/Interpreters/Cache/LRUFileCachePriority.h | 34 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- .../tests/gtest_lru_file_cache.cpp | 72 +- src/Interpreters/threadPoolCallbackRunner.h | 19 +- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 2 +- 28 files changed, 2120 insertions(+), 1539 deletions(-) create mode 100644 src/Disks/IO/ElapsedTimeProfileEventIncrement.h create mode 100644 src/Interpreters/Cache/FileCacheKey.cpp create mode 100644 src/Interpreters/Cache/Guards.h diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 25ba56fa046..12ca4fa0e5e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -341,10 +341,45 @@ The server successfully detected this situation and will download merged part fr M(CachedReadBufferReadFromCacheMicroseconds, "Time reading from filesystem cache") \ M(CachedReadBufferReadFromSourceBytes, "Bytes read from filesystem cache source (from remote fs, etc)") \ M(CachedReadBufferReadFromCacheBytes, "Bytes read from filesystem cache") \ + M(CachedReadBufferGetFileSegments, "Bytes read from filesystem cache") \ + M(CachedReadBufferGetCacheReadBuffer, "Bytes read from filesystem cache") \ + M(CachedReadBufferGetRemoteReadBuffer, "Bytes read from filesystem cache") \ + M(CachedReadBufferPredownload, "Bytes read from filesystem cache") \ M(CachedReadBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedReadBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ M(CachedWriteBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedWriteBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ + M(CachedReadBufferWaitForReaderInit, "Time") \ + M(CachedReadBufferWaitForReaderInit1, "Time") \ + M(CachedReadBufferWaitForReaderInit2, "Time") \ + M(CachedReadBufferGetImplementationBuffer, "Time") \ + M(CachedReadBufferInitialize, "Time") \ + M(CachedReadBufferGetOrSet, "Time") \ + M(CachedReadBufferRead, "Time") \ + M(CachedReadBufferRead1, "Time") \ + M(CachedReadBufferRead2, "Time") \ + M(CachedReadBufferRead3, "Time") \ + M(CachedReadBufferRead4, "Time") \ + M(CachedReadBufferRead5, "Time") \ + M(CachedReadBufferRead6, "Time") \ + M(CachedReadBufferRead7, "Time") \ + M(CachedReadBufferRead8, "Time") \ + M(CachedReadBufferRead9, "Time") \ + M(CachedReadBufferRead10, "Time") \ + M(FileCacheGetOrSet, "Time") \ + M(FileCacheGetOrSet1, "Time") \ + M(FileCacheGetOrSet2, "Time") \ + M(FileCacheGetOrSet3, "Time") \ + M(FileCacheGetOrSet4, "Time") \ + M(FileCacheGetImpl, "Time") \ + M(FileCacheGetImpl1, "Time") \ + M(FileCacheGetImpl2, "Time") \ + M(FileCacheGetImpl3, "Time") \ + M(FileCacheGetImpl4, "Time") \ + M(FileCacheGetImpl5, "Time") \ + M(FileCacheLockKey, "Time") \ + M(FileCacheCreateFile1, "Time") \ + M(FileCacheCreateFile2, "Time") \ \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ @@ -358,6 +393,18 @@ The server successfully detected this situation and will download merged part fr \ M(ThreadpoolReaderTaskMicroseconds, "Time spent getting the data in asynchronous reading") \ M(ThreadpoolReaderReadBytes, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolSubmit, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolSubmit2, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun2, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun3, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun4, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun5, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun6, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun7, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun8, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun9, "Bytes read from a threadpool task in asynchronous reading") \ + M(ThreadpoolRun10, "Bytes read from a threadpool task in asynchronous reading") \ \ M(FileSegmentWaitReadBufferMicroseconds, "Metric per file segment. Time spend waiting for internal read buffer (includes cache waiting)") \ M(FileSegmentReadMicroseconds, "Metric per file segment. Time spend reading from file") \ @@ -379,6 +426,7 @@ The server successfully detected this situation and will download merged part fr M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ + M(AsynchronousRemoteReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ \ M(ExternalDataSourceLocalCacheReadBytes, "Bytes read from local cache buffer in RemoteReadBufferCache")\ \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5b5d746ab55..f88bf281732 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -7,6 +7,7 @@ #include #include #include +#include "ElapsedTimeProfileEventIncrement.h" namespace ProfileEvents @@ -23,6 +24,28 @@ extern const Event CachedReadBufferCacheWriteMicroseconds; extern const Event CachedReadBufferReadFromSourceBytes; extern const Event CachedReadBufferReadFromCacheBytes; extern const Event CachedReadBufferCacheWriteBytes; + +extern const Event CachedReadBufferRead; +extern const Event CachedReadBufferInitialize; +extern const Event CachedReadBufferGetFileSegments; +extern const Event CachedReadBufferGetCacheReadBuffer; +extern const Event CachedReadBufferGetRemoteReadBuffer; +extern const Event CachedReadBufferPredownload; + +extern const Event CachedReadBufferWaitForReaderInit; +extern const Event CachedReadBufferWaitForReaderInit1; +extern const Event CachedReadBufferWaitForReaderInit2; +extern const Event CachedReadBufferRead1; +extern const Event CachedReadBufferRead2; +extern const Event CachedReadBufferRead3; +extern const Event CachedReadBufferRead4; +extern const Event CachedReadBufferRead5; +extern const Event CachedReadBufferRead6; +extern const Event CachedReadBufferRead7; +extern const Event CachedReadBufferRead8; +extern const Event CachedReadBufferRead9; +extern const Event CachedReadBufferRead10; +extern const Event CachedReadBufferGetImplementationBuffer; } namespace DB @@ -49,7 +72,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( std::optional read_until_position_) : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0, file_size_) #ifndef NDEBUG - , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile(" + source_file_path_ + ")")) + , log(&Poco::Logger::get(fmt::format("CachedOnDiskReadBufferFromFile({}:{})", source_file_path_, cache_key_.toString()))) #else , log(&Poco::Logger::get("CachedOnDiskReadBufferFromFile")) #endif @@ -69,6 +92,23 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( { } +FileSegmentsHolderPtr CachedOnDiskReadBufferFromFile::getFileSegments(size_t offset, size_t size) const +{ + ElapsedUSProfileEventIncrement watch_get_or_set(ProfileEvents::CachedReadBufferGetFileSegments); + + if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) + { + return cache->get(cache_key, offset, size); + } + + CreateFileSegmentSettings create_settings{ + .is_persistent = is_persistent + }; + + return cache->getOrSet(cache_key, offset, size, create_settings); +} + + void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type) { @@ -105,65 +145,35 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( cache_log->add(elem); } -void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) -{ - if (initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Caching buffer already initialized"); - - implementation_buffer.reset(); - - if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) - { - file_segments_holder.emplace(cache->get(cache_key, offset, size)); - } - else - { - CreateFileSegmentSettings create_settings{ - .is_persistent = is_persistent - }; - - file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); - } - - /** - * Segments in returned list are ordered in ascending order and represent a full contiguous - * interval (no holes). Each segment in returned list has state: DOWNLOADED, DOWNLOADING or EMPTY. - */ - if (file_segments_holder->file_segments.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "List of file segments cannot be empty"); - - LOG_TEST( - log, - "Having {} file segments to read: {}, current offset: {}", - file_segments_holder->file_segments.size(), file_segments_holder->toString(), file_offset_of_buffer_end); - - current_file_segment_it = file_segments_holder->file_segments.begin(); - - initialized = true; -} - CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segment) const +CachedOnDiskReadBufferFromFile::getCacheReadBuffer(FileSegment & file_segment) const { - /// Use is_persistent flag from in-memory state of the filesegment, - /// because it is consistent with what is written on disk. + ElapsedUSProfileEventIncrement watch(ProfileEvents::CachedReadBufferGetCacheReadBuffer); + auto path = file_segment.getPathInLocalCache(); - ReadSettings local_read_settings{settings}; - /// Do not allow to use asynchronous version of LocalFSReadMethod. - local_read_settings.local_fs_method = LocalFSReadMethod::pread; + if (!file_segment.file) + { + file_segment.file = OpenedFileCache::instance().get(path, -1); + file_segment.fd = file_segment.file->getFD(); + } - auto buf = createReadBufferFromFileBase(path, local_read_settings); + auto buf = std::make_unique( + path, settings.local_fs_buffer_size, -1, nullptr, 0, std::nullopt, file_segment.file, file_segment.fd); +#ifndef NDEBUG if (getFileSizeFromReadBuffer(*buf) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read from an empty cache file: {}", path); +#endif return buf; } CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_) +CachedOnDiskReadBufferFromFile::getRemoteReadBuffer(FileSegment & file_segment, ReadType read_type_) { + ElapsedUSProfileEventIncrement watch(ProfileEvents::CachedReadBufferGetRemoteReadBuffer); + switch (read_type_) { case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: @@ -237,22 +247,22 @@ bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, co } CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment) +CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_segment) { - auto download_state = file_segment->state(); + auto download_state = file_segment.state(); if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) { if (download_state == FileSegment::State::DOWNLOADED) { read_type = ReadType::CACHED; - return getCacheReadBuffer(*file_segment); + return getCacheReadBuffer(file_segment); } else { LOG_TEST(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(*file_segment, read_type); + return getRemoteReadBuffer(file_segment, read_type); } } @@ -264,11 +274,11 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil { LOG_TRACE(log, "Bypassing cache because file segment state is `SKIP_CACHE`"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(*file_segment, read_type); + return getRemoteReadBuffer(file_segment, read_type); } case FileSegment::State::DOWNLOADING: { - if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) + if (canStartFromCache(file_offset_of_buffer_end, file_segment)) { /// segment{k} state: DOWNLOADING /// cache: [______|___________ @@ -279,21 +289,21 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// file_offset_of_buffer_end read_type = ReadType::CACHED; - return getCacheReadBuffer(*file_segment); + return getCacheReadBuffer(file_segment); } - download_state = file_segment->wait(); + download_state = file_segment.wait(); continue; } case FileSegment::State::DOWNLOADED: { read_type = ReadType::CACHED; - return getCacheReadBuffer(*file_segment); + return getCacheReadBuffer(file_segment); } case FileSegment::State::EMPTY: case FileSegment::State::PARTIALLY_DOWNLOADED: { - if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) + if (canStartFromCache(file_offset_of_buffer_end, file_segment)) { /// segment{k} state: PARTIALLY_DOWNLOADED /// cache: [______|___________ @@ -304,13 +314,13 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// file_offset_of_buffer_end read_type = ReadType::CACHED; - return getCacheReadBuffer(*file_segment); + return getCacheReadBuffer(file_segment); } - auto downloader_id = file_segment->getOrSetDownloader(); - if (downloader_id == file_segment->getCallerId()) + auto downloader_id = file_segment.getOrSetDownloader(); + if (downloader_id == file_segment.getCallerId()) { - if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) + if (canStartFromCache(file_offset_of_buffer_end, file_segment)) { /// segment{k} /// cache: [______|___________ @@ -321,11 +331,11 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// file_offset_of_buffer_end read_type = ReadType::CACHED; - file_segment->resetDownloader(); - return getCacheReadBuffer(*file_segment); + file_segment.resetDownloader(); + return getCacheReadBuffer(file_segment); } - if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end) + if (file_segment.getCurrentWriteOffset() < file_offset_of_buffer_end) { /// segment{1} /// cache: [_____|___________ @@ -335,25 +345,25 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// ^ /// file_offset_of_buffer_end - LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog()); - chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset()); - bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset(); - chassert(bytes_to_predownload < file_segment->range().size()); + bytes_to_predownload = file_offset_of_buffer_end - file_segment.getCurrentWriteOffset(); + + chassert(file_offset_of_buffer_end > file_segment.getCurrentWriteOffset()); + chassert(bytes_to_predownload < file_segment.range().size()); } read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; - return getRemoteFSReadBuffer(*file_segment, read_type); + return getRemoteReadBuffer(file_segment, read_type); } - download_state = file_segment->state(); + download_state = file_segment.state(); continue; } case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { - if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) + if (canStartFromCache(file_offset_of_buffer_end, file_segment)) { read_type = ReadType::CACHED; - return getCacheReadBuffer(*file_segment); + return getCacheReadBuffer(file_segment); } else { @@ -361,7 +371,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil log, "Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - return getRemoteFSReadBuffer(*file_segment, read_type); + return getRemoteReadBuffer(file_segment, read_type); } } } @@ -369,12 +379,14 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil } CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment) +CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segment) { - chassert(!file_segment->isDownloader()); - chassert(file_offset_of_buffer_end >= file_segment->range().left); + ElapsedUSProfileEventIncrement increment(ProfileEvents::CachedReadBufferGetImplementationBuffer); - auto range = file_segment->range(); + chassert(!file_segment.isDownloader()); + chassert(file_offset_of_buffer_end >= file_segment.range().left); + + const auto & range = file_segment.range(); bytes_to_predownload = 0; Stopwatch watch(CLOCK_MONOTONIC); @@ -382,13 +394,14 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se auto read_buffer_for_file_segment = getReadBufferForFileSegment(file_segment); watch.stop(); - current_file_segment_counters.increment( - ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds()); + auto elapsed_ms = watch.elapsedMicroseconds(); + current_file_segment_counters.increment(ProfileEvents::FileSegmentWaitReadBufferMicroseconds, elapsed_ms); + ProfileEvents::increment(ProfileEvents::CachedReadBufferWaitForReaderInit, elapsed_ms); [[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; - chassert(download_current_segment == file_segment->isDownloader()); + chassert(download_current_segment == file_segment.isDownloader()); - chassert(file_segment->range() == range); + chassert(file_segment.range() == range); chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right); LOG_TEST( @@ -438,11 +451,11 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se } case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: { - chassert(file_segment->isDownloader()); + chassert(file_segment.isDownloader()); if (bytes_to_predownload) { - size_t current_write_offset = file_segment->getCurrentWriteOffset(); + size_t current_write_offset = file_segment.getCurrentWriteOffset(); read_buffer_for_file_segment->seek(current_write_offset, SEEK_SET); } else @@ -453,7 +466,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se assert(static_cast(read_buffer_for_file_segment->getFileOffsetOfBufferEnd()) == file_offset_of_buffer_end); } - auto current_write_offset = file_segment->getCurrentWriteOffset(); + auto current_write_offset = file_segment.getCurrentWriteOffset(); if (current_write_offset != static_cast(read_buffer_for_file_segment->getPosition())) { throw Exception( @@ -461,7 +474,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {} implementation buffer offset: {}, " "implementation buffer remaining range: {}, file segment info: {}", file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(), - read_buffer_for_file_segment->getRemainingReadRange().toString(), file_segment->getInfoForLog()); + read_buffer_for_file_segment->getRemainingReadRange().toString(), file_segment.getInfoForLog()); } break; @@ -473,207 +486,189 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() { - LOG_TEST(log, "Completed segment: {}", (*current_file_segment_it)->range().toString()); + auto * current_file_segment = &file_segments->front(); + const auto & completed_range = current_file_segment->range(); if (enable_logging) - appendFilesystemCacheLog((*current_file_segment_it)->range(), read_type); + appendFilesystemCacheLog(completed_range, read_type); - auto file_segment_it = current_file_segment_it++; - auto & file_segment = *file_segment_it; + chassert(file_offset_of_buffer_end > completed_range.right); + chassert(read_type != ReadType::CACHED || current_file_segment->state() == FileSegment::State::DOWNLOADED); - [[maybe_unused]] const auto & range = file_segment->range(); - chassert(file_offset_of_buffer_end > range.right); - - LOG_TEST( - log, - "Removing file segment: {}, downloader: {}, state: {}", - file_segment->range().toString(), - file_segment->getDownloader(), - file_segment->state()); - - /// Do not hold pointer to file segment if it is not needed anymore - /// so can become releasable and can be evicted from cache. - file_segment->completeWithoutState(); - file_segments_holder->file_segments.erase(file_segment_it); - - if (current_file_segment_it == file_segments_holder->file_segments.end()) + file_segments->popFront(); + if (file_segments->empty()) return false; - implementation_buffer = getImplementationBuffer(*current_file_segment_it); + current_file_segment = &file_segments->front(); + implementation_buffer = getImplementationBuffer(*current_file_segment); if (read_type == ReadType::CACHED) - (*current_file_segment_it)->incrementHitsCount(); + current_file_segment->incrementHitsCount(); + + LOG_TEST( + log, "New segment range: {}, old range: {}", + current_file_segment->range().toString(), completed_range.toString()); - LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString()); return true; } CachedOnDiskReadBufferFromFile::~CachedOnDiskReadBufferFromFile() { - if (enable_logging - && file_segments_holder - && current_file_segment_it != file_segments_holder->file_segments.end()) + if (enable_logging && file_segments && file_segments->empty()) { - appendFilesystemCacheLog((*current_file_segment_it)->range(), read_type); + appendFilesystemCacheLog(file_segments->front().range(), read_type); } } -void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) +void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) { - Stopwatch predownload_watch(CLOCK_MONOTONIC); - SCOPE_EXIT({ - predownload_watch.stop(); - current_file_segment_counters.increment( - ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds()); - }); + if (!bytes_to_predownload) + return; - if (bytes_to_predownload) + ElapsedUSProfileEventIncrement watch_predownload(ProfileEvents::CachedReadBufferPredownload); + /// Consider this case. Some user needed segment [a, b] and downloaded it partially. + /// But before he called complete(state) or his holder called complete(), + /// some other user, who needed segment [a', b'], a < a' < b', started waiting on [a, b] to be + /// downloaded because it intersects with the range he needs. + /// But then first downloader fails and second must continue. In this case we need to + /// download from offset a'' < a', but return buffer from offset a'. + + chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment.getCurrentWriteOffset()); + size_t current_offset = file_segment.getCurrentWriteOffset(); + const auto & current_range = file_segment.range(); + + LOG_TEST(log, "Predownloading for {}", file_segment.getInfoForLog()); + + while (true) { - /// Consider this case. Some user needed segment [a, b] and downloaded it partially. - /// But before he called complete(state) or his holder called complete(), - /// some other user, who needed segment [a', b'], a < a' < b', started waiting on [a, b] to be - /// downloaded because it intersects with the range he needs. - /// But then first downloader fails and second must continue. In this case we need to - /// download from offset a'' < a', but return buffer from offset a'. - LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId()); - - chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getCurrentWriteOffset()); - size_t current_offset = file_segment->getCurrentWriteOffset(); - const auto & current_range = file_segment->range(); - - while (true) + bool has_more_data; { - bool has_more_data; - { - Stopwatch watch(CLOCK_MONOTONIC); + Stopwatch watch(CLOCK_MONOTONIC); - has_more_data = !implementation_buffer->eof(); + has_more_data = !implementation_buffer->eof(); - watch.stop(); - auto elapsed = watch.elapsedMicroseconds(); - current_file_segment_counters.increment(ProfileEvents::FileSegmentReadMicroseconds, elapsed); - ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceMicroseconds, elapsed); - } + watch.stop(); + auto elapsed = watch.elapsedMicroseconds(); + current_file_segment_counters.increment(ProfileEvents::FileSegmentReadMicroseconds, elapsed); + ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceMicroseconds, elapsed); + } - if (!bytes_to_predownload || !has_more_data) - { - if (bytes_to_predownload) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Failed to predownload remaining {} bytes. Current file segment: {}, " - "current download offset: {}, expected: {}, eof: {}", - bytes_to_predownload, - current_range.toString(), - file_segment->getCurrentWriteOffset(), - file_offset_of_buffer_end, - implementation_buffer->eof()); - - auto result = implementation_buffer->hasPendingData(); - - if (result) - { - nextimpl_working_buffer_offset = implementation_buffer->offset(); - - auto current_write_offset = file_segment->getCurrentWriteOffset(); - if (current_write_offset != static_cast(implementation_buffer->getPosition()) - || current_write_offset != file_offset_of_buffer_end) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Buffer's offsets mismatch after predownloading; download offset: {}, " - "cached buffer offset: {}, implementation buffer offset: {}, " - "file segment info: {}", - current_write_offset, - file_offset_of_buffer_end, - implementation_buffer->getPosition(), - file_segment->getInfoForLog()); - } - } - - break; - } - - size_t current_impl_buffer_size = implementation_buffer->buffer().size(); - size_t current_predownload_size = std::min(current_impl_buffer_size, bytes_to_predownload); - - ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); - - bool continue_predownload = file_segment->reserve(current_predownload_size); - if (continue_predownload) - { - LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); - - chassert(file_segment->getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); - - bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment); - if (success) - { - current_offset += current_predownload_size; - - bytes_to_predownload -= current_predownload_size; - implementation_buffer->position() += current_predownload_size; - } - else - { - LOG_TEST(log, "Bypassing cache because writeCache (in predownload) method failed"); - continue_predownload = false; - } - } - - if (!continue_predownload) - { - /// We were predownloading: - /// segment{1} - /// cache: [_____|___________ - /// ^ - /// current_write_offset - /// requested_range: [__________] - /// ^ - /// file_offset_of_buffer_end - /// But space reservation failed. - /// So get working and internal buffer from predownload buffer, get new download buffer, - /// return buffer back, seek to actual position. - /// We could reuse predownload buffer and just seek to needed position, but for now - /// seek is only allowed once for ReadBufferForS3 - before call to nextImpl. - /// TODO: allow seek more than once with seek avoiding. - - bytes_to_predownload = 0; - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - - LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog()); - - read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; - - swap(*implementation_buffer); - resetWorkingBuffer(); - - implementation_buffer = getRemoteFSReadBuffer(*file_segment, read_type); - - swap(*implementation_buffer); - - implementation_buffer->setReadUntilPosition(file_segment->range().right + 1); /// [..., range.right] - implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET); - - LOG_TRACE( - log, - "Predownload failed because of space limit. " - "Will read from remote filesystem starting from offset: {}", + if (!bytes_to_predownload || !has_more_data) + { + if (bytes_to_predownload) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to predownload remaining {} bytes. Current file segment: {}, " + "current download offset: {}, expected: {}", + bytes_to_predownload, + current_range.toString(), + file_segment.getCurrentWriteOffset(), file_offset_of_buffer_end); - break; + auto result = implementation_buffer->hasPendingData(); + if (result) + { + nextimpl_working_buffer_offset = implementation_buffer->offset(); + + auto current_write_offset = file_segment.getCurrentWriteOffset(); + if (current_write_offset != static_cast(implementation_buffer->getPosition()) + || current_write_offset != file_offset_of_buffer_end) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Buffer's offsets mismatch after predownloading; download offset: {}, " + "cached buffer offset: {}, implementation buffer offset: {}, " + "file segment info: {}", + current_write_offset, + file_offset_of_buffer_end, + implementation_buffer->getPosition(), + file_segment.getInfoForLog()); + } } + + break; + } + + size_t current_impl_buffer_size = implementation_buffer->buffer().size(); + size_t current_predownload_size = std::min(current_impl_buffer_size, bytes_to_predownload); + + ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); + + bool continue_predownload = file_segment.reserve(current_predownload_size); + if (continue_predownload) + { + LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); + + chassert(file_segment.getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); + + bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, file_segment); + if (success) + { + current_offset += current_predownload_size; + + bytes_to_predownload -= current_predownload_size; + implementation_buffer->position() += current_predownload_size; + } + else + { + LOG_TEST(log, "Bypassing cache because writeCache (in predownload) method failed"); + continue_predownload = false; + } + } + + if (!continue_predownload) + { + /// We were predownloading: + /// segment{1} + /// cache: [_____|___________ + /// ^ + /// current_write_offset + /// requested_range: [__________] + /// ^ + /// file_offset_of_buffer_end + /// But space reservation failed. + /// So get working and internal buffer from predownload buffer, get new download buffer, + /// return buffer back, seek to actual position. + /// We could reuse predownload buffer and just seek to needed position, but for now + /// seek is only allowed once for ReadBufferForS3 - before call to nextImpl. + /// TODO: allow seek more than once with seek avoiding. + + bytes_to_predownload = 0; + file_segment.setBroken(); + + LOG_TEST(log, "Bypassing cache because for {}", file_segment.getInfoForLog()); + + read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; + + swap(*implementation_buffer); + resetWorkingBuffer(); + + implementation_buffer = getRemoteReadBuffer(file_segment, read_type); + + swap(*implementation_buffer); + + implementation_buffer->setReadUntilPosition(file_segment.range().right + 1); /// [..., range.right] + implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET); + + LOG_TRACE( + log, + "Predownload failed because of space limit. " + "Will read from remote filesystem starting from offset: {}", + file_offset_of_buffer_end); + + break; } } } bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() { - auto & file_segment = *current_file_segment_it; - auto current_read_range = file_segment->range(); - auto current_state = file_segment->state(); + auto & file_segment = file_segments->front(); + const auto & current_read_range = file_segment.range(); + auto current_state = file_segment.state(); chassert(current_read_range.left <= file_offset_of_buffer_end); - chassert(!file_segment->isDownloader()); + chassert(!file_segment.isDownloader()); if (file_offset_of_buffer_end > current_read_range.right) { @@ -692,7 +687,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() /// ^ /// file_offset_of_buffer_end - auto current_write_offset = file_segment->getCurrentWriteOffset(); + auto current_write_offset = file_segment.getCurrentWriteOffset(); bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end; LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end); @@ -700,7 +695,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() if (cached_part_is_finished) { /// TODO: makes sense to reuse local file reader if we return here with CACHED read type again? - implementation_buffer = getImplementationBuffer(*current_file_segment_it); + implementation_buffer = getImplementationBuffer(file_segment); return true; } @@ -728,7 +723,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() * to read by marks range given to him. Therefore, each nextImpl() call, in case of * READ_AND_PUT_IN_CACHE, starts with getOrSetDownloader(). */ - implementation_buffer = getImplementationBuffer(*current_file_segment_it); + implementation_buffer = getImplementationBuffer(file_segment); } return true; @@ -764,52 +759,29 @@ bool CachedOnDiskReadBufferFromFile::writeCache(char * data, size_t size, size_t bool CachedOnDiskReadBufferFromFile::nextImpl() { - try - { - return nextImplStep(); - } - catch (Exception & e) - { - e.addMessage("Cache info: {}", nextimpl_step_log_info); - throw; - } -} - -bool CachedOnDiskReadBufferFromFile::nextImplStep() -{ - last_caller_id = FileSegment::getCallerId(); - - assertCorrectness(); - - if (file_offset_of_buffer_end == read_until_position) - return false; - - if (!initialized) - initialize(file_offset_of_buffer_end, getTotalSizeToRead()); - - if (current_file_segment_it == file_segments_holder->file_segments.end()) - return false; + ElapsedUSProfileEventIncrement watch(ProfileEvents::CachedReadBufferRead); SCOPE_EXIT({ + ElapsedUSProfileEventIncrement increment3(ProfileEvents::CachedReadBufferRead3); try { - /// Save state of current file segment before it is completed. - nextimpl_step_log_info = getInfoForLog(); - - if (current_file_segment_it == file_segments_holder->file_segments.end()) + if (file_segments->empty()) return; - auto & file_segment = *current_file_segment_it; + auto & file_segment = file_segments->front(); + + if (file_segment.isCompleted()) + return; bool download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; if (download_current_segment) { - bool need_complete_file_segment = file_segment->isDownloader(); + bool need_complete_file_segment = file_segment.isDownloader(); if (need_complete_file_segment) - file_segment->completePartAndResetDownloader(); + file_segment.completePartAndResetDownloader(); } - chassert(!file_segment->isDownloader()); + chassert(!file_segment.isDownloader()); } catch (...) { @@ -817,8 +789,55 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } }); + try + { + return nextImplStep(); + } + catch (Exception & e) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + if (!file_segments->empty()) + { + e.addMessage( + "Current reading thread: {}, cache info " + "(cache info is added on each error while reading data, the error can be unrelated to cache): {}", + getThreadId(), file_segments->front().getInfoForLog()); + } + throw; + } +} + +bool CachedOnDiskReadBufferFromFile::nextImplStep() +{ + ElapsedUSProfileEventIncrement increment(ProfileEvents::CachedReadBufferRead1); + + if (file_offset_of_buffer_end == read_until_position) + return false; + + if (initialized && file_segments->empty()) + return false; + + if (!initialized) + { + implementation_buffer.reset(); + + file_segments = getFileSegments(file_offset_of_buffer_end, getTotalSizeToRead()); + if (file_segments->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "List of file segments cannot be empty"); + + LOG_TEST( + log, "Having {} file segments to read: {}, current offset: {}", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + + initialized = true; + } + + ElapsedUSProfileEventIncrement increment2(ProfileEvents::CachedReadBufferRead2); + bytes_to_predownload = 0; + ElapsedUSProfileEventIncrement increment4(ProfileEvents::CachedReadBufferRead4); if (implementation_buffer) { bool can_read_further = updateImplementationBufferIfNeeded(); @@ -827,26 +846,22 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } else { - implementation_buffer = getImplementationBuffer(*current_file_segment_it); + implementation_buffer = getImplementationBuffer(file_segments->front()); if (read_type == ReadType::CACHED) - (*current_file_segment_it)->incrementHitsCount(); + file_segments->front().incrementHitsCount(); } + ElapsedUSProfileEventIncrement increment5(ProfileEvents::CachedReadBufferRead5); + chassert(!internal_buffer.empty()); swap(*implementation_buffer); - auto & file_segment = *current_file_segment_it; - auto current_read_range = file_segment->range(); + auto & file_segment = file_segments->front(); + const auto & current_read_range = file_segment.range(); - LOG_TEST( - log, - "Current count: {}, position: {}, read range: {}, file segment: {}", - implementation_buffer->count(), - implementation_buffer->getPosition(), - implementation_buffer->getRemainingReadRange().toString(), - file_segment->getInfoForLog()); + LOG_TEST(log, "Current file segment: {}", file_segment.getInfoForLog()); chassert(current_read_range.left <= file_offset_of_buffer_end); chassert(current_read_range.right >= file_offset_of_buffer_end); @@ -854,8 +869,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() bool result = false; size_t size = 0; - size_t needed_to_predownload = bytes_to_predownload; - if (needed_to_predownload) + ElapsedUSProfileEventIncrement increment6(ProfileEvents::CachedReadBufferRead6); + + if (bytes_to_predownload) { predownload(file_segment); @@ -864,14 +880,18 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; - if (download_current_segment != file_segment->isDownloader()) + +#ifndef NDEBUG + if (download_current_segment != file_segment.isDownloader()) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect segment state. Having read type: {}, file segment info: {}", - toString(read_type), file_segment->getInfoForLog()); + toString(read_type), file_segment.getInfoForLog()); } +#endif + ElapsedUSProfileEventIncrement increment7(ProfileEvents::CachedReadBufferRead7); if (!result) { #ifndef NDEBUG @@ -894,12 +914,25 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() assert(!implementation_buffer->hasPendingData()); #endif - Stopwatch watch(CLOCK_MONOTONIC); + uint64_t elapsed; + { + Stopwatch watch(CLOCK_MONOTONIC); - result = implementation_buffer->next(); + try + { + result = implementation_buffer->next(); + } + catch (Exception & e) + { + e.addMessage("(" + file_segment.getInfoForLog() + ")"); + throw; + } - watch.stop(); - auto elapsed = watch.elapsedMicroseconds(); + watch.stop(); + elapsed = watch.elapsedMicroseconds(); + } + + ElapsedUSProfileEventIncrement increment8(ProfileEvents::CachedReadBufferRead8); current_file_segment_counters.increment(ProfileEvents::FileSegmentReadMicroseconds, elapsed); size = implementation_buffer->buffer().size(); @@ -922,41 +955,42 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } } + ElapsedUSProfileEventIncrement increment9(ProfileEvents::CachedReadBufferRead9); if (result) { if (download_current_segment) { - chassert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right); + chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); - bool success = file_segment->reserve(size); + bool success = file_segment.reserve(size); if (success) { - chassert(file_segment->getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); + chassert(file_segment.getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); - success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment); + success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, file_segment); if (success) { - chassert(file_segment->getCurrentWriteOffset() <= file_segment->range().right + 1); + chassert(file_segment.getCurrentWriteOffset() <= file_segment.range().right + 1); chassert( - std::next(current_file_segment_it) == file_segments_holder->file_segments.end() - || file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); + /* last_file_segment */file_segments->size() == 1 + || file_segment.getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); LOG_TEST(log, "Successfully written {} bytes", size); } else { - chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + chassert(file_segment.state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); LOG_TRACE(log, "Bypassing cache because writeCache method failed"); } } else { LOG_TRACE(log, "No space left in cache, will continue without cache download"); - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); } if (!success) { + file_segment.setBroken(); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; download_current_segment = false; } @@ -968,7 +1002,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() /// Therefore need to resize to a smaller size. And resize must be done after write into cache. /// - If last file segment was read from local fs, then we could read more than /// file_segemnt->range().right, so resize is also needed. - if (std::next(current_file_segment_it) == file_segments_holder->file_segments.end()) + if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; @@ -984,31 +1018,34 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available()); - if (download_current_segment) - file_segment->completePartAndResetDownloader(); + ElapsedUSProfileEventIncrement increment10(ProfileEvents::CachedReadBufferRead10); - chassert(!file_segment->isDownloader()); + if (download_current_segment) + { + file_segment.completePartAndResetDownloader(); + } + + chassert(!file_segment.isDownloader()); LOG_TEST( log, - "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " + "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}), " "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " "remaining ranges: {}", - getHexUIntLowercase(cache_key), + cache_key.toString(), working_buffer.size(), getPosition(), offset(), - needed_to_predownload, available(), current_read_range.toString(), file_offset_of_buffer_end, - FileSegment::stateToString(file_segment->state()), - file_segment->getCurrentWriteOffset(), + FileSegment::stateToString(file_segment.state()), + file_segment.getCurrentWriteOffset(), toString(read_type), read_until_position, first_offset, - file_segments_holder->toString()); + file_segments->toString()); if (size == 0 && file_offset_of_buffer_end < read_until_position) { @@ -1027,7 +1064,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() cache_file_size ? std::to_string(cache_file_size) : "None", cache_file_path, implementation_buffer->getFileOffsetOfBufferEnd(), - file_segment->getInfoForLog()); + file_segment.getInfoForLog()); } return result; @@ -1075,13 +1112,13 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence) first_offset = file_offset_of_buffer_end = new_pos; resetWorkingBuffer(); - // if (file_segments_holder && current_file_segment_it != file_segments_holder->file_segments.end()) + // if (file_segments && current_file_segment_it != file_segments->file_segments.end()) // { - // auto & file_segments = file_segments_holder->file_segments; + // auto & file_segments = file_segments->file_segments; // LOG_TRACE( // log, // "Having {} file segments to read: {}, current offset: {}", - // file_segments_holder->file_segments.size(), file_segments_holder->toString(), file_offset_of_buffer_end); + // file_segments->file_segments.size(), file_segments->toString(), file_offset_of_buffer_end); // auto it = std::upper_bound( // file_segments.begin(), @@ -1112,7 +1149,7 @@ off_t CachedOnDiskReadBufferFromFile::seek(off_t offset, int whence) // } // } - file_segments_holder.reset(); + file_segments.reset(); implementation_buffer.reset(); initialized = false; @@ -1155,29 +1192,6 @@ off_t CachedOnDiskReadBufferFromFile::getPosition() return file_offset_of_buffer_end - available(); } -std::optional CachedOnDiskReadBufferFromFile::getLastNonDownloadedOffset() const -{ - if (!file_segments_holder) - throw Exception(ErrorCodes::LOGICAL_ERROR, "File segments holder not initialized"); - - const auto & file_segments = file_segments_holder->file_segments; - for (auto it = file_segments.rbegin(); it != file_segments.rend(); ++it) - { - const auto & file_segment = *it; - if (file_segment->state() != FileSegment::State::DOWNLOADED) - return file_segment->range().right; - } - - return std::nullopt; -} - -void CachedOnDiskReadBufferFromFile::assertCorrectness() const -{ - if (FileCache::isReadOnly() - && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed"); -} - String CachedOnDiskReadBufferFromFile::getInfoForLog() { String implementation_buffer_read_range_str; @@ -1187,21 +1201,20 @@ String CachedOnDiskReadBufferFromFile::getInfoForLog() implementation_buffer_read_range_str = "None"; String current_file_segment_info; - if (current_file_segment_it == file_segments_holder->file_segments.end()) - current_file_segment_info = (*current_file_segment_it)->getInfoForLog(); - else + if (file_segments->empty()) current_file_segment_info = "None"; + else + current_file_segment_info = file_segments->front().getInfoForLog(); return fmt::format( "Buffer path: {}, hash key: {}, file_offset_of_buffer_end: {}, " "internal buffer remaining read range: {}, " - "read_type: {}, last caller: {}, file segment info: {}", + "read_type: {}, file segment info: {}", source_file_path, - getHexUIntLowercase(cache_key), + cache_key.toString(), file_offset_of_buffer_end, implementation_buffer_read_range_str, toString(read_type), - last_caller_id, current_file_segment_info); } diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 14e8ea6c7e7..c4229a233a2 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -62,26 +62,26 @@ public: }; private: - void initialize(size_t offset, size_t size); + /** + * Return a list of file segments ordered in ascending order. This list represents + * a full contiguous interval (without holes). + */ + FileSegmentsHolderPtr getFileSegments(size_t offset, size_t size) const; - ImplementationBufferPtr getImplementationBuffer(FileSegmentPtr & file_segment); + ImplementationBufferPtr getImplementationBuffer(FileSegment & file_segment); - ImplementationBufferPtr getReadBufferForFileSegment(FileSegmentPtr & file_segment); + ImplementationBufferPtr getReadBufferForFileSegment(FileSegment & file_segment); - ImplementationBufferPtr getCacheReadBuffer(const FileSegment & file_segment) const; + ImplementationBufferPtr getCacheReadBuffer(FileSegment & file_segment) const; - std::optional getLastNonDownloadedOffset() const; + ImplementationBufferPtr getRemoteReadBuffer(FileSegment & file_segment, ReadType read_type_); bool updateImplementationBufferIfNeeded(); - void predownload(FileSegmentPtr & file_segment); + void predownload(FileSegment & file_segment); bool nextImplStep(); - void assertCorrectness() const; - - std::shared_ptr getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_); - size_t getTotalSizeToRead(); bool completeFileSegmentAndGetNext(); @@ -108,8 +108,7 @@ private: /// Remote read buffer, which can only be owned by current buffer. FileSegment::RemoteFileReaderPtr remote_file_reader; - std::optional file_segments_holder; - FileSegments::iterator current_file_segment_it; + FileSegmentsHolderPtr file_segments; ImplementationBufferPtr implementation_buffer; bool initialized = false; @@ -131,8 +130,6 @@ private: } size_t first_offset = 0; - String nextimpl_step_log_info; - String last_caller_id; String query_id; bool enable_logging = false; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 823e4125f06..c7d3b9d551c 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -64,18 +64,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset offset, expected_write_offset); } - auto & file_segments = file_segments_holder.file_segments; + FileSegment * file_segment; - if (file_segments.empty() || file_segments.back()->isDownloaded()) + if (file_segments.empty() || file_segments.back().isDownloaded()) { - allocateFileSegment(expected_write_offset, is_persistent); + file_segment = &allocateFileSegment(expected_write_offset, is_persistent); + } + else + { + file_segment = &file_segments.back(); } - auto & file_segment = file_segments.back(); - SCOPE_EXIT({ - if (file_segments.back()->isDownloader()) - file_segments.back()->completePartAndResetDownloader(); + if (file_segments.back().isDownloader()) + file_segments.back().completePartAndResetDownloader(); }); while (size > 0) @@ -84,7 +86,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset if (available_size == 0) { completeFileSegment(*file_segment); - file_segment = allocateFileSegment(expected_write_offset, is_persistent); + file_segment = &allocateFileSegment(expected_write_offset, is_persistent); continue; } @@ -100,7 +102,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset bool reserved = file_segment->reserve(size_to_write); if (!reserved) { - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + file_segment->setBroken(); appendFilesystemCacheLog(*file_segment); LOG_DEBUG( @@ -127,11 +129,10 @@ void FileSegmentRangeWriter::finalize() if (finalized) return; - auto & file_segments = file_segments_holder.file_segments; if (file_segments.empty()) return; - completeFileSegment(*file_segments.back()); + completeFileSegment(file_segments.back()); finalized = true; } @@ -148,15 +149,13 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter() } } -FileSegmentPtr & FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent) +FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent) { /** * Allocate a new file segment starting `offset`. * File segment capacity will equal `max_file_segment_size`, but actual size is 0. */ - std::lock_guard cache_lock(cache->mutex); - CreateFileSegmentSettings create_settings { .is_persistent = is_persistent, @@ -165,10 +164,9 @@ FileSegmentPtr & FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. auto file_segment = cache->createFileSegmentForDownload( - key, offset, cache->max_file_segment_size, create_settings, cache_lock); + key, offset, cache->max_file_segment_size, create_settings); - auto & file_segments = file_segments_holder.file_segments; - return *file_segments.insert(file_segments.end(), file_segment); + return file_segments.add(std::move(file_segment)); } void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_segment) @@ -202,7 +200,7 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) if (file_segment.isDetached() || file_segment.isCompleted()) return; - file_segment.completeWithoutState(); + file_segment.complete(); appendFilesystemCacheLog(file_segment); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 280005734c0..b72da2d3a40 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -39,7 +39,7 @@ public: ~FileSegmentRangeWriter(); private: - FileSegmentPtr & allocateFileSegment(size_t offset, bool is_persistent); + FileSegment & allocateFileSegment(size_t offset, bool is_persistent); void appendFilesystemCacheLog(const FileSegment & file_segment); @@ -53,7 +53,7 @@ private: String query_id; String source_path; - FileSegmentsHolder file_segments_holder{}; + FileSegmentsHolder file_segments{}; size_t expected_write_offset = 0; diff --git a/src/Disks/IO/ElapsedTimeProfileEventIncrement.h b/src/Disks/IO/ElapsedTimeProfileEventIncrement.h new file mode 100644 index 00000000000..34a366a1ccc --- /dev/null +++ b/src/Disks/IO/ElapsedTimeProfileEventIncrement.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct ElapsedMSProfileEventIncrement +{ + explicit ElapsedMSProfileEventIncrement(ProfileEvents::Event event_) : event(event_), watch((CLOCK_MONOTONIC)) {} + + ~ElapsedMSProfileEventIncrement() + { + watch.stop(); + ProfileEvents::increment(event, watch.elapsedMilliseconds()); + } + + ProfileEvents::Event event; + Stopwatch watch; +}; + +struct ElapsedUSProfileEventIncrement +{ + explicit ElapsedUSProfileEventIncrement(ProfileEvents::Event event_, bool cancel_ = false) + : event(event_), watch((CLOCK_MONOTONIC)), cancel(cancel_) {} + + ~ElapsedUSProfileEventIncrement() + { + if (!cancel) + { + watch.stop(); + ProfileEvents::increment(event, watch.elapsedMicroseconds()); + } + } + + ProfileEvents::Event event; + Stopwatch watch; + bool cancel; +}; +} diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 01d4154199a..a1492d28028 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -35,7 +36,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( with_cache = settings.remote_fs_cache && settings.enable_filesystem_cache - && (!FileCache::isReadOnly() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); + && (CachedObjectStorage::canUseReadThroughCache() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); } SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const String & path, size_t file_size) @@ -54,7 +55,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c if (with_cache) { - auto cache_key = settings.remote_fs_cache->hash(path); + auto cache_key = settings.remote_fs_cache->createKeyForPath(path); return std::make_shared( path, cache_key, diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 505b26ebb3a..3c5a05187c8 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -43,7 +43,7 @@ DataSourceDescription CachedObjectStorage::getDataSourceDescription() const FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const { - return cache->hash(path); + return cache->createKeyForPath(path); } String CachedObjectStorage::getCachePath(const std::string & path) const @@ -62,7 +62,7 @@ ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settin ReadSettings modified_settings{read_settings}; modified_settings.remote_fs_cache = cache; - if (FileCache::isReadOnly()) + if (!canUseReadThroughCache()) modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; return IObjectStorage::patchSettings(modified_settings); @@ -308,4 +308,11 @@ String CachedObjectStorage::getObjectsNamespace() const return object_storage->getObjectsNamespace(); } +bool CachedObjectStorage::canUseReadThroughCache() +{ + return CurrentThread::isInitialized() + && CurrentThread::get().getQueryContext() + && !CurrentThread::getQueryId().empty(); +} + } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 2d67203be0f..07310772a81 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -113,6 +113,8 @@ public: WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; + static bool canUseReadThroughCache(); + private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index c7e2f9b0c41..84f7f11f362 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -108,6 +108,8 @@ bool AsynchronousReadBufferFromFileDescriptor::nextImpl() else { /// No pending request. Do synchronous read. + Stopwatch watch; + ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); auto [size, offset] = asyncReadInto(memory.data(), memory.size()).get(); file_offset_of_buffer_end += size; diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index 5ef75ad01ec..141caa7018a 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -9,6 +9,8 @@ namespace ProfileEvents { extern const Event FileOpen; + extern const Event FileCacheCreateFile1; + extern const Event FileCacheCreateFile2; } @@ -89,4 +91,24 @@ void ReadBufferFromFile::close() metric_increment.destroy(); } +ReadBufferFromFilePReadWithDescriptorsCache::ReadBufferFromFilePReadWithDescriptorsCache( + const std::string & file_name_, size_t buf_size, int flags, + char * existing_memory, size_t alignment, std::optional file_size_, + OpenedFileCache::OpenedFilePtr file_, + int fd_) + : ReadBufferFromFileDescriptorPRead(-1, buf_size, existing_memory, alignment, file_size_) + , file_name(file_name_) +{ + if (file_) + { + file = file_; + fd = fd_; + } + else + { + file = OpenedFileCache::instance().get(file_name, flags); + fd = file->getFD(); + } +} + } diff --git a/src/IO/ReadBufferFromFile.h b/src/IO/ReadBufferFromFile.h index a720f8dd36d..21233ab9aa2 100644 --- a/src/IO/ReadBufferFromFile.h +++ b/src/IO/ReadBufferFromFile.h @@ -88,13 +88,9 @@ public: int flags = -1, char * existing_memory = nullptr, size_t alignment = 0, - std::optional file_size_ = std::nullopt) - : ReadBufferFromFileDescriptorPRead(-1, buf_size, existing_memory, alignment, file_size_) - , file_name(file_name_) - { - file = OpenedFileCache::instance().get(file_name, flags); - fd = file->getFD(); - } + std::optional file_size_ = std::nullopt, + OpenedFileCache::OpenedFilePtr file_ = nullptr, + int fd = 0); std::string getFileName() const override { diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 2551e236f7b..4bdf05d81a5 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1,7 +1,6 @@ #include "FileCache.h" #include -#include #include #include #include @@ -10,10 +9,28 @@ #include #include #include +#include #include +#include namespace fs = std::filesystem; +namespace ProfileEvents +{ +extern const Event FileCacheGetOrSet; +extern const Event FileCacheGetOrSet1; +extern const Event FileCacheGetOrSet2; +extern const Event FileCacheGetOrSet3; +extern const Event FileCacheGetOrSet4; +extern const Event FileCacheGetImpl; +extern const Event FileCacheGetImpl1; +extern const Event FileCacheGetImpl2; +extern const Event FileCacheGetImpl3; +extern const Event FileCacheGetImpl4; +extern const Event FileCacheGetImpl5; +extern const Event FileCacheLockKey; +} + namespace DB { namespace ErrorCodes @@ -29,20 +46,18 @@ FileCache::FileCache( , max_element_size(cache_settings_.max_elements) , max_file_segment_size(cache_settings_.max_file_segment_size) , allow_persistent_files(cache_settings_.do_not_evict_index_and_mark_files) - , enable_cache_hits_threshold(cache_settings_.enable_cache_hits_threshold) , enable_filesystem_query_cache_limit(cache_settings_.enable_filesystem_query_cache_limit) - , enable_bypass_cache_with_threashold(cache_settings_.enable_bypass_cache_with_threashold) - , bypass_cache_threashold(cache_settings_.bypass_cache_threashold) + , enable_bypass_cache_with_threshold(cache_settings_.enable_bypass_cache_with_threashold) + , bypass_cache_threshold(enable_bypass_cache_with_threshold ? cache_settings_.bypass_cache_threashold : 0) , log(&Poco::Logger::get("FileCache")) , main_priority(std::make_unique()) - , stash_priority(std::make_unique()) - , max_stash_element_size(cache_settings_.max_elements) + , stash(cache_settings_.max_elements, cache_settings_.enable_cache_hits_threshold, std::make_unique()) { } -FileCache::Key FileCache::hash(const String & path) +FileCache::Key FileCache::createKeyForPath(const String & path) { - return Key(sipHash128(path.data(), path.size())); + return Key(path); } String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const @@ -60,7 +75,7 @@ String FileCache::getPathInLocalCache(const Key & key) const return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str; } -void FileCache::removeKeyDirectoryIfExists(const Key & key, std::lock_guard & /* cache_lock */) const +void FileCache::removeKeyDirectoryIfExists(const Key & key, const KeyPrefixGuard::Lock &) const { /// Note: it is guaranteed that there is no concurrency here with files deletion /// because cache key directories are create only in FileCache class under cache_lock. @@ -85,136 +100,82 @@ static bool isQueryInitialized() && !CurrentThread::getQueryId().empty(); } -bool FileCache::isReadOnly() +bool FileCache::readThrowCacheAllowed() { return !isQueryInitialized(); } -void FileCache::assertInitialized(std::lock_guard & /* cache_lock */) const -{ - if (!is_initialized) - { - if (initialization_exception) - std::rethrow_exception(initialization_exception); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache not initialized"); - } -} - -void FileCache::initialize() -{ - std::lock_guard cache_lock(mutex); - if (!is_initialized) - { - if (fs::exists(cache_base_path)) - { - try - { - loadCacheInfoIntoMemory(cache_lock); - } - catch (...) - { - initialization_exception = std::current_exception(); - throw; - } - } - else - { - fs::create_directories(cache_base_path); - } - - is_initialized = true; - } -} - -void FileCache::useCell( - const FileSegmentCell & cell, FileSegments & result, std::lock_guard & cache_lock) -{ - auto file_segment = cell.file_segment; - - if (file_segment->isDownloaded()) - { - fs::path path = file_segment->getPathInLocalCache(); - if (!fs::exists(path)) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "File path does not exist, but file has DOWNLOADED state. {}", - file_segment->getInfoForLog()); - } - - if (fs::file_size(path) == 0) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot have zero size downloaded file segments. {}", - file_segment->getInfoForLog()); - } - } - - result.push_back(cell.file_segment); - - /** - * A cell receives a queue iterator on first successful space reservation attempt - * (space is reserved incrementally on each read buffer nextImpl() call). - */ - if (cell.queue_iterator) - { - /// Move to the end of the queue. The iterator remains valid. - cell.queue_iterator->use(cache_lock); - } -} - -FileCache::FileSegmentCell * FileCache::getCell( - const Key & key, size_t offset, std::lock_guard & /* cache_lock */) -{ - auto it = files.find(key); - if (it == files.end()) - return nullptr; - - auto & offsets = it->second; - auto cell_it = offsets.find(offset); - if (cell_it == offsets.end()) - return nullptr; - - return &cell_it->second; -} - FileSegments FileCache::getImpl( - const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock) + const Key & key, const FileSegment::Range & range, const KeyTransaction & key_transaction) const { /// Given range = [left, right] and non-overlapping ordered set of file segments, /// find list [segment1, ..., segmentN] of segments which intersect with given range. - FileSegments result; + ElapsedUSProfileEventIncrement increment(ProfileEvents::FileCacheGetImpl); - if (enable_bypass_cache_with_threashold && (range.size() > bypass_cache_threashold)) + if (bypass_cache_threshold && range.size() > bypass_cache_threshold) { auto file_segment = std::make_shared( - range.left, range.size(), key, this, FileSegment::State::SKIP_CACHE, CreateFileSegmentSettings{}); - { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detachAssumeStateFinalized(segment_lock); - } - result.emplace_back(file_segment); - return result; + range.left, range.size(), key, nullptr, nullptr, + FileSegment::State::SKIP_CACHE, CreateFileSegmentSettings{}); + return { file_segment }; } - auto it = files.find(key); - if (it == files.end()) - return {}; - - const auto & file_segments = it->second; + const auto & file_segments = key_transaction.getOffsets(); if (file_segments.empty()) - { - files.erase(key); - removeKeyDirectoryIfExists(key, cache_lock); return {}; - } + + ElapsedUSProfileEventIncrement increment1(ProfileEvents::FileCacheGetImpl1); + + FileSegments result; + auto add_to_result = [&](const FileSegmentCell & cell) + { + if (cell.file_segment->isDownloaded()) + { + if (cell.file_segment->getDownloadedSize() == 0) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot have zero size downloaded file segments. {}", + cell.file_segment->getInfoForLog()); + } + + #ifndef NDEBUG + /** + * Check that in-memory state of the cache is consistent with the state on disk. + * Check only in debug build, because such checks can be done often and can be quite + * expensive compared to overall query execution time. + */ + + fs::path path = cell.file_segment->getPathInLocalCache(); + if (!fs::exists(path)) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "File path does not exist, but file has DOWNLOADED state. {}", + cell.file_segment->getInfoForLog()); + } + + if (fs::file_size(path) == 0) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot have zero size downloaded file segments. {}", + cell.file_segment->getInfoForLog()); + } + #endif + } + auto state = cell.file_segment->state(); + if (state == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + std::terminate(); + + result.push_back(cell.file_segment); + }; auto segment_it = file_segments.lower_bound(range.left); if (segment_it == file_segments.end()) { + ElapsedUSProfileEventIncrement increment2(ProfileEvents::FileCacheGetImpl2); /// N - last cached segment for given file key, segment{N}.offset < range.left: /// segment{N} segment{N} /// [________ [_______] @@ -226,10 +187,11 @@ FileSegments FileCache::getImpl( if (cell.file_segment->range().right < range.left) return {}; - useCell(cell, result, cache_lock); + add_to_result(cell); } else /// segment_it <-- segmment{k} { + ElapsedUSProfileEventIncrement increment3(ProfileEvents::FileCacheGetImpl3); if (segment_it != file_segments.begin()) { const auto & prev_cell = std::prev(segment_it)->second; @@ -242,7 +204,7 @@ FileSegments FileCache::getImpl( /// [___________ /// ^ /// range.left - useCell(prev_cell, result, cache_lock); + add_to_result(prev_cell); } } @@ -252,13 +214,14 @@ FileSegments FileCache::getImpl( /// ^ ^ ^ segment{k}.offset /// range.left range.left range.right + ElapsedUSProfileEventIncrement increment4(ProfileEvents::FileCacheGetImpl4); while (segment_it != file_segments.end()) { const auto & cell = segment_it->second; if (range.right < cell.file_segment->range().left) break; - useCell(cell, result, cache_lock); + add_to_result(cell); ++segment_it; } } @@ -272,7 +235,7 @@ FileSegments FileCache::splitRangeIntoCells( size_t size, FileSegment::State state, const CreateFileSegmentSettings & settings, - std::lock_guard & cache_lock) + KeyTransaction & key_transaction) { assert(size > 0); @@ -288,10 +251,8 @@ FileSegments FileCache::splitRangeIntoCells( current_cell_size = std::min(remaining_size, max_file_segment_size); remaining_size -= current_cell_size; - auto * cell = addCell(key, current_pos, current_cell_size, state, settings, cache_lock); - if (cell) - file_segments.push_back(cell->file_segment); - assert(cell); + auto cell_it = addCell(key, current_pos, current_cell_size, state, settings, key_transaction); + file_segments.push_back(cell_it->second.file_segment); current_pos += current_cell_size; } @@ -306,7 +267,7 @@ void FileCache::fillHolesWithEmptyFileSegments( const FileSegment::Range & range, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings, - std::lock_guard & cache_lock) + KeyTransaction & key_transaction) { /// There are segments [segment1, ..., segmentN] /// (non-overlapping, non-empty, ascending-ordered) which (maybe partially) @@ -318,6 +279,8 @@ void FileCache::fillHolesWithEmptyFileSegments( /// /// For each such hole create a cell with file segment state EMPTY. + assert(!file_segments.empty()); + auto it = file_segments.begin(); auto segment_range = (*it)->range(); @@ -352,16 +315,18 @@ void FileCache::fillHolesWithEmptyFileSegments( if (fill_with_detached_file_segments) { - auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings); - { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detachAssumeStateFinalized(segment_lock); - } + auto file_segment = std::make_shared( + current_pos, hole_size, key, + key_transaction.getCreator(), + this, FileSegment::State::SKIP_CACHE, settings); + file_segments.insert(it, file_segment); } else { - file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock)); + file_segments.splice( + it, splitRangeIntoCells( + key, current_pos, hole_size, FileSegment::State::EMPTY, settings, key_transaction)); } current_pos = segment_range.right + 1; @@ -379,399 +344,477 @@ void FileCache::fillHolesWithEmptyFileSegments( if (fill_with_detached_file_segments) { - auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings); - { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detachAssumeStateFinalized(segment_lock); - } + auto file_segment = std::make_shared( + current_pos, hole_size, key, + key_transaction.getCreator(), + this, FileSegment::State::SKIP_CACHE, settings); + file_segments.insert(file_segments.end(), file_segment); } else { file_segments.splice( file_segments.end(), - splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock)); + splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, key_transaction)); } } } -FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings) +KeyTransactionPtr FileCache::createKeyTransaction(const Key & key, KeyNotFoundPolicy key_not_found_policy, bool assert_initialized) { - std::lock_guard cache_lock(mutex); + ElapsedUSProfileEventIncrement increment(ProfileEvents::FileCacheLockKey); + auto lock = cache_guard.lock(); - assertInitialized(cache_lock); + if (assert_initialized) + assertInitializedUnlocked(lock); -#ifndef NDEBUG - assertCacheCorrectness(key, cache_lock); -#endif - - FileSegment::Range range(offset, offset + size - 1); - /// Get all segments which intersect with the given range. - auto file_segments = getImpl(key, range, cache_lock); - - if (file_segments.empty()) + auto it = files.find(key); + if (it == files.end()) { - file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); - } - else - { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock); - } - assert(!file_segments.empty()); - return FileSegmentsHolder(std::move(file_segments)); -} - -FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size) -{ - std::lock_guard cache_lock(mutex); - - assertInitialized(cache_lock); - -#ifndef NDEBUG - assertCacheCorrectness(key, cache_lock); -#endif - - FileSegment::Range range(offset, offset + size - 1); - - /// Get all segments which intersect with the given range. - auto file_segments = getImpl(key, range, cache_lock); - - if (file_segments.empty()) - { - auto file_segment = std::make_shared( - offset, size, key, this, FileSegment::State::EMPTY, CreateFileSegmentSettings{}); + switch (key_not_found_policy) { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detachAssumeStateFinalized(segment_lock); + case KeyNotFoundPolicy::THROW: + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key.toString()); + } + case KeyNotFoundPolicy::RETURN_NULL: + { + return nullptr; + } + case KeyNotFoundPolicy::CREATE_EMPTY: + { + it = files.emplace(key, std::make_shared()).first; + break; + } } - file_segments = { file_segment }; + } + + auto lock_it = keys_locks.find(key.key_prefix); + if (lock_it == keys_locks.end()) + { + lock_it = keys_locks.emplace(key.key_prefix, std::make_shared()).first; + } + + return std::make_unique(lock_it->second, it->second); +} + +FileSegmentsHolderPtr FileCache::getOrSet( + const Key & key, + size_t offset, + size_t size, + const CreateFileSegmentSettings & settings) +{ + ElapsedUSProfileEventIncrement increment(ProfileEvents::FileCacheGetOrSet); + + FileSegment::Range range(offset, offset + size - 1); + + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::CREATE_EMPTY); + + /// Get all segments which intersect with the given range. + auto file_segments = getImpl(key, range, *key_transaction); + if (file_segments.empty()) + { + file_segments = splitRangeIntoCells( + key, offset, size, FileSegment::State::EMPTY, settings, *key_transaction); } else { - fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, {}, cache_lock); + fillHolesWithEmptyFileSegments( + file_segments, key, range, /* fill_with_detached */false, settings, *key_transaction); } - return FileSegmentsHolder(std::move(file_segments)); + chassert(!file_segments.empty()); + return std::make_unique(std::move(file_segments)); } -FileCache::FileSegmentCell * FileCache::addCell( - const Key & key, size_t offset, size_t size, - FileSegment::State state, const CreateFileSegmentSettings & settings, - std::lock_guard & cache_lock) +FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size) +{ + /// Get all segments which intersect with the given range. + + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::RETURN_NULL); + if (key_transaction) + { + FileSegment::Range range(offset, offset + size - 1); + auto file_segments = getImpl(key, range, *key_transaction); + if (!file_segments.empty()) + { + fillHolesWithEmptyFileSegments( + file_segments, key, range, /* fill_with_detached */true, + CreateFileSegmentSettings{}, *key_transaction); + + return std::make_unique(std::move(file_segments)); + } + } + + auto file_segment = std::make_shared( + offset, size, key, + nullptr, this, FileSegment::State::SKIP_CACHE, CreateFileSegmentSettings{}); + + return std::make_unique(FileSegments{file_segment}); +} + +FileCache::CacheCells::iterator FileCache::addCell( + const Key & key, + size_t offset, + size_t size, + FileSegment::State state, + const CreateFileSegmentSettings & settings, + KeyTransaction & key_transaction) { /// Create a file segment cell and put it in `files` map by [key][offset]. - if (!size) - return nullptr; /// Empty files are not cached. + chassert(size > 0); /// Empty cells are not allowed. + chassert(!key.key_prefix.empty()); - if (files[key].contains(offset)) + auto it = key_transaction.getOffsets().find(offset); + if (it != key_transaction.getOffsets().end()) + { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Cache cell already exists for key: `{}`, offset: {}, size: {}.\nCurrent cache structure: {}", - key.toString(), offset, size, dumpStructureUnlocked(key, cache_lock)); - - auto skip_or_download = [&]() -> FileSegmentPtr - { - FileSegment::State result_state = state; - if (state == FileSegment::State::EMPTY && enable_cache_hits_threshold) - { - auto record = stash_records.find({key, offset}); - - if (record == stash_records.end()) - { - auto priority_iter = stash_priority->add(key, offset, 0, cache_lock); - stash_records.insert({{key, offset}, priority_iter}); - - if (stash_priority->getElementsNum(cache_lock) > max_stash_element_size) - { - auto remove_priority_iter = stash_priority->getLowestPriorityWriteIterator(cache_lock); - stash_records.erase({remove_priority_iter->key(), remove_priority_iter->offset()}); - remove_priority_iter->removeAndGetNext(cache_lock); - } - - /// For segments that do not reach the download threshold, - /// we do not download them, but directly read them - result_state = FileSegment::State::SKIP_CACHE; - } - else - { - auto priority_iter = record->second; - priority_iter->use(cache_lock); - - result_state = priority_iter->hits() >= enable_cache_hits_threshold - ? FileSegment::State::EMPTY - : FileSegment::State::SKIP_CACHE; - } - } - - return std::make_shared(offset, size, key, this, result_state, settings); - }; - - FileSegmentCell cell(skip_or_download(), this, cache_lock); - auto & offsets = files[key]; - - if (offsets.empty()) - { - auto key_path = getPathInLocalCache(key); - - if (!fs::exists(key_path)) - fs::create_directories(key_path); + "Cache cell already exists for key: `{}`, offset: {}, size: {}.", + key.toString(), offset, size); } - auto [it, inserted] = offsets.insert({offset, std::move(cell)}); - if (!inserted) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Failed to insert into cache key: `{}`, offset: {}, size: {}", - key.toString(), offset, size); + FileSegment::State result_state; - return &(it->second); + /// `stash` - a queue of "stashed" key-offset pairs. Implements counting of + /// cache entries and allows caching only if cache hit threadhold is reached. + if (stash.cache_hits_threshold && state == FileSegment::State::EMPTY) + { + // auto stash_lock = stash.lock(); + // KeyAndOffset stash_key(key, offset); + + // auto record_it = stash.records.find(stash_key); + // if (record_it == stash.records.end()) + // { + // auto & stash_queue = *stash.queue; + // auto & stash_records = stash.records; + + // stash_records.insert({stash_key, stash_queue.add(key, offset, 0, key_transaction.getCreator(), lock)}); + + // if (stash_queue.getElementsNum(lock) > stash.max_stash_queue_size) + // stash_records.erase(stash_queue.pop(lock)); + + // result_state = FileSegment::State::SKIP_CACHE; + // } + // else + // { + // result_state = record_it->second->use() >= stash.cache_hits_threshold + // ? FileSegment::State::EMPTY + // : FileSegment::State::SKIP_CACHE; + // } + } + else + { + result_state = state; + } + + auto file_segment = std::make_shared( + offset, size, key, key_transaction.getCreator(), this, result_state, settings); + + FileSegmentCell cell(std::move(file_segment), key_transaction, *main_priority); + + auto [cell_it, inserted] = key_transaction.getOffsets().emplace(offset, std::move(cell)); + assert(inserted); + + return cell_it; } FileSegmentPtr FileCache::createFileSegmentForDownload( const Key & key, size_t offset, size_t size, - const CreateFileSegmentSettings & settings, - std::lock_guard & cache_lock) + const CreateFileSegmentSettings & settings) { -#ifndef NDEBUG - assertCacheCorrectness(key, cache_lock); -#endif - if (size > max_file_segment_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Requested size exceeds max file segment size"); - auto * cell = getCell(key, offset, cache_lock); - if (cell) + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::CREATE_EMPTY); + + if (key_transaction->getOffsets().contains(offset)) + { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cache cell already exists for key `{}` and offset {}", key.toString(), offset); + } - cell = addCell(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock); - - if (!cell) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to add a new cell for download"); - - return cell->file_segment; + auto cell_it = addCell( + key, offset, size, FileSegment::State::EMPTY, settings, *key_transaction); + return cell_it->second.file_segment; } -bool FileCache::tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock) +bool FileCache::tryReserve(const Key & key, size_t offset, size_t size) { - auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr; + auto main_priority_lock = main_priority->lockShared(); + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::THROW); + key_transaction->queue_lock = main_priority_lock; + return tryReserveUnlocked(key, offset, size, *key_transaction); +} + +bool FileCache::tryReserveUnlocked( + const Key & key, + size_t offset, + size_t size, + KeyTransaction & key_transaction) +{ + // auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr; + QueryContextPtr query_context = nullptr; + + bool reserved; if (!query_context) { - return tryReserveForMainList(key, offset, size, nullptr, cache_lock); + reserved = tryReserveInCache(key, offset, size, nullptr, key_transaction); } /// The maximum cache capacity of the request is not reached, thus the - //// cache block is evicted from the main LRU queue by tryReserveForMainList(). + //// cache block is evicted from the main LRU queue by tryReserveInCache(). else if (query_context->getCacheSize() + size <= query_context->getMaxCacheSize()) { - return tryReserveForMainList(key, offset, size, query_context, cache_lock); + reserved = tryReserveInCache(key, offset, size, query_context, key_transaction); } /// When skip_download_if_exceeds_query_cache is true, there is no need /// to evict old data, skip the cache and read directly from remote fs. else if (query_context->isSkipDownloadIfExceed()) { - return false; + reserved = false; } /// The maximum cache size of the query is reached, the cache will be /// evicted from the history cache accessed by the current query. else { - size_t removed_size = 0; - size_t queue_size = main_priority->getElementsNum(cache_lock); - - auto * cell_for_reserve = getCell(key, offset, cache_lock); - - struct Segment - { - Key key; - size_t offset; - size_t size; - - Segment(Key key_, size_t offset_, size_t size_) - : key(key_), offset(offset_), size(size_) {} - }; - - std::vector ghost; - std::vector trash; - std::vector to_evict; - - auto is_overflow = [&] - { - return (max_size != 0 && main_priority->getCacheSize(cache_lock) + size - removed_size > max_size) - || (max_element_size != 0 && queue_size > max_element_size) - || (query_context->getCacheSize() + size - removed_size > query_context->getMaxCacheSize()); - }; - - /// Select the cache from the LRU queue held by query for expulsion. - for (auto iter = query_context->getPriority()->getLowestPriorityWriteIterator(cache_lock); iter->valid();) - { - if (!is_overflow()) - break; - - auto * cell = getCell(iter->key(), iter->offset(), cache_lock); - - if (!cell) - { - /// The cache corresponding to this record may be swapped out by - /// other queries, so it has become invalid. - removed_size += iter->size(); - ghost.push_back(Segment(iter->key(), iter->offset(), iter->size())); - /// next() - iter->removeAndGetNext(cache_lock); - } - else - { - size_t cell_size = cell->size(); - assert(iter->size() == cell_size); - - if (cell->releasable()) - { - auto & file_segment = cell->file_segment; - - if (file_segment->isPersistent() && allow_persistent_files) - { - LOG_DEBUG(log, "File segment will not be removed, because it is persistent: {}", file_segment->getInfoForLog()); - continue; - } - - std::lock_guard segment_lock(file_segment->mutex); - - switch (file_segment->download_state) - { - case FileSegment::State::DOWNLOADED: - { - to_evict.push_back(cell); - break; - } - default: - { - trash.push_back(cell); - break; - } - } - removed_size += cell_size; - --queue_size; - } - - iter->next(); - } - } - - auto remove_file_segment = [&](FileSegmentPtr file_segment, size_t file_segment_size) - { - query_context->remove(file_segment->key(), file_segment->offset(), file_segment_size, cache_lock); - remove(file_segment, cache_lock); - }; - - assert(trash.empty()); - for (auto & cell : trash) - { - if (auto file_segment = cell->file_segment) - remove_file_segment(file_segment, cell->size()); - } - - for (auto & entry : ghost) - query_context->remove(entry.key, entry.offset, entry.size, cache_lock); - - if (is_overflow()) - return false; - - if (cell_for_reserve) - { - auto queue_iterator = cell_for_reserve->queue_iterator; - if (queue_iterator) - queue_iterator->incrementSize(size, cache_lock); - else - cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, cache_lock); - } - - for (auto & cell : to_evict) - { - if (auto file_segment = cell->file_segment) - remove_file_segment(file_segment, cell->size()); - } - - query_context->reserve(key, offset, size, cache_lock); - return true; + reserved = tryReserveInQueryCache(key, offset, size, query_context, key_transaction); } + + if (reserved && !key_transaction.getOffsets().created_base_directory) + { + fs::create_directories(getPathInLocalCache(key)); + key_transaction.getOffsets().created_base_directory = true; + } + return reserved; } -bool FileCache::tryReserveForMainList( - const Key & key, size_t offset, size_t size, QueryContextPtr query_context, std::lock_guard & cache_lock) +bool FileCache::tryReserveInQueryCache( + const Key &, size_t, size_t, QueryContextPtr, KeyTransaction &) { - auto removed_size = 0; - size_t queue_size = main_priority->getElementsNum(cache_lock); - assert(queue_size <= max_element_size); +// auto * cell_for_reserve = key_transaction.getOffsets().get(offset); +// +// struct Segment +// { +// Key key; +// size_t offset; +// size_t size; +// +// Segment(Key key_, size_t offset_, size_t size_) +// : key(key_), offset(offset_), size(size_) {} +// }; +// +// std::vector ghost; +// std::vector trash; +// std::vector to_evict; +// +// auto & query_priority_queue = query_context->getPriority(); +// +// auto main_priority_lock = main_priority->lock(); +// auto query_priority_lock = query_priority_queue.lock(); +// +// size_t queue_size = main_priority->getElementsNum(); +// size_t removed_size = 0; +// +// auto is_overflow = [&] +// { +// return (max_size != 0 && main_priority->getCacheSize() + size - removed_size > max_size) +// || (max_element_size != 0 && queue_size > max_element_size) +// || (query_context->getCacheSize() + size - removed_size > query_context->getMaxCacheSize()); +// }; +// +// /// Select the cache from the LRU queue held by query for expulsion. +// for (auto iter = query_priority_queue.getLowestPriorityWriteIterator(); iter->valid();) +// { +// if (!is_overflow()) +// break; +// +// auto * cell = key_transaction.getOffsets().tryGet(iter->offset()); +// +// if (!cell) +// { +// /// The cache corresponding to this record may be swapped out by +// /// other queries, so it has become invalid. +// removed_size += iter->size(); +// ghost.push_back(Segment(iter->key(), iter->offset(), iter->size())); +// /// next() +// iter->removeAndGetNext(); +// } +// else +// { +// size_t cell_size = cell->size(); +// assert(iter->size() == cell_size); +// +// if (cell->releasable()) +// { +// auto & file_segment = cell->file_segment; +// +// if (file_segment->isPersistent() && allow_persistent_files) +// continue; +// +// switch (file_segment->state()) +// { +// case FileSegment::State::DOWNLOADED: +// { +// to_evict.push_back(cell); +// break; +// } +// default: +// { +// trash.push_back(cell); +// break; +// } +// } +// removed_size += cell_size; +// --queue_size; +// } +// +// iter->next(); +// } +// } +// +// // auto remove_file_segment = [&](FileSegmentPtr file_segment, size_t file_segment_size) +// // { +// // /// FIXME: key transaction is incorrect +// // query_context->remove(file_segment->key(), file_segment->offset(), file_segment_size, key_transaction); +// // remove(file_segment); +// // }; +// +// assert(trash.empty()); +// //for (auto & cell : trash) +// //{ +// // if (auto file_segment = cell->file_segment) +// // // remove_file_segment(file_segment, cell->size()); +// //} +// +// for (auto & entry : ghost) +// /// FIXME: key transaction is incorrect +// query_context->remove(entry.key, entry.offset, entry.size, key_transaction); +// +// if (is_overflow()) +// return false; +// +// if (cell_for_reserve) +// { +// auto queue_iterator = cell_for_reserve->queue_iterator; +// if (queue_iterator) +// queue_iterator->incrementSize(size); +// else +// cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, key_transaction.getCreator()); +// } +// +// // for (auto & cell : to_evict) +// // { +// // if (auto file_segment = cell->file_segment) +// // remove_file_segment(file_segment, cell->size()); +// // } +// +// query_context->reserve(key, offset, size, key_transaction); +// +// if (reserved && !key_transaction.getOffsets().created_base_directory) +// { +// fs::create_directories(getPathInLocalCache(key)); +// key_transaction.getOffsets().created_base_directory = true; +// } + return true; +} - /// Since space reservation is incremental, cache cell already exists if it's state is EMPTY. - /// And it cache cell does not exist on startup -- as we first check for space and then add a cell. - auto * cell_for_reserve = getCell(key, offset, cache_lock); +bool FileCache::tryReserveInCache( + const Key & key, + size_t offset, + size_t size, + QueryContextPtr query_context, + KeyTransaction & key_transaction) +{ + LOG_TEST(log, "Reserving space for {}:{}", key.toString(), offset); + const auto & lock = key_transaction.getQueueLock(); + + size_t queue_size = main_priority->getElementsNum(lock); + chassert(queue_size <= max_element_size); + + auto * cell_for_reserve = key_transaction.getOffsets().tryGet(offset); /// A cell acquires a LRUQueue iterator on first successful space reservation attempt. - /// cell_for_reserve can be nullptr here when we call tryReserve() from loadCacheInfoIntoMemory(). if (!cell_for_reserve || !cell_for_reserve->queue_iterator) queue_size += 1; + size_t removed_size = 0; auto is_overflow = [&] { + size_t expected_size = main_priority->getCacheSize(lock) + size - removed_size; + LOG_TEST(log, "SIZE: {}/{} ({})", expected_size, max_size, key.toString()); /// max_size == 0 means unlimited cache size, max_element_size means unlimited number of cache elements. - return (max_size != 0 && main_priority->getCacheSize(cache_lock) + size - removed_size > max_size) + return (max_size != 0 && expected_size > max_size) || (max_element_size != 0 && queue_size > max_element_size); }; - std::vector to_evict; - std::vector trash; + using KeyTransactionsMap = std::unordered_map; + KeyTransactionsMap locked_transactions; + std::vector locked_transactions_holder; + locked_transactions.emplace(key.key_prefix, key_transaction); + std::optional tmp_transaction; - for (auto it = main_priority->getLowestPriorityReadIterator(cache_lock); it->valid(); it->next()) + auto it = main_priority->getLowestPriorityIterator(lock); + for (; it->valid(lock) && is_overflow();) { - const auto & entry_key = it->key(); - auto entry_offset = it->offset(); + tmp_transaction.reset(); + KeyTransaction * current_key_transaction; - if (!is_overflow()) - break; + auto locked_it = locked_transactions.find(it->key().key_prefix); + if (locked_it == locked_transactions.end()) + { + tmp_transaction = it->createKeyTransaction(lock); + current_key_transaction = tmp_transaction->get(); + } + else + current_key_transaction = &locked_it->second; - auto * cell = getCell(entry_key, entry_offset, cache_lock); - if (!cell) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cache became inconsistent. Key: {}, offset: {}", - key.toString(), offset); + auto * cell = current_key_transaction->getOffsets().get(it->offset()); - size_t cell_size = cell->size(); - assert(it->size() == cell_size); + chassert(it->size() == cell->size()); + const size_t cell_size = cell->size(); + bool remove_current_it = false; /// It is guaranteed that cell is not removed from cache as long as /// pointer to corresponding file segment is hold by any other thread. if (cell->releasable()) { - auto & file_segment = cell->file_segment; + auto file_segment = cell->file_segment; + chassert(it->offset() == file_segment->offset()); if (file_segment->isPersistent() && allow_persistent_files) { - LOG_DEBUG(log, "File segment will not be removed, because it is persistent: {}", file_segment->getInfoForLog()); + it->next(lock); continue; } - std::lock_guard segment_lock(file_segment->mutex); - - switch (file_segment->download_state) + switch (file_segment->state()) { case FileSegment::State::DOWNLOADED: { - /// Cell will actually be removed only if - /// we managed to reserve enough space. + /// Cell will actually be removed only if we managed to reserve enough space. - to_evict.push_back(cell); + if (tmp_transaction) + { + locked_transactions_holder.push_back(std::move(*tmp_transaction)); + locked_transactions.emplace(it->key().key_prefix, *locked_transactions_holder.back()); + current_key_transaction->queue_lock = key_transaction.queue_lock; + } + + current_key_transaction->delete_offsets.push_back(file_segment->offset()); break; } default: { - trash.push_back(cell); + remove_current_it = true; + cell->queue_iterator = {}; + current_key_transaction->remove(file_segment); break; } } @@ -779,21 +822,16 @@ bool FileCache::tryReserveForMainList( removed_size += cell_size; --queue_size; } - } - /// This case is very unlikely, can happen in case of exception from - /// file_segment->complete(), which would be a logical error. - assert(trash.empty()); - for (auto & cell : trash) - { - if (auto file_segment = cell->file_segment) - remove(file_segment, cache_lock); + if (remove_current_it) + it = it->remove(lock); + else + it->next(lock); } if (is_overflow()) return false; - /// cache cell is nullptr on server startup because we first check for space and then add a cell. if (cell_for_reserve) { /// queue_iteratir is std::nullopt here if no space has been reserved yet, a cache cell @@ -801,152 +839,127 @@ bool FileCache::tryReserveForMainList( /// If queue iterator already exists, we need to update the size after each space reservation. auto queue_iterator = cell_for_reserve->queue_iterator; if (queue_iterator) - queue_iterator->incrementSize(size, cache_lock); + queue_iterator->incrementSize(size, lock); else - cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, cache_lock); + { + /// Space reservation is incremental, so cache cell is created first (with state empty), + /// and queue_iterator is assigned on first space reservation attempt. + cell_for_reserve->queue_iterator = main_priority->add(key, offset, size, key_transaction.getCreator(), lock); + } } - for (auto & cell : to_evict) + for (auto & [_, transaction] : locked_transactions) { - if (auto file_segment = cell->file_segment) - remove(file_segment, cache_lock); + for (const auto & offset_to_delete : transaction.delete_offsets) + { + auto * cell = transaction.getOffsets().get(offset_to_delete); + transaction.remove(cell->file_segment); + } } - if (main_priority->getCacheSize(cache_lock) > (1ull << 63)) + if (main_priority->getCacheSize(lock) > (1ull << 63)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache became inconsistent. There must be a bug"); if (query_context) - query_context->reserve(key, offset, size, cache_lock); + query_context->reserve(key, offset, size, key_transaction); return true; } void FileCache::removeIfExists(const Key & key) { - std::lock_guard cache_lock(mutex); - - assertInitialized(cache_lock); - - auto it = files.find(key); - if (it == files.end()) + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::RETURN_NULL); + if (!key_transaction) return; - auto & offsets = it->second; + auto & offsets = key_transaction->getOffsets(); + // bool remove_directory = true; - std::vector to_remove; - to_remove.reserve(offsets.size()); - - for (auto & [offset, cell] : offsets) - to_remove.push_back(&cell); - - bool some_cells_were_skipped = false; - for (auto & cell : to_remove) + if (!offsets.empty()) { - /// In ordinary case we remove data from cache when it's not used by anyone. - /// But if we have multiple replicated zero-copy tables on the same server - /// it became possible to start removing something from cache when it is used - /// by other "zero-copy" tables. That is why it's not an error. - if (!cell->releasable()) - { - some_cells_were_skipped = true; - continue; - } + std::vector remove_cells; + remove_cells.reserve(offsets.size()); + for (auto & [offset, cell] : offsets) + remove_cells.push_back(&cell); - auto file_segment = cell->file_segment; - if (file_segment) + for (auto & cell : remove_cells) { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detach(cache_lock, segment_lock); - remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); + /// In ordinary case we remove data from cache when it's not used by anyone. + /// But if we have multiple replicated zero-copy tables on the same server + /// it became possible to start removing something from cache when it is used + /// by other "zero-copy" tables. That is why it's not an error. + if (!cell->releasable()) + { + // remove_directory = false; + continue; + } + + key_transaction->remove(cell->file_segment); } } - if (!some_cells_were_skipped) - { - files.erase(key); - removeKeyDirectoryIfExists(key, cache_lock); - } + // if (remove_directory) + // { + // files.erase(key); + // removeKeyDirectoryIfExists(key, lock); + // } } -void FileCache::removeIfReleasable() +void FileCache::removeAllReleasable() { /// Try remove all cached files by cache_base_path. /// Only releasable file segments are evicted. /// `remove_persistent_files` defines whether non-evictable by some criteria files /// (they do not comply with the cache eviction policy) should also be removed. - std::lock_guard cache_lock(mutex); - + auto lock = main_priority->lockShared(); std::vector to_remove; - for (auto it = main_priority->getLowestPriorityReadIterator(cache_lock); it->valid(); it->next()) - { - const auto & key = it->key(); - auto offset = it->offset(); - auto * cell = getCell(key, offset, cache_lock); - if (!cell) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cache is in inconsistent state: LRU queue contains entries with no cache cell"); - } + for (auto it = main_priority->getLowestPriorityIterator(*lock); it->valid(*lock);) + { + auto key_transaction = it->createKeyTransaction(*lock); + auto * cell = key_transaction->getOffsets().get(it->offset()); if (cell->releasable()) { - auto file_segment = cell->file_segment; - - if (file_segment) - { - to_remove.emplace_back(file_segment); - } + it = it->remove(*lock); + cell->queue_iterator = {}; + key_transaction->remove(cell->file_segment); } - } - - for (auto & file_segment : to_remove) - { - std::unique_lock segment_lock(file_segment->mutex); - file_segment->detach(cache_lock, segment_lock); - remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); + else + it->next(*lock); } /// Remove all access information. - stash_records.clear(); - stash_priority->removeAll(cache_lock); + stash.records.clear(); + stash.queue->removeAll(*lock); -#ifndef NDEBUG - assertCacheCorrectness(cache_lock); -#endif +//#ifndef NDEBUG +// assertCacheCorrectness(cache_lock); +//#endif } -void FileCache::remove(FileSegmentPtr file_segment, std::lock_guard & cache_lock) +void KeyTransaction::remove(FileSegmentPtr file_segment) { - std::unique_lock segment_lock(file_segment->mutex); - remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); + /// We must hold pointer to file segment while removing it. + remove(file_segment->key(), file_segment->offset(), file_segment->lock()); } -void FileCache::remove( - Key key, size_t offset, - std::lock_guard & cache_lock, std::unique_lock & /* segment_lock */) +void KeyTransaction::remove(const Key & key, size_t offset, const FileSegmentGuard::Lock & segment_lock) { - LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset); + LOG_DEBUG( + log, "Remove from cache. Key: {}, offset: {}", + key.toString(), offset); - String cache_file_path; + auto * cell = offsets->get(offset); - { - auto * cell = getCell(key, offset, cache_lock); - if (!cell) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No cache cell for key: {}, offset: {}", key.toString(), offset); + if (cell->queue_iterator) + cell->queue_iterator->remove(getQueueLock()); - if (cell->queue_iterator) - { - cell->queue_iterator->removeAndGetNext(cache_lock); - } + const auto cache_file_path = cell->file_segment->getPathInLocalCache(); + cell->file_segment->detach(segment_lock, *this); - cache_file_path = cell->file_segment->getPathInLocalCache(); - } - - auto & offsets = files[key]; - offsets.erase(offset); + offsets->erase(offset); if (fs::exists(cache_file_path)) { @@ -954,11 +967,11 @@ void FileCache::remove( { fs::remove(cache_file_path); - if (is_initialized && offsets.empty()) - { - files.erase(key); - removeKeyDirectoryIfExists(key, cache_lock); - } + // if (is_initialized && offsets.empty()) + // { + // files.erase(key); + // removeKeyDirectoryIfExists(key, cache_lock); + // } } catch (...) { @@ -970,20 +983,23 @@ void FileCache::remove( } } -void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock) +void FileCache::loadCacheInfoIntoMemory() { - Key key; + auto queue_lock = main_priority->lockShared(); + UInt64 offset = 0; size_t size = 0; - std::vector>> queue_entries; + std::vector>> queue_entries; /// cache_base_path / key_prefix / key / offset if (!files.empty()) + { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cache initialization is partially made. " "This can be a result of a failed first attempt to initialize cache. " "Please, check log for error messages"); + } fs::directory_iterator key_prefix_it{cache_base_path}; for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it) @@ -991,7 +1007,11 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock if (!key_prefix_it->is_directory()) { if (key_prefix_it->path().filename() != "status") - LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_prefix_it->path().string()); + { + LOG_DEBUG( + log, "Unexpected file {} (not a directory), will skip it", + key_prefix_it->path().string()); + } continue; } @@ -1000,11 +1020,15 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock { if (!key_it->is_directory()) { - LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + LOG_DEBUG( + log, + "Unexpected file: {} (not a directory). Expected a directory", + key_it->path().string()); continue; } - key = Key(unhexUInt(key_it->path().filename().string().data())); + auto key = Key(unhexUInt(key_it->path().filename().string().data())); + fs::directory_iterator offset_it{key_it->path()}; for (; offset_it != fs::directory_iterator(); ++offset_it) { @@ -1034,21 +1058,25 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock continue; } - if (tryReserve(key, offset, size, cache_lock)) - { - auto * cell = addCell( - key, offset, size, FileSegment::State::DOWNLOADED, - CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock); + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::CREATE_EMPTY, false); + key_transaction->queue_lock = queue_lock; + key_transaction->getOffsets().created_base_directory = true; - if (cell) - queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); + if (tryReserveUnlocked(key, offset, size, *key_transaction)) + { + auto cell_it = addCell( + key, offset, size, FileSegment::State::DOWNLOADED, + CreateFileSegmentSettings{ .is_persistent = is_persistent }, *key_transaction); + + queue_entries.emplace_back(cell_it->second.queue_iterator, cell_it->second.file_segment); } else { LOG_WARNING( log, - "Cache capacity changed (max size: {}, available: {}), cached file `{}` does not fit in cache anymore (size: {})", - max_size, getAvailableCacheSizeUnlocked(cache_lock), key_it->path().string(), size); + "Cache capacity changed (max size: {}, used: {}), " + "cached file `{}` does not fit in cache anymore (size: {})", + max_size, getUsedCacheSize(), key_it->path().string(), size); fs::remove(offset_it->path()); } @@ -1067,32 +1095,23 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock if (file_segment.expired()) continue; - it->use(cache_lock); + it->use(*queue_lock); } -#ifndef NDEBUG - assertCacheCorrectness(cache_lock); -#endif + +//#ifndef NDEBUG +// auto cache_lock = cache_guard.lock(); +// assertCacheCorrectness(cache_lock); +//#endif } -void FileCache::reduceSizeToDownloaded( - const Key & key, size_t offset, - std::lock_guard & cache_lock, std::unique_lock & segment_lock) +void KeyTransaction::reduceSizeToDownloaded(const Key & key, size_t offset, const FileSegmentGuard::Lock & segment_lock) { /** * In case file was partially downloaded and it's download cannot be continued * because of no space left in cache, we need to be able to cut cell's size to downloaded_size. */ - auto * cell = getCell(key, offset, cache_lock); - - if (!cell) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "No cell found for key: {}, offset: {}", - key.toString(), offset); - } - + auto * cell = offsets->get(offset); const auto & file_segment = cell->file_segment; size_t downloaded_size = file_segment->downloaded_size; @@ -1107,88 +1126,69 @@ void FileCache::reduceSizeToDownloaded( } CreateFileSegmentSettings create_settings{ .is_persistent = file_segment->is_persistent }; + assert(file_segment->downloaded_size <= file_segment->reserved_size); + assert(cell->queue_iterator->size() == file_segment->reserved_size); + assert(cell->queue_iterator->size() >= file_segment->downloaded_size); + + if (file_segment->reserved_size > file_segment->downloaded_size) + { + int64_t extra_size = static_cast(cell->file_segment->reserved_size) - static_cast(file_segment->downloaded_size); + cell->queue_iterator->incrementSize(-extra_size, getQueueLock()); + } cell->file_segment = std::make_shared( - offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED, create_settings); + offset, downloaded_size, key, getCreator(), file_segment->cache, + FileSegment::State::DOWNLOADED, create_settings); assert(file_segment->reserved_size == downloaded_size); -} - -bool FileCache::isLastFileSegmentHolder( - const Key & key, size_t offset, - std::lock_guard & cache_lock, std::unique_lock & /* segment_lock */) -{ - auto * cell = getCell(key, offset, cache_lock); - - if (!cell) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No cell found for key: {}, offset: {}", key.toString(), offset); - - /// The caller of this method is last file segment holder if use count is 2 (the second pointer is cache itself) - return cell->file_segment.use_count() == 2; + assert(cell->size() == cell->queue_iterator->size()); } FileSegments FileCache::getSnapshot() const { - std::lock_guard cache_lock(mutex); + auto lock = cache_guard.lock(); FileSegments file_segments; for (const auto & [key, cells_by_offset] : files) { - for (const auto & [offset, cell] : cells_by_offset) - file_segments.push_back(FileSegment::getSnapshot(cell.file_segment, cache_lock)); + for (const auto & [offset, cell] : *cells_by_offset) + file_segments.push_back(FileSegment::getSnapshot(cell.file_segment)); } return file_segments; } std::vector FileCache::tryGetCachePaths(const Key & key) { - std::lock_guard cache_lock(mutex); + auto lock = cache_guard.lock(); std::vector cache_paths; - const auto & cells_by_offset = files[key]; - for (const auto & [offset, cell] : cells_by_offset) + for (const auto & [offset, cell] : *cells_by_offset) { if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent())); } - return cache_paths; } size_t FileCache::getUsedCacheSize() const { - std::lock_guard cache_lock(mutex); - return getUsedCacheSizeUnlocked(cache_lock); -} - -size_t FileCache::getUsedCacheSizeUnlocked(std::lock_guard & cache_lock) const -{ - return main_priority->getCacheSize(cache_lock); -} - -size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard & cache_lock) const -{ - return max_size - getUsedCacheSizeUnlocked(cache_lock); + auto lock = main_priority->lock(); + return main_priority->getCacheSize(lock); } size_t FileCache::getFileSegmentsNum() const { - std::lock_guard cache_lock(mutex); - return getFileSegmentsNumUnlocked(cache_lock); -} - -size_t FileCache::getFileSegmentsNumUnlocked(std::lock_guard & cache_lock) const -{ - return main_priority->getElementsNum(cache_lock); + auto lock = main_priority->lock(); + return main_priority->getElementsNum(lock); } FileCache::FileSegmentCell::FileSegmentCell( FileSegmentPtr file_segment_, - FileCache * cache, - std::lock_guard & cache_lock) + KeyTransaction & key_transaction, + IFileCachePriority & priority_queue) : file_segment(file_segment_) { /** @@ -1201,8 +1201,11 @@ FileCache::FileSegmentCell::FileSegmentCell( { case FileSegment::State::DOWNLOADED: { - queue_iterator = cache->main_priority->add( - file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock); + queue_iterator = priority_queue.add( + file_segment->key(), file_segment->offset(), file_segment->range().size(), + key_transaction.getCreator(), key_transaction.getQueueLock()); + + /// TODO: add destructor break; } case FileSegment::State::SKIP_CACHE: @@ -1219,68 +1222,185 @@ FileCache::FileSegmentCell::FileSegmentCell( } } -String FileCache::dumpStructure(const Key & key) +const FileCache::FileSegmentCell * FileCache::CacheCells::get(size_t offset) const { - std::lock_guard cache_lock(mutex); - return dumpStructureUnlocked(key, cache_lock); + auto it = find(offset); + if (it == end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset); + return &(it->second); } -String FileCache::dumpStructureUnlocked(const Key & key, std::lock_guard &) +FileCache::FileSegmentCell * FileCache::CacheCells::get(size_t offset) +{ + auto it = find(offset); + if (it == end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset); + return &(it->second); +} + +const FileCache::FileSegmentCell * FileCache::CacheCells::tryGet(size_t offset) const +{ + auto it = find(offset); + if (it == end()) + return nullptr; + return &(it->second); +} + +FileCache::FileSegmentCell * FileCache::CacheCells::tryGet(size_t offset) +{ + auto it = find(offset); + if (it == end()) + return nullptr; + return &(it->second); +} + +std::string FileCache::CacheCells::toString() const +{ + std::string result; + for (auto it = begin(); it != end(); ++it) + { + if (it != begin()) + result += ", "; + result += std::to_string(it->first); + } + return result; +} + +KeyTransaction::KeyTransaction(KeyPrefixGuardPtr guard_, FileCache::CacheCellsPtr offsets_, std::shared_ptr queue_lock_) + : guard(guard_) + , lock(guard->lock()) + , offsets(offsets_) + , log(&Poco::Logger::get("KeyTransaction")) + , queue_lock(queue_lock_) +{ +} + +void FileCache::assertInitializedUnlocked(CacheGuard::Lock & lock) const +{ + using State = InitializationState; + + switch (initialization_state) + { + case State::NOT_INITIALIZED: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache not initialized"); + } + case State::FAILED: + { + assert(initialization_exception); + std::rethrow_exception(initialization_exception); + } + case State::INITIALIZED: + { + return; + } + case State::INITIALIZING: + { + initialization_cv.wait(lock.lock, [this]() { return initialization_state != State::INITIALIZING; }); + assertInitializedUnlocked(lock); + } + } +} + +void FileCache::initialize() +{ + using State = InitializationState; + { + auto lock = cache_guard.lock(); + + switch (initialization_state) + { + case State::NOT_INITIALIZED: + { + if (!fs::exists(cache_base_path)) + { + fs::create_directories(cache_base_path); + initialization_state = InitializationState::INITIALIZED; + return; + } + else + { + initialization_state = InitializationState::INITIALIZING; + } + break; + } + case State::FAILED: + { + assert(initialization_exception); + std::rethrow_exception(initialization_exception); + } + case State::INITIALIZED: + case State::INITIALIZING: + { + return; + } + } + } + + try + { + loadCacheInfoIntoMemory(); + { + auto lock = cache_guard.lock(); + initialization_state = State::INITIALIZED; + } + } + catch (...) + { + initialization_exception = std::current_exception(); + { + auto lock = cache_guard.lock(); + initialization_state = State::FAILED; + } + throw; + } +} + + +String FileCache::dumpStructure(const Key & key) +{ + auto lock = cache_guard.lock(); + return dumpStructureUnlocked(key, lock); +} + +String FileCache::dumpStructureUnlocked(const Key & key, const CacheGuard::Lock &) { WriteBufferFromOwnString result; const auto & cells_by_offset = files[key]; - for (const auto & [offset, cell] : cells_by_offset) + for (const auto & [offset, cell] : *cells_by_offset) result << cell.file_segment->getInfoForLog() << "\n"; return result.str(); } -void FileCache::assertCacheCellsCorrectness( - const FileSegmentsByOffset & cells_by_offset, [[maybe_unused]] std::lock_guard & cache_lock) -{ - for (const auto & [_, cell] : cells_by_offset) - { - const auto & file_segment = cell.file_segment; - file_segment->assertCorrectness(); - - if (file_segment->reserved_size != 0) - { - assert(cell.queue_iterator); - assert(main_priority->contains(file_segment->key(), file_segment->offset(), cache_lock)); - } - } -} - -void FileCache::assertCacheCorrectness(const Key & key, std::lock_guard & cache_lock) -{ - assertCacheCellsCorrectness(files[key], cache_lock); - assertPriorityCorrectness(cache_lock); -} - -void FileCache::assertCacheCorrectness(std::lock_guard & cache_lock) +void FileCache::assertCacheCorrectness() { for (const auto & [key, cells_by_offset] : files) - assertCacheCellsCorrectness(files[key], cache_lock); - assertPriorityCorrectness(cache_lock); -} + { + for (const auto & [_, cell] : *cells_by_offset) + { + const auto & file_segment = cell.file_segment; + file_segment->assertCorrectness(); + + if (file_segment->reserved_size != 0) + { + assert(cell.queue_iterator); + // assert(main_priority->contains(file_segment->key(), file_segment->offset())); + } + } + } -void FileCache::assertPriorityCorrectness(std::lock_guard & cache_lock) -{ [[maybe_unused]] size_t total_size = 0; - for (auto it = main_priority->getLowestPriorityReadIterator(cache_lock); it->valid(); it->next()) + auto main_priority_lock = main_priority->lock(); + for (auto it = main_priority->getLowestPriorityIterator(main_priority_lock); it->valid(main_priority_lock); it->next(main_priority_lock)) { const auto & key = it->key(); auto offset = it->offset(); auto size = it->size(); - auto * cell = getCell(key, offset, cache_lock); - if (!cell) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cache is in inconsistent state: LRU queue contains entries with no cache cell (assertCorrectness())"); - } + auto key_transaction = createKeyTransaction(key, KeyNotFoundPolicy::THROW, false); + auto * cell = key_transaction->getOffsets().get(offset); if (cell->size() != size) { @@ -1293,9 +1413,9 @@ void FileCache::assertPriorityCorrectness(std::lock_guard & cache_lo total_size += size; } - assert(total_size == main_priority->getCacheSize(cache_lock)); - assert(main_priority->getCacheSize(cache_lock) <= max_size); - assert(main_priority->getElementsNum(cache_lock) <= max_element_size); + assert(total_size == main_priority->getCacheSize(main_priority_lock)); + assert(main_priority->getCacheSize(main_priority_lock) <= max_size); + assert(main_priority->getElementsNum(main_priority_lock) <= max_element_size); } FileCache::QueryContextHolder::QueryContextHolder( @@ -1316,15 +1436,16 @@ FileCache::QueryContextHolder::~QueryContextHolder() cache->removeQueryContext(query_id); } -FileCache::QueryContextPtr FileCache::getCurrentQueryContext(std::lock_guard & cache_lock) +FileCache::QueryContextPtr FileCache::getCurrentQueryContext() { if (!isQueryInitialized()) return nullptr; - return getQueryContext(std::string(CurrentThread::getQueryId()), cache_lock); + std::lock_guard lock(query_context_mutex); + return getQueryContextUnlocked(std::string(CurrentThread::getQueryId()), lock); } -FileCache::QueryContextPtr FileCache::getQueryContext(const String & query_id, std::lock_guard & /* cache_lock */) +FileCache::QueryContextPtr FileCache::getQueryContextUnlocked(const String & query_id, std::lock_guard & /* context_lock */) { auto query_iter = query_map.find(query_id); return (query_iter == query_map.end()) ? nullptr : query_iter->second; @@ -1332,9 +1453,9 @@ FileCache::QueryContextPtr FileCache::getQueryContext(const String & query_id, s void FileCache::removeQueryContext(const String & query_id) { - std::lock_guard cache_lock(mutex); - auto query_iter = query_map.find(query_id); + auto lock = cache_guard.lock(); + auto query_iter = query_map.find(query_id); if (query_iter == query_map.end()) { throw Exception( @@ -1346,13 +1467,14 @@ void FileCache::removeQueryContext(const String & query_id) query_map.erase(query_iter); } -FileCache::QueryContextPtr FileCache::getOrSetQueryContext( - const String & query_id, const ReadSettings & settings, std::lock_guard & cache_lock) +FileCache::QueryContextPtr FileCache::getOrSetQueryContext(const String & query_id, const ReadSettings & settings) { if (query_id.empty()) return nullptr; - auto context = getQueryContext(query_id, cache_lock); + std::lock_guard lock(query_context_mutex); + + auto context = getQueryContextUnlocked(query_id, lock); if (context) return context; @@ -1363,19 +1485,18 @@ FileCache::QueryContextPtr FileCache::getOrSetQueryContext( FileCache::QueryContextHolder FileCache::getQueryContextHolder(const String & query_id, const ReadSettings & settings) { - std::lock_guard cache_lock(mutex); - if (!enable_filesystem_query_cache_limit || settings.max_query_cache_size == 0) return {}; /// if enable_filesystem_query_cache_limit is true, and max_query_cache_size large than zero, /// we create context query for current query. - auto context = getOrSetQueryContext(query_id, settings, cache_lock); + auto context = getOrSetQueryContext(query_id, settings); return QueryContextHolder(query_id, this, context); } -void FileCache::QueryContext::remove(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock) +void FileCache::QueryContext::remove(const Key & key, size_t offset, size_t size, KeyTransaction & key_transaction) { + std::lock_guard lock(mutex); if (cache_size < size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Deleted cache size exceeds existing cache size"); @@ -1384,15 +1505,16 @@ void FileCache::QueryContext::remove(const Key & key, size_t offset, size_t size auto record = records.find({key, offset}); if (record != records.end()) { - record->second->removeAndGetNext(cache_lock); + record->second->remove(key_transaction.getQueueLock()); records.erase({key, offset}); } } cache_size -= size; } -void FileCache::QueryContext::reserve(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock) +void FileCache::QueryContext::reserve(const Key & key, size_t offset, size_t size, KeyTransaction & key_transaction) { + std::lock_guard lock(mutex); if (cache_size + size > max_cache_size) { throw Exception( @@ -1406,22 +1528,28 @@ void FileCache::QueryContext::reserve(const Key & key, size_t offset, size_t siz auto record = records.find({key, offset}); if (record == records.end()) { - auto queue_iter = priority->add(key, offset, 0, cache_lock); - record = records.insert({{key, offset}, queue_iter}).first; + auto queue_iter = priority->add(key, offset, 0, key_transaction.getCreator(), key_transaction.getQueueLock()); + record = records.insert({{key, offset}, queue_iter}).first; } - record->second->incrementSize(size, cache_lock); + record->second->incrementSize(size, key_transaction.getQueueLock()); } cache_size += size; } -void FileCache::QueryContext::use(const Key & key, size_t offset, std::lock_guard & cache_lock) +void FileCache::QueryContext::use(const Key & key, size_t offset, KeyTransaction & key_transaction) { if (skip_download_if_exceeds_query_cache) return; + std::lock_guard lock(mutex); auto record = records.find({key, offset}); if (record != records.end()) - record->second->use(cache_lock); + record->second->use(key_transaction.getQueueLock()); +} + +KeyTransactionPtr KeyTransactionCreator::create() +{ + return std::make_unique(guard, offsets); } } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 706762b6915..7307be0908e 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -28,11 +29,12 @@ namespace DB /// Different caching algorithms are implemented using IFileCachePriority. class FileCache : private boost::noncopyable { - -friend class FileSegment; friend class IFileCachePriority; -friend struct FileSegmentsHolder; friend class FileSegmentRangeWriter; +friend struct KeyTransaction; +friend struct KeyTransactionCreator; +friend struct FileSegmentsHolder; +friend class FileSegment; struct QueryContext; using QueryContextPtr = std::shared_ptr; @@ -59,7 +61,7 @@ public: * As long as pointers to returned file segments are hold * it is guaranteed that these file segments are not removed from cache. */ - FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); + FileSegmentsHolderPtr getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); /** * Segments in returned list are ordered in ascending order and represent a full contiguous @@ -70,15 +72,15 @@ public: * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change * it's state (and become DOWNLOADED). */ - FileSegmentsHolder get(const Key & key, size_t offset, size_t size); + FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size); /// Remove files by `key`. Removes files which might be used at the moment. void removeIfExists(const Key & key); /// Remove files by `key`. Will not remove files which are used at the moment. - void removeIfReleasable(); + void removeAllReleasable(); - static Key hash(const String & path); + static Key createKeyForPath(const String & path); String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const; @@ -92,7 +94,7 @@ public: size_t getFileSegmentsNum() const; - static bool isReadOnly(); + static bool readThrowCacheAllowed(); /** * Create a file segment of exactly requested size with EMPTY state. @@ -104,14 +106,15 @@ public: const Key & key, size_t offset, size_t size, - const CreateFileSegmentSettings & create_settings, - std::lock_guard & cache_lock); + const CreateFileSegmentSettings & create_settings); FileSegments getSnapshot() const; /// For debug. String dumpStructure(const Key & key); + bool tryReserve(const Key & key, size_t offset, size_t size); + /// Save a query context information, and adopt different cache policies /// for different queries through the context cache layer. struct QueryContextHolder : private boost::noncopyable @@ -130,6 +133,8 @@ public: QueryContextHolder getQueryContextHolder(const String & query_id, const ReadSettings & settings); private: + using KeyAndOffset = FileCacheKeyAndOffset; + String cache_base_path; const size_t max_size; @@ -137,50 +142,18 @@ private: const size_t max_file_segment_size; const bool allow_persistent_files; - const size_t enable_cache_hits_threshold; const bool enable_filesystem_query_cache_limit; + const bool enable_bypass_cache_with_threshold; + const size_t bypass_cache_threshold; - const bool enable_bypass_cache_with_threashold; - const size_t bypass_cache_threashold; - - mutable std::mutex mutex; Poco::Logger * log; - bool is_initialized = false; - std::exception_ptr initialization_exception; - - void assertInitialized(std::lock_guard & cache_lock) const; - - bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock); - - void remove( - Key key, - size_t offset, - std::lock_guard & cache_lock, - std::unique_lock & segment_lock); - - void remove( - FileSegmentPtr file_segment, - std::lock_guard & cache_lock); - - bool isLastFileSegmentHolder( - const Key & key, - size_t offset, - std::lock_guard & cache_lock, - std::unique_lock & segment_lock); - - void reduceSizeToDownloaded( - const Key & key, - size_t offset, - std::lock_guard & cache_lock, - std::unique_lock & segment_lock); - struct FileSegmentCell : private boost::noncopyable { FileSegmentPtr file_segment; /// Iterator is put here on first reservation attempt, if successful. - IFileCachePriority::WriteIterator queue_iterator; + IFileCachePriority::Iterator queue_iterator; /// Pointer to file segment is always hold by the cache itself. /// Apart from pointer in cache, it can be hold by cache users, when they call @@ -189,54 +162,92 @@ private: size_t size() const { return file_segment->reserved_size; } - FileSegmentCell(FileSegmentPtr file_segment_, FileCache * cache, std::lock_guard & cache_lock); + FileSegmentCell( + FileSegmentPtr file_segment_, + KeyTransaction & key_transaction, + IFileCachePriority & priority_queue); FileSegmentCell(FileSegmentCell && other) noexcept : file_segment(std::move(other.file_segment)), queue_iterator(std::move(other.queue_iterator)) {} }; - using AccessKeyAndOffset = std::pair; - struct KeyAndOffsetHash + struct CacheCells : public std::map { - std::size_t operator()(const AccessKeyAndOffset & key) const - { - return std::hash()(key.first.key) ^ std::hash()(key.second); - } + const FileSegmentCell * get(size_t offset) const; + FileSegmentCell * get(size_t offset); + + const FileSegmentCell * tryGet(size_t offset) const; + FileSegmentCell * tryGet(size_t offset); + + std::string toString() const; + + bool created_base_directory = false; + }; + using CacheCellsPtr = std::shared_ptr; + + mutable CacheGuard cache_guard; + + enum class InitializationState + { + NOT_INITIALIZED, + INITIALIZING, + INITIALIZED, + FAILED, + }; + InitializationState initialization_state = InitializationState::NOT_INITIALIZED; + mutable std::condition_variable initialization_cv; + std::exception_ptr initialization_exception; + + using CachedFiles = std::unordered_map; + CachedFiles files; + + using KeyPrefix = std::string; + using KeysLocksMap = std::unordered_map; + KeysLocksMap keys_locks; + + enum class KeyNotFoundPolicy + { + THROW, + CREATE_EMPTY, + RETURN_NULL, }; - using FileSegmentsByOffset = std::map; - using CachedFiles = std::unordered_map; - using FileCacheRecords = std::unordered_map; + KeyTransactionPtr createKeyTransaction(const Key & key, KeyNotFoundPolicy key_not_found_policy, bool assert_initialized = true); - CachedFiles files; - std::unique_ptr main_priority; + KeyTransactionCreatorPtr getKeyTransactionCreator(const Key & key, KeyTransaction & key_transaction); - FileCacheRecords stash_records; - std::unique_ptr stash_priority; - size_t max_stash_element_size; + FileCachePriorityPtr main_priority; - void loadCacheInfoIntoMemory(std::lock_guard & cache_lock); + struct HitsCountStash + { + HitsCountStash(size_t max_stash_queue_size_, size_t cache_hits_threshold_, FileCachePriorityPtr queue_) + : max_stash_queue_size(max_stash_queue_size_) + , cache_hits_threshold(cache_hits_threshold_) + , queue(std::move(queue_)) {} - FileSegments getImpl(const Key & key, const FileSegment::Range & range, std::lock_guard & cache_lock); + const size_t max_stash_queue_size; + const size_t cache_hits_threshold; - FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard & cache_lock); + auto lock() const { return queue->lock(); } - FileSegmentCell * addCell( + FileCachePriorityPtr queue; + + using Records = std::unordered_map; + Records records; + }; + + mutable HitsCountStash stash; + +protected: + void assertCacheCorrectness(); + + void assertInitializedUnlocked(CacheGuard::Lock & cache_lock) const; + +private: + FileSegments getImpl( const Key & key, - size_t offset, - size_t size, - FileSegment::State state, - const CreateFileSegmentSettings & create_settings, - std::lock_guard & cache_lock); - - static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard & cache_lock); - - bool tryReserveForMainList( - const Key & key, - size_t offset, - size_t size, - QueryContextPtr query_context, - std::lock_guard & cache_lock); + const FileSegment::Range & range, + const KeyTransaction & key_transaction) const; FileSegments splitRangeIntoCells( const Key & key, @@ -244,9 +255,7 @@ private: size_t size, FileSegment::State state, const CreateFileSegmentSettings & create_settings, - std::lock_guard & cache_lock); - - String dumpStructureUnlocked(const Key & key_, std::lock_guard & cache_lock); + KeyTransaction & key_transaction); void fillHolesWithEmptyFileSegments( FileSegments & file_segments, @@ -254,23 +263,50 @@ private: const FileSegment::Range & range, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings, - std::lock_guard & cache_lock); + KeyTransaction & key_transaction); - size_t getUsedCacheSizeUnlocked(std::lock_guard & cache_lock) const; + void loadCacheInfoIntoMemory(); - size_t getAvailableCacheSizeUnlocked(std::lock_guard & cache_lock) const; + CacheCells::iterator addCell( + const Key & key, + size_t offset, + size_t size, + FileSegment::State state, + const CreateFileSegmentSettings & create_settings, + KeyTransaction & key_transaction); + + bool tryReserveUnlocked( + const Key & key, + size_t offset, + size_t size, + KeyTransaction & key_transaction); + + bool tryReserveInCache( + const Key & key, + size_t offset, + size_t size, + QueryContextPtr query_context, + KeyTransaction & key_transaction); + + bool tryReserveInQueryCache( + const Key & key, + size_t offset, + size_t size, + QueryContextPtr query_context, + KeyTransaction & key_transaction); size_t getFileSegmentsNumUnlocked(std::lock_guard & cache_lock) const; - void assertCacheCellsCorrectness(const FileSegmentsByOffset & cells_by_offset, std::lock_guard & cache_lock); + void removeKeyDirectoryIfExists(const Key & key, const KeyPrefixGuard::Lock & lock) const; - void removeKeyDirectoryIfExists(const Key & key, std::lock_guard & cache_lock) const; + String dumpStructureUnlocked(const Key & key_, const CacheGuard::Lock & lock); /// Used to track and control the cache access of each query. /// Through it, we can realize the processing of different queries by the cache layer. struct QueryContext { - FileCacheRecords records; + std::mutex mutex; + HitsCountStash::Records records; FileCachePriorityPtr priority; size_t cache_size = 0; @@ -286,34 +322,82 @@ private: size_t getCacheSize() const { return cache_size; } - FileCachePriorityPtr getPriority() const { return priority; } + IFileCachePriority & getPriority() const { return *priority; } bool isSkipDownloadIfExceed() const { return skip_download_if_exceeds_query_cache; } - void remove(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock); + void remove(const Key & key, size_t offset, size_t size, KeyTransaction & key_transaction); - void reserve(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock); + void reserve(const Key & key, size_t offset, size_t size, KeyTransaction & key_transaction); - void use(const Key & key, size_t offset, std::lock_guard & cache_lock); + void use(const Key & key, size_t offset, KeyTransaction & key_transaction); }; using QueryContextMap = std::unordered_map; QueryContextMap query_map; + std::mutex query_context_mutex; - QueryContextPtr getCurrentQueryContext(std::lock_guard & cache_lock); + QueryContextPtr getCurrentQueryContext(); - QueryContextPtr getQueryContext(const String & query_id, std::lock_guard & cache_lock); + QueryContextPtr getQueryContextUnlocked(const String & query_id, std::lock_guard &); void removeQueryContext(const String & query_id); - QueryContextPtr getOrSetQueryContext(const String & query_id, const ReadSettings & settings, std::lock_guard &); + QueryContextPtr getOrSetQueryContext(const String & query_id, const ReadSettings & settings); +}; +struct KeyTransaction; +using KeyTransactionPtr = std::unique_ptr; + +struct KeyTransactionCreator +{ + KeyTransactionCreator( + KeyPrefixGuardPtr guard_, FileCache::CacheCellsPtr offsets_) + : guard(guard_) , offsets(offsets_) {} + + KeyTransactionPtr create(); + + KeyPrefixGuardPtr guard; + FileCache::CacheCellsPtr offsets; +}; +using KeyTransactionCreatorPtr = std::unique_ptr; + +struct KeyTransaction : private boost::noncopyable +{ + using Key = FileCacheKey; + + KeyTransaction(KeyPrefixGuardPtr guard_, FileCache::CacheCellsPtr offsets_, std::shared_ptr queue_lock_ = nullptr); + + KeyTransactionCreatorPtr getCreator() { return std::make_unique(guard, offsets); } + + void remove(FileSegmentPtr file_segment); + + void reduceSizeToDownloaded(const Key & key, size_t offset, const FileSegmentGuard::Lock &); + + void remove(const Key & key, size_t offset, const FileSegmentGuard::Lock &); + + FileCache::CacheCells & getOffsets() { return *offsets; } + const FileCache::CacheCells & getOffsets() const { return *offsets; } + + std::vector delete_offsets; + + const CachePriorityQueueGuard::Lock & getQueueLock() const + { + if (!queue_lock) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Queue is not locked"); + return *queue_lock; + } + +private: + KeyPrefixGuardPtr guard; + const KeyPrefixGuard::Lock lock; + + FileCache::CacheCellsPtr offsets; + + Poco::Logger * log; public: - void assertCacheCorrectness(const Key & key, std::lock_guard & cache_lock); + std::shared_ptr queue_lock; - void assertCacheCorrectness(std::lock_guard & cache_lock); - - void assertPriorityCorrectness(std::lock_guard & cache_lock); }; } diff --git a/src/Interpreters/Cache/FileCacheKey.cpp b/src/Interpreters/Cache/FileCacheKey.cpp new file mode 100644 index 00000000000..7614d954635 --- /dev/null +++ b/src/Interpreters/Cache/FileCacheKey.cpp @@ -0,0 +1,27 @@ +#include "FileCacheKey.h" + +#include +#include + + +namespace DB +{ + +FileCacheKey::FileCacheKey(const std::string & path) + : key(sipHash128(path.data(), path.size())) + , key_prefix(toString().substr(0, 3)) +{ +} + +FileCacheKey::FileCacheKey(const UInt128 & key_) + : key(key_) + , key_prefix(toString().substr(0, 3)) +{ +} + +std::string FileCacheKey::toString() const +{ + return getHexUIntLowercase(key); +} + +} diff --git a/src/Interpreters/Cache/FileCacheKey.h b/src/Interpreters/Cache/FileCacheKey.h index cf4ab5d20c5..3881a7aec7e 100644 --- a/src/Interpreters/Cache/FileCacheKey.h +++ b/src/Interpreters/Cache/FileCacheKey.h @@ -1,23 +1,34 @@ #pragma once #include -#include namespace DB { struct FileCacheKey { + /// Hash of the path. UInt128 key; + /// Prefix of the path. + std::string key_prefix; - String toString() const { return getHexUIntLowercase(key); } + std::string toString() const; - FileCacheKey() = default; + explicit FileCacheKey(const std::string & path); - explicit FileCacheKey(const UInt128 & key_) : key(key_) { } + explicit FileCacheKey(const UInt128 & path); bool operator==(const FileCacheKey & other) const { return key == other.key; } }; +using FileCacheKeyAndOffset = std::pair; +struct FileCacheKeyAndOffsetHash +{ + std::size_t operator()(const FileCacheKeyAndOffset & key) const + { + return std::hash()(key.first.key) ^ std::hash()(key.second); + } +}; + } namespace std diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 9c48f16d15e..8a42ddde31c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -26,15 +26,18 @@ FileSegment::FileSegment( size_t offset_, size_t size_, const Key & key_, + KeyTransactionCreatorPtr key_transaction_creator_, FileCache * cache_, State download_state_, const CreateFileSegmentSettings & settings) : segment_range(offset_, offset_ + size_ - 1) , download_state(download_state_) + , key_transaction_creator(std::move(key_transaction_creator_)) , file_key(key_) + , file_path(cache_->getPathInLocalCache(key(), offset(), settings.is_persistent)) , cache(cache_) #ifndef NDEBUG - , log(&Poco::Logger::get(fmt::format("FileSegment({}) : {}", getHexUIntLowercase(key_), range().toString()))) + , log(&Poco::Logger::get(fmt::format("FileSegment({}) : {}", key_.toString(), range().toString()))) #else , log(&Poco::Logger::get("FileSegment")) #endif @@ -55,11 +58,17 @@ FileSegment::FileSegment( { reserved_size = downloaded_size = size_; is_downloaded = true; - chassert(std::filesystem::file_size(getPathInLocalCache()) == size_); + is_completed = true; + file = OpenedFileCache::instance().get(file_path, -1); + fd = file->getFD(); + chassert(std::filesystem::file_size(file_path) == size_); break; } case (State::SKIP_CACHE): { + is_detached = true; + is_completed = true; + //CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments); break; } default: @@ -71,18 +80,13 @@ FileSegment::FileSegment( } } -String FileSegment::getPathInLocalCache() const -{ - return cache->getPathInLocalCache(key(), offset(), isPersistent()); -} - FileSegment::State FileSegment::state() const { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); return download_state; } -void FileSegment::setDownloadState(State state) +void FileSegment::setDownloadState(State state, const FileSegmentGuard::Lock &) { LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state)); download_state = state; @@ -90,33 +94,33 @@ void FileSegment::setDownloadState(State state) size_t FileSegment::getFirstNonDownloadedOffset() const { - std::unique_lock segment_lock(mutex); - return getFirstNonDownloadedOffsetUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return getFirstNonDownloadedOffsetUnlocked(lock); } -size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock & segment_lock) const +size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(const FileSegmentGuard::Lock & lock) const { - return range().left + getDownloadedSizeUnlocked(segment_lock); + return range().left + getDownloadedSizeUnlocked(lock); } size_t FileSegment::getCurrentWriteOffset() const { - std::unique_lock segment_lock(mutex); - return getCurrentWriteOffsetUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return getCurrentWriteOffsetUnlocked(lock); } -size_t FileSegment::getCurrentWriteOffsetUnlocked(std::unique_lock & segment_lock) const +size_t FileSegment::getCurrentWriteOffsetUnlocked(const FileSegmentGuard::Lock & lock) const { - return getFirstNonDownloadedOffsetUnlocked(segment_lock); + return getFirstNonDownloadedOffsetUnlocked(lock); } size_t FileSegment::getDownloadedSize() const { - std::unique_lock segment_lock(mutex); - return getDownloadedSizeUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return getDownloadedSizeUnlocked(lock); } -size_t FileSegment::getDownloadedSizeUnlocked(std::unique_lock & /* segment_lock */) const +size_t FileSegment::getDownloadedSizeUnlocked(const FileSegmentGuard::Lock &) const { if (download_state == State::DOWNLOADED) return downloaded_size; @@ -127,7 +131,7 @@ size_t FileSegment::getDownloadedSizeUnlocked(std::unique_lock & /* bool FileSegment::isDownloaded() const { - std::lock_guard segment_lock(mutex); + auto lock = segment_guard.lock(); return is_downloaded; } @@ -143,70 +147,71 @@ String FileSegment::getCallerId() String FileSegment::getDownloader() const { - std::unique_lock segment_lock(mutex); - return getDownloaderUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return getDownloaderUnlocked(lock); } -String FileSegment::getDownloaderUnlocked(std::unique_lock & /* segment_lock */) const +String FileSegment::getDownloaderUnlocked(const FileSegmentGuard::Lock &) const { return downloader_id; } String FileSegment::getOrSetDownloader() { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); - assertNotDetachedUnlocked(segment_lock); + assertNotDetachedUnlocked(lock); - auto current_downloader = getDownloaderUnlocked(segment_lock); + auto current_downloader = getDownloaderUnlocked(lock); if (current_downloader.empty()) { - bool allow_new_downloader = download_state == State::EMPTY || download_state == State::PARTIALLY_DOWNLOADED; + const auto caller_id = getCallerId(); + bool allow_new_downloader = download_state == State::EMPTY || download_state == State::PARTIALLY_DOWNLOADED || !caller_id.starts_with("None"); if (!allow_new_downloader) return "notAllowed:" + stateToString(download_state); - current_downloader = downloader_id = getCallerId(); - setDownloadState(State::DOWNLOADING); + current_downloader = downloader_id = caller_id; + setDownloadState(State::DOWNLOADING, lock); } return current_downloader; } -void FileSegment::resetDownloadingStateUnlocked([[maybe_unused]] std::unique_lock & segment_lock) +void FileSegment::resetDownloadingStateUnlocked([[maybe_unused]] const FileSegmentGuard::Lock & lock) { - assert(isDownloaderUnlocked(segment_lock)); + assert(isDownloaderUnlocked(lock)); assert(download_state == State::DOWNLOADING); - size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + size_t current_downloaded_size = getDownloadedSizeUnlocked(lock); /// range().size() can equal 0 in case of write-though cache. if (current_downloaded_size != 0 && current_downloaded_size == range().size()) - setDownloadedUnlocked(segment_lock); + setDownloadedUnlocked(lock); else - setDownloadState(State::PARTIALLY_DOWNLOADED); + setDownloadState(State::PARTIALLY_DOWNLOADED, lock); } void FileSegment::resetDownloader() { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); - assertNotDetachedUnlocked(segment_lock); - assertIsDownloaderUnlocked("resetDownloader", segment_lock); + assertNotDetachedUnlocked(lock); + assertIsDownloaderUnlocked("resetDownloader", lock); - resetDownloadingStateUnlocked(segment_lock); - resetDownloaderUnlocked(segment_lock); + resetDownloadingStateUnlocked(lock); + resetDownloaderUnlocked(lock); } -void FileSegment::resetDownloaderUnlocked(std::unique_lock & /* segment_lock */) +void FileSegment::resetDownloaderUnlocked(const FileSegmentGuard::Lock &) { LOG_TEST(log, "Resetting downloader from {}", downloader_id); downloader_id.clear(); } -void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const +void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, const FileSegmentGuard::Lock & lock) const { auto caller = getCallerId(); - auto current_downloader = getDownloaderUnlocked(segment_lock); + auto current_downloader = getDownloaderUnlocked(lock); LOG_TEST(log, "Downloader id: {}, caller id: {}", current_downloader, caller); if (caller != current_downloader) @@ -221,41 +226,53 @@ void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std: bool FileSegment::isDownloader() const { - std::unique_lock segment_lock(mutex); - return isDownloaderUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return isDownloaderUnlocked(lock); } -bool FileSegment::isDownloaderUnlocked(std::unique_lock & segment_lock) const +bool FileSegment::isDownloaderUnlocked(const FileSegmentGuard::Lock & lock) const { - return getCallerId() == getDownloaderUnlocked(segment_lock); + return getCallerId() == getDownloaderUnlocked(lock); } FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() { - std::unique_lock segment_lock(mutex); - assertIsDownloaderUnlocked("getRemoteFileReader", segment_lock); + auto lock = segment_guard.lock(); + assertIsDownloaderUnlocked("getRemoteFileReader", lock); return remote_file_reader; } +bool FileSegment::isLastHolder(const KeyTransaction & key_transaction) const +{ + chassert(!is_detached); + const auto * cell = key_transaction.getOffsets().get(offset()); + return cell->file_segment.use_count() == 2; +} + FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - std::lock_guard cache_lock(cache->mutex); - std::unique_lock segment_lock(mutex); - - if (!is_detached) + auto key_transaction = createKeyTransaction(false); + if (!key_transaction) { - bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); - if (!downloader_id.empty() || !is_last_holder) - return nullptr; + assert(isDetached()); + return std::move(remote_file_reader); } + auto segment_lock = segment_guard.lock(); + + assert(!is_detached); + + bool is_last_holder = isLastHolder(*key_transaction); + if (!downloader_id.empty() || !is_last_holder) + return nullptr; + return std::move(remote_file_reader); } void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) { - std::unique_lock segment_lock(mutex); - assertIsDownloaderUnlocked("setRemoteFileReader", segment_lock); + auto lock = segment_guard.lock(); + assertIsDownloaderUnlocked("setRemoteFileReader", lock); if (remote_file_reader) throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists"); @@ -265,8 +282,8 @@ void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) void FileSegment::resetRemoteFileReader() { - std::unique_lock segment_lock(mutex); - assertIsDownloaderUnlocked("resetRemoteFileReader", segment_lock); + auto lock = segment_guard.lock(); + assertIsDownloaderUnlocked("resetRemoteFileReader", lock); if (!remote_file_reader) throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader does not exist"); @@ -280,24 +297,24 @@ void FileSegment::write(const char * from, size_t size, size_t offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); - assertIsDownloaderUnlocked("write", segment_lock); - assertNotDetachedUnlocked(segment_lock); + assertIsDownloaderUnlocked("write", lock); + assertNotDetachedUnlocked(lock); if (download_state != State::DOWNLOADING) throw Exception( ErrorCodes::LOGICAL_ERROR, "Expected DOWNLOADING state, got {}", stateToString(download_state)); - size_t first_non_downloaded_offset = getFirstNonDownloadedOffsetUnlocked(segment_lock); + size_t first_non_downloaded_offset = getFirstNonDownloadedOffsetUnlocked(lock); if (offset != first_non_downloaded_offset) throw Exception( ErrorCodes::LOGICAL_ERROR, "Attempt to write {} bytes to offset: {}, but current write offset is {}", size, offset, first_non_downloaded_offset); - size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + size_t current_downloaded_size = getDownloadedSizeUnlocked(lock); chassert(reserved_size >= current_downloaded_size); size_t free_reserved_size = reserved_size - current_downloaded_size; @@ -317,8 +334,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) "Cache writer was finalized (downloaded size: {}, state: {})", current_downloaded_size, stateToString(download_state)); - auto download_path = getPathInLocalCache(); - cache_writer = std::make_unique(download_path); + cache_writer = std::make_unique(file_path); } } @@ -332,15 +348,15 @@ void FileSegment::write(const char * from, size_t size, size_t offset) downloaded_size += size; - chassert(std::filesystem::file_size(getPathInLocalCache()) == downloaded_size); + chassert(std::filesystem::file_size(file_path) == downloaded_size); } catch (Exception & e) { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); - wrapWithCacheInfo(e, "while writing into cache", segment_lock); + wrapWithCacheInfo(e, "while writing into cache", lock); - setDownloadFailedUnlocked(segment_lock); + setDownloadFailedUnlocked(lock); cv.notify_all(); @@ -352,7 +368,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) FileSegment::State FileSegment::wait() { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); if (is_detached) throw Exception( @@ -369,15 +385,27 @@ FileSegment::State FileSegment::wait() { LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id); - chassert(!getDownloaderUnlocked(segment_lock).empty()); - chassert(!isDownloaderUnlocked(segment_lock)); + chassert(!getDownloaderUnlocked(lock).empty()); + chassert(!isDownloaderUnlocked(lock)); - cv.wait_for(segment_lock, std::chrono::seconds(60)); + cv.wait_for(lock.lock, std::chrono::seconds(60)); } return download_state; } +KeyTransactionPtr FileSegment::createKeyTransaction(bool assert_exists) const +{ + if (!key_transaction_creator) + { + if (assert_exists) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key transaction: creator does not exist"); + else + return nullptr; + } + return key_transaction_creator->create(); +} + bool FileSegment::reserve(size_t size_to_reserve) { if (!size_to_reserve) @@ -386,12 +414,14 @@ bool FileSegment::reserve(size_t size_to_reserve) size_t expected_downloaded_size; { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); - assertNotDetachedUnlocked(segment_lock); - assertIsDownloaderUnlocked("reserve", segment_lock); + LOG_TRACE(log, "Try reserve for {}", getInfoForLogUnlocked(lock)); - expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + assertNotDetachedUnlocked(lock); + assertIsDownloaderUnlocked("reserve", lock); + + expected_downloaded_size = getDownloadedSizeUnlocked(lock); if (expected_downloaded_size + size_to_reserve > range().size()) throw Exception( @@ -413,14 +443,12 @@ bool FileSegment::reserve(size_t size_to_reserve) bool reserved = already_reserved_size >= size_to_reserve; if (!reserved) { - std::lock_guard cache_lock(cache->mutex); - size_to_reserve = size_to_reserve - already_reserved_size; - reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock); + reserved = cache->tryReserve(key(), offset(), size_to_reserve); if (reserved) { - std::lock_guard segment_lock(mutex); + auto lock = segment_guard.lock(); reserved_size += size_to_reserve; } } @@ -428,7 +456,25 @@ bool FileSegment::reserve(size_t size_to_reserve) return reserved; } -void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock & segment_lock) +OpenedFileCache::OpenedFilePtr FileSegment::getFile() const +{ + auto lock = segment_guard.lock(); + if (!is_downloaded) + throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is not downloaded"); + chassert(file); + return file; +} + +int FileSegment::getFD() const +{ + auto lock = segment_guard.lock(); + if (!is_downloaded) + throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is not downloaded"); + chassert(file); + return fd; +} + +void FileSegment::setDownloadedUnlocked([[maybe_unused]] const FileSegmentGuard::Lock & lock) { if (is_downloaded) return; @@ -443,16 +489,19 @@ void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock 0); - assert(std::filesystem::file_size(getPathInLocalCache()) > 0); + file = OpenedFileCache::instance().get(file_path, -1); + fd = file->getFD(); + + assert(getDownloadedSizeUnlocked(lock) > 0); + assert(std::filesystem::file_size(file_path) > 0); } -void FileSegment::setDownloadFailedUnlocked(std::unique_lock & segment_lock) +void FileSegment::setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock) { - LOG_INFO(log, "Settings download as failed: {}", getInfoForLogUnlocked(segment_lock)); + LOG_INFO(log, "Settings download as failed: {}", getInfoForLogUnlocked(lock)); - setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - resetDownloaderUnlocked(segment_lock); + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION, lock); + resetDownloaderUnlocked(lock); if (cache_writer) { @@ -464,64 +513,59 @@ void FileSegment::setDownloadFailedUnlocked(std::unique_lock & segme void FileSegment::completePartAndResetDownloader() { - std::unique_lock segment_lock(mutex); - completePartAndResetDownloaderUnlocked(segment_lock); + auto lock = segment_guard.lock(); + completePartAndResetDownloaderUnlocked(lock); } -void FileSegment::completePartAndResetDownloaderUnlocked(std::unique_lock & segment_lock) +void FileSegment::completePartAndResetDownloaderUnlocked(const FileSegmentGuard::Lock & lock) { - assertNotDetachedUnlocked(segment_lock); - assertIsDownloaderUnlocked("completePartAndResetDownloader", segment_lock); + assertNotDetachedUnlocked(lock); + assertIsDownloaderUnlocked("completePartAndResetDownloader", lock); - resetDownloadingStateUnlocked(segment_lock); - resetDownloaderUnlocked(segment_lock); + resetDownloadingStateUnlocked(lock); + resetDownloaderUnlocked(lock); - LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(segment_lock)); + LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(lock)); cv.notify_all(); } -void FileSegment::completeWithState(State state) +void FileSegment::setBroken() { - std::lock_guard cache_lock(cache->mutex); - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); + assertNotDetachedUnlocked(lock); + assertIsDownloaderUnlocked("setBroken", lock); + resetDownloadingStateUnlocked(lock); + resetDownloaderUnlocked(lock); +} - assertNotDetachedUnlocked(segment_lock); - assertIsDownloaderUnlocked("complete", segment_lock); +void FileSegment::complete() +{ + auto lock = cache->main_priority->lockShared(); + auto key_transaction = createKeyTransaction(); + key_transaction->queue_lock = lock; + return completeUnlocked(*key_transaction); +} - if (state != State::DOWNLOADED - && state != State::PARTIALLY_DOWNLOADED - && state != State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) +void FileSegment::completeUnlocked(KeyTransaction & key_transaction) +{ + auto segment_lock = segment_guard.lock(); + + if (is_detached) { - cv.notify_all(); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot complete file segment with state: {}", stateToString(state)); + assertDetachedStatus(segment_lock); + return; } - setDownloadState(state); - completeBasedOnCurrentState(cache_lock, segment_lock); -} - -void FileSegment::completeWithoutState() -{ - std::lock_guard cache_lock(cache->mutex); - completeWithoutStateUnlocked(cache_lock); -} - -void FileSegment::completeWithoutStateUnlocked(std::lock_guard & cache_lock) -{ - std::unique_lock segment_lock(mutex); - completeBasedOnCurrentState(cache_lock, segment_lock); -} - -void FileSegment::completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock) -{ - if (is_detached) + if (is_completed) return; - bool is_downloader = isDownloaderUnlocked(segment_lock); - bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); - size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + const bool is_downloader = isDownloaderUnlocked(segment_lock); + const bool is_last_holder = isLastHolder(key_transaction); + const size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); + + auto queue_iter = key_transaction.getOffsets().get(offset())->queue_iterator; + if (queue_iter) + queue_iter->use(key_transaction.getQueueLock()); SCOPE_EXIT({ if (is_downloader) @@ -531,38 +575,26 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach }); LOG_TEST( - log, - "Complete based on current state (is_last_holder: {}, {})", + log, "Complete based on current state (is_last_holder: {}, {})", is_last_holder, getInfoForLogUnlocked(segment_lock)); if (is_downloader) { - if (download_state == State::DOWNLOADING) /// != in case of completeWithState + if (download_state == State::DOWNLOADING) resetDownloadingStateUnlocked(segment_lock); resetDownloaderUnlocked(segment_lock); } - if (cache_writer && (is_downloader || is_last_holder)) - { - cache_writer->finalize(); - cache_writer.reset(); - remote_file_reader.reset(); - } - switch (download_state) { - case State::SKIP_CACHE: - { - if (is_last_holder) - cache->remove(key(), offset(), cache_lock, segment_lock); - break; - } case State::DOWNLOADED: { chassert(getDownloadedSizeUnlocked(segment_lock) == range().size()); - chassert(getDownloadedSizeUnlocked(segment_lock) == std::filesystem::file_size(getPathInLocalCache())); + chassert(getDownloadedSizeUnlocked(segment_lock) == std::filesystem::file_size(file_path)); chassert(is_downloaded); chassert(!cache_writer); + + is_completed = true; break; } case State::DOWNLOADING: @@ -580,8 +612,8 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach { LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString()); - setDownloadState(State::SKIP_CACHE); - cache->remove(key(), offset(), cache_lock, segment_lock); + setDownloadState(State::SKIP_CACHE, segment_lock); + key_transaction.remove(key(), offset(), segment_lock); } else { @@ -593,42 +625,51 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach * in FileSegmentsHolder represent a contiguous range, so we can resize * it only when nobody needs it. */ - setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION, segment_lock); /// Resize this file segment by creating a copy file segment with DOWNLOADED state, /// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state, /// because otherwise an invariant that getOrSet() returns a contiguous range of file segments will be broken /// (this will be crucial for other file segment holder, not for current one). - cache->reduceSizeToDownloaded(key(), offset(), cache_lock, segment_lock); + key_transaction.reduceSizeToDownloaded(key(), offset(), segment_lock); + } + + if (cache_writer) + { + cache_writer->finalize(); + cache_writer.reset(); + remote_file_reader.reset(); } detachAssumeStateFinalized(segment_lock); + is_completed = true; } break; } + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state while completing file segment"); } - is_completed = true; LOG_TEST(log, "Completed file segment: {}", getInfoForLogUnlocked(segment_lock)); } String FileSegment::getInfoForLog() const { - std::unique_lock segment_lock(mutex); - return getInfoForLogUnlocked(segment_lock); + auto lock = segment_guard.lock(); + return getInfoForLogUnlocked(lock); } -String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment_lock) const +String FileSegment::getInfoForLogUnlocked(const FileSegmentGuard::Lock & lock) const { WriteBufferFromOwnString info; info << "File segment: " << range().toString() << ", "; info << "key: " << key().toString() << ", "; info << "state: " << download_state << ", "; - info << "downloaded size: " << getDownloadedSizeUnlocked(segment_lock) << ", "; + info << "downloaded size: " << getDownloadedSizeUnlocked(lock) << ", "; info << "reserved size: " << reserved_size << ", "; info << "downloader id: " << (downloader_id.empty() ? "None" : downloader_id) << ", "; - info << "current write offset: " << getCurrentWriteOffsetUnlocked(segment_lock) << ", "; - info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", "; + info << "current write offset: " << getCurrentWriteOffsetUnlocked(lock) << ", "; + info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(lock) << ", "; info << "caller id: " << getCallerId() << ", "; info << "detached: " << is_detached << ", "; info << "persistent: " << is_persistent; @@ -636,9 +677,9 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment return info.str(); } -void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock & segment_lock) const +void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, const FileSegmentGuard::Lock & lock) const { - e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogUnlocked(segment_lock))); + e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogUnlocked(lock))); } String FileSegment::stateToString(FileSegment::State state) @@ -663,169 +704,134 @@ String FileSegment::stateToString(FileSegment::State state) void FileSegment::assertCorrectness() const { - std::unique_lock segment_lock(mutex); - assertCorrectnessUnlocked(segment_lock); + auto lock = segment_guard.lock(); + assertCorrectnessUnlocked(lock); } -void FileSegment::assertCorrectnessUnlocked(std::unique_lock & segment_lock) const +void FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) const { - auto current_downloader = getDownloaderUnlocked(segment_lock); + auto current_downloader = getDownloaderUnlocked(lock); chassert(current_downloader.empty() == (download_state != FileSegment::State::DOWNLOADING)); chassert(!current_downloader.empty() == (download_state == FileSegment::State::DOWNLOADING)); - chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0); + chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(file_path) > 0); } -void FileSegment::throwIfDetachedUnlocked(std::unique_lock & segment_lock) const +void FileSegment::throwIfDetachedUnlocked(const FileSegmentGuard::Lock & lock) const { throw Exception( ErrorCodes::LOGICAL_ERROR, "Cache file segment is in detached state, operation not allowed. " "It can happen when cache was concurrently dropped with SYSTEM DROP FILESYSTEM CACHE FORCE. " - "Please, retry. File segment info: {}", getInfoForLogUnlocked(segment_lock)); + "Please, retry. File segment info: {}", getInfoForLogUnlocked(lock)); } void FileSegment::assertNotDetached() const { - std::unique_lock segment_lock(mutex); - assertNotDetachedUnlocked(segment_lock); + auto lock = segment_guard.lock(); + assertNotDetachedUnlocked(lock); } -void FileSegment::assertNotDetachedUnlocked(std::unique_lock & segment_lock) const +void FileSegment::assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock) const { if (is_detached) - throwIfDetachedUnlocked(segment_lock); + throwIfDetachedUnlocked(lock); } -void FileSegment::assertDetachedStatus(std::unique_lock & segment_lock) const +void FileSegment::assertDetachedStatus(const FileSegmentGuard::Lock & lock) const { /// Detached file segment is allowed to have only a certain subset of states. /// It should be either EMPTY or one of the finalized states. - if (download_state != State::EMPTY && !hasFinalizedStateUnlocked(segment_lock)) + if (download_state != State::EMPTY + && download_state != State::PARTIALLY_DOWNLOADED_NO_CONTINUATION + && !hasFinalizedStateUnlocked(lock)) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Detached file segment has incorrect state: {}", - getInfoForLogUnlocked(segment_lock)); + getInfoForLogUnlocked(lock)); } } -FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & /* cache_lock */) +FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment) { - std::unique_lock segment_lock(file_segment->mutex); + auto lock = file_segment->segment_guard.lock(); auto snapshot = std::make_shared( file_segment->offset(), file_segment->range().size(), file_segment->key(), nullptr, + nullptr, State::EMPTY, CreateFileSegmentSettings{}); snapshot->hits_count = file_segment->getHitsCount(); snapshot->ref_count = file_segment.use_count(); - snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock); + snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(lock); snapshot->download_state = file_segment->download_state; snapshot->is_persistent = file_segment->isPersistent(); return snapshot; } -bool FileSegment::hasFinalizedStateUnlocked(std::unique_lock & /* segment_lock */) const +bool FileSegment::hasFinalizedStateUnlocked(const FileSegmentGuard::Lock &) const { return download_state == State::DOWNLOADED - || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION || download_state == State::SKIP_CACHE; } bool FileSegment::isDetached() const { - std::unique_lock segment_lock(mutex); + auto lock = segment_guard.lock(); return is_detached; } -bool FileSegment::isCompleted() const -{ - std::unique_lock segment_lock(mutex); - return is_completed; -} - -void FileSegment::detach(std::lock_guard & /* cache_lock */, std::unique_lock & segment_lock) +void FileSegment::detach(const FileSegmentGuard::Lock & lock, const KeyTransaction &) { if (is_detached) return; if (download_state == State::DOWNLOADING) - resetDownloadingStateUnlocked(segment_lock); + resetDownloadingStateUnlocked(lock); else - setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION, lock); - resetDownloaderUnlocked(segment_lock); - detachAssumeStateFinalized(segment_lock); + key_transaction_creator = nullptr; + resetDownloaderUnlocked(lock); + detachAssumeStateFinalized(lock); } -void FileSegment::detachAssumeStateFinalized(std::unique_lock & segment_lock) +void FileSegment::detachAssumeStateFinalized(const FileSegmentGuard::Lock & lock) { is_detached = true; - CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments); - LOG_TEST(log, "Detached file segment: {}", getInfoForLogUnlocked(segment_lock)); + is_completed = true; + //CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments); + LOG_TEST(log, "Detached file segment: {}", getInfoForLogUnlocked(lock)); } -FileSegment::~FileSegment() +FileSegments::iterator FileSegmentsHolder::completeAndPopFrontImpl() { - std::unique_lock segment_lock(mutex); - if (is_detached) - CurrentMetrics::sub(CurrentMetrics::CacheDetachedFileSegments); + if (file_segments.front()->isCompleted()) + { + return file_segments.erase(file_segments.begin()); + } + + auto lock = file_segments.front()->cache->main_priority->lockShared(); + /// File segment pointer must be reset right after calling complete() and + /// under the same mutex, because complete() checks for segment pointers. + auto key_transaction = file_segments.front()->createKeyTransaction(/* assert_exists */false); + key_transaction->queue_lock = lock; + if (key_transaction) + file_segments.front()->completeUnlocked(*key_transaction); + + return file_segments.erase(file_segments.begin()); } FileSegmentsHolder::~FileSegmentsHolder() { - /// In CacheableReadBufferFromRemoteFS file segment's downloader removes file segments from - /// FileSegmentsHolder right after calling file_segment->complete(), so on destruction here - /// remain only uncompleted file segments. - - FileCache * cache = nullptr; - for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();) - { - auto current_file_segment_it = file_segment_it; - auto & file_segment = *current_file_segment_it; - - if (!cache) - cache = file_segment->cache; - - try - { - bool is_detached = false; - - { - std::unique_lock segment_lock(file_segment->mutex); - is_detached = file_segment->isDetached(segment_lock); - if (is_detached) - file_segment->assertDetachedStatus(segment_lock); - } - - if (is_detached) - { - /// This file segment is not owned by cache, so it will be destructed - /// at this point, therefore no completion required. - file_segment_it = file_segments.erase(current_file_segment_it); - continue; - } - - /// File segment pointer must be reset right after calling complete() and - /// under the same mutex, because complete() checks for segment pointers. - std::lock_guard cache_lock(cache->mutex); - - file_segment->completeWithoutStateUnlocked(cache_lock); - - file_segment_it = file_segments.erase(current_file_segment_it); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + file_segment_it = completeAndPopFrontImpl(); } String FileSegmentsHolder::toString() diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index df2e54c4d78..0ceb887d1ba 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -2,11 +2,13 @@ #include #include +#include #include #include #include #include +#include #include #include #include @@ -28,7 +30,10 @@ class ReadBufferFromFileBase; class FileSegment; using FileSegmentPtr = std::shared_ptr; using FileSegments = std::list; - +struct KeyTransaction; +using KeyTransactionPtr = std::unique_ptr; +struct KeyTransactionCreator; +using KeyTransactionCreatorPtr = std::unique_ptr; struct CreateFileSegmentSettings { @@ -42,6 +47,7 @@ friend class FileCache; friend struct FileSegmentsHolder; friend class FileSegmentRangeWriter; friend class StorageSystemFilesystemCache; +friend struct KeyTransaction; public: using Key = FileCacheKey; @@ -88,12 +94,11 @@ public: size_t offset_, size_t size_, const Key & key_, + KeyTransactionCreatorPtr key_transaction_creator, FileCache * cache_, State download_state_, const CreateFileSegmentSettings & create_settings); - ~FileSegment(); - State state() const; static String stateToString(FileSegment::State state); @@ -132,7 +137,7 @@ public: using UniqueId = std::pair; UniqueId getUniqueId() const { return std::pair(key(), offset()); } - String getPathInLocalCache() const; + String getPathInLocalCache() const { return file_path; } /** * ========== Methods for _any_ file segment's owner ======================== @@ -161,6 +166,12 @@ public: size_t getDownloadedSize() const; + OpenedFileCache::OpenedFilePtr getFile() const; + int getFD() const; + + OpenedFileCache::OpenedFilePtr file; + int fd; + /// Now detached status can be used in the following cases: /// 1. there is only 1 remaining file segment holder /// && it does not need this segment anymore @@ -175,13 +186,13 @@ public: /// 2. Detached file segment can still be hold by some cache users, but it's state became /// immutable at the point it was detached, any non-const / stateful method will throw an /// exception. - void detach(std::lock_guard & cache_lock, std::unique_lock & segment_lock); + void detach(const FileSegmentGuard::Lock &, const KeyTransaction &); - static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & cache_lock); + static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment); bool isDetached() const; - bool isCompleted() const; + bool isCompleted() const { return is_completed; } void assertCorrectness() const; @@ -201,10 +212,9 @@ public: /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); - /// Complete file segment with a certain state. - void completeWithState(State state); + void setBroken(); - void completeWithoutState(); + void complete(); /// Complete file segment's part which was last written. void completePartAndResetDownloader(); @@ -219,46 +229,53 @@ public: void resetRemoteFileReader(); + FileSegmentGuard::Lock lock() const { return segment_guard.lock(); } + private: - size_t getFirstNonDownloadedOffsetUnlocked(std::unique_lock & segment_lock) const; - size_t getCurrentWriteOffsetUnlocked(std::unique_lock & segment_lock) const; - size_t getDownloadedSizeUnlocked(std::unique_lock & segment_lock) const; + size_t getFirstNonDownloadedOffsetUnlocked(const FileSegmentGuard::Lock & lock) const; + size_t getCurrentWriteOffsetUnlocked(const FileSegmentGuard::Lock & lock) const; + size_t getDownloadedSizeUnlocked(const FileSegmentGuard::Lock & lock) const; - String getInfoForLogUnlocked(std::unique_lock & segment_lock) const; + String getInfoForLogUnlocked(const FileSegmentGuard::Lock & lock) const; - String getDownloaderUnlocked(std::unique_lock & segment_lock) const; - void resetDownloaderUnlocked(std::unique_lock & segment_lock); - void resetDownloadingStateUnlocked(std::unique_lock & segment_lock); + String getDownloaderUnlocked(const FileSegmentGuard::Lock & lock) const; + void resetDownloaderUnlocked(const FileSegmentGuard::Lock & lock); + void resetDownloadingStateUnlocked(const FileSegmentGuard::Lock & lock); - void setDownloadState(State state); + void setDownloadState(State state, const FileSegmentGuard::Lock & lock); - void setDownloadedUnlocked(std::unique_lock & segment_lock); - void setDownloadFailedUnlocked(std::unique_lock & segment_lock); + void setDownloadedUnlocked(const FileSegmentGuard::Lock & lock); + void setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock); - bool hasFinalizedStateUnlocked(std::unique_lock & segment_lock) const; + /// Finalized state is such a state that does not need to be completed (with complete()). + bool hasFinalizedStateUnlocked(const FileSegmentGuard::Lock & lock) const; - bool isDownloaderUnlocked(std::unique_lock & segment_lock) const; + bool isDownloaderUnlocked(const FileSegmentGuard::Lock & segment_lock) const; - bool isDetached(std::unique_lock & /* segment_lock */) const { return is_detached; } - void detachAssumeStateFinalized(std::unique_lock & segment_lock); - [[noreturn]] void throwIfDetachedUnlocked(std::unique_lock & segment_lock) const; + bool isDetached(const FileSegmentGuard::Lock &) const { return is_detached; } + void detachAssumeStateFinalized(const FileSegmentGuard::Lock & lock); + [[noreturn]] void throwIfDetachedUnlocked(const FileSegmentGuard::Lock & lock) const; - void assertDetachedStatus(std::unique_lock & segment_lock) const; + void assertDetachedStatus(const FileSegmentGuard::Lock & lock) const; void assertNotDetached() const; - void assertNotDetachedUnlocked(std::unique_lock & segment_lock) const; - void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const; - void assertCorrectnessUnlocked(std::unique_lock & segment_lock) const; + void assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock) const; + void assertIsDownloaderUnlocked(const std::string & operation, const FileSegmentGuard::Lock & lock) const; + void assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) const; + + KeyTransactionPtr createKeyTransaction(bool assert_exists = true) const; /// complete() without any completion state is called from destructor of /// FileSegmentsHolder. complete() might check if the caller of the method /// is the last alive holder of the segment. Therefore, complete() and destruction /// of the file segment pointer must be done under the same cache mutex. - void completeWithoutStateUnlocked(std::lock_guard & cache_lock); - void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock); + void completeUnlocked(KeyTransaction & key_transaction); - void completePartAndResetDownloaderUnlocked(std::unique_lock & segment_lock); + void completePartAndResetDownloaderUnlocked(const FileSegmentGuard::Lock & segment_lock); - void wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock & segment_lock) const; + void wrapWithCacheInfo( + Exception & e, const String & message, const FileSegmentGuard::Lock & segment_lock) const; + + bool isLastHolder(const KeyTransaction & key_transaction) const; Range segment_range; @@ -277,7 +294,8 @@ private: /// 1. cache lock /// 2. segment lock - mutable std::mutex mutex; + mutable FileSegmentGuard segment_guard; + KeyTransactionCreatorPtr key_transaction_creator; std::condition_variable cv; /// Protects downloaded_size access with actual write into fs. @@ -289,6 +307,7 @@ private: mutable std::mutex download_mutex; Key file_key; + const std::string file_path; FileCache * cache; Poco::Logger * log; @@ -296,7 +315,7 @@ private: /// "detached" file segment means that it is not owned by cache ("detached" from cache). /// In general case, all file segments are owned by cache. bool is_detached = false; - bool is_completed = false; + std::atomic is_completed = false; bool is_downloaded{false}; @@ -314,13 +333,35 @@ struct FileSegmentsHolder : private boost::noncopyable explicit FileSegmentsHolder(FileSegments && file_segments_) : file_segments(std::move(file_segments_)) {} - FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {} - ~FileSegmentsHolder(); + bool empty() const { return file_segments.empty(); } + + size_t size() const { return file_segments.size(); } + String toString(); + void popFront() { completeAndPopFrontImpl(); } + + FileSegment & front() { return *file_segments.front(); } + + FileSegment & back() { return *file_segments.back(); } + + FileSegment & add(FileSegmentPtr && file_segment) + { + file_segments.push_back(file_segment); + return *file_segments.back(); + } + + FileSegments::iterator begin() { return file_segments.begin(); } + FileSegments::iterator end() { return file_segments.end(); } + +private: FileSegments file_segments{}; + + FileSegments::iterator completeAndPopFrontImpl(); }; +using FileSegmentsHolderPtr = std::unique_ptr; + } diff --git a/src/Interpreters/Cache/Guards.h b/src/Interpreters/Cache/Guards.h new file mode 100644 index 00000000000..14109f11cb9 --- /dev/null +++ b/src/Interpreters/Cache/Guards.h @@ -0,0 +1,79 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +/** + * Guard for the whole cache object. + */ +struct CacheGuard +{ + struct Lock + { + explicit Lock(CacheGuard & guard) : lock(guard.mutex) {} + std::unique_lock lock; + }; + + std::mutex mutex; + + Lock lock() { return Lock(*this); } +}; + +/** + * Guard for a set of keys. + * One guard per key prefix (first three digits of the path hash). + */ +struct KeyPrefixGuard +{ + struct Lock + { + explicit Lock(KeyPrefixGuard & guard) : lock(guard.mutex) {} + std::unique_lock lock; + }; + + std::mutex mutex; + + Lock lock() { return Lock(*this); } + + KeyPrefixGuard() = default; +}; +using KeyPrefixGuardPtr = std::shared_ptr; + +struct CachePriorityQueueGuard +{ + struct Lock + { + explicit Lock(CachePriorityQueueGuard & guard) : lock(guard.mutex) {} + std::unique_lock lock; + }; + + std::mutex mutex; + + Lock lock() { return Lock(*this); } + std::shared_ptr lockShared() { return std::make_shared(*this); } + + CachePriorityQueueGuard() = default; +}; + +/** + * Guard for a file segment. + * Cache guard > key prefix guard > file segment guard. + */ +struct FileSegmentGuard +{ + struct Lock + { + explicit Lock(FileSegmentGuard & guard) : lock(guard.mutex) {} + std::unique_lock lock; + }; + + std::mutex mutex; + + Lock lock() { return Lock(*this); } +}; + +} diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index de5e3f1428c..2abf0ba37e4 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -5,12 +5,17 @@ #include #include #include +#include namespace DB { class IFileCachePriority; -using FileCachePriorityPtr = std::shared_ptr; +using FileCachePriorityPtr = std::unique_ptr; +struct KeyTransaction; +using KeyTransactionPtr = std::unique_ptr; +struct KeyTransactionCreator; +using KeyTransactionCreatorPtr = std::unique_ptr; /// IFileCachePriority is used to maintain the priority of cached data. class IFileCachePriority @@ -18,8 +23,10 @@ class IFileCachePriority public: class IIterator; using Key = FileCacheKey; - using ReadIterator = std::unique_ptr; - using WriteIterator = std::shared_ptr; + using KeyAndOffset = FileCacheKeyAndOffset; + using Iterator = std::shared_ptr; + using ConstIterator = std::shared_ptr; + using Lock = CachePriorityQueueGuard::Lock; struct FileCacheRecord { @@ -27,8 +34,11 @@ public: size_t offset; size_t size; size_t hits = 0; + KeyTransactionCreatorPtr key_transaction_creator; - FileCacheRecord(const Key & key_, size_t offset_, size_t size_) : key(key_), offset(offset_), size(size_) { } + FileCacheRecord( + const Key & key_, size_t offset_, size_t size_, KeyTransactionCreatorPtr key_transaction_creator_) + : key(key_), offset(offset_), size(size_), key_transaction_creator(std::move(key_transaction_creator_)) { } }; /// It provides an iterator to traverse the cache priority. Under normal circumstances, @@ -48,48 +58,59 @@ public: virtual size_t hits() const = 0; - /// Point the iterator to the next higher priority cache record. - virtual void next() const = 0; + virtual KeyTransactionPtr createKeyTransaction(const CachePriorityQueueGuard::Lock &) = 0; - virtual bool valid() const = 0; + /// Point the iterator to the next higher priority cache record. + virtual void next(const CachePriorityQueueGuard::Lock &) const = 0; + + virtual bool valid(const CachePriorityQueueGuard::Lock &) const = 0; /// Mark a cache record as recently used, it will update the priority /// of the cache record according to different cache algorithms. - virtual void use(std::lock_guard &) = 0; + /// Return result hits count. + virtual size_t use(const CachePriorityQueueGuard::Lock &) = 0; - /// Deletes an existing cached record. And to avoid pointer suspension - /// the iterator should automatically point to the next record. - virtual void removeAndGetNext(std::lock_guard &) = 0; + /// Deletes an existing cached record. Return iterator to the next value. + virtual Iterator remove(const CachePriorityQueueGuard::Lock &) = 0; - virtual void incrementSize(size_t, std::lock_guard &) = 0; + virtual void incrementSize(ssize_t, const CachePriorityQueueGuard::Lock &) = 0; }; -public: virtual ~IFileCachePriority() = default; + /// Lock current priority queue. All methods must be called under this lock. + CachePriorityQueueGuard::Lock lock() { return guard.lock(); } + std::shared_ptr lockShared() { return guard.lockShared(); } + /// Add a cache record that did not exist before, and throw a /// logical exception if the cache block already exists. - virtual WriteIterator add(const Key & key, size_t offset, size_t size, std::lock_guard & cache_lock) = 0; + virtual Iterator add( + const Key & key, + size_t offset, + size_t size, + KeyTransactionCreatorPtr key_transaction_creator, + const CachePriorityQueueGuard::Lock &) = 0; /// This method is used for assertions in debug mode. So we do not care about complexity here. /// Query whether a cache record exists. If it exists, return true. If not, return false. - virtual bool contains(const Key & key, size_t offset, std::lock_guard & cache_lock) = 0; + virtual bool contains(const Key & key, size_t offset, const CachePriorityQueueGuard::Lock &) = 0; - virtual void removeAll(std::lock_guard & cache_lock) = 0; - - /// Returns an iterator pointing to the lowest priority cached record. - /// We can traverse all cached records through the iterator's next(). - virtual ReadIterator getLowestPriorityReadIterator(std::lock_guard & cache_lock) = 0; + virtual void removeAll(const CachePriorityQueueGuard::Lock &) = 0; /// The same as getLowestPriorityReadIterator(), but it is writeable. - virtual WriteIterator getLowestPriorityWriteIterator(std::lock_guard & cache_lock) = 0; + virtual Iterator getLowestPriorityIterator(const CachePriorityQueueGuard::Lock &) = 0; - virtual size_t getElementsNum(std::lock_guard & cache_lock) const = 0; + virtual size_t getElementsNum(const CachePriorityQueueGuard::Lock &) const = 0; - size_t getCacheSize(std::lock_guard &) const { return cache_size; } + size_t getCacheSize(const CachePriorityQueueGuard::Lock &) const { return cache_size; } protected: + CachePriorityQueueGuard guard; + size_t max_cache_size = 0; size_t cache_size = 0; }; + +using FileCachePriorityPtr = std::unique_ptr; + }; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 8010b9c682b..6c370bec367 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace CurrentMetrics @@ -15,7 +16,12 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, size_t offset, size_t size, std::lock_guard &) +IFileCachePriority::Iterator LRUFileCachePriority::add( + const Key & key, + size_t offset, + size_t size, + KeyTransactionCreatorPtr key_transaction_creator, + const CachePriorityQueueGuard::Lock &) { #ifndef NDEBUG for (const auto & entry : queue) @@ -28,7 +34,7 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz } #endif - auto iter = queue.insert(queue.end(), FileCacheRecord(key, offset, size)); + auto iter = queue.insert(queue.end(), FileCacheRecord(key, offset, size, std::move(key_transaction_creator))); cache_size += size; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); @@ -39,7 +45,12 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz return std::make_shared(this, iter); } -bool LRUFileCachePriority::contains(const Key & key, size_t offset, std::lock_guard &) +KeyTransactionPtr LRUFileCachePriority::LRUFileCacheIterator::createKeyTransaction(const CachePriorityQueueGuard::Lock &) +{ + return queue_iter->key_transaction_creator->create(); +} + +bool LRUFileCachePriority::contains(const Key & key, size_t offset, const CachePriorityQueueGuard::Lock &) { for (const auto & record : queue) { @@ -49,7 +60,7 @@ bool LRUFileCachePriority::contains(const Key & key, size_t offset, std::lock_gu return false; } -void LRUFileCachePriority::removeAll(std::lock_guard &) +void LRUFileCachePriority::removeAll(const CachePriorityQueueGuard::Lock &) { CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, cache_size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements, queue.size()); @@ -60,28 +71,31 @@ void LRUFileCachePriority::removeAll(std::lock_guard &) cache_size = 0; } +// LRUFileCachePriority::KeyAndOffset LRUFileCachePriority::pop(const CachePriorityQueueGuard::Lock & lock) +// { +// auto remove_it = getLowestPriorityIterator(lock); +// KeyAndOffset result(remove_it->key(), remove_it->offset()); +// remove_it->removeAndGetNext(lock); +// return result; +// } + LRUFileCachePriority::LRUFileCacheIterator::LRUFileCacheIterator( LRUFileCachePriority * cache_priority_, LRUFileCachePriority::LRUQueueIterator queue_iter_) : cache_priority(cache_priority_), queue_iter(queue_iter_) { } -IFileCachePriority::ReadIterator LRUFileCachePriority::getLowestPriorityReadIterator(std::lock_guard &) -{ - return std::make_unique(this, queue.begin()); -} - -IFileCachePriority::WriteIterator LRUFileCachePriority::getLowestPriorityWriteIterator(std::lock_guard &) +IFileCachePriority::Iterator LRUFileCachePriority::getLowestPriorityIterator(const CachePriorityQueueGuard::Lock &) { return std::make_shared(this, queue.begin()); } -size_t LRUFileCachePriority::getElementsNum(std::lock_guard &) const +size_t LRUFileCachePriority::getElementsNum(const CachePriorityQueueGuard::Lock &) const { return queue.size(); } -void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guard &) +LRUFileCachePriority::Iterator LRUFileCachePriority::LRUFileCacheIterator::remove(const CachePriorityQueueGuard::Lock &) { cache_priority->cache_size -= queue_iter->size; @@ -90,20 +104,22 @@ void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guar LOG_TRACE(cache_priority->log, "Removed entry from LRU queue, key: {}, offset: {}", queue_iter->key.toString(), queue_iter->offset); - queue_iter = cache_priority->queue.erase(queue_iter); + auto next = std::make_shared(cache_priority, cache_priority->queue.erase(queue_iter)); + queue_iter = cache_priority->queue.end(); + return next; } -void LRUFileCachePriority::LRUFileCacheIterator::incrementSize(size_t size_increment, std::lock_guard &) +void LRUFileCachePriority::LRUFileCacheIterator::incrementSize(ssize_t size_increment, const CachePriorityQueueGuard::Lock &) { cache_priority->cache_size += size_increment; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size_increment); queue_iter->size += size_increment; } -void LRUFileCachePriority::LRUFileCacheIterator::use(std::lock_guard &) +size_t LRUFileCachePriority::LRUFileCacheIterator::use(const CachePriorityQueueGuard::Lock &) { - queue_iter->hits++; cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); + return ++queue_iter->hits; } }; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 2345d3c47db..16565da8bd8 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -19,17 +20,20 @@ private: public: LRUFileCachePriority() = default; - WriteIterator add(const Key & key, size_t offset, size_t size, std::lock_guard &) override; + Iterator add( + const Key & key, + size_t offset, + size_t size, + KeyTransactionCreatorPtr key_transaction_creator, + const CachePriorityQueueGuard::Lock &) override; - bool contains(const Key & key, size_t offset, std::lock_guard &) override; + bool contains(const Key & key, size_t offset, const CachePriorityQueueGuard::Lock &) override; - void removeAll(std::lock_guard &) override; + void removeAll(const CachePriorityQueueGuard::Lock &) override; - ReadIterator getLowestPriorityReadIterator(std::lock_guard &) override; + Iterator getLowestPriorityIterator(const CachePriorityQueueGuard::Lock &) override; - WriteIterator getLowestPriorityWriteIterator(std::lock_guard &) override; - - size_t getElementsNum(std::lock_guard &) const override; + size_t getElementsNum(const CachePriorityQueueGuard::Lock &) const override; private: LRUQueue queue; @@ -39,11 +43,13 @@ private: class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator { public: - LRUFileCacheIterator(LRUFileCachePriority * cache_priority_, LRUFileCachePriority::LRUQueueIterator queue_iter_); + LRUFileCacheIterator( + LRUFileCachePriority * cache_priority_, + LRUFileCachePriority::LRUQueueIterator queue_iter_); - void next() const override { queue_iter++; } + void next(const CachePriorityQueueGuard::Lock &) const override { queue_iter++; } - bool valid() const override { return queue_iter != cache_priority->queue.end(); } + bool valid(const CachePriorityQueueGuard::Lock &) const override { return queue_iter != cache_priority->queue.end(); } const Key & key() const override { return queue_iter->key; } @@ -53,11 +59,13 @@ public: size_t hits() const override { return queue_iter->hits; } - void removeAndGetNext(std::lock_guard &) override; + KeyTransactionPtr createKeyTransaction(const CachePriorityQueueGuard::Lock &) override; - void incrementSize(size_t size_increment, std::lock_guard &) override; + Iterator remove(const CachePriorityQueueGuard::Lock &) override; - void use(std::lock_guard &) override; + void incrementSize(ssize_t size_increment, const CachePriorityQueueGuard::Lock &) override; + + size_t use(const CachePriorityQueueGuard::Lock &) override; private: LRUFileCachePriority * cache_priority; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e56aba9c0a..b0d4e9719ba 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3628,7 +3628,7 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co { if (!shared->asynchronous_remote_fs_reader) { - auto pool_size = config.getUInt(".threadpool_remote_fs_reader_pool_size", 100); + auto pool_size = config.getUInt(".threadpool_remote_fs_reader_pool_size", 200); auto queue_size = config.getUInt(".threadpool_remote_fs_reader_queue_size", 1000000); shared->asynchronous_remote_fs_reader = std::make_unique(pool_size, queue_size); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d05fd70e074..563c7db510c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -334,12 +334,12 @@ BlockIO InterpreterSystemQuery::execute() { auto caches = FileCacheFactory::instance().getAll(); for (const auto & [_, cache_data] : caches) - cache_data->cache->removeIfReleasable(); + cache_data->cache->removeAllReleasable(); } else { auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path); - cache->removeIfReleasable(); + cache->removeAllReleasable(); } break; } diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 5bad3e9bba2..a81e716c347 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -34,6 +34,20 @@ void assertRange( ASSERT_EQ(range.right, expected_range.right); ASSERT_EQ(file_segment->state(), expected_state); } +void assertRange( + [[maybe_unused]] size_t assert_n, DB::FileSegment & file_segment, + const DB::FileSegment::Range & expected_range, DB::FileSegment::State expected_state) +{ + auto range = file_segment.range(); + + std::cerr << fmt::format("\nAssert #{} : {} == {} (state: {} == {})\n", assert_n, + range.toString(), expected_range.toString(), + toString(file_segment.state()), toString(expected_state)); + + ASSERT_EQ(range.left, expected_range.left); + ASSERT_EQ(range.right, expected_range.right); + ASSERT_EQ(file_segment.state(), expected_state); +} void printRanges(const auto & segments) { @@ -42,9 +56,9 @@ void printRanges(const auto & segments) std::cerr << '\n' << segment->range().toString() << " (state: " + DB::FileSegment::stateToString(segment->state()) + ")" << "\n"; } -std::vector fromHolder(const DB::FileSegmentsHolder & holder) +std::vector fromHolder(const DB::FileSegmentsHolderPtr & holder) { - return std::vector(holder.file_segments.begin(), holder.file_segments.end()); + return std::vector(holder->begin(), holder->end()); } String getFileSegmentPath(const String & base_path, const DB::FileCache::Key & key, size_t offset) @@ -74,13 +88,13 @@ void prepareAndDownload(DB::FileSegmentPtr file_segment) download(file_segment); } -void complete(const DB::FileSegmentsHolder & holder) +void complete(DB::FileSegmentsHolderPtr holder) { - for (const auto & file_segment : holder.file_segments) + for (auto it = holder->begin(); it != holder->end(); ++it) { - ASSERT_TRUE(file_segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(file_segment); - file_segment->completeWithoutState(); + ASSERT_TRUE((*it)->getOrSetDownloader() == DB::FileSegment::getCallerId()); + prepareAndDownload(*it); + (*it)->complete(); } } @@ -107,7 +121,7 @@ TEST(FileCache, get) { auto cache = DB::FileCache(cache_base_path, settings); cache.initialize(); - auto key = cache.hash("key1"); + auto key = cache.createKeyForPath("key1"); { auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9] @@ -127,7 +141,7 @@ TEST(FileCache, get) assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); download(segments[0]); - segments[0]->completeWithoutState(); + segments[0]->complete(); assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } @@ -148,7 +162,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(segments[1]); - segments[1]->completeWithoutState(); + segments[1]->complete(); assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } @@ -205,7 +219,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(segments[2]); - segments[2]->completeWithoutState(); + segments[2]->complete(); assertRange(14, segments[3], DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); @@ -246,7 +260,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(segments[3]); - segments[3]->completeWithoutState(); + segments[3]->complete(); ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); } @@ -269,13 +283,19 @@ TEST(FileCache, get) ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(segments[0]); prepareAndDownload(segments[2]); - segments[0]->completeWithoutState(); - segments[2]->completeWithoutState(); + segments[0]->complete(); + segments[2]->complete(); } /// Current cache: [____][_] [][___][__] /// ^ ^ ^^^ ^^ ^ - /// 17 21 2324 26 28 + /// 17 21 2324 26 27 + ASSERT_EQ(cache.getFileSegmentsNum(), 5); + assertRange(25, cache.get(key, 17, 4)->front(), DB::FileSegment::Range(17, 20), DB::FileSegment::State::DOWNLOADED); + assertRange(26, cache.get(key, 21, 1)->front(), DB::FileSegment::Range(21, 21), DB::FileSegment::State::DOWNLOADED); + assertRange(27, cache.get(key, 23, 1)->front(), DB::FileSegment::Range(23, 23), DB::FileSegment::State::DOWNLOADED); + assertRange(28, cache.get(key, 24, 3)->front(), DB::FileSegment::Range(24, 26), DB::FileSegment::State::DOWNLOADED); + assertRange(29, cache.get(key, 27, 1)->front(), DB::FileSegment::Range(27, 27), DB::FileSegment::State::DOWNLOADED); { auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4] @@ -292,8 +312,8 @@ TEST(FileCache, get) ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(s5[0]); prepareAndDownload(s1[0]); - s5[0]->completeWithoutState(); - s1[0]->completeWithoutState(); + s5[0]->complete(); + s1[0]->complete(); /// Current cache: [___] [_][___][_] [__] /// ^ ^ ^ ^ ^ ^ ^ ^ @@ -313,6 +333,8 @@ TEST(FileCache, get) /// All cache is now unreleasable because pointers are still hold auto holder6 = cache.getOrSet(key, 0, 40, {}); + std::cerr << "kssenii: " << holder6->toString() << "\n\n"; + std::cerr << "kssenii: " << cache.dumpStructure(key) << "\n\n"; auto f = fromHolder(holder6); ASSERT_EQ(f.size(), 9); @@ -395,7 +417,7 @@ TEST(FileCache, get) } prepareAndDownload(segments[2]); - segments[2]->completeWithoutState(); + segments[2]->complete(); ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); other_1.join(); @@ -410,10 +432,10 @@ TEST(FileCache, get) /// state is changed not manually via segment->completeWithState(state) but from destructor of holder /// and notify_all() is also called from destructor of holder. - std::optional holder; - holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25] + DB::FileSegmentsHolderPtr holder; + holder = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] - auto segments = fromHolder(*holder); + auto segments = fromHolder(holder); ASSERT_EQ(segments.size(), 3); assertRange(38, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); @@ -438,7 +460,7 @@ TEST(FileCache, get) thread_status_1.attachQueryContext(query_context_1); auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again - auto segments_2 = fromHolder(*holder); + auto segments_2 = fromHolder(holder); ASSERT_EQ(segments_2.size(), 3); assertRange(41, segments_2[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); @@ -460,7 +482,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); prepareAndDownload(segments_2[1]); - segments_2[1]->completeWithoutState(); + segments_2[1]->complete(); }); { @@ -484,7 +506,7 @@ TEST(FileCache, get) auto cache2 = DB::FileCache(cache_base_path, settings); cache2.initialize(); - auto key = cache2.hash("key1"); + auto key = cache2.createKeyForPath("key1"); auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29] @@ -505,7 +527,7 @@ TEST(FileCache, get) settings2.max_file_segment_size = 10; auto cache2 = DB::FileCache(caches_dir / "cache2", settings2); cache2.initialize(); - auto key = cache2.hash("key1"); + auto key = cache2.createKeyForPath("key1"); auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24] auto segments1 = fromHolder(holder1); diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 9cadcc0ebb0..4965e8beb58 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -4,8 +4,16 @@ #include #include #include +#include #include +namespace ProfileEvents +{ + extern const Event ThreadpoolRun; + extern const Event ThreadpoolRun2; + extern const Event ThreadpoolRun3; +} + namespace DB { @@ -15,11 +23,14 @@ using ThreadPoolCallbackRunner = std::function(std::function /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. template -ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) +ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, std::string && thread_name) { - return [pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](std::function && callback, size_t priority) mutable -> std::future + return [pool = &pool, thread_group = CurrentThread::getGroup(), thread_name = std::move(thread_name)] + (std::function && callback, size_t priority) mutable -> std::future { - auto task = std::make_shared>([thread_group, thread_name, callback = std::move(callback)]() -> Result + ElapsedUSProfileEventIncrement measure_time(ProfileEvents::ThreadpoolRun); + auto task = std::make_shared>( + [thread_group, thread_name, callback = std::move(callback)]() -> Result { if (thread_group) CurrentThread::attachTo(thread_group); @@ -37,7 +48,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, con auto future = task->get_future(); /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". - pool->scheduleOrThrow([task]{ (*task)(); }, -priority); + pool->scheduleOrThrow([task = std::move(task)]{ (*task)(); }, -priority); return future; }; diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index b2194020dca..c95a1104f83 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -66,7 +66,7 @@ bool AsynchronousReadBufferFromHDFS::hasPendingDataToRead() std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size) { IAsynchronousReader::Request request; - request.descriptor = std::make_shared(impl); + request.descriptor = std::make_shared(*impl); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 20076603522..5f22edd1032 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -80,7 +80,7 @@ Pipe StorageSystemRemoteDataPaths::read( if (cache) { - auto cache_paths = cache->tryGetCachePaths(cache->hash(object.getPathKeyForCache())); + auto cache_paths = cache->tryGetCachePaths(cache->createKeyForPath(object.getPathKeyForCache())); col_cache_paths->insert(Array(cache_paths.begin(), cache_paths.end())); } else