ClickHouse/src/Common/FileCache.h

180 lines
5.1 KiB
C++
Raw Normal View History

2022-01-13 11:57:56 +00:00
#pragma once
#include <atomic>
#include <chrono>
#include <list>
#include <memory>
#include <mutex>
#include <unordered_map>
#include <unordered_set>
#include <boost/noncopyable.hpp>
#include <map>
2022-01-22 22:56:24 +00:00
2022-01-13 11:57:56 +00:00
#include <base/logger_useful.h>
2022-01-22 22:56:24 +00:00
#include <Common/FileSegment.h>
#include <Core/Types.h>
2022-01-13 11:57:56 +00:00
namespace DB
{
/**
* Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments.
*/
class FileCache : boost::noncopyable
{
friend class FileSegment;
public:
using Key = UInt128;
FileCache(const String & cache_base_path_, size_t max_size_, size_t max_element_size_);
virtual ~FileCache() = default;
2022-01-24 22:07:02 +00:00
static bool shouldBypassCache();
2022-01-13 11:57:56 +00:00
size_t capacity() const { return max_size; }
static Key hash(const String & path);
String path(const Key & key, size_t offset);
String path(const Key & key);
/**
* Given an `offset` and `size` representing [offset, offset + size) bytes interval,
* return list of cached non-overlapping non-empty
* file segments `[segment1, ..., segmentN]` which intersect with given interval.
*
* 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.
*
* As long as pointers to returned file segments are hold
* it is guaranteed that these file segments are not removed from cache.
*/
virtual FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) = 0;
virtual void remove(const Key & key) = 0;
2022-01-22 22:56:24 +00:00
/// For debug.
2022-01-26 09:35:46 +00:00
virtual String dumpStructure(const Key & key) = 0;
2022-01-21 15:39:34 +00:00
2022-01-13 11:57:56 +00:00
protected:
String cache_base_path;
size_t max_size = 0;
size_t max_element_size = 0;
mutable std::mutex mutex;
2022-01-21 15:39:34 +00:00
virtual bool tryReserve(
2022-01-23 16:51:18 +00:00
const Key & key, size_t offset, size_t size,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) = 0;
2022-01-21 15:39:34 +00:00
2022-01-13 11:57:56 +00:00
virtual void remove(
2022-01-21 15:39:34 +00:00
Key key, size_t offset,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) = 0;
2022-01-21 15:39:34 +00:00
2022-01-23 16:51:18 +00:00
virtual bool isLastFileSegmentHolder(
const Key & key, size_t offset,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) = 0;
2022-01-13 11:57:56 +00:00
2022-01-22 22:56:24 +00:00
virtual void reduceSizeToDownloaded(
2022-01-24 22:07:02 +00:00
const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock) = 0;
2022-01-13 11:57:56 +00:00
};
2022-01-22 22:56:24 +00:00
using FileCachePtr = std::shared_ptr<FileCache>;
2022-01-13 11:57:56 +00:00
class LRUFileCache final : public FileCache
{
public:
LRUFileCache(const String & cache_base_path_, size_t max_size_, size_t max_element_size_ = 0);
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override;
void remove(const Key & key) override;
private:
using FileKeyAndOffset = std::pair<Key, size_t>;
using LRUQueue = std::list<FileKeyAndOffset>;
using LRUQueueIterator = typename LRUQueue::iterator;
struct FileSegmentCell : boost::noncopyable
{
FileSegmentPtr file_segment;
2022-01-21 15:39:34 +00:00
/// Iterator is put here on first reservation attempt, if successful.
std::optional<LRUQueueIterator> queue_iterator;
2022-01-13 11:57:56 +00:00
bool releasable() const { return file_segment.unique(); }
2022-01-23 16:51:18 +00:00
size_t size() const { return file_segment->reserved_size; }
2022-01-13 11:57:56 +00:00
2022-01-22 22:56:24 +00:00
FileSegmentCell(FileSegmentPtr file_segment_, LRUQueue & queue_);
2022-01-13 11:57:56 +00:00
FileSegmentCell(FileSegmentCell && other)
: file_segment(std::move(other.file_segment))
, queue_iterator(std::move(other.queue_iterator)) {}
2022-01-21 15:39:34 +00:00
std::pair<Key, size_t> getKeyAndOffset() const { return std::make_pair(file_segment->key(), file_segment->range().left); }
2022-01-13 11:57:56 +00:00
};
using FileSegmentsByOffset = std::map<size_t, FileSegmentCell>;
using CachedFiles = std::unordered_map<Key, FileSegmentsByOffset>;
CachedFiles files;
LRUQueue queue;
size_t current_size = 0;
Poco::Logger * log;
2022-01-21 15:39:34 +00:00
bool startup_restore_finished = false;
2022-01-13 11:57:56 +00:00
FileSegments getImpl(
const Key & key, const FileSegment::Range & range,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock);
2022-01-13 11:57:56 +00:00
2022-01-22 22:56:24 +00:00
FileSegmentCell * getCell(
2022-01-24 22:07:02 +00:00
const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock);
2022-01-13 11:57:56 +00:00
2022-01-21 15:39:34 +00:00
FileSegmentCell * addCell(
const Key & key, size_t offset, size_t size,
2022-01-24 22:07:02 +00:00
FileSegment::State state, std::lock_guard<std::mutex> & cache_lock);
2022-01-22 22:56:24 +00:00
2022-01-24 22:07:02 +00:00
void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard<std::mutex> & cache_lock);
2022-01-13 11:57:56 +00:00
2022-01-21 15:39:34 +00:00
bool tryReserve(
2022-01-23 16:51:18 +00:00
const Key & key, size_t offset, size_t size,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) override;
2022-01-13 11:57:56 +00:00
2022-01-21 15:39:34 +00:00
void remove(
Key key, size_t offset,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) override;
2022-01-13 11:57:56 +00:00
2022-01-23 16:51:18 +00:00
bool isLastFileSegmentHolder(
const Key & key, size_t offset,
2022-01-24 22:07:02 +00:00
std::lock_guard<std::mutex> & cache_lock) override;
2022-01-13 11:57:56 +00:00
2022-01-21 15:39:34 +00:00
void removeFileKey(const Key & key);
2022-01-13 11:57:56 +00:00
2022-01-22 22:56:24 +00:00
void reduceSizeToDownloaded(
2022-01-24 22:07:02 +00:00
const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock) override;
2022-01-22 22:56:24 +00:00
2022-01-24 22:07:02 +00:00
size_t availableSize() const { return max_size - current_size; }
void restore();
2022-01-13 11:57:56 +00:00
public:
struct Stat
{
size_t size;
size_t available;
size_t downloaded_size;
size_t downloading_size;
};
Stat getStat();
2022-01-21 15:39:34 +00:00
2022-01-26 09:35:46 +00:00
String dumpStructure(const Key & key_) override;
2022-01-13 11:57:56 +00:00
};
}