mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 01:41:59 +00:00
Refactoring
This commit is contained in:
parent
00177a8016
commit
dc5e5f3b20
@ -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)
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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};
|
||||||
|
@ -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 &)
|
||||||
|
@ -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;
|
||||||
|
@ -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");
|
||||||
|
@ -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;
|
||||||
|
Loading…
Reference in New Issue
Block a user