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-02-18 09:06:13 +00:00
|
|
|
#include "FileCache_fwd.h"
|
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.
|
|
|
|
*/
|
2022-02-18 15:38:23 +00:00
|
|
|
class IFileCache : private boost::noncopyable
|
2022-01-13 11:57:56 +00:00
|
|
|
{
|
|
|
|
friend class FileSegment;
|
2022-03-17 19:29:07 +00:00
|
|
|
friend struct FileSegmentsHolder;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
using Key = UInt128;
|
2022-01-26 18:43:23 +00:00
|
|
|
using Downloader = std::unique_ptr<SeekableReadBuffer>;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
IFileCache(
|
2022-01-26 18:43:23 +00:00
|
|
|
const String & cache_base_path_,
|
2022-02-18 09:06:13 +00:00
|
|
|
size_t max_size_,
|
|
|
|
size_t max_element_size_,
|
|
|
|
size_t max_file_segment_size_);
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
virtual ~IFileCache() = default;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-03-01 17:12:34 +00:00
|
|
|
/// Restore cache from local filesystem.
|
|
|
|
virtual void initialize() = 0;
|
|
|
|
|
2022-03-07 13:30:57 +00:00
|
|
|
virtual void remove(const Key & key) = 0;
|
|
|
|
|
2022-01-24 22:07:02 +00:00
|
|
|
static bool shouldBypassCache();
|
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
/// Cache capacity in bytes.
|
2022-01-13 11:57:56 +00:00
|
|
|
size_t capacity() const { return max_size; }
|
|
|
|
|
|
|
|
static Key hash(const String & path);
|
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
String getPathInLocalCache(const Key & key, size_t offset);
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
String getPathInLocalCache(const Key & key);
|
2022-01-13 11:57:56 +00:00
|
|
|
|
|
|
|
/**
|
|
|
|
* 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;
|
|
|
|
|
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;
|
2022-02-18 09:06:13 +00:00
|
|
|
size_t max_size;
|
|
|
|
size_t max_element_size;
|
|
|
|
size_t max_file_segment_size;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-03-01 17:12:34 +00:00
|
|
|
bool is_initialized = false;
|
|
|
|
|
2022-01-13 11:57:56 +00:00
|
|
|
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-02-23 10:12:14 +00:00
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_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-02-23 10:12:14 +00:00
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_lock) = 0;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
/// If file segment was partially downloaded and then space reservation fails (because of no
|
|
|
|
/// space left), then update corresponding cache cell metadata (file segment size).
|
2022-01-22 22:56:24 +00:00
|
|
|
virtual void reduceSizeToDownloaded(
|
2022-02-23 10:12:14 +00:00
|
|
|
const Key & key, size_t offset,
|
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_lock) = 0;
|
2022-03-09 09:36:52 +00:00
|
|
|
|
|
|
|
void assertInitialized() const;
|
2022-01-13 11:57:56 +00:00
|
|
|
};
|
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
using FileCachePtr = std::shared_ptr<IFileCache>;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
class LRUFileCache final : public IFileCache
|
2022-01-13 11:57:56 +00:00
|
|
|
{
|
|
|
|
public:
|
2022-02-18 09:06:13 +00:00
|
|
|
LRUFileCache(
|
|
|
|
const String & cache_base_path_,
|
|
|
|
size_t max_size_,
|
|
|
|
size_t max_element_size_ = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS,
|
|
|
|
size_t max_file_segment_size_ = REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE);
|
2022-01-13 11:57:56 +00:00
|
|
|
|
|
|
|
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override;
|
|
|
|
|
2022-03-01 17:12:34 +00:00
|
|
|
void initialize() override;
|
|
|
|
|
2022-03-07 13:30:57 +00:00
|
|
|
void remove(const Key & key) override;
|
|
|
|
|
2022-01-13 11:57:56 +00:00
|
|
|
private:
|
|
|
|
using FileKeyAndOffset = std::pair<Key, size_t>;
|
|
|
|
using LRUQueue = std::list<FileKeyAndOffset>;
|
|
|
|
using LRUQueueIterator = typename LRUQueue::iterator;
|
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
struct FileSegmentCell : private boost::noncopyable
|
2022-01-13 11:57:56 +00:00
|
|
|
{
|
|
|
|
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
|
|
|
|
2022-02-21 12:54:03 +00:00
|
|
|
/// 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
|
|
|
|
/// getorSet(), but cache users always hold it via FileSegmentsHolder.
|
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-02-18 15:38:23 +00:00
|
|
|
|
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-02-23 10:12:14 +00:00
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_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-02-23 10:12:14 +00:00
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_lock) override;
|
2022-01-13 11:57:56 +00:00
|
|
|
|
2022-01-22 22:56:24 +00:00
|
|
|
void reduceSizeToDownloaded(
|
2022-02-23 10:12:14 +00:00
|
|
|
const Key & key, size_t offset,
|
|
|
|
std::lock_guard<std::mutex> & cache_lock,
|
|
|
|
std::lock_guard<std::mutex> & segment_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; }
|
2022-01-23 17:33:22 +00:00
|
|
|
|
2022-02-18 15:38:23 +00:00
|
|
|
void loadCacheInfoIntoMemory();
|
2022-01-23 17:33:22 +00:00
|
|
|
|
2022-02-18 09:06:13 +00:00
|
|
|
FileSegments splitRangeIntoEmptyCells(
|
|
|
|
const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
|
|
|
|
|
2022-03-16 12:27:58 +00:00
|
|
|
String dumpStructureImpl(const Key & key_, std::lock_guard<std::mutex> & cache_lock);
|
|
|
|
|
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
|
|
|
};
|
|
|
|
|
|
|
|
}
|