Refactoring

This commit is contained in:
kssenii 2023-11-23 13:16:04 +01:00
parent 00177a8016
commit dc5e5f3b20
12 changed files with 112 additions and 109 deletions

View File

@ -1046,7 +1046,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
} }
bool limits_satisfied; bool limits_satisfied;
IFileCachePriority::Iterator cache_it; IFileCachePriority::IteratorPtr cache_it;
{ {
auto lock = lockCache(); auto lock = lockCache();
limits_satisfied = (size_limit == 0 || main_priority->getSize(lock) + size <= size_limit) limits_satisfied = (size_limit == 0 || main_priority->getSize(lock) + size <= size_limit)

View File

@ -74,7 +74,6 @@ public:
using QueryLimit = DB::FileCacheQueryLimit; using QueryLimit = DB::FileCacheQueryLimit;
using Priority = IFileCachePriority; using Priority = IFileCachePriority;
using PriorityEntry = IFileCachePriority::Entry; using PriorityEntry = IFileCachePriority::Entry;
using PriorityIterator = IFileCachePriority::Iterator;
FileCache(const std::string & cache_name, const FileCacheSettings & settings); FileCache(const std::string & cache_name, const FileCacheSettings & settings);
@ -205,7 +204,7 @@ private:
const size_t queue_size; const size_t queue_size;
std::unique_ptr<LRUFileCachePriority> queue; std::unique_ptr<LRUFileCachePriority> queue;
using Records = std::unordered_map<KeyAndOffset, PriorityIterator, FileCacheKeyAndOffsetHash>; using Records = std::unordered_map<KeyAndOffset, Priority::IteratorPtr, FileCacheKeyAndOffsetHash>;
Records records; Records records;
}; };

View File

@ -54,7 +54,7 @@ FileSegment::FileSegment(
bool background_download_enabled_, bool background_download_enabled_,
FileCache * cache_, FileCache * cache_,
std::weak_ptr<KeyMetadata> key_metadata_, std::weak_ptr<KeyMetadata> key_metadata_,
Priority::Iterator queue_iterator_) Priority::IteratorPtr queue_iterator_)
: file_key(key_) : file_key(key_)
, segment_range(offset_, offset_ + size_ - 1) , segment_range(offset_, offset_ + size_ - 1)
, segment_kind(settings.kind) , segment_kind(settings.kind)
@ -146,13 +146,13 @@ size_t FileSegment::getReservedSize() const
return reserved_size; return reserved_size;
} }
FileSegment::Priority::Iterator FileSegment::getQueueIterator() const FileSegment::Priority::IteratorPtr FileSegment::getQueueIterator() const
{ {
auto lock = lockFileSegment(); auto lock = lockFileSegment();
return queue_iterator; return queue_iterator;
} }
void FileSegment::setQueueIterator(Priority::Iterator iterator) void FileSegment::setQueueIterator(Priority::IteratorPtr iterator)
{ {
auto lock = lockFileSegment(); auto lock = lockFileSegment();
if (queue_iterator) if (queue_iterator)
@ -775,7 +775,7 @@ bool FileSegment::assertCorrectness() const
bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const
{ {
auto check_iterator = [this](const Priority::Iterator & it) auto check_iterator = [this](const Priority::IteratorPtr & it)
{ {
UNUSED(this); UNUSED(this);
if (!it) if (!it)
@ -917,6 +917,10 @@ void FileSegment::increasePriority()
return; return;
} }
/// Priority can be increased only for downloaded file segments.
if (download_state != State::DOWNLOADED)
return;
auto it = getQueueIterator(); auto it = getQueueIterator();
if (it) if (it)
{ {

View File

@ -113,7 +113,7 @@ public:
bool background_download_enabled_ = false, bool background_download_enabled_ = false,
FileCache * cache_ = nullptr, FileCache * cache_ = nullptr,
std::weak_ptr<KeyMetadata> key_metadata_ = std::weak_ptr<KeyMetadata>(), std::weak_ptr<KeyMetadata> key_metadata_ = std::weak_ptr<KeyMetadata>(),
Priority::Iterator queue_iterator_ = Priority::Iterator{}); Priority::IteratorPtr queue_iterator_ = nullptr);
~FileSegment() = default; ~FileSegment() = default;
@ -219,9 +219,9 @@ public:
FileSegmentGuard::Lock lock() const { return segment_guard.lock(); } FileSegmentGuard::Lock lock() const { return segment_guard.lock(); }
Priority::Iterator getQueueIterator() const; Priority::IteratorPtr getQueueIterator() const;
void setQueueIterator(Priority::Iterator iterator); void setQueueIterator(Priority::IteratorPtr iterator);
KeyMetadataPtr tryGetKeyMetadata() const; KeyMetadataPtr tryGetKeyMetadata() const;
@ -309,7 +309,7 @@ private:
mutable FileSegmentGuard segment_guard; mutable FileSegmentGuard segment_guard;
std::weak_ptr<KeyMetadata> key_metadata; std::weak_ptr<KeyMetadata> key_metadata;
mutable Priority::Iterator queue_iterator; /// Iterator is put here on first reservation attempt, if successful. mutable Priority::IteratorPtr queue_iterator; /// Iterator is put here on first reservation attempt, if successful.
FileCache * cache; FileCache * cache;
std::condition_variable cv; std::condition_variable cv;

View File

@ -12,12 +12,10 @@ namespace DB
struct FileCacheReserveStat; struct FileCacheReserveStat;
class EvictionCandidates; class EvictionCandidates;
/// IFileCachePriority is used to maintain the priority of cached data.
class IFileCachePriority : private boost::noncopyable class IFileCachePriority : private boost::noncopyable
{ {
public: public:
using Key = FileCacheKey; using Key = FileCacheKey;
using KeyAndOffset = FileCacheKeyAndOffset;
struct Entry struct Entry
{ {
@ -32,10 +30,10 @@ public:
size_t hits = 0; size_t hits = 0;
}; };
class IIterator class Iterator
{ {
public: public:
virtual ~IIterator() = default; virtual ~Iterator() = default;
virtual const Entry & getEntry() const = 0; virtual const Entry & getEntry() const = 0;
@ -47,7 +45,7 @@ public:
virtual void invalidate() = 0; virtual void invalidate() = 0;
}; };
using Iterator = std::shared_ptr<IIterator>; using IteratorPtr = std::shared_ptr<Iterator>;
IFileCachePriority(size_t max_size_, size_t max_elements_); IFileCachePriority(size_t max_size_, size_t max_elements_);
@ -61,19 +59,18 @@ public:
virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0;
virtual Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; virtual IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0;
virtual void shuffle(const CacheGuard::Lock &) = 0; virtual void shuffle(const CacheGuard::Lock &) = 0;
virtual FileSegments dump(const CacheGuard::Lock &) = 0; virtual FileSegments dump(const CacheGuard::Lock &) = 0;
using FinalizeEvictionFunc = std::function<void(const CacheGuard::Lock & lk)>; using FinalizeEvictionFunc = std::function<void(const CacheGuard::Lock & lk)>;
virtual bool collectCandidatesForEviction( virtual bool collectCandidatesForEviction(
size_t size, size_t size,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::Iterator it, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const CacheGuard::Lock &) = 0; const CacheGuard::Lock &) = 0;

View File

@ -26,7 +26,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
IFileCachePriority::Iterator LRUFileCachePriority::add( IFileCachePriority::IteratorPtr LRUFileCachePriority::add(
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
@ -67,7 +67,7 @@ std::unique_ptr<LRUFileCachePriority::LRUIterator> LRUFileCachePriority::add(Ent
entry.key, entry.offset, entry.size, current_size, size_limit); entry.key, entry.offset, entry.size, current_size, size_limit);
} }
auto it = queue.insert(queue.end(), entry); auto iterator = queue.insert(queue.end(), entry);
updateSize(entry.size); updateSize(entry.size);
updateElementsCount(1); updateElementsCount(1);
@ -76,7 +76,7 @@ std::unique_ptr<LRUFileCachePriority::LRUIterator> LRUFileCachePriority::add(Ent
log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", log, "Added entry into LRU queue, key: {}, offset: {}, size: {}",
entry.key, entry.offset, entry.size); entry.key, entry.offset, entry.size);
return std::make_unique<LRUIterator>(this, it); return std::make_unique<LRUIterator>(this, iterator);
} }
LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::iterator it, const CacheGuard::Lock &) LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::iterator it, const CacheGuard::Lock &)
@ -107,9 +107,11 @@ void LRUFileCachePriority::updateElementsCount(int64_t num)
CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num); CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num);
} }
LRUFileCachePriority::LRUIterator::LRUIterator(
LRUFileCachePriority::LRUIterator::LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_) LRUFileCachePriority * cache_priority_,
: cache_priority(cache_priority_), queue_iter(queue_iter_) LRUQueue::iterator iterator_)
: cache_priority(cache_priority_)
, iterator(iterator_)
{ {
} }
@ -166,7 +168,11 @@ bool LRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) co
return canFit(size, 0, 0, lock); return canFit(size, 0, 0, lock);
} }
bool LRUFileCachePriority::canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const bool LRUFileCachePriority::canFit(
size_t size,
size_t released_size_assumption,
size_t released_elements_assumption,
const CacheGuard::Lock &) const
{ {
return (max_size == 0 || (current_size + size - released_size_assumption <= max_size)) return (max_size == 0 || (current_size + size - released_size_assumption <= max_size))
&& (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements); && (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements);
@ -176,7 +182,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
size_t size, size_t size,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::Iterator, IFileCachePriority::IteratorPtr,
FinalizeEvictionFunc &, FinalizeEvictionFunc &,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
@ -217,7 +223,8 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
return can_fit(); return can_fit();
} }
std::unique_ptr<LRUFileCachePriority::LRUIterator> LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) std::unique_ptr<LRUFileCachePriority::LRUIterator>
LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &)
{ {
const auto & entry = it.getEntry(); const auto & entry = it.getEntry();
if (entry.size == 0) if (entry.size == 0)
@ -240,14 +247,14 @@ std::unique_ptr<LRUFileCachePriority::LRUIterator> LRUFileCachePriority::move(LR
} }
#endif #endif
queue.splice(queue.end(), other.queue, it.queue_iter); queue.splice(queue.end(), other.queue, it.iterator);
updateSize(entry.size); updateSize(entry.size);
updateElementsCount(1); updateElementsCount(1);
other.updateSize(-entry.size); other.updateSize(-entry.size);
other.updateElementsCount(-1); other.updateElementsCount(-1);
return std::make_unique<LRUIterator>(this, it.queue_iter); return std::make_unique<LRUIterator>(this, it.iterator);
} }
FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock)
@ -263,48 +270,48 @@ FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock)
void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock) void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock)
{ {
checkUsable(); assertValid();
cache_priority->remove(queue_iter, lock); cache_priority->remove(iterator, lock);
queue_iter = LRUQueue::iterator{}; iterator = LRUQueue::iterator{};
} }
void LRUFileCachePriority::LRUIterator::invalidate() void LRUFileCachePriority::LRUIterator::invalidate()
{ {
checkUsable(); assertValid();
LOG_TEST( LOG_TEST(
cache_priority->log, cache_priority->log,
"Invalidating entry in LRU queue. Key: {}, offset: {}, previous size: {}", "Invalidating entry in LRU queue. Key: {}, offset: {}, previous size: {}",
queue_iter->key, queue_iter->offset, queue_iter->size); iterator->key, iterator->offset, iterator->size);
cache_priority->updateSize(-queue_iter->size); cache_priority->updateSize(-iterator->size);
cache_priority->updateElementsCount(-1); cache_priority->updateElementsCount(-1);
queue_iter->size = 0; iterator->size = 0;
} }
void LRUFileCachePriority::LRUIterator::updateSize(int64_t size) void LRUFileCachePriority::LRUIterator::updateSize(int64_t size)
{ {
checkUsable(); assertValid();
LOG_TEST( LOG_TEST(
cache_priority->log, cache_priority->log,
"Update size with {} in LRU queue for key: {}, offset: {}, previous size: {}", "Update size with {} in LRU queue for key: {}, offset: {}, previous size: {}",
size, queue_iter->key, queue_iter->offset, queue_iter->size); size, iterator->key, iterator->offset, iterator->size);
cache_priority->updateSize(size); cache_priority->updateSize(size);
queue_iter->size += size; iterator->size += size;
} }
size_t LRUFileCachePriority::LRUIterator::increasePriority(const CacheGuard::Lock &) size_t LRUFileCachePriority::LRUIterator::increasePriority(const CacheGuard::Lock &)
{ {
checkUsable(); assertValid();
cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, iterator);
return ++queue_iter->hits; return ++iterator->hits;
} }
void LRUFileCachePriority::LRUIterator::checkUsable() const void LRUFileCachePriority::LRUIterator::assertValid() const
{ {
if (queue_iter == LRUQueue::iterator{}) if (iterator == LRUQueue::iterator{})
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator");
} }

View File

@ -11,7 +11,7 @@ namespace DB
/// Based on the LRU algorithm implementation, the record with the lowest priority is stored at /// Based on the LRU algorithm implementation, the record with the lowest priority is stored at
/// the head of the queue, and the record with the highest priority is stored at the tail. /// the head of the queue, and the record with the highest priority is stored at the tail.
class LRUFileCachePriority : public IFileCachePriority class LRUFileCachePriority final : public IFileCachePriority
{ {
private: private:
class LRUIterator; class LRUIterator;
@ -25,13 +25,13 @@ public:
size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; } size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; }
Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override;
bool collectCandidatesForEviction( bool collectCandidatesForEviction(
size_t size, size_t size,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::Iterator it, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const CacheGuard::Lock &) override; const CacheGuard::Lock &) override;
@ -71,14 +71,14 @@ private:
std::unique_ptr<LRUIterator> add(Entry && entry, const CacheGuard::Lock &); std::unique_ptr<LRUIterator> add(Entry && entry, const CacheGuard::Lock &);
}; };
class LRUFileCachePriority::LRUIterator : public IFileCachePriority::IIterator class LRUFileCachePriority::LRUIterator : public IFileCachePriority::Iterator
{ {
friend class LRUFileCachePriority; friend class LRUFileCachePriority;
friend class SLRUFileCachePriority; friend class SLRUFileCachePriority;
public: public:
LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_); LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator iterator_);
const Entry & getEntry() const override { return *queue_iter; } const Entry & getEntry() const override { return *iterator; }
size_t increasePriority(const CacheGuard::Lock &) override; size_t increasePriority(const CacheGuard::Lock &) override;
@ -89,10 +89,10 @@ public:
void updateSize(int64_t size) override; void updateSize(int64_t size) override;
private: private:
void checkUsable() const; void assertValid() const;
LRUFileCachePriority * cache_priority; LRUFileCachePriority * cache_priority;
mutable LRUQueue::iterator queue_iter; mutable LRUQueue::iterator iterator;
}; };
} }

View File

@ -29,7 +29,7 @@ struct FileSegmentMetadata : private boost::noncopyable
bool evicting() const { return removal_candidate.load(); } bool evicting() const { return removal_candidate.load(); }
Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } Priority::IteratorPtr getQueueIterator() const { return file_segment->getQueueIterator(); }
FileSegmentPtr file_segment; FileSegmentPtr file_segment;
std::atomic<bool> removal_candidate{false}; std::atomic<bool> removal_candidate{false};

View File

@ -95,7 +95,7 @@ void FileCacheQueryLimit::QueryContext::remove(
records.erase({key, offset}); records.erase({key, offset});
} }
IFileCachePriority::Iterator FileCacheQueryLimit::QueryContext::tryGet( IFileCachePriority::IteratorPtr FileCacheQueryLimit::QueryContext::tryGet(
const Key & key, const Key & key,
size_t offset, size_t offset,
const CacheGuard::Lock &) const CacheGuard::Lock &)

View File

@ -27,7 +27,6 @@ public:
public: public:
using Key = FileCacheKey; using Key = FileCacheKey;
using Priority = IFileCachePriority; using Priority = IFileCachePriority;
using PriorityIterator = IFileCachePriority::Iterator;
QueryContext(size_t query_cache_size, bool recache_on_query_limit_exceeded_); QueryContext(size_t query_cache_size, bool recache_on_query_limit_exceeded_);
@ -36,7 +35,7 @@ public:
bool recacheOnFileCacheQueryLimitExceeded() const { return recache_on_query_limit_exceeded; } bool recacheOnFileCacheQueryLimitExceeded() const { return recache_on_query_limit_exceeded; }
Priority::Iterator tryGet( Priority::IteratorPtr tryGet(
const Key & key, const Key & key,
size_t offset, size_t offset,
const CacheGuard::Lock &); const CacheGuard::Lock &);
@ -53,7 +52,7 @@ public:
const CacheGuard::Lock &); const CacheGuard::Lock &);
private: private:
using Records = std::unordered_map<FileCacheKeyAndOffset, Priority::Iterator, FileCacheKeyAndOffsetHash>; using Records = std::unordered_map<FileCacheKeyAndOffset, Priority::IteratorPtr, FileCacheKeyAndOffsetHash>;
Records records; Records records;
LRUFileCachePriority priority; LRUFileCachePriority priority;
const bool recache_on_query_limit_exceeded; const bool recache_on_query_limit_exceeded;

View File

@ -52,38 +52,37 @@ size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) co
return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock); return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock);
} }
IFileCachePriority::Iterator SLRUFileCachePriority::add( IFileCachePriority::IteratorPtr SLRUFileCachePriority::add(
KeyMetadataPtr key_metadata, KeyMetadataPtr key_metadata,
size_t offset, size_t offset,
size_t size, size_t size,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
auto it = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock);
return std::make_shared<SLRUIterator>(this, std::move(it), false); return std::make_shared<SLRUIterator>(this, std::move(lru_iterator), false);
} }
bool SLRUFileCachePriority::collectCandidatesForEviction( bool SLRUFileCachePriority::collectCandidatesForEviction(
size_t size, size_t size,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::Iterator it, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const CacheGuard::Lock & lock) const CacheGuard::Lock & lock)
{ {
/// `it` is a pointer to entry we want to evict in favour of.
/// If `it` is nullptr, then it is the first space reservation attempt /// If `it` is nullptr, then it is the first space reservation attempt
/// for a corresponding file segment, so it will be directly put into probationary queue. /// for a corresponding file segment, so it will be directly put into probationary queue.
if (!it) if (!reservee)
{ {
return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock);
} }
/// If `it` not nullptr (e.g. is already in some queue), /// If `it` not nullptr (e.g. is already in some queue),
/// we need to check in which queue (protected/probationary) it currently is /// we need to check in which queue (protected/probationary) it currently is
/// (in order to know where we need to free space). /// (in order to know where we need to free space).
if (!assert_cast<SLRUIterator *>(it.get())->is_protected) if (!assert_cast<SLRUIterator *>(reservee.get())->is_protected)
{ {
return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock);
} }
/// Entry is in protected queue. /// Entry is in protected queue.
@ -100,13 +99,13 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
FileCacheReserveStat downgrade_stat; FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop; FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, it, noop, lock)) if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, lock))
return false; return false;
const size_t size_to_downgrade = downgrade_stat.stat.releasable_size; const size_t size_to_downgrade = downgrade_stat.stat.releasable_size;
if (!probationary_queue.canFit(size_to_downgrade, lock) if (!probationary_queue.canFit(size_to_downgrade, lock)
&& !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, it, noop, lock)) && !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, reservee, noop, lock))
return false; return false;
finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable
@ -127,75 +126,73 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock) void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock)
{ {
auto & lru_it = iterator.lru_iterator; /// If entry is already in protected queue,
const bool is_protected = iterator.is_protected;
const auto & entry = lru_it->getEntry();
/// If entry (`it` is the pointer to the entry) is already in protected queue,
/// we only need to increase its priority within the protected queue. /// we only need to increase its priority within the protected queue.
if (is_protected) if (iterator.is_protected)
{ {
lru_it->increasePriority(lock); iterator.lru_iterator->increasePriority(lock);
return; return;
} }
/// Entry is in probationary queue. /// Entry is in probationary queue.
/// We need to move it to protected queue. /// We need to move it to protected queue.
if (entry.size > protected_queue.getSizeLimit()) const size_t size = iterator.getEntry().size;
if (size > protected_queue.getSizeLimit())
{ {
/// Entry size is bigger than the whole protected queue limit. /// Entry size is bigger than the whole protected queue limit.
/// This is only possible if protected_queue_size_limit is less than max_file_segment_size, /// This is only possible if protected_queue_size_limit is less than max_file_segment_size,
/// which is not possible in any realistic cache configuration. /// which is not possible in any realistic cache configuration.
lru_it->increasePriority(lock); iterator.lru_iterator->increasePriority(lock);
return; return;
} }
/// Check if there is enough space in protected queue to move entry there. /// Check if there is enough space in protected queue to move entry there.
/// If not - we need to "downgrade" lowest priority entries from protected /// If not - we need to "downgrade" lowest priority entries from protected
/// queue to probationary queue. /// queue to probationary queue.
/// The amount of such "downgraded" entries is equal to the amount
/// required to make space for entry we want to insert.
EvictionCandidates downgrade_candidates; EvictionCandidates downgrade_candidates;
FileCacheReserveStat downgrade_stat; FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop; FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(entry.size, downgrade_stat, downgrade_candidates, {}, noop, lock)) if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, downgrade_candidates, {}, noop, lock))
{ {
/// We cannot make space for entry to be moved to protected queue /// We cannot make space for entry to be moved to protected queue
/// (not enough releasable file segments). /// (not enough releasable file segments).
/// Then just increase its priority within probationary queue. /// Then just increase its priority within probationary queue.
lru_it->increasePriority(lock); iterator.lru_iterator->increasePriority(lock);
return; return;
} }
/// The amount of such "downgraded" entries is equal to the amount
/// required to make space for entry we want to insert.
const size_t size_to_downgrade = downgrade_stat.stat.releasable_count;
size_t size_to_free = 0;
if (size_to_downgrade && size_to_downgrade > size)
size_to_free = size_to_downgrade - size;
/// Now we need to check if those "downgrade" candidates can actually /// Now we need to check if those "downgrade" candidates can actually
/// be moved to probationary queue. /// be moved to probationary queue.
const size_t size_to_downgrade = downgrade_stat.stat.releasable_count;
size_t size_to_free = 0;
if (size_to_downgrade && size_to_downgrade > entry.size)
size_to_free = size_to_downgrade - entry.size;
EvictionCandidates eviction_candidates; EvictionCandidates eviction_candidates;
FileCacheReserveStat stat; FileCacheReserveStat stat;
if (size_to_free if (size_to_free)
&& !probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock))
{ {
/// "downgrade" canidates cannot be moved to probationary queue, if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock))
/// so entry cannot be moved to protected queue as well. {
/// Then just increase its priority within probationary queue. /// "downgrade" canidates cannot be moved to probationary queue,
lru_it->increasePriority(lock); /// so entry cannot be moved to protected queue as well.
return; /// Then just increase its priority within probationary queue.
iterator.lru_iterator->increasePriority(lock);
return;
}
/// Make space for "downgrade" candidates.
eviction_candidates.evict(nullptr, lock);
} }
/// Make space for "downgrade" candidates.
eviction_candidates.evict(nullptr, lock);
/// All checks passed, now we can move downgrade candidates to /// All checks passed, now we can move downgrade candidates to
/// probationary queue and our entry to protected queue. /// probationary queue and our entry to protected queue.
Entry entry_copy = lru_it->getEntry(); Entry entry_copy = iterator.getEntry();
lru_it->remove(lock); iterator.lru_iterator->remove(lock);
for (const auto & [key, key_candidates] : downgrade_candidates) for (const auto & [key, key_candidates] : downgrade_candidates)
{ {
@ -237,37 +234,37 @@ SLRUFileCachePriority::SLRUIterator::SLRUIterator(
const SLRUFileCachePriority::Entry & SLRUFileCachePriority::SLRUIterator::getEntry() const const SLRUFileCachePriority::Entry & SLRUFileCachePriority::SLRUIterator::getEntry() const
{ {
checkUsable(); assertValid();
return lru_iterator->getEntry(); return lru_iterator->getEntry();
} }
size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::Lock & lock) size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::Lock & lock)
{ {
checkUsable(); assertValid();
cache_priority->increasePriority(*this, lock); cache_priority->increasePriority(*this, lock);
return getEntry().hits; return getEntry().hits;
} }
void SLRUFileCachePriority::SLRUIterator::updateSize(int64_t size) void SLRUFileCachePriority::SLRUIterator::updateSize(int64_t size)
{ {
checkUsable(); assertValid();
lru_iterator->updateSize(size); lru_iterator->updateSize(size);
} }
void SLRUFileCachePriority::SLRUIterator::invalidate() void SLRUFileCachePriority::SLRUIterator::invalidate()
{ {
checkUsable(); assertValid();
lru_iterator->invalidate(); lru_iterator->invalidate();
} }
void SLRUFileCachePriority::SLRUIterator::remove(const CacheGuard::Lock & lock) void SLRUFileCachePriority::SLRUIterator::remove(const CacheGuard::Lock & lock)
{ {
checkUsable(); assertValid();
lru_iterator->remove(lock); lru_iterator->remove(lock);
lru_iterator = nullptr; lru_iterator = nullptr;
} }
void SLRUFileCachePriority::SLRUIterator::checkUsable() const void SLRUFileCachePriority::SLRUIterator::assertValid() const
{ {
if (!lru_iterator) if (!lru_iterator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator");

View File

@ -32,13 +32,13 @@ public:
size_t getElementsCount(const CacheGuard::Lock &) const override; size_t getElementsCount(const CacheGuard::Lock &) const override;
Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override;
bool collectCandidatesForEviction( bool collectCandidatesForEviction(
size_t size, size_t size,
FileCacheReserveStat & stat, FileCacheReserveStat & stat,
EvictionCandidates & res, EvictionCandidates & res,
IFileCachePriority::Iterator, IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func, FinalizeEvictionFunc & finalize_eviction_func,
const CacheGuard::Lock &) override; const CacheGuard::Lock &) override;
@ -54,7 +54,7 @@ private:
void increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock); void increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock);
}; };
class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::IIterator class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::Iterator
{ {
friend class SLRUFileCachePriority; friend class SLRUFileCachePriority;
public: public:
@ -76,7 +76,7 @@ public:
bool isProtected() const { return is_protected; } bool isProtected() const { return is_protected; }
private: private:
void checkUsable() const; void assertValid() const;
SLRUFileCachePriority * cache_priority; SLRUFileCachePriority * cache_priority;
mutable std::unique_ptr<LRUIterator> lru_iterator; mutable std::unique_ptr<LRUIterator> lru_iterator;